You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by yi...@apache.org on 2022/07/05 12:33:41 UTC

[doris] branch master updated: [refactor] remove alpha rowset related code and vectorized row batch related code (#10584)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 89e56ea67f [refactor] remove alpha rowset related code and vectorized row batch related code (#10584)
89e56ea67f is described below

commit 89e56ea67f09a9bf5a3cfa46c8e8b0a07117d150
Author: yiguolei <67...@qq.com>
AuthorDate: Tue Jul 5 20:33:34 2022 +0800

    [refactor] remove alpha rowset related code and vectorized row batch related code (#10584)
---
 be/src/common/config.h                             |    6 -
 be/src/exprs/expr.cpp                              |    3 +-
 be/src/exprs/expr.h                                |   22 -
 be/src/exprs/slot_ref.h                            |    5 -
 be/src/olap/CMakeLists.txt                         |   10 -
 be/src/olap/bloom_filter_predicate.h               |   17 -
 be/src/olap/bloom_filter_reader.cpp                |   62 -
 be/src/olap/bloom_filter_reader.h                  |   66 -
 be/src/olap/bloom_filter_writer.cpp                |  109 -
 be/src/olap/bloom_filter_writer.h                  |   42 -
 be/src/olap/column_predicate.h                     |    4 -
 be/src/olap/comparison_predicate.h                 |   49 -
 be/src/olap/convert_rowset.cpp                     |  140 -
 be/src/olap/convert_rowset.h                       |   42 -
 be/src/olap/file_stream.cpp                        |  187 --
 be/src/olap/file_stream.h                          |  276 --
 be/src/olap/in_list_predicate.h                    |   54 -
 be/src/olap/in_stream.cpp                          |  305 --
 be/src/olap/in_stream.h                            |  191 --
 be/src/olap/lru_cache.cpp                          |    1 -
 be/src/olap/memtable.cpp                           |    1 -
 be/src/olap/null_predicate.cpp                     |   38 -
 be/src/olap/null_predicate.h                       |    4 -
 be/src/olap/olap_index.cpp                         |  525 ----
 be/src/olap/olap_index.h                           |  380 ---
 be/src/olap/olap_server.cpp                        |   50 -
 be/src/olap/out_stream.cpp                         |  401 ---
 be/src/olap/out_stream.h                           |  169 --
 be/src/olap/push_handler.cpp                       |    1 -
 be/src/olap/reader.h                               |    1 -
 be/src/olap/row_block.h                            |    3 -
 be/src/olap/row_block2.h                           |    1 -
 be/src/olap/rowset/CMakeLists.txt                  |   16 -
 be/src/olap/rowset/alpha_rowset.cpp                |  407 ---
 be/src/olap/rowset/alpha_rowset.h                  |   94 -
 be/src/olap/rowset/alpha_rowset_reader.cpp         |  400 ---
 be/src/olap/rowset/alpha_rowset_reader.h           |  136 -
 be/src/olap/rowset/alpha_rowset_writer.cpp         |  313 ---
 be/src/olap/rowset/alpha_rowset_writer.h           |   85 -
 be/src/olap/rowset/bit_field_reader.cpp            |  127 -
 be/src/olap/rowset/bit_field_reader.h              |   53 -
 be/src/olap/rowset/bit_field_writer.cpp            |   99 -
 be/src/olap/rowset/bit_field_writer.h              |   52 -
 be/src/olap/rowset/column_data.cpp                 |  661 -----
 be/src/olap/rowset/column_data.h                   |  230 --
 be/src/olap/rowset/column_data_writer.cpp          |  320 ---
 be/src/olap/rowset/column_data_writer.h            |   82 -
 be/src/olap/rowset/column_reader.cpp               | 1186 --------
 be/src/olap/rowset/column_reader.h                 |  841 ------
 be/src/olap/rowset/column_writer.cpp               |  873 ------
 be/src/olap/rowset/column_writer.h                 |  587 ----
 be/src/olap/rowset/rowset_factory.cpp              |    8 +-
 be/src/olap/rowset/run_length_byte_reader.cpp      |  145 -
 be/src/olap/rowset/run_length_byte_reader.h        |   57 -
 be/src/olap/rowset/run_length_byte_writer.cpp      |  142 -
 be/src/olap/rowset/run_length_byte_writer.h        |   57 -
 be/src/olap/rowset/run_length_integer_reader.cpp   |  429 ---
 be/src/olap/rowset/run_length_integer_reader.h     |   75 -
 be/src/olap/rowset/run_length_integer_writer.cpp   |  734 -----
 be/src/olap/rowset/run_length_integer_writer.h     |  283 --
 be/src/olap/rowset/segment_group.cpp               | 1029 -------
 be/src/olap/rowset/segment_group.h                 |  312 ---
 be/src/olap/rowset/segment_reader.cpp              |  885 ------
 be/src/olap/rowset/segment_reader.h                |  334 ---
 be/src/olap/rowset/segment_v2/bloom_filter.cpp     |   12 +-
 be/src/olap/rowset/segment_v2/bloom_filter.h       |    2 +
 be/src/olap/rowset/segment_writer.cpp              |  298 --
 be/src/olap/rowset/segment_writer.h                |   66 -
 be/src/olap/serialize.cpp                          |  334 ---
 be/src/olap/serialize.h                            |  184 --
 be/src/olap/short_key_index.h                      |    3 +-
 be/src/olap/snapshot_manager.h                     |    1 -
 be/src/olap/storage_engine.cpp                     |    8 -
 be/src/olap/storage_engine.h                       |    5 -
 be/src/olap/stream_index_reader.cpp                |  145 -
 be/src/olap/stream_index_reader.h                  |   91 -
 be/src/olap/stream_index_writer.cpp                |  178 --
 be/src/olap/stream_index_writer.h                  |   78 -
 be/src/olap/tablet.cpp                             |    9 -
 be/src/olap/tablet.h                               |    2 -
 be/src/olap/tablet_manager.cpp                     |   15 -
 be/src/olap/tablet_manager.h                       |    2 -
 be/src/olap/txn_manager.cpp                        |    1 -
 be/src/runtime/CMakeLists.txt                      |    1 -
 be/src/runtime/vectorized_row_batch.cpp            |  140 -
 be/src/runtime/vectorized_row_batch.h              |  125 -
 be/test/CMakeLists.txt                             |   17 +-
 be/test/exprs/binary_predicate_test.cpp            |   86 -
 be/test/exprs/in_op_test.cpp                       |   84 -
 be/test/olap/bit_field_test.cpp                    |  180 --
 be/test/olap/block_column_predicate_test.cpp       |    1 -
 .../olap/bloom_filter_column_predicate_test.cpp    |   52 +-
 be/test/olap/bloom_filter_index_test.cpp           |  108 -
 be/test/olap/column_reader_test.cpp                | 2939 --------------------
 be/test/olap/comparison_predicate_test.cpp         |  622 +----
 be/test/olap/in_list_predicate_test.cpp            |  399 +--
 be/test/olap/null_predicate_test.cpp               |  297 +-
 be/test/olap/rowset/alpha_rowset_test.cpp          |  333 ---
 be/test/olap/run_length_byte_test.cpp              |  834 ------
 be/test/olap/run_length_integer_test.cpp           | 1479 ----------
 be/test/olap/schema_change_test.cpp                | 1462 ----------
 be/test/olap/serialize_test.cpp                    |  234 --
 be/test/olap/stream_index_test.cpp                 |  385 ---
 be/test/olap/tablet_mgr_test.cpp                   |    1 -
 104 files changed, 42 insertions(+), 25388 deletions(-)

diff --git a/be/src/common/config.h b/be/src/common/config.h
index 648643b6a9..8b63ccf00c 100644
--- a/be/src/common/config.h
+++ b/be/src/common/config.h
@@ -295,12 +295,6 @@ CONF_mInt64(min_compaction_failure_interval_sec, "5"); // 5 seconds
 CONF_mInt32(max_base_compaction_threads, "4");
 CONF_mInt32(max_cumu_compaction_threads, "10");
 
-// This config can be set to limit thread number in convert rowset thread pool.
-CONF_mInt32(convert_rowset_thread_num, "0");
-
-// initial sleep interval in seconds of scan alpha rowset
-CONF_mInt32(scan_alpha_rowset_min_interval_sec, "3");
-
 // This config can be set to limit thread number in  smallcompaction thread pool.
 CONF_mInt32(quick_compaction_max_threads, "10");
 
diff --git a/be/src/exprs/expr.cpp b/be/src/exprs/expr.cpp
index 37f8bc4c31..695bc5a44e 100644
--- a/be/src/exprs/expr.cpp
+++ b/be/src/exprs/expr.cpp
@@ -90,8 +90,7 @@ Expr::Expr(const Expr& expr)
           _output_column(expr._output_column),
           _fn(expr._fn),
           _fn_context_index(expr._fn_context_index),
-          _constant_val(expr._constant_val),
-          _vector_compute_fn(expr._vector_compute_fn) {}
+          _constant_val(expr._constant_val) {}
 
 Expr::Expr(const TypeDescriptor& type)
         : _opcode(TExprOpcode::INVALID_OPCODE),
diff --git a/be/src/exprs/expr.h b/be/src/exprs/expr.h
index 56ff05e8a3..2fe979cc17 100644
--- a/be/src/exprs/expr.h
+++ b/be/src/exprs/expr.h
@@ -51,7 +51,6 @@ class TColumnValue;
 class TExpr;
 class TExprNode;
 class TupleIsNullPredicate;
-class VectorizedRowBatch;
 class Literal;
 class MemTracker;
 struct UserFunctionCacheEntry;
@@ -62,9 +61,6 @@ public:
     // typedef for compute functions.
     typedef void* (*ComputeFn)(Expr*, TupleRow*);
 
-    // typdef for vectorize compute functions.
-    typedef bool (*VectorComputeFn)(Expr*, VectorizedRowBatch*);
-
     // Empty virtual destructor
     virtual ~Expr();
 
@@ -77,10 +73,6 @@ public:
     // TODO: stop having the result cached in this Expr object
     void* get_value(TupleRow* row) { return nullptr; }
 
-    // Vectorize Evalute expr and return result column index.
-    // Result cached in batch and valid as long as batch.
-    bool evaluate(VectorizedRowBatch* batch);
-
     bool is_null_scalar_function(std::string& str) {
         // name and function_name both are required
         if (_fn.name.function_name.compare("is_null_pred") == 0) {
@@ -374,9 +366,6 @@ protected:
     // get_const_val().
     std::shared_ptr<AnyVal> _constant_val;
 
-    // function to evaluate vectorize expr; typically set in prepare()
-    VectorComputeFn _vector_compute_fn;
-
     /// Simple debug string that provides no expr subclass-specific information
     std::string debug_string(const std::string& expr_name) const {
         std::stringstream out;
@@ -458,17 +447,6 @@ private:
     int _fn_ctx_idx_end = 0;
 };
 
-inline bool Expr::evaluate(VectorizedRowBatch* batch) {
-    DCHECK(_type.type != INVALID_TYPE);
-
-    if (_is_slotref) {
-        // return SlotRef::vector_compute_fn(this, batch);
-        return false;
-    } else {
-        return _vector_compute_fn(this, batch);
-    }
-}
-
 template <typename T>
 Status create_texpr_literal_node(const void* data, TExprNode* node) {
     if constexpr (std::is_same_v<bool, T>) {
diff --git a/be/src/exprs/slot_ref.h b/be/src/exprs/slot_ref.h
index 35ba68d66c..3e63bda69c 100644
--- a/be/src/exprs/slot_ref.h
+++ b/be/src/exprs/slot_ref.h
@@ -50,7 +50,6 @@ public:
     void* get_slot(TupleRow* row);
     Tuple* get_tuple(TupleRow* row);
     bool is_null_bit_set(TupleRow* row);
-    static bool vector_compute_fn(Expr* expr, VectorizedRowBatch* batch);
     static bool is_nullable(Expr* expr);
     virtual std::string debug_string() const override;
     virtual bool is_constant() const override { return false; }
@@ -83,10 +82,6 @@ private:
     bool _is_nullable;
 };
 
-inline bool SlotRef::vector_compute_fn(Expr* expr, VectorizedRowBatch* /* batch */) {
-    return true;
-}
-
 inline void* SlotRef::get_value(Expr* expr, TupleRow* row) {
     SlotRef* ref = (SlotRef*)expr;
     Tuple* t = row->get_tuple(ref->_tuple_idx);
diff --git a/be/src/olap/CMakeLists.txt b/be/src/olap/CMakeLists.txt
index 805cc9458f..1853c62523 100644
--- a/be/src/olap/CMakeLists.txt
+++ b/be/src/olap/CMakeLists.txt
@@ -29,25 +29,20 @@ add_library(Olap STATIC
     base_compaction.cpp
     base_tablet.cpp
     bloom_filter.hpp
-    bloom_filter_reader.cpp
-    bloom_filter_writer.cpp
     block_column_predicate.cpp
     byte_buffer.cpp
     collect_iterator.cpp
     compaction.cpp
     compaction_permit_limiter.cpp
     compress.cpp
-    convert_rowset.cpp
     cumulative_compaction.cpp
     cumulative_compaction_policy.cpp
     delete_handler.cpp
     delta_writer.cpp
     file_helper.cpp
-    file_stream.cpp
     generic_iterators.cpp
     hll.cpp
     bloom_filter_predicate.cpp
-    in_stream.cpp
     key_coder.cpp
     lru_cache.cpp
     memtable.cpp
@@ -55,11 +50,9 @@ add_library(Olap STATIC
     merger.cpp
     null_predicate.cpp
     olap_cond.cpp
-    olap_index.cpp
     olap_meta.cpp
     olap_server.cpp
     options.cpp
-    out_stream.cpp
     page_cache.cpp
     push_handler.cpp
     reader.cpp
@@ -71,14 +64,11 @@ add_library(Olap STATIC
     schema.cpp
     schema_change.cpp
     storage_migration_v2.cpp
-    serialize.cpp
     storage_engine.cpp
     data_dir.cpp
     short_key_index.cpp
     snapshot_manager.cpp
     stream_index_common.cpp
-    stream_index_reader.cpp
-    stream_index_writer.cpp
     stream_name.cpp
     tablet.cpp
     tablet_manager.cpp
diff --git a/be/src/olap/bloom_filter_predicate.h b/be/src/olap/bloom_filter_predicate.h
index c25258f55a..b4ed077709 100644
--- a/be/src/olap/bloom_filter_predicate.h
+++ b/be/src/olap/bloom_filter_predicate.h
@@ -23,7 +23,6 @@
 
 #include "exprs/bloomfilter_predicate.h"
 #include "olap/column_predicate.h"
-#include "runtime/vectorized_row_batch.h"
 #include "vec/columns/column_dictionary.h"
 #include "vec/columns/column_nullable.h"
 #include "vec/columns/column_vector.h"
@@ -31,8 +30,6 @@
 
 namespace doris {
 
-class VectorizedRowBatch;
-
 // only use in runtime filter and segment v2
 template <PrimitiveType T>
 class BloomFilterColumnPredicate : public ColumnPredicate {
@@ -48,8 +45,6 @@ public:
 
     PredicateType type() const override { return PredicateType::BF; }
 
-    void evaluate(VectorizedRowBatch* batch) const override;
-
     void evaluate(ColumnBlock* block, uint16_t* sel, uint16_t* size) const override;
 
     void evaluate_or(ColumnBlock* block, uint16_t* sel, uint16_t size,
@@ -136,18 +131,6 @@ private:
     mutable bool _enable_pred = true;
 };
 
-// bloom filter column predicate do not support in segment v1
-template <PrimitiveType T>
-void BloomFilterColumnPredicate<T>::evaluate(VectorizedRowBatch* batch) const {
-    uint16_t n = batch->size();
-    uint16_t* sel = batch->selected();
-    if (!batch->selected_in_use()) {
-        for (uint16_t i = 0; i != n; ++i) {
-            sel[i] = i;
-        }
-    }
-}
-
 template <PrimitiveType T>
 void BloomFilterColumnPredicate<T>::evaluate(ColumnBlock* block, uint16_t* sel,
                                              uint16_t* size) const {
diff --git a/be/src/olap/bloom_filter_reader.cpp b/be/src/olap/bloom_filter_reader.cpp
deleted file mode 100644
index 1c3bfc2431..0000000000
--- a/be/src/olap/bloom_filter_reader.cpp
+++ /dev/null
@@ -1,62 +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 "olap/bloom_filter_reader.h"
-
-namespace doris {
-
-BloomFilterIndexReader::~BloomFilterIndexReader() {
-    _entry.reset();
-
-    if (!_is_using_cache) {
-        SAFE_DELETE_ARRAY(_buffer);
-    }
-}
-
-Status BloomFilterIndexReader::init(char* buffer, size_t buffer_size, bool is_using_cache,
-                                    uint32_t hash_function_num, uint32_t bit_num) {
-    Status res = Status::OK();
-
-    _buffer = buffer;
-    _buffer_size = buffer_size;
-    _is_using_cache = is_using_cache;
-
-    BloomFilterIndexHeader* header = reinterpret_cast<BloomFilterIndexHeader*>(_buffer);
-    _step_size = bit_num >> 3;
-    _entry_count = header->block_count;
-    _hash_function_num = hash_function_num;
-    _start_offset = sizeof(BloomFilterIndexHeader);
-    if (_step_size * _entry_count + _start_offset > _buffer_size) {
-        LOG(WARNING) << "invalid param found.[buffer_size=" << buffer_size << " bit_num=" << bit_num
-                     << " block_count=" << _entry_count << " header_size=" << _start_offset << "]";
-        return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
-    }
-
-    return res;
-}
-
-const BloomFilter& BloomFilterIndexReader::entry(uint64_t entry_id) {
-    _entry.init((uint64_t*)(_buffer + _start_offset + _step_size * entry_id),
-                _step_size / sizeof(uint64_t), _hash_function_num);
-    return _entry;
-}
-
-size_t BloomFilterIndexReader::entry_count() {
-    return _entry_count;
-}
-
-} // namespace doris
diff --git a/be/src/olap/bloom_filter_reader.h b/be/src/olap/bloom_filter_reader.h
deleted file mode 100644
index 06408a0ada..0000000000
--- a/be/src/olap/bloom_filter_reader.h
+++ /dev/null
@@ -1,66 +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.
-
-#pragma once
-
-#include <vector>
-
-#include "olap/bloom_filter.hpp"
-#include "olap/bloom_filter_writer.h"
-
-namespace doris {
-
-// Each bloom filter index contains multiple bloom filter entries,
-//     each of which is related to a data block.
-//     BloomFilterIndexReader allow caller to get specified bloom filter entry
-//     by parsing bloom filter index buffer.
-class BloomFilterIndexReader {
-public:
-    BloomFilterIndexReader() {}
-    ~BloomFilterIndexReader();
-
-    // Init BloomFilterIndexReader with given bloom filter index buffer
-    Status init(char* buffer, size_t buffer_size, bool is_using_cache, uint32_t hash_function_num,
-                uint32_t bit_num);
-
-    // Get specified bloom filter entry
-    const BloomFilter& entry(uint64_t entry_id);
-
-    // Get total number of bloom filter entries in current bloom filter index buffer
-    size_t entry_count();
-
-private:
-    // Bloom filter index buffer and length
-    char* _buffer;
-    size_t _buffer_size;
-
-    // Total bloom filter entries count, start offset, each bloom filter entry size
-    size_t _entry_count;
-    size_t _start_offset;
-    size_t _step_size;
-
-    // Bloom filter param
-    uint32_t _hash_function_num;
-
-    // BloomFilterIndexReader will not release bloom filter index buffer in destructor
-    // when it is cached in memory
-    bool _is_using_cache;
-
-    BloomFilter _entry;
-};
-
-} // namespace doris
diff --git a/be/src/olap/bloom_filter_writer.cpp b/be/src/olap/bloom_filter_writer.cpp
deleted file mode 100644
index 28517be782..0000000000
--- a/be/src/olap/bloom_filter_writer.cpp
+++ /dev/null
@@ -1,109 +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 "olap/bloom_filter_writer.h"
-
-#include <vector>
-
-namespace doris {
-
-BloomFilterIndexWriter::~BloomFilterIndexWriter() {
-    for (std::vector<BloomFilter*>::iterator it = _bloom_filters.begin();
-         it != _bloom_filters.end(); ++it) {
-        SAFE_DELETE(*it);
-    }
-}
-
-Status BloomFilterIndexWriter::add_bloom_filter(BloomFilter* bf) {
-    try {
-        _bloom_filters.push_back(bf);
-    } catch (...) {
-        LOG(WARNING) << "add bloom filter to vector fail";
-        return Status::OLAPInternalError(OLAP_ERR_STL_ERROR);
-    }
-
-    return Status::OK();
-}
-
-uint64_t BloomFilterIndexWriter::estimate_buffered_memory() {
-    uint64_t buffered_size = sizeof(_header);
-    if (_bloom_filters.size() > 0) {
-        buffered_size +=
-                _bloom_filters.size() * _bloom_filters[0]->bit_set_data_len() * sizeof(uint64_t);
-    }
-    return buffered_size;
-}
-
-Status BloomFilterIndexWriter::write_to_buffer(OutStream* out_stream) {
-    Status res = Status::OK();
-    if (nullptr == out_stream) {
-        LOG(WARNING) << "out stream is null";
-        return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
-    }
-
-    // write header
-    _header.block_count = _bloom_filters.size();
-    res = out_stream->write(reinterpret_cast<char*>(&_header), sizeof(_header));
-    if (!res.ok()) {
-        LOG(WARNING) << "write bloom filter index header fail";
-        return res;
-    }
-
-    // write bloom filters
-    for (size_t i = 0; i < _bloom_filters.size(); ++i) {
-        uint64_t* data = _bloom_filters[i]->bit_set_data();
-        uint32_t data_len = _bloom_filters[i]->bit_set_data_len();
-        res = out_stream->write(reinterpret_cast<char*>(data), sizeof(uint64_t) * data_len);
-        if (!res.ok()) {
-            LOG(WARNING) << "write bloom filter index fail, i=" << i;
-            return res;
-        }
-    }
-
-    return res;
-}
-
-Status BloomFilterIndexWriter::write_to_buffer(char* buffer, size_t buffer_size) {
-    Status res = Status::OK();
-    if (nullptr == buffer) {
-        LOG(WARNING) << "out stream is nullptr.";
-        return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
-    }
-
-    if (estimate_buffered_memory() > buffer_size) {
-        LOG(WARNING) << "need more buffer. [scr_size=" << estimate_buffered_memory()
-                     << " buffer_size=" << buffer_size << "]";
-        return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
-    }
-
-    // write header
-    _header.block_count = _bloom_filters.size();
-    memcpy(buffer, reinterpret_cast<char*>(&_header), sizeof(_header));
-    buffer += sizeof(_header);
-
-    // write bloom filters
-    for (size_t i = 0; i < _bloom_filters.size(); ++i) {
-        uint64_t* data = _bloom_filters[i]->bit_set_data();
-        uint32_t data_len = _bloom_filters[i]->bit_set_data_len();
-        memcpy(buffer, reinterpret_cast<char*>(data), sizeof(uint64_t) * data_len);
-        buffer += sizeof(uint64_t) * data_len;
-    }
-
-    return res;
-}
-
-} // namespace doris
diff --git a/be/src/olap/bloom_filter_writer.h b/be/src/olap/bloom_filter_writer.h
deleted file mode 100644
index b2c8224c3f..0000000000
--- a/be/src/olap/bloom_filter_writer.h
+++ /dev/null
@@ -1,42 +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.
-
-#pragma once
-
-#include <vector>
-
-#include "olap/bloom_filter.hpp"
-#include "olap/out_stream.h"
-
-namespace doris {
-
-class BloomFilterIndexWriter {
-public:
-    BloomFilterIndexWriter() {}
-    ~BloomFilterIndexWriter();
-
-    Status add_bloom_filter(BloomFilter* bf);
-    uint64_t estimate_buffered_memory();
-    Status write_to_buffer(OutStream* out_stream);
-    Status write_to_buffer(char* buffer, size_t buffer_size);
-
-private:
-    std::vector<BloomFilter*> _bloom_filters;
-    BloomFilterIndexHeader _header;
-};
-
-} // namespace doris
diff --git a/be/src/olap/column_predicate.h b/be/src/olap/column_predicate.h
index b02cdc0146..be7a1bdc59 100644
--- a/be/src/olap/column_predicate.h
+++ b/be/src/olap/column_predicate.h
@@ -28,7 +28,6 @@ using namespace doris::segment_v2;
 
 namespace doris {
 
-class VectorizedRowBatch;
 class Schema;
 class RowBlockV2;
 
@@ -64,9 +63,6 @@ public:
 
     virtual PredicateType type() const = 0;
 
-    //evaluate predicate on VectorizedRowBatch
-    virtual void evaluate(VectorizedRowBatch* batch) const = 0;
-
     // evaluate predicate on ColumnBlock
     virtual void evaluate(ColumnBlock* block, uint16_t* sel, uint16_t* size) const = 0;
     virtual void evaluate_or(ColumnBlock* block, uint16_t* sel, uint16_t size,
diff --git a/be/src/olap/comparison_predicate.h b/be/src/olap/comparison_predicate.h
index d1fdfcbb81..f0401aa4cc 100644
--- a/be/src/olap/comparison_predicate.h
+++ b/be/src/olap/comparison_predicate.h
@@ -20,7 +20,6 @@
 #include <cstdint>
 
 #include "olap/column_predicate.h"
-#include "runtime/vectorized_row_batch.h"
 #include "vec/columns/column_dictionary.h"
 
 namespace doris {
@@ -40,54 +39,6 @@ public:
 
     PredicateType type() const override { return PT; }
 
-    void evaluate(VectorizedRowBatch* batch) const override {
-        uint16_t n = batch->size();
-        if (n == 0) {
-            return;
-        }
-        uint16_t* sel = batch->selected();
-        const T* col_vector = reinterpret_cast<const T*>(batch->column(_column_id)->col_data());
-        uint16_t new_size = 0;
-        if (batch->column(_column_id)->no_nulls()) {
-            if (batch->selected_in_use()) {
-                for (uint16_t j = 0; j != n; ++j) {
-                    uint16_t i = sel[j];
-                    sel[new_size] = i;
-                    new_size += _operator(col_vector[i], _value);
-                }
-                batch->set_size(new_size);
-            } else {
-                for (uint16_t i = 0; i != n; ++i) {
-                    sel[new_size] = i;
-                    new_size += _operator(col_vector[i], _value);
-                }
-                if (new_size < n) {
-                    batch->set_size(new_size);
-                    batch->set_selected_in_use(true);
-                }
-            }
-        } else {
-            bool* is_null = batch->column(_column_id)->is_null();
-            if (batch->selected_in_use()) {
-                for (uint16_t j = 0; j != n; ++j) {
-                    uint16_t i = sel[j];
-                    sel[new_size] = i;
-                    new_size += (!is_null[i] && _operator(col_vector[i], _value));
-                }
-                batch->set_size(new_size);
-            } else {
-                for (uint16_t i = 0; i != n; ++i) {
-                    sel[new_size] = i;
-                    new_size += (!is_null[i] && _operator(col_vector[i], _value));
-                }
-                if (new_size < n) {
-                    batch->set_size(new_size);
-                    batch->set_selected_in_use(true);
-                }
-            }
-        }
-    }
-
     void evaluate(ColumnBlock* block, uint16_t* sel, uint16_t* size) const override {
         uint16_t new_size = 0;
         if (block->is_nullable()) {
diff --git a/be/src/olap/convert_rowset.cpp b/be/src/olap/convert_rowset.cpp
deleted file mode 100644
index c71e66c246..0000000000
--- a/be/src/olap/convert_rowset.cpp
+++ /dev/null
@@ -1,140 +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 "olap/convert_rowset.h"
-
-namespace doris {
-
-Status ConvertRowset::do_convert() {
-    if (!_tablet->init_succeeded()) {
-        return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
-    }
-    std::unique_lock<std::mutex> base_compaction_lock(_tablet->get_base_compaction_lock(),
-                                                      std::try_to_lock);
-    std::unique_lock<std::mutex> cumulative_compaction_lock(
-            _tablet->get_cumulative_compaction_lock(), std::try_to_lock);
-    if (!base_compaction_lock.owns_lock() || !cumulative_compaction_lock.owns_lock()) {
-        LOG(INFO) << "The tablet is under compaction. tablet=" << _tablet->full_name();
-        return Status::OLAPInternalError(OLAP_ERR_CE_TRY_CE_LOCK_ERROR);
-    }
-
-    std::vector<RowsetSharedPtr> alpah_rowsets;
-    _tablet->find_alpha_rowsets(&alpah_rowsets);
-
-    Merger::Statistics stats;
-    Status res;
-    const size_t max_convert_row_count = 20000000;
-    size_t row_count = 0;
-    for (size_t i = 0; i < alpah_rowsets.size(); ++i) {
-        Version output_version =
-                Version(alpah_rowsets[i]->start_version(), alpah_rowsets[i]->end_version());
-
-        RowsetReaderSharedPtr input_rs_reader;
-        RETURN_NOT_OK(alpah_rowsets[i]->create_reader(&input_rs_reader));
-
-        std::unique_ptr<RowsetWriter> output_rs_writer;
-        RETURN_NOT_OK(_tablet->create_rowset_writer(output_version, VISIBLE, NONOVERLAPPING,
-                                                    &output_rs_writer));
-        res = Merger::merge_rowsets(_tablet, ReaderType::READER_BASE_COMPACTION, {input_rs_reader},
-                                    output_rs_writer.get(), &stats);
-
-        if (!res.ok()) {
-            LOG(WARNING) << "fail to convert rowset. res=" << res
-                         << ", tablet=" << _tablet->full_name();
-            return res;
-        } else {
-            auto output_rowset = output_rs_writer->build();
-            if (output_rowset == nullptr) {
-                LOG(WARNING) << "rowset writer build failed"
-                             << ", tablet=" << _tablet->full_name();
-                return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
-            }
-
-            RETURN_NOT_OK(check_correctness(alpah_rowsets[i], output_rowset, stats));
-
-            row_count += alpah_rowsets[i]->num_rows();
-
-            RETURN_NOT_OK(_modify_rowsets(alpah_rowsets[i], output_rowset));
-
-            LOG(INFO) << "succeed to convert rowset"
-                      << ". tablet=" << _tablet->full_name()
-                      << ", output_version=" << output_version
-                      << ", disk=" << _tablet->data_dir()->path();
-
-            if (row_count >= max_convert_row_count) {
-                break;
-            }
-        }
-    }
-    return Status::OK();
-}
-
-Status ConvertRowset::check_correctness(RowsetSharedPtr input_rowset, RowsetSharedPtr output_rowset,
-                                        const Merger::Statistics& stats) {
-    // 1. check row number
-    if (input_rowset->num_rows() !=
-        output_rowset->num_rows() + stats.merged_rows + stats.filtered_rows) {
-        LOG(WARNING) << "row_num does not match between input and output! "
-                     << "input_row_num=" << input_rowset->num_rows()
-                     << ", merged_row_num=" << stats.merged_rows
-                     << ", filtered_row_num=" << stats.filtered_rows
-                     << ", output_row_num=" << output_rowset->num_rows();
-
-        // ATTN(cmy): We found that the num_rows in some rowset meta may be set to the wrong value,
-        // but it is not known which version of the code has the problem. So when the convert
-        // result is inconsistent, we then try to verify by num_rows recorded in segment_groups.
-        // If the check passes, ignore the error and set the correct value in the output rowset meta
-        // to fix this problem.
-        // Only handle alpha rowset because we only find this bug in alpha rowset
-        int64_t num_rows = _get_input_num_rows_from_seg_grps(input_rowset);
-        if (num_rows == -1) {
-            return Status::OLAPInternalError(OLAP_ERR_CHECK_LINES_ERROR);
-        }
-        if (num_rows != output_rowset->num_rows() + stats.merged_rows + stats.filtered_rows) {
-            // If it is still incorrect, it may be another problem
-            LOG(WARNING) << "row_num got from seg groups does not match between cumulative input "
-                            "and output! "
-                         << "input_row_num=" << num_rows << ", merged_row_num=" << stats.merged_rows
-                         << ", filtered_row_num=" << stats.filtered_rows
-                         << ", output_row_num=" << output_rowset->num_rows();
-
-            return Status::OLAPInternalError(OLAP_ERR_CHECK_LINES_ERROR);
-        }
-    }
-    return Status::OK();
-}
-
-int64_t ConvertRowset::_get_input_num_rows_from_seg_grps(RowsetSharedPtr rowset) {
-    int64_t num_rows = 0;
-    for (auto& seg_grp : rowset->rowset_meta()->alpha_rowset_extra_meta_pb().segment_groups()) {
-        num_rows += seg_grp.num_rows();
-    }
-    return num_rows;
-}
-Status ConvertRowset::_modify_rowsets(RowsetSharedPtr input_rowset, RowsetSharedPtr output_rowset) {
-    std::vector<RowsetSharedPtr> input_rowsets;
-    input_rowsets.push_back(input_rowset);
-
-    std::vector<RowsetSharedPtr> output_rowsets;
-    output_rowsets.push_back(output_rowset);
-
-    std::lock_guard<std::shared_mutex> wrlock(_tablet->get_header_lock());
-    RETURN_NOT_OK(_tablet->modify_rowsets(output_rowsets, input_rowsets, true));
-    _tablet->save_meta();
-    return Status::OK();
-}
-} // namespace doris
\ No newline at end of file
diff --git a/be/src/olap/convert_rowset.h b/be/src/olap/convert_rowset.h
deleted file mode 100644
index a691d38624..0000000000
--- a/be/src/olap/convert_rowset.h
+++ /dev/null
@@ -1,42 +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.
-
-#pragma once
-
-#include "olap/merger.h"
-#include "olap/storage_engine.h"
-#include "olap/tablet.h"
-
-namespace doris {
-class DataDir;
-class ConvertRowset {
-public:
-    ConvertRowset(TabletSharedPtr tablet) : _tablet(tablet) {}
-    Status do_convert();
-
-private:
-    Status check_correctness(RowsetSharedPtr input_rowset, RowsetSharedPtr output_rowset,
-                             const Merger::Statistics& stats);
-    int64_t _get_input_num_rows_from_seg_grps(RowsetSharedPtr rowset);
-    Status _modify_rowsets(RowsetSharedPtr input_rowset, RowsetSharedPtr output_rowset);
-
-private:
-    TabletSharedPtr _tablet;
-
-    DISALLOW_COPY_AND_ASSIGN(ConvertRowset);
-};
-} // namespace doris
\ No newline at end of file
diff --git a/be/src/olap/file_stream.cpp b/be/src/olap/file_stream.cpp
deleted file mode 100644
index 54d76d9bac..0000000000
--- a/be/src/olap/file_stream.cpp
+++ /dev/null
@@ -1,187 +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 "olap/file_stream.h"
-
-#include "olap/byte_buffer.h"
-#include "olap/out_stream.h"
-
-namespace doris {
-
-ReadOnlyFileStream::ReadOnlyFileStream(FileHandler* handler, StorageByteBuffer** shared_buffer,
-                                       Decompressor decompressor, uint32_t compress_buffer_size,
-                                       OlapReaderStatistics* stats)
-        : _file_cursor(handler, 0, 0),
-          _compressed_helper(nullptr),
-          _uncompressed(nullptr),
-          _shared_buffer(shared_buffer),
-          _decompressor(decompressor),
-          _compress_buffer_size(compress_buffer_size + sizeof(StreamHead)),
-          _current_compress_position(std::numeric_limits<uint64_t>::max()),
-          _stats(stats) {}
-
-ReadOnlyFileStream::ReadOnlyFileStream(FileHandler* handler, StorageByteBuffer** shared_buffer,
-                                       uint64_t offset, uint64_t length, Decompressor decompressor,
-                                       uint32_t compress_buffer_size, OlapReaderStatistics* stats)
-        : _file_cursor(handler, offset, length),
-          _compressed_helper(nullptr),
-          _uncompressed(nullptr),
-          _shared_buffer(shared_buffer),
-          _decompressor(decompressor),
-          _compress_buffer_size(compress_buffer_size + sizeof(StreamHead)),
-          _current_compress_position(std::numeric_limits<uint64_t>::max()),
-          _stats(stats) {}
-
-Status ReadOnlyFileStream::_assure_data() {
-    // if still has data in uncompressed
-    if (OLAP_LIKELY(_uncompressed != nullptr && _uncompressed->remaining() > 0)) {
-        return Status::OK();
-    } else if (_file_cursor.eof()) {
-        VLOG_TRACE << "STREAM EOF. length=" << _file_cursor.length()
-                   << ", used=" << _file_cursor.position();
-        return Status::OLAPInternalError(OLAP_ERR_COLUMN_STREAM_EOF);
-    }
-
-    StreamHead header;
-    size_t file_cursor_used = _file_cursor.position();
-    Status res = Status::OK();
-    {
-        SCOPED_RAW_TIMER(&_stats->io_ns);
-        res = _file_cursor.read(reinterpret_cast<char*>(&header), sizeof(header));
-        if (OLAP_UNLIKELY(!res.ok())) {
-            LOG(WARNING) << "read header fail";
-            return res;
-        }
-        res = _fill_compressed(header.length);
-        if (OLAP_UNLIKELY(!res.ok())) {
-            LOG(WARNING) << "read header fail";
-            return res;
-        }
-        _stats->compressed_bytes_read += sizeof(header) + header.length;
-    }
-
-    if (header.type == StreamHead::UNCOMPRESSED) {
-        StorageByteBuffer* tmp = _compressed_helper;
-        _compressed_helper = *_shared_buffer;
-        *_shared_buffer = tmp;
-    } else {
-        _compressed_helper->set_position(0);
-        _compressed_helper->set_limit(_compress_buffer_size);
-        {
-            SCOPED_RAW_TIMER(&_stats->decompress_ns);
-            res = _decompressor(*_shared_buffer, _compressed_helper);
-            if (!res.ok()) {
-                LOG(WARNING) << "fail to decompress err=" << res;
-                return res;
-            }
-        }
-    }
-    _stats->uncompressed_bytes_read += _compressed_helper->limit();
-
-    _uncompressed = _compressed_helper;
-    _current_compress_position = file_cursor_used;
-    return res;
-}
-
-// 设置读取的位置
-Status ReadOnlyFileStream::seek(PositionProvider* position) {
-    Status res = Status::OK();
-    // 先seek到解压前的位置,也就是writer中写入的spilled byte
-    int64_t compressed_position = position->get_next();
-    int64_t uncompressed_bytes = position->get_next();
-    if (_current_compress_position == compressed_position && nullptr != _uncompressed) {
-        /*
-         * 多数情况下不会出现_uncompressed为NULL的情况,
-         * 但varchar类型的数据可能会导致查询中出现_uncompressed == nullptr 。
-         * 假设查询恰好命中A压缩块的最后一行, 而相临下一个
-         * 中压缩块varchar全是空串,会导致_uncompressed == nullptr。
-         * 如果后面的segmentreader中还需要再次遍历A压缩块,会出现空指针。
-         */
-    } else {
-        _file_cursor.seek(compressed_position);
-        _uncompressed = nullptr;
-
-        res = _assure_data();
-        if (OLAP_LIKELY(res.ok())) {
-            // assure data will be successful in most case
-        } else if (res.precise_code() == OLAP_ERR_COLUMN_STREAM_EOF) {
-            VLOG_TRACE << "file stream eof.";
-            return res;
-        } else {
-            LOG(WARNING) << "fail to assure data after seek";
-            return res;
-        }
-    }
-
-    res = _uncompressed->set_position(uncompressed_bytes);
-    if (!res.ok()) {
-        LOG(WARNING) << "fail to set position. res= " << res << ", position=" << uncompressed_bytes;
-        return res;
-    }
-
-    return Status::OK();
-}
-
-// 跳过指定的size的流
-Status ReadOnlyFileStream::skip(uint64_t skip_length) {
-    Status res = _assure_data();
-
-    if (!res.ok()) {
-        return res;
-    }
-
-    uint64_t skip_byte = 0;
-    uint64_t byte_to_skip = skip_length;
-
-    // 如果不够跳,则先尝试跳过整个数据块,直到当前的数据
-    // 剩下的字节数足够跳过 又或者时EOF
-    do {
-        skip_byte = std::min(_uncompressed->remaining(), byte_to_skip);
-        _uncompressed->set_position(_uncompressed->position() + skip_byte);
-        byte_to_skip -= skip_byte;
-        // 如果跳到当前的块尽头,那么assure可以换到下一个块
-        // 如果当前块就可以满足skip_length,那么_assure_data没任何作用。
-        res = _assure_data();
-        // while放下面,通常会少判断一次
-    } while (byte_to_skip != 0 && res.ok());
-
-    return res;
-}
-
-Status ReadOnlyFileStream::_fill_compressed(size_t length) {
-    if (length > _compress_buffer_size) {
-        LOG(WARNING) << "overflow when fill compressed."
-                     << ", length=" << length << ", compress_size" << _compress_buffer_size;
-        return Status::OLAPInternalError(OLAP_ERR_OUT_OF_BOUND);
-    }
-
-    Status res = _file_cursor.read((*_shared_buffer)->array(), length);
-    if (!res.ok()) {
-        LOG(WARNING) << "fail to fill compressed buffer.";
-        return res;
-    }
-
-    (*_shared_buffer)->set_position(0);
-    (*_shared_buffer)->set_limit(length);
-    return res;
-}
-
-uint64_t ReadOnlyFileStream::available() {
-    return _file_cursor.remain();
-}
-
-} // namespace doris
diff --git a/be/src/olap/file_stream.h b/be/src/olap/file_stream.h
deleted file mode 100644
index 176a3cd22c..0000000000
--- a/be/src/olap/file_stream.h
+++ /dev/null
@@ -1,276 +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.
-
-#pragma once
-
-#include <gen_cpp/column_data_file.pb.h>
-
-#include <iostream>
-#include <istream>
-#include <streambuf>
-#include <vector>
-
-#include "olap/byte_buffer.h"
-#include "olap/compress.h"
-#include "olap/file_helper.h"
-#include "olap/olap_common.h"
-#include "olap/stream_index_reader.h"
-#include "util/runtime_profile.h"
-
-namespace doris {
-
-// Define the input data stream interface.
-class ReadOnlyFileStream {
-public:
-    // Construct method, use a group of ByteBuffer to create an InStream The position of the input
-    // ByteBuffer in the stream can be discontinuous, for example, certain data is not required to be
-    // determined by Index. After reading, this part of the data is not read. However, InStream
-    // encapsulates the fact that the ByteBuffer is discontinuous. From the perspective of the
-    // upper-layer user, it is still accessing a continuous stream. The upper-layer user should
-    // ensure that the StorageByteBuffer is not read. Void locations with no data in between.
-    //
-    // When mmap is used, it will degenerate to only one ByteBuffer, whether to use mmap depends on
-    // the test results in the performance tuning phase.
-    //
-    // Input:
-    //     inputs - A set of ByteBuffer holds the data in a specific stream.
-    //     offsets - The offset position of the data of each ByteBuffer in the input stream in the stream.
-    //     length - The total byte length of the stream.
-    //     Decompressor - Provides a decompression function if the stream is compressed, otherwise it is 'NULL'.
-    //     compress_buffer_size - If compression is used, give the compressed block size.
-    ReadOnlyFileStream(FileHandler* handler, StorageByteBuffer** shared_buffer,
-                       Decompressor decompressor, uint32_t compress_buffer_size,
-                       OlapReaderStatistics* stats);
-
-    ReadOnlyFileStream(FileHandler* handler, StorageByteBuffer** shared_buffer, uint64_t offset,
-                       uint64_t length, Decompressor decompressor, uint32_t compress_buffer_size,
-                       OlapReaderStatistics* stats);
-
-    ~ReadOnlyFileStream() { SAFE_DELETE(_compressed_helper); }
-
-    Status init() {
-        _compressed_helper = StorageByteBuffer::create(_compress_buffer_size);
-        if (nullptr == _compressed_helper) {
-            LOG(WARNING) << "fail to create compressed buffer";
-            return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
-        }
-
-        _uncompressed = nullptr;
-        return Status::OK();
-    }
-
-    void reset(uint64_t offset, uint64_t length) { _file_cursor.reset(offset, length); }
-
-    // Read a byte from the data stream, move the internal pointer backward
-    // If the stream ends, it returns `Status::OLAPInternalError(OLAP_ERR_COLUMN_STREAM_EOF)`.
-    Status read(char* byte);
-
-    // Read a piece of data from a data stream.
-    // Input:
-    //     buffer - Store read data.
-    //     buf_size - The size of the buffer is given when inputting,
-    //                and the number of bytes actually read is given when returning.
-    // If the stream ends, it returns `Status::OLAPInternalError(OLAP_ERR_COLUMN_STREAM_EOF)`.
-    Status read(char* buffer, uint64_t* buf_size);
-
-    Status read_all(char* buffer, uint64_t* buf_size);
-    // set read position.
-    Status seek(PositionProvider* position);
-
-    // Skip streams of specified size.
-    Status skip(uint64_t skip_length);
-
-    // Returns the total length of the stream.
-    uint64_t stream_length() { return _file_cursor.length(); }
-
-    bool eof() {
-        if (_uncompressed == nullptr) {
-            return _file_cursor.eof();
-        } else {
-            return _file_cursor.eof() && _uncompressed->remaining() == 0;
-        }
-    }
-
-    // Returns the remaining readable bytes of the current block.
-    uint64_t available();
-
-    size_t get_buffer_size() { return _compress_buffer_size; }
-
-    void get_buf(char** buf, uint32_t* remaining_bytes) {
-        if (UNLIKELY(_uncompressed == nullptr)) {
-            *buf = nullptr;
-            *remaining_bytes = 0;
-        } else {
-            *buf = _uncompressed->array();
-            *remaining_bytes = _uncompressed->remaining();
-        }
-    }
-
-    void get_position(uint32_t* position) { *position = _uncompressed->position(); }
-
-    void set_position(uint32_t pos) { _uncompressed->set_position(pos); }
-
-    int remaining() {
-        if (_uncompressed == nullptr) {
-            return 0;
-        }
-        return _uncompressed->remaining();
-    }
-
-private:
-    // Use to read a specified range in file
-    class FileCursor {
-    public:
-        FileCursor(FileHandler* file_handler, size_t offset, size_t length)
-                : _file_handler(file_handler), _offset(offset), _length(length), _used(0) {}
-
-        ~FileCursor() {}
-
-        void reset(size_t offset, size_t length) {
-            _offset = offset;
-            _length = length;
-            _used = 0;
-        }
-
-        Status read(char* out_buffer, size_t length) {
-            if (_used + length <= _length) {
-                Status res = _file_handler->pread(out_buffer, length, _used + _offset);
-                if (!res.ok()) {
-                    LOG(WARNING) << "fail to read from file. res = " << res;
-                    return res;
-                }
-
-                _used += length;
-            } else {
-                return Status::OLAPInternalError(OLAP_ERR_COLUMN_STREAM_EOF);
-            }
-
-            return Status::OK();
-        }
-
-        size_t position() { return _used; }
-
-        size_t remain() { return _length - _used; }
-
-        size_t length() { return _length; }
-
-        bool eof() { return _used == _length; }
-
-        Status seek(size_t offset) {
-            if (offset > _length) {
-                return Status::OLAPInternalError(OLAP_ERR_OUT_OF_BOUND);
-            }
-
-            _used = offset;
-            return Status::OK();
-        }
-
-        const std::string& file_name() const { return _file_handler->file_name(); }
-
-        size_t offset() const { return _offset; }
-
-    private:
-        FileHandler* _file_handler;
-        size_t _offset; // start from where
-        size_t _length; // length limit
-        size_t _used;
-    };
-
-    Status _assure_data();
-    Status _fill_compressed(size_t length);
-
-    FileCursor _file_cursor;
-    StorageByteBuffer* _compressed_helper;
-    StorageByteBuffer* _uncompressed;
-    StorageByteBuffer** _shared_buffer;
-
-    Decompressor _decompressor;
-    size_t _compress_buffer_size;
-    size_t _current_compress_position;
-
-    OlapReaderStatistics* _stats;
-
-    DISALLOW_COPY_AND_ASSIGN(ReadOnlyFileStream);
-};
-
-inline Status ReadOnlyFileStream::read(char* byte) {
-    Status res = _assure_data();
-
-    if (!res.ok()) {
-        return res;
-    }
-
-    res = _uncompressed->get(byte);
-    return res;
-}
-
-inline Status ReadOnlyFileStream::read(char* buffer, uint64_t* buf_size) {
-    Status res;
-    uint64_t read_length = *buf_size;
-    *buf_size = 0;
-
-    do {
-        res = _assure_data();
-        if (!res.ok()) {
-            break;
-        }
-
-        uint64_t actual_length = std::min(read_length - *buf_size, _uncompressed->remaining());
-
-        res = _uncompressed->get(buffer, actual_length);
-        if (!res.ok()) {
-            break;
-        }
-
-        *buf_size += actual_length;
-        buffer += actual_length;
-    } while (*buf_size < read_length);
-
-    return res;
-}
-
-inline Status ReadOnlyFileStream::read_all(char* buffer, uint64_t* buffer_size) {
-    Status res;
-    uint64_t read_length = 0;
-    uint64_t buffer_remain = *buffer_size;
-
-    while (_assure_data()) {
-        read_length = _uncompressed->remaining();
-
-        if (buffer_remain < read_length) {
-            res = Status::OLAPInternalError(OLAP_ERR_BUFFER_OVERFLOW);
-            break;
-        }
-
-        res = _uncompressed->get(buffer, read_length);
-        if (!res.ok()) {
-            break;
-        }
-
-        buffer_remain -= read_length;
-        buffer += read_length;
-    }
-
-    if (eof()) {
-        *buffer_size -= buffer_remain;
-        return Status::OK();
-    }
-
-    return res;
-}
-
-} // namespace doris
diff --git a/be/src/olap/in_list_predicate.h b/be/src/olap/in_list_predicate.h
index 2a0fcb0901..3d22ecb83f 100644
--- a/be/src/olap/in_list_predicate.h
+++ b/be/src/olap/in_list_predicate.h
@@ -26,7 +26,6 @@
 #include "decimal12.h"
 #include "olap/column_predicate.h"
 #include "runtime/string_value.h"
-#include "runtime/vectorized_row_batch.h"
 #include "uint24.h"
 #include "vec/columns/column_dictionary.h"
 #include "vec/core/types.h"
@@ -77,8 +76,6 @@ struct equal_to<doris::uint24_t> {
 
 namespace doris {
 
-class VectorizedRowBatch;
-
 template <class T, PredicateType PT>
 class InListPredicateBase : public ColumnPredicate {
 public:
@@ -88,57 +85,6 @@ public:
 
     PredicateType type() const override { return PT; }
 
-    void evaluate(VectorizedRowBatch* batch) const override {
-        uint16_t n = batch->size();
-        if (!n) {
-            return;
-        }
-
-        uint16_t* sel = batch->selected();
-        const T* col_vector = reinterpret_cast<const T*>(batch->column(_column_id)->col_data());
-        uint16_t new_size = 0;
-        if (batch->column(_column_id)->no_nulls()) {
-            if (batch->selected_in_use()) {
-                for (uint16_t j = 0; j != n; ++j) {
-                    uint16_t i = sel[j];
-                    sel[new_size] = i;
-                    new_size += _operator(_values.find(col_vector[i]), _values.end());
-                }
-                batch->set_size(new_size);
-            } else {
-                for (uint16_t i = 0; i != n; ++i) {
-                    sel[new_size] = i;
-                    new_size += _operator(_values.find(col_vector[i]), _values.end());
-                }
-                if (new_size < n) {
-                    batch->set_size(new_size);
-                    batch->set_selected_in_use(true);
-                }
-            }
-        } else {
-            bool* is_null = batch->column(_column_id)->is_null();
-            if (batch->selected_in_use()) {
-                for (uint16_t j = 0; j != n; ++j) {
-                    uint16_t i = sel[j];
-                    sel[new_size] = i;
-                    new_size +=
-                            (!is_null[i] && _operator(_values.find(col_vector[i]), _values.end()));
-                }
-                batch->set_size(new_size);
-            } else {
-                for (int i = 0; i != n; ++i) {
-                    sel[new_size] = i;
-                    new_size +=
-                            (!is_null[i] && _operator(_values.find(col_vector[i]), _values.end()));
-                }
-                if (new_size < n) {
-                    batch->set_size(new_size);
-                    batch->set_selected_in_use(true);
-                }
-            }
-        }
-    };
-
     void evaluate(ColumnBlock* block, uint16_t* sel, uint16_t* size) const override {
         if (block->is_nullable()) {
             _base_evaluate<true>(block, sel, size);
diff --git a/be/src/olap/in_stream.cpp b/be/src/olap/in_stream.cpp
deleted file mode 100644
index 40c45a6f22..0000000000
--- a/be/src/olap/in_stream.cpp
+++ /dev/null
@@ -1,305 +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 "olap/in_stream.h"
-
-#include "olap/byte_buffer.h"
-#include "olap/out_stream.h"
-
-namespace doris {
-
-InStream::InStream(std::vector<StorageByteBuffer*>* inputs, const std::vector<uint64_t>& offsets,
-                   uint64_t length, Decompressor decompressor, uint32_t compress_buffer_size)
-        : _inputs(*inputs),
-          _offsets(offsets),
-          _length(length),
-          _decompressor(decompressor),
-          _compress_buffer_size(compress_buffer_size),
-          _current_offset(0),
-          _current_range(0),
-          _compressed(nullptr),
-          _uncompressed(nullptr) {}
-
-InStream::~InStream() {
-    SAFE_DELETE(_compressed);
-    SAFE_DELETE(_uncompressed);
-}
-
-Status InStream::_slice(uint64_t chunk_size, StorageByteBuffer** out_slice) {
-    uint64_t len = chunk_size;
-    uint64_t old_offset = _current_offset;
-    StorageByteBuffer* slice = nullptr;
-
-    //如果buffer够读,拿出一个chunksize,并设置position
-    if (OLAP_LIKELY(_compressed->remaining() >= len)) {
-        slice = StorageByteBuffer::reference_buffer(_compressed, _compressed->position(), len);
-
-        if (OLAP_UNLIKELY(nullptr == slice)) {
-            return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
-        }
-
-        _compressed->set_position(_compressed->position() + len);
-        *out_slice = slice;
-        // 这里之前没有设置_current_offset
-        _current_offset += len;
-        return Status::OK();
-    } else if (_current_range >= _inputs.size() - 1) {
-        // 如果buffer用完了
-        LOG(WARNING) << "EOF in InStream. [Need=" << chunk_size << "]";
-        return Status::OLAPInternalError(OLAP_ERR_OUT_OF_BOUND);
-    }
-
-    // 这里并不分配chuck_size, 而是分配一个最大值, 这样利于减少内存碎片
-    slice = StorageByteBuffer::create(_compress_buffer_size);
-
-    if (OLAP_UNLIKELY(nullptr == slice)) {
-        return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
-    }
-
-    // 当前的compress里的buffer不够了
-    _current_offset += _compressed->remaining();
-    len -= _compressed->remaining();
-    // 先拿一部分出来
-    slice->put(&(_compressed->array()[_compressed->position()]), _compressed->remaining());
-
-    // 向后边的buffer移动
-    ++_current_range;
-
-    while (len > 0 && _current_range < _inputs.size()) {
-        SAFE_DELETE(_compressed);
-        // 再取一部分压缩过的buffer
-        _compressed = StorageByteBuffer::reference_buffer(_inputs[_current_range],
-                                                          _inputs[_current_range]->position(),
-                                                          _inputs[_current_range]->remaining());
-
-        if (OLAP_UNLIKELY(nullptr == _compressed)) {
-            SAFE_DELETE(slice);
-            return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
-        }
-
-        // 如果剩下的大于需要取的部分,拿出来
-        // 否则继续这个过程
-        if (_compressed->remaining() >= len) {
-            slice->put(&(_compressed->array()[_compressed->position()]), len);
-            _compressed->set_position(_compressed->position() + len);
-            _current_offset += len;
-            slice->flip();
-            *out_slice = slice;
-            return Status::OK();
-        } else {
-            _current_offset += _compressed->remaining();
-            len -= _compressed->remaining();
-            slice->put(&(_compressed->array()[_compressed->position()]), _compressed->remaining());
-        }
-
-        ++_current_range;
-    }
-
-    // 到这里就意味着上边的循环里没有取到足够的buf
-    // 回退到进来之前的状态
-    _seek(old_offset);
-    LOG(WARNING) << "EOF in InStream. [Need=" << chunk_size << "]";
-    return Status::OLAPInternalError(OLAP_ERR_OUT_OF_BOUND);
-}
-
-Status InStream::_assure_data() {
-    Status res = Status::OK();
-
-    if (OLAP_LIKELY(_uncompressed != nullptr && _uncompressed->remaining() > 0)) {
-        return Status::OK();
-    } else if (OLAP_UNLIKELY((_uncompressed == nullptr || _uncompressed->remaining() == 0) &&
-                             (_current_offset == _length))) {
-        return Status::OLAPInternalError(OLAP_ERR_COLUMN_STREAM_EOF);
-    }
-
-    // read head and data
-    SAFE_DELETE(_uncompressed);
-
-    // 到这里说明当前uncompress没有什么可以读了,input拿数据
-    // 如果没有compress。或者compress耗尽,用_seek向后一个buff移动
-    if (_compressed == nullptr || _compressed->remaining() == 0) {
-        res = _seek(_current_offset);
-        if (!res.ok()) {
-            return res;
-        }
-    }
-
-    // 取了compress之后,看下是不是能拿出一个head的长度,如果可以
-    // 则解析head,从outstream代码看header不会分开写在两个压缩块中
-    if (OLAP_LIKELY(_compressed->remaining() >= sizeof(StreamHead))) {
-        // 如果可以,从compress中拿出一个head,head是未压缩的。
-        StreamHead head;
-        _compressed->get((char*)&head, sizeof(head));
-
-        if (head.length > _compress_buffer_size) {
-            LOG(WARNING) << "chunk size is larger than buffer size. [chunk=" << head.length
-                         << " buffer_size=" << _compress_buffer_size << "]";
-            return Status::OLAPInternalError(OLAP_ERR_COLUMN_READ_STREAM);
-        }
-
-        // 向后移动整体偏移
-        _current_offset += sizeof(StreamHead);
-        StorageByteBuffer* slice = nullptr;
-
-        // 根据head取一块buf,这里应该要调整_current_offset
-        res = _slice(head.length, &slice);
-        if (OLAP_LIKELY(!res.ok())) {
-            LOG(WARNING) << "fail to slice data from stream.";
-            return Status::OLAPInternalError(OLAP_ERR_COLUMN_READ_STREAM);
-        }
-
-        // 如果没压缩,就直接读这块
-        // 否则需要解压下
-        if (head.type == StreamHead::UNCOMPRESSED) {
-            _uncompressed = slice;
-        } else {
-            _uncompressed = StorageByteBuffer::create(_compress_buffer_size);
-
-            if (OLAP_UNLIKELY(nullptr == _uncompressed)) {
-                res = Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
-            } else {
-                res = _decompressor(slice, _uncompressed);
-            }
-
-            // 一定要释放掉slice
-            SAFE_DELETE(slice);
-        }
-    } else {
-        LOG(WARNING) << "compressed remaining size less than stream head size. "
-                        "[compressed_remaining_size="
-                     << _compressed->remaining() << " stream_head_size=" << sizeof(StreamHead)
-                     << "]";
-        return Status::OLAPInternalError(OLAP_ERR_COLUMN_READ_STREAM);
-    }
-
-    return res;
-}
-
-uint64_t InStream::available() {
-    Status res;
-    res = _assure_data();
-
-    if (!res.ok()) {
-        return 0;
-    }
-
-    return _uncompressed->remaining();
-}
-
-// seek的是解压前的数据。
-Status InStream::_seek(uint64_t position) {
-    for (uint32_t i = 0; i < _inputs.size(); i++) {
-        if (_offsets[i] <= position && position - _offsets[i] < _inputs[i]->remaining()) {
-            // don't need to malloc _compressed if current range don't be changed.
-            if (!(_current_range == i && nullptr != _compressed)) {
-                _current_range = i;
-                SAFE_DELETE(_compressed);
-                _compressed =
-                        StorageByteBuffer::reference_buffer(_inputs[i], 0, _inputs[i]->remaining());
-            }
-
-            uint64_t pos = _inputs[i]->position() + position - _offsets[i];
-            _compressed->set_position(pos);
-            _current_offset = position;
-            return Status::OK();
-        }
-    }
-
-    SAFE_DELETE(_compressed);
-
-    if (!_inputs.empty() && position == _offsets.back() + _inputs.back()->remaining()) {
-        _current_range = _inputs.size() - 1;
-        _compressed = StorageByteBuffer::reference_buffer(_inputs[_current_range], 0,
-                                                          _inputs[_current_range]->limit());
-        _current_offset = position;
-        return Status::OK();
-    }
-
-    return Status::OLAPInternalError(OLAP_ERR_OUT_OF_BOUND);
-}
-
-Status InStream::seek(PositionProvider* position) {
-    Status res = Status::OK();
-
-    // 先seek到解压前的位置,也就是writer中写入的spilled byte
-    res = _seek(position->get_next());
-    if (!res.ok()) {
-        LOG(WARNING) << "fail to seek.res = " << res;
-        return res;
-    }
-
-    // 再定位到已经解压的内存中的位置
-    long uncompressed_bytes = position->get_next();
-
-    if (uncompressed_bytes != 0) {
-        SAFE_DELETE(_uncompressed);
-        res = _assure_data();
-
-        if (!res.ok()) {
-            LOG(WARNING) << "fail to assure data.res = " << res;
-            return res;
-        }
-
-        res = _uncompressed->set_position(uncompressed_bytes);
-
-        if (!res.ok()) {
-            LOG(WARNING) << "fail to set position. res= " << res
-                         << ", position=" << (_uncompressed->position() + uncompressed_bytes);
-            return res;
-        }
-    } else if (_uncompressed != nullptr) {
-        // mark the uncompressed buffer as done
-        res = _uncompressed->set_position(_uncompressed->limit());
-
-        if (!res.ok()) {
-            LOG(WARNING) << "fail to set position.res=" << res
-                         << ", position=" << _uncompressed->limit();
-            return res;
-        }
-    }
-
-    return Status::OK();
-}
-
-// skip的是解压后的数据
-Status InStream::skip(uint64_t skip_length) {
-    Status res = _assure_data();
-
-    if (!res.ok()) {
-        return res;
-    }
-
-    uint64_t skip_byte = 0;
-    uint64_t byte_to_skip = skip_length;
-
-    // 如果不够跳,则先尝试跳过整个数据块,直到当前的数据块剩下的字节数足够跳过
-    // 又或者是EOF
-    do {
-        skip_byte = std::min(_uncompressed->remaining(), byte_to_skip);
-        _uncompressed->set_position(_uncompressed->position() + skip_byte);
-
-        byte_to_skip -= skip_byte;
-        // 如果跳到当前块的尽头,那么assure可以换到下一个块。
-        // 如果当前块就可以满足skip_length,那么_assure_data没任何作用。
-        res = _assure_data();
-        // while 放下边,通常会少一次判断
-    } while (byte_to_skip != 0 && res.ok());
-
-    return res;
-}
-
-} // namespace doris
diff --git a/be/src/olap/in_stream.h b/be/src/olap/in_stream.h
deleted file mode 100644
index 6e054c9e1a..0000000000
--- a/be/src/olap/in_stream.h
+++ /dev/null
@@ -1,191 +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.
-
-#pragma once
-
-#include <gen_cpp/column_data_file.pb.h>
-
-#include <iostream>
-#include <istream>
-#include <streambuf>
-#include <vector>
-
-#include "olap/byte_buffer.h"
-#include "olap/compress.h"
-#include "olap/olap_common.h"
-#include "olap/stream_index_reader.h"
-
-namespace doris {
-
-// 提供Column Reader的Seek位置, 由于ColumnReader的seek需要多个position地址
-// PositionProvider 提供了next方法, 将对position的操作封装为stack的形式
-
-// 定义输入数据流接口
-class InStream {
-public:
-    // 构造方法, 使用一组ByteBuffer创建一个InStream
-    // 输入的ByteBuffer在流中的位置可以不连续,例如
-    // 通过Index确定某些数据不需要
-    // 读取后,则不读入这部分的数据. 但InStream封装
-    // 了ByteBuffer不连续这一事实,
-    // 从上层使用者来看,依旧是在访问一段连续的流.
-    // 上层使用者应该保证不读取StorageByteBuffer
-    // 之间没有数据的空洞位置.
-    //
-    // 当使用mmap的时候,这里会退化为只有一个ByteBuffer, 是
-    // ??使用mmap取决于在性能
-    // 调优阶段的测试结果
-    //
-    // Input:
-    //     inputs - 一组ByteBuffer保存具体的流中的数据
-    //     offsets - input中每个ByteBuffer的数据在流中的偏移位置
-    //     length - 流的总字节长度
-    //     Decompressor - 如果流被压缩过,则提供一个解压缩函数,否则为NULL
-    //     compress_buffer_size - 如果使用压缩,给出压缩的块大小
-    explicit InStream(std::vector<StorageByteBuffer*>* inputs, const std::vector<uint64_t>& offsets,
-                      uint64_t length, Decompressor decompressor, uint32_t compress_buffer_size);
-
-    ~InStream();
-
-    // 从数据流中读取一个字节,内部指针后移
-    // 如果数据流结束, 返回Status::OLAPInternalError(OLAP_ERR_COLUMN_STREAM_EOF)
-    Status read(char* byte);
-
-    // 从数据流读入一段数据
-    // Input:
-    //     buffer - 存储读入的数据
-    //     buf_size - 输入时给出buffer的大小,返回时给出实际读取的字节数
-    // 如果数据流结束, 返回Status::OLAPInternalError(OLAP_ERR_COLUMN_STREAM_EOF)
-    Status read(char* buffer, uint64_t* buf_size);
-
-    // 设置读取的位置
-    Status seek(PositionProvider* position);
-
-    // 跳过指定size的流
-    Status skip(uint64_t skip_length);
-
-    // 返回流的總長度
-    uint64_t stream_length() {
-        uint64_t length = 0;
-
-        for (size_t buffer_index = 0; buffer_index < _inputs.size(); ++buffer_index) {
-            length += _inputs[buffer_index]->limit();
-        }
-
-        return length;
-    }
-
-    uint64_t estimate_uncompressed_length() { return _inputs.size() * _compress_buffer_size; }
-
-    bool eof() { return _current_offset == _length; }
-
-    // 返回当前块剩余可读字节数
-    uint64_t available();
-
-    // 返回当前块剩余的内存
-    const char* available_buffer() {
-        if (_assure_data()) {
-            size_t offset = _uncompressed->position();
-            return _uncompressed->array(offset);
-        }
-
-        return nullptr;
-    }
-
-private:
-    Status _assure_data();
-    Status _slice(uint64_t chunk_size, StorageByteBuffer** out_slice);
-    Status _seek(uint64_t position);
-
-    std::vector<StorageByteBuffer*> _inputs;
-    std::vector<uint64_t> _offsets;
-    uint64_t _length;
-    Decompressor _decompressor;
-    uint32_t _compress_buffer_size;
-    uint64_t _current_offset;
-    uint64_t _current_range;
-    StorageByteBuffer* _compressed;
-    StorageByteBuffer* _uncompressed;
-
-    DISALLOW_COPY_AND_ASSIGN(InStream);
-};
-
-// byte buffer的封装, 用于流式读取(暂时用于支持pb的流式反序列化)
-// 其实也可以直接和instream合在一起,先这么写着
-class InStreamBufferWrapper : public std::streambuf {
-public:
-    InStreamBufferWrapper(InStream* input) : std::streambuf(), _stream(input), _skip_size(0) {}
-    virtual ~InStreamBufferWrapper() {}
-    virtual int_type underflow() {
-        if (nullptr != _stream) {
-            if (_stream->skip(_skip_size)) {
-                char* buf = const_cast<char*>(_stream->available_buffer());
-
-                if (nullptr != buf) {
-                    size_t read_length = _stream->available();
-                    setg(buf, buf, buf + read_length);
-                    _skip_size = read_length;
-                    return traits_type::to_int_type(*gptr());
-                }
-            }
-        }
-
-        return traits_type::eof();
-    }
-
-protected:
-    InStream* _stream;
-    size_t _skip_size;
-};
-
-inline Status InStream::read(char* byte) {
-    Status res;
-
-    if (!(res = _assure_data())) {
-        return res;
-    }
-
-    return _uncompressed->get(byte);
-}
-
-inline Status InStream::read(char* buffer, uint64_t* buf_size) {
-    Status res;
-
-    uint64_t read_length = *buf_size;
-    *buf_size = 0;
-
-    do {
-        res = _assure_data();
-        if (!res.ok()) {
-            break;
-        }
-
-        uint64_t actual_length = std::min(read_length - *buf_size, _uncompressed->remaining());
-
-        res = _uncompressed->get(buffer, actual_length);
-        if (!res.ok()) {
-            break;
-        }
-
-        *buf_size += actual_length;
-        buffer += actual_length;
-    } while (*buf_size < read_length);
-
-    return res;
-}
-
-} // namespace doris
diff --git a/be/src/olap/lru_cache.cpp b/be/src/olap/lru_cache.cpp
index 19d8b5d4e9..146ab34c4e 100644
--- a/be/src/olap/lru_cache.cpp
+++ b/be/src/olap/lru_cache.cpp
@@ -14,7 +14,6 @@
 #include "gutil/bits.h"
 #include "olap/olap_common.h"
 #include "olap/olap_define.h"
-#include "olap/olap_index.h"
 #include "olap/row_block.h"
 #include "olap/utils.h"
 #include "runtime/thread_context.h"
diff --git a/be/src/olap/memtable.cpp b/be/src/olap/memtable.cpp
index 1b44d0ee25..227312d421 100644
--- a/be/src/olap/memtable.cpp
+++ b/be/src/olap/memtable.cpp
@@ -19,7 +19,6 @@
 
 #include "common/logging.h"
 #include "olap/row.h"
-#include "olap/rowset/column_data_writer.h"
 #include "olap/rowset/rowset_writer.h"
 #include "olap/schema.h"
 #include "runtime/tuple.h"
diff --git a/be/src/olap/null_predicate.cpp b/be/src/olap/null_predicate.cpp
index bdafda6ee7..d755fd34eb 100644
--- a/be/src/olap/null_predicate.cpp
+++ b/be/src/olap/null_predicate.cpp
@@ -19,7 +19,6 @@
 
 #include "olap/field.h"
 #include "runtime/string_value.hpp"
-#include "runtime/vectorized_row_batch.h"
 #include "vec/columns/column_nullable.h"
 
 using namespace doris::vectorized;
@@ -33,43 +32,6 @@ PredicateType NullPredicate::type() const {
     return _is_null ? PredicateType::IS_NULL : PredicateType::IS_NOT_NULL;
 }
 
-void NullPredicate::evaluate(VectorizedRowBatch* batch) const {
-    uint16_t n = batch->size();
-    if (n == 0) {
-        return;
-    }
-    uint16_t* sel = batch->selected();
-    bool* null_array = batch->column(_column_id)->is_null();
-    if (batch->column(_column_id)->no_nulls() && _is_null) {
-        batch->set_size(0);
-        batch->set_selected_in_use(true);
-        return;
-    }
-
-    if (batch->column(_column_id)->no_nulls() && !_is_null) {
-        return;
-    }
-
-    uint16_t new_size = 0;
-    if (batch->selected_in_use()) {
-        for (uint16_t j = 0; j != n; ++j) {
-            uint16_t i = sel[j];
-            sel[new_size] = i;
-            new_size += (null_array[i] == _is_null);
-        }
-        batch->set_size(new_size);
-    } else {
-        for (uint16_t i = 0; i != n; ++i) {
-            sel[new_size] = i;
-            new_size += (null_array[i] == _is_null);
-        }
-        if (new_size < n) {
-            batch->set_size(new_size);
-            batch->set_selected_in_use(true);
-        }
-    }
-}
-
 void NullPredicate::evaluate(ColumnBlock* block, uint16_t* sel, uint16_t* size) const {
     uint16_t new_size = 0;
     if (!block->is_nullable() && _is_null) {
diff --git a/be/src/olap/null_predicate.h b/be/src/olap/null_predicate.h
index c1eee63268..caf05af16d 100644
--- a/be/src/olap/null_predicate.h
+++ b/be/src/olap/null_predicate.h
@@ -25,16 +25,12 @@
 
 namespace doris {
 
-class VectorizedRowBatch;
-
 class NullPredicate : public ColumnPredicate {
 public:
     NullPredicate(uint32_t column_id, bool is_null, bool opposite = false);
 
     PredicateType type() const override;
 
-    void evaluate(VectorizedRowBatch* batch) const override;
-
     void evaluate(ColumnBlock* block, uint16_t* sel, uint16_t* size) const override;
 
     void evaluate_or(ColumnBlock* block, uint16_t* sel, uint16_t size, bool* flags) const override;
diff --git a/be/src/olap/olap_index.cpp b/be/src/olap/olap_index.cpp
deleted file mode 100644
index 02d8a7d4eb..0000000000
--- a/be/src/olap/olap_index.cpp
+++ /dev/null
@@ -1,525 +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 "olap/olap_index.h"
-
-#include <algorithm>
-#include <cassert>
-#include <cmath>
-#include <fstream>
-
-#include "olap/row_block.h"
-#include "olap/row_cursor.h"
-#include "olap/rowset/column_data.h"
-#include "olap/utils.h"
-#include "olap/wrapper_field.h"
-
-using std::ifstream;
-using std::string;
-using std::vector;
-
-namespace doris {
-
-MemIndex::MemIndex()
-        : _key_length(0),
-          _num_entries(0),
-          _index_size(0),
-          _data_size(0),
-          _num_rows(0),
-          _mem_pool(new MemPool("MemIndex")) {}
-
-MemIndex::~MemIndex() {
-    _num_entries = 0;
-    for (vector<SegmentMetaInfo>::iterator it = _meta.begin(); it != _meta.end(); ++it) {
-        free(it->buffer.data);
-        it->buffer.data = nullptr;
-        it->buffer.length = 0;
-    }
-}
-
-Status MemIndex::load_segment(const char* file, size_t* current_num_rows_per_row_block,
-                              bool use_cache) {
-    Status res = Status::OK();
-
-    SegmentMetaInfo meta;
-    uint32_t adler_checksum = 0;
-    uint32_t num_entries = 0;
-
-    if (file == nullptr) {
-        res = Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
-        LOG(WARNING) << "load index error. file=" << file << ", res=" << res;
-        return res;
-    }
-
-    FileHandler file_handler;
-    if (use_cache) {
-        if ((res = file_handler.open_with_cache(file, O_RDONLY)) != Status::OK()) {
-            LOG(WARNING) << "open index error. file=" << file << ", res=" << res;
-            return res;
-        }
-    } else {
-        if ((res = file_handler.open(file, O_RDONLY)) != Status::OK()) {
-            LOG(WARNING) << "open index error. file=" << file << ", res=" << res;
-            return res;
-        }
-    }
-
-    if ((res = meta.file_header.unserialize(&file_handler)) != Status::OK()) {
-        LOG(WARNING) << "load index error. file=" << file << ", res=" << res;
-        file_handler.close();
-        return res;
-    }
-
-    // 允许索引内容为空
-    // 索引长度必须为索引项长度的整数倍
-    size_t storage_length = meta.file_header.file_length() - meta.file_header.size();
-    bool null_supported = false;
-    //null_supported是为了兼容之前没有NULL字节的数据。
-    //目前索引里面都加入了NULL的标志位,entry length都算了NULL标志位构成的bytes
-    //对于没有标志位的索引,读取数据之后需要对每个字段补齐这部分。
-    if (!meta.file_header.message().has_null_supported()) {
-        null_supported = false;
-    } else {
-        null_supported = meta.file_header.message().null_supported();
-    }
-    size_t num_short_key_columns = short_key_num();
-    bool is_align = false;
-    if (!null_supported) {
-        is_align = (0 == storage_length % (entry_length() - num_short_key_columns));
-    } else {
-        is_align = (0 == storage_length % entry_length());
-    }
-    if (!is_align) {
-        res = Status::OLAPInternalError(OLAP_ERR_INDEX_LOAD_ERROR);
-        LOG(WARNING) << "load index error. file=" << file << ", res=" << res;
-        file_handler.close();
-        return res;
-    }
-
-    // calculate the total size of all segments
-    if (!null_supported) {
-        _index_size += meta.file_header.file_length() + num_entries * num_short_key_columns;
-        num_entries = storage_length / (entry_length() - num_short_key_columns);
-    } else {
-        _index_size += meta.file_header.file_length();
-        num_entries = storage_length / entry_length();
-    }
-    _data_size += meta.file_header.extra().data_length;
-    _num_rows += meta.file_header.extra().num_rows;
-
-    meta.range.first = _num_entries;
-    meta.range.last = meta.range.first + num_entries;
-    _num_entries = meta.range.last;
-    _meta.push_back(meta);
-
-    (current_num_rows_per_row_block == nullptr ||
-     (*current_num_rows_per_row_block = meta.file_header.message().num_rows_per_block()));
-
-    if (OLAP_UNLIKELY(num_entries == 0)) {
-        file_handler.close();
-        return Status::OK();
-    }
-
-    // convert index memory layout for string type
-    // previous layout is size|data,
-    // target type is ptr|size, ptr pointer to data
-    char* storage_data = reinterpret_cast<char*>(calloc(storage_length, 1));
-    if (storage_data == nullptr) {
-        res = Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
-        LOG(WARNING) << "load segment for loading index error. file=" << file << " res=" << res;
-        file_handler.close();
-        return res;
-    }
-
-    // 读取索引内容
-    // 为了启动加速,此处可使用mmap方式。
-    if (file_handler.pread(storage_data, storage_length, meta.file_header.size()) != Status::OK()) {
-        res = Status::OLAPInternalError(OLAP_ERR_IO_ERROR);
-        LOG(WARNING) << "load segment for loading index error. file=" << file << "res=" << res;
-        file_handler.close();
-        free(storage_data);
-        return res;
-    }
-
-    // checksum validation
-    adler_checksum = olap_adler32(ADLER32_INIT, storage_data, storage_length);
-    if (adler_checksum != meta.file_header.checksum()) {
-        res = Status::OLAPInternalError(OLAP_ERR_INDEX_CHECKSUM_ERROR);
-        LOG(WARNING) << "checksum validation error.";
-        LOG(WARNING) << "load segment for loading index error. file=" << file << "res=" << res;
-        file_handler.close();
-        free(storage_data);
-        return res;
-    }
-
-    /*
-     * convert storage layout to memory layout for olap/index
-     * In this procedure, string type(Varchar/Char) should be
-     * converted with caution. Hyperloglog type will not be
-     * key, it can not to be handled.
-     */
-
-    size_t storage_row_bytes = entry_length();
-    storage_row_bytes -= (null_supported ? 0 : num_short_key_columns);
-    char* storage_ptr = storage_data;
-    size_t storage_field_offset = 0;
-
-    size_t mem_row_bytes = new_entry_length();
-    char* mem_buf = reinterpret_cast<char*>(calloc(num_entries * mem_row_bytes, 1));
-    memset(mem_buf, 0, num_entries * mem_row_bytes);
-    char* mem_ptr = mem_buf;
-    size_t mem_field_offset = 0;
-
-    size_t null_byte = null_supported ? 1 : 0;
-    for (size_t i = 0; i < num_short_key_columns; ++i) {
-        const TabletColumn& column = (*_short_key_columns)[i];
-        storage_ptr = storage_data + storage_field_offset;
-        storage_field_offset += column.index_length() + null_byte;
-        mem_ptr = mem_buf + mem_field_offset;
-        if (column.type() == OLAP_FIELD_TYPE_VARCHAR) {
-            mem_field_offset += sizeof(Slice) + 1;
-            for (size_t j = 0; j < num_entries; ++j) {
-                /*
-                 * Varchar is null_byte|length|content in OlapIndex storage
-                 * Varchar is in nullbyte|length|ptr in memory
-                 * We need copy three part: nullbyte|length|content
-                 * 1. copy null byte
-                 * 2. copy length and content into addrs pointed by ptr
-                 */
-
-                // 1. copy null_byte
-                memory_copy(mem_ptr, storage_ptr, null_byte);
-
-                // 2. copy length and content
-                bool is_null = *reinterpret_cast<bool*>(mem_ptr);
-                if (!is_null) {
-                    size_t storage_field_bytes =
-                            *reinterpret_cast<VarcharLengthType*>(storage_ptr + null_byte);
-                    Slice* slice = reinterpret_cast<Slice*>(mem_ptr + 1);
-                    char* data = reinterpret_cast<char*>(_mem_pool->allocate(storage_field_bytes));
-                    memory_copy(data, storage_ptr + sizeof(VarcharLengthType) + null_byte,
-                                storage_field_bytes);
-                    slice->data = data;
-                    slice->size = storage_field_bytes;
-                }
-
-                mem_ptr += mem_row_bytes;
-                storage_ptr += storage_row_bytes;
-            }
-        } else if (column.type() == OLAP_FIELD_TYPE_STRING) {
-            mem_field_offset += sizeof(Slice) + 1;
-            for (size_t j = 0; j < num_entries; ++j) {
-                /*
-                 * string is null_byte|length|content in OlapIndex storage
-                 * string is in nullbyte|length|ptr in memory
-                 * We need copy three part: nullbyte|length|content
-                 * 1. copy null byte
-                 * 2. copy length and content into addrs pointed by ptr
-                 */
-
-                // 1. copy null_byte
-                memory_copy(mem_ptr, storage_ptr, null_byte);
-
-                // 2. copy length and content
-                bool is_null = *reinterpret_cast<bool*>(mem_ptr);
-                if (!is_null) {
-                    size_t storage_field_bytes =
-                            *reinterpret_cast<StringLengthType*>(storage_ptr + null_byte);
-                    Slice* slice = reinterpret_cast<Slice*>(mem_ptr + 1);
-                    char* data = reinterpret_cast<char*>(_mem_pool->allocate(storage_field_bytes));
-                    memory_copy(data, storage_ptr + sizeof(StringLengthType) + null_byte,
-                                storage_field_bytes);
-                    slice->data = data;
-                    slice->size = storage_field_bytes;
-                }
-
-                mem_ptr += mem_row_bytes;
-                storage_ptr += storage_row_bytes;
-            }
-        } else if (column.type() == OLAP_FIELD_TYPE_CHAR) {
-            mem_field_offset += sizeof(Slice) + 1;
-            size_t storage_field_bytes = column.index_length();
-            for (size_t j = 0; j < num_entries; ++j) {
-                /*
-                 * Char is in nullbyte|content with fixed length in OlapIndex
-                 * Char is in nullbyte|length|ptr in memory
-                 * We need copy three part: nullbyte|length|content
-                 * 1. copy null byte
-                 * 2. copy length and content into addrs pointed by ptr
-                 */
-
-                // 1. copy null_byte
-                memory_copy(mem_ptr, storage_ptr, null_byte);
-
-                // 2. copy length and content
-                bool is_null = *reinterpret_cast<bool*>(mem_ptr);
-                if (!is_null) {
-                    Slice* slice = reinterpret_cast<Slice*>(mem_ptr + 1);
-                    char* data = reinterpret_cast<char*>(_mem_pool->allocate(storage_field_bytes));
-                    memory_copy(data, storage_ptr + null_byte, storage_field_bytes);
-                    slice->data = data;
-                    slice->size = storage_field_bytes;
-                }
-
-                mem_ptr += mem_row_bytes;
-                storage_ptr += storage_row_bytes;
-            }
-        } else {
-            size_t storage_field_bytes = column.index_length();
-            mem_field_offset += storage_field_bytes + 1;
-            for (size_t j = 0; j < num_entries; ++j) {
-                // 1. copy null_byte
-                memory_copy(mem_ptr, storage_ptr, null_byte);
-
-                // 2. copy content
-                bool is_null = *reinterpret_cast<bool*>(mem_ptr);
-                if (!is_null) {
-                    memory_copy(mem_ptr + 1, storage_ptr + null_byte, storage_field_bytes);
-                }
-
-                mem_ptr += mem_row_bytes;
-                storage_ptr += storage_row_bytes;
-            }
-        }
-    }
-
-    mem_ptr = mem_buf + mem_field_offset;
-    storage_ptr = storage_data + storage_field_offset;
-    size_t data_file_offset = sizeof(data_file_offset_t);
-    for (size_t j = 0; j < num_entries; ++j) {
-        memory_copy(mem_ptr, storage_ptr, data_file_offset);
-        mem_ptr += mem_row_bytes;
-        storage_ptr += storage_row_bytes;
-    }
-
-    _meta.back().buffer.data = mem_buf;
-    _meta.back().buffer.length = num_entries * mem_row_bytes;
-    free(storage_data);
-
-    file_handler.close();
-    return Status::OK();
-}
-
-Status MemIndex::init(size_t short_key_len, size_t new_short_key_len, size_t short_key_num,
-                      std::vector<TabletColumn>* short_key_columns) {
-    if (short_key_columns == nullptr) {
-        LOG(WARNING) << "fail to init MemIndex, nullptr short key columns.";
-        return Status::OLAPInternalError(OLAP_ERR_INDEX_LOAD_ERROR);
-    }
-
-    _key_length = short_key_len;
-    _new_key_length = new_short_key_len;
-    _key_num = short_key_num;
-    _short_key_columns = short_key_columns;
-
-    return Status::OK();
-}
-
-// Find and return the IndexOffset of the element prior to the first element which
-// is key's lower_bound, or upper_bound if key exists, or return the last element in MemIndex
-// This process is consists of two phases of binary search.
-// Here try to find the first segment which range covers k,
-// and find the index item inside the segment previously found.
-//
-// There 're a little more detail of the binary search.
-// The semantics here is to guarantee there's no
-// omissions for given k, consider the following case:
-// [4, offset] ---------------------> [(4, xxxx), (4, xxxy), (5, xxxx), (5, xxxy)]
-// [5, offset] ---------------------> [(5, yyyy), (5, yyyx), (6, ...)]
-// because of our sparse index, the first item which short key equals 5(5, xxxx) is indexed
-// by shortkey 4 in the first index item, if we want to find the first key not less than 6, we
-// should return the first index instead the second.
-const OLAPIndexOffset MemIndex::find(const RowCursor& k, RowCursor* helper_cursor,
-                                     bool find_last) const {
-    if (begin() == end()) {
-        return begin();
-    }
-
-    OLAPIndexOffset offset;
-    BinarySearchIterator it;
-    BinarySearchIterator seg_beg(0);
-    BinarySearchIterator seg_fin(segment_count());
-
-    try {
-        SegmentComparator seg_comparator(this, helper_cursor);
-
-        // first step, binary search for the correct segment
-        if (!find_last) {
-            it = std::lower_bound(seg_beg, seg_fin, k, seg_comparator);
-        } else {
-            it = std::upper_bound(seg_beg, seg_fin, k, seg_comparator);
-        }
-
-        iterator_offset_t off = 0;
-        if (*it > 0) {
-            off = *it - 1;
-        }
-
-        // set segment id
-        offset.segment = off;
-        IndexComparator index_comparator(this, helper_cursor);
-        // second step, binary search index item in given segment
-        BinarySearchIterator index_beg(0);
-        BinarySearchIterator index_fin(_meta[off].count());
-
-        if (index_comparator.set_segment_id(off) != Status::OK()) {
-            throw "index of range";
-        }
-
-        if (!find_last) {
-            it = std::lower_bound(index_beg, index_fin, k, index_comparator);
-        } else {
-            it = std::upper_bound(index_beg, index_fin, k, index_comparator);
-        }
-
-        offset.offset = *it;
-        VLOG_NOTICE << "show real offset iterator value. off=" << *it;
-        VLOG_NOTICE << "show result offset. seg_off=" << offset.segment
-                    << ", off=" << offset.offset;
-    } catch (...) {
-        LOG(WARNING) << "fail to compare value in memindex. [cursor='" << k.to_string()
-                     << "' find_last=" << find_last << "]";
-        return end();
-    }
-
-    return offset;
-}
-
-const OLAPIndexOffset MemIndex::next(const OLAPIndexOffset& pos) const {
-    OLAPIndexOffset off;
-
-    if (pos.segment >= segment_count()) {
-        return end();
-    } else if (pos.segment == segment_count() - 1) {
-        if (pos.offset >= _meta[pos.segment].count() - 1) {
-            return end();
-        } else {
-            off.segment = pos.segment;
-            off.offset = pos.offset + 1;
-            return off;
-        }
-    } else {
-        if (pos.offset >= _meta[pos.segment].count() - 1) {
-            off.segment = pos.segment + 1;
-            off.offset = 0;
-        } else {
-            off.segment = pos.segment;
-            off.offset = pos.offset + 1;
-        }
-
-        return off;
-    }
-}
-
-const OLAPIndexOffset MemIndex::prev(const OLAPIndexOffset& pos) const {
-    OLAPIndexOffset off;
-
-    if (pos.offset == 0) {
-        if (pos.segment == 0) {
-            off = pos;
-        } else {
-            off.segment = pos.segment - 1;
-            off.offset = _meta[off.segment].count() - 1;
-        }
-    } else {
-        off.segment = pos.segment;
-        off.offset = pos.offset - 1;
-    }
-
-    return off;
-}
-
-const OLAPIndexOffset MemIndex::get_offset(const RowBlockPosition& pos) const {
-    uint32_t file_header_size = _meta[pos.segment].file_header.size();
-    if (pos.segment >= segment_count() ||
-        pos.index_offset > file_header_size + _meta[pos.segment].buffer.length ||
-        (pos.index_offset - file_header_size) % new_entry_length() != 0) {
-        return end();
-    }
-
-    OLAPIndexOffset off;
-    off.segment = pos.segment;
-    off.offset = (pos.index_offset - _meta[pos.segment].file_header.size()) / new_entry_length();
-
-    return off;
-}
-
-Status MemIndex::get_entry(const OLAPIndexOffset& pos, EntrySlice* slice) const {
-    if (pos.segment >= segment_count() || pos.offset >= _meta[pos.segment].count()) {
-        return Status::OLAPInternalError(OLAP_ERR_INDEX_EOF);
-    }
-
-    slice->length = new_entry_length();
-    slice->data = _meta[pos.segment].buffer.data + pos.offset * new_entry_length();
-
-    return Status::OK();
-}
-
-Status MemIndex::get_row_block_position(const OLAPIndexOffset& pos, RowBlockPosition* rbp) const {
-    if (zero_num_rows()) {
-        return Status::OLAPInternalError(OLAP_ERR_INDEX_EOF);
-    }
-
-    if (pos.segment >= segment_count() || pos.offset >= _meta[pos.segment].count()) {
-        LOG(WARNING) << "fail to get RowBlockPosition from OLAPIndexOffset. "
-                        "[IndexOffset={segment="
-                     << pos.segment << " offset=" << pos.offset
-                     << "} segment_count=" << segment_count() << " items_count="
-                     << (pos.segment < segment_count() ? _meta[pos.segment].count() : 0) << "]";
-        return Status::OLAPInternalError(OLAP_ERR_INDEX_EOF);
-    }
-
-    rbp->segment = pos.segment;
-    rbp->data_offset =
-            *reinterpret_cast<uint32_t*>(_meta[pos.segment].buffer.data +
-                                         pos.offset * new_entry_length() + new_short_key_length());
-    rbp->index_offset = _meta[pos.segment].file_header.size() + pos.offset * new_entry_length();
-
-    if (pos.offset == _meta[pos.segment].count() - 1) {
-        rbp->block_size = _meta[pos.segment].file_header.extra().data_length - rbp->data_offset;
-    } else {
-        uint32_t next_offset = *reinterpret_cast<uint32_t*>(_meta[pos.segment].buffer.data +
-                                                            (pos.offset + 1) * new_entry_length() +
-                                                            new_short_key_length());
-        rbp->block_size = next_offset - rbp->data_offset;
-    }
-
-    return Status::OK();
-}
-
-const OLAPIndexOffset MemIndex::get_relative_offset(iterator_offset_t absolute_offset) const {
-    int begin = 0;
-    int end = segment_count() - 1;
-    OLAPIndexOffset offset(0, 0);
-
-    while (begin <= end) {
-        size_t mid = (begin + end) / 2;
-        if (absolute_offset >= _meta[mid].range.last) {
-            begin = mid + 1;
-        } else if (absolute_offset < _meta[mid].range.first) {
-            end = mid - 1;
-        } else {
-            offset.segment = mid;
-            break;
-        }
-    }
-
-    // 这里不考虑没有找到的情况
-    offset.offset = absolute_offset - _meta[offset.segment].range.first;
-    return offset;
-}
-} // namespace doris
diff --git a/be/src/olap/olap_index.h b/be/src/olap/olap_index.h
deleted file mode 100644
index d7a3deaa33..0000000000
--- a/be/src/olap/olap_index.h
+++ /dev/null
@@ -1,380 +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.
-
-#pragma once
-
-#include <condition_variable>
-#include <iterator>
-#include <mutex>
-#include <string>
-#include <unordered_map>
-#include <utility>
-#include <vector>
-
-#include "gen_cpp/column_data_file.pb.h"
-#include "gen_cpp/olap_file.pb.h"
-#include "olap/field.h"
-#include "olap/file_helper.h"
-#include "olap/olap_common.h"
-#include "olap/olap_define.h"
-#include "olap/row.h"
-#include "olap/row_cursor.h"
-#include "olap/utils.h"
-
-namespace doris {
-class IndexComparator;
-class SegmentGroup;
-class RowBlock;
-class RowCursor;
-class SegmentComparator;
-class WrapperField;
-
-typedef uint32_t data_file_offset_t;
-
-struct OLAPIndexFixedHeader {
-    uint32_t data_length;
-    uint64_t num_rows;
-};
-
-struct EntrySlice {
-    char* data;
-    size_t length;
-    EntrySlice() : data(nullptr), length(0) {}
-};
-
-// Range of offset in one segment
-struct IDRange {
-    uint32_t first;
-    uint32_t last;
-};
-
-// index offset
-// 2-Dimension Offset, the first is segment id, the second is offset inside segment
-struct OLAPIndexOffset {
-    OLAPIndexOffset() : segment(0), offset(0) {}
-    OLAPIndexOffset(const iterator_offset_t& seg, const iterator_offset_t& off)
-            : segment(seg), offset(off) {}
-    OLAPIndexOffset(const OLAPIndexOffset& off) : segment(off.segment), offset(off.offset) {}
-
-    bool operator==(const OLAPIndexOffset& other) const {
-        return segment == other.segment && offset == other.offset;
-    }
-
-    iterator_offset_t segment;
-    iterator_offset_t offset;
-};
-
-// 唯一标识一个RowBlock在Data文件和Index文件中的位置
-struct RowBlockPosition {
-    RowBlockPosition() : segment(0), block_size(0), data_offset(0), index_offset(0) {}
-
-    bool operator==(const RowBlockPosition& other) const {
-        return (segment == other.segment && data_offset == other.data_offset &&
-                block_size == other.block_size && index_offset == other.index_offset);
-    }
-
-    bool operator!=(const RowBlockPosition& other) const { return !(*this == other); }
-
-    bool operator>(const RowBlockPosition& other) const {
-        if (segment < other.segment) {
-            return false;
-        } else if (segment > other.segment) {
-            return true;
-        } else {
-            if (data_offset > other.data_offset) {
-                return true;
-            } else {
-                return false;
-            }
-        }
-    }
-
-    bool operator>=(const RowBlockPosition& other) const {
-        if (segment < other.segment) {
-            return false;
-        } else if (segment > other.segment) {
-            return true;
-        } else {
-            if (data_offset >= other.data_offset) {
-                return true;
-            } else {
-                return false;
-            }
-        }
-    }
-
-    // 供日志输出
-    std::string to_string() const {
-        char message[1024] = {'\0'};
-        snprintf(message, sizeof(message),
-                 "{segment=%u block_size=%u data_offset=%u index_offset=%u}", segment, block_size,
-                 data_offset, index_offset);
-        return std::string(message);
-    }
-
-    uint32_t segment;
-    uint32_t block_size;
-    uint32_t data_offset;  // offset in data file
-    uint32_t index_offset; // offset in index file
-};
-
-// In memory presentation of index meta information
-struct SegmentMetaInfo {
-    SegmentMetaInfo() {
-        range.first = range.last = 0;
-        buffer.length = 0;
-        buffer.data = nullptr;
-    }
-
-    const size_t count() const { return range.last - range.first; }
-
-    IDRange range;
-    EntrySlice buffer;
-    FileHeader<OLAPIndexHeaderMessage, OLAPIndexFixedHeader> file_header;
-};
-
-// In memory index structure, all index hold here
-class MemIndex {
-public:
-    friend class SegmentGroup;
-    friend class IndexComparator;
-    friend class SegmentComparator;
-
-    MemIndex();
-    ~MemIndex();
-
-    // 初始化MemIndex, 传入short_key的总长度和对应的Field数组
-    Status init(size_t short_key_len, size_t new_short_key_len, size_t short_key_num,
-                std::vector<TabletColumn>* short_key_columns);
-
-    // 加载一个segment到内存
-    Status load_segment(const char* file, size_t* current_num_rows_per_row_block,
-                        bool use_cache = true);
-
-    // Return the IndexOffset of the first element, physically, it's (0, 0)
-    const OLAPIndexOffset begin() const {
-        OLAPIndexOffset off;
-        off.segment = off.offset = 0;
-        return off;
-    }
-
-    // Indicates a logical IndexOffset beyond MemIndex
-    const OLAPIndexOffset end() const {
-        OLAPIndexOffset off;
-        off.segment = segment_count();
-        off.offset = _num_entries;
-        return off;
-    }
-
-    // Return the IndexOffset of position prior to the first element which is either
-    // not less than(find_last is false) or greater than (find_last is true) 'key';
-    // or, return the IndexOffset of last element inside MemIndex
-    // in case of nothing matched with 'key'
-    const OLAPIndexOffset find(const RowCursor& key, RowCursor* helper_cursor,
-                               bool find_last) const;
-
-    // Same with begin()
-    const OLAPIndexOffset find_first() const { return begin(); }
-
-    // Return IndexOffset of last element if exists, or return (0, 0) if MemIndex is empty()
-    const OLAPIndexOffset find_last() const {
-        if (_num_entries == 0 || segment_count() == 0) {
-            return end();
-        } else {
-            OLAPIndexOffset off;
-            off.segment = segment_count() - 1;
-            off.offset = _meta[segment_count() - 1].count() - 1;
-            return off;
-        }
-    }
-
-    // Return IndexOffset of next element
-    const OLAPIndexOffset next(const OLAPIndexOffset& pos) const;
-    // Return IndexOffset of prev element
-    const OLAPIndexOffset prev(const OLAPIndexOffset& pos) const;
-    // Calculate IndexOffset from RowBlockPosition
-    const OLAPIndexOffset get_offset(const RowBlockPosition& pos) const;
-
-    // Return the 1-dimension, plain offset from IndexOffset
-    // For example, there're 1000 index items, divided into 10 segments, 100 items in each segment,
-    // the 2-dimension offset of the first element of first segment is (0, 0),
-    // it's plain offset is 0,
-    // the 2-dimension offset of the first element of second segment is (1, 0),
-    // it's plain offset is 100
-    const iterator_offset_t get_absolute_offset(const OLAPIndexOffset& offset) const {
-        if (offset.segment >= segment_count() || offset.offset >= _meta[offset.segment].count()) {
-            return _num_entries;
-        } else {
-            return _meta[offset.segment].range.first + offset.offset;
-        }
-    }
-
-    // Return the 2-dimension, logical Offset from plain offset
-    const OLAPIndexOffset get_relative_offset(iterator_offset_t absolute_offset) const;
-
-    // Return content of index item, which IndexOffset is pos
-    Status get_entry(const OLAPIndexOffset& pos, EntrySlice* slice) const;
-
-    // Return RowBlockPosition from IndexOffset
-    Status get_row_block_position(const OLAPIndexOffset& pos, RowBlockPosition* rbp) const;
-
-    const size_t short_key_num() const { return _key_num; }
-
-    // Return length of short keys in bytes, for example, there're two short key columns:
-    // uint32_t/uint64_t the length is sizeof(uint32_t) + sizeof(uint64_t)
-    const size_t short_key_length() const { return _key_length; }
-
-    const size_t new_short_key_length() const { return _new_key_length; }
-
-    // Return length of full index item,
-    // which actually equals to short_key_length() plus sizeof(data_file_offset_t)
-    const size_t entry_length() const { return short_key_length() + sizeof(data_file_offset_t); }
-
-    const size_t new_entry_length() const { return _new_key_length + sizeof(data_file_offset_t); }
-
-    // Return short key FieldInfo array
-    const std::vector<TabletColumn>& short_key_columns() const { return *_short_key_columns; }
-
-    // Return the number of indices in MemIndex
-    size_t count() const { return _num_entries; }
-
-    // Return the number of segments in MemIndex
-    size_t segment_count() const { return _meta.size(); }
-
-    bool zero_num_rows() const { return _num_entries == 0; }
-
-    const size_t index_size() const { return _index_size; };
-
-    const size_t data_size() const { return _data_size; };
-
-    const size_t num_rows() const { return _num_rows; }
-
-    bool delete_flag() const {
-        if (_meta[0].file_header.message().has_delete_flag()) {
-            return _meta[0].file_header.message().delete_flag();
-        } else {
-            return false;
-        }
-    }
-
-    bool get_null_supported(uint32_t seg_id) {
-        if (false == _meta[seg_id].file_header.message().has_null_supported()) {
-            return false;
-        } else {
-            return _meta[seg_id].file_header.message().null_supported();
-        }
-    }
-
-private:
-    std::vector<SegmentMetaInfo> _meta;
-    size_t _key_length;
-    size_t _new_key_length;
-    size_t _key_num;
-    size_t _num_entries;
-    size_t _index_size;
-    size_t _data_size;
-    size_t _num_rows;
-    std::vector<TabletColumn>* _short_key_columns;
-
-    std::unique_ptr<MemPool> _mem_pool;
-    DISALLOW_COPY_AND_ASSIGN(MemIndex);
-};
-
-// 在同一个Segment内进行二分查找的比较类
-class IndexComparator {
-public:
-    IndexComparator(const MemIndex* index, RowCursor* cursor)
-            : _index(index), _cur_seg(0), _helper_cursor(cursor) {}
-
-    // Destructor do nothing
-    ~IndexComparator() {}
-
-    bool operator()(const iterator_offset_t& index, const RowCursor& key) {
-        return _compare(index, key, COMPARATOR_LESS);
-    }
-
-    bool operator()(const RowCursor& key, const iterator_offset_t& index) {
-        return _compare(index, key, COMPARATOR_LARGER);
-    }
-
-    Status set_segment_id(const iterator_offset_t& segment_id) {
-        if (segment_id >= _index->segment_count()) {
-            return Status::OLAPInternalError(OLAP_ERR_INDEX_EOF);
-        }
-
-        _cur_seg = segment_id;
-        return Status::OK();
-    }
-
-private:
-    bool _compare(const iterator_offset_t& index, const RowCursor& key, ComparatorEnum comparator) {
-        EntrySlice slice;
-        OLAPIndexOffset offset(_cur_seg, index);
-        _index->get_entry(offset, &slice);
-
-        _helper_cursor->attach(slice.data);
-
-        if (comparator == COMPARATOR_LESS) {
-            return index_compare_row(*_helper_cursor, key) < 0;
-        } else {
-            return index_compare_row(*_helper_cursor, key) > 0;
-        }
-    }
-
-    const MemIndex* _index;
-    iterator_offset_t _cur_seg;
-    RowCursor* _helper_cursor;
-};
-
-// 用于寻找索引所在的Segment的比较类
-class SegmentComparator {
-public:
-    SegmentComparator(const MemIndex* index, RowCursor* cursor)
-            : _index(index), _helper_cursor(cursor) {}
-
-    // Destructor do nothing
-    ~SegmentComparator() {}
-
-    bool operator()(const iterator_offset_t index, const RowCursor& key) {
-        return _compare(index, key, COMPARATOR_LESS);
-    }
-
-    bool operator()(const RowCursor& key, iterator_offset_t index) {
-        return _compare(index, key, COMPARATOR_LARGER);
-    }
-
-private:
-    bool _compare(const iterator_offset_t& index, const RowCursor& key, ComparatorEnum comparator) {
-        EntrySlice slice;
-        slice.data = _index->_meta[index].buffer.data;
-        //slice.length = _index->short_key_length();
-        slice.length = _index->new_short_key_length();
-
-        _helper_cursor->attach(slice.data);
-
-        if (comparator == COMPARATOR_LESS) {
-            return index_compare_row(*_helper_cursor, key) < 0;
-        } else {
-            return index_compare_row(*_helper_cursor, key) > 0;
-        }
-    }
-
-    const MemIndex* _index;
-    RowCursor* _helper_cursor;
-};
-
-} // namespace doris
diff --git a/be/src/olap/olap_server.cpp b/be/src/olap/olap_server.cpp
index e08ef5ca5b..2b893cac75 100644
--- a/be/src/olap/olap_server.cpp
+++ b/be/src/olap/olap_server.cpp
@@ -28,7 +28,6 @@
 #include "agent/cgroups_mgr.h"
 #include "common/status.h"
 #include "gutil/strings/substitute.h"
-#include "olap/convert_rowset.h"
 #include "olap/cumulative_compaction.h"
 #include "olap/olap_common.h"
 #include "olap/olap_define.h"
@@ -78,21 +77,6 @@ Status StorageEngine::start_bg_threads() {
             .set_max_threads(max_thread_num)
             .build(&_cumu_compaction_thread_pool);
 
-    int32_t convert_rowset_thread_num = config::convert_rowset_thread_num;
-    if (convert_rowset_thread_num > 0) {
-        ThreadPoolBuilder("ConvertRowsetTaskThreadPool")
-                .set_min_threads(convert_rowset_thread_num)
-                .set_max_threads(convert_rowset_thread_num)
-                .build(&_convert_rowset_thread_pool);
-
-        // alpha rowset scan thread
-        RETURN_IF_ERROR(Thread::create(
-                "StorageEngine", "alpha_rowset_scan_thread",
-                [this]() { this->_alpha_rowset_scan_thread_callback(); },
-                &_alpha_rowset_scan_thread));
-        LOG(INFO) << "alpha rowset scan thread started";
-    }
-
     ThreadPoolBuilder("BaseCompactionTaskThreadPool")
             .set_min_threads(config::max_base_compaction_threads)
             .set_max_threads(config::max_base_compaction_threads)
@@ -338,40 +322,6 @@ void StorageEngine::_tablet_checkpoint_callback(const std::vector<DataDir*>& dat
     } while (!_stop_background_threads_latch.wait_for(std::chrono::seconds(interval)));
 }
 
-void StorageEngine::_alpha_rowset_scan_thread_callback() {
-    LOG(INFO) << "try to start alpha rowset scan thread!";
-
-    auto scan_interval_sec = config::scan_alpha_rowset_min_interval_sec;
-    auto max_convert_task = config::convert_rowset_thread_num * 2;
-    do {
-        std::vector<TabletSharedPtr> tablet_have_alpha_rowset;
-        _tablet_manager->find_tablet_have_alpha_rowset(tablet_have_alpha_rowset);
-
-        std::random_device rd;
-        std::mt19937 g(rd());
-        std::shuffle(tablet_have_alpha_rowset.begin(), tablet_have_alpha_rowset.end(), g);
-
-        for (int i = 0; i < max_convert_task && i < tablet_have_alpha_rowset.size(); ++i) {
-            auto tablet = tablet_have_alpha_rowset[i];
-            auto st = _convert_rowset_thread_pool->submit_func([=]() {
-                CgroupsMgr::apply_system_cgroup();
-                auto convert_rowset = std::make_shared<ConvertRowset>(tablet);
-                convert_rowset->do_convert();
-            });
-            if (!st.ok()) {
-                LOG(WARNING) << "submit convert tablet tasks failed.";
-            }
-        }
-
-        if (tablet_have_alpha_rowset.size() == 0) {
-            scan_interval_sec = std::min(3600, scan_interval_sec * 2);
-        } else {
-            _convert_rowset_thread_pool->wait();
-            scan_interval_sec = config::scan_alpha_rowset_min_interval_sec;
-        }
-    } while (!_stop_background_threads_latch.wait_for(std::chrono::seconds(scan_interval_sec)));
-}
-
 void StorageEngine::_adjust_compaction_thread_num() {
     if (_base_compaction_thread_pool->max_threads() != config::max_base_compaction_threads) {
         int old_max_threads = _base_compaction_thread_pool->max_threads();
diff --git a/be/src/olap/out_stream.cpp b/be/src/olap/out_stream.cpp
deleted file mode 100644
index 3cca7c3803..0000000000
--- a/be/src/olap/out_stream.cpp
+++ /dev/null
@@ -1,401 +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 "olap/out_stream.h"
-
-#include "olap/byte_buffer.h"
-#include "olap/file_helper.h"
-#include "olap/utils.h"
-#include "util/mem_util.hpp"
-
-namespace doris {
-
-OutStreamFactory::OutStreamFactory(CompressKind compress_kind, uint32_t stream_buffer_size)
-        : _stream_buffer_size(stream_buffer_size) {
-    switch (compress_kind) {
-    case COMPRESS_NONE:
-        _compressor = nullptr;
-        break;
-
-#ifdef DORIS_WITH_LZO
-    case COMPRESS_LZO:
-        _compressor = lzo_compress;
-        break;
-#endif
-
-    case COMPRESS_LZ4:
-        _compressor = lz4_compress;
-        break;
-
-    default:
-        LOG(FATAL) << "unknown compress kind. kind=" << compress_kind;
-    }
-}
-
-OutStreamFactory::~OutStreamFactory() {
-    for (std::map<StreamName, OutStream*>::iterator it = _streams.begin(); it != _streams.end();
-         ++it) {
-        SAFE_DELETE(it->second);
-    }
-}
-
-OutStream* OutStreamFactory::create_stream(uint32_t column_unique_id,
-                                           StreamInfoMessage::Kind kind) {
-    OutStream* stream = nullptr;
-
-    if (StreamInfoMessage::ROW_INDEX == kind || StreamInfoMessage::BLOOM_FILTER == kind) {
-        stream = new (std::nothrow) OutStream(_stream_buffer_size, nullptr);
-    } else {
-        stream = new (std::nothrow) OutStream(_stream_buffer_size, _compressor);
-    }
-
-    if (nullptr == stream) {
-        LOG(WARNING) << "fail to allocate OutStream.";
-        return nullptr;
-    }
-
-    StreamName stream_name(column_unique_id, kind);
-    _streams[stream_name] = stream;
-    return stream;
-}
-
-OutStream::OutStream(uint32_t buffer_size, Compressor compressor)
-        : _buffer_size(buffer_size),
-          _compressor(compressor),
-          _is_suppressed(false),
-          _current(nullptr),
-          _compressed(nullptr),
-          _overflow(nullptr),
-          _spilled_bytes(0) {}
-
-OutStream::~OutStream() {
-    SAFE_DELETE(_current);
-    SAFE_DELETE(_compressed);
-    SAFE_DELETE(_overflow);
-
-    for (std::vector<StorageByteBuffer*>::iterator it = _output_buffers.begin();
-         it != _output_buffers.end(); ++it) {
-        SAFE_DELETE(*it);
-    }
-}
-
-Status OutStream::_create_new_input_buffer() {
-    SAFE_DELETE(_current);
-    _current = StorageByteBuffer::create(_buffer_size + sizeof(StreamHead));
-
-    if (nullptr != _current) {
-        _current->set_position(sizeof(StreamHead));
-        return Status::OK();
-    } else {
-        return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
-    }
-}
-
-Status OutStream::_write_head(StorageByteBuffer* buf, uint64_t position,
-                              StreamHead::StreamType type, uint32_t length) {
-    if (buf->limit() < sizeof(StreamHead) + length) {
-        return Status::OLAPInternalError(OLAP_ERR_BUFFER_OVERFLOW);
-    }
-
-    StreamHead* head = reinterpret_cast<StreamHead*>(&(buf->array()[position]));
-    head->type = type;
-    head->length = length;
-    head->checksum = 0;
-    return Status::OK();
-}
-
-Status OutStream::_compress(StorageByteBuffer* input, StorageByteBuffer* output,
-                            StorageByteBuffer* overflow, bool* smaller) {
-    Status res = Status::OK();
-
-    res = _compressor(input, overflow, smaller);
-
-    if (res.ok() && *smaller) {
-        if (output->remaining() >= overflow->position()) {
-            memory_copy(&(output->array()[output->position()]), overflow->array(),
-                        overflow->position());
-            output->set_position(output->position() + overflow->position());
-            overflow->set_position(0);
-        } else if (0 != output->remaining()) {
-            uint64_t to_copy = output->remaining();
-            memory_copy(&(output->array()[output->position()]), overflow->array(), to_copy);
-            output->set_position(output->limit());
-
-            memmove(overflow->array(), &(overflow->array()[to_copy]),
-                    overflow->position() - to_copy);
-            overflow->set_position(overflow->position() - to_copy);
-        }
-    }
-
-    return Status::OK();
-}
-
-void OutStream::_output_uncompress() {
-    _spilled_bytes += _current->limit();
-    _write_head(_current, 0, StreamHead::UNCOMPRESSED, _current->limit() - sizeof(StreamHead));
-    _output_buffers.push_back(_current);
-    _current = nullptr;
-}
-
-void OutStream::_output_compressed() {
-    _compressed->flip();
-    _output_buffers.push_back(_compressed);
-    _compressed = _overflow;
-    _overflow = nullptr;
-}
-
-Status OutStream::_make_sure_output_buffer() {
-    if (nullptr == _compressed) {
-        _compressed = StorageByteBuffer::create(_buffer_size + sizeof(StreamHead));
-
-        if (nullptr == _compressed) {
-            return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
-        }
-    }
-
-    if (nullptr == _overflow) {
-        _overflow = StorageByteBuffer::create(_buffer_size + sizeof(StreamHead));
-
-        if (nullptr == _overflow) {
-            return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
-        }
-    }
-
-    return Status::OK();
-}
-
-Status OutStream::_spill() {
-    Status res = Status::OK();
-
-    if (_current == nullptr || _current->position() == sizeof(StreamHead)) {
-        return Status::OK();
-    }
-
-    // If it is not compressed, read current directly. Note that current will be cleared and set to NULL after output
-    if (_compressor == nullptr) {
-        _current->flip();
-        _output_uncompress();
-    } else {
-        //If compression is required,
-        // The current moves to the position behind the head, leaving space for the head
-        _current->set_limit(_current->position());
-        _current->set_position(sizeof(StreamHead));
-
-        //Allocate compress and overflow, the two buffer sizes are actually the same
-        if (!(res = _make_sure_output_buffer())) {
-            return res;
-        }
-
-        // Decompress current to compress and overflow
-        uint64_t head_pos = _compressed->position();
-        _compressed->set_position(head_pos + sizeof(StreamHead));
-        bool smaller = false;
-        res = _compress(_current, _compressed, _overflow, &smaller);
-
-        if (!res.ok()) {
-            LOG(WARNING) << "fail to compress data.";
-            return Status::OLAPInternalError(OLAP_ERR_COMPRESS_ERROR);
-        }
-
-        if (smaller) {
-            // Data are compressed into _output and _overflow, reset _current
-            // Note that in this case, current is not released, because the compress actually output
-            _current->set_position(sizeof(StreamHead));
-            _current->set_limit(_current->capacity());
-
-            uint32_t output_bytes = _compressed->position() - head_pos - sizeof(StreamHead);
-            output_bytes += _overflow->position();
-            _write_head(_compressed, head_pos, StreamHead::COMPRESSED, output_bytes);
-
-            if (_compressed->remaining() < sizeof(StreamHead)) {
-                _output_compressed();
-            }
-
-            _spilled_bytes += sizeof(StreamHead) + output_bytes;
-        } else {
-            // directly output _current
-            // If there is _compress before, output m_compress first
-            // Note that there must be no _overflow at this time
-            _compressed->set_position(head_pos);
-
-            if (head_pos != 0) {
-                // There was data in _compressed before, in this case, output compressed first,
-                // At this time _overflow must be empty
-                _output_compressed();
-            }
-
-            _output_uncompress();
-        }
-    }
-
-    return Status::OK();
-}
-
-Status OutStream::write(const char* buffer, uint64_t length) {
-    Status res = Status::OK();
-    uint64_t offset = 0;
-    uint64_t remain = length;
-
-    while (remain > 0) {
-        // The reason why it was thrown in is because in the case of compression, _current will only be created once
-        // It has been multiplexing since then, and the output is compress
-        // In the case of uncompressed, current will be put into the list and cannot be reused. The reason is
-        // If it is reused, the previous content will be modified, so it needs to be redistributed.
-        // Only allocate once and the second block will hang up
-        if (nullptr == _current) {
-            res = _create_new_input_buffer();
-            if (!res.ok()) {
-                return res;
-            }
-        }
-
-        uint64_t to_put = std::min(_current->remaining(), remain);
-
-        if (OLAP_LIKELY(0 != to_put)) {
-            res = _current->put(&buffer[offset], to_put);
-            if (!res.ok()) {
-                LOG(WARNING) << "fail to put buffer.";
-                return res;
-            }
-
-            offset += to_put;
-            remain -= to_put;
-        }
-
-        if (_current->remaining() == 0) {
-            res = _spill();
-            if (!res.ok()) {
-                LOG(WARNING) << "fail to spill current buffer.";
-                return res;
-            }
-        }
-    }
-
-    return Status::OK();
-}
-
-void OutStream::get_position(PositionEntryWriter* index_entry) const {
-    index_entry->add_position(_spilled_bytes);
-
-    if (nullptr != _current) {
-        index_entry->add_position(_current->position() - sizeof(StreamHead));
-    } else {
-        index_entry->add_position(0);
-    }
-}
-
-uint64_t OutStream::get_stream_length() const {
-    uint64_t result = 0;
-
-    for (std::vector<StorageByteBuffer*>::const_iterator it = _output_buffers.begin();
-         it != _output_buffers.end(); ++it) {
-        result += (*it)->limit();
-    }
-
-    return result;
-}
-
-uint64_t OutStream::get_total_buffer_size() const {
-    uint64_t result = 0;
-
-    for (std::vector<StorageByteBuffer*>::const_iterator it = _output_buffers.begin();
-         it != _output_buffers.end(); ++it) {
-        result += (*it)->capacity();
-    }
-
-    if (_current) {
-        result += _current->capacity();
-    }
-
-    if (_compressed) {
-        result += _compressed->capacity();
-    }
-
-    if (_overflow) {
-        result += _overflow->capacity();
-    }
-
-    return result;
-}
-
-Status OutStream::write_to_file(FileHandler* file_handle, uint32_t write_mbytes_per_sec) const {
-    Status res = Status::OK();
-
-    uint64_t total_stream_len = 0;
-    OlapStopWatch speed_limit_watch;
-
-    speed_limit_watch.reset();
-
-    for (std::vector<StorageByteBuffer*>::const_iterator it = _output_buffers.begin();
-         it != _output_buffers.end(); ++it) {
-        VLOG_TRACE << "write stream begin:" << file_handle->tell();
-
-        res = file_handle->write((*it)->array(), (*it)->limit());
-        if (!res.ok()) {
-            LOG(WARNING) << "fail to write stream to fail.";
-            return res;
-        }
-
-        VLOG_TRACE << "write stream end:" << file_handle->tell();
-
-        total_stream_len += (*it)->limit();
-        if (write_mbytes_per_sec > 0) {
-            uint64_t delta_time_us = speed_limit_watch.get_elapse_time_us();
-            int64_t sleep_time = total_stream_len / write_mbytes_per_sec - delta_time_us;
-            if (sleep_time > 0) {
-                VLOG_TRACE << "sleep to limit merge speed. time=" << sleep_time
-                           << ", bytes=" << total_stream_len;
-                std::this_thread::sleep_for(std::chrono::microseconds(sleep_time));
-            }
-        }
-    }
-
-    return res;
-}
-
-Status OutStream::flush() {
-    Status res = Status::OK();
-
-    res = _spill();
-    if (!res.ok()) {
-        LOG(WARNING) << "fail to spill stream.";
-        return res;
-    }
-
-    if (nullptr != _compressed && 0 != _compressed->position()) {
-        _output_compressed();
-        SAFE_DELETE(_compressed);
-    }
-
-    SAFE_DELETE(_current);
-    SAFE_DELETE(_overflow);
-
-    return res;
-}
-
-uint32_t OutStream::crc32(uint32_t checksum) const {
-    uint32_t result = CRC32_INIT;
-
-    for (std::vector<StorageByteBuffer*>::const_iterator it = _output_buffers.begin();
-         it != _output_buffers.end(); ++it) {
-        result = olap_crc32(result, (*it)->array(), (*it)->limit());
-    }
-
-    return result;
-}
-
-} // namespace doris
diff --git a/be/src/olap/out_stream.h b/be/src/olap/out_stream.h
deleted file mode 100644
index 50f1914649..0000000000
--- a/be/src/olap/out_stream.h
+++ /dev/null
@@ -1,169 +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.
-
-#pragma once
-
-#include "olap/byte_buffer.h"
-#include "olap/compress.h"
-#include "olap/olap_define.h"
-#include "olap/stream_index_writer.h"
-#include "olap/stream_name.h"
-
-namespace doris {
-class FileHandler;
-
-// Unlike OrcFile, we cannot guarantee the reliability of stored data without HDFS at the bottom, so we must write
-// Check value, check this check value when reading data
-// Adopt TLV type header, which has sufficient scalability
-struct StreamHead {
-    enum StreamType { UNCOMPRESSED = 0, COMPRESSED = 1 };
-    uint8_t type;         // 256 types, should be enough for future expansion
-    uint32_t length : 24; // 24-bit length
-    uint32_t checksum;    // 32-bit check value
-    StreamHead() : type(COMPRESSED), length(0), checksum(0) {}
-} __attribute__((packed));
-
-// Output stream, use a set of ByteBuffer to buffer all data
-class OutStream {
-public:
-    // The output stream supports two modes: compressed or uncompressed. If compression is enabled, the compression function is given
-    explicit OutStream(uint32_t buffer_size, Compressor compressor);
-
-    ~OutStream();
-
-    // Output a byte to the stream
-    Status write(char byte) {
-        Status res = Status::OK();
-        if (_current == nullptr) {
-            res = _create_new_input_buffer();
-            if (!res.ok()) {
-                return res;
-            }
-        }
-        if (_current->remaining() < 1) {
-            res = _spill();
-            if (!res.ok()) {
-                LOG(WARNING) << "fail to spill current buffer.";
-                return res;
-            }
-            if (_current == nullptr) {
-                res = _create_new_input_buffer();
-                if (!res.ok()) {
-                    return res;
-                }
-            }
-        }
-        return _current->put(byte);
-    }
-
-    // Output a piece of data to the stream
-    Status write(const char* buffer, uint64_t length);
-
-    // Record the current position of the stream in the index entry
-    void get_position(PositionEntryWriter* index_entry) const;
-
-    // Returns the size of all data in the stream
-    uint64_t get_stream_length() const;
-
-    // Returns the size of the buffer that has been allocated
-    uint64_t get_total_buffer_size() const;
-
-    // Output the cached data stream to a file
-    Status write_to_file(FileHandler* file_handle, uint32_t write_mbytes_per_sec) const;
-
-    bool is_suppressed() const { return _is_suppressed; }
-    void suppress() { _is_suppressed = true; }
-    // Output data to output_buffers
-    Status flush();
-    // Calculate the crc32 value of the output data
-    uint32_t crc32(uint32_t checksum) const;
-    const std::vector<StorageByteBuffer*>& output_buffers() { return _output_buffers; }
-
-    void print_position_debug_info() {
-        VLOG_TRACE << "compress: " << _spilled_bytes;
-
-        if (_current != nullptr) {
-            VLOG_TRACE << "uncompress=" << (_current->position() - sizeof(StreamHead));
-        } else {
-            VLOG_TRACE << "uncompress 0";
-        }
-    }
-
-private:
-    Status _create_new_input_buffer();
-    Status _write_head(StorageByteBuffer* buf, uint64_t position, StreamHead::StreamType type,
-                       uint32_t length);
-    Status _spill();
-    Status _compress(StorageByteBuffer* input, StorageByteBuffer* output,
-                     StorageByteBuffer* overflow, bool* smaller);
-    void _output_uncompress();
-    void _output_compressed();
-    Status _make_sure_output_buffer();
-
-    uint32_t _buffer_size;  // Compressed block size
-    Compressor _compressor; // Compression function, if NULL means no compression
-    std::vector<StorageByteBuffer*> _output_buffers; // Buffer all output
-    bool _is_suppressed;                             // Whether the stream is terminated
-    StorageByteBuffer* _current;                     // Cache uncompressed data
-    StorageByteBuffer* _compressed;                  // Bytes to be output to output_buffers
-    StorageByteBuffer* _overflow;                    // Bytes that can't fit in _output
-    uint64_t _spilled_bytes; // The number of bytes that have been output to output
-
-    DISALLOW_COPY_AND_ASSIGN(OutStream);
-};
-
-// Define the factory method of the output stream
-// Host all output streams, and encapsulate information such as compression algorithm, whether to enable Index, block size, etc.
-class OutStreamFactory {
-public:
-    explicit OutStreamFactory(CompressKind compress_kind, uint32_t stream_buffer_size);
-
-    ~OutStreamFactory();
-
-    //The lifetime of the stream after creation is still managed by OutStreamFactory
-    OutStream* create_stream(uint32_t column_unique_id, StreamInfoMessage::Kind kind);
-
-    const std::map<StreamName, OutStream*>& streams() const { return _streams; }
-
-private:
-    std::map<StreamName, OutStream*> _streams; // All created streams
-    Compressor _compressor;
-    uint32_t _stream_buffer_size;
-
-    DISALLOW_COPY_AND_ASSIGN(OutStreamFactory);
-};
-
-/*
-class OutStreamBufferWrapper : public std::streambuf {
-public:
-    OutStreamBufferWrapper(OutStream* output)
-        : std::streambuf(),
-        _stream(output),
-        _skip_size(0) {
-
-    }
-    virtual ~OutStreamBufferWrapper() {}
-    virtual int_type overflow(typename traits::int_type c = traits::eof()) {
-        return c;
-    }
-protected:
-    OutStream* _stream;
-    size_t _skip_size;
-};
-*/
-
-} // namespace doris
diff --git a/be/src/olap/push_handler.cpp b/be/src/olap/push_handler.cpp
index 4ab7ecfddd..5b3d1b2cbc 100644
--- a/be/src/olap/push_handler.cpp
+++ b/be/src/olap/push_handler.cpp
@@ -162,7 +162,6 @@ Status PushHandler::_do_streaming_ingestion(TabletSharedPtr tablet, const TPushR
                     request.partition_id, tablet_var.tablet, request.transaction_id);
             // has to check rollback status to ensure not delete a committed rowset
             if (rollback_status.ok()) {
-                // actually, olap_index may has been deleted in delete_transaction()
                 StorageEngine::instance()->add_unused_rowset(tablet_var.rowset_to_add);
             }
         }
diff --git a/be/src/olap/reader.h b/be/src/olap/reader.h
index 2593a51bae..c0fb42ebf8 100644
--- a/be/src/olap/reader.h
+++ b/be/src/olap/reader.h
@@ -77,7 +77,6 @@ public:
         std::vector<TCondition> conditions;
         std::vector<std::pair<string, std::shared_ptr<IBloomFilterFuncBase>>> bloom_filters;
 
-        // The ColumnData will be set when using Merger, eg Cumulative, BE.
         std::vector<RowsetReaderSharedPtr> rs_readers;
         std::vector<uint32_t> return_columns;
         RuntimeProfile* profile = nullptr;
diff --git a/be/src/olap/row_block.h b/be/src/olap/row_block.h
index f40f2fd93b..ac157991eb 100644
--- a/be/src/olap/row_block.h
+++ b/be/src/olap/row_block.h
@@ -26,7 +26,6 @@
 #include "olap/olap_define.h"
 #include "olap/row_cursor.h"
 #include "olap/utils.h"
-#include "runtime/vectorized_row_batch.h"
 
 namespace doris {
 
@@ -53,7 +52,6 @@ class RowBlock {
     // Please keep these classes as 'friend'.  They have to use lots of private fields for
     // faster operation.
     friend class RowBlockChanger;
-    friend class VectorizedRowBatch;
 
 public:
     RowBlock(const TabletSchema* schema);
@@ -128,7 +126,6 @@ private:
     // Field offset of memory row format, used to get field ptr in memory row
     std::vector<size_t> _field_offset_in_memory;
 
-    // only used for SegmentReader to covert VectorizedRowBatch to RowBlock
     // Be careful to use this
     size_t _pos = 0;
     size_t _limit = 0;
diff --git a/be/src/olap/row_block2.h b/be/src/olap/row_block2.h
index fda9a49d6f..22a5745ddf 100644
--- a/be/src/olap/row_block2.h
+++ b/be/src/olap/row_block2.h
@@ -38,7 +38,6 @@ class RowBlockRow;
 class RowCursor;
 
 // This struct contains a block of rows, in which each column's data is stored
-// in a vector. We don't use VectorizedRowBatch because it doesn't own the data
 // in block, however it is used by old code, which we don't want to change.
 class RowBlockV2 {
 public:
diff --git a/be/src/olap/rowset/CMakeLists.txt b/be/src/olap/rowset/CMakeLists.txt
index 2f4720e5ac..131978a9c4 100644
--- a/be/src/olap/rowset/CMakeLists.txt
+++ b/be/src/olap/rowset/CMakeLists.txt
@@ -22,26 +22,10 @@ set(LIBRARY_OUTPUT_PATH "${BUILD_DIR}/src/olap/rowset")
 set(EXECUTABLE_OUTPUT_PATH "${BUILD_DIR}/src/olap/rowset")
   
 add_library(Rowset STATIC
-    bit_field_reader.cpp
-    bit_field_writer.cpp
-    column_data.cpp
-    column_reader.cpp
-    column_writer.cpp
-    column_data_writer.cpp
-    segment_group.cpp
-    run_length_byte_reader.cpp
-    run_length_byte_writer.cpp
-    run_length_integer_reader.cpp
-    run_length_integer_writer.cpp
     unique_rowset_id_generator.cpp
-    segment_reader.cpp
-    segment_writer.cpp
     rowset.cpp
     rowset_factory.cpp
     rowset_meta_manager.cpp
-    alpha_rowset.cpp
-    alpha_rowset_reader.cpp
-    alpha_rowset_writer.cpp
     alpha_rowset_meta.cpp
     beta_rowset.cpp
     beta_rowset_reader.cpp
diff --git a/be/src/olap/rowset/alpha_rowset.cpp b/be/src/olap/rowset/alpha_rowset.cpp
deleted file mode 100644
index 7f2e74c744..0000000000
--- a/be/src/olap/rowset/alpha_rowset.cpp
+++ /dev/null
@@ -1,407 +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 "olap/rowset/alpha_rowset.h"
-
-#include <util/file_utils.h>
-
-#include "olap/row.h"
-#include "olap/rowset/alpha_rowset_meta.h"
-#include "olap/rowset/alpha_rowset_reader.h"
-#include "olap/rowset/rowset_meta_manager.h"
-#include "util/hash_util.hpp"
-
-namespace doris {
-
-AlphaRowset::AlphaRowset(const TabletSchema* schema, const FilePathDesc& rowset_path_desc,
-                         RowsetMetaSharedPtr rowset_meta)
-        : Rowset(schema, rowset_path_desc, std::move(rowset_meta)) {}
-
-Status AlphaRowset::do_load(bool use_cache) {
-    for (auto& segment_group : _segment_groups) {
-        // validate segment group
-        if (segment_group->validate() != Status::OK()) {
-            LOG(WARNING) << "fail to validate segment_group. [version=" << start_version() << "-"
-                         << end_version();
-            // if load segment group failed, rowset init failed
-            return Status::OLAPInternalError(OLAP_ERR_TABLE_INDEX_VALIDATE_ERROR);
-        }
-        Status res = segment_group->load(use_cache);
-        if (!res.ok()) {
-            LOG(WARNING) << "fail to load segment_group. res=" << res << ", "
-                         << "version=" << start_version() << "-" << end_version();
-            return res;
-        }
-    }
-    return Status::OK();
-}
-
-Status AlphaRowset::create_reader(std::shared_ptr<RowsetReader>* result) {
-    result->reset(new AlphaRowsetReader(_schema->num_rows_per_row_block(),
-                                        std::static_pointer_cast<AlphaRowset>(shared_from_this())));
-    return Status::OK();
-}
-
-Status AlphaRowset::remove() {
-    VLOG_NOTICE << "begin to remove files in rowset " << unique_id()
-                << ", version:" << start_version() << "-" << end_version()
-                << ", tabletid:" << _rowset_meta->tablet_id();
-    for (auto segment_group : _segment_groups) {
-        bool ret = segment_group->delete_all_files();
-        if (!ret) {
-            LOG(WARNING) << "delete segment group files failed."
-                         << " tablet id:" << segment_group->get_tablet_id()
-                         << ", rowset path:" << segment_group->rowset_path_prefix();
-            return Status::OLAPInternalError(OLAP_ERR_ROWSET_DELETE_FILE_FAILED);
-        }
-    }
-    return Status::OK();
-}
-
-void AlphaRowset::make_visible_extra(Version version) {
-    AlphaRowsetMetaSharedPtr alpha_rowset_meta =
-            std::dynamic_pointer_cast<AlphaRowsetMeta>(_rowset_meta);
-    std::vector<SegmentGroupPB> published_segment_groups;
-    alpha_rowset_meta->get_segment_groups(&published_segment_groups);
-    int32_t segment_group_idx = 0;
-    for (auto& segment_group : _segment_groups) {
-        segment_group->set_version(version);
-        segment_group->set_pending_finished();
-        published_segment_groups.at(segment_group_idx).clear_load_id();
-        ++segment_group_idx;
-    }
-    alpha_rowset_meta->clear_segment_group();
-    for (auto& segment_group_meta : published_segment_groups) {
-        alpha_rowset_meta->add_segment_group(segment_group_meta);
-    }
-}
-
-Status AlphaRowset::link_files_to(const FilePathDesc& dir_desc, RowsetId new_rowset_id) {
-    for (auto& segment_group : _segment_groups) {
-        auto status = segment_group->link_segments_to_path(dir_desc.filepath, new_rowset_id);
-        if (!status.ok()) {
-            LOG(WARNING) << "create hard links failed for segment group:"
-                         << segment_group->segment_group_id();
-            return status;
-        }
-    }
-    return Status::OK();
-}
-
-Status AlphaRowset::copy_files_to(const std::string& dir, const RowsetId& new_rowset_id) {
-    for (auto& segment_group : _segment_groups) {
-        Status status = segment_group->copy_files_to(dir);
-        if (!status.ok()) {
-            LOG(WARNING) << "copy files failed for segment group."
-                         << " segment_group_id:" << segment_group->segment_group_id()
-                         << ", dest_path:" << dir;
-            return status;
-        }
-    }
-    return Status::OK();
-}
-
-Status AlphaRowset::convert_from_old_files(const std::string& snapshot_path,
-                                           std::vector<std::string>* success_files) {
-    for (auto& segment_group : _segment_groups) {
-        Status status = segment_group->convert_from_old_files(snapshot_path, success_files);
-        if (!status.ok()) {
-            LOG(WARNING) << "create hard links failed for segment group:"
-                         << segment_group->segment_group_id();
-            return status;
-        }
-    }
-    return Status::OK();
-}
-
-Status AlphaRowset::convert_to_old_files(const std::string& snapshot_path,
-                                         std::vector<std::string>* success_files) {
-    for (auto& segment_group : _segment_groups) {
-        Status status = segment_group->convert_to_old_files(snapshot_path, success_files);
-        if (!status.ok()) {
-            LOG(WARNING) << "create hard links failed for segment group:"
-                         << segment_group->segment_group_id();
-            return status;
-        }
-    }
-    return Status::OK();
-}
-
-Status AlphaRowset::remove_old_files(std::vector<std::string>* files_to_remove) {
-    for (auto& segment_group : _segment_groups) {
-        Status status = segment_group->remove_old_files(files_to_remove);
-        if (!status.ok()) {
-            LOG(WARNING) << "remove old files failed for segment group:"
-                         << segment_group->segment_group_id();
-            return status;
-        }
-    }
-    return Status::OK();
-}
-
-Status AlphaRowset::split_range(const RowCursor& start_key, const RowCursor& end_key,
-                                uint64_t request_block_row_count, size_t key_num,
-                                std::vector<OlapTuple>* ranges) {
-    if (key_num > _schema->num_short_key_columns()) {
-        // should not happen
-        // But since aloha rowset is deprecated in future and it will not fail the query,
-        // just use VLOG to avoid too many warning logs.
-        VLOG_NOTICE << "key num " << key_num
-                    << " should less than or equal to short key column number: "
-                    << _schema->num_short_key_columns();
-        return Status::OLAPInternalError(OLAP_ERR_INVALID_SCHEMA);
-    }
-    EntrySlice entry;
-    RowBlockPosition start_pos;
-    RowBlockPosition end_pos;
-    RowBlockPosition step_pos;
-
-    std::shared_ptr<SegmentGroup> largest_segment_group = _segment_group_with_largest_size();
-    if (largest_segment_group == nullptr ||
-        largest_segment_group->current_num_rows_per_row_block() == 0) {
-        VLOG_NOTICE << "failed to get largest_segment_group. is null: "
-                    << (largest_segment_group == nullptr) << ". version: " << start_version() << "-"
-                    << end_version() << ". tablet: " << rowset_meta()->tablet_id();
-        ranges->emplace_back(start_key.to_tuple());
-        ranges->emplace_back(end_key.to_tuple());
-        return Status::OK();
-    }
-    uint64_t expected_rows =
-            request_block_row_count / largest_segment_group->current_num_rows_per_row_block();
-    if (expected_rows == 0) {
-        LOG(WARNING) << "expected_rows less than 1. [request_block_row_count = "
-                     << request_block_row_count << "]";
-        return Status::OLAPInternalError(OLAP_ERR_INVALID_SCHEMA);
-    }
-
-    // find the start position of start key
-    RowCursor helper_cursor;
-    if (helper_cursor.init(*_schema, key_num) != Status::OK()) {
-        LOG(WARNING) << "fail to parse strings to key with RowCursor type.";
-        return Status::OLAPInternalError(OLAP_ERR_INVALID_SCHEMA);
-    }
-    if (!largest_segment_group->find_short_key(start_key, &helper_cursor, false, &start_pos)) {
-        if (!largest_segment_group->find_first_row_block(&start_pos)) {
-            LOG(WARNING) << "fail to get first block pos";
-            return Status::OLAPInternalError(OLAP_ERR_TABLE_INDEX_FIND_ERROR);
-        }
-    }
-
-    step_pos = start_pos;
-    VLOG_NOTICE << "start_pos=" << start_pos.segment << ", " << start_pos.index_offset;
-
-    //find last row_block is end_key is given, or using last_row_block
-    if (!largest_segment_group->find_short_key(end_key, &helper_cursor, false, &end_pos)) {
-        if (!largest_segment_group->find_last_row_block(&end_pos)) {
-            LOG(WARNING) << "fail find last row block.";
-            return Status::OLAPInternalError(OLAP_ERR_TABLE_INDEX_FIND_ERROR);
-        }
-    }
-
-    VLOG_NOTICE << "end_pos=" << end_pos.segment << ", " << end_pos.index_offset;
-
-    //get rows between first and last
-    Status res = Status::OK();
-    RowCursor cur_start_key;
-    RowCursor last_start_key;
-
-    if (cur_start_key.init(*_schema, key_num) != Status::OK() ||
-        last_start_key.init(*_schema, key_num) != Status::OK()) {
-        LOG(WARNING) << "fail to init cursor";
-        return Status::OLAPInternalError(OLAP_ERR_INIT_FAILED);
-    }
-
-    std::vector<uint32_t> cids;
-    for (uint32_t cid = 0; cid < key_num; ++cid) {
-        cids.push_back(cid);
-    }
-
-    if (largest_segment_group->get_row_block_entry(start_pos, &entry) != Status::OK()) {
-        LOG(WARNING) << "get block entry failed.";
-        return Status::OLAPInternalError(OLAP_ERR_ROWBLOCK_FIND_ROW_EXCEPTION);
-    }
-
-    cur_start_key.attach(entry.data);
-    last_start_key.allocate_memory_for_string_type(*_schema);
-    direct_copy_row(&last_start_key, cur_start_key);
-    // start_key是last start_key, 但返回的实际上是查询层给出的key
-    ranges->emplace_back(start_key.to_tuple());
-
-    while (end_pos > step_pos) {
-        res = largest_segment_group->advance_row_block(expected_rows, &step_pos);
-        if (res == Status::OLAPInternalError(OLAP_ERR_INDEX_EOF) || !(end_pos > step_pos)) {
-            break;
-        } else if (!res.ok()) {
-            LOG(WARNING) << "advance_row_block failed.";
-            return Status::OLAPInternalError(OLAP_ERR_ROWBLOCK_FIND_ROW_EXCEPTION);
-        }
-
-        if (largest_segment_group->get_row_block_entry(step_pos, &entry) != Status::OK()) {
-            LOG(WARNING) << "get block entry failed.";
-            return Status::OLAPInternalError(OLAP_ERR_ROWBLOCK_FIND_ROW_EXCEPTION);
-        }
-        cur_start_key.attach(entry.data);
-
-        if (!equal_row(cids, cur_start_key, last_start_key)) {
-            ranges->emplace_back(cur_start_key.to_tuple()); // end of last section
-            ranges->emplace_back(cur_start_key.to_tuple()); // start a new section
-            direct_copy_row(&last_start_key, cur_start_key);
-        }
-    }
-
-    ranges->emplace_back(end_key.to_tuple());
-    return Status::OK();
-}
-
-bool AlphaRowset::check_path(const std::string& path) {
-    std::set<std::string> valid_paths;
-    for (auto& segment_group : _segment_groups) {
-        for (int i = 0; i < segment_group->num_segments(); ++i) {
-            std::string data_path = segment_group->construct_data_file_path(i);
-            std::string index_path = segment_group->construct_index_file_path(i);
-            valid_paths.insert(data_path);
-            valid_paths.insert(index_path);
-        }
-    }
-    return valid_paths.find(path) != valid_paths.end();
-}
-
-bool AlphaRowset::check_file_exist() {
-    for (auto& segment_group : _segment_groups) {
-        for (int i = 0; i < segment_group->num_segments(); ++i) {
-            std::string data_path = segment_group->construct_data_file_path(i);
-            if (!FileUtils::check_exist(data_path)) {
-                LOG(WARNING) << "data file not existed: " << data_path
-                             << " for rowset_id: " << rowset_id();
-                return false;
-            }
-            std::string index_path = segment_group->construct_index_file_path(i);
-            if (!FileUtils::check_exist(index_path)) {
-                LOG(WARNING) << "index file not existed: " << index_path
-                             << " for rowset_id: " << rowset_id();
-                return false;
-            }
-        }
-    }
-    return true;
-}
-
-Status AlphaRowset::init() {
-    std::vector<SegmentGroupPB> segment_group_metas;
-    AlphaRowsetMetaSharedPtr _alpha_rowset_meta =
-            std::dynamic_pointer_cast<AlphaRowsetMeta>(_rowset_meta);
-    _alpha_rowset_meta->get_segment_groups(&segment_group_metas);
-    for (auto& segment_group_meta : segment_group_metas) {
-        std::shared_ptr<SegmentGroup> segment_group;
-        if (_is_pending) {
-            segment_group.reset(new SegmentGroup(
-                    _rowset_meta->tablet_id(), _rowset_meta->rowset_id(), _schema,
-                    _rowset_path_desc.filepath, false, segment_group_meta.segment_group_id(),
-                    segment_group_meta.num_segments(), true, _rowset_meta->partition_id(),
-                    _rowset_meta->txn_id()));
-        } else {
-            segment_group.reset(new SegmentGroup(
-                    _rowset_meta->tablet_id(), _rowset_meta->rowset_id(), _schema,
-                    _rowset_path_desc.filepath, _rowset_meta->version(), false,
-                    segment_group_meta.segment_group_id(), segment_group_meta.num_segments()));
-        }
-        if (segment_group == nullptr) {
-            LOG(WARNING) << "fail to create olap segment_group. rowset_id='"
-                         << _rowset_meta->rowset_id();
-            return Status::OLAPInternalError(OLAP_ERR_CREATE_FILE_ERROR);
-        }
-        if (segment_group_meta.has_empty()) {
-            segment_group->set_empty(segment_group_meta.empty());
-        }
-
-        if (segment_group_meta.zone_maps_size() != 0) {
-            size_t zone_maps_size = segment_group_meta.zone_maps_size();
-            // after 0.12.10 the value column in duplicate table also has zone map.
-            // after 0.14 the value column in duplicate table also has zone map.
-            size_t expect_zone_maps_num = _schema->keys_type() != KeysType::AGG_KEYS
-                                                  ? _schema->num_columns()
-                                                  : _schema->num_key_columns();
-            if ((_schema->keys_type() == KeysType::AGG_KEYS &&
-                 expect_zone_maps_num != zone_maps_size) ||
-                (_schema->keys_type() != KeysType::AGG_KEYS &&
-                 expect_zone_maps_num < zone_maps_size)) {
-                LOG(ERROR) << "column pruning size is error. "
-                           << "KeysType=" << KeysType_Name(_schema->keys_type()) << ", "
-                           << "zone_maps_size=" << zone_maps_size << ", "
-                           << "num_key_columns=" << _schema->num_key_columns() << ", "
-                           << "num_columns=" << _schema->num_columns();
-                return Status::OLAPInternalError(OLAP_ERR_TABLE_INDEX_VALIDATE_ERROR);
-            }
-            // Before 0.12.10, the zone map columns number in duplicate/unique table is the same with the key column numbers,
-            // but after 0.12.10 we build zone map for duplicate table value column, after 0.14 we build zone map for unique
-            // table value column, so when first start the two number is not the same,
-            // it causes start failed. When `expect_zone_maps_num > zone_maps_size` it may be the first start after upgrade
-            if (expect_zone_maps_num > zone_maps_size) {
-                VLOG_CRITICAL
-                        << "tablet: " << _rowset_meta->tablet_id() << " expect zone map size is "
-                        << expect_zone_maps_num << ", actual num is " << zone_maps_size
-                        << ". If this is not the first start after upgrade, please pay attention!";
-            }
-            zone_maps_size = std::min(zone_maps_size, expect_zone_maps_num);
-            std::vector<std::pair<std::string, std::string>> zone_map_strings(zone_maps_size);
-            std::vector<bool> null_vec(zone_maps_size);
-            for (size_t j = 0; j < zone_maps_size; ++j) {
-                const ZoneMap& zone_map = segment_group_meta.zone_maps(j);
-                zone_map_strings[j].first = zone_map.min();
-                zone_map_strings[j].second = zone_map.max();
-                if (zone_map.has_null_flag()) {
-                    null_vec[j] = zone_map.null_flag();
-                } else {
-                    null_vec[j] = false;
-                }
-            }
-            Status status = segment_group->add_zone_maps(zone_map_strings, null_vec);
-            if (!status.ok()) {
-                LOG(WARNING) << "segment group add column statistics failed, status:" << status;
-                return status;
-            }
-        }
-        _segment_groups.push_back(segment_group);
-    }
-    if (_is_cumulative && _segment_groups.size() > 1) {
-        LOG(WARNING) << "invalid segment group meta for cumulative rowset. segment group size:"
-                     << _segment_groups.size();
-        return Status::OLAPInternalError(OLAP_ERR_ENGINE_LOAD_INDEX_TABLE_ERROR);
-    }
-    return Status::OK();
-}
-
-std::shared_ptr<SegmentGroup> AlphaRowset::_segment_group_with_largest_size() {
-    std::shared_ptr<SegmentGroup> largest_segment_group = nullptr;
-    size_t largest_segment_group_sizes = 0;
-
-    for (auto segment_group : _segment_groups) {
-        if (!segment_group->index_loaded()) {
-            continue;
-        }
-        if (segment_group->empty() || segment_group->zero_num_rows()) {
-            continue;
-        }
-        if (segment_group->index_size() > largest_segment_group_sizes) {
-            largest_segment_group = segment_group;
-            largest_segment_group_sizes = segment_group->index_size();
-        }
-    }
-    return largest_segment_group;
-}
-
-} // namespace doris
diff --git a/be/src/olap/rowset/alpha_rowset.h b/be/src/olap/rowset/alpha_rowset.h
deleted file mode 100644
index 9f4c838723..0000000000
--- a/be/src/olap/rowset/alpha_rowset.h
+++ /dev/null
@@ -1,94 +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.
-
-#ifndef DORIS_BE_SRC_OLAP_ROWSET_ALPHA_ROWSET_H
-#define DORIS_BE_SRC_OLAP_ROWSET_ALPHA_ROWSET_H
-
-#include <memory>
-#include <vector>
-
-#include "olap/data_dir.h"
-#include "olap/rowset/rowset.h"
-#include "olap/rowset/rowset_meta.h"
-#include "olap/rowset/segment_group.h"
-#include "olap/tuple.h"
-
-namespace doris {
-
-class AlphaRowset;
-using AlphaRowsetSharedPtr = std::shared_ptr<AlphaRowset>;
-class AlphaRowsetWriter;
-class AlphaRowsetReader;
-class RowsetFactory;
-
-class AlphaRowset : public Rowset {
-public:
-    virtual ~AlphaRowset() {}
-
-    Status create_reader(std::shared_ptr<RowsetReader>* result) override;
-
-    Status split_range(const RowCursor& start_key, const RowCursor& end_key,
-                       uint64_t request_block_row_count, size_t key_num,
-                       std::vector<OlapTuple>* ranges) override;
-
-    Status remove() override;
-
-    Status link_files_to(const FilePathDesc& dir_desc, RowsetId new_rowset_id) override;
-
-    Status copy_files_to(const std::string& dir, const RowsetId& new_rowset_id) override;
-
-    Status convert_from_old_files(const std::string& snapshot_path,
-                                  std::vector<std::string>* success_files);
-
-    Status convert_to_old_files(const std::string& snapshot_path,
-                                std::vector<std::string>* success_files);
-
-    Status remove_old_files(std::vector<std::string>* files_to_remove) override;
-
-    bool check_path(const std::string& path) override;
-
-    bool check_file_exist() override;
-
-protected:
-    friend class RowsetFactory;
-
-    AlphaRowset(const TabletSchema* schema, const FilePathDesc& rowset_path_desc,
-                RowsetMetaSharedPtr rowset_meta);
-
-    // init segment groups
-    Status init() override;
-
-    Status do_load(bool use_cache) override;
-
-    void do_close() override {}
-
-    // add custom logic when rowset is published
-    void make_visible_extra(Version version) override;
-
-private:
-    std::shared_ptr<SegmentGroup> _segment_group_with_largest_size();
-
-private:
-    friend class AlphaRowsetWriter;
-    friend class AlphaRowsetReader;
-
-    std::vector<std::shared_ptr<SegmentGroup>> _segment_groups;
-};
-
-} // namespace doris
-
-#endif // DORIS_BE_SRC_OLAP_ROWSET_ALPHA_ROWSET_H
diff --git a/be/src/olap/rowset/alpha_rowset_reader.cpp b/be/src/olap/rowset/alpha_rowset_reader.cpp
deleted file mode 100644
index 3b0d7472ee..0000000000
--- a/be/src/olap/rowset/alpha_rowset_reader.cpp
+++ /dev/null
@@ -1,400 +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 "olap/rowset/alpha_rowset_reader.h"
-
-#include "olap/row.h"
-#include "olap/rowset/alpha_rowset.h"
-
-namespace doris {
-
-AlphaRowsetReader::AlphaRowsetReader(int num_rows_per_row_block, AlphaRowsetSharedPtr rowset)
-        : _num_rows_per_row_block(num_rows_per_row_block),
-          _rowset(std::move(rowset)),
-          _alpha_rowset_meta(
-                  std::static_pointer_cast<AlphaRowsetMeta>(_rowset->rowset_meta()).get()),
-          _segment_groups(_rowset->_segment_groups),
-          _key_range_size(0) {
-    _rowset->acquire();
-}
-
-AlphaRowsetReader::~AlphaRowsetReader() {
-    delete _dst_cursor;
-    _rowset->release();
-    while (!_merge_heap.empty()) {
-        auto ctx = _merge_heap.top();
-        _merge_heap.pop();
-        delete ctx;
-    }
-    for (auto ctx : _sequential_ctxs) {
-        delete ctx;
-    }
-    _sequential_ctxs.clear();
-}
-
-Status AlphaRowsetReader::init(RowsetReaderContext* read_context) {
-    RETURN_NOT_OK(_rowset->load());
-    if (read_context == nullptr) {
-        return Status::OLAPInternalError(OLAP_ERR_INIT_FAILED);
-    }
-    _current_read_context = read_context;
-    if (_current_read_context->stats != nullptr) {
-        _stats = _current_read_context->stats;
-    }
-
-    _is_segments_overlapping = _alpha_rowset_meta->is_segments_overlapping();
-
-    RETURN_NOT_OK(_init_merge_ctxs(read_context));
-
-    // needs to sort merge only when
-    // 1) we are told to return sorted result (need_ordered_result)
-    // 2) we have several segment groups (_is_segments_overlapping && _sequential_ctxs.size() > 1)
-    if (_current_read_context->need_ordered_result && _is_segments_overlapping &&
-        _sequential_ctxs.size() > 1) {
-        _next_block = &AlphaRowsetReader::_merge_block;
-        _read_block.reset(new (std::nothrow) RowBlock(_current_read_context->tablet_schema));
-        if (_read_block == nullptr) {
-            LOG(WARNING) << "new row block failed in reader";
-            return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
-        }
-        RowBlockInfo block_info;
-        block_info.row_num = _current_read_context->tablet_schema->num_rows_per_row_block();
-        block_info.null_supported = true;
-        _read_block->init(block_info);
-        _dst_cursor = new (std::nothrow) RowCursor();
-        if (_dst_cursor == nullptr) {
-            LOG(WARNING) << "allocate memory for row cursor failed";
-            return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
-        }
-        if (_current_read_context->reader_type == READER_ALTER_TABLE) {
-            // Upon rollup/alter table, seek_columns is nullptr.
-            // Under this circumstance, init RowCursor with all columns.
-            _dst_cursor->init(*(_current_read_context->tablet_schema));
-            for (auto ctx : _sequential_ctxs) {
-                ctx->row_cursor.reset(new (std::nothrow) RowCursor());
-                ctx->row_cursor->init(*(_current_read_context->tablet_schema));
-            }
-        } else {
-            _dst_cursor->init(*(_current_read_context->tablet_schema),
-                              *(_current_read_context->seek_columns));
-            for (auto ctx : _sequential_ctxs) {
-                ctx->row_cursor.reset(new (std::nothrow) RowCursor());
-                ctx->row_cursor->init(*(_current_read_context->tablet_schema),
-                                      *(_current_read_context->seek_columns));
-            }
-        }
-        RETURN_NOT_OK(_init_merge_heap());
-    } else {
-        _next_block = &AlphaRowsetReader::_union_block;
-        _cur_ctx = *(_sequential_ctxs.begin());
-    }
-    return Status::OK();
-}
-
-Status AlphaRowsetReader::next_block(RowBlock** block) {
-    return (this->*_next_block)(block);
-}
-
-bool AlphaRowsetReader::delete_flag() {
-    return _alpha_rowset_meta->delete_flag();
-}
-
-Version AlphaRowsetReader::version() {
-    return _alpha_rowset_meta->version();
-}
-
-int64_t AlphaRowsetReader::filtered_rows() {
-    return _stats->rows_del_filtered;
-}
-
-Status AlphaRowsetReader::_union_block(RowBlock** block) {
-    while (_cur_ctx != nullptr) {
-        // union block only use one block to store
-        Status status = _pull_next_block(_cur_ctx);
-        if (status == Status::OLAPInternalError(OLAP_ERR_DATA_EOF)) {
-            delete _cur_ctx;
-            _cur_ctx = nullptr;
-            _sequential_ctxs.pop_front();
-            if (!_sequential_ctxs.empty()) {
-                _cur_ctx = *(_sequential_ctxs.begin());
-            }
-        } else if (!status.ok()) {
-            return status;
-        } else {
-            (*block) = _cur_ctx->row_block;
-            return Status::OK();
-        }
-    }
-    if (_sequential_ctxs.empty()) {
-        *block = nullptr;
-        return Status::OLAPInternalError(OLAP_ERR_DATA_EOF);
-    }
-
-    return Status::OK();
-}
-
-Status AlphaRowsetReader::_merge_block(RowBlock** block) {
-    // Row among different segment groups may overlap with each other.
-    // Iterate all row_blocks to fetch min row each round.
-    Status status = Status::OK();
-    _read_block->clear();
-    size_t num_rows_in_block = 0;
-    while (_read_block->pos() < _num_rows_per_row_block) {
-        // 1. Read one row from heap
-        RowCursor* row_cursor = nullptr;
-        status = _pull_next_row_for_merge_rowset_v2(&row_cursor);
-        if (status == Status::OLAPInternalError(OLAP_ERR_DATA_EOF) && _read_block->pos() > 0) {
-            status = Status::OK();
-            break;
-        } else if (!status.ok()) {
-            return status;
-        }
-
-        VLOG_TRACE << "get merged row: " << row_cursor->to_string();
-
-        // 2. Copy the row to buffer block
-        _read_block->get_row(_read_block->pos(), _dst_cursor);
-        copy_row(_dst_cursor, *row_cursor, _read_block->mem_pool());
-        _read_block->pos_inc();
-        num_rows_in_block++;
-
-        // 3. Adjust heap
-        // MergeHeap should advance one step after row been read.
-        // This function must be called after copy_row
-        // Otherwise, the row has read will be modified instantly before handled.
-        // For example:
-        // If I have (1, 1), (2, 2), (3, 3) three records.
-        // Now I have read (1, 1).
-        // Before copy_row, I rebuild the heap
-        // The returned row will be (2, 2) instead of (1, 1)
-        AlphaMergeContext* merge_ctx = _merge_heap.top();
-        _merge_heap.pop();
-        // merge_ctx will not be pushed back into heap if it is EOF
-        RETURN_NOT_OK(_update_merge_ctx_and_build_merge_heap(merge_ctx));
-    }
-    _read_block->set_pos(0);
-    _read_block->set_limit(num_rows_in_block);
-    _read_block->finalize(num_rows_in_block);
-    *block = _read_block.get();
-    return status;
-}
-
-Status AlphaRowsetReader::_init_merge_heap() {
-    DCHECK(_merge_heap.empty());
-    DCHECK(!_sequential_ctxs.empty());
-    for (auto merge_ctx : _sequential_ctxs) {
-        RETURN_NOT_OK(_update_merge_ctx_and_build_merge_heap(merge_ctx));
-    }
-    _sequential_ctxs.clear();
-    return Status::OK();
-}
-
-Status AlphaRowsetReader::_update_merge_ctx_and_build_merge_heap(AlphaMergeContext* merge_ctx) {
-    if (OLAP_UNLIKELY(merge_ctx->is_eof)) {
-        // nothing in this merge ctx, release and return
-        delete merge_ctx;
-        return Status::OK();
-    }
-
-    // get next row block of this merge ctx
-    if (merge_ctx->row_block == nullptr || !merge_ctx->row_block->has_remaining()) {
-        Status status = _pull_next_block(merge_ctx);
-        if (status == Status::OLAPInternalError(OLAP_ERR_DATA_EOF)) {
-            // nothing in this merge ctx, release and return
-            delete merge_ctx;
-            return Status::OK();
-        } else if (!status.ok()) {
-            delete merge_ctx;
-            LOG(WARNING) << "read next row of singleton rowset failed:" << status;
-            return status;
-        }
-    }
-
-    // read the first row, push it into merge heap, and step forward
-    RowCursor* current_row = merge_ctx->row_cursor.get();
-    merge_ctx->row_block->get_row(merge_ctx->row_block->pos(), current_row);
-    _merge_heap.push(merge_ctx);
-    merge_ctx->row_block->pos_inc();
-    return Status::OK();
-}
-
-Status AlphaRowsetReader::_pull_next_row_for_merge_rowset_v2(RowCursor** row) {
-    // if _merge_heap is not empty, return the row at top, and insert a new row
-    // from corresponding merge_ctx
-    if (OLAP_LIKELY(!_merge_heap.empty())) {
-        AlphaMergeContext* merge_ctx = _merge_heap.top();
-        *row = merge_ctx->row_cursor.get();
-        // Must not rebuild merge_heap in this place.
-        // Because row have not been copied and is a pointer.
-        // If rebuild merge_heap, content in row will be modified.
-        return Status::OK();
-    } else {
-        // all rows are read
-        return Status::OLAPInternalError(OLAP_ERR_DATA_EOF);
-    }
-}
-
-Status AlphaRowsetReader::_pull_next_block(AlphaMergeContext* merge_ctx) {
-    Status status = Status::OK();
-    if (OLAP_UNLIKELY(merge_ctx->first_read_symbol)) {
-        if (_key_range_size > 0) {
-            status = _pull_first_block(merge_ctx);
-        } else {
-            status = merge_ctx->column_data->get_first_row_block(&(merge_ctx->row_block));
-            if (!status.ok() && status != Status::OLAPInternalError(OLAP_ERR_DATA_EOF)) {
-                LOG(WARNING) << "get first row block failed, status:" << status;
-            }
-        }
-        merge_ctx->first_read_symbol = false;
-        return status;
-    } else {
-        // get next block
-        status = merge_ctx->column_data->get_next_block(&(merge_ctx->row_block));
-        if (status == Status::OLAPInternalError(OLAP_ERR_DATA_EOF) && _key_range_size > 0) {
-            // reach the end of one predicate
-            // currently, SegmentReader can only support filter one key range a time
-            // refresh the predicate and continue read
-            return _pull_first_block(merge_ctx);
-        }
-    }
-    return status;
-}
-
-Status AlphaRowsetReader::_pull_first_block(AlphaMergeContext* merge_ctx) {
-    Status status = Status::OK();
-    merge_ctx->key_range_index++;
-    while (merge_ctx->key_range_index < _key_range_size) {
-        status = merge_ctx->column_data->prepare_block_read(
-                &_current_read_context->lower_bound_keys->at(merge_ctx->key_range_index),
-                _current_read_context->is_lower_keys_included->at(merge_ctx->key_range_index),
-                &_current_read_context->upper_bound_keys->at(merge_ctx->key_range_index),
-                _current_read_context->is_upper_keys_included->at(merge_ctx->key_range_index),
-                &(merge_ctx->row_block));
-        if (status == Status::OLAPInternalError(OLAP_ERR_DATA_EOF)) {
-            merge_ctx->key_range_index++;
-            continue;
-        } else if (!status.ok()) {
-            LOG(WARNING) << "prepare block read failed. status=" << status;
-            return status;
-        } else {
-            break;
-        }
-    }
-    if (merge_ctx->key_range_index >= _key_range_size) {
-        merge_ctx->row_block = nullptr;
-        return Status::OLAPInternalError(OLAP_ERR_DATA_EOF);
-    }
-    return status;
-}
-
-Status AlphaRowsetReader::_init_merge_ctxs(RowsetReaderContext* read_context) {
-    if (read_context->reader_type == READER_QUERY) {
-        if (read_context->lower_bound_keys->size() !=
-                    read_context->is_lower_keys_included->size() ||
-            read_context->lower_bound_keys->size() != read_context->upper_bound_keys->size() ||
-            read_context->upper_bound_keys->size() !=
-                    read_context->is_upper_keys_included->size()) {
-            std::string error_msg = "invalid key range arguments";
-            LOG(WARNING) << error_msg;
-            return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
-        }
-        _key_range_size = read_context->lower_bound_keys->size();
-    }
-
-    // avoid polluting index stream cache by non-query workload (compaction/alter/checksum)
-    const bool use_index_stream_cache = read_context->reader_type == READER_QUERY;
-
-    for (auto& segment_group : _segment_groups) {
-        std::unique_ptr<ColumnData> new_column_data(ColumnData::create(segment_group.get()));
-        Status status = new_column_data->init();
-        if (!status.ok()) {
-            LOG(WARNING) << "init column data failed";
-            return Status::OLAPInternalError(OLAP_ERR_READER_READING_ERROR);
-        }
-        new_column_data->set_delete_handler(read_context->delete_handler);
-        new_column_data->set_stats(_stats);
-        if (read_context->reader_type == READER_ALTER_TABLE) {
-            new_column_data->schema_change_init();
-            new_column_data->set_using_cache(use_index_stream_cache);
-            if (new_column_data->empty() && new_column_data->zero_num_rows()) {
-                continue;
-            }
-        } else {
-            std::shared_ptr<std::vector<ColumnPredicate*>> predicates =
-                    std::make_shared<std::vector<ColumnPredicate*>>();
-            if (read_context->predicates != nullptr) {
-                predicates->insert(predicates->end(), read_context->predicates->begin(),
-                                   read_context->predicates->end());
-            }
-            // if unique table with rowset [0-x] or [0-1] [2-y] [...],
-            // value column predicates can be pushdown on rowset [0-x] or [2-y]
-            if (read_context->value_predicates != nullptr && _rowset->keys_type() == UNIQUE_KEYS &&
-                (_rowset->start_version() == 0 || _rowset->start_version() == 2)) {
-                predicates->insert(predicates->end(), read_context->value_predicates->begin(),
-                                   read_context->value_predicates->end());
-            }
-            new_column_data->set_read_params(
-                    *read_context->return_columns, *read_context->seek_columns,
-                    *read_context->load_bf_columns, *read_context->conditions, predicates,
-                    use_index_stream_cache, read_context->runtime_state);
-            // filter
-            if (new_column_data->rowset_pruning_filter()) {
-                _stats->rows_stats_filtered += new_column_data->num_rows();
-                VLOG_NOTICE << "filter segment group in query in condition. version="
-                            << new_column_data->version();
-                continue;
-            }
-        }
-
-        int ret = new_column_data->delete_pruning_filter();
-        if (ret == DEL_SATISFIED) {
-            _stats->rows_del_filtered += new_column_data->num_rows();
-            VLOG_NOTICE << "filter segment group in delete predicate:"
-                        << new_column_data->version();
-            continue;
-        } else if (ret == DEL_PARTIAL_SATISFIED) {
-            VLOG_NOTICE << "filter segment group partially in delete predicate:"
-                        << new_column_data->version();
-            new_column_data->set_delete_status(DEL_PARTIAL_SATISFIED);
-        } else {
-            VLOG_NOTICE << "not filter segment group in delete predicate:"
-                        << new_column_data->version();
-            new_column_data->set_delete_status(DEL_NOT_SATISFIED);
-        }
-        auto merge_ctx = new AlphaMergeContext();
-        merge_ctx->column_data = std::move(new_column_data);
-        _sequential_ctxs.emplace_back(merge_ctx);
-    }
-
-    if (!_is_segments_overlapping && _sequential_ctxs.size() > 1) {
-        LOG(WARNING) << "invalid column_data for cumulative rowset. column_data size:"
-                     << _sequential_ctxs.size();
-        return Status::OLAPInternalError(OLAP_ERR_READER_READING_ERROR);
-    }
-    return Status::OK();
-}
-
-RowsetSharedPtr AlphaRowsetReader::rowset() {
-    return std::static_pointer_cast<Rowset>(_rowset);
-}
-
-bool AlphaMergeContextComparator::operator()(const AlphaMergeContext* x,
-                                             const AlphaMergeContext* y) const {
-    return compare_row(*(x->row_cursor.get()), *(y->row_cursor.get())) > 0;
-}
-
-} // namespace doris
diff --git a/be/src/olap/rowset/alpha_rowset_reader.h b/be/src/olap/rowset/alpha_rowset_reader.h
deleted file mode 100644
index 3dffec2457..0000000000
--- a/be/src/olap/rowset/alpha_rowset_reader.h
+++ /dev/null
@@ -1,136 +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.
-
-#ifndef DORIS_BE_SRC_OLAP_ROWSET_ALPHA_ROWSET_READER_H
-#define DORIS_BE_SRC_OLAP_ROWSET_ALPHA_ROWSET_READER_H
-
-#include <queue>
-#include <vector>
-
-#include "olap/rowset/alpha_rowset.h"
-#include "olap/rowset/alpha_rowset_meta.h"
-#include "olap/rowset/column_data.h"
-#include "olap/rowset/rowset_reader.h"
-#include "olap/rowset/segment_group.h"
-
-namespace doris {
-
-// Each segment group corresponds to a MergeContext, which is able to produce ordered rows.
-struct AlphaMergeContext {
-    std::unique_ptr<ColumnData> column_data = nullptr;
-
-    int key_range_index = -1;
-
-    // Read data from ColumnData for the first time.
-    // ScanKey should be sought in this case.
-    bool first_read_symbol = true;
-
-    RowBlock* row_block = nullptr;
-
-    std::unique_ptr<RowCursor> row_cursor = nullptr;
-
-    bool is_eof = false;
-};
-
-struct AlphaMergeContextComparator {
-    bool operator()(const AlphaMergeContext* x, const AlphaMergeContext* y) const;
-};
-
-class AlphaRowsetReader : public RowsetReader {
-public:
-    AlphaRowsetReader(int num_rows_per_row_block, AlphaRowsetSharedPtr rowset);
-
-    ~AlphaRowsetReader() override;
-
-    // reader init
-    Status init(RowsetReaderContext* read_context) override;
-
-    // read next block data
-    // It's ok, because we only get ref here, the block's owner is this reader.
-    Status next_block(RowBlock** block) override;
-
-    Status next_block(vectorized::Block* block) override {
-        return Status::OLAPInternalError(OLAP_ERR_DATA_EOF);
-    }
-
-    bool delete_flag() override;
-
-    Version version() override;
-
-    RowsetSharedPtr rowset() override;
-
-    int64_t filtered_rows() override;
-
-    RowsetTypePB type() const override { return RowsetTypePB::ALPHA_ROWSET; }
-
-private:
-    Status _init_merge_ctxs(RowsetReaderContext* read_context);
-
-    Status _union_block(RowBlock** block);
-    Status _merge_block(RowBlock** block);
-    Status _pull_next_block(AlphaMergeContext* merge_ctx);
-
-    // Doris will split query predicates to several scan keys
-    // This function is used to fetch block when advancing
-    // current scan key to next scan key.
-    Status _pull_first_block(AlphaMergeContext* merge_ctx);
-
-    // merge by priority queue(_merge_heap)
-    Status _pull_next_row_for_merge_rowset_v2(RowCursor** row);
-    // init the merge heap, this should be call before calling _pull_next_row_for_merge_rowset_v2();
-    Status _init_merge_heap();
-    // update the merge ctx.
-    // 1. get next row block of this ctx, if current row block is empty.
-    // 2. read the current row of the row block and push it to merge heap.
-    // 3. point to the next row of the row block
-    Status _update_merge_ctx_and_build_merge_heap(AlphaMergeContext* merge_ctx);
-
-private:
-    int _num_rows_per_row_block;
-    AlphaRowsetSharedPtr _rowset;
-    std::string _rowset_path;
-    AlphaRowsetMeta* _alpha_rowset_meta;
-    const std::vector<std::shared_ptr<SegmentGroup>>& _segment_groups;
-
-    // In '_union_block' mode, it has items to traverse.
-    // In '_merge_block' mode, it will be cleared after '_merge_heap' has been built.
-    std::list<AlphaMergeContext*> _sequential_ctxs;
-
-    std::unique_ptr<RowBlock> _read_block;
-    Status (AlphaRowsetReader::*_next_block)(RowBlock** block) = nullptr;
-    RowCursor* _dst_cursor = nullptr;
-    int _key_range_size;
-
-    // In streaming ingestion, row among different segment
-    // groups may overlap, and is necessary to be taken
-    // into consideration deliberately.
-    bool _is_segments_overlapping;
-
-    // Current AlphaMergeContext to read data, just valid in '_union_block' mode.
-    AlphaMergeContext* _cur_ctx = nullptr;
-    // A priority queue for merging rowsets, just valid in '_merge_block' mode.
-    std::priority_queue<AlphaMergeContext*, vector<AlphaMergeContext*>, AlphaMergeContextComparator>
-            _merge_heap;
-
-    RowsetReaderContext* _current_read_context;
-    OlapReaderStatistics _owned_stats;
-    OlapReaderStatistics* _stats = &_owned_stats;
-};
-
-} // namespace doris
-
-#endif // DORIS_BE_SRC_OLAP_ROWSET_ALPHA_ROWSET_READER_H
diff --git a/be/src/olap/rowset/alpha_rowset_writer.cpp b/be/src/olap/rowset/alpha_rowset_writer.cpp
deleted file mode 100644
index 138b9a73cf..0000000000
--- a/be/src/olap/rowset/alpha_rowset_writer.cpp
+++ /dev/null
@@ -1,313 +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 "olap/rowset/alpha_rowset_writer.h"
-
-#include "olap/row.h"
-#include "olap/rowset/alpha_rowset.h"
-#include "olap/rowset/alpha_rowset_meta.h"
-#include "olap/rowset/rowset_factory.h"
-#include "olap/rowset/rowset_meta_manager.h"
-
-namespace doris {
-
-AlphaRowsetWriter::AlphaRowsetWriter()
-        : _segment_group_id(0),
-          _cur_segment_group(nullptr),
-          _column_data_writer(nullptr),
-          _current_rowset_meta(nullptr),
-          _is_pending_rowset(false),
-          _num_rows_written(0),
-          _rowset_build(false),
-          _writer_state(WRITER_CREATED),
-          _need_column_data_writer(true) {}
-
-AlphaRowsetWriter::~AlphaRowsetWriter() {
-    SAFE_DELETE(_column_data_writer);
-    if (!_rowset_build) {
-        _garbage_collection();
-    }
-    for (auto& segment_group : _segment_groups) {
-        segment_group->release();
-        delete segment_group;
-    }
-    _segment_groups.clear();
-}
-
-Status AlphaRowsetWriter::init(const RowsetWriterContext& rowset_writer_context) {
-    _rowset_writer_context = rowset_writer_context;
-    _current_rowset_meta.reset(new (std::nothrow) AlphaRowsetMeta());
-    _current_rowset_meta->set_rowset_id(_rowset_writer_context.rowset_id);
-    _current_rowset_meta->set_partition_id(_rowset_writer_context.partition_id);
-    _current_rowset_meta->set_tablet_uid(_rowset_writer_context.tablet_uid);
-    _current_rowset_meta->set_tablet_id(_rowset_writer_context.tablet_id);
-    _current_rowset_meta->set_tablet_schema_hash(_rowset_writer_context.tablet_schema_hash);
-    _current_rowset_meta->set_rowset_type(_rowset_writer_context.rowset_type);
-    _current_rowset_meta->set_rowset_state(rowset_writer_context.rowset_state);
-    _current_rowset_meta->set_segments_overlap(rowset_writer_context.segments_overlap);
-    RowsetStatePB rowset_state = _rowset_writer_context.rowset_state;
-    if (rowset_state == PREPARED || rowset_state == COMMITTED) {
-        _is_pending_rowset = true;
-    }
-    if (_is_pending_rowset) {
-        _current_rowset_meta->set_txn_id(_rowset_writer_context.txn_id);
-        _current_rowset_meta->set_load_id(_rowset_writer_context.load_id);
-    } else {
-        _current_rowset_meta->set_version(_rowset_writer_context.version);
-    }
-    RETURN_NOT_OK(_init());
-    return Status::OK();
-}
-
-template <typename RowType>
-Status AlphaRowsetWriter::_add_row(const RowType& row) {
-    if (_writer_state != WRITER_INITED) {
-        RETURN_NOT_OK(_init());
-    }
-    Status status = _column_data_writer->write(row);
-    if (!status.ok()) {
-        std::string error_msg = "add row failed";
-        LOG(WARNING) << error_msg;
-        return status;
-    }
-    ++_num_rows_written;
-    return Status::OK();
-}
-
-template Status AlphaRowsetWriter::_add_row(const RowCursor& row);
-template Status AlphaRowsetWriter::_add_row(const ContiguousRow& row);
-
-Status AlphaRowsetWriter::add_rowset(RowsetSharedPtr rowset) {
-    _need_column_data_writer = false;
-    // this api is for clone
-    AlphaRowsetSharedPtr alpha_rowset = std::dynamic_pointer_cast<AlphaRowset>(rowset);
-    for (auto& segment_group : alpha_rowset->_segment_groups) {
-        RETURN_NOT_OK(_init());
-        RETURN_NOT_OK(segment_group->link_segments_to_path(
-                _rowset_writer_context.path_desc.filepath, _rowset_writer_context.rowset_id));
-        _cur_segment_group->set_empty(segment_group->empty());
-        _cur_segment_group->set_num_segments(segment_group->num_segments());
-        _cur_segment_group->add_zone_maps(segment_group->get_zone_maps());
-        RETURN_NOT_OK(flush());
-        _num_rows_written += segment_group->num_rows();
-    }
-    // process delete predicate
-    if (rowset->rowset_meta()->has_delete_predicate()) {
-        _current_rowset_meta->set_delete_predicate(rowset->rowset_meta()->delete_predicate());
-    }
-    return Status::OK();
-}
-
-Status AlphaRowsetWriter::add_rowset_for_linked_schema_change(RowsetSharedPtr rowset,
-                                                              const SchemaMapping& schema_mapping) {
-    _need_column_data_writer = false;
-    // this api is for LinkedSchemaChange
-    // use create hard link to copy rowset for performance
-    // this is feasible because LinkedSchemaChange is done on the same disk
-    AlphaRowsetSharedPtr alpha_rowset = std::dynamic_pointer_cast<AlphaRowset>(rowset);
-    for (auto& segment_group : alpha_rowset->_segment_groups) {
-        RETURN_NOT_OK(_init());
-        RETURN_NOT_OK(segment_group->link_segments_to_path(
-                _rowset_writer_context.path_desc.filepath, _rowset_writer_context.rowset_id));
-        _cur_segment_group->set_empty(segment_group->empty());
-        _cur_segment_group->set_num_segments(segment_group->num_segments());
-        _cur_segment_group->add_zone_maps_for_linked_schema_change(segment_group->get_zone_maps(),
-                                                                   schema_mapping);
-        RETURN_NOT_OK(flush());
-        _num_rows_written += segment_group->num_rows();
-    }
-    return Status::OK();
-}
-
-Status AlphaRowsetWriter::add_rowset_for_migration(RowsetSharedPtr rowset) {
-    LOG(WARNING) << "alpha_rowset_writer doesn't support add_rowset_for_migration";
-    return Status::NotSupported("alpha_rowset_writer doesn't support add_rowset_for_migration");
-}
-
-Status AlphaRowsetWriter::flush() {
-    if (_writer_state == WRITER_FLUSHED) {
-        return Status::OK();
-    }
-    DCHECK(_writer_state == WRITER_INITED);
-    if (_need_column_data_writer) {
-        // column_data_writer finalize will call segment_group->set_empty()
-        RETURN_NOT_OK(_column_data_writer->finalize());
-    }
-    SAFE_DELETE(_column_data_writer);
-    _writer_state = WRITER_FLUSHED;
-    return Status::OK();
-}
-
-RowsetSharedPtr AlphaRowsetWriter::build() {
-    if (_current_rowset_meta->rowset_id().version == 0) {
-        LOG(WARNING) << "invalid rowset id, version == 0, rowset id="
-                     << _current_rowset_meta->rowset_id().to_string();
-        return nullptr;
-    }
-    if (_writer_state != WRITER_FLUSHED) {
-        LOG(WARNING) << "invalid writer state before build, state:" << _writer_state;
-        return nullptr;
-    }
-    int total_num_segments = 0;
-    for (auto& segment_group : _segment_groups) {
-        if (segment_group->load() != Status::OK()) {
-            return nullptr;
-        }
-        if (!segment_group->check()) {
-            return nullptr;
-        }
-        _current_rowset_meta->set_data_disk_size(_current_rowset_meta->data_disk_size() +
-                                                 segment_group->data_size());
-        _current_rowset_meta->set_index_disk_size(_current_rowset_meta->index_disk_size() +
-                                                  segment_group->index_size());
-        _current_rowset_meta->set_total_disk_size(_current_rowset_meta->total_disk_size() +
-                                                  segment_group->index_size() +
-                                                  segment_group->data_size());
-        SegmentGroupPB segment_group_pb;
-        segment_group_pb.set_segment_group_id(segment_group->segment_group_id());
-        segment_group_pb.set_num_segments(segment_group->num_segments());
-        total_num_segments += segment_group->num_segments();
-        segment_group_pb.set_index_size(segment_group->index_size());
-        segment_group_pb.set_data_size(segment_group->data_size());
-        segment_group_pb.set_num_rows(segment_group->num_rows());
-        const std::vector<KeyRange>& zone_maps = segment_group->get_zone_maps();
-        if (!zone_maps.empty()) {
-            for (size_t i = 0; i < zone_maps.size(); ++i) {
-                ZoneMap* new_zone_map = segment_group_pb.add_zone_maps();
-                new_zone_map->set_min(zone_maps.at(i).first->to_string());
-                new_zone_map->set_max(zone_maps.at(i).second->to_string());
-                new_zone_map->set_null_flag(zone_maps.at(i).first->is_null());
-            }
-        }
-        if (_is_pending_rowset) {
-            PUniqueId* unique_id = segment_group_pb.mutable_load_id();
-            unique_id->set_hi(_rowset_writer_context.load_id.hi());
-            unique_id->set_lo(_rowset_writer_context.load_id.lo());
-        }
-        segment_group_pb.set_empty(segment_group->empty());
-        AlphaRowsetMetaSharedPtr alpha_rowset_meta =
-                std::dynamic_pointer_cast<AlphaRowsetMeta>(_current_rowset_meta);
-        alpha_rowset_meta->add_segment_group(segment_group_pb);
-    }
-    _current_rowset_meta->set_num_segments(total_num_segments);
-    if (total_num_segments <= 1) {
-        _current_rowset_meta->set_segments_overlap(NONOVERLAPPING);
-    }
-    if (_is_pending_rowset) {
-        _current_rowset_meta->set_rowset_state(COMMITTED);
-    } else {
-        _current_rowset_meta->set_rowset_state(VISIBLE);
-    }
-
-    _current_rowset_meta->set_empty(_num_rows_written == 0);
-    _current_rowset_meta->set_num_rows(_num_rows_written);
-    _current_rowset_meta->set_creation_time(time(nullptr));
-
-    // validate rowset arguments before create rowset
-    bool ret = _validate_rowset();
-    if (!ret) {
-        LOG(FATAL) << "validate rowset arguments failed";
-        return nullptr;
-    }
-
-    RowsetSharedPtr rowset;
-    auto status = RowsetFactory::create_rowset(_rowset_writer_context.tablet_schema,
-                                               _rowset_writer_context.path_desc,
-                                               _current_rowset_meta, &rowset);
-    if (!status.ok()) {
-        LOG(WARNING) << "rowset init failed when build new rowset, res=" << status;
-        return nullptr;
-    }
-    _rowset_build = true;
-    return rowset;
-}
-
-Status AlphaRowsetWriter::_garbage_collection() {
-    for (auto& segment_group : _segment_groups) {
-        bool ret = segment_group->delete_all_files();
-        if (!ret) {
-            LOG(WARNING) << "delete segment group files failed."
-                         << " tablet id:" << segment_group->get_tablet_id()
-                         << ", rowset path:" << segment_group->rowset_path_prefix();
-            return Status::OLAPInternalError(OLAP_ERR_ROWSET_DELETE_FILE_FAILED);
-        }
-    }
-    return Status::OK();
-}
-
-Status AlphaRowsetWriter::_init() {
-    if (_writer_state == WRITER_INITED) {
-        return Status::OK();
-    }
-    if (_is_pending_rowset) {
-        _cur_segment_group = new (std::nothrow) SegmentGroup(
-                _rowset_writer_context.tablet_id, _rowset_writer_context.rowset_id,
-                _rowset_writer_context.tablet_schema, _rowset_writer_context.path_desc.filepath,
-                false, _segment_group_id, 0, true, _rowset_writer_context.partition_id,
-                _rowset_writer_context.txn_id);
-    } else {
-        _cur_segment_group = new (std::nothrow) SegmentGroup(
-                _rowset_writer_context.tablet_id, _rowset_writer_context.rowset_id,
-                _rowset_writer_context.tablet_schema, _rowset_writer_context.path_desc.filepath,
-                _rowset_writer_context.version, false, _segment_group_id, 0);
-    }
-    DCHECK(_cur_segment_group != nullptr) << "failed to malloc SegmentGroup";
-    _cur_segment_group->acquire();
-    //_cur_segment_group->set_load_id(_rowset_writer_context.load_id);
-    _segment_groups.push_back(_cur_segment_group);
-
-    _column_data_writer = ColumnDataWriter::create(
-            _cur_segment_group, true, _rowset_writer_context.tablet_schema->compress_kind(),
-            _rowset_writer_context.tablet_schema->bloom_filter_fpp());
-    DCHECK(_column_data_writer != nullptr) << "memory error occurs when creating writer";
-    Status res = _column_data_writer->init();
-    if (!res.ok()) {
-        LOG(WARNING) << "column data writer init failed";
-        return res;
-    }
-
-    _segment_group_id++;
-    _writer_state = WRITER_INITED;
-    return Status::OK();
-}
-
-bool AlphaRowsetWriter::_validate_rowset() {
-    if (_is_pending_rowset) {
-        int64_t partition_id = _current_rowset_meta->partition_id();
-        if (partition_id <= 0) {
-            LOG(WARNING) << "invalid partition id:" << partition_id << " for pending rowset."
-                         << ", rowset_id:" << _current_rowset_meta->rowset_id()
-                         << ", tablet_id:" << _current_rowset_meta->tablet_id()
-                         << ", schema_hash:" << _current_rowset_meta->tablet_schema_hash();
-            return false;
-        }
-    }
-    int64_t num_rows = 0;
-    for (auto& segment_group : _segment_groups) {
-        num_rows += segment_group->num_rows();
-    }
-    if (num_rows != _current_rowset_meta->num_rows()) {
-        LOG(WARNING) << "num_rows between rowset and segment_groups do not match. "
-                     << "num_rows of segment_groups:" << num_rows
-                     << ", num_rows of rowset:" << _current_rowset_meta->num_rows();
-
-        return false;
-    }
-    return true;
-}
-
-} // namespace doris
diff --git a/be/src/olap/rowset/alpha_rowset_writer.h b/be/src/olap/rowset/alpha_rowset_writer.h
deleted file mode 100644
index 411beabf62..0000000000
--- a/be/src/olap/rowset/alpha_rowset_writer.h
+++ /dev/null
@@ -1,85 +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.
-
-#pragma once
-
-#include <vector>
-
-#include "olap/rowset/column_data_writer.h"
-#include "olap/rowset/rowset_writer.h"
-#include "olap/rowset/segment_group.h"
-
-namespace doris {
-
-enum WriterState { WRITER_CREATED, WRITER_INITED, WRITER_FLUSHED };
-
-class AlphaRowsetWriter : public RowsetWriter {
-public:
-    AlphaRowsetWriter();
-    ~AlphaRowsetWriter() override;
-
-    Status init(const RowsetWriterContext& rowset_writer_context) override;
-
-    Status add_row(const RowCursor& row) override { return _add_row(row); }
-    Status add_row(const ContiguousRow& row) override { return _add_row(row); }
-
-    // add rowset by create hard link
-    Status add_rowset(RowsetSharedPtr rowset) override;
-    Status add_rowset_for_linked_schema_change(RowsetSharedPtr rowset,
-                                               const SchemaMapping& schema_mapping) override;
-
-    Status add_rowset_for_migration(RowsetSharedPtr rowset) override;
-    Status flush() override;
-
-    // get a rowset
-    RowsetSharedPtr build() override;
-
-    Version version() override { return _rowset_writer_context.version; }
-
-    int64_t num_rows() const override { return _num_rows_written; }
-
-    RowsetId rowset_id() override { return _rowset_writer_context.rowset_id; }
-
-    RowsetTypePB type() const override { return RowsetTypePB::ALPHA_ROWSET; }
-
-private:
-    Status _init();
-
-    template <typename RowType>
-    Status _add_row(const RowType& row);
-
-    // validate rowset build arguments before create rowset to make sure correctness
-    bool _validate_rowset();
-
-    Status _garbage_collection();
-
-private:
-    int32_t _segment_group_id;
-    SegmentGroup* _cur_segment_group;
-    ColumnDataWriter* _column_data_writer;
-    std::shared_ptr<RowsetMeta> _current_rowset_meta;
-    bool _is_pending_rowset;
-    int64_t _num_rows_written;
-    RowsetWriterContext _rowset_writer_context;
-    std::vector<SegmentGroup*> _segment_groups;
-    bool _rowset_build;
-    WriterState _writer_state;
-    // add_rowset does not need to call column_data_writer.finalize()
-    bool _need_column_data_writer;
-};
-
-} // namespace doris
diff --git a/be/src/olap/rowset/bit_field_reader.cpp b/be/src/olap/rowset/bit_field_reader.cpp
deleted file mode 100644
index 5fbcad3015..0000000000
--- a/be/src/olap/rowset/bit_field_reader.cpp
+++ /dev/null
@@ -1,127 +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 "olap/rowset/bit_field_reader.h"
-
-#include "olap/in_stream.h"
-#include "olap/rowset/column_reader.h"
-#include "olap/rowset/run_length_byte_reader.h"
-
-namespace doris {
-
-BitFieldReader::BitFieldReader(ReadOnlyFileStream* input)
-        : _input(input), _byte_reader(nullptr), _current('\0'), _bits_left(0) {}
-
-BitFieldReader::~BitFieldReader() {
-    SAFE_DELETE(_byte_reader);
-}
-
-Status BitFieldReader::init() {
-    if (nullptr == _byte_reader) {
-        _byte_reader = new (std::nothrow) RunLengthByteReader(_input);
-
-        if (nullptr == _byte_reader) {
-            LOG(WARNING) << "fail to create RunLengthByteReader";
-            return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
-        }
-    }
-
-    return Status::OK();
-}
-
-Status BitFieldReader::_read_byte() {
-    Status res = Status::OK();
-
-    if (_byte_reader->has_next()) {
-        if (!(res = _byte_reader->next(&_current))) {
-            return res;
-        }
-
-        _bits_left = 8;
-    } else {
-        return Status::OLAPInternalError(OLAP_ERR_DATA_EOF);
-    }
-
-    return Status::OK();
-}
-
-Status BitFieldReader::next(char* value) {
-    Status res = Status::OK();
-
-    if (0 == _bits_left) {
-        if (!(res = _read_byte())) {
-            return res;
-        }
-    }
-
-    --_bits_left;
-
-    *value = (_current >> _bits_left) & 0x01;
-
-    return Status::OK();
-}
-
-Status BitFieldReader::seek(PositionProvider* position) {
-    Status res = Status::OK();
-
-    if (!(res = _byte_reader->seek(position))) {
-        return res;
-    }
-
-    int64_t consumed = position->get_next();
-
-    if (consumed > 8) {
-        LOG(WARNING) << "read past end of bit field";
-        return Status::OLAPInternalError(OLAP_ERR_DATA_EOF);
-    } else if (consumed != 0) {
-        if (!(res = _read_byte())) {
-            return res;
-        }
-
-        _bits_left = 8 - consumed;
-    } else {
-        _bits_left = 0;
-    }
-
-    return Status::OK();
-}
-
-Status BitFieldReader::skip(uint64_t num_values) {
-    Status res = Status::OK();
-
-    uint64_t total_bits = num_values;
-
-    if (_bits_left >= total_bits) {
-        _bits_left -= total_bits;
-    } else {
-        total_bits -= _bits_left;
-
-        if (!(res = _byte_reader->skip(total_bits / 8))) {
-            return res;
-        }
-
-        if (!(res = _byte_reader->next(&_current))) {
-            return res;
-        }
-
-        _bits_left = 8 - (total_bits % 8);
-    }
-
-    return Status::OK();
-}
-
-} // namespace doris
\ No newline at end of file
diff --git a/be/src/olap/rowset/bit_field_reader.h b/be/src/olap/rowset/bit_field_reader.h
deleted file mode 100644
index 6498ae3826..0000000000
--- a/be/src/olap/rowset/bit_field_reader.h
+++ /dev/null
@@ -1,53 +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.
-
-#ifndef DORIS_BE_SRC_OLAP_ROWSET_BIT_FIELD_READER_H
-#define DORIS_BE_SRC_OLAP_ROWSET_BIT_FIELD_READER_H
-
-#include "olap/olap_define.h"
-#include "olap/stream_index_reader.h"
-
-namespace doris {
-
-class ReadOnlyFileStream;
-class RunLengthByteReader;
-
-class BitFieldReader {
-public:
-    BitFieldReader(ReadOnlyFileStream* input);
-    ~BitFieldReader();
-    Status init();
-    // 获取下一条数据, 如果没有更多的数据了, 返回Status::OLAPInternalError(OLAP_ERR_DATA_EOF)
-    // 返回的value只可能是0或1
-    Status next(char* value);
-    Status seek(PositionProvider* position);
-    Status skip(uint64_t num_values);
-
-private:
-    Status _read_byte();
-
-    ReadOnlyFileStream* _input;
-    RunLengthByteReader* _byte_reader;
-    char _current;
-    uint32_t _bits_left;
-
-    DISALLOW_COPY_AND_ASSIGN(BitFieldReader);
-};
-
-} // namespace doris
-
-#endif // DORIS_BE_SRC_OLAP_ROWSET_BIT_FIELD_READER_H
\ No newline at end of file
diff --git a/be/src/olap/rowset/bit_field_writer.cpp b/be/src/olap/rowset/bit_field_writer.cpp
deleted file mode 100644
index 5dc4553c3a..0000000000
--- a/be/src/olap/rowset/bit_field_writer.cpp
+++ /dev/null
@@ -1,99 +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 "olap/rowset/bit_field_writer.h"
-
-#include <gen_cpp/column_data_file.pb.h>
-
-#include "olap/rowset/run_length_byte_writer.h"
-
-namespace doris {
-
-BitFieldWriter::BitFieldWriter(OutStream* output)
-        : _output(output), _byte_writer(nullptr), _current(0), _bits_left(8) {}
-
-BitFieldWriter::~BitFieldWriter() {
-    SAFE_DELETE(_byte_writer);
-}
-
-Status BitFieldWriter::init() {
-    _byte_writer = new (std::nothrow) RunLengthByteWriter(_output);
-
-    if (nullptr == _byte_writer) {
-        LOG(WARNING) << "fail to create RunLengthByteWriter";
-        return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
-    }
-
-    return Status::OK();
-}
-
-Status BitFieldWriter::_write_byte() {
-    Status res = Status::OK();
-
-    if (!(res = _byte_writer->write(_current))) {
-        LOG(WARNING) << "fail to write byte to byte writer";
-        return res;
-    }
-
-    _current = 0;
-    _bits_left = 8;
-    return res;
-}
-
-Status BitFieldWriter::write(bool bit_value) {
-    Status res = Status::OK();
-
-    _bits_left--;
-
-    if (bit_value) {
-        _current |= 1 << _bits_left;
-    }
-
-    if (_bits_left == 0) {
-        res = _write_byte();
-    }
-
-    return res;
-}
-
-Status BitFieldWriter::flush() {
-    Status res = Status::OK();
-
-    if (_bits_left != 8) {
-        if (!(res = _write_byte())) {
-            return res;
-        }
-    }
-
-    return _byte_writer->flush();
-}
-
-void BitFieldWriter::get_position(PositionEntryWriter* index_entry) const {
-    if (nullptr != _byte_writer) {
-        _byte_writer->get_position(index_entry);
-    } else {
-        // for stream
-        index_entry->add_position(0);
-        index_entry->add_position(0);
-        // for rle byte writer
-        index_entry->add_position(0);
-    }
-
-    index_entry->add_position(8 - _bits_left);
-}
-
-} // namespace doris
\ No newline at end of file
diff --git a/be/src/olap/rowset/bit_field_writer.h b/be/src/olap/rowset/bit_field_writer.h
deleted file mode 100644
index 87480c51ab..0000000000
--- a/be/src/olap/rowset/bit_field_writer.h
+++ /dev/null
@@ -1,52 +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.
-
-#ifndef DORIS_BE_SRC_OLAP_ROWSET_BIT_FIELD_WRITER_H
-#define DORIS_BE_SRC_OLAP_ROWSET_BIT_FIELD_WRITER_H
-
-#include "olap/olap_define.h"
-#include "olap/stream_index_writer.h"
-
-namespace doris {
-
-class OutStream;
-class RunLengthByteWriter;
-
-class BitFieldWriter {
-public:
-    explicit BitFieldWriter(OutStream* output);
-    ~BitFieldWriter();
-    Status init();
-    // Write a bit, bit_value is true means writing 1, false means writing 0
-    Status write(bool bit_value);
-    Status flush();
-    void get_position(PositionEntryWriter* index_entry) const;
-
-private:
-    Status _write_byte();
-
-    OutStream* _output;
-    RunLengthByteWriter* _byte_writer;
-    char _current;
-    uint8_t _bits_left;
-
-    DISALLOW_COPY_AND_ASSIGN(BitFieldWriter);
-};
-
-} // namespace doris
-
-#endif // DORIS_BE_SRC_OLAP_ROWSET_BIT_FIELD_WRITER_H
\ No newline at end of file
diff --git a/be/src/olap/rowset/column_data.cpp b/be/src/olap/rowset/column_data.cpp
deleted file mode 100644
index 889581d8a9..0000000000
--- a/be/src/olap/rowset/column_data.cpp
+++ /dev/null
@@ -1,661 +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 "olap/rowset/column_data.h"
-
-#include "olap/olap_cond.h"
-#include "olap/row_block.h"
-#include "olap/rowset/segment_reader.h"
-#include "olap/storage_engine.h"
-
-namespace doris {
-
-ColumnData* ColumnData::create(SegmentGroup* segment_group) {
-    ColumnData* data = new (std::nothrow) ColumnData(segment_group);
-    return data;
-}
-
-ColumnData::ColumnData(SegmentGroup* segment_group)
-        : _segment_group(segment_group),
-          _eof(false),
-          _conditions(nullptr),
-          _col_predicates(nullptr),
-          _delete_status(DEL_NOT_SATISFIED),
-          _runtime_state(nullptr),
-          _schema(segment_group->get_tablet_schema()),
-          _is_using_cache(false),
-          _segment_reader(nullptr),
-          _lru_cache(nullptr) {
-    if (StorageEngine::instance() != nullptr) {
-        _lru_cache = StorageEngine::instance()->index_stream_lru_cache();
-    } else {
-        // for independent usage, eg: unit test/segment tool
-        _lru_cache = FileHandler::get_fd_cache();
-    }
-    _num_rows_per_block = _segment_group->get_num_rows_per_row_block();
-}
-
-ColumnData::~ColumnData() {
-    _segment_group->release();
-    SAFE_DELETE(_segment_reader);
-}
-
-Status ColumnData::init() {
-    _segment_group->acquire();
-
-    auto res = _short_key_cursor.init(_segment_group->short_key_columns());
-    if (!res.ok()) {
-        LOG(WARNING) << "key cursor init failed, res:" << res;
-        return res;
-    }
-    return res;
-}
-
-Status ColumnData::get_next_block(RowBlock** row_block) {
-    SCOPED_RAW_TIMER(&_stats->block_fetch_ns);
-    _is_normal_read = true;
-    auto res = _get_block(false);
-    if (!res.ok()) {
-        if (res.precise_code() != OLAP_ERR_DATA_EOF) {
-            LOG(WARNING) << "Get next block failed.";
-        }
-        *row_block = nullptr;
-        return res;
-    }
-    *row_block = _read_block.get();
-    return Status::OK();
-}
-
-Status ColumnData::_next_row(const RowCursor** row, bool without_filter) {
-    _read_block->pos_inc();
-    do {
-        if (_read_block->has_remaining()) {
-            // 1. get one row for vectorized_row_batch
-            size_t pos = _read_block->pos();
-            _read_block->get_row(pos, &_cursor);
-            if (without_filter) {
-                *row = &_cursor;
-                return Status::OK();
-            }
-
-            // when without_filter is true, _include_blocks is nullptr
-            if (_read_block->block_status() == DEL_NOT_SATISFIED) {
-                *row = &_cursor;
-                return Status::OK();
-            } else {
-                DCHECK(_read_block->block_status() == DEL_PARTIAL_SATISFIED);
-                bool row_del_filter =
-                        _delete_handler->is_filter_data(_segment_group->version().second, _cursor);
-                if (!row_del_filter) {
-                    *row = &_cursor;
-                    return Status::OK();
-                }
-                // This row is filtered, continue to process next row
-                _stats->rows_del_filtered++;
-                _read_block->pos_inc();
-            }
-        } else {
-            // get_next_block
-            auto res = _get_block(without_filter);
-            if (!res.ok()) {
-                return res;
-            }
-        }
-    } while (true);
-
-    return Status::OK();
-}
-
-Status ColumnData::_seek_to_block(const RowBlockPosition& block_pos, bool without_filter) {
-    // TODO(zc): _segment_readers???
-    // open segment reader if needed
-    if (_segment_reader == nullptr || block_pos.segment != _current_segment) {
-        if (block_pos.segment >= _segment_group->num_segments() ||
-            (_end_key_is_set && block_pos.segment > _end_segment)) {
-            _eof = true;
-            return Status::OLAPInternalError(OLAP_ERR_DATA_EOF);
-        }
-        SAFE_DELETE(_segment_reader);
-        std::string file_name;
-        file_name = segment_group()->construct_data_file_path(block_pos.segment);
-        _segment_reader = new (std::nothrow) SegmentReader(
-                file_name, segment_group(), block_pos.segment, _seek_columns, _load_bf_columns,
-                _conditions, _delete_handler, _delete_status, _lru_cache, _runtime_state, _stats);
-        if (_segment_reader == nullptr) {
-            LOG(WARNING) << "fail to malloc segment reader.";
-            return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
-        }
-
-        _current_segment = block_pos.segment;
-        auto res = _segment_reader->init(_is_using_cache);
-        if (!res.ok()) {
-            LOG(WARNING) << "fail to init segment reader. res = " << res;
-            return res;
-        }
-    }
-
-    uint32_t end_block;
-    if (_end_key_is_set && block_pos.segment == _end_segment) {
-        end_block = _end_block;
-    } else {
-        end_block = _segment_reader->block_count() - 1;
-    }
-
-    VLOG_NOTICE << "seek from " << block_pos.data_offset << " to " << end_block;
-    return _segment_reader->seek_to_block(block_pos.data_offset, end_block, without_filter,
-                                          &_next_block, &_segment_eof);
-}
-
-Status ColumnData::_find_position_by_short_key(const RowCursor& key, bool find_last_key,
-                                               RowBlockPosition* position) {
-    RowBlockPosition tmp_pos;
-    auto res = _segment_group->find_short_key(key, &_short_key_cursor, find_last_key, &tmp_pos);
-    if (!res.ok()) {
-        if (res.precise_code() == OLAP_ERR_INDEX_EOF) {
-            res = Status::OLAPInternalError(OLAP_ERR_DATA_EOF);
-        } else {
-            LOG(WARNING) << "find row block failed. res = " << res;
-        }
-        return res;
-    }
-    res = segment_group()->find_prev_point(tmp_pos, position);
-    if (!res.ok()) {
-        LOG(WARNING) << "find prev row block failed. res = " << res;
-        return res;
-    }
-    return Status::OK();
-}
-
-Status ColumnData::_find_position_by_full_key(const RowCursor& key, bool find_last_key,
-                                              RowBlockPosition* position) {
-    RowBlockPosition tmp_pos;
-    auto res = _segment_group->find_short_key(key, &_short_key_cursor, false, &tmp_pos);
-    if (!res.ok()) {
-        if (res.precise_code() == OLAP_ERR_INDEX_EOF) {
-            res = Status::OLAPInternalError(OLAP_ERR_DATA_EOF);
-        } else {
-            LOG(WARNING) << "find row block failed. res = " << res;
-        }
-        return res;
-    }
-    RowBlockPosition start_position;
-    res = segment_group()->find_prev_point(tmp_pos, &start_position);
-    if (!res.ok()) {
-        LOG(WARNING) << "find prev row block failed. res = " << res;
-        return res;
-    }
-
-    RowBlockPosition end_position;
-    res = _segment_group->find_short_key(key, &_short_key_cursor, true, &end_position);
-    if (!res.ok()) {
-        if (res.precise_code() == OLAP_ERR_INDEX_EOF) {
-            res = Status::OLAPInternalError(OLAP_ERR_DATA_EOF);
-        } else {
-            LOG(WARNING) << "find row block failed. res = " << res;
-        }
-        return res;
-    }
-
-    // choose min value of end_position and m_end_key_block_position as real end_position
-    if (_end_key_is_set) {
-        RowBlockPosition end_key_position;
-        end_key_position.segment = _end_segment;
-        end_key_position.data_offset = _end_block;
-        if (end_position > end_key_position) {
-            OLAPIndexOffset index_offset;
-            index_offset.segment = _end_segment;
-            index_offset.offset = _end_block;
-            res = segment_group()->get_row_block_position(index_offset, &end_position);
-            if (!res.ok()) {
-                LOG(WARNING) << "fail to get row block position. res = " << res;
-                return res;
-            }
-        }
-    }
-
-    // ????end_position
-    uint32_t distance = segment_group()->compute_distance(start_position, end_position);
-
-    BinarySearchIterator it_start(0u);
-    BinarySearchIterator it_end(distance + 1);
-    BinarySearchIterator it_result(0u);
-    ColumnDataComparator comparator(start_position, this, segment_group());
-    try {
-        if (!find_last_key) {
-            it_result = std::lower_bound(it_start, it_end, key, comparator);
-        } else {
-            it_result = std::upper_bound(it_start, it_end, key, comparator);
-        }
-        VLOG_NOTICE << "get result iterator. offset=" << *it_result
-                    << ", start_pos=" << start_position.to_string();
-    } catch (std::exception& e) {
-        LOG(WARNING) << "exception happens when doing seek. exception=" << e.what();
-        return Status::OLAPInternalError(OLAP_ERR_STL_ERROR);
-    }
-
-    if (*it_result != *it_start) {
-        it_result -= 1;
-    }
-
-    if (!(res = segment_group()->advance_row_block(*it_result, &start_position))) {
-        LOG(WARNING) << "fail to advance row_block. res=" << res << " it_offset=" << *it_result
-                     << " start_pos=" << start_position.to_string();
-        return res;
-    }
-
-    if (_end_key_is_set) {
-        RowBlockPosition end_key_position;
-        end_key_position.segment = _end_segment;
-        end_key_position.data_offset = _end_block;
-        if (end_position > end_key_position) {
-            return Status::OLAPInternalError(OLAP_ERR_DATA_EOF);
-        }
-    }
-
-    *position = start_position;
-    return Status::OK();
-}
-
-Status ColumnData::_seek_to_row(const RowCursor& key, bool find_last_key, bool is_end_key) {
-    RowBlockPosition position;
-    Status res = Status::OK();
-    const TabletSchema& tablet_schema = _segment_group->get_tablet_schema();
-    FieldType type = tablet_schema.column(key.field_count() - 1).type();
-    if (key.field_count() > _segment_group->get_num_short_key_columns() ||
-        OLAP_FIELD_TYPE_VARCHAR == type || OLAP_FIELD_TYPE_STRING == type) {
-        res = _find_position_by_full_key(key, find_last_key, &position);
-    } else {
-        res = _find_position_by_short_key(key, find_last_key, &position);
-    }
-    if (!res.ok()) {
-        if (res.precise_code() != OLAP_ERR_DATA_EOF) {
-            LOG(WARNING) << "Fail to find the key.[res=" << res << " key=" << key.to_string()
-                         << " find_last_key=" << find_last_key << "]";
-        }
-        return res;
-    }
-    bool without_filter = is_end_key;
-    res = _seek_to_block(position, without_filter);
-    if (!res.ok()) {
-        LOG(WARNING) << "fail to get row block. res=" << res << " segment=" << position.segment
-                     << " block_size=" << position.block_size
-                     << " data_offset=" << position.data_offset
-                     << " index_offset=" << position.index_offset;
-        return res;
-    }
-    res = _get_block(without_filter);
-    if (!res.ok()) {
-        if (res.precise_code() != OLAP_ERR_DATA_EOF) {
-            LOG(WARNING) << "Fail to find the key.[res=" << res << " key=" << key.to_string()
-                         << " find_last_key=" << find_last_key << "]";
-        }
-        return res;
-    }
-
-    const RowCursor* row_cursor = _current_row();
-    if (!find_last_key) {
-        // 不找last key。 那么应该返回大于等于这个key的第一个,也就是
-        // row_cursor >= key
-        // 此处比较2个block的行数,是存在一种极限情况:若未找到满足的block,
-        // Index模块会返回倒数第二个block,此时key可能是最后一个block的最后一行
-        while (res.ok() && compare_row_key(*row_cursor, key) < 0) {
-            res = _next_row(&row_cursor, without_filter);
-        }
-    } else {
-        // 找last key。返回大于这个key的第一个。也就是
-        // row_cursor > key
-        while (res.ok() && compare_row_key(*row_cursor, key) <= 0) {
-            res = _next_row(&row_cursor, without_filter);
-        }
-    }
-
-    return res;
-}
-
-const RowCursor* ColumnData::seek_and_get_current_row(const RowBlockPosition& position) {
-    auto res = _seek_to_block(position, true);
-    if (!res.ok()) {
-        LOG(WARNING) << "Fail to seek to block in seek_and_get_current_row, res=" << res
-                     << ", segment:" << position.segment << ", block:" << position.data_offset;
-        return nullptr;
-    }
-    res = _get_block(true, 1);
-    if (!res.ok()) {
-        LOG(WARNING) << "Fail to get block in seek_and_get_current_row, res=" << res
-                     << ", segment:" << position.segment << ", block:" << position.data_offset
-                     << ", tablet: " << _segment_group->get_tablet_id();
-        return nullptr;
-    }
-    return _current_row();
-}
-
-Status ColumnData::prepare_block_read(const RowCursor* start_key, bool find_start_key,
-                                      const RowCursor* end_key, bool find_end_key,
-                                      RowBlock** first_block) {
-    SCOPED_RAW_TIMER(&_stats->block_fetch_ns);
-    set_eof(false);
-    _end_key_is_set = false;
-    _is_normal_read = false;
-    // set end position
-    if (end_key != nullptr) {
-        auto res = _seek_to_row(*end_key, find_end_key, true);
-        if (res.ok()) {
-            // we find a
-            _end_segment = _current_segment;
-            _end_block = _current_block;
-            _end_row_index = _read_block->pos();
-            _end_key_is_set = true;
-        } else if (res.precise_code() != OLAP_ERR_DATA_EOF) {
-            LOG(WARNING) << "Find end key failed.key=" << end_key->to_string();
-            return res;
-        }
-        // res.precise_code() == OLAP_ERR_DATA_EOF means there is no end key, then we read to
-        // the end of this ColumnData
-    }
-    set_eof(false);
-    if (start_key != nullptr) {
-        auto res = _seek_to_row(*start_key, !find_start_key, false);
-        if (res.ok()) {
-            *first_block = _read_block.get();
-        } else if (res.precise_code() == OLAP_ERR_DATA_EOF) {
-            _eof = true;
-            *first_block = nullptr;
-            return res;
-        } else {
-            LOG(WARNING) << "start_key can't be found.key=" << start_key->to_string();
-            return res;
-        }
-    } else {
-        // This is used to
-        _is_normal_read = true;
-
-        RowBlockPosition pos;
-        pos.segment = 0u;
-        pos.data_offset = 0u;
-        auto res = _seek_to_block(pos, false);
-        if (!res.ok()) {
-            LOG(WARNING) << "failed to seek to block in, res=" << res << ", segment:" << pos.segment
-                         << ", block:" << pos.data_offset;
-            return res;
-        }
-        res = _get_block(false);
-        if (!res.ok()) {
-            LOG(WARNING) << "failed to get block in , res=" << res << ", segment:" << pos.segment
-                         << ", block:" << pos.data_offset;
-            return res;
-        }
-        *first_block = _read_block.get();
-    }
-    return Status::OK();
-}
-
-// ColumnData向上返回的列至少由几部分组成:
-// 1. return_columns中要求返回的列,即Fetch命令中指定要查询的列.
-// 2. condition中涉及的列, 绝大多数情况下这些列都已经在return_columns中.
-// 在这个函数里,合并上述几种情况
-void ColumnData::set_read_params(const std::vector<uint32_t>& return_columns,
-                                 const std::vector<uint32_t>& seek_columns,
-                                 const std::set<uint32_t>& load_bf_columns,
-                                 const Conditions& conditions,
-                                 std::shared_ptr<std::vector<ColumnPredicate*>> col_predicates,
-                                 bool is_using_cache, RuntimeState* runtime_state) {
-    _conditions = &conditions;
-    _col_predicates = col_predicates;
-    _need_eval_predicates = !col_predicates->empty();
-    _is_using_cache = is_using_cache;
-    _runtime_state = runtime_state;
-    _return_columns = return_columns;
-    _seek_columns = seek_columns;
-    _load_bf_columns = load_bf_columns;
-
-    auto res = _cursor.init(_segment_group->get_tablet_schema(), _seek_columns);
-    if (!res.ok()) {
-        LOG(WARNING) << "fail to init row_cursor";
-    }
-
-    _read_vector_batch.reset(new VectorizedRowBatch(&(_segment_group->get_tablet_schema()),
-                                                    _return_columns, _num_rows_per_block));
-
-    _seek_vector_batch.reset(new VectorizedRowBatch(&(_segment_group->get_tablet_schema()),
-                                                    _seek_columns, _num_rows_per_block));
-
-    _read_block.reset(new RowBlock(&(_segment_group->get_tablet_schema())));
-    RowBlockInfo block_info;
-    block_info.row_num = _num_rows_per_block;
-    block_info.null_supported = true;
-    block_info.column_ids = _seek_columns;
-    _read_block->init(block_info);
-}
-
-Status ColumnData::get_first_row_block(RowBlock** row_block) {
-    DCHECK(!_end_key_is_set) << "end key is set while use block interface.";
-    _is_normal_read = true;
-    _eof = false;
-
-    // to be same with OLAPData, we use segment_group.
-    RowBlockPosition block_pos;
-    Status res = segment_group()->find_first_row_block(&block_pos);
-    if (!res.ok()) {
-        if (res.precise_code() == OLAP_ERR_INDEX_EOF) {
-            *row_block = nullptr;
-            _eof = true;
-            return res;
-        }
-        LOG(WARNING) << "fail to find first row block with SegmentGroup.";
-        return res;
-    }
-
-    res = _seek_to_block(block_pos, false);
-    if (!res.ok()) {
-        if (res.precise_code() != OLAP_ERR_DATA_EOF) {
-            LOG(WARNING) << "seek to block fail. res = " << res;
-        }
-        *row_block = nullptr;
-        return res;
-    }
-
-    res = _get_block(false);
-    if (!res.ok()) {
-        if (res.precise_code() != OLAP_ERR_DATA_EOF) {
-            LOG(WARNING) << "fail to load data to row block. res=" << res
-                         << ", version=" << version().first << "-" << version().second;
-        }
-        *row_block = nullptr;
-        return res;
-    }
-
-    *row_block = _read_block.get();
-    return Status::OK();
-}
-
-bool ColumnData::rowset_pruning_filter() {
-    if (empty() || zero_num_rows()) {
-        return true;
-    }
-
-    if (!_segment_group->has_zone_maps()) {
-        return false;
-    }
-
-    return _conditions->rowset_pruning_filter(_segment_group->get_zone_maps());
-}
-
-int ColumnData::delete_pruning_filter() {
-    if (empty() || zero_num_rows()) {
-        // should return DEL_NOT_SATISFIED, because that when creating rollup tablet,
-        // the delete version file should preserved for filter data.
-        return DEL_NOT_SATISFIED;
-    }
-
-    int num_zone_maps = _schema.keys_type() == KeysType::DUP_KEYS ? _schema.num_columns()
-                                                                  : _schema.num_key_columns();
-    // _segment_group->get_zone_maps().size() < num_zone_maps for a table is schema changed from older version that not support
-    // generate zone map for duplicated mode value column, using DEL_PARTIAL_SATISFIED
-    if (!_segment_group->has_zone_maps() ||
-        _segment_group->get_zone_maps().size() < num_zone_maps) {
-        /*
-         * if segment_group has no column statistics, we cannot judge whether the data can be filtered or not
-         */
-        return DEL_PARTIAL_SATISFIED;
-    }
-
-    /*
-     * the relationship between delete condition A and B is A || B.
-     * if any delete condition is satisfied, the data can be filtered.
-     * elseif all delete condition is not satisfied, the data can't be filtered.
-     * else is the partial satisfied.
-    */
-    int ret = DEL_PARTIAL_SATISFIED;
-    bool del_partial_satisfied = false;
-    bool del_satisfied = false;
-    for (auto& delete_condition : _delete_handler->get_delete_conditions()) {
-        if (delete_condition.filter_version <= _segment_group->version().first) {
-            continue;
-        }
-
-        Conditions* del_cond = delete_condition.del_cond;
-        int del_ret = del_cond->delete_pruning_filter(_segment_group->get_zone_maps());
-        if (DEL_SATISFIED == del_ret) {
-            del_satisfied = true;
-            break;
-        } else if (DEL_PARTIAL_SATISFIED == del_ret) {
-            del_partial_satisfied = true;
-        } else {
-            continue;
-        }
-    }
-
-    if (del_satisfied) {
-        ret = DEL_SATISFIED;
-    } else if (del_partial_satisfied) {
-        ret = DEL_PARTIAL_SATISFIED;
-    } else {
-        ret = DEL_NOT_SATISFIED;
-    }
-
-    return ret;
-}
-
-uint64_t ColumnData::get_filtered_rows() {
-    return _stats->rows_del_filtered;
-}
-
-Status ColumnData::schema_change_init() {
-    _is_using_cache = false;
-
-    for (int i = 0; i < _segment_group->get_tablet_schema().num_columns(); ++i) {
-        _return_columns.push_back(i);
-        _seek_columns.push_back(i);
-    }
-
-    auto res = _cursor.init(_segment_group->get_tablet_schema());
-    if (!res.ok()) {
-        LOG(WARNING) << "fail to init row_cursor";
-        return res;
-    }
-
-    _read_vector_batch.reset(new VectorizedRowBatch(&(_segment_group->get_tablet_schema()),
-                                                    _return_columns, _num_rows_per_block));
-
-    _read_block.reset(new RowBlock(&(_segment_group->get_tablet_schema())));
-
-    RowBlockInfo block_info;
-    block_info.row_num = _num_rows_per_block;
-    block_info.null_supported = true;
-    _read_block->init(block_info);
-    return Status::OK();
-}
-
-Status ColumnData::_get_block_from_reader(VectorizedRowBatch** got_batch, bool without_filter,
-                                          int rows_read) {
-    VectorizedRowBatch* vec_batch = nullptr;
-    if (_is_normal_read) {
-        vec_batch = _read_vector_batch.get();
-    } else {
-        vec_batch = _seek_vector_batch.get();
-    }
-    // If this is normal read
-    do {
-        vec_batch->clear();
-        if (rows_read > 0) {
-            vec_batch->set_limit(rows_read);
-        }
-        // If we are going to read last block, we need to set batch limit to the end of key
-        // if without_filter is true and _end_key_is_set is true, this must seek to start row's
-        // block, we must load the entire block.
-        if (OLAP_UNLIKELY(!without_filter && _end_key_is_set && _next_block == _end_block &&
-                          _current_segment == _end_segment)) {
-            vec_batch->set_limit(_end_row_index);
-            if (_end_row_index == 0) {
-                _segment_eof = true;
-            }
-        }
-
-        if (!_segment_eof) {
-            _current_block = _next_block;
-            auto res = _segment_reader->get_block(vec_batch, &_next_block, &_segment_eof);
-            if (!res.ok()) {
-                return res;
-            }
-            // Normal case
-            *got_batch = vec_batch;
-            return Status::OK();
-        }
-        // When this segment is read over, we reach here.
-        // Seek to next segment
-        RowBlockPosition block_pos;
-        block_pos.segment = _current_segment + 1;
-        block_pos.data_offset = 0;
-        auto res = _seek_to_block(block_pos, without_filter);
-        if (!res.ok()) {
-            return res;
-        }
-    } while (true);
-
-    return Status::OK();
-}
-
-Status ColumnData::_get_block(bool without_filter, int rows_read) {
-    do {
-        VectorizedRowBatch* vec_batch = nullptr;
-        auto res = _get_block_from_reader(&vec_batch, without_filter, rows_read);
-        if (!res.ok()) {
-            return res;
-        }
-        // evaluate predicates
-        if (!without_filter && _need_eval_predicates) {
-            SCOPED_RAW_TIMER(&_stats->vec_cond_ns);
-            size_t old_size = vec_batch->size();
-            for (auto pred : *_col_predicates) {
-                pred->evaluate(vec_batch);
-            }
-            _stats->rows_vec_cond_filtered += old_size - vec_batch->size();
-        }
-        // if vector is empty after predicate evaluate, get next block
-        if (vec_batch->size() == 0) {
-            continue;
-        }
-        SCOPED_RAW_TIMER(&_stats->block_convert_ns);
-        // when reach here, we have already read a block successfully
-        _read_block->clear();
-        vec_batch->dump_to_row_block(_read_block.get());
-        return Status::OK();
-    } while (true);
-    return Status::OK();
-}
-
-} // namespace doris
diff --git a/be/src/olap/rowset/column_data.h b/be/src/olap/rowset/column_data.h
deleted file mode 100644
index c2c91fab79..0000000000
--- a/be/src/olap/rowset/column_data.h
+++ /dev/null
@@ -1,230 +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.
-
-#ifndef DORIS_BE_SRC_OLAP_ROWSET_COLUMN_DATA_H
-#define DORIS_BE_SRC_OLAP_ROWSET_COLUMN_DATA_H
-
-#include <string>
-
-#include "gen_cpp/olap_file.pb.h"
-#include "olap/column_predicate.h"
-#include "olap/delete_handler.h"
-#include "olap/olap_common.h"
-#include "olap/olap_cond.h"
-#include "olap/row.h"
-#include "olap/row_block.h"
-#include "olap/row_cursor.h"
-#include "olap/rowset/segment_group.h"
-#include "util/runtime_profile.h"
-
-namespace doris {
-
-class Tablet;
-class SegmentReader;
-
-// This class is column data reader. this class will be used in two case.
-class ColumnData {
-public:
-    static ColumnData* create(SegmentGroup* segment_group);
-    ColumnData(SegmentGroup* segment_group);
-    ~ColumnData();
-
-    // 为了与之前兼容, 暴露部分index的接口
-    Version version() const { return _segment_group->version(); }
-    bool delete_flag() const { return _segment_group->delete_flag(); }
-    uint32_t num_segments() const { return _segment_group->num_segments(); }
-
-    Status init();
-
-    Status prepare_block_read(const RowCursor* start_key, bool find_start_key,
-                              const RowCursor* end_key, bool find_end_key, RowBlock** first_block);
-
-    Status get_next_block(RowBlock** row_block);
-
-    void set_read_params(const std::vector<uint32_t>& return_columns,
-                         const std::vector<uint32_t>& seek_columns,
-                         const std::set<uint32_t>& load_bf_columns, const Conditions& conditions,
-                         std::shared_ptr<std::vector<ColumnPredicate*>> col_predicates,
-                         bool is_using_cache, RuntimeState* runtime_state);
-
-    Status get_first_row_block(RowBlock** row_block);
-
-    // Only used to binary search in full-key find row
-    const RowCursor* seek_and_get_current_row(const RowBlockPosition& position);
-
-    void set_using_cache(bool is_using_cache) { _is_using_cache = is_using_cache; }
-
-    void set_stats(OlapReaderStatistics* stats) { _stats = stats; }
-
-    void set_delete_handler(const DeleteHandler* delete_handler) {
-        _delete_handler = delete_handler;
-    }
-
-    void set_delete_status(const DelCondSatisfied delete_status) { _delete_status = delete_status; }
-
-    // 开放接口查询_eof,让外界知道数据读取是否正常终止
-    // 因为这个函数被频繁访问, 从性能考虑, 放在基类而不是虚函数
-    bool eof() { return _eof; }
-    void set_eof(bool eof) { _eof = eof; }
-    bool* eof_ptr() { return &_eof; }
-
-    bool empty() const { return _segment_group->empty(); }
-    bool zero_num_rows() const { return _segment_group->zero_num_rows(); }
-
-    // Return true if should be filtered out
-    bool rowset_pruning_filter();
-    int delete_pruning_filter();
-    uint64_t get_filtered_rows();
-
-    SegmentGroup* segment_group() const { return _segment_group; }
-    void set_segment_group(SegmentGroup* segment_group) { _segment_group = segment_group; }
-    int64_t num_rows() const { return _segment_group->num_rows(); }
-
-    // To compatible with schema change read, use this function to init column data
-    // for schema change read. Only called in get_first_row_block
-    Status schema_change_init();
-
-private:
-    DISALLOW_COPY_AND_ASSIGN(ColumnData);
-
-    // Try to seek to 'key'. If this function returned with OLAP_SUCCESS, current_row()
-    // point to the first row meet the requirement.
-    // If there is no such row, Status::OLAPInternalError(OLAP_ERR_DATA_EOF) will return.
-    // If error happened, other code will return
-    Status _seek_to_row(const RowCursor& key, bool find_key, bool is_end_key);
-
-    // seek to block_pos without load that block, caller must call _get_block()
-    // to load _read_block with data. If without_filter is false, this will seek to
-    // other block. Because the seeked block may be filtered by condition or delete.
-    Status _seek_to_block(const RowBlockPosition& block_pos, bool without_filter);
-
-    Status _find_position_by_short_key(const RowCursor& key, bool find_last_key,
-                                       RowBlockPosition* position);
-    Status _find_position_by_full_key(const RowCursor& key, bool find_last_key,
-                                      RowBlockPosition* position);
-
-    // Used in _seek_to_row, this function will goto next row that valid for this
-    // ColumnData
-    Status _next_row(const RowCursor** row, bool without_filter);
-
-    // get block from reader, just read vector batch from _current_segment.
-    // The read batch return by got_batch.
-    Status _get_block_from_reader(VectorizedRowBatch** got_batch, bool without_filter,
-                                  int rows_read);
-
-    // get block from segment reader. If this function returns OLAP_SUCCESS
-    Status _get_block(bool without_filter, int rows_read = 0);
-
-    const RowCursor* _current_row() {
-        _read_block->get_row(_read_block->pos(), &_cursor);
-        return &_cursor;
-    }
-
-private:
-    SegmentGroup* _segment_group;
-    // 当到达文件末尾或者到达end key时设置此标志
-    bool _eof;
-    const Conditions* _conditions;
-    std::shared_ptr<std::vector<ColumnPredicate*>> _col_predicates;
-    const DeleteHandler* _delete_handler = nullptr;
-    DelCondSatisfied _delete_status;
-    RuntimeState* _runtime_state;
-    OlapReaderStatistics* _stats;
-
-    const TabletSchema& _schema;
-    // whether in normal read, use return columns to load block
-    bool _is_normal_read = false;
-    bool _end_key_is_set = false;
-    bool _is_using_cache;
-    bool _segment_eof = false;
-    bool _need_eval_predicates = false;
-
-    std::vector<uint32_t> _return_columns;
-    std::vector<uint32_t> _seek_columns;
-    std::set<uint32_t> _load_bf_columns;
-
-    SegmentReader* _segment_reader;
-
-    std::unique_ptr<VectorizedRowBatch> _seek_vector_batch;
-    std::unique_ptr<VectorizedRowBatch> _read_vector_batch;
-
-    std::unique_ptr<RowBlock> _read_block = nullptr;
-    RowCursor _cursor;
-    RowCursor _short_key_cursor;
-
-    // Record when last key is found
-    uint32_t _current_block = 0;
-    uint32_t _current_segment;
-    uint32_t _next_block;
-
-    uint32_t _end_segment;
-    uint32_t _end_block;
-    int64_t _end_row_index = 0;
-
-    size_t _num_rows_per_block;
-    Cache* _lru_cache;
-};
-
-class ColumnDataComparator {
-public:
-    ColumnDataComparator(RowBlockPosition position, ColumnData* olap_data,
-                         const SegmentGroup* segment_group)
-            : _start_block_position(position),
-              _olap_data(olap_data),
-              _segment_group(segment_group) {}
-
-    ~ColumnDataComparator() {}
-
-    // less comparator function
-    bool operator()(const iterator_offset_t& index, const RowCursor& key) const {
-        return _compare(index, key, COMPARATOR_LESS);
-    }
-    // larger comparator function
-    bool operator()(const RowCursor& key, const iterator_offset_t& index) const {
-        return _compare(index, key, COMPARATOR_LARGER);
-    }
-
-private:
-    bool _compare(const iterator_offset_t& index, const RowCursor& key,
-                  ComparatorEnum comparator_enum) const {
-        Status res = Status::OK();
-        RowBlockPosition position = _start_block_position;
-        if (!(res = _segment_group->advance_row_block(index, &position))) {
-            LOG(WARNING) << "fail to advance row block. res=" << res;
-            throw ComparatorException();
-        }
-        const RowCursor* helper_cursor = _olap_data->seek_and_get_current_row(position);
-        if (helper_cursor == nullptr) {
-            LOG(WARNING) << "fail to seek and get current row.";
-            throw ComparatorException();
-        }
-
-        if (COMPARATOR_LESS == comparator_enum) {
-            return compare_row_key(*helper_cursor, key) < 0;
-        } else {
-            return compare_row_key(*helper_cursor, key) > 0;
-        }
-    }
-
-    const RowBlockPosition _start_block_position;
-    ColumnData* _olap_data;
-    const SegmentGroup* _segment_group;
-};
-
-} // namespace doris
-
-#endif // DORIS_BE_SRC_OLAP_ROWSET_COLUMN_DATA_H
diff --git a/be/src/olap/rowset/column_data_writer.cpp b/be/src/olap/rowset/column_data_writer.cpp
deleted file mode 100644
index f1eb60b36d..0000000000
--- a/be/src/olap/rowset/column_data_writer.cpp
+++ /dev/null
@@ -1,320 +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 "olap/rowset/column_data_writer.h"
-
-#include <math.h>
-
-#include "olap/row.h"
-#include "olap/row_block.h"
-#include "olap/rowset/segment_group.h"
-#include "olap/rowset/segment_writer.h"
-
-namespace doris {
-
-ColumnDataWriter* ColumnDataWriter::create(SegmentGroup* segment_group, bool is_push_write,
-                                           CompressKind compress_kind, double bloom_filter_fpp) {
-    ColumnDataWriter* writer = new (std::nothrow)
-            ColumnDataWriter(segment_group, is_push_write, compress_kind, bloom_filter_fpp);
-    return writer;
-}
-
-ColumnDataWriter::ColumnDataWriter(SegmentGroup* segment_group, bool is_push_write,
-                                   CompressKind compress_kind, double bloom_filter_fpp)
-        : _segment_group(segment_group),
-          _is_push_write(is_push_write),
-          _compress_kind(compress_kind),
-          _bloom_filter_fpp(bloom_filter_fpp),
-          _zone_maps(segment_group->get_num_zone_map_columns(), KeyRange(nullptr, nullptr)),
-          _row_index(0),
-          _row_block(nullptr),
-          _segment_writer(nullptr),
-          _num_rows(0),
-          _block_id(0),
-          _max_segment_size(OLAP_MAX_COLUMN_SEGMENT_FILE_SIZE),
-          _segment(0),
-          _all_num_rows(0),
-          _new_segment_created(false) {}
-
-ColumnDataWriter::~ColumnDataWriter() {
-    for (size_t i = 0; i < _zone_maps.size(); ++i) {
-        SAFE_DELETE(_zone_maps[i].first);
-        SAFE_DELETE(_zone_maps[i].second);
-    }
-    SAFE_DELETE(_row_block);
-    SAFE_DELETE(_segment_writer);
-}
-
-Status ColumnDataWriter::init() {
-    Status res = Status::OK();
-
-    for (size_t i = 0; i < _zone_maps.size(); ++i) {
-        _zone_maps[i].first = WrapperField::create(_segment_group->get_tablet_schema().column(i));
-        DCHECK(_zone_maps[i].first != nullptr) << "fail to create column statistics field.";
-        _zone_maps[i].first->set_to_max();
-
-        _zone_maps[i].second = WrapperField::create(_segment_group->get_tablet_schema().column(i));
-        DCHECK(_zone_maps[i].second != nullptr) << "fail to create column statistics field.";
-        _zone_maps[i].second->set_null();
-        _zone_maps[i].second->set_to_min();
-    }
-
-    double size = static_cast<double>(OLAP_MAX_COLUMN_SEGMENT_FILE_SIZE);
-    size *= OLAP_COLUMN_FILE_SEGMENT_SIZE_SCALE;
-    _max_segment_size = static_cast<uint32_t>(lround(size));
-
-    _row_block = new (std::nothrow) RowBlock(&(_segment_group->get_tablet_schema()));
-
-    if (nullptr == _row_block) {
-        LOG(WARNING) << "fail to new RowBlock.";
-        return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
-    }
-
-    res = _cursor.init(_segment_group->get_tablet_schema());
-    if (!res.ok()) {
-        LOG(WARNING) << "fail to initiate row cursor. [res=" << res << "]";
-        return res;
-    }
-
-    VLOG_NOTICE << "init ColumnData writer. segment_group_id=" << _segment_group->segment_group_id()
-                << ", block_row_number=" << _segment_group->get_num_rows_per_row_block();
-    RowBlockInfo block_info(0U, _segment_group->get_num_rows_per_row_block());
-    block_info.null_supported = true;
-
-    _row_block->init(block_info);
-    return Status::OK();
-}
-
-Status ColumnDataWriter::_init_segment() {
-    Status res = _add_segment();
-    if (!res.ok()) {
-        LOG(WARNING) << "fail to add segment. [res=" << res << "]";
-        return res;
-    }
-
-    res = _segment_group->add_segment();
-    if (!res.ok()) {
-        LOG(WARNING) << "fail to add index segment. [res=" << res << "]";
-        return res;
-    }
-
-    _new_segment_created = true;
-    return res;
-}
-
-template <typename RowType>
-Status ColumnDataWriter::write(const RowType& row) {
-    // copy input row to row block
-    _row_block->get_row(_row_index, &_cursor);
-    copy_row(&_cursor, row, _row_block->mem_pool());
-    next(row);
-    if (_row_index >= _segment_group->get_num_rows_per_row_block()) {
-        if (!_flush_row_block(false)) {
-            LOG(WARNING) << "failed to flush data while attaching row cursor.";
-            return Status::OLAPInternalError(OLAP_ERR_OTHER_ERROR);
-        }
-        RETURN_NOT_OK(_flush_segment_with_verification());
-    }
-    return Status::OK();
-}
-
-template <typename RowType>
-void ColumnDataWriter::next(const RowType& row) {
-    for (size_t cid = 0; cid < _segment_group->get_num_zone_map_columns(); ++cid) {
-        auto field = row.schema()->column(cid);
-        auto cell = row.cell(cid);
-
-        if (field->compare_cell(*_zone_maps[cid].first, cell) > 0) {
-            field->direct_copy(_zone_maps[cid].first, cell);
-        }
-
-        if (field->compare_cell(*_zone_maps[cid].second, cell) < 0) {
-            field->direct_copy(_zone_maps[cid].second, cell);
-        }
-    }
-
-    ++_row_index;
-}
-
-Status ColumnDataWriter::finalize() {
-    if (_all_num_rows == 0 && _row_index == 0) {
-        _segment_group->set_empty(true);
-        return Status::OK();
-    }
-
-    // Segment which size reaches OLAP_MAX_COLUMN_SEGMENT_FILE_SIZE
-    // will be flushed into disk. If the previous segment reach
-    // the threshold just right, and been flushed into disk.
-    // The following finalize() when closing ColumnDataWriter
-    // will generate a non-sense segment.
-    // In this scenario, undefined behavior will happens.
-    if (_num_rows == 0 && _row_index == 0) {
-        // If the two conditions are all satisfied,
-        // it dedicates that there is no necessity
-        // to generate segment object and file.
-        // Return OLAP_SUCCESS is OK.
-        return Status::OK();
-    }
-
-    Status res = _flush_row_block(true);
-    if (!res.ok()) {
-        LOG(WARNING) << "failed to flush data while attaching row cursor.res = " << res;
-        return res;
-    }
-
-    res = _finalize_segment();
-    if (!res.ok()) {
-        LOG(WARNING) << "fail to finalize segment. res=" << res << ", _row_index=" << _row_index
-                     << ", _all_num_rows=" << _all_num_rows;
-        return res;
-    }
-
-    res = _segment_group->add_zone_maps(_zone_maps);
-    if (!res.ok()) {
-        LOG(WARNING) << "Fail to set zone_map! res=" << res;
-        return res;
-    }
-
-    return Status::OK();
-}
-
-Status ColumnDataWriter::_flush_row_block(bool finalize) {
-    if (!_new_segment_created) {
-        RETURN_NOT_OK(_init_segment());
-    }
-
-    if (_row_index < 1) {
-        return Status::OK();
-    }
-    // 与OLAPDataWriter不同,这里不是真的写RowBlock,所以并不需要finalize RowBlock
-    // 但考虑到兼容Row Block的使用方式,还是调用了finalize
-    Status res = _row_block->finalize(_row_index);
-    if (!res.ok()) {
-        LOG(WARNING) << "fail to finalize row block. num_rows=" << _row_index << "res=" << res;
-        return Status::OLAPInternalError(OLAP_ERR_WRITER_ROW_BLOCK_ERROR);
-    }
-
-    // 目标是将自己的block按条写入目标block中。
-    res = _segment_writer->write_batch(_row_block, &_cursor, finalize);
-    if (!res.ok()) {
-        LOG(WARNING) << "fail to write row to segment. res = " << res;
-        return Status::OLAPInternalError(OLAP_ERR_WRITER_DATA_WRITE_ERROR);
-    }
-
-    // 在SegmentGroup中记录的不是数据文件的偏移,而是block的编号
-    if (!_segment_group->add_row_block(*_row_block, _block_id++)) {
-        LOG(WARNING) << "fail to update index.";
-        return Status::OLAPInternalError(OLAP_ERR_WRITER_INDEX_WRITE_ERROR);
-    }
-
-    // In order to reuse row_block, clear the row_block after finalize
-    _row_block->clear();
-    _num_rows += _row_index;
-    _all_num_rows += _row_index;
-    _row_index = 0;
-    return Status::OK();
-}
-
-Status ColumnDataWriter::_add_segment() {
-    std::string file_name;
-
-    if (nullptr != _segment_writer) {
-        LOG(WARNING) << "previous segment is not finalized before add new segment.";
-        return Status::OLAPInternalError(OLAP_ERR_WRITER_SEGMENT_NOT_FINALIZED);
-    }
-
-    file_name = _segment_group->construct_data_file_path(_segment);
-    _segment_writer = new (std::nothrow)
-            SegmentWriter(file_name, _segment_group, OLAP_DEFAULT_COLUMN_STREAM_BUFFER_SIZE,
-                          _compress_kind, _bloom_filter_fpp);
-
-    if (nullptr == _segment_writer) {
-        LOG(WARNING) << "fail to allocate SegmentWriter";
-        return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
-    }
-
-    Status res = Status::OK();
-    if (_is_push_write) {
-        res = _segment_writer->init(config::push_write_mbytes_per_sec);
-    } else {
-        res = _segment_writer->init(config::base_compaction_write_mbytes_per_sec);
-    }
-
-    if (!res.ok()) {
-        LOG(WARNING) << "fail to init segment writer";
-        return res;
-    }
-
-    ++_segment;
-    _block_id = 0;
-    return Status::OK();
-}
-
-Status ColumnDataWriter::_flush_segment_with_verification() {
-    uint64_t segment_size = _segment_writer->estimate_segment_size();
-    if (UNLIKELY(segment_size < _max_segment_size)) {
-        return Status::OK();
-    }
-
-    Status res = _finalize_segment();
-    if (!res.ok()) {
-        LOG(WARNING) << "fail to finalize segment. res = " << res;
-        return res;
-    }
-
-    _new_segment_created = false;
-    _num_rows = 0;
-    return Status::OK();
-}
-
-Status ColumnDataWriter::_finalize_segment() {
-    uint32_t data_segment_size;
-    Status res = _segment_writer->finalize(&data_segment_size);
-    if (res != Status::OK()) {
-        LOG(WARNING) << "fail to finish segment from olap_data.";
-        return res;
-    }
-    res = _segment_group->finalize_segment(data_segment_size, _num_rows);
-    if (res != Status::OK()) {
-        LOG(WARNING) << "fail to finish segment from olap_index.";
-        return res;
-    }
-
-    SAFE_DELETE(_segment_writer);
-    return res;
-}
-
-uint64_t ColumnDataWriter::written_bytes() {
-    uint64_t size = _segment * _max_segment_size + _segment_writer->estimate_segment_size();
-    return size;
-}
-
-MemPool* ColumnDataWriter::mem_pool() {
-    return _row_block->mem_pool();
-}
-
-CompressKind ColumnDataWriter::compress_kind() {
-    return _compress_kind;
-}
-
-template Status ColumnDataWriter::write<RowCursor>(const RowCursor& row);
-template Status ColumnDataWriter::write<ContiguousRow>(const ContiguousRow& row);
-
-template void ColumnDataWriter::next<RowCursor>(const RowCursor& row);
-template void ColumnDataWriter::next<ContiguousRow>(const ContiguousRow& row);
-
-} // namespace doris
diff --git a/be/src/olap/rowset/column_data_writer.h b/be/src/olap/rowset/column_data_writer.h
deleted file mode 100644
index 66b72c69ce..0000000000
--- a/be/src/olap/rowset/column_data_writer.h
+++ /dev/null
@@ -1,82 +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.
-
-#ifndef DORIS_BE_SRC_OLAP_ROWSET_COLUMN_DATA_WRITER_H
-#define DORIS_BE_SRC_OLAP_ROWSET_COLUMN_DATA_WRITER_H
-
-#include "gen_cpp/olap_common.pb.h"
-#include "olap/row_block.h"
-#include "olap/rowset/segment_group.h"
-#include "olap/schema.h"
-#include "olap/wrapper_field.h"
-
-namespace doris {
-class RowBlock;
-class SegmentWriter;
-
-class ColumnDataWriter {
-public:
-    // Factory function
-    // 调用者获得新建的对象, 并负责delete释放
-    static ColumnDataWriter* create(SegmentGroup* segment_group, bool is_push_write,
-                                    CompressKind compress_kind, double bloom_filter_fpp);
-    ColumnDataWriter(SegmentGroup* segment_group, bool is_push_write, CompressKind compress_kind,
-                     double bloom_filter_fpp);
-    ~ColumnDataWriter();
-    Status init();
-
-    template <typename RowType>
-    Status write(const RowType& row);
-
-    template <typename RowType>
-    void next(const RowType& row);
-
-    Status finalize();
-    uint64_t written_bytes();
-    MemPool* mem_pool();
-    CompressKind compress_kind();
-
-private:
-    Status _add_segment();
-    Status _flush_segment_with_verification();
-    Status _finalize_segment();
-    Status _flush_row_block(bool finalize);
-    Status _init_segment();
-
-private:
-    SegmentGroup* _segment_group;
-    bool _is_push_write;
-    CompressKind _compress_kind;
-    double _bloom_filter_fpp;
-    // first is min, second is max
-    std::vector<std::pair<WrapperField*, WrapperField*>> _zone_maps;
-    uint32_t _row_index;
-
-    RowBlock* _row_block; // 使用RowBlock缓存要写入的数据
-    RowCursor _cursor;
-    SegmentWriter* _segment_writer;
-    int64_t _num_rows;
-    uint32_t _block_id; // 当前Segment内的block编号
-    uint32_t _max_segment_size;
-    uint32_t _segment;
-    int64_t _all_num_rows;
-    bool _new_segment_created;
-};
-
-} // namespace doris
-
-#endif // DORIS_BE_SRC_OLAP_ROWSET_COLUMN_DATA_WRITER_H
diff --git a/be/src/olap/rowset/column_reader.cpp b/be/src/olap/rowset/column_reader.cpp
deleted file mode 100644
index 8c6cc1676c..0000000000
--- a/be/src/olap/rowset/column_reader.cpp
+++ /dev/null
@@ -1,1186 +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 "olap/rowset/column_reader.h"
-
-#include <cstring>
-
-#include "olap/file_stream.h"
-#include "olap/olap_define.h"
-#include "olap/rowset/bit_field_reader.h"
-
-namespace doris {
-IntegerColumnReader::IntegerColumnReader(uint32_t column_unique_id)
-        : _eof(false), _column_unique_id(column_unique_id), _data_reader(nullptr) {}
-
-IntegerColumnReader::~IntegerColumnReader() {
-    SAFE_DELETE(_data_reader);
-}
-
-Status IntegerColumnReader::init(std::map<StreamName, ReadOnlyFileStream*>* streams, bool is_sign) {
-    if (nullptr == streams) {
-        LOG(WARNING) << "input streams is nullptr";
-        return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
-    }
-
-    // Get data stream according to column id and type
-    ReadOnlyFileStream* data_stream =
-            extract_stream(_column_unique_id, StreamInfoMessage::DATA, streams);
-
-    if (data_stream == nullptr) {
-        LOG(WARNING) << "specified stream is nullptr";
-        return Status::OLAPInternalError(OLAP_ERR_COLUMN_STREAM_NOT_EXIST);
-    }
-
-    _data_reader = new (std::nothrow) RunLengthIntegerReader(data_stream, is_sign);
-
-    if (nullptr == _data_reader) {
-        LOG(WARNING) << "fail to malloc RunLengthIntegerReader";
-        return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
-    }
-
-    // reset eof flag when init, to support reinit
-    return Status::OK();
-}
-
-Status IntegerColumnReader::seek(PositionProvider* position) {
-    return _data_reader->seek(position);
-}
-
-Status IntegerColumnReader::skip(uint64_t row_count) {
-    return _data_reader->skip(row_count);
-}
-
-Status IntegerColumnReader::next(int64_t* value) {
-    return _data_reader->next(value);
-}
-
-StringColumnDirectReader::StringColumnDirectReader(uint32_t column_unique_id,
-                                                   uint32_t dictionary_size)
-        : _column_unique_id(column_unique_id),
-          _values(nullptr),
-          _data_stream(nullptr),
-          _length_reader(nullptr) {}
-
-StringColumnDirectReader::~StringColumnDirectReader() {
-    SAFE_DELETE(_length_reader);
-}
-
-Status StringColumnDirectReader::init(std::map<StreamName, ReadOnlyFileStream*>* streams, int size,
-                                      MemPool* mem_pool) {
-    if (nullptr == streams) {
-        LOG(WARNING) << "input streams is nullptr";
-        return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
-    }
-
-    // Get data stream according to column id and type
-    _data_stream = extract_stream(_column_unique_id, StreamInfoMessage::DATA, streams);
-
-    if (nullptr == _data_stream) {
-        LOG(WARNING) << "specified stream not found. [unique_id = " << _column_unique_id << "]";
-        return Status::OLAPInternalError(OLAP_ERR_COLUMN_STREAM_NOT_EXIST);
-    }
-
-    _values = reinterpret_cast<Slice*>(mem_pool->allocate(size * sizeof(Slice)));
-
-    ReadOnlyFileStream* length_stream =
-            extract_stream(_column_unique_id, StreamInfoMessage::LENGTH, streams);
-
-    if (nullptr == length_stream) {
-        LOG(WARNING) << "specified stream not found. [unique_id = " << _column_unique_id << "]";
-        return Status::OLAPInternalError(OLAP_ERR_COLUMN_STREAM_NOT_EXIST);
-    }
-
-    _length_reader = new (std::nothrow) RunLengthIntegerReader(length_stream, false);
-
-    if (nullptr == _length_reader) {
-        LOG(WARNING) << "fail to malloc RunLengthIntegerReader";
-        return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
-    }
-
-    return Status::OK();
-}
-
-Status StringColumnDirectReader::seek(PositionProvider* position) {
-    Status res = _data_stream->seek(position);
-
-    // All strings in segment may be empty, so the data stream is EOF and
-    // and length stream is not EOF.
-    if (res.ok() || OLAP_ERR_COLUMN_STREAM_EOF == res.precise_code()) {
-        res = _length_reader->seek(position);
-    }
-
-    return res;
-}
-
-Status StringColumnDirectReader::skip(uint64_t row_count) {
-    Status res = Status::OK();
-    int64_t skip_length = 0;
-    int64_t tmp_length = 0;
-
-    for (size_t i = 0; i < row_count; ++i) {
-        res = _length_reader->next(&tmp_length);
-
-        if (!res.ok()) {
-            return res;
-        }
-
-        skip_length += tmp_length;
-    }
-
-    if (res.ok()) {
-        // TODO: skip function of instream is implemented, but not tested
-        return _data_stream->skip(skip_length);
-    }
-
-    return res;
-}
-
-// Return string field of current row_count
-Status StringColumnDirectReader::next(char* buffer, uint32_t* length) {
-    int64_t read_length = 0;
-    Status res = _length_reader->next(&read_length);
-    *length = read_length;
-    while (res.ok() && read_length > 0) {
-        uint64_t buf_size = read_length;
-        res = _data_stream->read(buffer, &buf_size);
-        read_length -= buf_size;
-        buffer += buf_size;
-    }
-    *length -= read_length;
-    return res;
-}
-
-Status StringColumnDirectReader::next_vector(ColumnVector* column_vector, uint32_t size,
-                                             MemPool* mem_pool, int64_t* read_bytes) {
-    /*
-     * MemPool here is not the same as MemPool in init function
-     * 1. MemPool is created by VectorizedRowBatch,
-     *    and reset when load row batch
-     * 2. MemPool in init function is created by SegmentReader,
-     *    and free by SegmentReader deconstruction.
-     */
-    Status res = Status::OK();
-    int64_t length = 0;
-    int64_t string_buffer_size = 0;
-
-    column_vector->set_col_data(_values);
-    if (column_vector->no_nulls()) {
-        for (int i = 0; i < size; ++i) {
-            res = _length_reader->next(&length);
-            if (!res.ok()) {
-                return res;
-            }
-            _values[i].size = length;
-            string_buffer_size += length;
-        }
-
-        uint8_t* allocated_mem = nullptr;
-        res = mem_pool->allocate_safely(string_buffer_size, allocated_mem);
-        if (!res.ok()) {
-            return res;
-        }
-        char* string_buffer = reinterpret_cast<char*>(allocated_mem);
-
-        for (int i = 0; i < size; ++i) {
-            length = _values[i].size;
-            if (UNLIKELY(length == 0)) {
-                _values[i].data = nullptr;
-            } else {
-                _values[i].data = string_buffer;
-            }
-            while (length > 0) {
-                uint64_t buf_size = length;
-                res = _data_stream->read(string_buffer, &buf_size);
-                if (!res.ok()) {
-                    return res;
-                }
-                length -= buf_size;
-                string_buffer += buf_size;
-            }
-        }
-    } else {
-        bool* is_null = column_vector->is_null();
-        for (int i = 0; i < size; ++i) {
-            if (!is_null[i]) {
-                res = _length_reader->next(&length);
-                if (!res.ok()) {
-                    return res;
-                }
-                _values[i].size = length;
-                string_buffer_size += length;
-            } else {
-                _values[i].size = 0;
-            }
-        }
-
-        uint8_t* allocated_mem = nullptr;
-        res = mem_pool->allocate_safely(string_buffer_size, allocated_mem);
-        if (!res.ok()) {
-            return res;
-        }
-        char* string_buffer = reinterpret_cast<char*>(allocated_mem);
-
-        for (int i = 0; i < size; ++i) {
-            if (!is_null[i]) {
-                length = _values[i].size;
-                if (UNLIKELY(length == 0)) {
-                    _values[i].data = nullptr;
-                } else {
-                    _values[i].data = string_buffer;
-                }
-                while (length > 0) {
-                    uint64_t buf_size = length;
-                    res = _data_stream->read(string_buffer, &buf_size);
-                    if (!res.ok()) {
-                        return res;
-                    }
-                    length -= buf_size;
-                    string_buffer += buf_size;
-                }
-            } else {
-                _values[i].data = nullptr;
-                _values[i].size = 0;
-            }
-        }
-    }
-    *read_bytes += string_buffer_size;
-
-    return res;
-}
-
-StringColumnDictionaryReader::StringColumnDictionaryReader(uint32_t column_unique_id,
-                                                           uint32_t dictionary_size)
-        : _eof(false),
-          _dictionary_size(dictionary_size),
-          _column_unique_id(column_unique_id),
-          _values(nullptr),
-          //_dictionary_size(0),
-          //_offset_dictionary(nullptr),
-          //_dictionary_data_buffer(nullptr),
-          _read_buffer(nullptr),
-          _data_reader(nullptr) {}
-
-StringColumnDictionaryReader::~StringColumnDictionaryReader() {
-    //SAFE_DELETE_ARRAY(_offset_dictionary);
-    //SAFE_DELETE(_dictionary_data_buffer);
-    SAFE_DELETE(_data_reader);
-    SAFE_DELETE_ARRAY(_read_buffer);
-}
-
-Status StringColumnDictionaryReader::init(std::map<StreamName, ReadOnlyFileStream*>* streams,
-                                          int size, MemPool* mem_pool) {
-    ReadOnlyFileStream* dictionary_data_stream =
-            extract_stream(_column_unique_id, StreamInfoMessage::DICTIONARY_DATA, streams);
-
-    if (nullptr == dictionary_data_stream) {
-        LOG(WARNING) << "dictionary data stream not found. [unique id = " << _column_unique_id
-                     << "]";
-        return Status::OLAPInternalError(OLAP_ERR_COLUMN_STREAM_NOT_EXIST);
-    }
-
-    ReadOnlyFileStream* dictionary_length_stream =
-            extract_stream(_column_unique_id, StreamInfoMessage::LENGTH, streams);
-
-    if (nullptr == dictionary_length_stream) {
-        LOG(WARNING) << "dictionary length stream not found. [unique id = " << _column_unique_id
-                     << "]";
-        return Status::OLAPInternalError(OLAP_ERR_COLUMN_STREAM_NOT_EXIST);
-    }
-
-    RunLengthIntegerReader* dictionary_length_reader =
-            new (std::nothrow) RunLengthIntegerReader(dictionary_length_stream, false);
-    Status res = Status::OK();
-
-    _values = reinterpret_cast<Slice*>(mem_pool->allocate(size * sizeof(Slice)));
-    int64_t read_buffer_size = 1024;
-    char* _read_buffer = new (std::nothrow) char[read_buffer_size];
-
-    if (nullptr == _read_buffer) {
-        LOG(WARNING) << "fail to malloc read buffer. [size = " << read_buffer_size << "]";
-        return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
-    }
-
-    int64_t length = 0;
-    uint64_t read_length = 0;
-    std::string dictionary_item;
-
-    for (size_t dictionary_entry = 0; dictionary_entry < _dictionary_size; ++dictionary_entry) {
-        res = dictionary_length_reader->next(&length);
-        // 理论上应该足够读,读出eof也是不对的。
-        if (!res.ok() || length < 0) {
-            LOG(WARNING) << "build offset dictionary failed. res = " << res;
-            return res;
-        }
-
-        if (length > read_buffer_size) {
-            SAFE_DELETE_ARRAY(_read_buffer);
-            read_buffer_size = length;
-
-            if (nullptr == (_read_buffer = new (std::nothrow) char[read_buffer_size])) {
-                LOG(WARNING) << "fail to malloc read buffer. [size = " << read_buffer_size << "]";
-                return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
-            }
-        }
-
-        read_length = length;
-        dictionary_data_stream->read(_read_buffer, &read_length);
-
-        if (static_cast<int64_t>(read_length) != length) {
-            LOG(WARNING) << "read stream fail.";
-            return Status::OLAPInternalError(OLAP_ERR_COLUMN_READ_STREAM);
-        }
-
-        dictionary_item.assign(_read_buffer, length);
-        _dictionary.push_back(dictionary_item);
-    }
-
-    // 建立数据流读取器
-    ReadOnlyFileStream* data_stream =
-            extract_stream(_column_unique_id, StreamInfoMessage::DATA, streams);
-
-    if (nullptr == data_stream) {
-        LOG(WARNING) << "data stream not found. [unique id = " << _column_unique_id << "]";
-        return Status::OLAPInternalError(OLAP_ERR_COLUMN_STREAM_NOT_EXIST);
-    }
-
-    _data_reader = new (std::nothrow) RunLengthIntegerReader(data_stream, false);
-
-    if (nullptr == _data_reader) {
-        LOG(WARNING) << "fail to malloc data reader";
-        return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
-    }
-
-    SAFE_DELETE_ARRAY(_read_buffer);
-    SAFE_DELETE(dictionary_length_reader);
-    return Status::OK();
-}
-
-Status StringColumnDictionaryReader::seek(PositionProvider* position) {
-    return _data_reader->seek(position);
-}
-
-Status StringColumnDictionaryReader::skip(uint64_t row_count) {
-    return _data_reader->skip(row_count);
-}
-
-Status StringColumnDictionaryReader::next(char* buffer, uint32_t* length) {
-    int64_t value;
-    Status res = _data_reader->next(&value);
-    // 错误或是EOF
-    if (!res.ok()) {
-        if (OLAP_ERR_DATA_EOF == res.precise_code()) {
-            _eof = true;
-        }
-
-        return res;
-    }
-
-    if (value >= static_cast<int64_t>(_dictionary.size())) {
-        LOG(WARNING) << "value may indicated an invalid dictionary entry. "
-                        "[value = "
-                     << value << ", dictionary_size = " << _dictionary.size() << "]";
-        return Status::OLAPInternalError(OLAP_ERR_BUFFER_OVERFLOW);
-    }
-
-    memcpy(buffer, _dictionary[value].c_str(), _dictionary[value].size());
-    *length = _dictionary[value].size();
-
-    return Status::OK();
-}
-
-Status StringColumnDictionaryReader::next_vector(ColumnVector* column_vector, uint32_t size,
-                                                 MemPool* mem_pool, int64_t* read_bytes) {
-    int64_t index[size];
-    int64_t buffer_size = 0;
-    Status res = Status::OK();
-
-    column_vector->set_col_data(_values);
-    if (column_vector->no_nulls()) {
-        for (int i = 0; i < size; ++i) {
-            res = _data_reader->next(&index[i]);
-            if (!res.ok()) {
-                return res;
-            }
-            if (index[i] >= static_cast<int64_t>(_dictionary.size())) {
-                LOG(WARNING) << "value may indicated an invalid dictionary entry. "
-                                "[index = "
-                             << index[i] << ", dictionary_size = " << _dictionary.size() << "]";
-                return Status::OLAPInternalError(OLAP_ERR_BUFFER_OVERFLOW);
-            }
-            _values[i].size = _dictionary[index[i]].size();
-            buffer_size += _values[i].size;
-        }
-
-        char* string_buffer = reinterpret_cast<char*>(mem_pool->allocate(buffer_size));
-        for (int i = 0; i < size; ++i) {
-            memory_copy(string_buffer, _dictionary[index[i]].c_str(), _values[i].size);
-            _values[i].data = string_buffer;
-            string_buffer += _values[i].size;
-        }
-    } else {
-        bool* is_null = column_vector->is_null();
-        for (int i = 0; i < size; ++i) {
-            if (!is_null[i]) {
-                res = _data_reader->next(&index[i]);
-                if (!res.ok()) {
-                    return res;
-                }
-                if (index[i] >= static_cast<int64_t>(_dictionary.size())) {
-                    LOG(WARNING) << "value may indicated an invalid dictionary entry. "
-                                    "[index = "
-                                 << index[i] << ", dictionary_size = " << _dictionary.size() << "]";
-                    return Status::OLAPInternalError(OLAP_ERR_BUFFER_OVERFLOW);
-                }
-                _values[i].size = _dictionary[index[i]].size();
-                buffer_size += _values[i].size;
-            }
-        }
-
-        char* string_buffer = reinterpret_cast<char*>(mem_pool->allocate(buffer_size));
-        for (int i = 0; i < size; ++i) {
-            if (!is_null[i]) {
-                memory_copy(string_buffer, _dictionary[index[i]].c_str(), _values[i].size);
-                _values[i].data = string_buffer;
-                string_buffer += _values[i].size;
-            }
-        }
-    }
-    *read_bytes += buffer_size;
-
-    return res;
-}
-
-ColumnReader::ColumnReader(uint32_t column_id, uint32_t column_unique_id)
-        : _value_present(false),
-          _is_null(nullptr),
-          _column_id(column_id),
-          _column_unique_id(column_unique_id),
-          _present_reader(nullptr) {}
-
-ColumnReader* ColumnReader::create(uint32_t column_id, const TabletSchema& schema,
-                                   const UniqueIdToColumnIdMap& included,
-                                   UniqueIdToColumnIdMap& segment_included,
-                                   const UniqueIdEncodingMap& encodings) {
-    return create(column_id, schema.columns(), included, segment_included, encodings);
-}
-
-ColumnReader* ColumnReader::create(uint32_t column_id, const std::vector<TabletColumn>& schema,
-                                   const UniqueIdToColumnIdMap& included,
-                                   UniqueIdToColumnIdMap& segment_included,
-                                   const UniqueIdEncodingMap& encodings) {
-    if (column_id >= schema.size()) {
-        LOG(WARNING) << "invalid column_id, column_id=" << column_id
-                     << ", columns_size=" << schema.size();
-        return nullptr;
-    }
-
-    const TabletColumn& column = schema[column_id];
-    ColumnReader* reader = nullptr;
-    int32_t column_unique_id = column.unique_id();
-
-    if (0 == included.count(column_unique_id)) {
-        return nullptr;
-    }
-
-    if (0 == segment_included.count(column_unique_id)) {
-        if (column.has_default_value()) {
-            if (0 == strcasecmp("NULL", column.default_value().c_str()) && column.is_nullable()) {
-                return new (std::nothrow) NullValueReader(column_id, column_unique_id);
-            } else {
-                return new (std::nothrow)
-                        DefaultValueReader(column_id, column_unique_id, column.default_value(),
-                                           column.type(), column.length());
-            }
-        } else if (column.is_nullable()) {
-            return new (std::nothrow) NullValueReader(column_id, column_unique_id);
-        } else {
-            LOG(WARNING) << "not null field has no default value";
-            return nullptr;
-        }
-    }
-
-    uint32_t dictionary_size = 0;
-    ColumnEncodingMessage::Kind encode_kind = ColumnEncodingMessage::DIRECT;
-    UniqueIdEncodingMap::const_iterator it = encodings.find(column_unique_id);
-
-    if (it != encodings.end()) {
-        encode_kind = (*it).second.kind();
-        dictionary_size = (*it).second.dictionary_size();
-    }
-
-    switch (column.type()) {
-    case OLAP_FIELD_TYPE_BOOL:
-    case OLAP_FIELD_TYPE_TINYINT:
-    case OLAP_FIELD_TYPE_UNSIGNED_TINYINT: {
-        reader = new (std::nothrow) TinyColumnReader(column_id, column_unique_id);
-        break;
-    }
-
-    case OLAP_FIELD_TYPE_SMALLINT: {
-        reader = new (std::nothrow)
-                IntegerColumnReaderWrapper<int16_t, true>(column_id, column_unique_id);
-        break;
-    }
-
-    case OLAP_FIELD_TYPE_UNSIGNED_SMALLINT: {
-        reader = new (std::nothrow)
-                IntegerColumnReaderWrapper<uint16_t, false>(column_id, column_unique_id);
-        break;
-    }
-
-    case OLAP_FIELD_TYPE_INT: {
-        reader = new (std::nothrow)
-                IntegerColumnReaderWrapper<int32_t, true>(column_id, column_unique_id);
-        break;
-    }
-
-    case OLAP_FIELD_TYPE_UNSIGNED_INT: {
-        reader = new (std::nothrow)
-                IntegerColumnReaderWrapper<uint32_t, false>(column_id, column_unique_id);
-        break;
-    }
-
-    case OLAP_FIELD_TYPE_BIGINT: {
-        reader = new (std::nothrow)
-                IntegerColumnReaderWrapper<int64_t, true>(column_id, column_unique_id);
-        break;
-    }
-
-    case OLAP_FIELD_TYPE_UNSIGNED_BIGINT: {
-        reader = new (std::nothrow)
-                IntegerColumnReaderWrapper<uint64_t, false>(column_id, column_unique_id);
-        break;
-    }
-
-    case OLAP_FIELD_TYPE_FLOAT: {
-        reader = new (std::nothrow) FloatColumnReader(column_id, column_unique_id);
-        break;
-    }
-
-    case OLAP_FIELD_TYPE_DOUBLE: {
-        reader = new (std::nothrow) DoubleColumnReader(column_id, column_unique_id);
-        break;
-    }
-
-    case OLAP_FIELD_TYPE_DISCRETE_DOUBLE: {
-        reader = new (std::nothrow) DiscreteDoubleColumnReader(column_id, column_unique_id);
-        break;
-    }
-
-    case OLAP_FIELD_TYPE_CHAR: {
-        if (ColumnEncodingMessage::DIRECT == encode_kind) {
-            reader = new (std::nothrow) FixLengthStringColumnReader<StringColumnDirectReader>(
-                    column_id, column_unique_id, column.length(), dictionary_size);
-        } else if (ColumnEncodingMessage::DICTIONARY == encode_kind) {
-            reader = new (std::nothrow) FixLengthStringColumnReader<StringColumnDictionaryReader>(
-                    column_id, column_unique_id, column.length(), dictionary_size);
-        } else {
-            LOG(WARNING) << "known encoding format. data may be generated by higher version,try "
-                            "updating olap/ngine binary to solve this problem";
-            // TODO. define a new return code
-            return nullptr;
-        }
-
-        break;
-    }
-
-    case OLAP_FIELD_TYPE_DATETIME: {
-        reader = new (std::nothrow) DateTimeColumnReader(column_id, column_unique_id);
-        break;
-    }
-
-    case OLAP_FIELD_TYPE_DATE: {
-        reader = new (std::nothrow) DateColumnReader(column_id, column_unique_id);
-
-        break;
-    }
-
-    case OLAP_FIELD_TYPE_DATEV2: {
-        reader = new (std::nothrow) DateV2ColumnReader(column_id, column_unique_id);
-
-        break;
-    }
-
-    case OLAP_FIELD_TYPE_DECIMAL: {
-        reader = new (std::nothrow) DecimalColumnReader(column_id, column_unique_id);
-        break;
-    }
-
-    case OLAP_FIELD_TYPE_LARGEINT: {
-        reader = new (std::nothrow) LargeIntColumnReader(column_id, column_unique_id);
-        break;
-    }
-
-    case OLAP_FIELD_TYPE_VARCHAR:
-    case OLAP_FIELD_TYPE_OBJECT:
-    case OLAP_FIELD_TYPE_HLL:
-    case OLAP_FIELD_TYPE_STRING: {
-        if (ColumnEncodingMessage::DIRECT == encode_kind) {
-            reader = new (std::nothrow) VarStringColumnReader<StringColumnDirectReader>(
-                    column_id, column_unique_id, column.length(), dictionary_size);
-        } else if (ColumnEncodingMessage::DICTIONARY == encode_kind) {
-            reader = new (std::nothrow) VarStringColumnReader<StringColumnDictionaryReader>(
-                    column_id, column_unique_id, column.length(), dictionary_size);
-        } else {
-            LOG(WARNING) << "known encoding format. data may be generated by higher version, "
-                         << "try updating olap/ngine binary to solve this problem";
-            // TODO. define a new return code
-            return nullptr;
-        }
-
-        break;
-    }
-
-    case OLAP_FIELD_TYPE_STRUCT:
-    case OLAP_FIELD_TYPE_ARRAY:
-    case OLAP_FIELD_TYPE_MAP:
-    default: {
-        LOG(WARNING) << "unsupported field type. field=" << column.name()
-                     << ", type=" << column.type();
-        break;
-    }
-    }
-
-    return reader;
-}
-
-ColumnReader::~ColumnReader() {
-    SAFE_DELETE(_present_reader);
-}
-
-Status ColumnReader::init(std::map<StreamName, ReadOnlyFileStream*>* streams, int size,
-                          MemPool* mem_pool, OlapReaderStatistics* stats) {
-    if (nullptr == streams) {
-        LOG(WARNING) << "null parameters given.";
-        return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
-    }
-    _stats = stats;
-
-    // 从map中找到需要的流,ColumnReader的数据应该由一条PRESENT流和一条ROW_INDEX流组成
-    ReadOnlyFileStream* present_stream =
-            extract_stream(_column_unique_id, StreamInfoMessage::PRESENT, streams);
-
-    _is_null = reinterpret_cast<bool*>(mem_pool->allocate(size));
-    memset(_is_null, 0, size);
-
-    if (nullptr == present_stream) {
-        _present_reader = nullptr;
-        _value_present = false;
-    } else {
-        VLOG_TRACE << "create null present_stream for column_id:" << _column_unique_id;
-        _present_reader = new (std::nothrow) BitFieldReader(present_stream);
-
-        if (nullptr == _present_reader) {
-            LOG(WARNING) << "malloc present reader failed.";
-            return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
-        }
-
-        if (!_present_reader->init()) {
-            LOG(WARNING) << "fail to init present reader.";
-            return Status::OLAPInternalError(OLAP_ERR_INIT_FAILED);
-        }
-
-        _value_present = true;
-    }
-
-    return Status::OK();
-}
-
-Status ColumnReader::seek(PositionProvider* position) {
-    if (nullptr != _present_reader) {
-        return _present_reader->seek(position);
-    }
-
-    return Status::OK();
-}
-
-Status ColumnReader::skip(uint64_t row_count) {
-    return Status::OK();
-}
-
-Status ColumnReader::next_vector(ColumnVector* column_vector, uint32_t size, MemPool* mem_pool) {
-    Status res = Status::OK();
-    column_vector->set_is_null(_is_null);
-    if (nullptr != _present_reader) {
-        column_vector->set_no_nulls(false);
-        for (uint32_t i = 0; i < size; ++i) {
-            bool value = false;
-            res = _present_reader->next((char*)&value);
-            if (!res.ok()) {
-                break;
-            }
-            _is_null[i] = value;
-        }
-        _stats->bytes_read += size;
-    } else {
-        column_vector->set_no_nulls(true);
-    }
-
-    return res;
-}
-
-uint64_t ColumnReader::_count_none_nulls(uint64_t rows) {
-    if (_present_reader != nullptr) {
-        Status res = Status::OK();
-        uint64_t result = 0;
-
-        for (uint64_t counter = 0; counter < rows; ++counter) {
-            res = _present_reader->next(reinterpret_cast<char*>(&_value_present));
-
-            if (res.ok() && (false == _value_present)) {
-                result += 1;
-            } else {
-                break;
-            }
-        }
-
-        return result;
-    } else {
-        return rows;
-    }
-}
-
-TinyColumnReader::TinyColumnReader(uint32_t column_id, uint32_t column_unique_id)
-        : ColumnReader(column_id, column_unique_id),
-          _eof(false),
-          _values(nullptr),
-          _data_reader(nullptr) {}
-
-TinyColumnReader::~TinyColumnReader() {
-    SAFE_DELETE(_data_reader);
-}
-
-Status TinyColumnReader::init(std::map<StreamName, ReadOnlyFileStream*>* streams, int size,
-                              MemPool* mem_pool, OlapReaderStatistics* stats) {
-    if (nullptr == streams) {
-        LOG(WARNING) << "input streams is nullptr";
-        return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
-    }
-
-    ColumnReader::init(streams, size, mem_pool, stats);
-    ReadOnlyFileStream* data_stream =
-            extract_stream(_column_unique_id, StreamInfoMessage::DATA, streams);
-
-    if (nullptr == data_stream) {
-        LOG(WARNING) << "specified stream not exist";
-        return Status::OLAPInternalError(OLAP_ERR_COLUMN_STREAM_NOT_EXIST);
-    }
-
-    _values = reinterpret_cast<char*>(mem_pool->allocate(size));
-    _data_reader = new (std::nothrow) RunLengthByteReader(data_stream);
-
-    if (nullptr == _data_reader) {
-        LOG(WARNING) << "malloc data reader failed";
-        return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
-    }
-
-    return Status::OK();
-}
-
-Status TinyColumnReader::seek(PositionProvider* positions) {
-    Status res;
-    if (nullptr == _present_reader) {
-        res = _data_reader->seek(positions);
-        if (!res.ok()) {
-            return res;
-        }
-    } else {
-        res = ColumnReader::seek(positions);
-        if (!res.ok()) {
-            return res;
-        }
-        res = _data_reader->seek(positions);
-        if (!res.ok() && OLAP_ERR_COLUMN_STREAM_EOF != res.precise_code()) {
-            LOG(WARNING) << "fail to seek tinyint stream. res = " << res;
-            return res;
-        }
-    }
-
-    return Status::OK();
-}
-
-Status TinyColumnReader::skip(uint64_t row_count) {
-    // count_none_nulls 其实就是columnReader的跳过函数。
-    return _data_reader->skip(_count_none_nulls(row_count));
-}
-
-Status TinyColumnReader::next_vector(ColumnVector* column_vector, uint32_t size,
-                                     MemPool* mem_pool) {
-    Status res = ColumnReader::next_vector(column_vector, size, mem_pool);
-    if (!res.ok()) {
-        if (OLAP_ERR_DATA_EOF == res.precise_code()) {
-            _eof = true;
-        }
-        return res;
-    }
-
-    bool* is_null = column_vector->is_null();
-    column_vector->set_col_data(_values);
-    if (column_vector->no_nulls()) {
-        for (uint32_t i = 0; i < size; ++i) {
-            res = _data_reader->next(_values + i);
-            if (!res.ok()) {
-                break;
-            }
-        }
-    } else {
-        for (uint32_t i = 0; i < size; ++i) {
-            if (!is_null[i]) {
-                res = _data_reader->next(_values + i);
-                if (!res.ok()) {
-                    break;
-                }
-            }
-        }
-    }
-    _stats->bytes_read += size;
-
-    if (OLAP_ERR_DATA_EOF == res.precise_code()) {
-        _eof = true;
-    }
-
-    return res;
-}
-
-DecimalColumnReader::DecimalColumnReader(uint32_t column_id, uint32_t column_unique_id)
-        : ColumnReader(column_id, column_unique_id),
-          _eof(false),
-          _values(nullptr),
-          _int_reader(nullptr),
-          _frac_reader(nullptr) {}
-
-DecimalColumnReader::~DecimalColumnReader() {
-    SAFE_DELETE(_int_reader);
-    SAFE_DELETE(_frac_reader);
-}
-
-Status DecimalColumnReader::init(std::map<StreamName, ReadOnlyFileStream*>* streams, int size,
-                                 MemPool* mem_pool, OlapReaderStatistics* stats) {
-    if (nullptr == streams) {
-        LOG(WARNING) << "input streams is nullptr";
-        return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
-    }
-
-    // reset stream and reader
-    ColumnReader::init(streams, size, mem_pool, stats);
-
-    _values = reinterpret_cast<decimal12_t*>(mem_pool->allocate(size * sizeof(decimal12_t)));
-
-    // 从map中找到需要的流,StringColumnReader的数据应该由一条DATA流和一条LENGTH流组成
-    ReadOnlyFileStream* int_stream =
-            extract_stream(_column_unique_id, StreamInfoMessage::DATA, streams);
-
-    if (nullptr == int_stream) {
-        LOG(WARNING) << "specified stream not found. [unique_id = " << _column_unique_id << "]";
-        return Status::OLAPInternalError(OLAP_ERR_COLUMN_STREAM_NOT_EXIST);
-    }
-
-    ReadOnlyFileStream* frac_stream =
-            extract_stream(_column_unique_id, StreamInfoMessage::SECONDARY, streams);
-
-    if (nullptr == frac_stream) {
-        LOG(WARNING) << "specified stream not found. [unique_id = " << _column_unique_id << "]";
-        return Status::OLAPInternalError(OLAP_ERR_COLUMN_STREAM_NOT_EXIST);
-    }
-
-    _int_reader = new (std::nothrow) RunLengthIntegerReader(int_stream, true);
-
-    if (nullptr == _int_reader) {
-        LOG(WARNING) << "fail to malloc RunLengthIntegerReader";
-        return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
-    }
-
-    _frac_reader = new (std::nothrow) RunLengthIntegerReader(frac_stream, true);
-
-    if (nullptr == _frac_reader) {
-        LOG(WARNING) << "fail to malloc RunLengthIntegerReader";
-        return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
-    }
-
-    return Status::OK();
-}
-
-Status DecimalColumnReader::seek(PositionProvider* positions) {
-    Status res;
-    if (nullptr == _present_reader) {
-        res = _int_reader->seek(positions);
-        if (!res.ok()) {
-            return res;
-        }
-
-        res = _frac_reader->seek(positions);
-
-        if (!res.ok()) {
-            return res;
-        }
-    } else {
-        //all field in the segment can be nullptr, so the data stream is EOF
-        res = ColumnReader::seek(positions);
-        if (!res.ok()) {
-            return res;
-        }
-        res = _int_reader->seek(positions);
-        if (!res.ok() && OLAP_ERR_COLUMN_STREAM_EOF != res.precise_code()) {
-            LOG(WARNING) << "fail to seek int stream of decimal. res = " << res;
-            return res;
-        }
-
-        res = _frac_reader->seek(positions);
-        if (!res.ok() && OLAP_ERR_COLUMN_STREAM_EOF != res.precise_code()) {
-            LOG(WARNING) << "fail to seek frac stream of decimal. res = " << res;
-            return res;
-        }
-    }
-
-    return Status::OK();
-}
-
-Status DecimalColumnReader::skip(uint64_t row_count) {
-    Status res = _int_reader->skip(row_count);
-
-    if (!res.ok()) {
-        LOG(WARNING) << "fail to create int part reader";
-        return res;
-    }
-
-    res = _frac_reader->skip(row_count);
-
-    if (!res.ok()) {
-        LOG(WARNING) << "fail to create frac part reader";
-        return res;
-    }
-
-    return Status::OK();
-}
-
-Status DecimalColumnReader::next_vector(ColumnVector* column_vector, uint32_t size,
-                                        MemPool* mem_pool) {
-    Status res = ColumnReader::next_vector(column_vector, size, mem_pool);
-    if (!res.ok()) {
-        if (OLAP_ERR_DATA_EOF == res.precise_code()) {
-            _eof = true;
-        }
-        return res;
-    }
-
-    bool* is_null = column_vector->is_null();
-    column_vector->set_col_data(_values);
-
-    if (column_vector->no_nulls()) {
-        for (uint32_t i = 0; i < size; ++i) {
-            int64_t value = 0;
-            Status res = _int_reader->next(&value);
-            if (!res.ok()) {
-                LOG(WARNING) << "fail to read decimal int part";
-                break;
-            }
-            _values[i].integer = value;
-
-            res = _frac_reader->next(&value);
-            if (!res.ok()) {
-                LOG(WARNING) << "fail to read decimal frac part";
-                break;
-            }
-            _values[i].fraction = value;
-        }
-    } else {
-        for (uint32_t i = 0; i < size; ++i) {
-            int64_t value = 0;
-            if (!is_null[i]) {
-                Status res = _int_reader->next(&value);
-                if (!res.ok()) {
-                    LOG(WARNING) << "fail to read decimal int part";
-                    break;
-                }
-                _values[i].integer = value;
-
-                res = _frac_reader->next(&value);
-                if (!res.ok()) {
-                    LOG(WARNING) << "fail to read decimal frac part";
-                    break;
-                }
-                _values[i].fraction = value;
-            }
-        }
-    }
-    _stats->bytes_read += sizeof(decimal12_t) * size;
-
-    return res;
-}
-
-LargeIntColumnReader::LargeIntColumnReader(uint32_t column_id, uint32_t column_unique_id)
-        : ColumnReader(column_id, column_unique_id),
-          _eof(false),
-          _values(nullptr),
-          _high_reader(nullptr),
-          _low_reader(nullptr) {}
-
-LargeIntColumnReader::~LargeIntColumnReader() {
-    SAFE_DELETE(_high_reader);
-    SAFE_DELETE(_low_reader);
-}
-
-Status LargeIntColumnReader::init(std::map<StreamName, ReadOnlyFileStream*>* streams, int size,
-                                  MemPool* mem_pool, OlapReaderStatistics* stats) {
-    if (nullptr == streams) {
-        LOG(WARNING) << "input streams is nullptr";
-        return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
-    }
-
-    // reset stream and reader
-    ColumnReader::init(streams, size, mem_pool, stats);
-
-    _values = reinterpret_cast<int128_t*>(
-            mem_pool->try_allocate_aligned(size * sizeof(int128_t), alignof(int128_t)));
-
-    // 从map中找到需要的流,LargeIntColumnReader的数据应该由一条DATA流组成
-    ReadOnlyFileStream* high_stream =
-            extract_stream(_column_unique_id, StreamInfoMessage::DATA, streams);
-    if (nullptr == high_stream) {
-        LOG(WARNING) << "specified stream not found. [unique_id = " << _column_unique_id << "]";
-        return Status::OLAPInternalError(OLAP_ERR_COLUMN_STREAM_NOT_EXIST);
-    }
-
-    ReadOnlyFileStream* low_stream =
-            extract_stream(_column_unique_id, StreamInfoMessage::SECONDARY, streams);
-    if (nullptr == low_stream) {
-        LOG(WARNING) << "specified stream not found. [unique_id = " << _column_unique_id << "]";
-        return Status::OLAPInternalError(OLAP_ERR_COLUMN_STREAM_NOT_EXIST);
-    }
-
-    _high_reader = new (std::nothrow) RunLengthIntegerReader(high_stream, true);
-    if (nullptr == _high_reader) {
-        LOG(WARNING) << "fail to malloc RunLengthIntegerReader.";
-        return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
-    }
-
-    _low_reader = new (std::nothrow) RunLengthIntegerReader(low_stream, true);
-    if (nullptr == _low_reader) {
-        LOG(WARNING) << "fail to malloc RunLengthIntegerReader.";
-        return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
-    }
-
-    return Status::OK();
-}
-
-Status LargeIntColumnReader::seek(PositionProvider* positions) {
-    Status res;
-    if (nullptr == _present_reader) {
-        res = _high_reader->seek(positions);
-        if (!res.ok()) {
-            return res;
-        }
-
-        res = _low_reader->seek(positions);
-        if (!res.ok()) {
-            return res;
-        }
-    } else {
-        //all field in the segment can be nullptr, so the data stream is EOF
-        res = ColumnReader::seek(positions);
-        if (!res.ok()) {
-            LOG(WARNING) << "fail to seek null stream of largeint";
-            return res;
-        }
-
-        res = _high_reader->seek(positions);
-        if (!res.ok() && OLAP_ERR_COLUMN_STREAM_EOF != res.precise_code()) {
-            LOG(WARNING) << "fail to seek high int stream of largeint. res = " << res;
-            return res;
-        }
-
-        res = _low_reader->seek(positions);
-        if (!res.ok() && OLAP_ERR_COLUMN_STREAM_EOF != res.precise_code()) {
-            LOG(WARNING) << "fail to seek low int stream of largeint. res = " << res;
-            return res;
-        }
-    }
-
-    return Status::OK();
-}
-
-Status LargeIntColumnReader::skip(uint64_t row_count) {
-    Status res = _high_reader->skip(row_count);
-    if (!res.ok()) {
-        LOG(WARNING) << "fail to skip large int high part. res = " << res;
-        return res;
-    }
-
-    res = _low_reader->skip(row_count);
-    if (!res.ok()) {
-        LOG(WARNING) << "fail to skip large int low part reader. res = " << res;
-        return res;
-    }
-
-    return Status::OK();
-}
-
-Status LargeIntColumnReader::next_vector(ColumnVector* column_vector, uint32_t size,
-                                         MemPool* mem_pool) {
-    Status res = ColumnReader::next_vector(column_vector, size, mem_pool);
-    if (!res.ok()) {
-        if (OLAP_ERR_DATA_EOF == res.precise_code()) {
-            _eof = true;
-        }
-        return res;
-    }
-
-    bool* is_null = column_vector->is_null();
-    column_vector->set_col_data(_values);
-
-    if (column_vector->no_nulls()) {
-        for (uint32_t i = 0; i < size; ++i) {
-            int64_t* value = nullptr;
-            value = (int64_t*)(_values + i);
-            res = _high_reader->next(value);
-            if (!res.ok()) {
-                LOG(WARNING) << "fail to read decimal int part";
-                break;
-            }
-
-            res = _low_reader->next(++value);
-            if (!res.ok()) {
-                LOG(WARNING) << "fail to read decimal frac part";
-                break;
-            }
-        }
-    } else {
-        for (uint32_t i = 0; i < size; ++i) {
-            int64_t* value = nullptr;
-            if (!is_null[i]) {
-                value = (int64_t*)(_values + i);
-                res = _high_reader->next(value);
-                if (!res.ok()) {
-                    LOG(WARNING) << "fail to read decimal int part";
-                    break;
-                }
-
-                res = _low_reader->next(++value);
-                if (!res.ok()) {
-                    LOG(WARNING) << "fail to read decimal frac part";
-                    break;
-                }
-            }
-        }
-    }
-    _stats->bytes_read += 16 * size;
-
-    return res;
-}
-
-} // namespace doris
diff --git a/be/src/olap/rowset/column_reader.h b/be/src/olap/rowset/column_reader.h
deleted file mode 100644
index 2880cef335..0000000000
--- a/be/src/olap/rowset/column_reader.h
+++ /dev/null
@@ -1,841 +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.
-
-#ifndef DORIS_BE_SRC_OLAP_ROWSET_COLUMN_READER_H
-#define DORIS_BE_SRC_OLAP_ROWSET_COLUMN_READER_H
-
-#include "olap/file_stream.h"
-#include "olap/olap_common.h"
-#include "olap/rowset/run_length_byte_reader.h"
-#include "olap/rowset/run_length_integer_reader.h"
-#include "olap/stream_name.h"
-#include "runtime/large_int_value.h"
-#include "runtime/vectorized_row_batch.h"
-#include "util/date_func.h"
-
-namespace doris {
-
-class StreamName;
-class ReadOnlyFileStream;
-class BitFieldReader;
-class RowIndexEntryMessage;
-class ColumnEncodingMessage;
-
-// Solution flow
-inline ReadOnlyFileStream* extract_stream(uint32_t column_unique_id, StreamInfoMessage::Kind kind,
-                                          std::map<StreamName, ReadOnlyFileStream*>* streams) {
-    StreamName stream_name(column_unique_id, kind);
-    std::map<StreamName, ReadOnlyFileStream*>::iterator it = streams->find(stream_name);
-
-    if (streams->end() != it) {
-        return (*it).second;
-    }
-
-    return nullptr;
-}
-
-// Unique id -> PositionProvider
-typedef std::unordered_map<uint32_t, PositionProvider> UniqueIdPositionProviderMap;
-// Unique id -> ColumnEncodingMessage
-typedef std::map<uint32_t, ColumnEncodingMessage> UniqueIdEncodingMap;
-
-// Readers for Integer and String.
-// Although these readers are also named Reader, note that they do not inherit from ColumnReader
-// Therefore, the case of null values is not considered.
-
-// For SHORT/INT/LONG type data, use int64 as the stored data uniformly
-// Due to the use of variable length coding, it will not cause waste
-// IntegerColumnReader is a reader that reads the int64 data of the bottom layer, when the upper layer returns
-// Use IntColumnReaderWrapper to convert to a specific data type
-//
-// NOTE. Since the RLE reader can only read int64, which is different from java, java shaping does not consider symbols
-// Then this thing actually seems to be �
-// �� method supports unsigned integer shaping, you need to pay attention to whether to modify RLEReader in the future
-class IntegerColumnReader {
-public:
-    IntegerColumnReader(uint32_t column_unique_id);
-    ~IntegerColumnReader();
-    /**
-      * Initialize the Integer column reader
-      * @param streams contains the map of the required stream
-      * @param is_sign whether the number read has a sign
-      * @return [description]
-      */
-    Status init(std::map<StreamName, ReadOnlyFileStream*>* streams, bool is_sign);
-    // Position the internal pointer to positions
-    Status seek(PositionProvider* positions);
-    // Move the internal pointer back row_count rows
-    Status skip(uint64_t row_count);
-    // Return the data of the current row by moving the internal pointer to the next row
-    Status next(int64_t* value);
-    bool eof() { return _eof; }
-
-private:
-    bool _eof;
-    uint32_t _column_unique_id;
-    RunLengthIntegerReader* _data_reader;
-};
-
-// For readers of string columns encoded in Direct mode
-// Direct method of String can be read directly
-class StringColumnDirectReader {
-public:
-    StringColumnDirectReader(uint32_t column_unique_id, uint32_t dictionary_size);
-    ~StringColumnDirectReader();
-
-    Status init(std::map<StreamName, ReadOnlyFileStream*>* streams, int size, MemPool* mem_pool);
-    Status seek(PositionProvider* positions);
-    Status skip(uint64_t row_count);
-    // Return the data of the current row and move the internal pointer backward
-    // buffer - the buffer of the returned data
-    // length - the size of the buffer area when input, and the size of the string when returning
-    Status next(char* buffer, uint32_t* length);
-    Status next_vector(ColumnVector* column_vector, uint32_t size, MemPool* mem_pool,
-                       int64_t* read_bytes);
-
-    size_t get_buffer_size() { return sizeof(RunLengthByteReader); }
-
-private:
-    uint32_t _column_unique_id;
-    Slice* _values;
-    ReadOnlyFileStream* _data_stream;
-    RunLengthIntegerReader* _length_reader;
-};
-
-// For readers using dictionary-encoded string columns
-// The interface is the same as StringColumnDirectReader
-// Reading process:
-// 1. Read all the dictionary data and save it in the whole buffer
-// 2. Read the length data, construct the offset dictionary, the offset is �
-// ��The beginning of each string, combined with 1 can read data
-// 3. Read the actual saved data when needed.
-// �� is an int). Find the offset according to this int, and then read the dictionary data according to the offset
-class StringColumnDictionaryReader {
-public:
-    StringColumnDictionaryReader(uint32_t column_unique_id, uint32_t dictionary_size);
-    ~StringColumnDictionaryReader();
-    Status init(std::map<StreamName, ReadOnlyFileStream*>* streams, int size, MemPool* mem_pool);
-    Status seek(PositionProvider* positions);
-    Status skip(uint64_t row_count);
-    Status next(char* buffer, uint32_t* length);
-    Status next_vector(ColumnVector* column_vector, uint32_t size, MemPool* mem_pool,
-                       int64_t* read_bytes);
-
-    size_t get_buffer_size() { return sizeof(RunLengthByteReader) + _dictionary_size; }
-
-private:
-    bool _eof;
-    uint32_t _dictionary_size;
-    uint32_t _column_unique_id;
-    Slice* _values;
-    char* _read_buffer;
-    //uint64_t _dictionary_size;
-    //uint64_t* _offset_dictionary;   // The offset corresponding to the number used to find the response data
-    //StorageByteBuffer* _dictionary_data_buffer;   // Save dict data
-    std::vector<std::string> _dictionary;
-    // Used to read the actual data (represented by an integer)
-    RunLengthIntegerReader* _data_reader;
-};
-
-// ColumnReader is used to read a column and is the base class of other XXXColumnReader
-// ColumnReader maintains the NULL feature of the column through the bit field presented
-class ColumnReader {
-public:
-    // Factory method, create ColumnReader, if the column has sub-columns, recursively create a sub reader
-    // If the column to be read does not exist in segment_columns, then:
-    //       1.If the column allows Null values, create a NullValueReader
-    //       2.If the column does not allow Null values, but has a default value, create a DefaultValueReader
-    //       3.Otherwise the creation fails
-    // Input:
-    //       column_id - the position of the column to be created in the columns
-    //       columns - the schema of the table
-    //       included - column to be created, if the unique id of a column is included in included
-    //       segment_columns - a collection of unique ids of all columns in the segment
-    //       encodings - column encoding information, use encodings[_column_unique_id] to access
-    static ColumnReader* create(uint32_t column_id, const TabletSchema& schema,
-                                const UniqueIdToColumnIdMap& included,
-                                UniqueIdToColumnIdMap& segment_included,
-                                const UniqueIdEncodingMap& encodings);
-
-    static ColumnReader* create(uint32_t column_id, const std::vector<TabletColumn>& schema,
-                                const UniqueIdToColumnIdMap& included,
-                                UniqueIdToColumnIdMap& segment_included,
-                                const UniqueIdEncodingMap& encodings);
-
-    ColumnReader(uint32_t column_id, uint32_t column_unique_id);
-    virtual ~ColumnReader();
-
-    // Use streams to initialize Reader
-    // ColumnReader is initialized only once, and a new object is allocated each time it is used.
-    // Input:
-    //       streams-input stream
-    virtual Status init(std::map<StreamName, ReadOnlyFileStream*>* streams, int size,
-                        MemPool* mem_pool, OlapReaderStatistics* stats);
-
-    // Set the position of the next returned data
-    // positions are the positions where each column needs to seek, ColumnReader passes (*positions)[_column_unique_id]
-    // Get the seek position of this column
-    virtual Status seek(PositionProvider* positions);
-
-    virtual Status skip(uint64_t row_count);
-
-    virtual Status next_vector(ColumnVector* column_vector, uint32_t size, MemPool* mem_pool);
-
-    uint32_t column_unique_id() { return _column_unique_id; }
-
-    uint32_t column_id() { return _column_id; }
-
-    virtual size_t get_buffer_size() { return 0; }
-
-protected:
-    // NOTE. Count the non-blank rows in rows. This is because "blank lines" do not exist in actual storage.
-    // So for upper-level fields that may be empty (such as integer), the caller wants to skip 10 lines,
-    // but actually for
-    uint64_t _count_none_nulls(uint64_t rows);
-
-    bool _value_present;
-    bool* _is_null;
-    uint32_t _column_id;             // The id of the column in the schema
-    uint32_t _column_unique_id;      // the unique id of the column
-    BitFieldReader* _present_reader; // NULL value of NULLabel field
-    std::vector<ColumnReader*> _sub_readers;
-    OlapReaderStatistics* _stats = nullptr;
-};
-
-class DefaultValueReader : public ColumnReader {
-public:
-    DefaultValueReader(uint32_t column_id, uint32_t column_unique_id, std::string default_value,
-                       FieldType type, int length)
-            : ColumnReader(column_id, column_unique_id),
-              _default_value(default_value),
-              _values(nullptr),
-              _type(type),
-              _length(length) {}
-
-    virtual ~DefaultValueReader() {}
-
-    virtual Status init(std::map<StreamName, ReadOnlyFileStream*>* streams, int size,
-                        MemPool* mem_pool, OlapReaderStatistics* stats) {
-        switch (_type) {
-        case OLAP_FIELD_TYPE_TINYINT: {
-            _values = reinterpret_cast<void*>(mem_pool->allocate(size * sizeof(int8_t)));
-            int32_t value = 0;
-            std::stringstream ss(_default_value);
-            ss >> value;
-            for (int i = 0; i < size; ++i) {
-                ((int8_t*)_values)[i] = value;
-            }
-            break;
-        }
-        case OLAP_FIELD_TYPE_SMALLINT: {
-            _values = reinterpret_cast<void*>(mem_pool->allocate(size * sizeof(int16_t)));
-            int16_t value = 0;
-            std::stringstream ss(_default_value);
-            ss >> value;
-            for (int i = 0; i < size; ++i) {
-                ((int16_t*)_values)[i] = value;
-            }
-            break;
-        }
-        case OLAP_FIELD_TYPE_INT: {
-            _values = reinterpret_cast<void*>(mem_pool->allocate(size * sizeof(int32_t)));
-            int32_t value = 0;
-            std::stringstream ss(_default_value);
-            ss >> value;
-            for (int i = 0; i < size; ++i) {
-                ((int32_t*)_values)[i] = value;
-            }
-            break;
-        }
-        case OLAP_FIELD_TYPE_BIGINT: {
-            _values = reinterpret_cast<void*>(mem_pool->allocate(size * sizeof(int64_t)));
-            int64_t value = 0;
-            std::stringstream ss(_default_value);
-            ss >> value;
-            for (int i = 0; i < size; ++i) {
-                ((int64_t*)_values)[i] = value;
-            }
-            break;
-        }
-        case OLAP_FIELD_TYPE_LARGEINT: {
-            _values = reinterpret_cast<void*>(
-                    mem_pool->try_allocate_aligned(size * sizeof(int128_t), alignof(int128_t)));
-            int128_t value = 0;
-            std::stringstream ss(_default_value);
-            ss >> value;
-            for (int i = 0; i < size; ++i) {
-                ((int128_t*)_values)[i] = value;
-            }
-            break;
-        }
-        case OLAP_FIELD_TYPE_FLOAT: {
-            _values = reinterpret_cast<void*>(mem_pool->allocate(size * sizeof(float)));
-            float value = 0;
-            std::stringstream ss(_default_value);
-            ss >> value;
-            for (int i = 0; i < size; ++i) {
-                ((float*)_values)[i] = value;
-            }
-            break;
-        }
-        case OLAP_FIELD_TYPE_DOUBLE: {
-            _values = reinterpret_cast<void*>(mem_pool->allocate(size * sizeof(double)));
-            double value = 0;
-            std::stringstream ss(_default_value);
-            ss >> value;
-            for (int i = 0; i < size; ++i) {
-                ((double*)_values)[i] = value;
-            }
-            break;
-        }
-        case OLAP_FIELD_TYPE_DECIMAL: {
-            _values = reinterpret_cast<void*>(mem_pool->allocate(size * sizeof(decimal12_t)));
-            decimal12_t value = {0, 0};
-            value.from_string(_default_value);
-            for (int i = 0; i < size; ++i) {
-                ((decimal12_t*)_values)[i] = value;
-            }
-            break;
-        }
-        case OLAP_FIELD_TYPE_CHAR: {
-            _values = reinterpret_cast<void*>(mem_pool->allocate(size * sizeof(Slice)));
-            int32_t length = _length;
-            char* string_buffer = reinterpret_cast<char*>(mem_pool->allocate(size * length));
-            memset(string_buffer, 0, size * length);
-            for (int i = 0; i < size; ++i) {
-                memory_copy(string_buffer, _default_value.c_str(), _default_value.length());
-                ((Slice*)_values)[i].size = length;
-                ((Slice*)_values)[i].data = string_buffer;
-                string_buffer += length;
-            }
-            break;
-        }
-        case OLAP_FIELD_TYPE_VARCHAR:
-        case OLAP_FIELD_TYPE_OBJECT:
-        case OLAP_FIELD_TYPE_HLL:
-        case OLAP_FIELD_TYPE_STRING: {
-            _values = reinterpret_cast<void*>(mem_pool->allocate(size * sizeof(Slice)));
-            int32_t length = _default_value.length();
-            char* string_buffer = reinterpret_cast<char*>(mem_pool->allocate(size * length));
-            for (int i = 0; i < size; ++i) {
-                memory_copy(string_buffer, _default_value.c_str(), length);
-                ((Slice*)_values)[i].size = length;
-                ((Slice*)_values)[i].data = string_buffer;
-                string_buffer += length;
-            }
-            break;
-        }
-        case OLAP_FIELD_TYPE_DATE: {
-            _values = reinterpret_cast<void*>(mem_pool->allocate(size * sizeof(uint24_t)));
-            uint24_t value = timestamp_from_date(_default_value);
-            for (int i = 0; i < size; ++i) {
-                ((uint24_t*)_values)[i] = value;
-            }
-            break;
-        }
-        case OLAP_FIELD_TYPE_DATEV2: {
-            _values = reinterpret_cast<void*>(mem_pool->allocate(size * sizeof(uint32_t)));
-            uint32_t value = 0;
-            std::stringstream ss(_default_value);
-            ss >> value;
-            for (int i = 0; i < size; ++i) {
-                ((uint32_t*)_values)[i] = value;
-            }
-            break;
-        }
-        case OLAP_FIELD_TYPE_DATETIME: {
-            _values = reinterpret_cast<void*>(mem_pool->allocate(size * sizeof(uint64_t)));
-            uint64_t value = timestamp_from_datetime(_default_value);
-            for (int i = 0; i < size; ++i) {
-                ((uint64_t*)_values)[i] = value;
-            }
-            break;
-        }
-        default:
-            break;
-        }
-        _stats = stats;
-        return Status::OK();
-    }
-    virtual Status seek(PositionProvider* positions) { return Status::OK(); }
-    virtual Status skip(uint64_t row_count) { return Status::OK(); }
-
-    virtual Status next_vector(ColumnVector* column_vector, uint32_t size, MemPool* mem_pool) {
-        column_vector->set_no_nulls(true);
-        column_vector->set_col_data(_values);
-        _stats->bytes_read += _length * size;
-        return Status::OK();
-    }
-
-private:
-    std::string _default_value;
-    void* _values;
-    FieldType _type;
-    int32_t _length;
-};
-
-class NullValueReader : public ColumnReader {
-public:
-    NullValueReader(uint32_t column_id, uint32_t column_unique_id)
-            : ColumnReader(column_id, column_unique_id) {}
-    Status init(std::map<StreamName, ReadOnlyFileStream*>* streams, int size, MemPool* mem_pool,
-                OlapReaderStatistics* stats) override {
-        _is_null = reinterpret_cast<bool*>(mem_pool->allocate(size));
-        memset(_is_null, 1, size);
-        _stats = stats;
-        return Status::OK();
-    }
-    virtual Status seek(PositionProvider* positions) override { return Status::OK(); }
-    virtual Status skip(uint64_t row_count) override { return Status::OK(); }
-    virtual Status next_vector(ColumnVector* column_vector, uint32_t size,
-                               MemPool* mem_pool) override {
-        column_vector->set_no_nulls(false);
-        column_vector->set_is_null(_is_null);
-        _stats->bytes_read += size;
-        return Status::OK();
-    }
-};
-
-// 对于Tiny类型, 使用Byte作为存储
-class TinyColumnReader : public ColumnReader {
-public:
-    TinyColumnReader(uint32_t column_id, uint32_t column_unique_id);
-    virtual ~TinyColumnReader();
-
-    virtual Status init(std::map<StreamName, ReadOnlyFileStream*>* streams, int size,
-                        MemPool* mem_pool, OlapReaderStatistics* stats);
-    virtual Status seek(PositionProvider* positions);
-    virtual Status skip(uint64_t row_count);
-    virtual Status next_vector(ColumnVector* column_vector, uint32_t size, MemPool* mem_pool);
-
-    virtual size_t get_buffer_size() { return sizeof(RunLengthByteReader); }
-
-private:
-    bool _eof;
-    char* _values;
-    RunLengthByteReader* _data_reader;
-};
-
-// A wrapper for IntColumnReader, which implements the interface to ColumnReader
-template <class T, bool is_sign>
-class IntegerColumnReaderWrapper : public ColumnReader {
-public:
-    IntegerColumnReaderWrapper(uint32_t column_id, uint32_t column_unique_id)
-            : ColumnReader(column_id, column_unique_id),
-              _reader(column_unique_id),
-              _values(nullptr),
-              _eof(false) {}
-
-    virtual ~IntegerColumnReaderWrapper() {}
-
-    virtual Status init(std::map<StreamName, ReadOnlyFileStream*>* streams, int size,
-                        MemPool* mem_pool, OlapReaderStatistics* stats) {
-        Status res = ColumnReader::init(streams, size, mem_pool, stats);
-
-        if (res.ok()) {
-            res = _reader.init(streams, is_sign);
-        }
-
-        _values = reinterpret_cast<T*>(mem_pool->allocate(size * sizeof(T)));
-
-        return res;
-    }
-    virtual Status seek(PositionProvider* positions) {
-        Status res;
-        if (nullptr == _present_reader) {
-            res = _reader.seek(positions);
-            if (!res.ok()) {
-                return res;
-            }
-        } else {
-            //all field in the segment can be nullptr, so the data stream is EOF
-            res = ColumnReader::seek(positions);
-            if (!res.ok()) {
-                return res;
-            }
-            res = _reader.seek(positions);
-            if (!res.ok() && OLAP_ERR_COLUMN_STREAM_EOF != res.precise_code()) {
-                LOG(WARNING) << "fail to seek int stream. res = " << res;
-                return res;
-            }
-        }
-
-        return Status::OK();
-    }
-    virtual Status skip(uint64_t row_count) { return _reader.skip(_count_none_nulls(row_count)); }
-
-    virtual Status next_vector(ColumnVector* column_vector, uint32_t size, MemPool* mem_pool) {
-        Status res = ColumnReader::next_vector(column_vector, size, mem_pool);
-        if (!res.ok()) {
-            if (OLAP_ERR_DATA_EOF == res.precise_code()) {
-                _eof = true;
-            }
-            return res;
-        }
-
-        column_vector->set_col_data(_values);
-        if (column_vector->no_nulls()) {
-            for (uint32_t i = 0; i < size; ++i) {
-                int64_t value = 0;
-                res = _reader.next(&value);
-                if (!res.ok()) {
-                    break;
-                }
-                _values[i] = value;
-            }
-        } else {
-            bool* is_null = column_vector->is_null();
-            for (uint32_t i = 0; i < size; ++i) {
-                int64_t value = 0;
-                if (!is_null[i]) {
-                    res = _reader.next(&value);
-                    if (!res.ok()) {
-                        break;
-                    }
-                }
-                _values[i] = value;
-            }
-        }
-        _stats->bytes_read += sizeof(T) * size;
-
-        if (OLAP_ERR_DATA_EOF == res.precise_code()) {
-            _eof = true;
-        }
-        return res;
-    }
-
-    virtual size_t get_buffer_size() { return sizeof(RunLengthIntegerReader); }
-
-private:
-    IntegerColumnReader _reader; // Wrapped real reader
-    T* _values;
-    bool _eof;
-};
-
-// There are two types of strings in OLAP Engine, fixed-length strings and variable-length strings, using two wrappers respectively
-// class handles the return format of these two strings
-// FixLengthStringColumnReader handles fixed-length strings, the feature is that the part of insufficient length should be filled with 0
-template <class ReaderClass>
-class FixLengthStringColumnReader : public ColumnReader {
-public:
-    FixLengthStringColumnReader(uint32_t column_id, uint32_t column_unique_id,
-                                uint32_t string_length, uint32_t dictionary_size)
-            : ColumnReader(column_id, column_unique_id),
-              _eof(false),
-              _reader(column_unique_id, dictionary_size),
-              _string_length(string_length) {}
-    virtual ~FixLengthStringColumnReader() {}
-
-    virtual Status init(std::map<StreamName, ReadOnlyFileStream*>* streams, int size,
-                        MemPool* mem_pool, OlapReaderStatistics* stats) {
-        Status res = ColumnReader::init(streams, size, mem_pool, stats);
-
-        if (res.ok()) {
-            res = _reader.init(streams, size, mem_pool);
-        }
-
-        return res;
-    }
-
-    virtual Status seek(PositionProvider* positions) {
-        Status res;
-        if (nullptr == _present_reader) {
-            res = _reader.seek(positions);
-            if (!res.ok()) {
-                return res;
-            }
-        } else {
-            //all field in the segment can be nullptr, so the data stream is EOF
-            res = ColumnReader::seek(positions);
-            if (!res.ok()) {
-                return res;
-            }
-            res = _reader.seek(positions);
-            if (!res.ok() && OLAP_ERR_COLUMN_STREAM_EOF != res.precise_code()) {
-                LOG(WARNING) << "fail to read fixed string stream. res = " << res;
-                return res;
-            }
-        }
-
-        return Status::OK();
-    }
-    virtual Status skip(uint64_t row_count) { return _reader.skip(_count_none_nulls(row_count)); }
-    virtual Status next_vector(ColumnVector* column_vector, uint32_t size, MemPool* mem_pool) {
-        Status res = ColumnReader::next_vector(column_vector, size, mem_pool);
-        if (!res.ok()) {
-            if (OLAP_ERR_DATA_EOF == res.precise_code()) {
-                _eof = true;
-            }
-            return res;
-        }
-
-        return _reader.next_vector(column_vector, size, mem_pool, &_stats->bytes_read);
-    }
-
-    virtual size_t get_buffer_size() { return _reader.get_buffer_size() + _string_length; }
-
-private:
-    bool _eof;
-    ReaderClass _reader;
-    uint32_t _string_length;
-};
-
-// VarStringColumnReader handles variable length strings, characterized by using uint16 in the data header to indicate the length
-template <class ReaderClass>
-class VarStringColumnReader : public ColumnReader {
-public:
-    VarStringColumnReader(uint32_t column_id, uint32_t column_unique_id, uint32_t max_length,
-                          uint32_t dictionary_size)
-            : ColumnReader(column_id, column_unique_id),
-              _eof(false),
-              _reader(column_unique_id, dictionary_size),
-              _max_length(max_length) {}
-    virtual ~VarStringColumnReader() {}
-    virtual Status init(std::map<StreamName, ReadOnlyFileStream*>* streams, int size,
-                        MemPool* mem_pool, OlapReaderStatistics* stats) {
-        Status res = ColumnReader::init(streams, size, mem_pool, stats);
-        if (res.ok()) {
-            res = _reader.init(streams, size, mem_pool);
-        }
-
-        return res;
-    }
-
-    virtual Status seek(PositionProvider* position) {
-        Status res;
-        if (nullptr == _present_reader) {
-            res = _reader.seek(position);
-            if (!res.ok()) {
-                return res;
-            }
-        } else {
-            //all field in the segment can be nullptr, so the data stream is EOF
-            res = ColumnReader::seek(position);
-            if (!res.ok()) {
-                return res;
-            }
-            res = _reader.seek(position);
-            if (!res.ok() && OLAP_ERR_COLUMN_STREAM_EOF != res.precise_code()) {
-                LOG(WARNING) << "fail to seek varchar stream. res = " << res;
-                return res;
-            }
-        }
-
-        return Status::OK();
-    }
-    virtual Status skip(uint64_t row_count) { return _reader.skip(_count_none_nulls(row_count)); }
-
-    virtual Status next_vector(ColumnVector* column_vector, uint32_t size, MemPool* mem_pool) {
-        Status res = ColumnReader::next_vector(column_vector, size, mem_pool);
-        if (!res.ok()) {
-            if (OLAP_ERR_DATA_EOF == res.precise_code()) {
-                _eof = true;
-            }
-            return res;
-        }
-
-        return _reader.next_vector(column_vector, size, mem_pool, &_stats->bytes_read);
-    }
-
-    virtual size_t get_buffer_size() { return _reader.get_buffer_size() + _max_length; }
-
-private:
-    bool _eof;
-    ReaderClass _reader;
-    uint32_t _max_length;
-};
-
-template <typename FLOAT_TYPE>
-class FloatintPointColumnReader : public ColumnReader {
-public:
-    FloatintPointColumnReader(uint32_t column_id, uint32_t column_unique_id)
-            : ColumnReader(column_id, column_unique_id),
-              _eof(false),
-              _data_stream(nullptr),
-              _values(nullptr) {}
-
-    virtual ~FloatintPointColumnReader() {}
-
-    virtual Status init(std::map<StreamName, ReadOnlyFileStream*>* streams, int size,
-                        MemPool* mem_pool, OlapReaderStatistics* stats) {
-        if (nullptr == streams) {
-            LOG(WARNING) << "input streams is nullptr";
-            return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
-        }
-
-        // reset stream and reader
-        ColumnReader::init(streams, size, mem_pool, stats);
-        _data_stream = extract_stream(_column_unique_id, StreamInfoMessage::DATA, streams);
-
-        if (nullptr == _data_stream) {
-            LOG(WARNING) << "specified stream not exist";
-            return Status::OLAPInternalError(OLAP_ERR_COLUMN_STREAM_NOT_EXIST);
-        }
-
-        _values = reinterpret_cast<FLOAT_TYPE*>(mem_pool->allocate(size * sizeof(FLOAT_TYPE)));
-        return Status::OK();
-    }
-    virtual Status seek(PositionProvider* position) {
-        if (nullptr == position) {
-            LOG(WARNING) << "input positions is nullptr";
-            return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
-        }
-
-        if (nullptr == _data_stream) {
-            LOG(WARNING) << "reader not init.";
-            return Status::OLAPInternalError(OLAP_ERR_NOT_INITED);
-        }
-
-        Status res;
-        if (nullptr == _present_reader) {
-            res = _data_stream->seek(position);
-            if (!res.ok()) {
-                return res;
-            }
-        } else {
-            //all field in the segment can be nullptr, so the data stream is EOF
-            res = ColumnReader::seek(position);
-            if (!res.ok()) {
-                return res;
-            }
-            res = _data_stream->seek(position);
-            if (!res.ok() && OLAP_ERR_COLUMN_STREAM_EOF != res.precise_code()) {
-                LOG(WARNING) << "fail to seek float stream. res = " << res;
-                return res;
-            }
-        }
-
-        return Status::OK();
-    }
-    virtual Status skip(uint64_t row_count) {
-        if (nullptr == _data_stream) {
-            LOG(WARNING) << "reader not init.";
-            return Status::OLAPInternalError(OLAP_ERR_NOT_INITED);
-        }
-
-        uint64_t skip_values_count = _count_none_nulls(row_count);
-        return _data_stream->skip(skip_values_count * sizeof(FLOAT_TYPE));
-    }
-
-    virtual Status next_vector(ColumnVector* column_vector, uint32_t size, MemPool* mem_pool) {
-        if (nullptr == _data_stream) {
-            LOG(WARNING) << "reader not init.";
-            return Status::OLAPInternalError(OLAP_ERR_NOT_INITED);
-        }
-
-        Status res = ColumnReader::next_vector(column_vector, size, mem_pool);
-        if (!res.ok()) {
-            if (OLAP_ERR_DATA_EOF == res.precise_code()) {
-                _eof = true;
-            }
-            return res;
-        }
-
-        bool* is_null = column_vector->is_null();
-        column_vector->set_col_data(_values);
-        size_t length = sizeof(FLOAT_TYPE);
-        if (column_vector->no_nulls()) {
-            for (uint32_t i = 0; i < size; ++i) {
-                FLOAT_TYPE value = 0.0;
-                res = _data_stream->read(reinterpret_cast<char*>(&value), &length);
-                if (!res.ok()) {
-                    break;
-                }
-                _values[i] = value;
-            }
-        } else {
-            for (uint32_t i = 0; i < size; ++i) {
-                FLOAT_TYPE value = 0.0;
-                if (!is_null[i]) {
-                    res = _data_stream->read(reinterpret_cast<char*>(&value), &length);
-                    if (!res.ok()) {
-                        break;
-                    }
-                }
-                _values[i] = value;
-            }
-        }
-        _stats->bytes_read += sizeof(FLOAT_TYPE) * size;
-
-        if (OLAP_ERR_DATA_EOF == res.precise_code()) {
-            _eof = true;
-        }
-
-        return res;
-    }
-
-protected:
-    bool _eof;
-    ReadOnlyFileStream* _data_stream;
-    FLOAT_TYPE* _values;
-};
-
-class DecimalColumnReader : public ColumnReader {
-public:
-    DecimalColumnReader(uint32_t column_id, uint32_t column_unique_id);
-    virtual ~DecimalColumnReader();
-    Status init(std::map<StreamName, ReadOnlyFileStream*>* streams, int size, MemPool* mem_pool,
-                OlapReaderStatistics* stats) override;
-    virtual Status seek(PositionProvider* positions) override;
-    virtual Status skip(uint64_t row_count) override;
-    virtual Status next_vector(ColumnVector* column_vector, uint32_t size,
-                               MemPool* mem_pool) override;
-
-    virtual size_t get_buffer_size() override { return sizeof(RunLengthByteReader) * 2; }
-
-private:
-    bool _eof;
-    decimal12_t* _values;
-    RunLengthIntegerReader* _int_reader;
-    RunLengthIntegerReader* _frac_reader;
-};
-
-class LargeIntColumnReader : public ColumnReader {
-public:
-    LargeIntColumnReader(uint32_t column_id, uint32_t column_unique_id);
-    virtual ~LargeIntColumnReader();
-    virtual Status init(std::map<StreamName, ReadOnlyFileStream*>* streams, int size,
-                        MemPool* mem_pool, OlapReaderStatistics* stats);
-    virtual Status seek(PositionProvider* positions);
-    virtual Status skip(uint64_t row_count);
-    virtual Status next_vector(ColumnVector* column_vector, uint32_t size, MemPool* mem_pool);
-
-    virtual size_t get_buffer_size() { return sizeof(RunLengthByteReader) * 2; }
-
-private:
-    bool _eof;
-    int128_t* _values;
-    RunLengthIntegerReader* _high_reader;
-    RunLengthIntegerReader* _low_reader;
-};
-
-typedef FloatintPointColumnReader<float> FloatColumnReader;
-typedef FloatintPointColumnReader<double> DoubleColumnReader;
-typedef IntegerColumnReaderWrapper<int64_t, true> DiscreteDoubleColumnReader;
-
-// Use 3 bytes to store the date
-// Use IntegerColumnReader, truncated to 3 bytes length when returning data
-typedef IntegerColumnReaderWrapper<uint24_t, false> DateColumnReader;
-
-// Internal use LONG implementation
-typedef IntegerColumnReaderWrapper<uint64_t, false> DateTimeColumnReader;
-
-typedef IntegerColumnReaderWrapper<uint32_t, false> DateV2ColumnReader;
-
-} // namespace doris
-
-#endif // DORIS_BE_SRC_OLAP_ROWSET_COLUMN_READER_H
diff --git a/be/src/olap/rowset/column_writer.cpp b/be/src/olap/rowset/column_writer.cpp
deleted file mode 100644
index af212a6927..0000000000
--- a/be/src/olap/rowset/column_writer.cpp
+++ /dev/null
@@ -1,873 +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 "olap/rowset/column_writer.h"
-
-#include "olap/file_helper.h"
-#include "olap/rowset/bit_field_writer.h"
-
-namespace doris {
-
-ColumnWriter* ColumnWriter::create(uint32_t column_id, const TabletSchema& schema,
-                                   OutStreamFactory* stream_factory, size_t num_rows_per_row_block,
-                                   double bf_fpp) {
-    ColumnWriter* column_writer = nullptr;
-    const TabletColumn& column = schema.column(column_id);
-
-    switch (column.type()) {
-    case OLAP_FIELD_TYPE_BOOL:
-    case OLAP_FIELD_TYPE_TINYINT:
-    case OLAP_FIELD_TYPE_UNSIGNED_TINYINT: {
-        column_writer = new (std::nothrow)
-                ByteColumnWriter(column_id, stream_factory, column, num_rows_per_row_block, bf_fpp);
-        break;
-    }
-    case OLAP_FIELD_TYPE_SMALLINT: {
-        column_writer = new (std::nothrow) IntegerColumnWriterWrapper<int16_t, true>(
-                column_id, stream_factory, column, num_rows_per_row_block, bf_fpp);
-        break;
-    }
-    case OLAP_FIELD_TYPE_UNSIGNED_SMALLINT: {
-        column_writer = new (std::nothrow) IntegerColumnWriterWrapper<uint16_t, false>(
-                column_id, stream_factory, column, num_rows_per_row_block, bf_fpp);
-        break;
-    }
-    case OLAP_FIELD_TYPE_INT: {
-        column_writer = new (std::nothrow) IntegerColumnWriterWrapper<int32_t, true>(
-                column_id, stream_factory, column, num_rows_per_row_block, bf_fpp);
-        break;
-    }
-    case OLAP_FIELD_TYPE_UNSIGNED_INT: {
-        column_writer = new (std::nothrow) IntegerColumnWriterWrapper<uint32_t, false>(
-                column_id, stream_factory, column, num_rows_per_row_block, bf_fpp);
-        break;
-    }
-    case OLAP_FIELD_TYPE_BIGINT: {
-        column_writer = new (std::nothrow) IntegerColumnWriterWrapper<int64_t, true>(
-                column_id, stream_factory, column, num_rows_per_row_block, bf_fpp);
-        break;
-    }
-    case OLAP_FIELD_TYPE_UNSIGNED_BIGINT: {
-        column_writer = new (std::nothrow) IntegerColumnWriterWrapper<uint64_t, false>(
-                column_id, stream_factory, column, num_rows_per_row_block, bf_fpp);
-        break;
-    }
-    case OLAP_FIELD_TYPE_FLOAT: {
-        column_writer = new (std::nothrow) FloatColumnWriter(column_id, stream_factory, column,
-                                                             num_rows_per_row_block, bf_fpp);
-        break;
-    }
-    case OLAP_FIELD_TYPE_DOUBLE: {
-        column_writer = new (std::nothrow) DoubleColumnWriter(column_id, stream_factory, column,
-                                                              num_rows_per_row_block, bf_fpp);
-        break;
-    }
-    case OLAP_FIELD_TYPE_DISCRETE_DOUBLE: {
-        column_writer = new (std::nothrow) DiscreteDoubleColumnWriter(
-                column_id, stream_factory, column, num_rows_per_row_block, bf_fpp);
-        break;
-    }
-    case OLAP_FIELD_TYPE_CHAR: {
-        column_writer = new (std::nothrow) FixLengthStringColumnWriter(
-                column_id, stream_factory, column, num_rows_per_row_block, bf_fpp);
-        break;
-    }
-    case OLAP_FIELD_TYPE_DATETIME: {
-        column_writer = new (std::nothrow) DateTimeColumnWriter(column_id, stream_factory, column,
-                                                                num_rows_per_row_block, bf_fpp);
-        break;
-    }
-    case OLAP_FIELD_TYPE_DATE: {
-        column_writer = new (std::nothrow)
-                DateColumnWriter(column_id, stream_factory, column, num_rows_per_row_block, bf_fpp);
-        break;
-    }
-    case OLAP_FIELD_TYPE_DATEV2: {
-        column_writer = new (std::nothrow) DateV2ColumnWriter(column_id, stream_factory, column,
-                                                              num_rows_per_row_block, bf_fpp);
-        break;
-    }
-    case OLAP_FIELD_TYPE_DECIMAL: {
-        column_writer = new (std::nothrow) DecimalColumnWriter(column_id, stream_factory, column,
-                                                               num_rows_per_row_block, bf_fpp);
-        break;
-    }
-    case OLAP_FIELD_TYPE_LARGEINT: {
-        column_writer = new (std::nothrow) LargeIntColumnWriter(column_id, stream_factory, column,
-                                                                num_rows_per_row_block, bf_fpp);
-        break;
-    }
-    case OLAP_FIELD_TYPE_VARCHAR:
-    case OLAP_FIELD_TYPE_OBJECT:
-    case OLAP_FIELD_TYPE_HLL:
-    case OLAP_FIELD_TYPE_STRING: {
-        column_writer = new (std::nothrow) VarStringColumnWriter(column_id, stream_factory, column,
-                                                                 num_rows_per_row_block, bf_fpp);
-        break;
-    }
-    case OLAP_FIELD_TYPE_STRUCT:
-    case OLAP_FIELD_TYPE_ARRAY:
-    case OLAP_FIELD_TYPE_MAP:
-    default: {
-        LOG(WARNING) << "Unsupported field type. field=" << column.name()
-                     << ", type=" << column.type();
-        break;
-    }
-    }
-
-    return column_writer;
-}
-
-ColumnWriter::ColumnWriter(uint32_t column_id, OutStreamFactory* stream_factory,
-                           const TabletColumn& column, size_t num_rows_per_row_block, double bf_fpp)
-        : _column_id(column_id),
-          _column(column),
-          _stream_factory(stream_factory),
-          _index(column.type()),
-          _is_present(nullptr),
-          _is_present_stream(nullptr),
-          _index_stream(nullptr),
-          _is_found_nulls(false),
-          _bf(nullptr),
-          _num_rows_per_row_block(num_rows_per_row_block),
-          _bf_fpp(bf_fpp) {}
-
-ColumnWriter::~ColumnWriter() {
-    SAFE_DELETE(_is_present);
-    SAFE_DELETE(_bf);
-
-    for (std::vector<ColumnWriter*>::iterator it = _sub_writers.begin(); it != _sub_writers.end();
-         ++it) {
-        SAFE_DELETE(*it);
-    }
-}
-
-Status ColumnWriter::init() {
-    if (_column.is_nullable()) {
-        _is_present_stream =
-                _stream_factory->create_stream(unique_column_id(), StreamInfoMessage::PRESENT);
-
-        if (nullptr == _is_present_stream) {
-            LOG(WARNING) << "fail to allocate IS PRESENT STREAM";
-            return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
-        }
-
-        _is_present = new (std::nothrow) BitFieldWriter(_is_present_stream);
-
-        if (nullptr == _is_present) {
-            LOG(WARNING) << "fail to allocate IS PRESENT Writer";
-            return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
-        }
-
-        if (!_is_present->init()) {
-            LOG(WARNING) << "fail to init IS PRESENT Writer";
-            return Status::OLAPInternalError(OLAP_ERR_INIT_FAILED);
-        }
-    }
-
-    Status res = _block_statistics.init(_column.type(), true);
-
-    if (!res.ok()) {
-        LOG(WARNING) << "init block statistic failed";
-        return res;
-    }
-
-    res = _segment_statistics.init(_column.type(), true);
-
-    if (!res.ok()) {
-        LOG(WARNING) << "init segment statistic failed";
-        return res;
-    }
-
-    _index_stream =
-            _stream_factory->create_stream(unique_column_id(), StreamInfoMessage::ROW_INDEX);
-
-    if (nullptr == _index_stream) {
-        LOG(WARNING) << "fail to allocate Index STREAM";
-        return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
-    }
-
-    // bloom filter index
-    if (is_bf_column()) {
-        _bf_index_stream =
-                _stream_factory->create_stream(unique_column_id(), StreamInfoMessage::BLOOM_FILTER);
-        if (nullptr == _bf_index_stream) {
-            LOG(WARNING) << "fail to allocate bloom filter index stream";
-            return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
-        }
-
-        _bf = new (std::nothrow) BloomFilter();
-        if (nullptr == _bf) {
-            LOG(WARNING) << "fail to allocate bloom filter";
-            return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
-        }
-
-        if (!_bf->init(_num_rows_per_row_block, _bf_fpp)) {
-            LOG(WARNING) << "fail to init bloom filter. num rows: " << _num_rows_per_row_block
-                         << ", fpp: " << _bf_fpp;
-            return Status::OLAPInternalError(OLAP_ERR_INIT_FAILED);
-        }
-    }
-
-    return Status::OK();
-}
-
-Status ColumnWriter::write(RowCursor* row_cursor) {
-    Status res = Status::OK();
-
-    bool is_null = row_cursor->is_null(_column_id);
-    char* buf = row_cursor->cell_ptr(_column_id);
-    if (_is_present) {
-        res = _is_present->write(is_null);
-
-        if (is_null) {
-            _is_found_nulls = true;
-        }
-    }
-
-    if (is_bf_column()) {
-        if (!is_null) {
-            if (_column.type() == OLAP_FIELD_TYPE_CHAR ||
-                _column.type() == OLAP_FIELD_TYPE_VARCHAR ||
-                _column.type() == OLAP_FIELD_TYPE_HLL || _column.type() == OLAP_FIELD_TYPE_STRING) {
-                Slice* slice = reinterpret_cast<Slice*>(buf);
-                _bf->add_bytes(slice->data, slice->size);
-            } else {
-                _bf->add_bytes(buf, row_cursor->column_size(_column_id));
-            }
-        } else {
-            _bf->add_bytes(nullptr, 0);
-        }
-    }
-
-    return res;
-}
-
-Status ColumnWriter::flush() {
-    return _is_present->flush();
-}
-
-Status ColumnWriter::create_row_index_entry() {
-    Status res = Status::OK();
-    segment_statistics()->merge(&_block_statistics);
-    _index_entry.set_statistic(&_block_statistics);
-    _index.add_index_entry(_index_entry);
-    _index_entry.reset_write_offset();
-    _block_statistics.reset();
-    record_position();
-
-    if (is_bf_column()) {
-        _bf_index.add_bloom_filter(_bf);
-
-        _bf = new (std::nothrow) BloomFilter();
-        if (nullptr == _bf) {
-            LOG(WARNING) << "fail to allocate bloom filter";
-            return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
-        }
-
-        if (!_bf->init(_num_rows_per_row_block, _bf_fpp)) {
-            LOG(WARNING) << "fail to init bloom filter. num rows: " << _num_rows_per_row_block
-                         << ", fpp: " << _bf_fpp;
-            return Status::OLAPInternalError(OLAP_ERR_INIT_FAILED);
-        }
-    }
-
-    for (std::vector<ColumnWriter*>::iterator it = _sub_writers.begin(); it != _sub_writers.end();
-         ++it) {
-        if (!(res = (*it)->create_row_index_entry())) {
-            LOG(WARNING) << "fail to create sub column's index.";
-            return res;
-        }
-    }
-
-    return res;
-}
-
-uint64_t ColumnWriter::estimate_buffered_memory() {
-    uint64_t result = 0;
-
-    // bloom filter
-    if (is_bf_column()) {
-        result += _bf_index.estimate_buffered_memory();
-    }
-
-    for (std::vector<ColumnWriter*>::iterator it = _sub_writers.begin(); it != _sub_writers.end();
-         ++it) {
-        result += (*it)->estimate_buffered_memory();
-    }
-
-    return result;
-}
-
-// Delete the positions used by is_present_stream:
-// * OutStream uses 2
-// * ByteRunLength uses 1
-// * BitRunLength uses 1
-// Delete 4 in total
-void ColumnWriter::_remove_is_present_positions() {
-    for (uint32_t i = 0; i < _index.entry_size(); i++) {
-        PositionEntryWriter* entry = _index.mutable_entry(i);
-        entry->remove_written_position(0, 4);
-    }
-}
-
-Status ColumnWriter::finalize(ColumnDataHeaderMessage* header) {
-    Status res = Status::OK();
-
-    if (nullptr != _is_present) {
-        if (!(res = _is_present->flush())) {
-            return res;
-        }
-
-        if (!_is_found_nulls) {
-            _is_present_stream->suppress();
-            _remove_is_present_positions();
-        }
-    }
-
-    char* index_buf = nullptr;
-    // char* index_statistic_buf = NULL;
-    // Write index pb
-    size_t pb_size = _index.output_size();
-    index_buf = new (std::nothrow) char[pb_size];
-    ColumnMessage* column = nullptr;
-
-    if (!_index.write_to_buffer(index_buf, pb_size)) {
-        LOG(WARNING) << "fail to serialize index";
-        res = Status::OLAPInternalError(OLAP_ERR_SERIALIZE_PROTOBUF_ERROR);
-        goto FINALIZE_EXIT;
-    }
-
-    res = _index_stream->write(index_buf, pb_size);
-
-    if (!res.ok()) {
-        LOG(WARNING) << "fail to write index to stream";
-        goto FINALIZE_EXIT;
-    }
-
-    res = _index_stream->flush();
-
-    if (!res.ok()) {
-        LOG(WARNING) << "fail to flush index stream";
-        goto FINALIZE_EXIT;
-    }
-
-    // write bloom filter index
-    if (is_bf_column()) {
-        res = _bf_index.write_to_buffer(_bf_index_stream);
-        if (!res.ok()) {
-            LOG(WARNING) << "fail to write bloom filter stream";
-            OLAP_GOTO(FINALIZE_EXIT);
-        }
-
-        res = _bf_index_stream->flush();
-        if (!res.ok()) {
-            LOG(WARNING) << "fail to flush bloom filter stream";
-            OLAP_GOTO(FINALIZE_EXIT);
-        }
-    }
-
-    // Record a Schema information in the Segment header
-    // This makes it not affect the reading of the data in the existing segment after modifying the schema of the table
-    column = header->add_column();
-    column->set_name(_column.name());
-    column->set_type(TabletColumn::get_string_by_field_type(_column.type()));
-    column->set_aggregation(TabletColumn::get_string_by_aggregation_type(_column.aggregation()));
-    column->set_length(_column.length());
-    column->set_is_key(_column.is_key());
-    column->set_precision(_column.precision());
-    column->set_frac(_column.frac());
-    column->set_unique_id(_column.unique_id());
-    column->set_is_bf_column(is_bf_column());
-
-    save_encoding(header->add_column_encoding());
-
-FINALIZE_EXIT:
-    SAFE_DELETE_ARRAY(index_buf);
-    // SAFE_DELETE_ARRAY(index_statistic_buf);
-    return res;
-}
-
-void ColumnWriter::record_position() {
-    if (nullptr != _is_present) {
-        _is_present->get_position(&_index_entry);
-    }
-}
-
-// The default returns DIRECT, String type may return Dict
-void ColumnWriter::save_encoding(ColumnEncodingMessage* encoding) {
-    encoding->set_kind(ColumnEncodingMessage::DIRECT);
-}
-
-void ColumnWriter::get_bloom_filter_info(bool* has_bf_column, uint32_t* bf_hash_function_num,
-                                         uint32_t* bf_bit_num) {
-    if (is_bf_column()) {
-        *has_bf_column = true;
-        *bf_hash_function_num = _bf->hash_function_num();
-        *bf_bit_num = _bf->bit_num();
-        return;
-    }
-
-    for (std::vector<ColumnWriter*>::iterator it = _sub_writers.begin(); it != _sub_writers.end();
-         ++it) {
-        (*it)->get_bloom_filter_info(has_bf_column, bf_hash_function_num, bf_bit_num);
-        if (*has_bf_column) {
-            return;
-        }
-    }
-}
-
-////////////////////////////////////////////////////////////////////////////////
-ByteColumnWriter::ByteColumnWriter(uint32_t column_id, OutStreamFactory* stream_factory,
-                                   const TabletColumn& column, size_t num_rows_per_row_block,
-                                   double bf_fpp)
-        : ColumnWriter(column_id, stream_factory, column, num_rows_per_row_block, bf_fpp),
-          _writer(nullptr) {}
-
-ByteColumnWriter::~ByteColumnWriter() {
-    SAFE_DELETE(_writer);
-}
-
-Status ByteColumnWriter::init() {
-    Status res = Status::OK();
-
-    if (!(res = ColumnWriter::init())) {
-        return res;
-    }
-
-    OutStreamFactory* factory = stream_factory();
-    OutStream* stream = factory->create_stream(unique_column_id(), StreamInfoMessage::DATA);
-
-    if (nullptr == stream) {
-        LOG(WARNING) << "fail to allocate DATA STREAM";
-        return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
-    }
-
-    _writer = new (std::nothrow) RunLengthByteWriter(stream);
-
-    if (nullptr == _writer) {
-        LOG(WARNING) << "fail to allocate RunLengthByteWriter";
-        return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
-    }
-
-    record_position();
-    return Status::OK();
-}
-
-Status ByteColumnWriter::finalize(ColumnDataHeaderMessage* header) {
-    Status res = Status::OK();
-
-    if (!(res = ColumnWriter::finalize(header))) {
-        LOG(WARNING) << "fail to finalize ColumnWriter.";
-        return res;
-    }
-
-    if (!(res = _writer->flush())) {
-        LOG(WARNING) << "fail to flush.";
-        return res;
-    }
-
-    return Status::OK();
-}
-
-void ByteColumnWriter::record_position() {
-    ColumnWriter::record_position();
-    _writer->get_position(index_entry());
-}
-
-////////////////////////////////////////////////////////////////////////////////
-
-IntegerColumnWriter::IntegerColumnWriter(uint32_t column_id, uint32_t unique_column_id,
-                                         OutStreamFactory* stream_factory, bool is_singed)
-        : _unique_column_id(unique_column_id),
-          _stream_factory(stream_factory),
-          _writer(nullptr),
-          _is_signed(is_singed) {}
-
-IntegerColumnWriter::~IntegerColumnWriter() {
-    SAFE_DELETE(_writer);
-}
-
-Status IntegerColumnWriter::init() {
-    OutStream* stream = _stream_factory->create_stream(_unique_column_id, StreamInfoMessage::DATA);
-
-    if (nullptr == stream) {
-        LOG(WARNING) << "fail to allocate DATA STREAM";
-        return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
-    }
-
-    _writer = new (std::nothrow) RunLengthIntegerWriter(stream, _is_signed);
-
-    if (nullptr == _writer) {
-        LOG(WARNING) << "fail to allocate RunLengthIntegerWriter";
-        return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
-    }
-
-    return Status::OK();
-}
-
-////////////////////////////////////////////////////////////////////////////////
-
-VarStringColumnWriter::VarStringColumnWriter(uint32_t column_id, OutStreamFactory* stream_factory,
-                                             const TabletColumn& column,
-                                             size_t num_rows_per_row_block, double bf_fpp)
-        : ColumnWriter(column_id, stream_factory, column, num_rows_per_row_block, bf_fpp),
-          _use_dictionary_encoding(false),
-          _dict_total_size(0),
-          _dict_stream(nullptr),
-          _length_writer(nullptr),
-          _data_stream(nullptr),
-          _id_writer(nullptr) {}
-
-VarStringColumnWriter::~VarStringColumnWriter() {
-    SAFE_DELETE(_length_writer);
-    SAFE_DELETE(_id_writer);
-}
-
-Status VarStringColumnWriter::init() {
-    Status res = Status::OK();
-
-    if (!(res = ColumnWriter::init())) {
-        return res;
-    }
-
-    _dict_stream =
-            stream_factory()->create_stream(unique_column_id(), StreamInfoMessage::DICTIONARY_DATA);
-    _data_stream = stream_factory()->create_stream(unique_column_id(), StreamInfoMessage::DATA);
-    OutStream* length_stream =
-            stream_factory()->create_stream(unique_column_id(), StreamInfoMessage::LENGTH);
-
-    if (nullptr == _dict_stream || nullptr == length_stream || nullptr == _data_stream) {
-        LOG(WARNING) << "fail to create stream.";
-        return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
-    }
-
-    _length_writer = new (std::nothrow) RunLengthIntegerWriter(length_stream, false);
-    _id_writer = new (std::nothrow) RunLengthIntegerWriter(_data_stream, false);
-
-    if (nullptr == _length_writer || nullptr == _id_writer) {
-        LOG(WARNING) << "fail to create writer.";
-        return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
-    }
-
-    record_position();
-    return Status::OK();
-}
-
-Status VarStringColumnWriter::write(const char* str, uint32_t len) {
-    Status res = Status::OK();
-    // zdb shield the dictionary coding
-    //std::string key(str, len);
-
-    if (!(res = _data_stream->write(str, len))) {
-        LOG(WARNING) << "fail to write string content.";
-        return res;
-    }
-
-    if (!(res = _length_writer->write(len))) {
-        LOG(WARNING) << "fail to write string length.";
-        return res;
-    }
-
-    return Status::OK();
-}
-
-uint64_t VarStringColumnWriter::estimate_buffered_memory() {
-    // the length of _string_id is short after RLE
-    return _dict_total_size;
-}
-
-Status VarStringColumnWriter::_finalize_dict_encoding() {
-    Status res = Status::OK();
-    std::vector<uint32_t> dump_order;
-    uint32_t current_id = 0;
-
-    dump_order.resize(_string_keys.size());
-
-    for (StringDict::iterator it = _string_dict.begin(); it != _string_dict.end(); ++it) {
-        dump_order[it->second] = current_id;
-        current_id++;
-        const std::string& key = it->first.get();
-        res = _dict_stream->write(key.c_str(), key.length());
-
-        if (!res.ok()) {
-            LOG(WARNING) << "fail to write string dict to stream.";
-            return res;
-        }
-
-        if (!(res = _length_writer->write(key.length()))) {
-            LOG(WARNING) << "fail to write string length to stream.";
-            return res;
-        }
-    }
-
-    uint32_t block_id = 0;
-
-    // Suppose there are n ids in total. (total)
-    for (uint32_t i = 0; i <= _string_id.size(); i++) {
-        while (block_id < _block_row_count.size() - 1 && i == _block_row_count[block_id]) {
-            _id_writer->get_position(index()->mutable_entry(block_id), false);
-            block_id++;
-        }
-
-        if (i != _string_id.size()) {
-            res = _id_writer->write(dump_order[_string_id[i]]);
-
-            if (!res.ok()) {
-                LOG(WARNING) << "fail to write string id to stream.";
-                return res;
-            }
-        }
-    }
-
-    return Status::OK();
-}
-
-Status VarStringColumnWriter::_finalize_direct_encoding() {
-    //Status res = Status::OK();
-    //uint32_t block_id = 0;
-
-    _dict_stream->suppress();
-    return Status::OK();
-}
-
-Status VarStringColumnWriter::finalize(ColumnDataHeaderMessage* header) {
-    Status res = Status::OK();
-    uint64_t ratio_threshold = config::column_dictionary_key_ratio_threshold;
-    uint64_t size_threshold = config::column_dictionary_key_size_threshold;
-
-    // the dictionary condition:1 key size < size threshold; 2 key ratio < ratio threshold
-    _use_dictionary_encoding = (_string_keys.size() < size_threshold) &&
-                               (_string_keys.size() * 100UL < _string_id.size() * ratio_threshold);
-
-    if (_use_dictionary_encoding) {
-        res = _finalize_dict_encoding();
-        if (!res.ok()) {
-            LOG(WARNING) << "fail to finalize dict encoding.";
-            return res;
-        }
-    } else {
-        res = _finalize_direct_encoding();
-        if (!res.ok()) {
-            LOG(WARNING) << "fail to finalize direct encoding.";
-            return res;
-        }
-    }
-
-    // The index's supplementary writing has been completed, ColumnWriter::finalize will write the header
-    res = ColumnWriter::finalize(header);
-    if (!res.ok()) {
-        LOG(WARNING) << "fail to finalize ColumnWriter.";
-        return res;
-    }
-
-    // id_writer is practical to data_stream, it doesn't matter if you repeat flush
-    if (!_length_writer->flush() || !_id_writer->flush() || !_dict_stream->flush() ||
-        !_data_stream->flush()) {
-        LOG(WARNING) << "fail to flush stream.";
-        return Status::OLAPInternalError(OLAP_ERR_WRITER_DATA_WRITE_ERROR);
-    }
-
-    _string_keys.clear();
-    _string_dict.clear();
-    _string_id.clear();
-    _block_row_count.clear();
-    _dict_total_size = 0;
-    return Status::OK();
-}
-
-void VarStringColumnWriter::save_encoding(ColumnEncodingMessage* encoding) {
-    if (_use_dictionary_encoding) {
-        encoding->set_kind(ColumnEncodingMessage::DICTIONARY);
-        encoding->set_dictionary_size(_string_keys.size());
-    } else {
-        encoding->set_kind(ColumnEncodingMessage::DIRECT);
-    }
-}
-
-// Unlike other Writer, data is written to Stream only when it is finalized.
-// So it is impossible to record the position of the stream. For this reason, record the number of data written in each block, and when finalize
-// Use this information to add stream location information to Index
-void VarStringColumnWriter::record_position() {
-    ColumnWriter::record_position();
-    _block_row_count.push_back(_string_id.size());
-    //zdb shield dictionary coding
-    _data_stream->get_position(index_entry());
-    _length_writer->get_position(index_entry(), false);
-}
-
-////////////////////////////////////////////////////////////////////////////////
-
-FixLengthStringColumnWriter::FixLengthStringColumnWriter(uint32_t column_id,
-                                                         OutStreamFactory* stream_factory,
-                                                         const TabletColumn& column,
-                                                         size_t num_rows_per_row_block,
-                                                         double bf_fpp)
-        : VarStringColumnWriter(column_id, stream_factory, column, num_rows_per_row_block, bf_fpp),
-          _length(column.length()) {}
-
-FixLengthStringColumnWriter::~FixLengthStringColumnWriter() {}
-
-////////////////////////////////////////////////////////////////////////////////
-
-DecimalColumnWriter::DecimalColumnWriter(uint32_t column_id, OutStreamFactory* stream_factory,
-                                         const TabletColumn& column, size_t num_rows_per_row_block,
-                                         double bf_fpp)
-        : ColumnWriter(column_id, stream_factory, column, num_rows_per_row_block, bf_fpp),
-          _int_writer(nullptr),
-          _frac_writer(nullptr) {}
-
-DecimalColumnWriter::~DecimalColumnWriter() {
-    SAFE_DELETE(_int_writer);
-    SAFE_DELETE(_frac_writer);
-}
-
-Status DecimalColumnWriter::init() {
-    Status res = Status::OK();
-
-    res = ColumnWriter::init();
-    if (!res.ok()) {
-        return res;
-    }
-
-    OutStream* int_stream =
-            stream_factory()->create_stream(unique_column_id(), StreamInfoMessage::DATA);
-    OutStream* frac_stream =
-            stream_factory()->create_stream(unique_column_id(), StreamInfoMessage::SECONDARY);
-
-    if (nullptr == int_stream || nullptr == frac_stream) {
-        LOG(WARNING) << "fail to create stream.";
-        return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
-    }
-
-    _int_writer = new (std::nothrow) RunLengthIntegerWriter(int_stream, true);
-    _frac_writer = new (std::nothrow) RunLengthIntegerWriter(frac_stream, true);
-
-    if (nullptr == _int_writer || nullptr == _frac_writer) {
-        LOG(WARNING) << "fail to create writer.";
-        return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
-    }
-
-    record_position();
-    return Status::OK();
-}
-
-Status DecimalColumnWriter::finalize(ColumnDataHeaderMessage* header) {
-    Status res;
-
-    res = ColumnWriter::finalize(header);
-    if (!res.ok()) {
-        LOG(WARNING) << "fail to finalize ColumnWriter.";
-        return res;
-    }
-
-    res = _int_writer->flush();
-    if (!res.ok()) {
-        LOG(WARNING) << "fail to flush integer writer.";
-        return res;
-    }
-
-    res = _frac_writer->flush();
-    if (!res.ok()) {
-        LOG(WARNING) << "fail to flush fraction writer.";
-        return res;
-    }
-
-    return Status::OK();
-}
-
-void DecimalColumnWriter::record_position() {
-    ColumnWriter::record_position();
-    _int_writer->get_position(index_entry(), false);
-    _frac_writer->get_position(index_entry(), false);
-}
-
-////////////////////////////////////////////////////////////////////////////////
-
-LargeIntColumnWriter::LargeIntColumnWriter(uint32_t column_id, OutStreamFactory* stream_factory,
-                                           const TabletColumn& column,
-                                           size_t num_rows_per_row_block, double bf_fpp)
-        : ColumnWriter(column_id, stream_factory, column, num_rows_per_row_block, bf_fpp),
-          _high_writer(nullptr),
-          _low_writer(nullptr) {}
-
-LargeIntColumnWriter::~LargeIntColumnWriter() {
-    SAFE_DELETE(_high_writer);
-    SAFE_DELETE(_low_writer);
-}
-
-Status LargeIntColumnWriter::init() {
-    Status res = Status::OK();
-
-    res = ColumnWriter::init();
-    if (!res.ok()) {
-        return res;
-    }
-
-    OutStream* high_stream =
-            stream_factory()->create_stream(unique_column_id(), StreamInfoMessage::DATA);
-    OutStream* low_stream =
-            stream_factory()->create_stream(unique_column_id(), StreamInfoMessage::SECONDARY);
-
-    if (nullptr == high_stream || nullptr == low_stream) {
-        LOG(WARNING) << "fail to create stream.";
-        return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
-    }
-
-    _high_writer = new (std::nothrow) RunLengthIntegerWriter(high_stream, true);
-    _low_writer = new (std::nothrow) RunLengthIntegerWriter(low_stream, true);
-
-    if (nullptr == _high_writer || nullptr == _low_writer) {
-        LOG(WARNING) << "fail to create writer.";
-        return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
-    }
-
-    record_position();
-    return Status::OK();
-}
-
-Status LargeIntColumnWriter::finalize(ColumnDataHeaderMessage* header) {
-    Status res;
-
-    res = ColumnWriter::finalize(header);
-    if (!res.ok()) {
-        LOG(WARNING) << "fail to finalize ColumnWriter.";
-        return res;
-    }
-
-    res = _high_writer->flush();
-    if (!res.ok()) {
-        LOG(WARNING) << "fail to flush integer writer.";
-        return res;
-    }
-
-    res = _low_writer->flush();
-    if (!res.ok()) {
-        LOG(WARNING) << "fail to flush fraction writer.";
-        return res;
-    }
-
-    return Status::OK();
-}
-
-void LargeIntColumnWriter::record_position() {
-    ColumnWriter::record_position();
-    _high_writer->get_position(index_entry(), false);
-    _low_writer->get_position(index_entry(), false);
-}
-
-} // namespace doris
diff --git a/be/src/olap/rowset/column_writer.h b/be/src/olap/rowset/column_writer.h
deleted file mode 100644
index 7771189e74..0000000000
--- a/be/src/olap/rowset/column_writer.h
+++ /dev/null
@@ -1,587 +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.
-
-#ifndef DORIS_BE_SRC_OLAP_ROWSET_COLUMN_WRITER_H
-#define DORIS_BE_SRC_OLAP_ROWSET_COLUMN_WRITER_H
-
-#include <gen_cpp/column_data_file.pb.h>
-
-#include <map>
-
-#include "olap/bloom_filter.hpp"
-#include "olap/bloom_filter_writer.h"
-#include "olap/field.h"
-#include "olap/olap_common.h"
-#include "olap/olap_define.h"
-#include "olap/out_stream.h"
-#include "olap/row_block.h"
-#include "olap/row_cursor.h"
-#include "olap/rowset/run_length_byte_writer.h"
-#include "olap/rowset/run_length_integer_writer.h"
-#include "olap/stream_index_writer.h"
-
-namespace doris {
-
-class OutStream;
-class OutStreamFactory;
-class ColumnStatistics;
-class BitFieldWriter;
-class RunLengthByteWriter;
-class RunLengthIntegerWriter;
-
-class ColumnWriter {
-public:
-    // Create a ColumnWriter, the lifetime of the object after creation is owned by the caller
-    // That is, the caller is responsible for calling delete to destruct the ColumnWriter
-    // Args:
-    //      column_id: the position of the created column in columns
-    //      columns: all column information of the table
-    //      stream_factory: The factory object used to create the output stream, the lifetime of the object is owned by the caller
-    static ColumnWriter* create(uint32_t column_id, const TabletSchema& schema,
-                                OutStreamFactory* stream_factory, size_t num_rows_per_row_block,
-                                double bf_fpp);
-
-    virtual ~ColumnWriter();
-    virtual Status init();
-
-    Status write(RowCursor* cursor);
-
-    virtual Status write_batch(RowBlock* block, RowCursor* cursor) = 0;
-
-    // Write the previously recorded block location information and current statistical information into a new index entry
-    Status create_row_index_entry();
-    // Estimate the current cache memory size, excluding the memory that has been output to OutStream
-    virtual uint64_t estimate_buffered_memory();
-    virtual Status flush();
-    // End the segment, flush stream and update the header:
-    //   * column_unique_id
-    //   * column_type
-    //   * column_encoding
-    //   * zone_maps
-    virtual Status finalize(ColumnDataHeaderMessage* header);
-    virtual void save_encoding(ColumnEncodingMessage* encoding);
-    uint32_t column_id() const { return _column_id; }
-
-    uint32_t unique_column_id() const { return _column.unique_id(); }
-
-    virtual void get_bloom_filter_info(bool* has_bf_column, uint32_t* bf_hash_function_num,
-                                       uint32_t* bf_bit_num);
-
-    ColumnStatistics* segment_statistics() { return &_segment_statistics; }
-
-    ColumnStatistics* block_statistics() { return &_block_statistics; }
-
-protected:
-    ColumnWriter(uint32_t column_id, OutStreamFactory* stream_factory, const TabletColumn& column,
-                 size_t num_rows_per_row_block, double bf_fpp);
-
-    OutStreamFactory* stream_factory() { return _stream_factory; }
-    PositionEntryWriter* index_entry() { return &_index_entry; }
-    StreamIndexWriter* index() { return &_index; }
-    // Record the position of the current Stream, which is used to generate index entries
-    virtual void record_position();
-
-protected:
-    ColumnStatistics _block_statistics;
-    ColumnStatistics _segment_statistics;
-
-private:
-    void _remove_is_present_positions();
-
-    bool is_bf_column() { return _column.is_bf_column(); }
-
-    uint32_t _column_id;
-    const TabletColumn& _column;
-    OutStreamFactory* _stream_factory;       // The object is owned by the external caller
-    std::vector<ColumnWriter*> _sub_writers; // Writer to save the sub-column
-    PositionEntryWriter _index_entry;
-    StreamIndexWriter _index;
-    BitFieldWriter* _is_present; //Record NULL Bits for columns that allow NULL
-    OutStream* _is_present_stream;
-    OutStream* _index_stream; // Note that the ownership of the object is _stream_factory
-    bool _is_found_nulls;
-    BloomFilter* _bf;
-    BloomFilterIndexWriter _bf_index;
-    OutStream* _bf_index_stream;
-    size_t _num_rows_per_row_block;
-    double _bf_fpp;
-
-    DISALLOW_COPY_AND_ASSIGN(ColumnWriter);
-};
-
-class ByteColumnWriter : public ColumnWriter {
-public:
-    ByteColumnWriter(uint32_t column_id, OutStreamFactory* stream_factory,
-                     const TabletColumn& column, size_t num_rows_per_row_block, double bf_fpp);
-    virtual ~ByteColumnWriter();
-    virtual Status init() override;
-
-    Status write_batch(RowBlock* block, RowCursor* cursor) override {
-        for (uint32_t i = 0; i < block->row_block_info().row_num; i++) {
-            block->get_row(i, cursor);
-
-            Status res = ColumnWriter::write(cursor);
-            if (OLAP_UNLIKELY(!res.ok())) {
-                LOG(WARNING) << "fail to write ColumnWriter.";
-                return res;
-            }
-
-            auto cell = cursor->cell(column_id());
-            _block_statistics.add(cell);
-            if (!cell.is_null()) {
-                char value = *reinterpret_cast<const char*>(cell.cell_ptr());
-                res = _writer->write(value);
-                if (!res.ok()) {
-                    LOG(WARNING) << "fail to write double, res=" << res;
-                    return res;
-                }
-            }
-        }
-        return Status::OK();
-    }
-
-    virtual Status finalize(ColumnDataHeaderMessage* header) override;
-    virtual void record_position() override;
-    virtual Status flush() override { return Status::OK(); }
-
-private:
-    RunLengthByteWriter* _writer;
-
-    DISALLOW_COPY_AND_ASSIGN(ByteColumnWriter);
-};
-
-// For SHORT/INT/LONG type data, use int64 as the stored data uniformly
-class IntegerColumnWriter {
-public:
-    IntegerColumnWriter(uint32_t column_id, uint32_t unique_column_id,
-                        OutStreamFactory* stream_factory, bool is_singed);
-    ~IntegerColumnWriter();
... 17293 lines suppressed ...


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