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/incubator-doris.git


The following commit(s) were added to refs/heads/master by this push:
     new c872793a23 remove rowset converter since it is useless (#8974)
c872793a23 is described below

commit c872793a23e30860f1bdbf989e153162171186db
Author: yiguolei <guole...@tencent.com>
AuthorDate: Wed Apr 13 10:40:12 2022 +0800

    remove rowset converter since it is useless (#8974)
    
    Co-authored-by: yiguolei <yiguo...@gmail.com>
---
 be/src/olap/rowset/CMakeLists.txt             |   1 -
 be/src/olap/rowset/rowset_converter.cpp       | 116 ----------
 be/src/olap/rowset/rowset_converter.h         |  58 -----
 be/src/olap/snapshot_manager.cpp              |  54 +----
 be/src/olap/snapshot_manager.h                |   4 -
 be/src/olap/task/engine_clone_task.cpp        |   7 +-
 be/src/olap/task/engine_clone_task.h          |   2 +-
 be/test/CMakeLists.txt                        |   1 -
 be/test/olap/rowset/rowset_converter_test.cpp | 306 --------------------------
 9 files changed, 5 insertions(+), 544 deletions(-)

diff --git a/be/src/olap/rowset/CMakeLists.txt 
b/be/src/olap/rowset/CMakeLists.txt
index 228d76efc5..c5e0e39e15 100644
--- a/be/src/olap/rowset/CMakeLists.txt
+++ b/be/src/olap/rowset/CMakeLists.txt
@@ -38,7 +38,6 @@ add_library(Rowset STATIC
     segment_writer.cpp
     rowset.cpp
     rowset_factory.cpp
-    rowset_converter.cpp
     rowset_meta_manager.cpp
     alpha_rowset.cpp
     alpha_rowset_reader.cpp
diff --git a/be/src/olap/rowset/rowset_converter.cpp 
b/be/src/olap/rowset/rowset_converter.cpp
deleted file mode 100644
index a8bae5ebb4..0000000000
--- a/be/src/olap/rowset/rowset_converter.cpp
+++ /dev/null
@@ -1,116 +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/rowset_converter.h"
-
-#include "olap/row_block.h"
-#include "olap/row_block2.h"
-#include "olap/rowset/rowset_factory.h"
-
-namespace doris {
-
-OLAPStatus RowsetConverter::convert_beta_to_alpha(const RowsetMetaSharedPtr& 
src_rowset_meta,
-                                                  const FilePathDesc& 
rowset_path_desc,
-                                                  RowsetMetaPB* 
dst_rs_meta_pb) {
-    return _convert_rowset(src_rowset_meta, rowset_path_desc, ALPHA_ROWSET, 
dst_rs_meta_pb);
-}
-
-OLAPStatus RowsetConverter::convert_alpha_to_beta(const RowsetMetaSharedPtr& 
src_rowset_meta,
-                                                  const FilePathDesc& 
rowset_path_desc,
-                                                  RowsetMetaPB* 
dst_rs_meta_pb) {
-    return _convert_rowset(src_rowset_meta, rowset_path_desc, BETA_ROWSET, 
dst_rs_meta_pb);
-}
-
-OLAPStatus RowsetConverter::_convert_rowset(const RowsetMetaSharedPtr& 
src_rowset_meta,
-                                            const FilePathDesc& 
rowset_path_desc, RowsetTypePB dst_type,
-                                            RowsetMetaPB* dst_rs_meta_pb) {
-    const TabletSchema& tablet_schema = _tablet_meta->tablet_schema();
-    RowsetWriterContext context;
-    context.rowset_id = src_rowset_meta->rowset_id();
-    context.tablet_uid = _tablet_meta->tablet_uid();
-    context.tablet_id = _tablet_meta->tablet_id();
-    context.partition_id = _tablet_meta->partition_id();
-    context.tablet_schema_hash = _tablet_meta->schema_hash();
-    context.rowset_type = dst_type;
-    context.path_desc = rowset_path_desc;
-    context.tablet_schema = &tablet_schema;
-    context.rowset_state = src_rowset_meta->rowset_state();
-    context.segments_overlap = src_rowset_meta->segments_overlap();
-    if (context.rowset_state == VISIBLE) {
-        context.version = src_rowset_meta->version();
-    } else {
-        context.txn_id = src_rowset_meta->txn_id();
-        context.load_id = src_rowset_meta->load_id();
-    }
-    std::unique_ptr<RowsetWriter> rowset_writer;
-    RETURN_NOT_OK(RowsetFactory::create_rowset_writer(context, 
&rowset_writer));
-    if (!src_rowset_meta->empty()) {
-        RowsetSharedPtr rowset;
-        RETURN_NOT_OK(RowsetFactory::create_rowset(&tablet_schema, 
rowset_path_desc, src_rowset_meta,
-                                                   &rowset));
-        RowsetReaderSharedPtr rowset_reader;
-        RETURN_NOT_OK(rowset->create_reader(&rowset_reader));
-        std::vector<uint32_t> cids;
-        for (int i = 0; i < tablet_schema.num_columns(); ++i) {
-            cids.push_back(i);
-        }
-        DeleteHandler delete_handler;
-        RowsetReaderContext reader_context;
-        reader_context.reader_type = READER_ALTER_TABLE;
-        reader_context.tablet_schema = &tablet_schema;
-        reader_context.return_columns = &cids;
-        reader_context.seek_columns = &cids;
-        reader_context.delete_handler = &delete_handler;
-        reader_context.sequence_id_idx = 
reader_context.tablet_schema->sequence_col_idx();
-        RETURN_NOT_OK(rowset_reader->init(&reader_context));
-        // convert
-        RowBlock* row_block = nullptr;
-        RowCursor row_cursor;
-        row_cursor.init(tablet_schema);
-        while (true) {
-            if (row_block == nullptr || !row_block->has_remaining()) {
-                auto st = rowset_reader->next_block(&row_block);
-                if (st != OLAP_SUCCESS) {
-                    if (st == OLAP_ERR_DATA_EOF) {
-                        break;
-                    } else {
-                        return st;
-                    }
-                }
-            }
-            while (row_block->has_remaining()) {
-                size_t pos = row_block->pos();
-                row_block->get_row(pos, &row_cursor);
-                RETURN_NOT_OK(rowset_writer->add_row(row_cursor));
-                row_block->pos_inc();
-            }
-        }
-    }
-    RETURN_NOT_OK(rowset_writer->flush());
-    RowsetSharedPtr dst_rowset = rowset_writer->build();
-    if (dst_rowset == nullptr) {
-        return OLAP_ERR_MALLOC_ERROR;
-    }
-    if (src_rowset_meta->has_delete_predicate()) {
-        // should set the delete predicate to the rowset meta
-        
dst_rowset->rowset_meta()->set_delete_predicate(src_rowset_meta->delete_predicate());
-    }
-    dst_rowset->rowset_meta()->to_rowset_pb(dst_rs_meta_pb);
-    return OLAP_SUCCESS;
-}
-
-} // namespace doris
diff --git a/be/src/olap/rowset/rowset_converter.h 
b/be/src/olap/rowset/rowset_converter.h
deleted file mode 100644
index 4faa284c28..0000000000
--- a/be/src/olap/rowset/rowset_converter.h
+++ /dev/null
@@ -1,58 +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/olap_common.h"
-#include "olap/olap_define.h"
-#include "olap/rowset/alpha_rowset.h"
-#include "olap/rowset/alpha_rowset_writer.h"
-#include "olap/rowset/beta_rowset.h"
-#include "olap/rowset/beta_rowset_reader.h"
-#include "olap/rowset/rowset.h"
-#include "olap/rowset/rowset_meta.h"
-#include "olap/tablet_meta.h"
-
-namespace doris {
-
-class BetaRowset;
-using BetaRowsetSharedPtr = std::shared_ptr<BetaRowset>;
-class BetaRowsetReader;
-class RowsetFactory;
-
-class RowsetConverter {
-public:
-    ~RowsetConverter() = default;
-
-    RowsetConverter(const TabletMetaSharedPtr& tablet_meta) : 
_tablet_meta(tablet_meta) {}
-
-    OLAPStatus convert_beta_to_alpha(const RowsetMetaSharedPtr& 
src_rowset_meta,
-                                     const FilePathDesc& rowset_path_desc, 
RowsetMetaPB* dst_rs_meta_pb);
-
-    OLAPStatus convert_alpha_to_beta(const RowsetMetaSharedPtr& 
src_rowset_meta,
-                                     const FilePathDesc& rowset_path_desc, 
RowsetMetaPB* dst_rs_meta_pb);
-
-private:
-    OLAPStatus _convert_rowset(const RowsetMetaSharedPtr& src_rowset_meta,
-                               const FilePathDesc& rowset_path_desc,
-                               RowsetTypePB dst_type, RowsetMetaPB* 
dst_rs_meta_pb);
-
-private:
-    TabletMetaSharedPtr _tablet_meta;
-};
-
-} // namespace doris
diff --git a/be/src/olap/snapshot_manager.cpp b/be/src/olap/snapshot_manager.cpp
index 53e1718a23..da9d8f328f 100644
--- a/be/src/olap/snapshot_manager.cpp
+++ b/be/src/olap/snapshot_manager.cpp
@@ -31,7 +31,6 @@
 #include "gen_cpp/Types_constants.h"
 #include "olap/rowset/alpha_rowset_meta.h"
 #include "olap/rowset/rowset.h"
-#include "olap/rowset/rowset_converter.h"
 #include "olap/rowset/rowset_factory.h"
 #include "olap/rowset/rowset_id_generator.h"
 #include "olap/rowset/rowset_writer.h"
@@ -448,19 +447,8 @@ OLAPStatus SnapshotManager::_create_snapshot_files(const 
TabletSharedPtr& ref_ta
         // Clear it for safety reason.
         // Whether it is incremental or full snapshot, rowset information is 
stored in rs_meta.
         new_tablet_meta->revise_rs_metas(std::move(rs_metas));
-
-        if (snapshot_version == g_Types_constants.TSNAPSHOT_REQ_VERSION1) {
-            // convert beta rowset to alpha rowset
-            res = _convert_beta_rowsets_to_alpha(new_tablet_meta, 
new_tablet_meta->all_rs_metas(),
-                                                 schema_full_path_desc);
-            if (res != OLAP_SUCCESS) {
-                break;
-            }
-            res = new_tablet_meta->save(header_path);
-            LOG(INFO) << "finish convert beta to alpha, res:" << res
-                      << ", tablet:" << new_tablet_meta->tablet_id()
-                      << ", schema hash:" << new_tablet_meta->schema_hash();
-        } else if (snapshot_version == 
g_Types_constants.TSNAPSHOT_REQ_VERSION2) {
+        
+        if (snapshot_version == g_Types_constants.TSNAPSHOT_REQ_VERSION2) {
             res = new_tablet_meta->save(header_path);
         } else {
             res = OLAP_ERR_INVALID_SNAPSHOT_VERSION;
@@ -492,42 +480,4 @@ OLAPStatus SnapshotManager::_create_snapshot_files(const 
TabletSharedPtr& ref_ta
     return res;
 }
 
-OLAPStatus SnapshotManager::_convert_beta_rowsets_to_alpha(
-        const TabletMetaSharedPtr& new_tablet_meta,
-        const std::vector<RowsetMetaSharedPtr>& rowset_metas, const 
FilePathDesc& dst_path_desc) {
-    OLAPStatus res = OLAP_SUCCESS;
-    RowsetConverter rowset_converter(new_tablet_meta);
-    std::vector<RowsetMetaSharedPtr> new_rowset_metas;
-    bool modified = false;
-    for (auto& rowset_meta : rowset_metas) {
-        if (rowset_meta->rowset_type() == BETA_ROWSET) {
-            modified = true;
-            RowsetMetaPB rowset_meta_pb;
-            auto st = rowset_converter.convert_beta_to_alpha(rowset_meta, 
dst_path_desc,
-                                                             &rowset_meta_pb);
-            if (st != OLAP_SUCCESS) {
-                res = st;
-                LOG(WARNING) << "convert beta to alpha failed"
-                             << ", tablet_id:" << new_tablet_meta->tablet_id()
-                             << ", schema hash:" << 
new_tablet_meta->schema_hash()
-                             << ", src rowset:" << rowset_meta->rowset_id() << 
", error:" << st;
-                break;
-            }
-            RowsetMetaSharedPtr new_rowset_meta(new AlphaRowsetMeta());
-            bool ret = new_rowset_meta->init_from_pb(rowset_meta_pb);
-            if (!ret) {
-                res = OLAP_ERR_INIT_FAILED;
-                break;
-            }
-            new_rowset_metas.push_back(new_rowset_meta);
-        } else {
-            new_rowset_metas.push_back(rowset_meta);
-        }
-    }
-    if (res == OLAP_SUCCESS && modified) {
-        new_tablet_meta->revise_rs_metas(std::move(new_rowset_metas));
-    }
-    return res;
-}
-
 } // namespace doris
diff --git a/be/src/olap/snapshot_manager.h b/be/src/olap/snapshot_manager.h
index 0ae77784ad..790c3106ec 100644
--- a/be/src/olap/snapshot_manager.h
+++ b/be/src/olap/snapshot_manager.h
@@ -91,10 +91,6 @@ private:
                                  TabletSchema& tablet_schema, const RowsetId& 
next_id,
                                  RowsetMetaPB* new_rs_meta_pb);
 
-    OLAPStatus _convert_beta_rowsets_to_alpha(const TabletMetaSharedPtr& 
new_tablet_meta,
-                                              const 
vector<RowsetMetaSharedPtr>& rowset_metas,
-                                              const FilePathDesc& 
dst_path_desc);
-
 private:
     static SnapshotManager* _s_instance;
     static std::mutex _mlock;
diff --git a/be/src/olap/task/engine_clone_task.cpp 
b/be/src/olap/task/engine_clone_task.cpp
index 4d1f6c44b8..4cca849fa1 100644
--- a/be/src/olap/task/engine_clone_task.cpp
+++ b/be/src/olap/task/engine_clone_task.cpp
@@ -287,11 +287,10 @@ Status 
EngineCloneTask::_make_and_download_snapshots(DataDir& data_dir,
     for (auto& src : _clone_req.src_backends) {
         // Make snapshot in remote olap engine
         *src_host = src;
-        int32_t snapshot_version = 0;
         // make snapshot
         auto st = _make_snapshot(src.host, src.be_port, _clone_req.tablet_id,
                                  _clone_req.schema_hash, timeout_s, 
missed_versions, snapshot_path,
-                                 allow_incremental_clone, &snapshot_version);
+                                 allow_incremental_clone);
         if (st.ok()) {
             LOG(INFO) << "success to make snapshot. ip=" << src.host << ", 
port=" << src.be_port
                       << ", tablet=" << _clone_req.tablet_id
@@ -361,8 +360,7 @@ Status 
EngineCloneTask::_make_and_download_snapshots(DataDir& data_dir,
 Status EngineCloneTask::_make_snapshot(const std::string& ip, int port, 
TTableId tablet_id,
                                        TSchemaHash schema_hash, int timeout_s,
                                        const std::vector<Version>* 
missed_versions,
-                                       std::string* snapshot_path, bool* 
allow_incremental_clone,
-                                       int32_t* snapshot_version) {
+                                       std::string* snapshot_path, bool* 
allow_incremental_clone) {
     TSnapshotRequest request;
     request.__set_tablet_id(tablet_id);
     request.__set_schema_hash(schema_hash);
@@ -402,7 +400,6 @@ Status EngineCloneTask::_make_snapshot(const std::string& 
ip, int port, TTableId
         // should add a symbol to indicate it.
         *allow_incremental_clone = result.allow_incremental_clone;
     }
-    *snapshot_version = result.snapshot_version;
     return Status::OK();
 }
 
diff --git a/be/src/olap/task/engine_clone_task.h 
b/be/src/olap/task/engine_clone_task.h
index 3e8a34894b..ecfd0a91ff 100644
--- a/be/src/olap/task/engine_clone_task.h
+++ b/be/src/olap/task/engine_clone_task.h
@@ -64,7 +64,7 @@ private:
     Status _make_snapshot(const std::string& ip, int port, TTableId tablet_id,
                           TSchemaHash schema_hash, int timeout_s,
                           const std::vector<Version>* missed_versions, 
std::string* snapshot_path,
-                          bool* allow_incremental_clone, int32_t* 
snapshot_version);
+                          bool* allow_incremental_clone);
 
     Status _release_snapshot(const std::string& ip, int port, const 
std::string& snapshot_path);
 
diff --git a/be/test/CMakeLists.txt b/be/test/CMakeLists.txt
index 07d4093387..4c784c1ff9 100644
--- a/be/test/CMakeLists.txt
+++ b/be/test/CMakeLists.txt
@@ -200,7 +200,6 @@ set(OLAP_TEST_FILES
     olap/options_test.cpp
     olap/fs/file_block_manager_test.cpp
     olap/common_test.cpp
-    # olap/rowset/rowset_converter_test.cpp
     # olap/memtable_flush_executor_test.cpp
     # olap/push_handler_test.cpp
 )
diff --git a/be/test/olap/rowset/rowset_converter_test.cpp 
b/be/test/olap/rowset/rowset_converter_test.cpp
deleted file mode 100644
index 26fcf38f66..0000000000
--- a/be/test/olap/rowset/rowset_converter_test.cpp
+++ /dev/null
@@ -1,306 +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/rowset_converter.h"
-
-#include <fstream>
-#include <sstream>
-#include <string>
-
-#include "gmock/gmock.h"
-#include "gtest/gtest.h"
-#include "json2pb/json_to_pb.h"
-#include "olap/data_dir.h"
-#include "olap/olap_cond.h"
-#include "olap/olap_meta.h"
-#include "olap/rowset/rowset_factory.h"
-#include "olap/rowset/rowset_reader.h"
-#include "olap/rowset/rowset_reader_context.h"
-#include "olap/rowset/rowset_writer.h"
-#include "olap/rowset/rowset_writer_context.h"
-#include "olap/storage_engine.h"
-#include "olap/tablet_meta.h"
-#include "runtime/exec_env.h"
-#include "util/file_utils.h"
-#include "util/logging.h"
-
-#ifndef BE_TEST
-#define BE_TEST
-#endif
-
-using ::testing::_;
-using ::testing::Return;
-using ::testing::SetArgPointee;
-using std::string;
-
-namespace doris {
-
-static const uint32_t MAX_PATH_LEN = 1024;
-static StorageEngine* k_engine = nullptr;
-
-static void create_rowset_writer_context(TabletSchema* tablet_schema, 
RowsetTypePB dst_type,
-                                         RowsetWriterContext* 
rowset_writer_context) {
-    RowsetId rowset_id;
-    rowset_id.init(10000);
-    rowset_writer_context->rowset_id = rowset_id;
-    rowset_writer_context->tablet_id = 12345;
-    rowset_writer_context->tablet_schema_hash = 1111;
-    rowset_writer_context->partition_id = 10;
-    rowset_writer_context->rowset_type = dst_type;
-    rowset_writer_context->path_desc.filepath = config::storage_root_path + 
"/data/0/12345/1111";
-    rowset_writer_context->rowset_state = VISIBLE;
-    rowset_writer_context->tablet_schema = tablet_schema;
-    rowset_writer_context->version.first = 0;
-    rowset_writer_context->version.second = 1;
-}
-
-static void create_rowset_reader_context(TabletSchema* tablet_schema,
-                                         const std::vector<uint32_t>* 
return_columns,
-                                         const DeleteHandler* delete_handler,
-                                         std::vector<ColumnPredicate*>* 
predicates,
-                                         std::set<uint32_t>* load_bf_columns,
-                                         Conditions* conditions,
-                                         RowsetReaderContext* 
rowset_reader_context) {
-    rowset_reader_context->reader_type = READER_ALTER_TABLE;
-    rowset_reader_context->tablet_schema = tablet_schema;
-    rowset_reader_context->need_ordered_result = true;
-    rowset_reader_context->return_columns = return_columns;
-    rowset_reader_context->seek_columns = return_columns;
-    rowset_reader_context->delete_handler = delete_handler;
-    rowset_reader_context->lower_bound_keys = nullptr;
-    rowset_reader_context->is_lower_keys_included = nullptr;
-    rowset_reader_context->upper_bound_keys = nullptr;
-    rowset_reader_context->is_upper_keys_included = nullptr;
-    rowset_reader_context->predicates = predicates;
-    rowset_reader_context->load_bf_columns = load_bf_columns;
-    rowset_reader_context->conditions = conditions;
-}
-
-static void create_tablet_schema(KeysType keys_type, TabletSchema* 
tablet_schema) {
-    TabletSchemaPB tablet_schema_pb;
-    tablet_schema_pb.set_keys_type(keys_type);
-    tablet_schema_pb.set_num_short_key_columns(2);
-    tablet_schema_pb.set_num_rows_per_row_block(1024);
-    tablet_schema_pb.set_compress_kind(COMPRESS_NONE);
-    tablet_schema_pb.set_next_column_unique_id(4);
-
-    ColumnPB* column_1 = tablet_schema_pb.add_column();
-    column_1->set_unique_id(1);
-    column_1->set_name("k1");
-    column_1->set_type("INT");
-    column_1->set_is_key(true);
-    column_1->set_length(4);
-    column_1->set_index_length(4);
-    column_1->set_is_nullable(false);
-    column_1->set_is_bf_column(false);
-
-    ColumnPB* column_2 = tablet_schema_pb.add_column();
-    column_2->set_unique_id(2);
-    column_2->set_name("k2");
-    column_2->set_type("VARCHAR");
-    column_2->set_length(20);
-    column_2->set_index_length(20);
-    column_2->set_is_key(true);
-    column_2->set_is_nullable(false);
-    column_2->set_is_bf_column(false);
-
-    ColumnPB* column_3 = tablet_schema_pb.add_column();
-    column_3->set_unique_id(3);
-    column_3->set_name("v1");
-    column_3->set_type("INT");
-    column_3->set_length(4);
-    column_3->set_is_key(false);
-    column_3->set_is_nullable(false);
-    column_3->set_is_bf_column(false);
-    column_3->set_aggregation("SUM");
-
-    tablet_schema->init_from_pb(tablet_schema_pb);
-}
-
-static void create_tablet_meta(TabletSchema* tablet_schema, TabletMeta* 
tablet_meta) {
-    TabletMetaPB tablet_meta_pb;
-    tablet_meta_pb.set_table_id(10000);
-    tablet_meta_pb.set_tablet_id(12345);
-    tablet_meta_pb.set_schema_hash(1111);
-    tablet_meta_pb.set_partition_id(10);
-    tablet_meta_pb.set_shard_id(0);
-    tablet_meta_pb.set_creation_time(1575020449);
-    tablet_meta_pb.set_tablet_state(PB_RUNNING);
-    PUniqueId* tablet_uid = tablet_meta_pb.mutable_tablet_uid();
-    tablet_uid->set_hi(10);
-    tablet_uid->set_lo(10);
-
-    TabletSchemaPB* tablet_schema_pb = tablet_meta_pb.mutable_schema();
-    tablet_schema->to_schema_pb(tablet_schema_pb);
-
-    tablet_meta->init_from_pb(tablet_meta_pb);
-}
-
-class RowsetConverterTest : public testing::Test {
-public:
-    virtual void SetUp() {
-        config::tablet_map_shard_size = 1;
-        config::txn_map_shard_size = 1;
-        config::txn_shard_size = 1;
-        config::path_gc_check = false;
-        char buffer[MAX_PATH_LEN];
-        EXPECT_NE(getcwd(buffer, MAX_PATH_LEN), nullptr);
-        config::storage_root_path = std::string(buffer) + "/data_test";
-        FileUtils::remove_all(config::storage_root_path);
-        EXPECT_TRUE(FileUtils::create_dir(config::storage_root_path).ok());
-        std::vector<StorePath> paths;
-        paths.emplace_back(config::storage_root_path, -1);
-
-        doris::EngineOptions options;
-        options.store_paths = paths;
-        if (k_engine == nullptr) {
-            Status s = doris::StorageEngine::open(options, &k_engine);
-            EXPECT_TRUE(s.ok()) << s.to_string();
-        }
-
-        ExecEnv* exec_env = doris::ExecEnv::GetInstance();
-        exec_env->set_storage_engine(k_engine);
-
-        std::string data_path = config::storage_root_path + "/data";
-        EXPECT_TRUE(FileUtils::create_dir(data_path).ok());
-        std::string shard_path = data_path + "/0";
-        EXPECT_TRUE(FileUtils::create_dir(shard_path).ok());
-        std::string tablet_path = shard_path + "/12345";
-        EXPECT_TRUE(FileUtils::create_dir(tablet_path).ok());
-        _schema_hash_path = tablet_path + "/1111";
-        EXPECT_TRUE(FileUtils::create_dir(_schema_hash_path).ok());
-        _mem_tracker.reset(new MemTracker(-1));
-        _mem_pool.reset(new MemPool(_mem_tracker.get()));
-    }
-
-    virtual void TearDown() {
-        if (k_engine != nullptr) {
-            k_engine->stop();
-            delete k_engine;
-            k_engine = nullptr;
-        }
-        FileUtils::remove_all(config::storage_root_path);
-    }
-
-    void process(RowsetTypePB src_type, RowsetTypePB dst_type);
-
-private:
-    std::string _schema_hash_path;
-    std::shared_ptr<MemTracker> _mem_tracker;
-    std::unique_ptr<MemPool> _mem_pool;
-};
-
-void RowsetConverterTest::process(RowsetTypePB src_type, RowsetTypePB 
dst_type) {
-    // write
-    TabletSchema tablet_schema;
-    create_tablet_schema(AGG_KEYS, &tablet_schema);
-    RowsetWriterContext rowset_writer_context;
-    create_rowset_writer_context(&tablet_schema, src_type, 
&rowset_writer_context);
-    std::unique_ptr<RowsetWriter> _rowset_writer;
-    EXPECT_EQ(OLAP_SUCCESS,
-              RowsetFactory::create_rowset_writer(rowset_writer_context, 
&_rowset_writer));
-    RowCursor row;
-    OLAPStatus res = row.init(tablet_schema);
-    EXPECT_EQ(OLAP_SUCCESS, res);
-
-    std::vector<std::string> test_data;
-    for (int i = 0; i < 1024; ++i) {
-        test_data.push_back("well" + std::to_string(i));
-
-        int32_t field_0 = i;
-        row.set_field_content(0, reinterpret_cast<char*>(&field_0), 
_mem_pool.get());
-        Slice field_1(test_data[i]);
-        row.set_field_content(1, reinterpret_cast<char*>(&field_1), 
_mem_pool.get());
-        int32_t field_2 = 10000 + i;
-        row.set_field_content(2, reinterpret_cast<char*>(&field_2), 
_mem_pool.get());
-        _rowset_writer->add_row(row);
-    }
-    _rowset_writer->flush();
-    RowsetSharedPtr src_rowset = _rowset_writer->build();
-    EXPECT_TRUE(src_rowset != nullptr);
-    RowsetId src_rowset_id;
-    src_rowset_id.init(10000);
-    EXPECT_EQ(src_rowset_id, src_rowset->rowset_id());
-    EXPECT_EQ(1024, src_rowset->num_rows());
-
-    // convert
-    TabletMetaSharedPtr tablet_meta(new TabletMeta());
-    create_tablet_meta(&tablet_schema, tablet_meta.get());
-    RowsetConverter rowset_converter(tablet_meta);
-    RowsetMetaPB dst_rowset_meta_pb;
-    FilePathDesc schema_hash_path_desc;
-    schema_hash_path_desc.filepath = _schema_hash_path;
-    if (dst_type == BETA_ROWSET) {
-        EXPECT_EQ(OLAP_SUCCESS,
-                  rowset_converter.convert_alpha_to_beta(
-                          src_rowset->rowset_meta(), schema_hash_path_desc, 
&dst_rowset_meta_pb));
-    } else {
-        EXPECT_EQ(OLAP_SUCCESS,
-                  rowset_converter.convert_beta_to_alpha(
-                          src_rowset->rowset_meta(), schema_hash_path_desc, 
&dst_rowset_meta_pb));
-    }
-
-    EXPECT_EQ(dst_type, dst_rowset_meta_pb.rowset_type());
-    EXPECT_EQ(12345, dst_rowset_meta_pb.tablet_id());
-    EXPECT_EQ(1024, dst_rowset_meta_pb.num_rows());
-
-    // read
-    RowsetMetaSharedPtr dst_rowset_meta(new RowsetMeta());
-    EXPECT_TRUE(dst_rowset_meta->init_from_pb(dst_rowset_meta_pb));
-    RowsetSharedPtr dst_rowset;
-    EXPECT_EQ(OLAP_SUCCESS, RowsetFactory::create_rowset(&tablet_schema, 
schema_hash_path_desc,
-                                                         dst_rowset_meta, 
&dst_rowset));
-
-    RowsetReaderSharedPtr dst_rowset_reader;
-    EXPECT_EQ(OLAP_SUCCESS, dst_rowset->create_reader(&dst_rowset_reader));
-    RowsetReaderContext rowset_reader_context;
-    std::set<uint32_t> load_bf_columns;
-    std::vector<ColumnPredicate*> predicates;
-    Conditions conditions;
-    std::vector<uint32_t> return_columns;
-    for (int i = 0; i < tablet_schema.num_columns(); ++i) {
-        return_columns.push_back(i);
-    }
-    DeleteHandler delete_handler;
-    create_rowset_reader_context(&tablet_schema, &return_columns, 
&delete_handler, &predicates,
-                                 &load_bf_columns, &conditions, 
&rowset_reader_context);
-    res = dst_rowset_reader->init(&rowset_reader_context);
-    EXPECT_EQ(OLAP_SUCCESS, res);
-
-    RowBlock* row_block = nullptr;
-    res = dst_rowset_reader->next_block(&row_block);
-    EXPECT_EQ(OLAP_SUCCESS, res);
-    EXPECT_EQ(1024, row_block->remaining());
-    RowCursor row_cursor;
-    row_cursor.init(tablet_schema);
-    for (int i = 0; i < 1024; ++i) {
-        row_block->get_row(i, &row_cursor);
-        EXPECT_EQ(i, *(uint32_t*)row_cursor.cell_ptr(0));
-        EXPECT_EQ("well" + std::to_string(i), 
(*(Slice*)row_cursor.cell_ptr(1)).to_string());
-        EXPECT_EQ(10000 + i, *(uint32_t*)row_cursor.cell_ptr(2));
-    }
-}
-
-TEST_F(RowsetConverterTest, TestConvertAlphaRowsetToBeta) {
-    process(ALPHA_ROWSET, BETA_ROWSET);
-}
-
-TEST_F(RowsetConverterTest, TestConvertBetaRowsetToAlpha) {
-    process(ALPHA_ROWSET, BETA_ROWSET);
-}
-
-} // namespace doris


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

Reply via email to