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/18 00:45:54 UTC

[doris] branch master updated: [refactor] Remove alpha rowset meta (#10933)

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 523d395527 [refactor] Remove alpha rowset meta (#10933)
523d395527 is described below

commit 523d3955271b6b48de7b4135b49d8c8297743f41
Author: plat1ko <pl...@gmail.com>
AuthorDate: Mon Jul 18 08:45:46 2022 +0800

    [refactor] Remove alpha rowset meta (#10933)
    
    * remove alpha_rowset_meta
    * remove alpha rowset related codes in compaction
    * remove alpha rowset related codes in RowsetMeta
    * fix be ut because some ut use alpha rowsetmeta
---
 be/src/olap/compaction.cpp                         |  35 +------
 be/src/olap/compaction.h                           |   5 -
 be/src/olap/data_dir.cpp                           |   3 +-
 be/src/olap/rowset/CMakeLists.txt                  |   1 -
 be/src/olap/rowset/alpha_rowset_meta.cpp           |  48 ---------
 be/src/olap/rowset/alpha_rowset_meta.h             |  43 --------
 be/src/olap/rowset/rowset_meta.h                   |  28 ------
 be/src/olap/snapshot_manager.cpp                   |  18 +---
 be/src/olap/storage_engine.cpp                     |   3 +-
 be/src/olap/tablet_meta.cpp                        |   5 +-
 be/test/CMakeLists.txt                             |   6 +-
 be/test/olap/cumulative_compaction_policy_test.cpp | 110 +--------------------
 be/test/olap/rowset/rowset_meta_test.cpp           |  82 +--------------
 be/test/olap/tablet_mgr_test.cpp                   |   1 -
 be/test/olap/test_data/rowset.json                 |  57 +----------
 be/test/olap/test_data/rowset_meta.json            |  33 +------
 be/test/olap/test_data/rowset_meta2.json           |   2 +-
 be/test/olap/txn_manager_test.cpp                  |  65 ++++++------
 18 files changed, 51 insertions(+), 494 deletions(-)

diff --git a/be/src/olap/compaction.cpp b/be/src/olap/compaction.cpp
index e06e5a9b00..fa1b1ba326 100644
--- a/be/src/olap/compaction.cpp
+++ b/be/src/olap/compaction.cpp
@@ -306,44 +306,11 @@ Status Compaction::check_correctness(const Merger::Statistics& stats) {
                      << ", 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 compaction
-        // 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();
-        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::OLAPInternalError(OLAP_ERR_CHECK_LINES_ERROR);
     }
     return Status::OK();
 }
 
-int64_t Compaction::_get_input_num_rows_from_seg_grps() {
-    int64_t num_rows = 0;
-    for (auto& rowset : _input_rowsets) {
-        if (rowset->rowset_meta()->rowset_type() != RowsetTypePB::ALPHA_ROWSET) {
-            return -1;
-        }
-        for (auto& seg_grp : rowset->rowset_meta()->alpha_rowset_extra_meta_pb().segment_groups()) {
-            num_rows += seg_grp.num_rows();
-        }
-    }
-    return num_rows;
-}
-
 int64_t Compaction::get_compaction_permits() {
     int64_t permits = 0;
     for (auto rowset : _input_rowsets) {
diff --git a/be/src/olap/compaction.h b/be/src/olap/compaction.h
index 648ff78935..000b309e43 100644
--- a/be/src/olap/compaction.h
+++ b/be/src/olap/compaction.h
@@ -76,11 +76,6 @@ protected:
                                             std::vector<Version>* missing_version);
     int64_t get_compaction_permits();
 
-private:
-    // get num rows from segment group meta of input rowsets.
-    // return -1 if these are not alpha rowsets.
-    int64_t _get_input_num_rows_from_seg_grps();
-
 protected:
     // the root tracker for this compaction
     std::shared_ptr<MemTracker> _mem_tracker;
diff --git a/be/src/olap/data_dir.cpp b/be/src/olap/data_dir.cpp
index afc56f9d15..0f7acc749f 100644
--- a/be/src/olap/data_dir.cpp
+++ b/be/src/olap/data_dir.cpp
@@ -40,7 +40,6 @@
 #include "io/fs/path.h"
 #include "olap/file_helper.h"
 #include "olap/olap_define.h"
-#include "olap/rowset/alpha_rowset_meta.h"
 #include "olap/rowset/rowset_meta_manager.h"
 #include "olap/storage_engine.h"
 #include "olap/tablet_meta_manager.h"
@@ -366,7 +365,7 @@ Status DataDir::load() {
     auto load_rowset_func = [&dir_rowset_metas, &local_fs = fs()](
                                     TabletUid tablet_uid, RowsetId rowset_id,
                                     const std::string& meta_str) -> bool {
-        RowsetMetaSharedPtr rowset_meta(new AlphaRowsetMeta());
+        RowsetMetaSharedPtr rowset_meta(new RowsetMeta());
         bool parsed = rowset_meta->init(meta_str);
         if (!parsed) {
             LOG(WARNING) << "parse rowset meta string failed for rowset_id:" << rowset_id;
diff --git a/be/src/olap/rowset/CMakeLists.txt b/be/src/olap/rowset/CMakeLists.txt
index cd5a61636d..72ae74ec56 100644
--- a/be/src/olap/rowset/CMakeLists.txt
+++ b/be/src/olap/rowset/CMakeLists.txt
@@ -26,7 +26,6 @@ add_library(Rowset STATIC
     rowset.cpp
     rowset_factory.cpp
     rowset_meta_manager.cpp
-    alpha_rowset_meta.cpp
     beta_rowset.cpp
     beta_rowset_reader.cpp
     beta_rowset_writer.cpp
diff --git a/be/src/olap/rowset/alpha_rowset_meta.cpp b/be/src/olap/rowset/alpha_rowset_meta.cpp
deleted file mode 100644
index 4f6395ae03..0000000000
--- a/be/src/olap/rowset/alpha_rowset_meta.cpp
+++ /dev/null
@@ -1,48 +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_meta.h"
-
-#include "common/logging.h"
-
-namespace doris {
-
-void AlphaRowsetMeta::get_segment_groups(std::vector<SegmentGroupPB>* segment_groups) {
-    if (!_has_alpha_rowset_extra_meta_pb()) {
-        return;
-    }
-    const AlphaRowsetExtraMetaPB& alpha_rowset_extra_meta = alpha_rowset_extra_meta_pb();
-    for (auto& segment_group : alpha_rowset_extra_meta.segment_groups()) {
-        segment_groups->push_back(segment_group);
-    }
-}
-
-void AlphaRowsetMeta::add_segment_group(const SegmentGroupPB& segment_group) {
-    AlphaRowsetExtraMetaPB* alpha_rowset_extra_meta_pb = _mutable_alpha_rowset_extra_meta_pb();
-    SegmentGroupPB* new_segment_group = alpha_rowset_extra_meta_pb->add_segment_groups();
-    *new_segment_group = segment_group;
-}
-
-void AlphaRowsetMeta::clear_segment_group() {
-    if (!_has_alpha_rowset_extra_meta_pb()) {
-        return;
-    }
-    AlphaRowsetExtraMetaPB* alpha_rowset_extra_meta_pb = _mutable_alpha_rowset_extra_meta_pb();
-    alpha_rowset_extra_meta_pb->clear_segment_groups();
-}
-
-} // namespace doris
diff --git a/be/src/olap/rowset/alpha_rowset_meta.h b/be/src/olap/rowset/alpha_rowset_meta.h
deleted file mode 100644
index 2d87d7ba1b..0000000000
--- a/be/src/olap/rowset/alpha_rowset_meta.h
+++ /dev/null
@@ -1,43 +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_META_H
-#define DORIS_BE_SRC_OLAP_ROWSET_ALPHA_ROWSET_META_H
-
-#include <memory>
-#include <string>
-#include <vector>
-
-#include "olap/rowset/rowset_meta.h"
-
-namespace doris {
-
-class AlphaRowsetMeta;
-using AlphaRowsetMetaSharedPtr = std::shared_ptr<AlphaRowsetMeta>;
-
-class AlphaRowsetMeta : public RowsetMeta {
-public:
-    void get_segment_groups(std::vector<SegmentGroupPB>* segment_groups);
-
-    void add_segment_group(const SegmentGroupPB& segment_group);
-
-    void clear_segment_group();
-};
-
-} // namespace doris
-
-#endif // DORIS_BE_SRC_OLAP_ROWSET_ALPHA_ROWSET_META_H
diff --git a/be/src/olap/rowset/rowset_meta.h b/be/src/olap/rowset/rowset_meta.h
index d8f2c98249..93aefa4351 100644
--- a/be/src/olap/rowset/rowset_meta.h
+++ b/be/src/olap/rowset/rowset_meta.h
@@ -312,10 +312,6 @@ public:
         }
     }
 
-    const AlphaRowsetExtraMetaPB& alpha_rowset_extra_meta_pb() const {
-        return _rowset_meta_pb.alpha_rowset_extra_meta_pb();
-    }
-
     void set_oldest_write_timestamp(int64_t timestamp) {
         _rowset_meta_pb.set_oldest_write_timestamp(timestamp);
     }
@@ -337,7 +333,6 @@ public:
     const TabletSchema* tablet_schema() { return _schema.get(); }
 
 private:
-    friend class AlphaRowsetMeta;
     bool _deserialize_from_pb(const std::string& value) {
         return _rowset_meta_pb.ParseFromString(value);
     }
@@ -349,35 +344,12 @@ private:
         return _rowset_meta_pb.SerializeToString(value);
     }
 
-    bool _has_alpha_rowset_extra_meta_pb() {
-        return _rowset_meta_pb.has_alpha_rowset_extra_meta_pb();
-    }
-
-    AlphaRowsetExtraMetaPB* _mutable_alpha_rowset_extra_meta_pb() {
-        return _rowset_meta_pb.mutable_alpha_rowset_extra_meta_pb();
-    }
-
     void _init() {
         if (_rowset_meta_pb.rowset_id() > 0) {
             _rowset_id.init(_rowset_meta_pb.rowset_id());
         } else {
             _rowset_id.init(_rowset_meta_pb.rowset_id_v2());
         }
-
-        if (num_segments() == 0) {
-            // ATTN(cmy): the num segments should be read from rowset meta pb.
-            // But the previous code error caused this value not to be set in some cases.
-            // So when init the rowset meta and find that the num_segments is 0(not set),
-            // we will try to calculate the num segments from AlphaRowsetExtraMetaPB,
-            // and then set the num_segments field.
-            // This should only happen in some rowsets converted from old version.
-            // and for all newly created rowsets, the num_segments field must be set.
-            int32_t num_segments = 0;
-            for (auto& seg_grp : alpha_rowset_extra_meta_pb().segment_groups()) {
-                num_segments += seg_grp.num_segments();
-            }
-            set_num_segments(num_segments);
-        }
         if (_rowset_meta_pb.has_tablet_schema()) {
             _schema = std::make_shared<TabletSchema>();
             _schema->init_from_pb(_rowset_meta_pb.tablet_schema());
diff --git a/be/src/olap/snapshot_manager.cpp b/be/src/olap/snapshot_manager.cpp
index 436769bc3a..6950c1f025 100644
--- a/be/src/olap/snapshot_manager.cpp
+++ b/be/src/olap/snapshot_manager.cpp
@@ -30,7 +30,6 @@
 
 #include "env/env.h"
 #include "gen_cpp/Types_constants.h"
-#include "olap/rowset/alpha_rowset_meta.h"
 #include "olap/rowset/rowset.h"
 #include "olap/rowset/rowset_factory.h"
 #include "olap/rowset/rowset_writer.h"
@@ -113,9 +112,6 @@ Status SnapshotManager::release_snapshot(const string& snapshot_path) {
     return Status::OLAPInternalError(OLAP_ERR_CE_CMD_PARAMS_ERROR);
 }
 
-// TODO support beta rowset
-// For now, alpha and beta rowset meta have same fields, so we can just use
-// AlphaRowsetMeta here.
 Status SnapshotManager::convert_rowset_ids(const std::string& clone_dir, int64_t tablet_id,
                                            int64_t replica_id, const int32_t& schema_hash) {
     SCOPED_SWITCH_THREAD_LOCAL_MEM_TRACKER(_mem_tracker);
@@ -209,16 +205,10 @@ Status SnapshotManager::_rename_rowset_id(const RowsetMetaPB& rs_meta_pb,
                                           TabletSchema& tablet_schema, const RowsetId& rowset_id,
                                           RowsetMetaPB* new_rs_meta_pb) {
     Status res = Status::OK();
-    // TODO use factory to obtain RowsetMeta when SnapshotManager::convert_rowset_ids supports beta rowset
-    // TODO(cmy): now we only has AlphaRowsetMeta, and no BetaRowsetMeta.
-    //            AlphaRowsetMeta only add some functions about segment group, and no addition fields.
-    //            So we can use AlphaRowsetMeta here even if this is a beta rowset.
-    //            And the `rowset_type` field indicates the real type of rowset, so that the correct rowset
-    //            can be created.
-    RowsetMetaSharedPtr alpha_rowset_meta(new AlphaRowsetMeta());
-    alpha_rowset_meta->init_from_pb(rs_meta_pb);
+    RowsetMetaSharedPtr rowset_meta(new RowsetMeta());
+    rowset_meta->init_from_pb(rs_meta_pb);
     RowsetSharedPtr org_rowset;
-    RETURN_NOT_OK(RowsetFactory::create_rowset(&tablet_schema, new_tablet_path, alpha_rowset_meta,
+    RETURN_NOT_OK(RowsetFactory::create_rowset(&tablet_schema, new_tablet_path, rowset_meta,
                                                &org_rowset));
     // do not use cache to load index
     // because the index file may conflict
@@ -239,7 +229,7 @@ Status SnapshotManager::_rename_rowset_id(const RowsetMetaPB& rs_meta_pb,
     context.oldest_write_timestamp = org_rowset_meta->oldest_write_timestamp();
     context.newest_write_timestamp = org_rowset_meta->newest_write_timestamp();
     // keep segments_overlap same as origin rowset
-    context.segments_overlap = alpha_rowset_meta->segments_overlap();
+    context.segments_overlap = rowset_meta->segments_overlap();
 
     std::unique_ptr<RowsetWriter> rs_writer;
     RETURN_NOT_OK(RowsetFactory::create_rowset_writer(context, &rs_writer));
diff --git a/be/src/olap/storage_engine.cpp b/be/src/olap/storage_engine.cpp
index e06d788f9b..c7d33a4187 100644
--- a/be/src/olap/storage_engine.cpp
+++ b/be/src/olap/storage_engine.cpp
@@ -45,7 +45,6 @@
 #include "olap/memtable_flush_executor.h"
 #include "olap/push_handler.h"
 #include "olap/reader.h"
-#include "olap/rowset/alpha_rowset_meta.h"
 #include "olap/rowset/rowset_meta_manager.h"
 #include "olap/rowset/unique_rowset_id_generator.h"
 #include "olap/schema_change.h"
@@ -730,7 +729,7 @@ void StorageEngine::_clean_unused_rowset_metas() {
     auto clean_rowset_func = [this, &invalid_rowset_metas](TabletUid tablet_uid, RowsetId rowset_id,
                                                            const std::string& meta_str) -> bool {
         // return false will break meta iterator, return true to skip this error
-        RowsetMetaSharedPtr rowset_meta(new AlphaRowsetMeta());
+        RowsetMetaSharedPtr rowset_meta(new RowsetMeta());
         bool parsed = rowset_meta->init(meta_str);
         if (!parsed) {
             LOG(WARNING) << "parse rowset meta string failed for rowset_id:" << rowset_id;
diff --git a/be/src/olap/tablet_meta.cpp b/be/src/olap/tablet_meta.cpp
index f0ae3bfc85..eb5669fd4e 100644
--- a/be/src/olap/tablet_meta.cpp
+++ b/be/src/olap/tablet_meta.cpp
@@ -23,7 +23,6 @@
 #include "olap/file_helper.h"
 #include "olap/olap_common.h"
 #include "olap/olap_define.h"
-#include "olap/rowset/alpha_rowset_meta.h"
 #include "olap/tablet_meta_manager.h"
 #include "util/string_util.h"
 #include "util/uid_util.h"
@@ -437,7 +436,7 @@ void TabletMeta::init_from_pb(const TabletMetaPB& tablet_meta_pb) {
 
     // init _rs_metas
     for (auto& it : tablet_meta_pb.rs_metas()) {
-        RowsetMetaSharedPtr rs_meta(new AlphaRowsetMeta());
+        RowsetMetaSharedPtr rs_meta(new RowsetMeta());
         rs_meta->init_from_pb(it);
         if (rs_meta->has_delete_predicate()) {
             add_delete_predicate(rs_meta->delete_predicate(), rs_meta->version().first);
@@ -446,7 +445,7 @@ void TabletMeta::init_from_pb(const TabletMetaPB& tablet_meta_pb) {
     }
 
     for (auto& it : tablet_meta_pb.stale_rs_metas()) {
-        RowsetMetaSharedPtr rs_meta(new AlphaRowsetMeta());
+        RowsetMetaSharedPtr rs_meta(new RowsetMeta());
         rs_meta->init_from_pb(it);
         _stale_rs_metas.push_back(std::move(rs_meta));
     }
diff --git a/be/test/CMakeLists.txt b/be/test/CMakeLists.txt
index 84f9a42d2c..cdc39b7436 100644
--- a/be/test/CMakeLists.txt
+++ b/be/test/CMakeLists.txt
@@ -175,14 +175,12 @@ set(OLAP_TEST_FILES
     olap/tablet_meta_manager_test.cpp
     olap/tablet_mgr_test.cpp
     olap/tablet_test.cpp
-    # TODO yiguolei: it is using alpha rowset to test, should use beta rowset
-    #olap/rowset/rowset_meta_manager_test.cpp
+    olap/rowset/rowset_meta_manager_test.cpp
     olap/rowset/rowset_meta_test.cpp
     olap/rowset/beta_rowset_test.cpp
     olap/rowset/unique_rowset_id_generator_test.cpp
     olap/rowset/rowset_tree_test.cpp
-    # TODO yiguolei: it is using alpha rowset to test, should use beta rowset
-    #olap/txn_manager_test.cpp
+    olap/txn_manager_test.cpp
     olap/generic_iterators_test.cpp
     olap/key_coder_test.cpp
     olap/short_key_index_test.cpp
diff --git a/be/test/olap/cumulative_compaction_policy_test.cpp b/be/test/olap/cumulative_compaction_policy_test.cpp
index 3d2364bbec..deff7a6190 100644
--- a/be/test/olap/cumulative_compaction_policy_test.cpp
+++ b/be/test/olap/cumulative_compaction_policy_test.cpp
@@ -54,60 +54,7 @@ public:
                 "lo": -6717994719194512122
             },
             "creation_time": 1553765670,
-            "alpha_rowset_extra_meta_pb": {
-                "segment_groups": [
-                {
-                    "segment_group_id": 0,
-                    "num_segments": 2,
-                    "index_size": 132,
-                    "data_size": 576,
-                    "num_rows": 5,
-                    "zone_maps": [
-                    {
-                        "min": "MQ==",
-                        "max": "NQ==",
-                        "null_flag": false
-                    },
-                    {
-                        "min": "MQ==",
-                        "max": "Mw==",
-                        "null_flag": false
-                    },
-                    {
-                        "min": "J2J1c2gn",
-                        "max": "J3RvbSc=",
-                        "null_flag": false
-                    }
-                    ],
-                    "empty": false
-                },
-                {
-                    "segment_group_id": 1,
-                    "num_segments": 1,
-                    "index_size": 132,
-                    "data_size": 576,
-                    "num_rows": 5,
-                    "zone_maps": [
-                    {
-                        "min": "MQ==",
-                        "max": "NQ==",
-                        "null_flag": false
-                    },
-                    {
-                        "min": "MQ==",
-                        "max": "Mw==",
-                        "null_flag": false
-                    },
-                    {
-                        "min": "J2J1c2gn",
-                        "max": "J3RvbSc=",
-                        "null_flag": false
-                    }
-                    ],
-                    "empty": false
-                }
-                ]
-            }
+            "num_segments": 3
         })";
     }
     void TearDown() {}
@@ -343,60 +290,7 @@ public:
                 "lo": -6717994719194512122
             },
             "creation_time": 1553765670,
-            "alpha_rowset_extra_meta_pb": {
-                "segment_groups": [
-                {
-                    "segment_group_id": 0,
-                    "num_segments": 2,
-                    "index_size": 132,
-                    "data_size": 576,
-                    "num_rows": 5,
-                    "zone_maps": [
-                    {
-                        "min": "MQ==",
-                        "max": "NQ==",
-                        "null_flag": false
-                    },
-                    {
-                        "min": "MQ==",
-                        "max": "Mw==",
-                        "null_flag": false
-                    },
-                    {
-                        "min": "J2J1c2gn",
-                        "max": "J3RvbSc=",
-                        "null_flag": false
-                    }
-                    ],
-                    "empty": false
-                },
-                {
-                    "segment_group_id": 1,
-                    "num_segments": 1,
-                    "index_size": 132,
-                    "data_size": 576,
-                    "num_rows": 5,
-                    "zone_maps": [
-                    {
-                        "min": "MQ==",
-                        "max": "NQ==",
-                        "null_flag": false
-                    },
-                    {
-                        "min": "MQ==",
-                        "max": "Mw==",
-                        "null_flag": false
-                    },
-                    {
-                        "min": "J2J1c2gn",
-                        "max": "J3RvbSc=",
-                        "null_flag": false
-                    }
-                    ],
-                    "empty": false
-                }
-                ]
-            }
+            "num_segments": 3
         })";
     }
     void TearDown() {}
diff --git a/be/test/olap/rowset/rowset_meta_test.cpp b/be/test/olap/rowset/rowset_meta_test.cpp
index 0eb7957eee..6a998fc877 100644
--- a/be/test/olap/rowset/rowset_meta_test.cpp
+++ b/be/test/olap/rowset/rowset_meta_test.cpp
@@ -26,7 +26,6 @@
 #include "gtest/gtest.h"
 #include "json2pb/json_to_pb.h"
 #include "olap/olap_meta.h"
-#include "olap/rowset/alpha_rowset_meta.h"
 
 #ifndef BE_TEST
 #define BE_TEST
@@ -79,7 +78,7 @@ void do_check(RowsetMeta rowset_meta) {
     EXPECT_EQ(15673, rowset_meta.tablet_id());
     EXPECT_EQ(4042, rowset_meta.txn_id());
     EXPECT_EQ(567997577, rowset_meta.tablet_schema_hash());
-    EXPECT_EQ(ALPHA_ROWSET, rowset_meta.rowset_type());
+    EXPECT_EQ(BETA_ROWSET, rowset_meta.rowset_type());
     EXPECT_EQ(VISIBLE, rowset_meta.rowset_state());
     EXPECT_EQ(2, rowset_meta.start_version());
     EXPECT_EQ(2, rowset_meta.end_version());
@@ -113,83 +112,4 @@ TEST_F(RowsetMetaTest, TestInitWithInvalidData) {
     EXPECT_FALSE(rowset_meta.init("invalid pb meta data"));
 }
 
-void do_check_for_alpha(AlphaRowsetMeta alpha_rowset_meta) {
-    RowsetId rowset_id;
-    rowset_id.init(540081);
-    EXPECT_EQ(rowset_id, alpha_rowset_meta.rowset_id());
-    EXPECT_EQ(15673, alpha_rowset_meta.tablet_id());
-    EXPECT_EQ(4042, alpha_rowset_meta.txn_id());
-    EXPECT_EQ(567997577, alpha_rowset_meta.tablet_schema_hash());
-    EXPECT_EQ(ALPHA_ROWSET, alpha_rowset_meta.rowset_type());
-    EXPECT_EQ(VISIBLE, alpha_rowset_meta.rowset_state());
-    EXPECT_EQ(2, alpha_rowset_meta.start_version());
-    EXPECT_EQ(2, alpha_rowset_meta.end_version());
-    EXPECT_EQ(3929, alpha_rowset_meta.num_rows());
-    EXPECT_EQ(84699, alpha_rowset_meta.total_disk_size());
-    EXPECT_EQ(84464, alpha_rowset_meta.data_disk_size());
-    EXPECT_EQ(235, alpha_rowset_meta.index_disk_size());
-    EXPECT_EQ(false, alpha_rowset_meta.empty());
-    EXPECT_EQ(1553765670, alpha_rowset_meta.creation_time());
-    std::vector<SegmentGroupPB> segment_groups;
-    alpha_rowset_meta.get_segment_groups(&segment_groups);
-    EXPECT_EQ(2, segment_groups.size());
-}
-
-TEST_F(RowsetMetaTest, TestAlphaRowsetMeta) {
-    AlphaRowsetMeta rowset_meta;
-    rowset_meta.init_from_json(_json_rowset_meta);
-    do_check_for_alpha(rowset_meta);
-    RowsetMetaPB rowset_meta_pb;
-    rowset_meta.to_rowset_pb(&rowset_meta_pb);
-    AlphaRowsetMeta rowset_meta_2;
-    rowset_meta_2.init_from_pb(rowset_meta_pb);
-    do_check_for_alpha(rowset_meta_2);
-    std::string value = "";
-    rowset_meta_pb.SerializeToString(&value);
-    AlphaRowsetMeta rowset_meta_3;
-    rowset_meta_3.init(value);
-    do_check_for_alpha(rowset_meta_3);
-}
-
-TEST_F(RowsetMetaTest, TestAlphaRowsetMetaAdd) {
-    AlphaRowsetMeta rowset_meta;
-    rowset_meta.init_from_json(_json_rowset_meta);
-    do_check_for_alpha(rowset_meta);
-    SegmentGroupPB new_segment_group;
-    new_segment_group.set_segment_group_id(88888);
-    new_segment_group.set_num_segments(3);
-    new_segment_group.set_empty(true);
-    new_segment_group.set_index_size(100);
-    new_segment_group.set_data_size(1000);
-    new_segment_group.set_num_rows(1000);
-    rowset_meta.add_segment_group(new_segment_group);
-    std::vector<SegmentGroupPB> segment_groups;
-    rowset_meta.get_segment_groups(&segment_groups);
-    EXPECT_EQ(3, segment_groups.size());
-    std::string meta_pb_string = "";
-    EXPECT_TRUE(rowset_meta.serialize(&meta_pb_string));
-    AlphaRowsetMeta rowset_meta_2;
-    EXPECT_TRUE(rowset_meta_2.init(meta_pb_string));
-    segment_groups.clear();
-    rowset_meta_2.get_segment_groups(&segment_groups);
-    EXPECT_EQ(3, segment_groups.size());
-}
-
-TEST_F(RowsetMetaTest, TestAlphaRowsetMetaClear) {
-    AlphaRowsetMeta rowset_meta;
-    rowset_meta.init_from_json(_json_rowset_meta);
-    do_check_for_alpha(rowset_meta);
-    rowset_meta.clear_segment_group();
-    std::vector<SegmentGroupPB> segment_groups;
-    rowset_meta.get_segment_groups(&segment_groups);
-    EXPECT_EQ(0, segment_groups.size());
-    std::string meta_pb_string = "";
-    EXPECT_TRUE(rowset_meta.serialize(&meta_pb_string));
-    AlphaRowsetMeta rowset_meta_2;
-    EXPECT_TRUE(rowset_meta_2.init(meta_pb_string));
-    segment_groups.clear();
-    rowset_meta_2.get_segment_groups(&segment_groups);
-    EXPECT_EQ(0, segment_groups.size());
-}
-
 } // namespace doris
diff --git a/be/test/olap/tablet_mgr_test.cpp b/be/test/olap/tablet_mgr_test.cpp
index 5bfb775597..a47c1ddc95 100644
--- a/be/test/olap/tablet_mgr_test.cpp
+++ b/be/test/olap/tablet_mgr_test.cpp
@@ -24,7 +24,6 @@
 #include "gtest/gtest.h"
 #include "json2pb/json_to_pb.h"
 #include "olap/olap_meta.h"
-#include "olap/rowset/alpha_rowset_meta.h"
 #include "olap/rowset/rowset_meta_manager.h"
 #include "olap/storage_engine.h"
 #include "olap/tablet_meta_manager.h"
diff --git a/be/test/olap/test_data/rowset.json b/be/test/olap/test_data/rowset.json
index 025cd0066b..ceeeb008c3 100644
--- a/be/test/olap/test_data/rowset.json
+++ b/be/test/olap/test_data/rowset.json
@@ -3,7 +3,7 @@
     "tablet_id": 15673,
     "txn_id": 4042,
     "tablet_schema_hash": 567997577,
-    "rowset_type": "ALPHA_ROWSET",
+    "rowset_type": "BETA_ROWSET",
     "rowset_state": "VISIBLE",
     "start_version": 2,
     "end_version": 2,
@@ -17,58 +17,5 @@
         "lo": -6717994719194512122
     },
     "creation_time": 1553765670,
-    "alpha_rowset_extra_meta_pb": {
-        "segment_groups": [
-        {
-            "segment_group_id": 0,
-            "num_segments": 1,
-            "index_size": 132,
-            "data_size": 576,
-            "num_rows": 5,
-            "zone_maps": [
-            {
-                "min": "MQ==",
-                "max": "NQ==",
-                "null_flag": false
-            },
-            {
-                "min": "MQ==",
-                "max": "Mw==",
-                "null_flag": false
-            },
-            {
-                "min": "J2J1c2gn",
-                "max": "J3RvbSc=",
-                "null_flag": false
-            }
-            ],
-            "empty": false
-        },
-        {
-            "segment_group_id": 1,
-            "num_segments": 1,
-            "index_size": 132,
-            "data_size": 576,
-            "num_rows": 5,
-            "zone_maps": [
-            {
-                "min": "MQ==",
-                "max": "NQ==",
-                "null_flag": false
-            },
-            {
-                "min": "MQ==",
-                "max": "Mw==",
-                "null_flag": false
-            },
-            {
-                "min": "J2J1c2gn",
-                "max": "J3RvbSc=",
-                "null_flag": false
-            }
-            ],
-            "empty": false
-        }
-        ]
-    }
+    "num_segments": 2
 }
diff --git a/be/test/olap/test_data/rowset_meta.json b/be/test/olap/test_data/rowset_meta.json
index 1fdb9cf477..04016427ce 100644
--- a/be/test/olap/test_data/rowset_meta.json
+++ b/be/test/olap/test_data/rowset_meta.json
@@ -2,7 +2,7 @@
     "rowset_id": 10000,
     "tablet_id": 12046,
     "tablet_schema_hash": 365187263,
-    "rowset_type": "ALPHA_ROWSET",
+    "rowset_type": "BETA_ROWSET",
     "rowset_state": "VISIBLE",
     "start_version": 0,
     "end_version": 1,
@@ -16,34 +16,5 @@
         "hi": 10,
         "lo": 10
     },
-    "num_segments": 1,
-    "alpha_rowset_extra_meta_pb": {
-        "segment_groups": [
-            {
-                "segment_group_id": 0,
-                "num_segments": 1,
-                "index_size": 132,
-                "data_size": 576,
-                "num_rows": 5,
-                "zone_maps": [
-                    {
-                        "min": "MQ==",
-                        "max": "NQ==",
-                        "null_flag": false
-                    },
-                    {
-                        "min": "MQ==",
-                        "max": "Mw==",
-                        "null_flag": false
-                    },
-                    {
-                        "min": "J2J1c2gn",
-                        "max": "J3RvbSc=",
-                        "null_flag": false
-                    }
-                ],
-                "empty": false
-            }
-        ]
-    }
+    "num_segments": 1
 }
diff --git a/be/test/olap/test_data/rowset_meta2.json b/be/test/olap/test_data/rowset_meta2.json
index 15af654e15..cd7cfb4935 100644
--- a/be/test/olap/test_data/rowset_meta2.json
+++ b/be/test/olap/test_data/rowset_meta2.json
@@ -2,7 +2,7 @@
     "rowset_id": 10001,
     "tablet_id": 20487,
     "tablet_schema_hash": 1520686811,
-    "rowset_type": "ALPHA_ROWSET",
+    "rowset_type": "BETA_ROWSET",
     "rowset_state": "VISIBLE",
     "start_version": 2,
     "end_version": 3,
diff --git a/be/test/olap/txn_manager_test.cpp b/be/test/olap/txn_manager_test.cpp
index b4c6a3f178..18917353b9 100644
--- a/be/test/olap/txn_manager_test.cpp
+++ b/be/test/olap/txn_manager_test.cpp
@@ -26,7 +26,6 @@
 #include "gtest/gtest.h"
 #include "json2pb/json_to_pb.h"
 #include "olap/olap_meta.h"
-#include "olap/rowset/alpha_rowset_meta.h"
 #include "olap/rowset/rowset.h"
 #include "olap/rowset/rowset_factory.h"
 #include "olap/rowset/rowset_meta_manager.h"
@@ -130,13 +129,13 @@ public:
         _json_rowset_meta = _json_rowset_meta.substr(0, _json_rowset_meta.size() - 1);
         RowsetId rowset_id;
         rowset_id.init(10000);
-        RowsetMetaSharedPtr rowset_meta(new AlphaRowsetMeta());
+        RowsetMetaSharedPtr rowset_meta(new RowsetMeta());
         rowset_meta->init_from_json(_json_rowset_meta);
         EXPECT_EQ(rowset_meta->rowset_id(), rowset_id);
         EXPECT_EQ(Status::OK(), RowsetFactory::create_rowset(_schema.get(), rowset_meta_path,
-                                                             rowset_meta, &_alpha_rowset));
+                                                             rowset_meta, &_rowset));
         EXPECT_EQ(Status::OK(), RowsetFactory::create_rowset(_schema.get(), rowset_meta_path,
-                                                             rowset_meta, &_alpha_rowset_same_id));
+                                                             rowset_meta, &_rowset_same_id));
 
         // init rowset meta 2
         _json_rowset_meta = "";
@@ -148,11 +147,11 @@ public:
         }
         _json_rowset_meta = _json_rowset_meta.substr(0, _json_rowset_meta.size() - 1);
         rowset_id.init(10001);
-        RowsetMetaSharedPtr rowset_meta2(new AlphaRowsetMeta());
+        RowsetMetaSharedPtr rowset_meta2(new RowsetMeta());
         rowset_meta2->init_from_json(_json_rowset_meta);
         EXPECT_EQ(rowset_meta2->rowset_id(), rowset_id);
         EXPECT_EQ(Status::OK(), RowsetFactory::create_rowset(_schema.get(), rowset_meta_path_2,
-                                                             rowset_meta2, &_alpha_rowset_diff_id));
+                                                             rowset_meta2, &_rowset_diff_id));
         _tablet_uid = TabletUid(10, 10);
     }
 
@@ -172,9 +171,9 @@ private:
     TabletUid _tablet_uid {0, 0};
     PUniqueId load_id;
     std::unique_ptr<TabletSchema> _schema;
-    RowsetSharedPtr _alpha_rowset;
-    RowsetSharedPtr _alpha_rowset_same_id;
-    RowsetSharedPtr _alpha_rowset_diff_id;
+    RowsetSharedPtr _rowset;
+    RowsetSharedPtr _rowset_same_id;
+    RowsetSharedPtr _rowset_diff_id;
 };
 
 TEST_F(TxnManagerTest, PrepareNewTxn) {
@@ -190,20 +189,20 @@ TEST_F(TxnManagerTest, CommitTxnWithPrepare) {
     Status status = _txn_mgr->prepare_txn(partition_id, transaction_id, tablet_id, schema_hash,
                                           _tablet_uid, load_id);
     _txn_mgr->commit_txn(_meta, partition_id, transaction_id, tablet_id, schema_hash, _tablet_uid,
-                         load_id, _alpha_rowset, false);
+                         load_id, _rowset, false);
     EXPECT_TRUE(status == Status::OK());
-    RowsetMetaSharedPtr rowset_meta(new AlphaRowsetMeta());
-    status = RowsetMetaManager::get_rowset_meta(_meta, _tablet_uid, _alpha_rowset->rowset_id(),
+    RowsetMetaSharedPtr rowset_meta(new RowsetMeta());
+    status = RowsetMetaManager::get_rowset_meta(_meta, _tablet_uid, _rowset->rowset_id(),
                                                 rowset_meta);
     EXPECT_TRUE(status == Status::OK());
-    EXPECT_TRUE(rowset_meta->rowset_id() == _alpha_rowset->rowset_id());
+    EXPECT_TRUE(rowset_meta->rowset_id() == _rowset->rowset_id());
 }
 
 // 1. commit without prepare
 // 2. should success
 TEST_F(TxnManagerTest, CommitTxnWithNoPrepare) {
     Status status = _txn_mgr->commit_txn(_meta, partition_id, transaction_id, tablet_id,
-                                         schema_hash, _tablet_uid, load_id, _alpha_rowset, false);
+                                         schema_hash, _tablet_uid, load_id, _rowset, false);
     EXPECT_TRUE(status == Status::OK());
 }
 
@@ -211,10 +210,10 @@ TEST_F(TxnManagerTest, CommitTxnWithNoPrepare) {
 // 2. should failed
 TEST_F(TxnManagerTest, CommitTxnTwiceWithDiffRowsetId) {
     Status status = _txn_mgr->commit_txn(_meta, partition_id, transaction_id, tablet_id,
-                                         schema_hash, _tablet_uid, load_id, _alpha_rowset, false);
+                                         schema_hash, _tablet_uid, load_id, _rowset, false);
     EXPECT_TRUE(status == Status::OK());
     status = _txn_mgr->commit_txn(_meta, partition_id, transaction_id, tablet_id, schema_hash,
-                                  _tablet_uid, load_id, _alpha_rowset_diff_id, false);
+                                  _tablet_uid, load_id, _rowset_diff_id, false);
     EXPECT_TRUE(status != Status::OK());
 }
 
@@ -222,10 +221,10 @@ TEST_F(TxnManagerTest, CommitTxnTwiceWithDiffRowsetId) {
 // 2. should success
 TEST_F(TxnManagerTest, CommitTxnTwiceWithSameRowsetId) {
     Status status = _txn_mgr->commit_txn(_meta, partition_id, transaction_id, tablet_id,
-                                         schema_hash, _tablet_uid, load_id, _alpha_rowset, false);
+                                         schema_hash, _tablet_uid, load_id, _rowset, false);
     EXPECT_TRUE(status == Status::OK());
     status = _txn_mgr->commit_txn(_meta, partition_id, transaction_id, tablet_id, schema_hash,
-                                  _tablet_uid, load_id, _alpha_rowset_same_id, false);
+                                  _tablet_uid, load_id, _rowset_same_id, false);
     EXPECT_TRUE(status == Status::OK());
 }
 
@@ -247,8 +246,8 @@ TEST_F(TxnManagerTest, RollbackNotCommittedTxn) {
     status = _txn_mgr->rollback_txn(partition_id, transaction_id, tablet_id, schema_hash,
                                     _tablet_uid);
     EXPECT_TRUE(status == Status::OK());
-    RowsetMetaSharedPtr rowset_meta(new AlphaRowsetMeta());
-    status = RowsetMetaManager::get_rowset_meta(_meta, _tablet_uid, _alpha_rowset->rowset_id(),
+    RowsetMetaSharedPtr rowset_meta(new RowsetMeta());
+    status = RowsetMetaManager::get_rowset_meta(_meta, _tablet_uid, _rowset->rowset_id(),
                                                 rowset_meta);
     EXPECT_TRUE(status != Status::OK());
 }
@@ -256,33 +255,33 @@ TEST_F(TxnManagerTest, RollbackNotCommittedTxn) {
 // 1. txn could not be rollbacked if it is committed
 TEST_F(TxnManagerTest, RollbackCommittedTxn) {
     Status status = _txn_mgr->commit_txn(_meta, partition_id, transaction_id, tablet_id,
-                                         schema_hash, _tablet_uid, load_id, _alpha_rowset, false);
+                                         schema_hash, _tablet_uid, load_id, _rowset, false);
     EXPECT_TRUE(status == Status::OK());
     status = _txn_mgr->rollback_txn(partition_id, transaction_id, tablet_id, schema_hash,
                                     _tablet_uid);
     EXPECT_FALSE(status == Status::OK());
-    RowsetMetaSharedPtr rowset_meta(new AlphaRowsetMeta());
-    status = RowsetMetaManager::get_rowset_meta(_meta, _tablet_uid, _alpha_rowset->rowset_id(),
+    RowsetMetaSharedPtr rowset_meta(new RowsetMeta());
+    status = RowsetMetaManager::get_rowset_meta(_meta, _tablet_uid, _rowset->rowset_id(),
                                                 rowset_meta);
     EXPECT_TRUE(status == Status::OK());
-    EXPECT_TRUE(rowset_meta->rowset_id() == _alpha_rowset->rowset_id());
+    EXPECT_TRUE(rowset_meta->rowset_id() == _rowset->rowset_id());
 }
 
 // 1. publish version success
 TEST_F(TxnManagerTest, PublishVersionSuccessful) {
     Status status = _txn_mgr->commit_txn(_meta, partition_id, transaction_id, tablet_id,
-                                         schema_hash, _tablet_uid, load_id, _alpha_rowset, false);
+                                         schema_hash, _tablet_uid, load_id, _rowset, false);
     EXPECT_TRUE(status == Status::OK());
     Version new_version(10, 11);
     status = _txn_mgr->publish_txn(_meta, partition_id, transaction_id, tablet_id, schema_hash,
                                    _tablet_uid, new_version);
     EXPECT_TRUE(status == Status::OK());
 
-    RowsetMetaSharedPtr rowset_meta(new AlphaRowsetMeta());
-    status = RowsetMetaManager::get_rowset_meta(_meta, _tablet_uid, _alpha_rowset->rowset_id(),
+    RowsetMetaSharedPtr rowset_meta(new RowsetMeta());
+    status = RowsetMetaManager::get_rowset_meta(_meta, _tablet_uid, _rowset->rowset_id(),
                                                 rowset_meta);
     EXPECT_TRUE(status == Status::OK());
-    EXPECT_TRUE(rowset_meta->rowset_id() == _alpha_rowset->rowset_id());
+    EXPECT_TRUE(rowset_meta->rowset_id() == _rowset->rowset_id());
     EXPECT_TRUE(rowset_meta->start_version() == 10);
     EXPECT_TRUE(rowset_meta->end_version() == 11);
 }
@@ -306,17 +305,17 @@ TEST_F(TxnManagerTest, DeletePreparedTxn) {
 
 TEST_F(TxnManagerTest, DeleteCommittedTxn) {
     Status status = _txn_mgr->commit_txn(_meta, partition_id, transaction_id, tablet_id,
-                                         schema_hash, _tablet_uid, load_id, _alpha_rowset, false);
+                                         schema_hash, _tablet_uid, load_id, _rowset, false);
     EXPECT_TRUE(status == Status::OK());
-    RowsetMetaSharedPtr rowset_meta(new AlphaRowsetMeta());
-    status = RowsetMetaManager::get_rowset_meta(_meta, _tablet_uid, _alpha_rowset->rowset_id(),
+    RowsetMetaSharedPtr rowset_meta(new RowsetMeta());
+    status = RowsetMetaManager::get_rowset_meta(_meta, _tablet_uid, _rowset->rowset_id(),
                                                 rowset_meta);
     EXPECT_TRUE(status == Status::OK());
     status = _txn_mgr->delete_txn(_meta, partition_id, transaction_id, tablet_id, schema_hash,
                                   _tablet_uid);
     EXPECT_TRUE(status == Status::OK());
-    RowsetMetaSharedPtr rowset_meta2(new AlphaRowsetMeta());
-    status = RowsetMetaManager::get_rowset_meta(_meta, _tablet_uid, _alpha_rowset->rowset_id(),
+    RowsetMetaSharedPtr rowset_meta2(new RowsetMeta());
+    status = RowsetMetaManager::get_rowset_meta(_meta, _tablet_uid, _rowset->rowset_id(),
                                                 rowset_meta2);
     EXPECT_TRUE(status != Status::OK());
 }


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