From 57919102711f10a0a6296fea570322383345b02e Mon Sep 17 00:00:00 2001 From: Lei Zhang <27994433+SWJTU-ZhangLei@users.noreply.github.com> Date: Wed, 26 Jun 2024 10:45:22 +0800 Subject: [PATCH 1/5] [feature](cloud) Add lazy commit mechanism for `commit_txn` Motivation: * In cloud mode we use `foundationdb` for storing rowset meta, when we load data with many rowsets, `commit_txn` will failed because `foundationdb` currently limits all transactions to be below 10 MB in size, So we spilt `commit_txn` into several sub fdb txn to solve the problem How: `commit_txn` flow like this sub txn 1: 1. update partition `VersionPB` with txn_id 2. submit async task to txn_lazy_committer txn_lazy_committer: sub txn 2: convert tmp rowset meta per batch sub txn 3: convert tmp rowset meta per batch ...... sub txn n: make txn visible and remove txn_id in `VersionPB` `get_version` main flow like this: 1. `VersionPB` without txn_id, the same as before 2. `VersionPB` has txn_id field, it means this is a txn need be advanced by txn_lazy_committer, so submit a `txn_lazy_task` to make last txn visible and retry --- be/src/cloud/cloud_meta_mgr.cpp | 1 + be/src/cloud/config.cpp | 2 + be/src/cloud/config.h | 3 + cloud/src/common/config.h | 4 + cloud/src/meta-service/CMakeLists.txt | 1 + cloud/src/meta-service/meta_service.cpp | 256 ++-- cloud/src/meta-service/meta_service.h | 2 + cloud/src/meta-service/meta_service_txn.cpp | 1239 ++++++++++++----- cloud/src/meta-service/txn_lazy_committer.cpp | 459 ++++++ cloud/src/meta-service/txn_lazy_committer.h | 68 + cloud/src/recycler/recycler.cpp | 287 +++- cloud/src/recycler/recycler.h | 12 +- cloud/src/recycler/recycler_service.cpp | 18 +- cloud/src/recycler/recycler_service.h | 5 +- cloud/test/CMakeLists.txt | 4 + cloud/test/meta_service_test.cpp | 2 +- cloud/test/recycler_test.cpp | 112 +- cloud/test/txn_lazy_commit_test.cpp | 216 +++ .../java/org/apache/doris/common/Config.java | 4 + .../apache/doris/alter/CloudRollupJobV2.java | 1 + .../doris/alter/CloudSchemaChangeJobV2.java | 1 + .../datasource/CloudInternalCatalog.java | 3 +- .../CloudGlobalTransactionMgr.java | 6 +- gensrc/proto/cloud.proto | 3 + .../test_insert_with_many_buckets.out | 4 + .../test_insert_with_many_partitions.out | 4 + ...nsert_with_many_partitions_and_buckets.out | 4 + .../test_insert_with_many_buckets.groovy | 57 + .../test_insert_with_many_partitions.groovy | 73 + ...rt_with_many_partitions_and_buckets.groovy | 79 ++ 30 files changed, 2379 insertions(+), 551 deletions(-) create mode 100644 cloud/src/meta-service/txn_lazy_committer.cpp create mode 100644 cloud/src/meta-service/txn_lazy_committer.h create mode 100644 cloud/test/txn_lazy_commit_test.cpp create mode 100644 regression-test/data/large_txn_load_p1/test_insert_with_many_buckets.out create mode 100644 regression-test/data/large_txn_load_p1/test_insert_with_many_partitions.out create mode 100644 regression-test/data/large_txn_load_p1/test_insert_with_many_partitions_and_buckets.out create mode 100644 regression-test/suites/large_txn_load_p1/test_insert_with_many_buckets.groovy create mode 100644 regression-test/suites/large_txn_load_p1/test_insert_with_many_partitions.groovy create mode 100644 regression-test/suites/large_txn_load_p1/test_insert_with_many_partitions_and_buckets.groovy diff --git a/be/src/cloud/cloud_meta_mgr.cpp b/be/src/cloud/cloud_meta_mgr.cpp index 8105e41079da74..8b9b5a6360f1a2 100644 --- a/be/src/cloud/cloud_meta_mgr.cpp +++ b/be/src/cloud/cloud_meta_mgr.cpp @@ -847,6 +847,7 @@ Status CloudMetaMgr::commit_txn(const StreamLoadContext& ctx, bool is_2pc) { req.set_db_id(ctx.db_id); req.set_txn_id(ctx.txn_id); req.set_is_2pc(is_2pc); + req.set_lazy_commit(config::cloud_txn_lazy_commit); auto st = retry_rpc("commit txn", req, &res, &MetaService_Stub::commit_txn); if (st.ok()) { diff --git a/be/src/cloud/config.cpp b/be/src/cloud/config.cpp index 4cef5559eaa428..c97a1da4b29db3 100644 --- a/be/src/cloud/config.cpp +++ b/be/src/cloud/config.cpp @@ -62,4 +62,6 @@ DEFINE_mInt32(sync_load_for_tablets_thread, "32"); DEFINE_mBool(enable_new_tablet_do_compaction, "false"); +DEFINE_Bool(cloud_txn_lazy_commit, "false"); + } // namespace doris::config diff --git a/be/src/cloud/config.h b/be/src/cloud/config.h index 656c98a4343cb4..8c6d2b441f858c 100644 --- a/be/src/cloud/config.h +++ b/be/src/cloud/config.h @@ -95,4 +95,7 @@ DECLARE_mBool(save_load_error_log_to_s3); // the theads which sync the datas which loaded in other clusters DECLARE_mInt32(sync_load_for_tablets_thread); +// enable large txn lazy commit in meta-service `commit_txn` +DECLARE_mBool(cloud_txn_lazy_commit); + } // namespace doris::config diff --git a/cloud/src/common/config.h b/cloud/src/common/config.h index 2b31eed3f24944..7a6dc04d95c172 100644 --- a/cloud/src/common/config.h +++ b/cloud/src/common/config.h @@ -206,4 +206,8 @@ CONF_Validator(s3_client_http_scheme, [](const std::string& config) -> bool { // Max retry times for object storage request CONF_mInt64(max_s3_client_retry, "10"); +CONF_Bool(enable_txn_lazy_commit, "false"); +CONF_Int32(txn_lazy_commit_rowsets_thresold, "2048"); +CONF_Int32(txn_lazy_commit_num_threads, "8"); +CONF_Int32(txn_lazy_max_rowsets_per_batch, "2048"); } // namespace doris::cloud::config diff --git a/cloud/src/meta-service/CMakeLists.txt b/cloud/src/meta-service/CMakeLists.txt index 33dd9b6c0ceb7b..c7c4887a0686c1 100644 --- a/cloud/src/meta-service/CMakeLists.txt +++ b/cloud/src/meta-service/CMakeLists.txt @@ -24,4 +24,5 @@ add_library(MetaService doris_txn.cpp mem_txn_kv.cpp http_encode_key.cpp + txn_lazy_committer.cpp ) diff --git a/cloud/src/meta-service/meta_service.cpp b/cloud/src/meta-service/meta_service.cpp index d9f71ac22184e3..74fe5a562ba01d 100644 --- a/cloud/src/meta-service/meta_service.cpp +++ b/cloud/src/meta-service/meta_service.cpp @@ -75,6 +75,7 @@ MetaServiceImpl::MetaServiceImpl(std::shared_ptr txn_kv, resource_mgr_ = resource_mgr; rate_limiter_ = rate_limiter; rate_limiter_->init(this); + txn_lazy_committer_ = std::make_shared(txn_kv_); } MetaServiceImpl::~MetaServiceImpl() = default; @@ -237,6 +238,7 @@ void MetaServiceImpl::get_version(::google::protobuf::RpcController* controller, partition_version_key({instance_id, db_id, table_id, partition_id}, &ver_key); } + code = MetaServiceCode::OK; std::unique_ptr txn; TxnErrorCode err = txn_kv_->create_txn(&txn); if (err != TxnErrorCode::TXN_OK) { @@ -265,6 +267,20 @@ void MetaServiceImpl::get_version(::google::protobuf::RpcController* controller, msg = "malformed version value"; return; } + + if (version_pb.has_txn_id()) { + txn.reset(); + std::shared_ptr task = + txn_lazy_committer_->submit(instance_id, version_pb.txn_id()); + + std::tie(code, msg) = task->wait(); + if (code != MetaServiceCode::OK) { + LOG(WARNING) << "wait txn lazy commit failed, txn_id=" << version_pb.txn_id() + << " code=" << code << " msg=" << msg; + return; + } + } + response->set_version(version_pb.version()); response->add_version_update_time_ms(version_pb.update_time_ms()); } @@ -329,6 +345,7 @@ void MetaServiceImpl::batch_get_version(::google::protobuf::RpcController* contr std::vector> version_values; version_keys.reserve(BATCH_SIZE); version_values.reserve(BATCH_SIZE); + while ((code == MetaServiceCode::OK || code == MetaServiceCode::KV_TXN_TOO_OLD) && response->versions_size() < num_acquired) { std::unique_ptr txn; @@ -390,6 +407,20 @@ void MetaServiceImpl::batch_get_version(::google::protobuf::RpcController* contr msg = "malformed version value"; break; } + if (version_pb.has_txn_id()) { + txn.reset(); + std::shared_ptr task = + txn_lazy_committer_->submit(instance_id, version_pb.txn_id()); + std::tie(code, msg) = task->wait(); + if (code != MetaServiceCode::OK) { + LOG(WARNING) << "wait txn lazy commit failed, txn_id=" + << version_pb.txn_id(); + response->clear_partition_ids(); + response->clear_table_ids(); + response->clear_versions(); + return; + } + } response->add_versions(version_pb.version()); response->add_version_update_time_ms(version_pb.update_time_ms()); } @@ -403,9 +434,9 @@ void MetaServiceImpl::batch_get_version(::google::protobuf::RpcController* contr } } -void internal_create_tablet(MetaServiceCode& code, std::string& msg, - const doris::TabletMetaCloudPB& meta, std::shared_ptr txn_kv, - const std::string& instance_id, +void internal_create_tablet(const CreateTabletsRequest* request, MetaServiceCode& code, + std::string& msg, const doris::TabletMetaCloudPB& meta, + std::shared_ptr txn_kv, const std::string& instance_id, std::set>& saved_schema) { doris::TabletMetaCloudPB tablet_meta(meta); bool has_first_rowset = tablet_meta.rs_metas_size() > 0; @@ -496,7 +527,9 @@ void internal_create_tablet(MetaServiceCode& code, std::string& msg, MetaTabletIdxKeyInfo key_info1 {instance_id, tablet_id}; meta_tablet_idx_key(key_info1, &key1); TabletIndexPB tablet_table; - // tablet_table.set_db_id(db_id); + if (request->has_db_id()) { + tablet_table.set_db_id(request->db_id()); + } tablet_table.set_table_id(table_id); tablet_table.set_index_id(index_id); tablet_table.set_partition_id(partition_id); @@ -618,11 +651,12 @@ void MetaServiceImpl::create_tablets(::google::protobuf::RpcController* controll msg = fmt::format("failed to get vault id, vault name={}", name); return; } + // [index_id, schema_version] std::set> saved_schema; TEST_SYNC_POINT_RETURN_WITH_VOID("create_tablets"); for (auto& tablet_meta : request->tablet_metas()) { - internal_create_tablet(code, msg, tablet_meta, txn_kv_, instance_id, saved_schema); + internal_create_tablet(request, code, msg, tablet_meta, txn_kv_, instance_id, saved_schema); if (code != MetaServiceCode::OK) { return; } @@ -1420,104 +1454,152 @@ void MetaServiceImpl::get_rowset(::google::protobuf::RpcController* controller, int64_t req_cc_cnt = request->cumulative_compaction_cnt(); int64_t req_cp = request->cumulative_point(); - std::unique_ptr txn; - TxnErrorCode err = txn_kv_->create_txn(&txn); - if (err != TxnErrorCode::TXN_OK) { - code = cast_as(err); - msg = "failed to create txn"; - return; - } + do { + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = "failed to create txn"; + LOG(WARNING) << msg; + return; + } - TabletIndexPB idx(request->idx()); - // Get tablet id index from kv - if (!idx.has_table_id() || !idx.has_index_id() || !idx.has_partition_id()) { + TabletIndexPB idx; + // Get tablet id index from kv get_tablet_idx(code, msg, txn.get(), instance_id, tablet_id, idx); - if (code != MetaServiceCode::OK) return; - } - // TODO(plat1ko): Judge if tablet has been dropped (in dropped index/partition) - - TabletStatsPB tablet_stat; - internal_get_tablet_stats(code, msg, txn.get(), instance_id, idx, tablet_stat, true); - if (code != MetaServiceCode::OK) return; - VLOG_DEBUG << "tablet_id=" << tablet_id << " stats=" << proto_to_json(tablet_stat); + if (code != MetaServiceCode::OK) { + return; + } + DCHECK(request->has_idx()); - int64_t bc_cnt = tablet_stat.base_compaction_cnt(); - int64_t cc_cnt = tablet_stat.cumulative_compaction_cnt(); - int64_t cp = tablet_stat.cumulative_point(); + if (idx.has_db_id()) { + std::string ver_val; + std::string ver_key = partition_version_key( + {instance_id, idx.db_id(), idx.table_id(), idx.partition_id()}); + err = txn->get(ver_key, &ver_val); + if (TxnErrorCode::TXN_OK != err && TxnErrorCode::TXN_KEY_NOT_FOUND != err) { + code = cast_as(err); + ss << "failed to get partiton version, tablet_id=" << tablet_id + << " key=" << hex(ver_key) << " err=" << err; + msg = ss.str(); + LOG(WARNING) << msg; + return; + } - response->mutable_stats()->CopyFrom(tablet_stat); + if (TxnErrorCode::TXN_OK == err) { + VersionPB version_pb; + if (!version_pb.ParseFromString(ver_val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + ss << "failed to parse version pb tablet_id=" << tablet_id + << " key=" << hex(ver_key); + msg = ss.str(); + LOG(WARNING) << msg; + return; + } - int64_t req_start = request->start_version(); - int64_t req_end = request->end_version(); - req_end = req_end < 0 ? std::numeric_limits::max() - 1 : req_end; + if (version_pb.has_txn_id()) { + txn.reset(); + std::shared_ptr task = + txn_lazy_committer_->submit(instance_id, version_pb.txn_id()); - //========================================================================== - // Find version ranges to be synchronized due to compaction - //========================================================================== - if (req_bc_cnt > bc_cnt || req_cc_cnt > cc_cnt || req_cp > cp) { - code = MetaServiceCode::INVALID_ARGUMENT; - ss << "no valid compaction_cnt or cumulative_point given. req_bc_cnt=" << req_bc_cnt - << ", bc_cnt=" << bc_cnt << ", req_cc_cnt=" << req_cc_cnt << ", cc_cnt=" << cc_cnt - << ", req_cp=" << req_cp << ", cp=" << cp; - msg = ss.str(); - return; - } - auto versions = calc_sync_versions(req_bc_cnt, bc_cnt, req_cc_cnt, cc_cnt, req_cp, cp, - req_start, req_end); - for (auto [start, end] : versions) { - internal_get_rowset(txn.get(), start, end, instance_id, tablet_id, code, msg, response); - if (code != MetaServiceCode::OK) { - return; + std::tie(code, msg) = task->wait(); + if (code != MetaServiceCode::OK) { + LOG(WARNING) << "advance_last_txn failed last_txn=" << version_pb.txn_id() + << " code=" << code << "msg=" << msg; + return; + } + continue; + } + } } - } - // get referenced schema - std::unordered_map version_to_schema; - for (auto& rowset_meta : *response->mutable_rowset_meta()) { - if (rowset_meta.has_tablet_schema()) { - version_to_schema.emplace(rowset_meta.tablet_schema().schema_version(), - rowset_meta.mutable_tablet_schema()); - rowset_meta.set_schema_version(rowset_meta.tablet_schema().schema_version()); - } - rowset_meta.set_index_id(idx.index_id()); - } - bool need_read_schema_dict = false; - auto arena = response->GetArena(); - for (auto& rowset_meta : *response->mutable_rowset_meta()) { - if (rowset_meta.has_schema_dict_key_list()) { - need_read_schema_dict = true; - } - if (rowset_meta.has_tablet_schema()) continue; - if (!rowset_meta.has_schema_version()) { + // TODO(plat1ko): Judge if tablet has been dropped (in dropped index/partition) + + TabletStatsPB tablet_stat; + internal_get_tablet_stats(code, msg, txn.get(), instance_id, idx, tablet_stat, true); + if (code != MetaServiceCode::OK) return; + VLOG_DEBUG << "tablet_id=" << tablet_id << " stats=" << proto_to_json(tablet_stat); + + int64_t bc_cnt = tablet_stat.base_compaction_cnt(); + int64_t cc_cnt = tablet_stat.cumulative_compaction_cnt(); + int64_t cp = tablet_stat.cumulative_point(); + + response->mutable_stats()->CopyFrom(tablet_stat); + + int64_t req_start = request->start_version(); + int64_t req_end = request->end_version(); + req_end = req_end < 0 ? std::numeric_limits::max() - 1 : req_end; + + //========================================================================== + // Find version ranges to be synchronized due to compaction + //========================================================================== + if (req_bc_cnt > bc_cnt || req_cc_cnt > cc_cnt || req_cp > cp) { code = MetaServiceCode::INVALID_ARGUMENT; - msg = fmt::format( - "rowset_meta must have either schema or schema_version, " - "rowset_version=[{}-{}]", - rowset_meta.start_version(), rowset_meta.end_version()); + ss << "no valid compaction_cnt or cumulative_point given. req_bc_cnt=" << req_bc_cnt + << ", bc_cnt=" << bc_cnt << ", req_cc_cnt=" << req_cc_cnt << ", cc_cnt=" << cc_cnt + << ", req_cp=" << req_cp << ", cp=" << cp; + msg = ss.str(); return; } - if (auto it = version_to_schema.find(rowset_meta.schema_version()); - it != version_to_schema.end()) { - if (arena != nullptr) { - rowset_meta.set_allocated_tablet_schema(it->second); - } else { - rowset_meta.mutable_tablet_schema()->CopyFrom(*it->second); + auto versions = calc_sync_versions(req_bc_cnt, bc_cnt, req_cc_cnt, cc_cnt, req_cp, cp, + req_start, req_end); + for (auto [start, end] : versions) { + internal_get_rowset(txn.get(), start, end, instance_id, tablet_id, code, msg, response); + if (code != MetaServiceCode::OK) { + return; } - } else { - auto key = meta_schema_key({instance_id, idx.index_id(), rowset_meta.schema_version()}); - if (!try_fetch_and_parse_schema(txn.get(), rowset_meta, key, code, msg)) { + } + + // get referenced schema + std::unordered_map version_to_schema; + for (auto& rowset_meta : *response->mutable_rowset_meta()) { + if (rowset_meta.has_tablet_schema()) { + version_to_schema.emplace(rowset_meta.tablet_schema().schema_version(), + rowset_meta.mutable_tablet_schema()); + rowset_meta.set_schema_version(rowset_meta.tablet_schema().schema_version()); + } + rowset_meta.set_index_id(idx.index_id()); + } + bool need_read_schema_dict = false; + auto arena = response->GetArena(); + for (auto& rowset_meta : *response->mutable_rowset_meta()) { + if (rowset_meta.has_schema_dict_key_list()) { + need_read_schema_dict = true; + } + if (rowset_meta.has_tablet_schema()) continue; + if (!rowset_meta.has_schema_version()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = fmt::format( + "rowset_meta must have either schema or schema_version, " + "rowset_version=[{}-{}]", + rowset_meta.start_version(), rowset_meta.end_version()); return; } - version_to_schema.emplace(rowset_meta.schema_version(), - rowset_meta.mutable_tablet_schema()); + if (auto it = version_to_schema.find(rowset_meta.schema_version()); + it != version_to_schema.end()) { + if (arena != nullptr) { + rowset_meta.set_allocated_tablet_schema(it->second); + } else { + rowset_meta.mutable_tablet_schema()->CopyFrom(*it->second); + } + } else { + auto key = meta_schema_key( + {instance_id, idx.index_id(), rowset_meta.schema_version()}); + if (!try_fetch_and_parse_schema(txn.get(), rowset_meta, key, code, msg)) { + return; + } + version_to_schema.emplace(rowset_meta.schema_version(), + rowset_meta.mutable_tablet_schema()); + } } - } - if (need_read_schema_dict) { - read_schema_from_dict(code, msg, instance_id, idx.index_id(), txn.get(), - response->mutable_rowset_meta()); - if (code != MetaServiceCode::OK) return; - } + if (need_read_schema_dict) { + read_schema_from_dict(code, msg, instance_id, idx.index_id(), txn.get(), + response->mutable_rowset_meta()); + if (code != MetaServiceCode::OK) return; + } + break; + } while (true); } void MetaServiceImpl::get_tablet_stats(::google::protobuf::RpcController* controller, diff --git a/cloud/src/meta-service/meta_service.h b/cloud/src/meta-service/meta_service.h index 8b812aeffc9a94..5415e7ce1e6962 100644 --- a/cloud/src/meta-service/meta_service.h +++ b/cloud/src/meta-service/meta_service.h @@ -30,6 +30,7 @@ #include "common/config.h" #include "cpp/sync_point.h" #include "meta-service/txn_kv.h" +#include "meta-service/txn_lazy_committer.h" #include "rate-limiter/rate_limiter.h" #include "resource-manager/resource_manager.h" @@ -296,6 +297,7 @@ class MetaServiceImpl : public cloud::MetaService { std::shared_ptr txn_kv_; std::shared_ptr resource_mgr_; std::shared_ptr rate_limiter_; + std::shared_ptr txn_lazy_committer_; }; class MetaServiceProxy final : public MetaService { diff --git a/cloud/src/meta-service/meta_service_txn.cpp b/cloud/src/meta-service/meta_service_txn.cpp index af28e180e306d7..8a73b941f42d11 100644 --- a/cloud/src/meta-service/meta_service_txn.cpp +++ b/cloud/src/meta-service/meta_service_txn.cpp @@ -751,7 +751,7 @@ void MetaServiceImpl::reset_rl_progress(::google::protobuf::RpcController* contr } void scan_tmp_rowset( - const std::string& instance_id, int64_t txn_id, std::shared_ptr& txn_kv, + const std::string& instance_id, int64_t txn_id, std::shared_ptr txn_kv, MetaServiceCode& code, std::string& msg, int64_t* db_id, std::vector>* tmp_rowsets_meta) { // Create a readonly txn for scan tmp rowset @@ -760,7 +760,7 @@ void scan_tmp_rowset( TxnErrorCode err = txn_kv->create_txn(&txn); if (err != TxnErrorCode::TXN_OK) { code = cast_as(err); - ss << "filed to create txn, txn_id=" << txn_id << " err=" << err; + ss << "failed to create txn, txn_id=" << txn_id << " err=" << err; msg = ss.str(); LOG(WARNING) << msg; return; @@ -894,6 +894,7 @@ void update_tablet_stats(const StatsTabletKeyInfo& info, const TabletStats& stat txn->put(key, val); } } + /** * 0. Extract txn_id from request * 1. Get db id from TxnKv with txn_id @@ -910,94 +911,485 @@ void update_tablet_stats(const StatsTabletKeyInfo& info, const TabletStats& stat */ void commit_txn_immediately( const CommitTxnRequest* request, CommitTxnResponse* response, - std::shared_ptr& txn_kv, MetaServiceCode& code, std::string& msg, - const std::string& instance_id, int64_t db_id, + std::shared_ptr& txn_kv, std::shared_ptr& txn_lazy_committer, + MetaServiceCode& code, std::string& msg, const std::string& instance_id, int64_t db_id, std::vector>& tmp_rowsets_meta) { std::stringstream ss; int64_t txn_id = request->txn_id(); - std::unique_ptr txn; - TxnErrorCode err = txn_kv->create_txn(&txn); - if (err != TxnErrorCode::TXN_OK) { - code = cast_as(err); - ss << "filed to create txn, txn_id=" << txn_id << " err=" << err; - msg = ss.str(); - LOG(WARNING) << msg; - return; - } + bool need_advance_last_txn = false; + int64_t last_pending_txn_id = 0; + do { + std::unique_ptr txn; + TxnErrorCode err = txn_kv->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "failed to create txn, txn_id=" << txn_id << " err=" << err; + msg = ss.str(); + LOG(WARNING) << msg; + return; + } - // Get txn info with db_id and txn_id - std::string info_val; // Will be reused when saving updated txn - const std::string info_key = txn_info_key({instance_id, db_id, txn_id}); - err = txn->get(info_key, &info_val); - if (err != TxnErrorCode::TXN_OK) { - code = err == TxnErrorCode::TXN_KEY_NOT_FOUND ? MetaServiceCode::TXN_ID_NOT_FOUND - : cast_as(err); - if (err == TxnErrorCode::TXN_KEY_NOT_FOUND) { - ss << "transaction [" << txn_id << "] not found, db_id=" << db_id; - } else { - ss << "failed to get txn_info, db_id=" << db_id << " txn_id=" << txn_id - << " err=" << err; + // Get txn info with db_id and txn_id + std::string info_val; // Will be reused when saving updated txn + const std::string info_key = txn_info_key({instance_id, db_id, txn_id}); + err = txn->get(info_key, &info_val); + if (err != TxnErrorCode::TXN_OK) { + code = err == TxnErrorCode::TXN_KEY_NOT_FOUND ? MetaServiceCode::TXN_ID_NOT_FOUND + : cast_as(err); + if (err == TxnErrorCode::TXN_KEY_NOT_FOUND) { + ss << "transaction [" << txn_id << "] not found, db_id=" << db_id; + } else { + ss << "failed to get txn_info, db_id=" << db_id << " txn_id=" << txn_id + << " err=" << err; + } + msg = ss.str(); + LOG(WARNING) << msg; + return; } - msg = ss.str(); - LOG(WARNING) << msg; - return; - } - TxnInfoPB txn_info; - if (!txn_info.ParseFromString(info_val)) { - code = MetaServiceCode::PROTOBUF_PARSE_ERR; - ss << "failed to parse txn_info, db_id=" << db_id << " txn_id=" << txn_id; - msg = ss.str(); - LOG(WARNING) << msg; - return; - } + TxnInfoPB txn_info; + if (!txn_info.ParseFromString(info_val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + ss << "failed to parse txn_info, db_id=" << db_id << " txn_id=" << txn_id; + msg = ss.str(); + LOG(WARNING) << msg; + return; + } - // TODO: do more check like txn state, 2PC etc. - DCHECK(txn_info.txn_id() == txn_id); - if (txn_info.status() == TxnStatusPB::TXN_STATUS_ABORTED) { - code = MetaServiceCode::TXN_ALREADY_ABORTED; - ss << "transaction [" << txn_id << "] is already aborted, db_id=" << db_id; - msg = ss.str(); - LOG(WARNING) << msg; - return; - } + // TODO: do more check like txn state, 2PC etc. + DCHECK(txn_info.txn_id() == txn_id); + if (txn_info.status() == TxnStatusPB::TXN_STATUS_ABORTED) { + code = MetaServiceCode::TXN_ALREADY_ABORTED; + ss << "transaction [" << txn_id << "] is already aborted, db_id=" << db_id; + msg = ss.str(); + LOG(WARNING) << msg; + return; + } - if (txn_info.status() == TxnStatusPB::TXN_STATUS_VISIBLE) { - if (request->has_is_2pc() && request->is_2pc()) { - code = MetaServiceCode::TXN_ALREADY_VISIBLE; - ss << "transaction [" << txn_id << "] is already visible, not pre-committed."; + if (txn_info.status() == TxnStatusPB::TXN_STATUS_VISIBLE) { + if (request->has_is_2pc() && request->is_2pc()) { + code = MetaServiceCode::TXN_ALREADY_VISIBLE; + ss << "transaction [" << txn_id << "] is already visible, not pre-committed."; + msg = ss.str(); + LOG(INFO) << msg; + response->mutable_txn_info()->CopyFrom(txn_info); + return; + } + code = MetaServiceCode::OK; + ss << "transaction is already visible: db_id=" << db_id << " txn_id=" << txn_id; msg = ss.str(); LOG(INFO) << msg; response->mutable_txn_info()->CopyFrom(txn_info); return; } - code = MetaServiceCode::OK; - ss << "transaction is already visible: db_id=" << db_id << " txn_id=" << txn_id; - msg = ss.str(); - LOG(INFO) << msg; - response->mutable_txn_info()->CopyFrom(txn_info); - return; - } - if (request->has_is_2pc() && request->is_2pc() && - txn_info.status() == TxnStatusPB::TXN_STATUS_PREPARED) { - code = MetaServiceCode::TXN_INVALID_STATUS; - ss << "transaction is prepare, not pre-committed: db_id=" << db_id << " txn_id" << txn_id; - msg = ss.str(); - LOG(WARNING) << msg; - return; - } + if (request->has_is_2pc() && request->is_2pc() && + txn_info.status() == TxnStatusPB::TXN_STATUS_PREPARED) { + code = MetaServiceCode::TXN_INVALID_STATUS; + ss << "transaction is prepare, not pre-committed: db_id=" << db_id << " txn_id" + << txn_id; + msg = ss.str(); + LOG(WARNING) << msg; + return; + } - LOG(INFO) << "txn_id=" << txn_id << " txn_info=" << txn_info.ShortDebugString(); + LOG(INFO) << "txn_id=" << txn_id << " txn_info=" << txn_info.ShortDebugString(); - // Prepare rowset meta and new_versions + // Prepare rowset meta and new_versions + // Read tablet indexes in batch. + std::vector tablet_idx_keys; + for (auto& [_, i] : tmp_rowsets_meta) { + tablet_idx_keys.push_back(meta_tablet_idx_key({instance_id, i.tablet_id()})); + } + std::vector> tablet_idx_values; + err = txn->batch_get(&tablet_idx_values, tablet_idx_keys, + Transaction::BatchGetOptions(false)); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "failed to get tablet table index ids, err=" << err; + msg = ss.str(); + LOG(WARNING) << msg << " txn_id=" << txn_id; + return; + } + + size_t total_rowsets = tmp_rowsets_meta.size(); + // tablet_id -> {table/index/partition}_id + std::unordered_map tablet_ids; + // table_id -> tablets_ids + std::unordered_map> table_id_tablet_ids; + for (size_t i = 0; i < total_rowsets; i++) { + uint64_t tablet_id = tmp_rowsets_meta[i].second.tablet_id(); + if (!tablet_idx_values[i].has_value()) [[unlikely]] { + // The value must existed + code = MetaServiceCode::KV_TXN_GET_ERR; + ss << "failed to get tablet table index ids, err=not found" + << " tablet_id=" << tablet_id << " key=" << hex(tablet_idx_keys[i]); + msg = ss.str(); + LOG(WARNING) << msg << " err=" << err << " txn_id=" << txn_id; + return; + } + if (!tablet_ids[tablet_id].ParseFromString(tablet_idx_values[i].value())) [[unlikely]] { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + ss << "malformed tablet index value tablet_id=" << tablet_id + << " txn_id=" << txn_id; + msg = ss.str(); + LOG(WARNING) << msg; + return; + } + table_id_tablet_ids[tablet_ids[tablet_id].table_id()].push_back(tablet_id); + VLOG_DEBUG << "tablet_id:" << tablet_id + << " value:" << tablet_ids[tablet_id].ShortDebugString(); + } + + tablet_idx_keys.clear(); + tablet_idx_values.clear(); + + // {table/partition} -> version + std::unordered_map new_versions; + std::vector version_keys; + for (auto& [_, i] : tmp_rowsets_meta) { + int64_t tablet_id = i.tablet_id(); + int64_t table_id = tablet_ids[tablet_id].table_id(); + int64_t partition_id = i.partition_id(); + std::string ver_key = + partition_version_key({instance_id, db_id, table_id, partition_id}); + if (new_versions.count(ver_key) == 0) { + new_versions.insert({ver_key, 0}); + version_keys.push_back(std::move(ver_key)); + } + } + std::vector> version_values; + err = txn->batch_get(&version_values, version_keys); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "failed to get partition versions, err=" << err; + msg = ss.str(); + LOG(WARNING) << msg << " txn_id=" << txn_id; + return; + } + size_t total_versions = version_keys.size(); + for (size_t i = 0; i < total_versions; i++) { + int64_t version; + if (version_values[i].has_value()) { + VersionPB version_pb; + if (!version_pb.ParseFromString(version_values[i].value())) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + ss << "failed to parse version pb txn_id=" << txn_id + << " key=" << hex(version_keys[i]); + msg = ss.str(); + return; + } + if (version_pb.has_txn_id()) { + need_advance_last_txn = true; + last_pending_txn_id = version_pb.txn_id(); + break; + } + version = version_pb.version(); + } else { + version = 1; + } + new_versions[version_keys[i]] = version + 1; + need_advance_last_txn = false; + last_pending_txn_id = 0; + } + version_keys.clear(); + version_values.clear(); + + if (need_advance_last_txn) { + txn.reset(); + DCHECK(last_pending_txn_id > 0); + std::shared_ptr task = + txn_lazy_committer->submit(instance_id, last_pending_txn_id); + + std::tie(code, msg) = task->wait(); + need_advance_last_txn = false; + last_pending_txn_id = 0; + if (code != MetaServiceCode::OK) { + LOG(WARNING) << "advance_last_txn failed last_txn=" << last_pending_txn_id + << " code=" << code << "msg=" << msg; + return; + } + continue; + } + + std::vector> rowsets; + std::unordered_map tablet_stats; // tablet_id -> stats + rowsets.reserve(tmp_rowsets_meta.size()); + for (auto& [_, i] : tmp_rowsets_meta) { + int64_t tablet_id = i.tablet_id(); + int64_t table_id = tablet_ids[tablet_id].table_id(); + int64_t partition_id = i.partition_id(); + std::string ver_key = + partition_version_key({instance_id, db_id, table_id, partition_id}); + if (new_versions[ver_key] == 0) [[unlikely]] { + // it is impossible. + code = MetaServiceCode::UNDEFINED_ERR; + ss << "failed to get partition version key, the target version not exists in " + "new_versions." + << " txn_id=" << txn_id; + msg = ss.str(); + LOG(ERROR) << msg; + return; + } + + // Update rowset version + int64_t new_version = new_versions[ver_key]; + i.set_start_version(new_version); + i.set_end_version(new_version); + + std::string key = meta_rowset_key({instance_id, tablet_id, i.end_version()}); + std::string val; + if (!i.SerializeToString(&val)) { + code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; + ss << "failed to serialize rowset_meta, txn_id=" << txn_id; + msg = ss.str(); + return; + } + rowsets.emplace_back(std::move(key), std::move(val)); + + // Accumulate affected rows + auto& stats = tablet_stats[tablet_id]; + stats.data_size += i.data_disk_size(); + stats.num_rows += i.num_rows(); + ++stats.num_rowsets; + stats.num_segs += i.num_segments(); + } // for tmp_rowsets_meta + + // process mow table, check lock and remove pending key + std::vector lock_keys; + lock_keys.reserve(request->mow_table_ids().size()); + for (auto table_id : request->mow_table_ids()) { + lock_keys.push_back(meta_delete_bitmap_update_lock_key({instance_id, table_id, -1})); + } + std::vector> lock_values; + err = txn->batch_get(&lock_values, lock_keys); + if (err != TxnErrorCode::TXN_OK) { + ss << "failed to get delete bitmap update lock key info, instance_id=" << instance_id + << " err=" << err; + msg = ss.str(); + code = cast_as(err); + LOG(WARNING) << msg << " txn_id=" << txn_id; + return; + } + size_t total_locks = lock_keys.size(); + for (size_t i = 0; i < total_locks; i++) { + int64_t table_id = request->mow_table_ids(i); + // When the key does not exist, it means the lock has been acquired + // by another transaction and successfully committed. + if (!lock_values[i].has_value()) { + ss << "get delete bitmap update lock info, lock is expired" + << " table_id=" << table_id << " key=" << hex(lock_keys[i]); + code = MetaServiceCode::LOCK_EXPIRED; + msg = ss.str(); + LOG(WARNING) << msg << " txn_id=" << txn_id; + return; + } + + DeleteBitmapUpdateLockPB lock_info; + if (!lock_info.ParseFromString(lock_values[i].value())) [[unlikely]] { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = "failed to parse DeleteBitmapUpdateLockPB"; + LOG(WARNING) << msg << " txn_id=" << txn_id; + return; + } + if (lock_info.lock_id() != request->txn_id()) { + msg = "lock is expired"; + code = MetaServiceCode::LOCK_EXPIRED; + return; + } + txn->remove(lock_keys[i]); + LOG(INFO) << "xxx remove delete bitmap lock, lock_key=" << hex(lock_keys[i]) + << " txn_id=" << txn_id; + + for (auto tablet_id : table_id_tablet_ids[table_id]) { + std::string pending_key = meta_pending_delete_bitmap_key({instance_id, tablet_id}); + txn->remove(pending_key); + LOG(INFO) << "xxx remove delete bitmap pending key, pending_key=" + << hex(pending_key) << " txn_id=" << txn_id; + } + } + lock_keys.clear(); + lock_values.clear(); + + // Save rowset meta + for (auto& i : rowsets) { + size_t rowset_size = i.first.size() + i.second.size(); + txn->put(i.first, i.second); + LOG(INFO) << "xxx put rowset_key=" << hex(i.first) << " txn_id=" << txn_id + << " rowset_size=" << rowset_size; + } + + // Save versions + int64_t version_update_time_ms = + duration_cast(system_clock::now().time_since_epoch()).count(); + response->set_version_update_time_ms(version_update_time_ms); + for (auto& i : new_versions) { + std::string ver_val; + VersionPB version_pb; + version_pb.set_version(i.second); + version_pb.set_update_time_ms(version_update_time_ms); + if (!version_pb.SerializeToString(&ver_val)) { + code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; + ss << "failed to serialize version_pb when saving, txn_id=" << txn_id; + msg = ss.str(); + return; + } + + txn->put(i.first, ver_val); + LOG(INFO) << "xxx put partition_version_key=" << hex(i.first) << " version:" << i.second + << " txn_id=" << txn_id << " update_time=" << version_update_time_ms; + + std::string_view ver_key = i.first; + ver_key.remove_prefix(1); // Remove key space + // PartitionVersionKeyInfo {instance_id, db_id, table_id, partition_id} + std::vector, int, int>> out; + int ret = decode_key(&ver_key, &out); + if (ret != 0) [[unlikely]] { + // decode version key error means this is something wrong, + // we can not continue this txn + LOG(WARNING) << "failed to decode key, ret=" << ret << " key=" << hex(ver_key); + code = MetaServiceCode::UNDEFINED_ERR; + msg = "decode version key error"; + return; + } + + int64_t table_id = std::get(std::get<0>(out[4])); + int64_t partition_id = std::get(std::get<0>(out[5])); + VLOG_DEBUG << " table_id=" << table_id << " partition_id=" << partition_id; + + response->add_table_ids(table_id); + response->add_partition_ids(partition_id); + response->add_versions(i.second); + } + + // Save table versions + for (auto& i : table_id_tablet_ids) { + std::string ver_key = table_version_key({instance_id, db_id, i.first}); + txn->atomic_add(ver_key, 1); + LOG(INFO) << "xxx atomic add table_version_key=" << hex(ver_key) + << " txn_id=" << txn_id; + } + + LOG(INFO) << " before update txn_info=" << txn_info.ShortDebugString(); + + // Update txn_info + txn_info.set_status(TxnStatusPB::TXN_STATUS_VISIBLE); + + auto now_time = system_clock::now(); + uint64_t commit_time = duration_cast(now_time.time_since_epoch()).count(); + if ((txn_info.prepare_time() + txn_info.timeout_ms()) < commit_time) { + code = MetaServiceCode::UNDEFINED_ERR; + msg = fmt::format("txn is expired, not allow to commit txn_id={}", txn_id); + LOG(INFO) << msg << " prepare_time=" << txn_info.prepare_time() + << " timeout_ms=" << txn_info.timeout_ms() << " commit_time=" << commit_time; + return; + } + txn_info.set_commit_time(commit_time); + txn_info.set_finish_time(commit_time); + if (request->has_commit_attachment()) { + txn_info.mutable_commit_attachment()->CopyFrom(request->commit_attachment()); + } + LOG(INFO) << "after update txn_info=" << txn_info.ShortDebugString(); + info_val.clear(); + if (!txn_info.SerializeToString(&info_val)) { + code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; + ss << "failed to serialize txn_info when saving, txn_id=" << txn_id; + msg = ss.str(); + return; + } + txn->put(info_key, info_val); + LOG(INFO) << "xxx put info_key=" << hex(info_key) << " txn_id=" << txn_id; + + // Update stats of affected tablet + for (auto& [tablet_id, stats] : tablet_stats) { + DCHECK(tablet_ids.count(tablet_id)); + auto& tablet_idx = tablet_ids[tablet_id]; + StatsTabletKeyInfo info {instance_id, tablet_idx.table_id(), tablet_idx.index_id(), + tablet_idx.partition_id(), tablet_id}; + update_tablet_stats(info, stats, txn, code, msg); + if (code != MetaServiceCode::OK) return; + } + // Remove tmp rowset meta + for (auto& [k, _] : tmp_rowsets_meta) { + txn->remove(k); + LOG(INFO) << "xxx remove tmp_rowset_key=" << hex(k) << " txn_id=" << txn_id; + } + + const std::string running_key = txn_running_key({instance_id, db_id, txn_id}); + LOG(INFO) << "xxx remove running_key=" << hex(running_key) << " txn_id=" << txn_id; + txn->remove(running_key); + + std::string recycle_val; + std::string recycle_key = recycle_txn_key({instance_id, db_id, txn_id}); + RecycleTxnPB recycle_pb; + recycle_pb.set_creation_time(commit_time); + recycle_pb.set_label(txn_info.label()); + + if (!recycle_pb.SerializeToString(&recycle_val)) { + code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; + ss << "failed to serialize recycle_pb, txn_id=" << txn_id; + msg = ss.str(); + return; + } + txn->put(recycle_key, recycle_val); + + if (txn_info.load_job_source_type() == + LoadJobSourceTypePB::LOAD_JOB_SRC_TYPE_ROUTINE_LOAD_TASK) { + put_routine_load_progress(code, msg, instance_id, request, txn.get(), db_id); + } + + LOG(INFO) << "xxx commit_txn put recycle_key key=" << hex(recycle_key) + << " txn_id=" << txn_id; + LOG(INFO) << "commit_txn put_size=" << txn->put_bytes() + << " del_size=" << txn->delete_bytes() << " num_put_keys=" << txn->num_put_keys() + << " num_del_keys=" << txn->num_del_keys() + << " txn_size=" << txn->approximate_bytes() << " txn_id=" << txn_id; + + // Finally we are done... + err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "failed to commit kv txn, txn_id=" << txn_id << " err=" << err; + msg = ss.str(); + return; + } + + // calculate table stats from tablets stats + std::map table_stats; + std::vector base_tablet_ids(request->base_tablet_ids().begin(), + request->base_tablet_ids().end()); + calc_table_stats(tablet_ids, tablet_stats, table_stats, base_tablet_ids); + for (const auto& pair : table_stats) { + TableStatsPB* stats_pb = response->add_table_stats(); + auto table_id = pair.first; + auto stats = pair.second; + get_pb_from_tablestats(stats, stats_pb); + stats_pb->set_table_id(table_id); + VLOG_DEBUG << "Add TableStats to CommitTxnResponse. txn_id=" << txn_id + << " table_id=" << table_id + << " updated_row_count=" << stats_pb->updated_row_count(); + } + response->mutable_txn_info()->CopyFrom(txn_info); + break; + } while (true); +} // end commit_txn_immediately + +void get_tablet_indexes( + const std::string& instance_id, int64_t txn_id, + const std::vector>& tmp_rowsets_meta, + std::unique_ptr& txn, MetaServiceCode& code, std::string& msg, + std::unordered_map* tablet_ids, + std::unordered_map>* table_id_tablet_ids, + bool* need_repair_tablet_idx) { // Read tablet indexes in batch. + std::stringstream ss; std::vector tablet_idx_keys; for (auto& [_, i] : tmp_rowsets_meta) { tablet_idx_keys.push_back(meta_tablet_idx_key({instance_id, i.tablet_id()})); } std::vector> tablet_idx_values; - err = txn->batch_get(&tablet_idx_values, tablet_idx_keys, Transaction::BatchGetOptions(false)); + TxnErrorCode err = + txn->batch_get(&tablet_idx_values, tablet_idx_keys, Transaction::BatchGetOptions(true)); if (err != TxnErrorCode::TXN_OK) { code = cast_as(err); ss << "failed to get tablet table index ids, err=" << err; @@ -1006,12 +1398,7 @@ void commit_txn_immediately( return; } - size_t total_rowsets = tmp_rowsets_meta.size(); - // tablet_id -> {table/index/partition}_id - std::unordered_map tablet_ids; - // table_id -> tablets_ids - std::unordered_map> table_id_tablet_ids; - for (size_t i = 0; i < total_rowsets; i++) { + for (size_t i = 0; i < tmp_rowsets_meta.size(); i++) { uint64_t tablet_id = tmp_rowsets_meta[i].second.tablet_id(); if (!tablet_idx_values[i].has_value()) [[unlikely]] { // The value must existed @@ -1022,353 +1409,476 @@ void commit_txn_immediately( LOG(WARNING) << msg << " err=" << err << " txn_id=" << txn_id; return; } - if (!tablet_ids[tablet_id].ParseFromString(tablet_idx_values[i].value())) [[unlikely]] { + if (!(*tablet_ids)[tablet_id].ParseFromString(tablet_idx_values[i].value())) [[unlikely]] { code = MetaServiceCode::PROTOBUF_PARSE_ERR; ss << "malformed tablet index value tablet_id=" << tablet_id << " txn_id=" << txn_id; msg = ss.str(); LOG(WARNING) << msg; return; } - table_id_tablet_ids[tablet_ids[tablet_id].table_id()].push_back(tablet_id); + if (!(*tablet_ids)[tablet_id].has_db_id()) { + *need_repair_tablet_idx = true; + } + (*table_id_tablet_ids)[(*tablet_ids)[tablet_id].table_id()].push_back(tablet_id); VLOG_DEBUG << "tablet_id:" << tablet_id - << " value:" << tablet_ids[tablet_id].ShortDebugString(); + << " value:" << (*tablet_ids)[tablet_id].ShortDebugString(); } tablet_idx_keys.clear(); tablet_idx_values.clear(); +} - // {table/partition} -> version - std::unordered_map new_versions; - std::vector version_keys; +void repair_tablet_index( + std::shared_ptr& txn_kv, MetaServiceCode& code, std::string& msg, + const std::string& instance_id, int64_t db_id, int64_t txn_id, + const std::vector>& tmp_rowsets_meta) { + std::stringstream ss; + std::vector tablet_idx_keys; for (auto& [_, i] : tmp_rowsets_meta) { - int64_t tablet_id = i.tablet_id(); - int64_t table_id = tablet_ids[tablet_id].table_id(); - int64_t partition_id = i.partition_id(); - std::string ver_key = partition_version_key({instance_id, db_id, table_id, partition_id}); - if (new_versions.count(ver_key) == 0) { - new_versions.insert({ver_key, 0}); - version_keys.push_back(std::move(ver_key)); - } - } - std::vector> version_values; - err = txn->batch_get(&version_values, version_keys); - if (err != TxnErrorCode::TXN_OK) { - code = cast_as(err); - ss << "failed to get partition versions, err=" << err; - msg = ss.str(); - LOG(WARNING) << msg << " txn_id=" << txn_id; - return; + tablet_idx_keys.push_back(meta_tablet_idx_key({instance_id, i.tablet_id()})); } - size_t total_versions = version_keys.size(); - for (size_t i = 0; i < total_versions; i++) { - int64_t version; - if (version_values[i].has_value()) { - VersionPB version_pb; - if (!version_pb.ParseFromString(version_values[i].value())) { + +#define MAX_TABLET_INDEX_NUM 2048 + for (size_t i = 0; i < tablet_idx_keys.size(); i += MAX_TABLET_INDEX_NUM) { + size_t end = (i + MAX_TABLET_INDEX_NUM) > tablet_idx_keys.size() ? tablet_idx_keys.size() + : i + MAX_TABLET_INDEX_NUM; + const std::vector sub_tablet_idx_keys(tablet_idx_keys.begin() + i, + tablet_idx_keys.begin() + end); + std::unique_ptr txn; + TxnErrorCode err = txn_kv->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "failed to create txn, txn_id=" << txn_id << " err=" << err; + msg = ss.str(); + LOG(WARNING) << msg; + return; + } + + std::vector> tablet_idx_values; + // batch get snapshot is false + err = txn->batch_get(&tablet_idx_values, tablet_idx_keys, + Transaction::BatchGetOptions(false)); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "failed to get tablet table index ids, err=" << err; + msg = ss.str(); + LOG(WARNING) << msg << " txn_id=" << txn_id; + return; + } + + for (size_t j = 0; j < sub_tablet_idx_keys.size(); j++) { + if (!tablet_idx_values[j].has_value()) [[unlikely]] { + // The value must existed + code = MetaServiceCode::KV_TXN_GET_ERR; + ss << "failed to get tablet table index ids, err=not found" + << " key=" << hex(tablet_idx_keys[j]); + msg = ss.str(); + LOG(WARNING) << msg << " err=" << err << " txn_id=" << txn_id; + return; + } + TabletIndexPB tablet_idx_pb; + if (!tablet_idx_pb.ParseFromString(tablet_idx_values[j].value())) [[unlikely]] { code = MetaServiceCode::PROTOBUF_PARSE_ERR; - ss << "failed to parse version pb txn_id=" << txn_id - << " key=" << hex(version_keys[i]); + ss << "malformed tablet index value key=" << hex(tablet_idx_keys[j]) + << " txn_id=" << txn_id; msg = ss.str(); + LOG(WARNING) << msg; return; } - version = version_pb.version(); - } else { - version = 1; + + if (!tablet_idx_pb.has_db_id()) { + tablet_idx_pb.set_db_id(db_id); + std::string idx_val; + if (!tablet_idx_pb.SerializeToString(&idx_val)) { + code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; + ss << "failed to serialize tablet index value key=" << hex(tablet_idx_keys[j]) + << " txn_id=" << txn_id; + msg = ss.str(); + LOG(WARNING) << msg; + return; + } + txn->put(sub_tablet_idx_keys[j], idx_val); + } + } + + err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "failed to commit kv txn, txn_id=" << txn_id << " err=" << err; + msg = ss.str(); + LOG(WARNING) << msg; + return; + } + } +} + +void commit_txn_eventually( + const CommitTxnRequest* request, CommitTxnResponse* response, + std::shared_ptr& txn_kv, std::shared_ptr& txn_lazy_committer, + MetaServiceCode& code, std::string& msg, const std::string& instance_id, int64_t db_id, + const std::vector>& tmp_rowsets_meta) { + std::stringstream ss; + TxnErrorCode err = TxnErrorCode::TXN_OK; + int64_t txn_id = request->txn_id(); + bool need_advance_last_txn = false; + int64_t last_pending_txn_id = 0; + + do { + std::unique_ptr txn; + err = txn_kv->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "failed to create txn, txn_id=" << txn_id << " err=" << err; + msg = ss.str(); + LOG(WARNING) << msg; + return; + } + + // tablet_id -> {table/index/partition}_id + std::unordered_map tablet_ids; + // table_id -> tablets_ids + std::unordered_map> table_id_tablet_ids; + bool need_repair_tablet_idx = false; + get_tablet_indexes(instance_id, txn_id, tmp_rowsets_meta, txn, code, msg, &tablet_ids, + &table_id_tablet_ids, &need_repair_tablet_idx); + if (code != MetaServiceCode::OK) { + LOG(WARNING) << "get_tablet_indexes failed, txn_id=" << txn_id << " code=" << code; + return; + } + + if (need_repair_tablet_idx) { + txn.reset(); + repair_tablet_index(txn_kv, code, msg, instance_id, db_id, txn_id, tmp_rowsets_meta); + if (code != MetaServiceCode::OK) { + LOG(WARNING) << "repair_tablet_index failed, txn_id=" << txn_id << " code=" << code; + return; + } + continue; + } + + std::unordered_map new_versions; + std::vector version_keys; + for (auto& [_, i] : tmp_rowsets_meta) { + int64_t tablet_id = i.tablet_id(); + int64_t table_id = tablet_ids[tablet_id].table_id(); + int64_t partition_id = i.partition_id(); + std::string ver_key = + partition_version_key({instance_id, db_id, table_id, partition_id}); + if (new_versions.count(ver_key) == 0) { + new_versions.insert({ver_key, 0}); + version_keys.push_back(std::move(ver_key)); + } } - new_versions[version_keys[i]] = version + 1; - } - version_keys.clear(); - version_values.clear(); - std::vector> rowsets; - std::unordered_map tablet_stats; // tablet_id -> stats - rowsets.reserve(tmp_rowsets_meta.size()); - for (auto& [_, i] : tmp_rowsets_meta) { - int64_t tablet_id = i.tablet_id(); - int64_t table_id = tablet_ids[tablet_id].table_id(); - int64_t partition_id = i.partition_id(); - std::string ver_key = partition_version_key({instance_id, db_id, table_id, partition_id}); - if (new_versions[ver_key] == 0) [[unlikely]] { - // it is impossible. - code = MetaServiceCode::UNDEFINED_ERR; - ss << "failed to get partition version key, the target version not exists in " - "new_versions." - << " txn_id=" << txn_id; + std::vector> version_values; + err = txn->batch_get(&version_values, version_keys); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "failed to get partition versions, err=" << err; msg = ss.str(); - LOG(ERROR) << msg; + LOG(WARNING) << msg << " txn_id=" << txn_id; return; } - // Update rowset version - int64_t new_version = new_versions[ver_key]; - i.set_start_version(new_version); - i.set_end_version(new_version); + for (size_t i = 0; i < version_keys.size(); i++) { + int64_t version; + if (version_values[i].has_value()) { + VersionPB version_pb; + if (!version_pb.ParseFromString(version_values[i].value())) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + ss << "failed to parse version pb txn_id=" << txn_id + << " key=" << hex(version_keys[i]); + msg = ss.str(); + return; + } + if (version_pb.has_txn_id()) { + need_advance_last_txn = true; + last_pending_txn_id = version_pb.txn_id(); + break; + } + version = version_pb.version(); + } else { + version = 1; + } + new_versions[version_keys[i]] = version + 1; + need_advance_last_txn = false; + last_pending_txn_id = 0; + } + + if (need_advance_last_txn) { + txn.reset(); + DCHECK(last_pending_txn_id > 0); + std::shared_ptr task = + txn_lazy_committer->submit(instance_id, last_pending_txn_id); + + std::tie(code, msg) = task->wait(); + need_advance_last_txn = false; + last_pending_txn_id = 0; + if (code != MetaServiceCode::OK) { + LOG(WARNING) << "advance_last_txn failed last_txn=" << last_pending_txn_id + << " code=" << code << "msg=" << msg; + return; + } - std::string key = meta_rowset_key({instance_id, tablet_id, i.end_version()}); - std::string val; - if (!i.SerializeToString(&val)) { - code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; - ss << "failed to serialize rowset_meta, txn_id=" << txn_id; + // there maybe concurrent commit_txn_eventually, so we need continue to make sure + // partition versionPB has no txn_id + continue; + } + + std::string info_val; + const std::string info_key = txn_info_key({instance_id, db_id, txn_id}); + err = txn->get(info_key, &info_val); + if (err != TxnErrorCode::TXN_OK) { + code = err == TxnErrorCode::TXN_KEY_NOT_FOUND ? MetaServiceCode::TXN_ID_NOT_FOUND + : cast_as(err); + ss << "failed to get txn_info, db_id=" << db_id << " txn_id=" << txn_id + << " err=" << err; msg = ss.str(); + LOG(WARNING) << msg; return; } - rowsets.emplace_back(std::move(key), std::move(val)); - - // Accumulate affected rows - auto& stats = tablet_stats[tablet_id]; - stats.data_size += i.data_disk_size(); - stats.num_rows += i.num_rows(); - ++stats.num_rowsets; - stats.num_segs += i.num_segments(); - } // for tmp_rowsets_meta - // process mow table, check lock and remove pending key - std::vector lock_keys; - lock_keys.reserve(request->mow_table_ids().size()); - for (auto table_id : request->mow_table_ids()) { - lock_keys.push_back(meta_delete_bitmap_update_lock_key({instance_id, table_id, -1})); - } - std::vector> lock_values; - err = txn->batch_get(&lock_values, lock_keys); - if (err != TxnErrorCode::TXN_OK) { - ss << "failed to get delete bitmap update lock key info, instance_id=" << instance_id - << " err=" << err; - msg = ss.str(); - code = cast_as(err); - LOG(WARNING) << msg << " txn_id=" << txn_id; - return; - } - size_t total_locks = lock_keys.size(); - for (size_t i = 0; i < total_locks; i++) { - int64_t table_id = request->mow_table_ids(i); - // When the key does not exist, it means the lock has been acquired - // by another transaction and successfully committed. - if (!lock_values[i].has_value()) { - ss << "get delete bitmap update lock info, lock is expired" - << " table_id=" << table_id << " key=" << hex(lock_keys[i]); - code = MetaServiceCode::LOCK_EXPIRED; + TxnInfoPB txn_info; + if (!txn_info.ParseFromString(info_val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + ss << "failed to parse txn_info, db_id=" << db_id << " txn_id=" << txn_id; msg = ss.str(); - LOG(WARNING) << msg << " txn_id=" << txn_id; + LOG(WARNING) << msg; return; } - DeleteBitmapUpdateLockPB lock_info; - if (!lock_info.ParseFromString(lock_values[i].value())) [[unlikely]] { - code = MetaServiceCode::PROTOBUF_PARSE_ERR; - msg = "failed to parse DeleteBitmapUpdateLockPB"; - LOG(WARNING) << msg << " txn_id=" << txn_id; + DCHECK(txn_info.txn_id() == txn_id); + if (txn_info.status() == TxnStatusPB::TXN_STATUS_ABORTED) { + code = MetaServiceCode::TXN_ALREADY_ABORTED; + ss << "transaction is already aborted: db_id=" << db_id << " txn_id=" << txn_id; + msg = ss.str(); + LOG(WARNING) << msg; return; } - if (lock_info.lock_id() != request->txn_id()) { - msg = "lock is expired"; - code = MetaServiceCode::LOCK_EXPIRED; + + if (txn_info.status() == TxnStatusPB::TXN_STATUS_VISIBLE) { + if (request->has_is_2pc() && request->is_2pc()) { + code = MetaServiceCode::TXN_ALREADY_VISIBLE; + ss << "transaction [" << txn_id << "] is already visible, not pre-committed."; + msg = ss.str(); + LOG(INFO) << msg; + response->mutable_txn_info()->CopyFrom(txn_info); + return; + } + code = MetaServiceCode::OK; + ss << "transaction is already visible: db_id=" << db_id << " txn_id=" << txn_id; + msg = ss.str(); + LOG(INFO) << msg; + response->mutable_txn_info()->CopyFrom(txn_info); return; } - txn->remove(lock_keys[i]); - LOG(INFO) << "xxx remove delete bitmap lock, lock_key=" << hex(lock_keys[i]) - << " txn_id=" << txn_id; - for (auto tablet_id : table_id_tablet_ids[table_id]) { - std::string pending_key = meta_pending_delete_bitmap_key({instance_id, tablet_id}); - txn->remove(pending_key); - LOG(INFO) << "xxx remove delete bitmap pending key, pending_key=" << hex(pending_key) - << " txn_id=" << txn_id; + if (request->has_is_2pc() && request->is_2pc() && + txn_info.status() == TxnStatusPB::TXN_STATUS_PREPARED) { + code = MetaServiceCode::TXN_INVALID_STATUS; + ss << "transaction is prepare, not pre-committed: db_id=" << db_id << " txn_id" + << txn_id; + msg = ss.str(); + LOG(WARNING) << msg; + return; } - } - lock_keys.clear(); - lock_values.clear(); + LOG(INFO) << "txn_id=" << txn_id << " txn_info=" << txn_info.ShortDebugString(); - // Save rowset meta - for (auto& i : rowsets) { - size_t rowset_size = i.first.size() + i.second.size(); - txn->put(i.first, i.second); - LOG(INFO) << "xxx put rowset_key=" << hex(i.first) << " txn_id=" << txn_id - << " rowset_size=" << rowset_size; - } + auto now_time = system_clock::now(); + uint64_t commit_time = duration_cast(now_time.time_since_epoch()).count(); + if ((txn_info.prepare_time() + txn_info.timeout_ms()) < commit_time) { + code = MetaServiceCode::UNDEFINED_ERR; + msg = fmt::format("txn is expired, not allow to commit txn_id={}", txn_id); + LOG(INFO) << msg << " prepare_time=" << txn_info.prepare_time() + << " timeout_ms=" << txn_info.timeout_ms() << " commit_time=" << commit_time; + return; + } + txn_info.set_commit_time(commit_time); + txn_info.set_finish_time(commit_time); + if (request->has_commit_attachment()) { + txn_info.mutable_commit_attachment()->CopyFrom(request->commit_attachment()); + } + DCHECK(txn_info.status() != TxnStatusPB::TXN_STATUS_COMMITTED); + // lazy commit set status TXN_STATUS_COMMITTED not TXN_STATUS_VISIBLE !!! + txn_info.set_status(TxnStatusPB::TXN_STATUS_COMMITTED); - // Save versions - int64_t version_update_time_ms = - duration_cast(system_clock::now().time_since_epoch()).count(); - response->set_version_update_time_ms(version_update_time_ms); - for (auto& i : new_versions) { - std::string ver_val; - VersionPB version_pb; - version_pb.set_version(i.second); - version_pb.set_update_time_ms(version_update_time_ms); - if (!version_pb.SerializeToString(&ver_val)) { + LOG(INFO) << "after update txn_info=" << txn_info.ShortDebugString(); + info_val.clear(); + if (!txn_info.SerializeToString(&info_val)) { code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; - ss << "failed to serialize version_pb when saving, txn_id=" << txn_id; + ss << "failed to serialize txn_info when saving, txn_id=" << txn_id; msg = ss.str(); return; } - txn->put(i.first, ver_val); - LOG(INFO) << "xxx put partition_version_key=" << hex(i.first) << " version:" << i.second - << " txn_id=" << txn_id << " update_time=" << version_update_time_ms; + txn->put(info_key, info_val); + LOG(INFO) << "xxx put info_key=" << hex(info_key) << " txn_id=" << txn_id; - std::string_view ver_key = i.first; - ver_key.remove_prefix(1); // Remove key space - // PartitionVersionKeyInfo {instance_id, db_id, table_id, partition_id} - std::vector, int, int>> out; - int ret = decode_key(&ver_key, &out); - if (ret != 0) [[unlikely]] { - // decode version key error means this is something wrong, - // we can not continue this txn - LOG(WARNING) << "failed to decode key, ret=" << ret << " key=" << hex(ver_key); - code = MetaServiceCode::UNDEFINED_ERR; - msg = "decode version key error"; - return; + if (txn_info.load_job_source_type() == + LoadJobSourceTypePB::LOAD_JOB_SRC_TYPE_ROUTINE_LOAD_TASK) { + put_routine_load_progress(code, msg, instance_id, request, txn.get(), db_id); } - int64_t table_id = std::get(std::get<0>(out[4])); - int64_t partition_id = std::get(std::get<0>(out[5])); - VLOG_DEBUG << " table_id=" << table_id << " partition_id=" << partition_id; + // save versions for partition + int64_t version_update_time_ms = + duration_cast(system_clock::now().time_since_epoch()).count(); + response->set_version_update_time_ms(version_update_time_ms); + for (auto& i : new_versions) { + std::string ver_val; + VersionPB version_pb; + version_pb.set_version(i.second); + version_pb.set_txn_id(txn_id); + version_pb.set_update_time_ms(version_update_time_ms); + if (!version_pb.SerializeToString(&ver_val)) { + code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; + ss << "failed to serialize version_pb when saving, txn_id=" << txn_id; + msg = ss.str(); + return; + } - response->add_table_ids(table_id); - response->add_partition_ids(partition_id); - response->add_versions(i.second); - } + txn->put(i.first, ver_val); + LOG(INFO) << "xxx put partition_version_key=" << hex(i.first) << " version:" << i.second + << " txn_id=" << txn_id << " update_time=" << version_update_time_ms; + + std::string_view ver_key = i.first; + ver_key.remove_prefix(1); // Remove key space + // PartitionVersionKeyInfo {instance_id, db_id, table_id, partition_id} + std::vector, int, int>> out; + int ret = decode_key(&ver_key, &out); + if (ret != 0) [[unlikely]] { + // decode version key error means this is something wrong, + // we can not continue this txn + LOG(WARNING) << "failed to decode key, ret=" << ret << " key=" << hex(ver_key); + code = MetaServiceCode::UNDEFINED_ERR; + msg = "decode version key error"; + return; + } - // Save table versions - for (auto& i : table_id_tablet_ids) { - std::string ver_key = table_version_key({instance_id, db_id, i.first}); - txn->atomic_add(ver_key, 1); - LOG(INFO) << "xxx atomic add table_version_key=" << hex(ver_key) << " txn_id=" << txn_id; - } + int64_t table_id = std::get(std::get<0>(out[4])); + int64_t partition_id = std::get(std::get<0>(out[5])); + VLOG_DEBUG << "txn_id=" << txn_id << " table_id=" << table_id + << " partition_id=" << partition_id << " version=" << i.second; - LOG(INFO) << " before update txn_info=" << txn_info.ShortDebugString(); + response->add_table_ids(table_id); + response->add_partition_ids(partition_id); + response->add_versions(i.second); + } - // Update txn_info - txn_info.set_status(TxnStatusPB::TXN_STATUS_VISIBLE); + // process mow table, check lock and remove pending key + std::vector lock_keys; + lock_keys.reserve(request->mow_table_ids().size()); + for (auto table_id : request->mow_table_ids()) { + lock_keys.push_back(meta_delete_bitmap_update_lock_key({instance_id, table_id, -1})); + } + std::vector> lock_values; + err = txn->batch_get(&lock_values, lock_keys); + if (err != TxnErrorCode::TXN_OK) { + ss << "failed to get delete bitmap update lock key info, instance_id=" << instance_id + << " err=" << err; + msg = ss.str(); + code = cast_as(err); + LOG(WARNING) << msg << " txn_id=" << txn_id; + return; + } - auto now_time = system_clock::now(); - uint64_t commit_time = duration_cast(now_time.time_since_epoch()).count(); - if ((txn_info.prepare_time() + txn_info.timeout_ms()) < commit_time) { - code = MetaServiceCode::UNDEFINED_ERR; - msg = fmt::format("txn is expired, not allow to commit txn_id={}", txn_id); - LOG(INFO) << msg << " prepare_time=" << txn_info.prepare_time() - << " timeout_ms=" << txn_info.timeout_ms() << " commit_time=" << commit_time; - return; - } - txn_info.set_commit_time(commit_time); - txn_info.set_finish_time(commit_time); - if (request->has_commit_attachment()) { - txn_info.mutable_commit_attachment()->CopyFrom(request->commit_attachment()); - } - LOG(INFO) << "after update txn_info=" << txn_info.ShortDebugString(); - info_val.clear(); - if (!txn_info.SerializeToString(&info_val)) { - code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; - ss << "failed to serialize txn_info when saving, txn_id=" << txn_id; - msg = ss.str(); - return; - } - txn->put(info_key, info_val); - LOG(INFO) << "xxx put info_key=" << hex(info_key) << " txn_id=" << txn_id; + for (size_t i = 0; i < lock_keys.size(); i++) { + int64_t table_id = request->mow_table_ids(i); + // When the key does not exist, it means the lock has been acquired + // by another transaction and successfully committed. + if (!lock_values[i].has_value()) { + ss << "get delete bitmap update lock info, lock is expired" + << " table_id=" << table_id << " key=" << hex(lock_keys[i]); + code = MetaServiceCode::LOCK_EXPIRED; + msg = ss.str(); + LOG(WARNING) << msg << " txn_id=" << txn_id; + return; + } - // Update stats of affected tablet - for (auto& [tablet_id, stats] : tablet_stats) { - DCHECK(tablet_ids.count(tablet_id)); - auto& tablet_idx = tablet_ids[tablet_id]; - StatsTabletKeyInfo info {instance_id, tablet_idx.table_id(), tablet_idx.index_id(), - tablet_idx.partition_id(), tablet_id}; - update_tablet_stats(info, stats, txn, code, msg); - if (code != MetaServiceCode::OK) return; - } - // Remove tmp rowset meta - for (auto& [k, _] : tmp_rowsets_meta) { - txn->remove(k); - LOG(INFO) << "xxx remove tmp_rowset_key=" << hex(k) << " txn_id=" << txn_id; - } + DeleteBitmapUpdateLockPB lock_info; + if (!lock_info.ParseFromString(lock_values[i].value())) [[unlikely]] { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = "failed to parse DeleteBitmapUpdateLockPB"; + LOG(WARNING) << msg << " txn_id=" << txn_id; + return; + } + if (lock_info.lock_id() != request->txn_id()) { + msg = "lock is expired"; + code = MetaServiceCode::LOCK_EXPIRED; + return; + } + txn->remove(lock_keys[i]); + LOG(INFO) << "xxx remove delete bitmap lock, lock_key=" << hex(lock_keys[i]) + << " txn_id=" << txn_id; - const std::string running_key = txn_running_key({instance_id, db_id, txn_id}); - LOG(INFO) << "xxx remove running_key=" << hex(running_key) << " txn_id=" << txn_id; - txn->remove(running_key); + for (auto tablet_id : table_id_tablet_ids[table_id]) { + std::string pending_key = meta_pending_delete_bitmap_key({instance_id, tablet_id}); + txn->remove(pending_key); + LOG(INFO) << "xxx remove delete bitmap pending key, pending_key=" + << hex(pending_key) << " txn_id=" << txn_id; + } + } + lock_keys.clear(); + lock_values.clear(); - std::string recycle_val; - std::string recycle_key = recycle_txn_key({instance_id, db_id, txn_id}); - RecycleTxnPB recycle_pb; - recycle_pb.set_creation_time(commit_time); - recycle_pb.set_label(txn_info.label()); + // Save table versions + for (auto& i : table_id_tablet_ids) { + std::string ver_key = table_version_key({instance_id, db_id, i.first}); + txn->atomic_add(ver_key, 1); + LOG(INFO) << "xxx atomic add table_version_key=" << hex(ver_key) + << " txn_id=" << txn_id; + } - if (!recycle_pb.SerializeToString(&recycle_val)) { - code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; - ss << "failed to serialize recycle_pb, txn_id=" << txn_id; - msg = ss.str(); - return; - } - txn->put(recycle_key, recycle_val); + LOG(INFO) << "commit_txn_eventually put_size=" << txn->put_bytes() + << " del_size=" << txn->delete_bytes() << " num_put_keys=" << txn->num_put_keys() + << " num_del_keys=" << txn->num_del_keys() + << " txn_size=" << txn->approximate_bytes() << " txn_id=" << txn_id; - if (txn_info.load_job_source_type() == - LoadJobSourceTypePB::LOAD_JOB_SRC_TYPE_ROUTINE_LOAD_TASK) { - put_routine_load_progress(code, msg, instance_id, request, txn.get(), db_id); - } + err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "failed to commit kv txn, txn_id=" << txn_id << " err=" << err; + msg = ss.str(); + return; + } - LOG(INFO) << "xxx commit_txn put recycle_key key=" << hex(recycle_key) << " txn_id=" << txn_id; - LOG(INFO) << "commit_txn put_size=" << txn->put_bytes() << " del_size=" << txn->delete_bytes() - << " num_put_keys=" << txn->num_put_keys() << " num_del_keys=" << txn->num_del_keys() - << " txn_size=" << txn->approximate_bytes() << " txn_id=" << txn_id; + std::shared_ptr task = txn_lazy_committer->submit(instance_id, txn_id); + std::pair ret = task->wait(); + if (ret.first != MetaServiceCode::OK) { + LOG(WARNING) << "lazy commit txn failed txn_id=" << txn_id << " code=" << ret.first + << "msg=" << ret.second; + } - // Finally we are done... - err = txn->commit(); - if (err != TxnErrorCode::TXN_OK) { - if ((err == TxnErrorCode::TXN_VALUE_TOO_LARGE || - err == TxnErrorCode::TXN_BYTES_TOO_LARGE) && - !tmp_rowsets_meta.empty()) { - size_t max_size = 0, max_idx = 0, max_num_segments = 0, - min_num_segments = std::numeric_limits::max(), avg_num_segments = 0; - for (size_t i = 0; i < tmp_rowsets_meta.size(); i++) { - auto& [k, v] = tmp_rowsets_meta[i]; - if (v.ByteSizeLong() > max_size) { - max_size = v.ByteSizeLong(); - max_idx = i; - } - if (v.num_segments() > max_num_segments) { - max_num_segments = v.num_segments(); - } - if (v.num_segments() < min_num_segments) { - min_num_segments = v.num_segments(); - } - avg_num_segments += v.num_segments(); - } - if (!tmp_rowsets_meta.empty()) { - avg_num_segments /= tmp_rowsets_meta.size(); - } - LOG(WARNING) << "failed to commit kv txn" - << ", err=" << err << ", txn_id=" << txn_id - << ", total_rowsets=" << tmp_rowsets_meta.size() - << ", avg_num_segments=" << avg_num_segments - << ", min_num_segments=" << min_num_segments - << ", max_num_segments=" << max_num_segments - << ", largest_rowset_size=" << max_size - << ", largest_rowset_key=" << hex(tmp_rowsets_meta[max_idx].first) - << ", largest_rowset_value=" - << tmp_rowsets_meta[max_idx].second.ShortDebugString(); + std::unordered_map tablet_stats; // tablet_id -> stats + for (auto& [_, i] : tmp_rowsets_meta) { + // Accumulate affected rows + auto& stats = tablet_stats[i.tablet_id()]; + stats.data_size += i.data_disk_size(); + stats.num_rows += i.num_rows(); + ++stats.num_rowsets; + stats.num_segs += i.num_segments(); } - code = cast_as(err); - ss << "failed to commit kv txn, txn_id=" << txn_id << " err=" << err; - msg = ss.str(); - return; - } - // calculate table stats from tablets stats - std::map table_stats; - std::vector base_tablet_ids(request->base_tablet_ids().begin(), - request->base_tablet_ids().end()); - calc_table_stats(tablet_ids, tablet_stats, table_stats, base_tablet_ids); - for (const auto& pair : table_stats) { - TableStatsPB* stats_pb = response->add_table_stats(); - auto table_id = pair.first; - auto stats = pair.second; - get_pb_from_tablestats(stats, stats_pb); - stats_pb->set_table_id(table_id); - VLOG_DEBUG << "Add TableStats to CommitTxnResponse. txn_id=" << txn_id - << " table_id=" << table_id - << " updated_row_count=" << stats_pb->updated_row_count(); - } + // calculate table stats from tablets stats + std::map table_stats; + std::vector base_tablet_ids(request->base_tablet_ids().begin(), + request->base_tablet_ids().end()); + calc_table_stats(tablet_ids, tablet_stats, table_stats, base_tablet_ids); + for (const auto& pair : table_stats) { + TableStatsPB* stats_pb = response->add_table_stats(); + auto table_id = pair.first; + auto stats = pair.second; + get_pb_from_tablestats(stats, stats_pb); + stats_pb->set_table_id(table_id); + VLOG_DEBUG << "Add TableStats to CommitTxnResponse. txn_id=" << txn_id + << " table_id=" << table_id + << " updated_row_count=" << stats_pb->updated_row_count(); + } - response->mutable_txn_info()->CopyFrom(txn_info); -} // end commit_txn + txn_info.set_status(TxnStatusPB::TXN_STATUS_VISIBLE); + response->mutable_txn_info()->CopyFrom(txn_info); + break; + } while (true); +} /** * This process is generally the same as commit_txn, the difference is that @@ -1992,8 +2502,19 @@ void MetaServiceImpl::commit_txn(::google::protobuf::RpcController* controller, LOG(WARNING) << "scan_tmp_rowset failed, txn_id=" << txn_id << " code=" << code; return; } - commit_txn_immediately(request, response, txn_kv_, code, msg, instance_id, db_id, - tmp_rowsets_meta); + + if (request->has_is_2pc() && !request->is_2pc() && request->has_lazy_commit() && + request->lazy_commit() && config::enable_txn_lazy_commit && + (tmp_rowsets_meta.size() >= config::txn_lazy_commit_rowsets_thresold)) { + LOG(INFO) << "txn_id=" << txn_id << " commit_txn_eventually" + << " size=" << tmp_rowsets_meta.size(); + commit_txn_eventually(request, response, txn_kv_, txn_lazy_committer_, code, msg, + instance_id, db_id, tmp_rowsets_meta); + return; + } + + commit_txn_immediately(request, response, txn_kv_, txn_lazy_committer_, code, msg, instance_id, + db_id, tmp_rowsets_meta); } static void _abort_txn(const std::string& instance_id, const AbortTxnRequest* request, diff --git a/cloud/src/meta-service/txn_lazy_committer.cpp b/cloud/src/meta-service/txn_lazy_committer.cpp new file mode 100644 index 00000000000000..23eb61bc9b929f --- /dev/null +++ b/cloud/src/meta-service/txn_lazy_committer.cpp @@ -0,0 +1,459 @@ +// 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 "txn_lazy_committer.h" + +#include + +#include "common/logging.h" +#include "common/util.h" +#include "meta-service/keys.h" +#include "meta-service/meta_service_helper.h" +#include "meta-service/meta_service_tablet_stats.h" + +using namespace std::chrono; + +namespace doris::cloud { + +void scan_tmp_rowset( + const std::string& instance_id, int64_t txn_id, std::shared_ptr txn_kv, + MetaServiceCode& code, std::string& msg, int64_t* db_id, + std::vector>* tmp_rowsets_meta); + +void update_tablet_stats(const StatsTabletKeyInfo& info, const TabletStats& stats, + std::unique_ptr& txn, MetaServiceCode& code, + std::string& msg); + +void convert_tmp_rowsets( + const std::string& instance_id, int64_t txn_id, std::shared_ptr txn_kv, + MetaServiceCode& code, std::string& msg, int64_t db_id, + std::vector>& tmp_rowsets_meta, + std::unordered_map& tablet_ids) { + std::stringstream ss; + std::unique_ptr txn; + TxnErrorCode err = txn_kv->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "failed to create txn, txn_id=" << txn_id << " err=" << err; + msg = ss.str(); + LOG(WARNING) << msg; + return; + } + + // partition_id -> VersionPB + std::unordered_map partition_versions; + // tablet_id -> stats + std::unordered_map tablet_stats; + + for (auto& [tmp_rowset_key, tmp_rowset_pb] : tmp_rowsets_meta) { + std::string tmp_rowst_data; + err = txn->get(tmp_rowset_key, &tmp_rowst_data); + if (TxnErrorCode::TXN_KEY_NOT_FOUND == err) { + // the tmp rowset has been converted + VLOG_DEBUG << "tmp rowset has been converted, key=" << hex(tmp_rowset_key); + continue; + } + + if (TxnErrorCode::TXN_OK != err) { + code = cast_as(err); + ss << "failed to get tmp_rowset_key, txn_id=" << txn_id + << " key=" << hex(tmp_rowset_key) << " err=" << err; + msg = ss.str(); + LOG(WARNING) << msg; + return; + } + + if (!tablet_ids.contains(tmp_rowset_pb.tablet_id())) { + std::string tablet_idx_key = + meta_tablet_idx_key({instance_id, tmp_rowset_pb.tablet_id()}); + std::string tablet_idx_val; + err = txn->get(tablet_idx_key, &tablet_idx_val, true); + if (TxnErrorCode::TXN_OK != err) { + code = err == TxnErrorCode::TXN_KEY_NOT_FOUND ? MetaServiceCode::TXN_ID_NOT_FOUND + : cast_as(err); + ss << "failed to get tablet idx, txn_id=" << txn_id + << " key=" << hex(tablet_idx_key) << " err=" << err; + msg = ss.str(); + LOG(WARNING) << msg; + return; + } + + TabletIndexPB tablet_idx_pb; + if (!tablet_idx_pb.ParseFromString(tablet_idx_val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + ss << "failed to parse tablet idx pb txn_id=" << txn_id + << " key=" << hex(tablet_idx_key); + msg = ss.str(); + return; + } + tablet_ids.emplace(tmp_rowset_pb.tablet_id(), tablet_idx_pb); + } + const TabletIndexPB& tablet_idx_pb = tablet_ids[tmp_rowset_pb.tablet_id()]; + + if (!partition_versions.contains(tmp_rowset_pb.partition_id())) { + std::string ver_val; + std::string ver_key = partition_version_key( + {instance_id, db_id, tablet_idx_pb.table_id(), tmp_rowset_pb.partition_id()}); + err = txn->get(ver_key, &ver_val); + if (TxnErrorCode::TXN_OK != err) { + code = err == TxnErrorCode::TXN_KEY_NOT_FOUND ? MetaServiceCode::TXN_ID_NOT_FOUND + : cast_as(err); + ss << "failed to get partiton version, txn_id=" << txn_id << " key=" << hex(ver_key) + << " err=" << err; + msg = ss.str(); + LOG(WARNING) << msg; + return; + } + VersionPB version_pb; + if (!version_pb.ParseFromString(ver_val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + ss << "failed to parse version pb txn_id=" << txn_id << " key=" << hex(ver_key); + msg = ss.str(); + return; + } + LOG(INFO) << "txn_id=" << txn_id << " key=" << hex(ver_key) + << " version_pb:" << version_pb.ShortDebugString(); + DCHECK(version_pb.has_txn_id()); + DCHECK(version_pb.txn_id() == txn_id); + partition_versions.emplace(tmp_rowset_pb.partition_id(), version_pb); + } + + const VersionPB& version_pb = partition_versions[tmp_rowset_pb.partition_id()]; + + std::string rowset_key = + meta_rowset_key({instance_id, tmp_rowset_pb.tablet_id(), version_pb.version()}); + std::string rowset_val; + err = txn->get(rowset_key, &rowset_val); + if (TxnErrorCode::TXN_OK == err) { + continue; + } + + if (err != TxnErrorCode::TXN_KEY_NOT_FOUND) { + code = cast_as(err); + ss << "failed to get rowset_key, txn_id=" << txn_id << " key=" << hex(rowset_key) + << " err=" << err; + msg = ss.str(); + LOG(WARNING) << msg; + return; + } + DCHECK(err == TxnErrorCode::TXN_KEY_NOT_FOUND); + + tmp_rowset_pb.set_start_version(version_pb.version()); + tmp_rowset_pb.set_end_version(version_pb.version()); + + rowset_val.clear(); + if (!tmp_rowset_pb.SerializeToString(&rowset_val)) { + code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; + ss << "failed to serialize rowset_meta, txn_id=" << txn_id + << " key=" << hex(rowset_key); + msg = ss.str(); + return; + } + + txn->put(rowset_key, rowset_val); + LOG(INFO) << "xxx put rowset_key=" << hex(rowset_key) << " txn_id=" << txn_id + << " rowset_size=" << rowset_key.size() + rowset_val.size(); + + // Accumulate affected rows + auto& stats = tablet_stats[tmp_rowset_pb.tablet_id()]; + stats.data_size += tmp_rowset_pb.data_disk_size(); + stats.num_rows += tmp_rowset_pb.num_rows(); + ++stats.num_rowsets; + stats.num_segs += tmp_rowset_pb.num_segments(); + } + + DCHECK(partition_versions.size() == 1); + + for (auto& [tablet_id, stats] : tablet_stats) { + DCHECK(tablet_ids.count(tablet_id)); + auto& tablet_idx = tablet_ids[tablet_id]; + StatsTabletKeyInfo info {instance_id, tablet_idx.table_id(), tablet_idx.index_id(), + tablet_idx.partition_id(), tablet_id}; + update_tablet_stats(info, stats, txn, code, msg); + if (code != MetaServiceCode::OK) return; + } + + err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "failed to commit kv txn, txn_id=" << txn_id << " err=" << err; + msg = ss.str(); + return; + } +} + +void make_committed_txn_visible(const std::string& instance_id, int64_t db_id, int64_t txn_id, + std::shared_ptr txn_kv, MetaServiceCode& code, + std::string& msg) { + // 1. visible txn info + // 2. remove running key and put recycle txn key + + std::stringstream ss; + std::unique_ptr txn; + TxnErrorCode err = txn_kv->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "failed to create txn, txn_id=" << txn_id << " err=" << err; + msg = ss.str(); + LOG(WARNING) << msg; + return; + } + + std::string info_val; + const std::string info_key = txn_info_key({instance_id, db_id, txn_id}); + err = txn->get(info_key, &info_val); + if (err != TxnErrorCode::TXN_OK) { + code = err == TxnErrorCode::TXN_KEY_NOT_FOUND ? MetaServiceCode::TXN_ID_NOT_FOUND + : cast_as(err); + ss << "failed to get txn_info, db_id=" << db_id << " txn_id=" << txn_id << " err=" << err; + msg = ss.str(); + LOG(WARNING) << msg; + return; + } + + TxnInfoPB txn_info; + if (!txn_info.ParseFromString(info_val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + ss << "failed to parse txn_info, txn_id=" << txn_id; + msg = ss.str(); + LOG(WARNING) << msg; + return; + } + + VLOG_DEBUG << "txn_info:" << txn_info.ShortDebugString(); + DCHECK((txn_info.status() == TxnStatusPB::TXN_STATUS_COMMITTED) || + (txn_info.status() == TxnStatusPB::TXN_STATUS_VISIBLE)); + + if (txn_info.status() == TxnStatusPB::TXN_STATUS_COMMITTED) { + txn_info.set_status(TxnStatusPB::TXN_STATUS_VISIBLE); + + if (!txn_info.SerializeToString(&info_val)) { + code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; + ss << "failed to serialize txn_info when saving, txn_id=" << txn_id; + msg = ss.str(); + return; + } + txn->put(info_key, info_val); + LOG(INFO) << "xxx put info_key=" << hex(info_key) << " txn_id=" << txn_id; + + const std::string running_key = txn_running_key({instance_id, db_id, txn_id}); + LOG(INFO) << "xxx remove running_key=" << hex(running_key) << " txn_id=" << txn_id; + txn->remove(running_key); + + std::string recycle_val; + std::string recycle_key = recycle_txn_key({instance_id, db_id, txn_id}); + RecycleTxnPB recycle_pb; + auto now_time = system_clock::now(); + uint64_t visible_time = duration_cast(now_time.time_since_epoch()).count(); + recycle_pb.set_creation_time(visible_time); + recycle_pb.set_label(txn_info.label()); + + if (!recycle_pb.SerializeToString(&recycle_val)) { + code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; + ss << "failed to serialize recycle_pb, txn_id=" << txn_id; + msg = ss.str(); + return; + } + txn->put(recycle_key, recycle_val); + + err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "failed to commit kv txn, txn_id=" << txn_id << " err=" << err; + msg = ss.str(); + return; + } + } +} + +TxnLazyCommitTask::TxnLazyCommitTask(const std::string& instance_id, int64_t txn_id, + std::shared_ptr txn_kv, + TxnLazyCommitter* txn_lazy_committer) + : instance_id_(instance_id), + txn_id_(txn_id), + txn_kv_(txn_kv), + txn_lazy_committer_(txn_lazy_committer) { + DCHECK(txn_id > 0); +} + +void TxnLazyCommitTask::commit() { + std::stringstream ss; + int retry_times = 0; + do { + LOG(INFO) << "lazy task commit txn_id=" << txn_id_ << " retry_times=" << retry_times; + do { + code_ = MetaServiceCode::OK; + msg_.clear(); + int64_t db_id; + std::vector> tmp_rowset_metas; + scan_tmp_rowset(instance_id_, txn_id_, txn_kv_, code_, msg_, &db_id, &tmp_rowset_metas); + if (code_ != MetaServiceCode::OK) { + LOG(WARNING) << "scan_tmp_rowset failed, txn_id=" << txn_id_ << " code=" << code_; + break; + } + + VLOG_DEBUG << "txn_id=" << txn_id_ + << " tmp_rowset_metas.size()=" << tmp_rowset_metas.size(); + if (tmp_rowset_metas.size() == 0) { + LOG(INFO) << "empty tmp_rowset_metas, txn_id=" << txn_id_; + } + + // + std::unordered_map>> + partition_to_tmp_rowset_metas; + for (auto& [tmp_rowset_key, tmp_rowset_pb] : tmp_rowset_metas) { + partition_to_tmp_rowset_metas[tmp_rowset_pb.partition_id()].emplace_back(); + partition_to_tmp_rowset_metas[tmp_rowset_pb.partition_id()].back().first = + tmp_rowset_key; + partition_to_tmp_rowset_metas[tmp_rowset_pb.partition_id()].back().second = + tmp_rowset_pb; + } + + // tablet_id -> TabletIndexPB + std::unordered_map tablet_ids; + for (auto& [partition_id, tmp_rowset_metas] : partition_to_tmp_rowset_metas) { + for (size_t i = 0; i < tmp_rowset_metas.size(); + i += config::txn_lazy_max_rowsets_per_batch) { + size_t end = + (i + config::txn_lazy_max_rowsets_per_batch) > tmp_rowset_metas.size() + ? tmp_rowset_metas.size() + : i + config::txn_lazy_max_rowsets_per_batch; + std::vector> + sub_partition_tmp_rowset_metas(tmp_rowset_metas.begin() + i, + tmp_rowset_metas.begin() + end); + convert_tmp_rowsets(instance_id_, txn_id_, txn_kv_, code_, msg_, db_id, + sub_partition_tmp_rowset_metas, tablet_ids); + if (code_ != MetaServiceCode::OK) break; + } + if (code_ != MetaServiceCode::OK) break; + + DCHECK(tmp_rowset_metas.size() > 0); + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code_ = cast_as(err); + ss << "failed to create txn, txn_id=" << txn_id_ << " err=" << err; + msg_ = ss.str(); + LOG(WARNING) << msg_; + break; + } + + int64_t table_id = -1; + for (auto& [tmp_rowset_key, tmp_rowset_pb] : tmp_rowset_metas) { + if (table_id <= 0) { + table_id = tablet_ids[tmp_rowset_pb.tablet_id()].table_id(); + } + txn->remove(tmp_rowset_key); + } + + DCHECK(table_id > 0); + DCHECK(partition_id > 0); + + std::string ver_val; + std::string ver_key = + partition_version_key({instance_id_, db_id, table_id, partition_id}); + err = txn->get(ver_key, &ver_val); + if (TxnErrorCode::TXN_OK != err) { + code_ = err == TxnErrorCode::TXN_KEY_NOT_FOUND + ? MetaServiceCode::TXN_ID_NOT_FOUND + : cast_as(err); + ss << "failed to get partiton version, txn_id=" << txn_id_ + << " key=" << hex(ver_key) << " err=" << err; + msg_ = ss.str(); + LOG(WARNING) << msg_; + break; + } + VersionPB version_pb; + if (!version_pb.ParseFromString(ver_val)) { + code_ = MetaServiceCode::PROTOBUF_PARSE_ERR; + ss << "failed to parse version pb txn_id=" << txn_id_ + << " key=" << hex(ver_key); + msg_ = ss.str(); + break; + } + + if (version_pb.has_txn_id() && version_pb.txn_id() == txn_id_) { + version_pb.clear_txn_id(); + ver_val.clear(); + if (!version_pb.SerializeToString(&ver_val)) { + code_ = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; + ss << "failed to serialize version_pb when saving, txn_id=" << txn_id_; + msg_ = ss.str(); + return; + } + txn->put(ver_key, ver_val); + + err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + code_ = cast_as(err); + ss << "failed to commit kv txn, txn_id=" << txn_id_ << " err=" << err; + msg_ = ss.str(); + break; + } + } + } + make_committed_txn_visible(instance_id_, db_id, txn_id_, txn_kv_, code_, msg_); + } while (false); + } while (code_ == MetaServiceCode::KV_TXN_CONFLICT && + retry_times++ < config::txn_store_retry_times); + { + std::unique_lock lock(mutex_); + this->finished_ = true; + } + this->cond_.notify_all(); +} + +std::pair TxnLazyCommitTask::wait() { + { + std::unique_lock lock(mutex_); + cond_.wait(lock, [this]() { return this->finished_ == true; }); + } + txn_lazy_committer_->remove(txn_id_); + return std::make_pair(this->code_, this->msg_); +} + +TxnLazyCommitter::TxnLazyCommitter(std::shared_ptr txn_kv) : txn_kv_(txn_kv) { + worker_pool_ = std::make_unique(config::txn_lazy_commit_num_threads); + worker_pool_->start(); +} + +std::shared_ptr TxnLazyCommitter::submit(const std::string& instance_id, + int64_t txn_id) { + std::shared_ptr task; + { + std::unique_lock lock(mutex_); + auto iter = running_tasks_.find(txn_id); + if (iter != running_tasks_.end()) { + return iter->second; + } + + task = std::make_shared(instance_id, txn_id, txn_kv_, this); + running_tasks_.emplace(txn_id, task); + } + + worker_pool_->submit([task]() { task->commit(); }); + DCHECK(task != nullptr); + return task; +} + +void TxnLazyCommitter::remove(int64_t txn_id) { + std::unique_lock lock(mutex_); + running_tasks_.erase(txn_id); +} + +} // namespace doris::cloud \ No newline at end of file diff --git a/cloud/src/meta-service/txn_lazy_committer.h b/cloud/src/meta-service/txn_lazy_committer.h new file mode 100644 index 00000000000000..dca618739ec8ba --- /dev/null +++ b/cloud/src/meta-service/txn_lazy_committer.h @@ -0,0 +1,68 @@ +// 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 + +#include + +#include "common/simple_thread_pool.h" +#include "meta-service/txn_kv.h" + +namespace doris::cloud { + +class TxnLazyCommitter; + +class TxnLazyCommitTask { +public: + TxnLazyCommitTask(const std::string& instance_id, int64_t txn_id, std::shared_ptr txn_kv, + TxnLazyCommitter* txn_lazy_committer); + void commit(); + + std::pair wait(); + +private: + friend class TxnLazyCommitter; + + std::string instance_id_; + int64_t txn_id_; + std::shared_ptr txn_kv_; + std::mutex mutex_; + std::condition_variable cond_; + bool finished_ = false; + MetaServiceCode code_ = MetaServiceCode::OK; + std::string msg_; + TxnLazyCommitter* txn_lazy_committer_; +}; + +class TxnLazyCommitter { +public: + TxnLazyCommitter(std::shared_ptr txn_kv); + std::shared_ptr submit(const std::string& instance_id, int64_t txn_id); + void remove(int64_t txn_id); + +private: + std::shared_ptr txn_kv_; + + std::unique_ptr worker_pool_; + + std::mutex mutex_; + // + std::unordered_map> running_tasks_; +}; +} // namespace doris::cloud \ No newline at end of file diff --git a/cloud/src/recycler/recycler.cpp b/cloud/src/recycler/recycler.cpp index 0e5293b916e730..df60a827c0afcf 100644 --- a/cloud/src/recycler/recycler.cpp +++ b/cloud/src/recycler/recycler.cpp @@ -167,6 +167,7 @@ static inline void check_recycle_task(const std::string& instance_id, const std: Recycler::Recycler(std::shared_ptr txn_kv) : txn_kv_(std::move(txn_kv)) { ip_port_ = std::string(butil::my_ip_cstr()) + ":" + std::to_string(config::brpc_listen_port); + auto s3_producer_pool = std::make_shared(config::recycle_pool_parallelism); s3_producer_pool->start(); auto recycle_tablet_pool = std::make_shared(config::recycle_pool_parallelism); @@ -177,6 +178,8 @@ Recycler::Recycler(std::shared_ptr txn_kv) : txn_kv_(std::move(txn_kv)) { _thread_pool_group = RecyclerThreadPoolGroup(std::move(s3_producer_pool), std::move(recycle_tablet_pool), std::move(group_recycle_function_pool)); + + txn_lazy_committer_ = std::make_shared(txn_kv_); } Recycler::~Recycler() { @@ -236,8 +239,9 @@ void Recycler::recycle_callback() { // skip instance in recycling if (recycling_instance_map_.count(instance_id)) continue; } - auto instance_recycler = - std::make_shared(txn_kv_, instance, _thread_pool_group); + auto instance_recycler = std::make_shared( + txn_kv_, instance, _thread_pool_group, txn_lazy_committer_); + if (instance_recycler->init() != 0) { LOG(WARNING) << "failed to init instance recycler, instance_id=" << instance_id; continue; @@ -337,7 +341,8 @@ int Recycler::start(brpc::Server* server) { if (server) { // Add service - auto recycler_service = new RecyclerServiceImpl(txn_kv_, this, checker_.get()); + auto recycler_service = + new RecyclerServiceImpl(txn_kv_, this, checker_.get(), txn_lazy_committer_); server->AddService(recycler_service, brpc::SERVER_OWNS_SERVICE); } @@ -454,12 +459,14 @@ class InstanceRecycler::InvertedIndexIdCache { }; InstanceRecycler::InstanceRecycler(std::shared_ptr txn_kv, const InstanceInfoPB& instance, - RecyclerThreadPoolGroup thread_pool_group) + RecyclerThreadPoolGroup thread_pool_group, + std::shared_ptr txn_lazy_committer) : txn_kv_(std::move(txn_kv)), instance_id_(instance.instance_id()), instance_info_(instance), inverted_index_id_cache_(std::make_unique(instance_id_, txn_kv_)), - _thread_pool_group(std::move(thread_pool_group)) {} + _thread_pool_group(std::move(thread_pool_group)), + txn_lazy_committer_(std::move(txn_lazy_committer)) {}; InstanceRecycler::~InstanceRecycler() = default; @@ -586,7 +593,7 @@ int InstanceRecycler::do_recycle() { [this]() -> int { return InstanceRecycler::recycle_tmp_rowsets(); }, [this]() -> int { return InstanceRecycler::recycle_rowsets(); })) .add(task_wrapper( - [this]() { return InstanceRecycler::abort_timeout_txn(); }, + [this]() { return InstanceRecycler::advance_pending_txn(); }, [this]() { return InstanceRecycler::recycle_expired_txn_label(); })) .add(task_wrapper([this]() { return InstanceRecycler::recycle_copy_jobs(); })) .add(task_wrapper([this]() { return InstanceRecycler::recycle_stage(); })) @@ -824,6 +831,78 @@ int InstanceRecycler::recycle_indexes() { return scan_and_recycle(index_key0, index_key1, std::move(recycle_func), std::move(loop_done)); } +bool check_lazy_txn_finished(std::shared_ptr txn_kv, const std::string instance_id, + int64_t tablet_id) { + std::unique_ptr txn; + TxnErrorCode err = txn_kv->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to create txn, instance_id=" << instance_id + << " tablet_id=" << tablet_id << " err=" << err; + return false; + } + + std::string tablet_idx_key = meta_tablet_idx_key({instance_id, tablet_id}); + std::string tablet_idx_val; + err = txn->get(tablet_idx_key, &tablet_idx_val); + if (TxnErrorCode::TXN_OK != err) { + LOG(WARNING) << "failed to get tablet index, instance_id=" << instance_id + << " tablet_id=" << tablet_id << " err=" << err + << " key=" << hex(tablet_idx_key); + return false; + } + + TabletIndexPB tablet_idx_pb; + if (!tablet_idx_pb.ParseFromString(tablet_idx_val)) { + LOG(WARNING) << "failed to parse tablet_idx_pb, instance_id=" << instance_id + << " tablet_id=" << tablet_id; + return false; + } + + if (!tablet_idx_pb.has_db_id()) { + return true; + } + + std::string ver_val; + std::string ver_key = + partition_version_key({instance_id, tablet_idx_pb.db_id(), tablet_idx_pb.table_id(), + tablet_idx_pb.partition_id()}); + err = txn->get(ver_key, &ver_val); + + if (TxnErrorCode::TXN_KEY_NOT_FOUND == err) { + return true; + } + + if (TxnErrorCode::TXN_OK != err) { + LOG(WARNING) << "failed to get partition version, instance_id=" << instance_id + << " db_id=" << tablet_idx_pb.db_id() + << " table_id=" << tablet_idx_pb.table_id() + << " partition_id=" << tablet_idx_pb.partition_id() + << " tablet_id=" << tablet_id << " key=" << hex(ver_key) << " err=" << err; + return false; + } + + VersionPB version_pb; + if (!version_pb.ParseFromString(ver_val)) { + LOG(WARNING) << "failed to parse version_pb, instance_id=" << instance_id + << " db_id=" << tablet_idx_pb.db_id() + << " table_id=" << tablet_idx_pb.table_id() + << " partition_id=" << tablet_idx_pb.partition_id() + << " tablet_id=" << tablet_id << " key=" << hex(ver_key); + return false; + } + + if (version_pb.has_txn_id()) { + LOG(WARNING) << "lazy txn not finished, instance_id=" << instance_id + << " db_id=" << tablet_idx_pb.db_id() + << " table_id=" << tablet_idx_pb.table_id() + << " partition_id=" << tablet_idx_pb.partition_id() + << " tablet_id=" << tablet_id << " txn_id=" << version_pb.txn_id() + << " key=" << hex(ver_key); + return false; + } + return true; +} + int InstanceRecycler::recycle_partitions() { const std::string task_name = "recycle_partitions"; int num_scanned = 0; @@ -923,6 +1002,7 @@ int InstanceRecycler::recycle_partitions() { return -1; } } + int ret = 0; for (int64_t index_id : part_pb.index_id()) { if (recycle_tablets(part_pb.table_id(), index_id, partition_id, is_empty_tablet) != 0) { @@ -1035,7 +1115,8 @@ int InstanceRecycler::recycle_versions() { partition_version_key({instance_id_, db_id, table_id, INT64_MAX}); txn->remove(partition_version_key_begin, partition_version_key_end); LOG(WARNING) << "remove partition version kv, begin=" << hex(partition_version_key_begin) - << " end=" << hex(partition_version_key_end); + << " end=" << hex(partition_version_key_end) << " db_id=" << db_id + << " table_id=" << table_id; // 2. Remove the table version kv of this table auto tbl_version_key = table_version_key({instance_id_, db_id, table_id}); txn->remove(tbl_version_key); @@ -1119,6 +1200,12 @@ int InstanceRecycler::recycle_tablets(int64_t table_id, int64_t index_id, int64_ return -1; } int64_t tablet_id = tablet_meta_pb.tablet_id(); + + if (!check_lazy_txn_finished(txn_kv_, instance_id_, tablet_meta_pb.tablet_id())) { + LOG(WARNING) << "lazy txn not finished tablet_id=" << tablet_meta_pb.tablet_id(); + return -1; + } + tablet_idx_keys.push_back(meta_tablet_idx_key({instance_id_, tablet_id})); if (!is_empty_tablet) { sync_executor.add([this, &num_recycled, tid = tablet_id, range_move = use_range_remove, @@ -1222,6 +1309,10 @@ int InstanceRecycler::recycle_tablets(int64_t table_id, int64_t index_id, int64_ int ret = scan_and_recycle(tablet_key_begin, tablet_key_end, std::move(recycle_func), std::move(loop_done)); + if (ret != 0) { + LOG(WARNING) << "failed to scan_and_recycle, instance_id=" << instance_id_; + return ret; + } // directly remove tablet stats and tablet jobs of these dropped index or partition std::unique_ptr txn; @@ -1707,6 +1798,64 @@ int InstanceRecycler::recycle_rowsets() { return ret; } +bool check_txn_abort(std::shared_ptr txn_kv, const std::string& instance_id, + int64_t txn_id) { + std::unique_ptr txn; + TxnErrorCode err = txn_kv->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to create txn, txn_id=" << txn_id << " instance_id=" << instance_id; + return false; + } + + std::string index_val; + const std::string index_key = txn_index_key({instance_id, txn_id}); + err = txn->get(index_key, &index_val); + if (err != TxnErrorCode::TXN_OK) { + if (TxnErrorCode::TXN_KEY_NOT_FOUND == err) { + // txn has been recycled; + return true; + } + LOG(WARNING) << "failed to get txn index key, txn_id=" << txn_id + << " instance_id=" << instance_id << " key=" << hex(index_key) + << " err=" << err; + return false; + } + + TxnIndexPB index_pb; + if (!index_pb.ParseFromString(index_val)) { + LOG(WARNING) << "failed to parse txn_index_pb, txn_id=" << txn_id + << " instance_id=" << instance_id; + return false; + } + + DCHECK(index_pb.has_tablet_index() == true); + DCHECK(index_pb.tablet_index().has_db_id() == true); + int64_t db_id = index_pb.tablet_index().db_id(); + + std::string info_val; + const std::string info_key = txn_info_key({instance_id, db_id, txn_id}); + err = txn->get(info_key, &info_val); + if (err != TxnErrorCode::TXN_OK) { + DCHECK(err != TxnErrorCode::TXN_KEY_NOT_FOUND); + LOG(WARNING) << "failed to get txn info key, txn_id=" << txn_id + << " instance_id=" << instance_id << " key=" << hex(info_key) + << " err=" << err; + return false; + } + + TxnInfoPB txn_info; + if (!txn_info.ParseFromString(info_val)) { + LOG(WARNING) << "failed to parse txn_info, txn_id=" << txn_id + << " instance_id=" << instance_id; + return false; + } + DCHECK(txn_info.txn_id() == txn_id); + if (TxnStatusPB::TXN_STATUS_ABORTED == txn_info.status()) { + return true; + } + return false; +} + int InstanceRecycler::recycle_tmp_rowsets() { const std::string task_name = "recycle_tmp_rowsets"; int num_scanned = 0; @@ -1764,6 +1913,11 @@ int InstanceRecycler::recycle_tmp_rowsets() { // not expired return 0; } + + if (!check_txn_abort(txn_kv_, instance_id_, rowset.txn_id())) { + return 0; + } + ++num_expired; expired_rowset_size += v.size(); if (!rowset.has_resource_id()) { @@ -1843,11 +1997,12 @@ int InstanceRecycler::scan_and_recycle( return ret; } -int InstanceRecycler::abort_timeout_txn() { - const std::string task_name = "abort_timeout_txn"; +int InstanceRecycler::advance_pending_txn() { + const std::string task_name = "advance_pending_txn"; int num_scanned = 0; int num_timeout = 0; int num_abort = 0; + int num_advance = 0; TxnRunningKeyInfo txn_running_key_info0 {instance_id_, 0, 0}; TxnRunningKeyInfo txn_running_key_info1 {instance_id_, INT64_MAX, INT64_MAX}; @@ -1869,24 +2024,17 @@ int InstanceRecycler::abort_timeout_txn() { .tag("instance_id", instance_id_) .tag("num_scanned", num_scanned) .tag("num_timeout", num_timeout) - .tag("num_abort", num_abort); + .tag("num_abort", num_abort) + .tag("num_advance", num_advance); }); int64_t current_time = duration_cast(system_clock::now().time_since_epoch()).count(); - auto handle_txn_running_kv = [&num_scanned, &num_timeout, &num_abort, ¤t_time, this]( - std::string_view k, std::string_view v) -> int { + auto handle_txn_running_kv = [&num_scanned, &num_timeout, &num_abort, &num_advance, + ¤t_time, + this](std::string_view k, std::string_view v) -> int { ++num_scanned; - TxnRunningPB txn_running_pb; - if (!txn_running_pb.ParseFromArray(v.data(), v.size())) { - LOG_WARNING("malformed txn_running_pb").tag("key", hex(k)); - return -1; - } - if (txn_running_pb.timeout_time() > current_time) { - return 0; - } - ++num_timeout; std::unique_ptr txn; TxnErrorCode err = txn_kv_->create_txn(&txn); @@ -1918,42 +2066,69 @@ int InstanceRecycler::abort_timeout_txn() { LOG_WARNING("failed to parse txn info").tag("key", hex(k)); return -1; } - txn_info.set_status(TxnStatusPB::TXN_STATUS_ABORTED); - txn_info.set_finish_time(current_time); - txn_info.set_reason("timeout"); - VLOG_DEBUG << "txn_info=" << txn_info.DebugString(); - txn_inf_val.clear(); - if (!txn_info.SerializeToString(&txn_inf_val)) { - LOG_WARNING("failed to serialize txn info").tag("key", hex(k)); - return -1; - } - txn->put(txn_inf_key, txn_inf_val); - VLOG_DEBUG << "txn->put, txn_inf_key=" << hex(txn_inf_key); - // Put recycle txn key - std::string recyc_txn_key, recyc_txn_val; - recycle_txn_key({instance_id_, db_id, txn_id}, &recyc_txn_key); - RecycleTxnPB recycle_txn_pb; - recycle_txn_pb.set_creation_time(current_time); - recycle_txn_pb.set_label(txn_info.label()); - if (!recycle_txn_pb.SerializeToString(&recyc_txn_val)) { - LOG_WARNING("failed to serialize txn recycle info") - .tag("key", hex(k)) - .tag("db_id", db_id) - .tag("txn_id", txn_id); - return -1; - } - txn->put(recyc_txn_key, recyc_txn_val); - // Remove txn running key - txn->remove(k); - err = txn->commit(); - if (err != TxnErrorCode::TXN_OK) { - LOG_WARNING("failed to commit txn err={}", err) - .tag("key", hex(k)) - .tag("db_id", db_id) - .tag("txn_id", txn_id); - return -1; + + if (TxnStatusPB::TXN_STATUS_COMMITTED == txn_info.status()) { + txn.reset(); + std::shared_ptr task = + txn_lazy_committer_->submit(instance_id_, txn_info.txn_id()); + std::pair ret = task->wait(); + if (ret.first != MetaServiceCode::OK) { + LOG(WARNING) << "lazy commit txn failed txn_id=" << txn_id << " code=" << ret.first + << "msg=" << ret.second; + return -1; + } + ++num_advance; + return 0; + } else { + TxnRunningPB txn_running_pb; + if (!txn_running_pb.ParseFromArray(v.data(), v.size())) { + LOG_WARNING("malformed txn_running_pb").tag("key", hex(k)); + return -1; + } + if (txn_running_pb.timeout_time() > current_time) { + return 0; + } + ++num_timeout; + + DCHECK(txn_info.status() != TxnStatusPB::TXN_STATUS_VISIBLE); + txn_info.set_status(TxnStatusPB::TXN_STATUS_ABORTED); + txn_info.set_finish_time(current_time); + txn_info.set_reason("timeout"); + VLOG_DEBUG << "txn_info=" << txn_info.DebugString(); + txn_inf_val.clear(); + if (!txn_info.SerializeToString(&txn_inf_val)) { + LOG_WARNING("failed to serialize txn info").tag("key", hex(k)); + return -1; + } + txn->put(txn_inf_key, txn_inf_val); + VLOG_DEBUG << "txn->put, txn_inf_key=" << hex(txn_inf_key); + // Put recycle txn key + std::string recyc_txn_key, recyc_txn_val; + recycle_txn_key({instance_id_, db_id, txn_id}, &recyc_txn_key); + RecycleTxnPB recycle_txn_pb; + recycle_txn_pb.set_creation_time(current_time); + recycle_txn_pb.set_label(txn_info.label()); + if (!recycle_txn_pb.SerializeToString(&recyc_txn_val)) { + LOG_WARNING("failed to serialize txn recycle info") + .tag("key", hex(k)) + .tag("db_id", db_id) + .tag("txn_id", txn_id); + return -1; + } + txn->put(recyc_txn_key, recyc_txn_val); + // Remove txn running key + txn->remove(k); + err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + LOG_WARNING("failed to commit txn err={}", err) + .tag("key", hex(k)) + .tag("db_id", db_id) + .tag("txn_id", txn_id); + return -1; + } + ++num_abort; } - ++num_abort; + return 0; }; diff --git a/cloud/src/recycler/recycler.h b/cloud/src/recycler/recycler.h index 950c0193f77b98..c37b8ca860b17f 100644 --- a/cloud/src/recycler/recycler.h +++ b/cloud/src/recycler/recycler.h @@ -28,6 +28,7 @@ #include #include +#include "meta-service/txn_lazy_committer.h" #include "recycler/storage_vault_accessor.h" #include "recycler/white_black_list.h" @@ -101,13 +102,17 @@ class Recycler { WhiteBlackList instance_filter_; std::unique_ptr checker_; + RecyclerThreadPoolGroup _thread_pool_group; + + std::shared_ptr txn_lazy_committer_; }; class InstanceRecycler { public: explicit InstanceRecycler(std::shared_ptr txn_kv, const InstanceInfoPB& instance, - RecyclerThreadPoolGroup thread_pool_group); + RecyclerThreadPoolGroup thread_pool_group, + std::shared_ptr txn_lazy_committer); ~InstanceRecycler(); // returns 0 for success otherwise error @@ -161,7 +166,7 @@ class InstanceRecycler { // scan and abort timeout txn label // returns 0 for success otherwise error - int abort_timeout_txn(); + int advance_pending_txn(); //scan and recycle expire txn label // returns 0 for success otherwise error @@ -239,7 +244,10 @@ class InstanceRecycler { std::mutex recycle_tasks_mutex; // > std::map running_recycle_tasks; + RecyclerThreadPoolGroup _thread_pool_group; + + std::shared_ptr txn_lazy_committer_; }; } // namespace doris::cloud diff --git a/cloud/src/recycler/recycler_service.cpp b/cloud/src/recycler/recycler_service.cpp index 52c510fb2e7da0..3c1a5b2ab48d95 100644 --- a/cloud/src/recycler/recycler_service.cpp +++ b/cloud/src/recycler/recycler_service.cpp @@ -42,8 +42,12 @@ extern int reset_s3_rate_limiter(S3RateLimitType type, size_t max_speed, size_t extern std::tuple convert_ms_code_to_http_code(MetaServiceCode ret); RecyclerServiceImpl::RecyclerServiceImpl(std::shared_ptr txn_kv, Recycler* recycler, - Checker* checker) - : txn_kv_(std::move(txn_kv)), recycler_(recycler), checker_(checker) {} + Checker* checker, + std::shared_ptr txn_lazy_committer) + : txn_kv_(std::move(txn_kv)), + recycler_(recycler), + checker_(checker), + txn_lazy_committer_(std::move(txn_lazy_committer)) {} RecyclerServiceImpl::~RecyclerServiceImpl() = default; @@ -152,7 +156,8 @@ void RecyclerServiceImpl::check_instance(const std::string& instance_id, MetaSer void recycle_copy_jobs(const std::shared_ptr& txn_kv, const std::string& instance_id, MetaServiceCode& code, std::string& msg, - RecyclerThreadPoolGroup thread_pool_group) { + RecyclerThreadPoolGroup thread_pool_group, + std::shared_ptr txn_lazy_committer) { std::unique_ptr txn; TxnErrorCode err = txn_kv->create_txn(&txn); if (err != TxnErrorCode::TXN_OK) { @@ -190,7 +195,8 @@ void recycle_copy_jobs(const std::shared_ptr& txn_kv, const std::string& } } - auto recycler = std::make_unique(txn_kv, instance, thread_pool_group); + auto recycler = std::make_unique(txn_kv, instance, thread_pool_group, + txn_lazy_committer); if (recycler->init() != 0) { LOG(WARNING) << "failed to init InstanceRecycler recycle_copy_jobs on instance " << instance_id; @@ -333,7 +339,9 @@ void RecyclerServiceImpl::http(::google::protobuf::RpcController* controller, status_code = 400; return; } - recycle_copy_jobs(txn_kv_, *instance_id, code, msg, recycler_->_thread_pool_group); + recycle_copy_jobs(txn_kv_, *instance_id, code, msg, recycler_->_thread_pool_group, + txn_lazy_committer_); + response_body = msg; return; } diff --git a/cloud/src/recycler/recycler_service.h b/cloud/src/recycler/recycler_service.h index 8f9faec1efeb7d..5ece69c86ba0da 100644 --- a/cloud/src/recycler/recycler_service.h +++ b/cloud/src/recycler/recycler_service.h @@ -20,6 +20,7 @@ #include #include "meta-service/txn_kv.h" +#include "meta-service/txn_lazy_committer.h" namespace doris::cloud { @@ -28,7 +29,8 @@ class Checker; class RecyclerServiceImpl : public cloud::RecyclerService { public: - RecyclerServiceImpl(std::shared_ptr txn_kv, Recycler* recycler, Checker* checker); + RecyclerServiceImpl(std::shared_ptr txn_kv, Recycler* recycler, Checker* checker, + std::shared_ptr txn_lazy_committer); ~RecyclerServiceImpl() override; void recycle_instance(::google::protobuf::RpcController* controller, @@ -48,6 +50,7 @@ class RecyclerServiceImpl : public cloud::RecyclerService { std::shared_ptr txn_kv_; Recycler* recycler_; // Ref Checker* checker_; // Ref + std::shared_ptr txn_lazy_committer_; }; } // namespace doris::cloud diff --git a/cloud/test/CMakeLists.txt b/cloud/test/CMakeLists.txt index ba5e2909918e53..b504a882534a16 100644 --- a/cloud/test/CMakeLists.txt +++ b/cloud/test/CMakeLists.txt @@ -56,6 +56,8 @@ add_executable(util_test util_test.cpp) add_executable(network_util_test network_util_test.cpp) +add_executable(txn_lazy_commit_test txn_lazy_commit_test.cpp) + message("Meta-service test dependencies: ${TEST_LINK_LIBS}") #target_link_libraries(sync_point_test ${TEST_LINK_LIBS}) @@ -91,6 +93,8 @@ target_link_libraries(util_test ${TEST_LINK_LIBS}) target_link_libraries(network_util_test ${TEST_LINK_LIBS}) +target_link_libraries(txn_lazy_commit_test ${TEST_LINK_LIBS}) + # FDB related tests need to be linked with libfdb_c set(FDB_LINKER_FLAGS "-lfdb_c -L${THIRDPARTY_DIR}/lib") diff --git a/cloud/test/meta_service_test.cpp b/cloud/test/meta_service_test.cpp index db187cdc0991ad..7920ce296c5c8d 100644 --- a/cloud/test/meta_service_test.cpp +++ b/cloud/test/meta_service_test.cpp @@ -163,7 +163,7 @@ static void commit_txn(MetaServiceProxy* meta_service, int64_t db_id, int64_t tx } static doris::RowsetMetaCloudPB create_rowset(int64_t txn_id, int64_t tablet_id, - int partition_id = 0, int64_t version = -1, + int partition_id = 10, int64_t version = -1, int num_rows = 100) { doris::RowsetMetaCloudPB rowset; rowset.set_rowset_id(0); // required diff --git a/cloud/test/recycler_test.cpp b/cloud/test/recycler_test.cpp index ca8ffbcee6139e..68dd1950ed08b0 100644 --- a/cloud/test/recycler_test.cpp +++ b/cloud/test/recycler_test.cpp @@ -260,17 +260,27 @@ static int create_tablet(TxnKv* txn_kv, int64_t table_id, int64_t index_id, int6 if (txn_kv->create_txn(&txn) != TxnErrorCode::TXN_OK) { return -1; } - auto key = meta_tablet_key({instance_id, table_id, index_id, partition_id, tablet_id}); doris::TabletMetaCloudPB tablet_meta; tablet_meta.set_tablet_id(tablet_id); auto val = tablet_meta.SerializeAsString(); + auto key = meta_tablet_key({instance_id, table_id, index_id, partition_id, tablet_id}); txn->put(key, val); - key = meta_tablet_idx_key({instance_id, tablet_id}); - txn->put(key, val); // val is not necessary key = stats_tablet_key({instance_id, table_id, index_id, partition_id, tablet_id}); txn->put(key, val); // val is not necessary key = job_tablet_key({instance_id, table_id, index_id, partition_id, tablet_id}); txn->put(key, val); // val is not necessary + + // mock tablet index + TabletIndexPB tablet_idx_pb; + tablet_idx_pb.set_db_id(db_id); + tablet_idx_pb.set_table_id(table_id); + tablet_idx_pb.set_partition_id(partition_id); + tablet_idx_pb.set_tablet_id(tablet_id); + auto idx_val = tablet_idx_pb.SerializeAsString(); + key = meta_tablet_idx_key({instance_id, tablet_id}); + txn->put(key, idx_val); + LOG(INFO) << "tablet_idx_pb:" << tablet_idx_pb.DebugString() << " key=" << hex(key); + if (txn->commit() != TxnErrorCode::TXN_OK) { return -1; } @@ -631,7 +641,8 @@ TEST(RecyclerTest, recycle_empty) { obj_info->set_bucket(config::test_s3_bucket); obj_info->set_prefix("recycle_empty"); - InstanceRecycler recycler(txn_kv, instance, thread_group); + InstanceRecycler recycler(txn_kv, instance, thread_group, + std::make_shared(txn_kv)); ASSERT_EQ(recycler.init(), 0); ASSERT_EQ(recycler.recycle_rowsets(), 0); @@ -664,7 +675,8 @@ TEST(RecyclerTest, recycle_rowsets) { sp->set_call_back("InvertedIndexIdCache::insert2", [&](auto&&) { ++insert_inverted_index; }); sp->enable_processing(); - InstanceRecycler recycler(txn_kv, instance, thread_group); + InstanceRecycler recycler(txn_kv, instance, thread_group, + std::make_shared(txn_kv)); ASSERT_EQ(recycler.init(), 0); std::vector schemas; @@ -729,7 +741,8 @@ TEST(RecyclerTest, bench_recycle_rowsets) { config::instance_recycler_worker_pool_size = 10; config::recycle_task_threshold_seconds = 0; - InstanceRecycler recycler(txn_kv, instance, thread_group); + InstanceRecycler recycler(txn_kv, instance, thread_group, + std::make_shared(txn_kv)); ASSERT_EQ(recycler.init(), 0); auto sp = SyncPoint::get_instance(); @@ -812,7 +825,8 @@ TEST(RecyclerTest, recycle_tmp_rowsets) { sp->set_call_back("InvertedIndexIdCache::insert2", [&](auto&&) { ++insert_inverted_index; }); sp->enable_processing(); - InstanceRecycler recycler(txn_kv, instance, thread_group); + InstanceRecycler recycler(txn_kv, instance, thread_group, + std::make_shared(txn_kv)); ASSERT_EQ(recycler.init(), 0); std::vector schemas; @@ -874,7 +888,8 @@ TEST(RecyclerTest, recycle_tablet) { obj_info->set_bucket(config::test_s3_bucket); obj_info->set_prefix("recycle_tablet"); - InstanceRecycler recycler(txn_kv, instance, thread_group); + InstanceRecycler recycler(txn_kv, instance, thread_group, + std::make_shared(txn_kv)); ASSERT_EQ(recycler.init(), 0); std::vector schemas; @@ -901,6 +916,8 @@ TEST(RecyclerTest, recycle_tablet) { create_committed_rowset(txn_kv.get(), accessor.get(), "recycle_tablet", tablet_id, i); } + ASSERT_EQ(create_partition_version_kv(txn_kv.get(), table_id, partition_id), 0); + ASSERT_EQ(0, recycler.recycle_tablets(table_id, index_id)); // check rowset does not exist on s3 @@ -949,7 +966,8 @@ TEST(RecyclerTest, recycle_indexes) { obj_info->set_bucket(config::test_s3_bucket); obj_info->set_prefix("recycle_indexes"); - InstanceRecycler recycler(txn_kv, instance, thread_group); + InstanceRecycler recycler(txn_kv, instance, thread_group, + std::make_shared(txn_kv)); ASSERT_EQ(recycler.init(), 0); std::vector schemas; @@ -984,6 +1002,8 @@ TEST(RecyclerTest, recycle_indexes) { create_committed_rowset(txn_kv.get(), accessor.get(), "recycle_indexes", tablet_id, j); } } + + ASSERT_EQ(create_partition_version_kv(txn_kv.get(), table_id, partition_id), 0); create_recycle_index(txn_kv.get(), table_id, index_id); ASSERT_EQ(recycler.recycle_indexes(), 0); @@ -1061,7 +1081,8 @@ TEST(RecyclerTest, recycle_partitions) { obj_info->set_bucket(config::test_s3_bucket); obj_info->set_prefix("recycle_partitions"); - InstanceRecycler recycler(txn_kv, instance, thread_group); + InstanceRecycler recycler(txn_kv, instance, thread_group, + std::make_shared(txn_kv)); ASSERT_EQ(recycler.init(), 0); std::vector schemas; @@ -1083,8 +1104,8 @@ TEST(RecyclerTest, recycle_partitions) { int64_t tablet_id_base = 10100; for (auto index_id : index_ids) { for (int i = 0; i < 20; ++i) { - int64_t tablet_id = tablet_id_base + i; - create_tablet(txn_kv.get(), table_id, index_id, partition_id, tablet_id); + int64_t tablet_id = tablet_id_base++; + ASSERT_EQ(create_tablet(txn_kv.get(), table_id, index_id, partition_id, tablet_id), 0); for (int j = 0; j < 10; ++j) { auto rowset = create_rowset("recycle_tablet", tablet_id, index_id, 5, schemas[j % 5]); @@ -1098,6 +1119,9 @@ TEST(RecyclerTest, recycle_partitions) { } } } + + ASSERT_EQ(create_partition_version_kv(txn_kv.get(), table_id, partition_id), 0); + create_recycle_partiton(txn_kv.get(), table_id, partition_id, index_ids); ASSERT_EQ(recycler.recycle_partitions(), 0); @@ -1153,7 +1177,7 @@ TEST(RecyclerTest, recycle_versions) { std::vector index_ids {20001, 20002, 20003, 20004, 20005}; std::vector partition_ids {30001, 30002, 30003, 30004, 30005, 30006}; - constexpr int64_t table_id = 10000; + constexpr int table_id = 10000; int64_t tablet_id = 40000; for (auto index_id : index_ids) { @@ -1172,7 +1196,8 @@ TEST(RecyclerTest, recycle_versions) { InstanceInfoPB instance; instance.set_instance_id(instance_id); - InstanceRecycler recycler(txn_kv, instance, thread_group); + InstanceRecycler recycler(txn_kv, instance, thread_group, + std::make_shared(txn_kv)); ASSERT_EQ(recycler.init(), 0); // Recycle all partitions in table except 30006 ASSERT_EQ(recycler.recycle_partitions(), 0); @@ -1210,7 +1235,7 @@ TEST(RecyclerTest, recycle_versions) { ASSERT_EQ(iter->size(), 0); } -TEST(RecyclerTest, abort_timeout_txn) { +TEST(RecyclerTest, advance_pending_txn) { auto txn_kv = std::dynamic_pointer_cast(std::make_shared()); ASSERT_NE(txn_kv.get(), nullptr); auto rs = std::make_shared(txn_kv); @@ -1228,7 +1253,7 @@ TEST(RecyclerTest, abort_timeout_txn) { req.set_cloud_unique_id("test_cloud_unique_id"); TxnInfoPB txn_info_pb; txn_info_pb.set_db_id(db_id); - txn_info_pb.set_label("abort_timeout_txn"); + txn_info_pb.set_label("advance_pending_txn"); txn_info_pb.add_table_ids(table_id); txn_info_pb.set_timeout_ms(1); req.mutable_txn_info()->CopyFrom(txn_info_pb); @@ -1241,16 +1266,17 @@ TEST(RecyclerTest, abort_timeout_txn) { } InstanceInfoPB instance; instance.set_instance_id(mock_instance); - InstanceRecycler recycler(txn_kv, instance, thread_group); + InstanceRecycler recycler(txn_kv, instance, thread_group, + std::make_shared(txn_kv)); ASSERT_EQ(recycler.init(), 0); sleep(1); - ASSERT_EQ(recycler.abort_timeout_txn(), 0); + ASSERT_EQ(recycler.advance_pending_txn(), 0); TxnInfoPB txn_info_pb; get_txn_info(txn_kv, mock_instance, db_id, txn_id, txn_info_pb); ASSERT_EQ(txn_info_pb.status(), TxnStatusPB::TXN_STATUS_ABORTED); } -TEST(RecyclerTest, abort_timeout_txn_and_rebegin) { +TEST(RecyclerTest, advance_pending_txn_and_rebegin) { config::label_keep_max_second = 0; auto txn_kv = std::dynamic_pointer_cast(std::make_shared()); ASSERT_NE(txn_kv.get(), nullptr); @@ -1263,7 +1289,7 @@ TEST(RecyclerTest, abort_timeout_txn_and_rebegin) { int64_t table_id = 1234; int64_t txn_id = -1; std::string cloud_unique_id = "test_cloud_unique_id22131"; - std::string label = "abort_timeout_txn_and_rebegin"; + std::string label = "advance_pending_txn_and_rebegin"; { brpc::Controller cntl; BeginTxnRequest req; @@ -1284,10 +1310,11 @@ TEST(RecyclerTest, abort_timeout_txn_and_rebegin) { } InstanceInfoPB instance; instance.set_instance_id(mock_instance); - InstanceRecycler recycler(txn_kv, instance, thread_group); + InstanceRecycler recycler(txn_kv, instance, thread_group, + std::make_shared(txn_kv)); ASSERT_EQ(recycler.init(), 0); sleep(1); - ASSERT_EQ(recycler.abort_timeout_txn(), 0); + ASSERT_EQ(recycler.advance_pending_txn(), 0); TxnInfoPB txn_info_pb; get_txn_info(txn_kv, mock_instance, db_id, txn_id, txn_info_pb); ASSERT_EQ(txn_info_pb.status(), TxnStatusPB::TXN_STATUS_ABORTED); @@ -1351,9 +1378,10 @@ TEST(RecyclerTest, recycle_expired_txn_label) { } InstanceInfoPB instance; instance.set_instance_id(mock_instance); - InstanceRecycler recycler(txn_kv, instance, thread_group); + InstanceRecycler recycler(txn_kv, instance, thread_group, + std::make_shared(txn_kv)); ASSERT_EQ(recycler.init(), 0); - recycler.abort_timeout_txn(); + recycler.advance_pending_txn(); TxnInfoPB txn_info_pb; ASSERT_EQ(get_txn_info(txn_kv, mock_instance, db_id, txn_id, txn_info_pb), 0); ASSERT_EQ(txn_info_pb.status(), TxnStatusPB::TXN_STATUS_PREPARED); @@ -1402,10 +1430,11 @@ TEST(RecyclerTest, recycle_expired_txn_label) { } InstanceInfoPB instance; instance.set_instance_id(mock_instance); - InstanceRecycler recycler(txn_kv, instance, thread_group); + InstanceRecycler recycler(txn_kv, instance, thread_group, + std::make_shared(txn_kv)); ASSERT_EQ(recycler.init(), 0); sleep(1); - recycler.abort_timeout_txn(); + recycler.advance_pending_txn(); TxnInfoPB txn_info_pb; get_txn_info(txn_kv, mock_instance, db_id, txn_id, txn_info_pb); ASSERT_EQ(txn_info_pb.status(), TxnStatusPB::TXN_STATUS_PREPARED); @@ -1454,10 +1483,11 @@ TEST(RecyclerTest, recycle_expired_txn_label) { } InstanceInfoPB instance; instance.set_instance_id(mock_instance); - InstanceRecycler recycler(txn_kv, instance, thread_group); + InstanceRecycler recycler(txn_kv, instance, thread_group, + std::make_shared(txn_kv)); ASSERT_EQ(recycler.init(), 0); sleep(1); - recycler.abort_timeout_txn(); + recycler.advance_pending_txn(); TxnInfoPB txn_info_pb; get_txn_info(txn_kv, mock_instance, db_id, txn_id, txn_info_pb); ASSERT_EQ(txn_info_pb.status(), TxnStatusPB::TXN_STATUS_PREPARED); @@ -1513,10 +1543,11 @@ TEST(RecyclerTest, recycle_expired_txn_label) { } InstanceInfoPB instance; instance.set_instance_id(mock_instance); - InstanceRecycler recycler(txn_kv, instance, thread_group); + InstanceRecycler recycler(txn_kv, instance, thread_group, + std::make_shared(txn_kv)); ASSERT_EQ(recycler.init(), 0); sleep(1); - recycler.abort_timeout_txn(); + recycler.advance_pending_txn(); TxnInfoPB txn_info_pb; get_txn_info(txn_kv, mock_instance, db_id, txn_id, txn_info_pb); ASSERT_EQ(txn_info_pb.status(), TxnStatusPB::TXN_STATUS_PREPARED); @@ -1650,7 +1681,8 @@ TEST(RecyclerTest, recycle_copy_jobs) { InstanceInfoPB instance_info; create_instance(internal_stage_id, external_stage_id, instance_info); - InstanceRecycler recycler(txn_kv, instance_info, thread_group); + InstanceRecycler recycler(txn_kv, instance_info, thread_group, + std::make_shared(txn_kv)); ASSERT_EQ(recycler.init(), 0); auto internal_accessor = recycler.accessor_map_.find(internal_stage_id)->second; @@ -1809,7 +1841,8 @@ TEST(RecyclerTest, recycle_batch_copy_jobs) { InstanceInfoPB instance_info; create_instance(internal_stage_id, external_stage_id, instance_info); - InstanceRecycler recycler(txn_kv, instance_info, thread_group); + InstanceRecycler recycler(txn_kv, instance_info, thread_group, + std::make_shared(txn_kv)); ASSERT_EQ(recycler.init(), 0); const auto& internal_accessor = recycler.accessor_map_.find(internal_stage_id)->second; @@ -1923,7 +1956,8 @@ TEST(RecyclerTest, recycle_stage) { instance.set_instance_id(mock_instance); instance.add_obj_info()->CopyFrom(object_info); - InstanceRecycler recycler(txn_kv, instance, thread_group); + InstanceRecycler recycler(txn_kv, instance, thread_group, + std::make_shared(txn_kv)); ASSERT_EQ(recycler.init(), 0); auto accessor = recycler.accessor_map_.begin()->second; for (int i = 0; i < 10; ++i) { @@ -1983,7 +2017,8 @@ TEST(RecyclerTest, recycle_deleted_instance) { InstanceInfoPB instance_info; create_instance(internal_stage_id, external_stage_id, instance_info); - InstanceRecycler recycler(txn_kv, instance_info, thread_group); + InstanceRecycler recycler(txn_kv, instance_info, thread_group, + std::make_shared(txn_kv)); ASSERT_EQ(recycler.init(), 0); // create txn key for (size_t i = 0; i < 100; i++) { @@ -2568,7 +2603,8 @@ TEST(RecyclerTest, delete_rowset_data) { } { - InstanceRecycler recycler(txn_kv, instance, thread_group); + InstanceRecycler recycler(txn_kv, instance, thread_group, + std::make_shared(txn_kv)); ASSERT_EQ(recycler.init(), 0); auto accessor = recycler.accessor_map_.begin()->second; int64_t txn_id_base = 114115; @@ -2602,7 +2638,8 @@ TEST(RecyclerTest, delete_rowset_data) { tmp_obj_info->set_bucket(config::test_s3_bucket); tmp_obj_info->set_prefix(resource_id); - InstanceRecycler recycler(txn_kv, tmp_instance, thread_group); + InstanceRecycler recycler(txn_kv, tmp_instance, thread_group, + std::make_shared(txn_kv)); ASSERT_EQ(recycler.init(), 0); auto accessor = recycler.accessor_map_.begin()->second; // Delete multiple rowset files using one series of RowsetPB @@ -2622,7 +2659,8 @@ TEST(RecyclerTest, delete_rowset_data) { ASSERT_FALSE(list_iter->has_next()); } { - InstanceRecycler recycler(txn_kv, instance, thread_group); + InstanceRecycler recycler(txn_kv, instance, thread_group, + std::make_shared(txn_kv)); ASSERT_EQ(recycler.init(), 0); auto accessor = recycler.accessor_map_.begin()->second; // Delete multiple rowset files using one series of RowsetPB diff --git a/cloud/test/txn_lazy_commit_test.cpp b/cloud/test/txn_lazy_commit_test.cpp new file mode 100644 index 00000000000000..2fb842186fba13 --- /dev/null +++ b/cloud/test/txn_lazy_commit_test.cpp @@ -0,0 +1,216 @@ +// 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 +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include + +#include "common/config.h" +#include "common/logging.h" +#include "common/util.h" +#include "cpp/sync_point.h" +#include "meta-service/keys.h" +#include "meta-service/mem_txn_kv.h" +#include "meta-service/meta_service.h" +#include "meta-service/meta_service_helper.h" +#include "meta-service/txn_kv_error.h" +#include "mock_resource_manager.h" +#include "rate-limiter/rate_limiter.h" +#include "resource-manager/resource_manager.h" + +int main(int argc, char** argv) { + const std::string conf_file = "doris_cloud.conf"; + if (!doris::cloud::config::init(conf_file.c_str(), true)) { + std::cerr << "failed to init config file, conf=" << conf_file << std::endl; + return -1; + } + + config::enable_txn_lazy_commit = true; + config::txn_lazy_commit_rowsets_thresold = 1; + config::txn_lazy_max_rowsets_per_batch = 1; + config::txn_lazy_commit_num_threads = 2; + + if (!doris::cloud::init_glog("txn_lazy_commit_test")) { + std::cerr << "failed to init glog" << std::endl; + return -1; + } + ::testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} +namespace doris::cloud { + +std::unique_ptr get_meta_service(std::shared_ptr txn_kv, + bool mock_resource_mgr) { + std::unique_ptr txn; + EXPECT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->remove("\x00", "\xfe"); // This is dangerous if the fdb is not correctly set + EXPECT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + auto rs = mock_resource_mgr ? std::make_shared(txn_kv) + : std::make_shared(txn_kv); + auto rl = std::make_shared(); + auto meta_service = std::make_unique(txn_kv, rs, rl); + return std::make_unique(std::move(meta_service)); +} + +static std::string next_rowset_id() { + static int cnt = 0; + return std::to_string(++cnt); +} + +static void add_tablet(CreateTabletsRequest& req, int64_t table_id, int64_t index_id, + int64_t partition_id, int64_t tablet_id) { + auto tablet = req.add_tablet_metas(); + tablet->set_table_id(table_id); + tablet->set_index_id(index_id); + tablet->set_partition_id(partition_id); + tablet->set_tablet_id(tablet_id); + auto schema = tablet->mutable_schema(); + schema->set_schema_version(0); + auto first_rowset = tablet->add_rs_metas(); + first_rowset->set_rowset_id(0); // required + first_rowset->set_rowset_id_v2(next_rowset_id()); + first_rowset->set_start_version(0); + first_rowset->set_end_version(1); + first_rowset->mutable_tablet_schema()->CopyFrom(*schema); +} + +static void create_tablet(MetaServiceProxy* meta_service, int64_t table_id, int64_t index_id, + int64_t partition_id, int64_t tablet_id) { + brpc::Controller cntl; + CreateTabletsRequest req; + CreateTabletsResponse res; + add_tablet(req, table_id, index_id, partition_id, tablet_id); + meta_service->create_tablets(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << tablet_id; +} + +static doris::RowsetMetaCloudPB create_rowset(int64_t txn_id, int64_t tablet_id, + int partition_id = 10, int64_t version = -1, + int num_rows = 100) { + doris::RowsetMetaCloudPB rowset; + rowset.set_rowset_id(0); // required + rowset.set_rowset_id_v2(next_rowset_id()); + rowset.set_tablet_id(tablet_id); + rowset.set_partition_id(partition_id); + rowset.set_txn_id(txn_id); + if (version > 0) { + rowset.set_start_version(version); + rowset.set_end_version(version); + } + rowset.set_num_segments(1); + rowset.set_num_rows(num_rows); + rowset.set_data_disk_size(num_rows * 100); + rowset.mutable_tablet_schema()->set_schema_version(0); + rowset.set_txn_expiration(::time(nullptr)); // Required by DCHECK + return rowset; +} + +static void commit_rowset(MetaServiceProxy* meta_service, const doris::RowsetMetaCloudPB& rowset, + CreateRowsetResponse& res) { + brpc::Controller cntl; + auto arena = res.GetArena(); + auto req = google::protobuf::Arena::CreateMessage(arena); + req->mutable_rowset_meta()->CopyFrom(rowset); + meta_service->commit_rowset(&cntl, req, &res, nullptr); + if (!arena) { + delete req; + } +} + +TEST(TxnLazyCommitTest, CommitTxnEventuallyTest) { + int ret = 0; + // MemKv + auto txn_kv = std::dynamic_pointer_cast(std::make_shared()); + if (txn_kv != nullptr) { + ret = txn_kv->init(); + [&] { ASSERT_EQ(ret, 0); }(); + } + [&] { ASSERT_NE(txn_kv.get(), nullptr); }(); + + int64_t db_id = 3213235; + int64_t table_id = 93213131; + + auto meta_service = get_meta_service(txn_kv, true); + brpc::Controller cntl; + BeginTxnRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + TxnInfoPB txn_info_pb; + txn_info_pb.set_db_id(db_id); + txn_info_pb.set_label("test_label"); + txn_info_pb.add_table_ids(table_id); + txn_info_pb.set_timeout_ms(36000); + req.mutable_txn_info()->CopyFrom(txn_info_pb); + BeginTxnResponse res; + meta_service->begin_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, &res, + nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + int64_t txn_id = res.txn_id(); + + // mock rowset and tablet + int64_t tablet_id_base = 1103; + for (int i = 0; i < 5; ++i) { + create_tablet(meta_service.get(), table_id, 1235, 1236, tablet_id_base + i); + auto tmp_rowset = create_rowset(txn_id, tablet_id_base + i); + CreateRowsetResponse res; + commit_rowset(meta_service.get(), tmp_rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } + + { + brpc::Controller cntl; + CommitTxnRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + req.set_db_id(db_id); + req.set_txn_id(txn_id); + req.set_is_2pc(false); + req.set_lazy_commit(true); + CommitTxnResponse res; + meta_service->commit_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, + &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } + + { + std::unique_ptr txn; + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + std::string mock_instance = "test_instance"; + int64_t tablet_id_base = 1103; + for (int i = 0; i < 5; ++i) { + int64_t tablet_id = tablet_id_base + i; + std::string key = meta_tablet_idx_key({mock_instance, tablet_id}); + std::string val; + ASSERT_EQ(txn->get(key, &val), TxnErrorCode::TXN_OK); + TabletIndexPB tablet_idx_pb; + tablet_idx_pb.ParseFromString(val); + ASSERT_EQ(tablet_idx_pb.db_id(), db_id); + } + } +} +} // namespace doris::cloud \ No newline at end of file diff --git a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java index 69d382c7a5c2e7..c5698fecb4400e 100644 --- a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java +++ b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java @@ -3017,6 +3017,10 @@ public static int metaServiceRpcRetryTimes() { @ConfField(mutable = true, description = {"存算分离模式下commit阶段等锁超时时间,默认5s"}) public static int try_commit_lock_timeout_seconds = 5; + + @ConfField(mutable = true, description = {"存算分离模式下是否开启大事务提交,默认false"}) + public static boolean cloud_txn_lazy_commit = false; + // ATTN: DONOT add any config not related to cloud mode here // ATTN: DONOT add any config not related to cloud mode here // ATTN: DONOT add any config not related to cloud mode here diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/CloudRollupJobV2.java b/fe/fe-core/src/main/java/org/apache/doris/alter/CloudRollupJobV2.java index 9ebbcae4e538b3..57143ed47d72f6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/CloudRollupJobV2.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/CloudRollupJobV2.java @@ -229,6 +229,7 @@ private void createRollupReplicaForPartition(OlapTable tbl) throws Exception { tbl.variantEnableFlattenNested()); requestBuilder.addTabletMetas(builder); } // end for rollupTablets + requestBuilder.setDbId(dbId); ((CloudInternalCatalog) Env.getCurrentInternalCatalog()) .sendCreateTabletsRpc(requestBuilder); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/CloudSchemaChangeJobV2.java b/fe/fe-core/src/main/java/org/apache/doris/alter/CloudSchemaChangeJobV2.java index c126efc8d53b09..0a59ec4c93cbaf 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/CloudSchemaChangeJobV2.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/CloudSchemaChangeJobV2.java @@ -270,6 +270,7 @@ private void createShadowIndexReplicaForPartition(OlapTable tbl) throws Exceptio tbl.variantEnableFlattenNested()); requestBuilder.addTabletMetas(builder); } // end for rollupTablets + requestBuilder.setDbId(dbId); ((CloudInternalCatalog) Env.getCurrentInternalCatalog()) .sendCreateTabletsRpc(requestBuilder); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/datasource/CloudInternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/datasource/CloudInternalCatalog.java index 48871b3fb898c7..a809e9187d2ce7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/cloud/datasource/CloudInternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/datasource/CloudInternalCatalog.java @@ -181,6 +181,7 @@ protected Partition createPartitionWithIndices(long dbId, OlapTable tbl, long pa if (!storageVaultIdSet && ((CloudEnv) Env.getCurrentEnv()).getEnableStorageVault()) { requestBuilder.setStorageVaultName(storageVaultName); } + requestBuilder.setDbId(dbId); LOG.info("create tablets, dbId: {}, tableId: {}, tableName: {}, partitionId: {}, partitionName: {}, " + "indexId: {}, vault name {}", @@ -682,7 +683,7 @@ public void checkMaterializedIndex(long dbId, long tableId, List indexIds) sendCreateTabletsRpc(Cloud.CreateTabletsRequest.Builder requestBuilder) throws DdlException { requestBuilder.setCloudUniqueId(Config.cloud_unique_id); Cloud.CreateTabletsRequest createTabletsReq = requestBuilder.build(); - + Preconditions.checkState(createTabletsReq.hasDbId(), "createTabletsReq must set dbId"); if (LOG.isDebugEnabled()) { LOG.debug("send create tablets rpc, createTabletsReq: {}", createTabletsReq); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/CloudGlobalTransactionMgr.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/CloudGlobalTransactionMgr.java index 3940976f3d90d9..03057735cddbb5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/CloudGlobalTransactionMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/CloudGlobalTransactionMgr.java @@ -472,7 +472,8 @@ private void commitTransaction(long dbId, List tableList, long transactio .setTxnId(transactionId) .setIs2Pc(is2PC) .setCloudUniqueId(Config.cloud_unique_id) - .addAllBaseTabletIds(getBaseTabletsFromTables(tableList, tabletCommitInfos)); + .addAllBaseTabletIds(getBaseTabletsFromTables(tableList, tabletCommitInfos)) + .setLazyCommit(Config.cloud_txn_lazy_commit); // if tablet commit info is empty, no need to pass mowTableList to meta service. if (tabletCommitInfos != null && !tabletCommitInfos.isEmpty()) { @@ -933,7 +934,8 @@ public boolean commitAndPublishTransaction(DatabaseIf db, long transactionId, .setTxnId(transactionId) .setIs2Pc(false) .setCloudUniqueId(Config.cloud_unique_id) - .setIsTxnLoad(true); + .setIsTxnLoad(true) + .setLazyCommit(Config.cloud_txn_lazy_commit); // add sub txn infos for (SubTransactionState subTransactionState : subTransactionStates) { builder.addSubTxnInfos(SubTxnInfo.newBuilder().setSubTxnId(subTransactionState.getSubTransactionId()) diff --git a/gensrc/proto/cloud.proto b/gensrc/proto/cloud.proto index bd56b6db588c60..8035a3a96a3675 100644 --- a/gensrc/proto/cloud.proto +++ b/gensrc/proto/cloud.proto @@ -406,6 +406,7 @@ message TxnRunningPB { message VersionPB { optional int64 version = 1; optional int64 update_time_ms = 2; + optional int64 txn_id = 3; } message RecycleTxnPB { @@ -667,6 +668,7 @@ message CommitTxnRequest { // for transaction load optional bool is_txn_load = 9; repeated SubTxnInfo sub_txn_infos = 10; + optional bool lazy_commit = 11; } message SubTxnInfo { @@ -896,6 +898,7 @@ message CreateTabletsRequest { optional string cloud_unique_id = 1; // For auth repeated doris.TabletMetaCloudPB tablet_metas = 2; optional string storage_vault_name = 3; + optional int64 db_id = 4; } message CreateTabletsResponse { diff --git a/regression-test/data/large_txn_load_p1/test_insert_with_many_buckets.out b/regression-test/data/large_txn_load_p1/test_insert_with_many_buckets.out new file mode 100644 index 00000000000000..a507ec715665f4 --- /dev/null +++ b/regression-test/data/large_txn_load_p1/test_insert_with_many_buckets.out @@ -0,0 +1,4 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !sql -- +999 + diff --git a/regression-test/data/large_txn_load_p1/test_insert_with_many_partitions.out b/regression-test/data/large_txn_load_p1/test_insert_with_many_partitions.out new file mode 100644 index 00000000000000..ce114af904d082 --- /dev/null +++ b/regression-test/data/large_txn_load_p1/test_insert_with_many_partitions.out @@ -0,0 +1,4 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !sql -- +1000 + diff --git a/regression-test/data/large_txn_load_p1/test_insert_with_many_partitions_and_buckets.out b/regression-test/data/large_txn_load_p1/test_insert_with_many_partitions_and_buckets.out new file mode 100644 index 00000000000000..475b9ff01e933a --- /dev/null +++ b/regression-test/data/large_txn_load_p1/test_insert_with_many_partitions_and_buckets.out @@ -0,0 +1,4 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !sql -- +400 + diff --git a/regression-test/suites/large_txn_load_p1/test_insert_with_many_buckets.groovy b/regression-test/suites/large_txn_load_p1/test_insert_with_many_buckets.groovy new file mode 100644 index 00000000000000..ddfabe468e42fe --- /dev/null +++ b/regression-test/suites/large_txn_load_p1/test_insert_with_many_buckets.groovy @@ -0,0 +1,57 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("test_insert_with_many_buckets") { + def tableName = 'test_insert_with_many_buckets' + + // [start, end) + def makeInsertStmt = { start, end -> + def stmt = "" + for (int i = start; i < end; i++) { + if (i == (end - 1)) { + stmt += """(${i}, false, 1, null, "c5", "2023-09-01", null, "c8", 1.1234, "c10")\n""" + } else { + stmt += """(${i}, false, 1, null, "c5", "2023-09-01", null, "c8", 1.1234, "c10"),\n""" + } + } + return stmt + } + + sql """ DROP TABLE IF EXISTS ${tableName} FORCE""" + sql """ CREATE TABLE IF NOT EXISTS ${tableName} ( + `c1` bigint(20) not null, + `c2` boolean REPLACE_IF_NOT_NULL null, + `c3` tinyint(4) REPLACE_IF_NOT_NULL null, + `c4` decimalv3(9, 3) REPLACE_IF_NOT_NULL null, + `c5` char(36) REPLACE_IF_NOT_NULL null, + `c6` date REPLACE_IF_NOT_NULL null, + `c7` datetime REPLACE_IF_NOT_NULL null, + `c8` varchar(64) REPLACE_IF_NOT_NULL null, + `c9` double REPLACE_IF_NOT_NULL null, + `c10` string REPLACE_IF_NOT_NULL null + ) engine=olap + AGGREGATE KEY(`c1`) + DISTRIBUTED BY HASH(`c1`) BUCKETS 1000 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + sql """ insert into ${tableName} values ${makeInsertStmt(1 , 1000)};""" + qt_sql """ select count(*) from ${tableName};""" + sql """ DROP TABLE IF EXISTS ${tableName} FORCE""" +} diff --git a/regression-test/suites/large_txn_load_p1/test_insert_with_many_partitions.groovy b/regression-test/suites/large_txn_load_p1/test_insert_with_many_partitions.groovy new file mode 100644 index 00000000000000..a530f797a0c38c --- /dev/null +++ b/regression-test/suites/large_txn_load_p1/test_insert_with_many_partitions.groovy @@ -0,0 +1,73 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("test_insert_with_many_parititions") { + def tableName = 'test_insert_with_many_parititions' + + def makePartitionsStmt = { partitionNum -> + def stmt = "" + for (int i = 0; i < partitionNum; i++) { + if (i == (partitionNum - 1)) { + stmt += """partition `p${i}` values [("${i}"), ("${i + 1}"))""" + } else { + stmt += """partition `p${i}` values [("${i}"), ("${i + 1}")),\n\t\t\t""" + } + } + return stmt + } + + // [start, end) + def makeInsertStmt = { start, end -> + def stmt = "" + for (int i = start; i < end; i++) { + if (i == (end - 1)) { + stmt += """(${i}, false, 1, null, "c5", "2023-09-01", null, "c8", 1.1234, "c10")\n""" + } else { + stmt += """(${i}, false, 1, null, "c5", "2023-09-01", null, "c8", 1.1234, "c10"),\n""" + } + } + return stmt + } + + sql """ DROP TABLE IF EXISTS ${tableName} FORCE""" + sql """ CREATE TABLE IF NOT EXISTS ${tableName} ( + `c1` bigint(20) not null, + `c2` boolean REPLACE_IF_NOT_NULL null, + `c3` tinyint(4) REPLACE_IF_NOT_NULL null, + `c4` decimalv3(9, 3) REPLACE_IF_NOT_NULL null, + `c5` char(36) REPLACE_IF_NOT_NULL null, + `c6` date REPLACE_IF_NOT_NULL null, + `c7` datetime REPLACE_IF_NOT_NULL null, + `c8` varchar(64) REPLACE_IF_NOT_NULL null, + `c9` double REPLACE_IF_NOT_NULL null, + `c10` string REPLACE_IF_NOT_NULL null + ) engine=olap + AGGREGATE KEY(`c1`) + PARTITION BY RANGE(`c1`) + ( + ${makePartitionsStmt(1000)} + ) + DISTRIBUTED BY HASH(`c1`) BUCKETS 1 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + sql """ insert into ${tableName} values ${makeInsertStmt(0, 1000)};""" + qt_sql """ select count(*) from ${tableName};""" + sql """ DROP TABLE IF EXISTS ${tableName} FORCE""" +} diff --git a/regression-test/suites/large_txn_load_p1/test_insert_with_many_partitions_and_buckets.groovy b/regression-test/suites/large_txn_load_p1/test_insert_with_many_partitions_and_buckets.groovy new file mode 100644 index 00000000000000..b303ce130276b2 --- /dev/null +++ b/regression-test/suites/large_txn_load_p1/test_insert_with_many_partitions_and_buckets.groovy @@ -0,0 +1,79 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("test_insert_with_many_parititions_and_buckets") { + def tableName = 'test_insert_with_many_parititions_and_buckets' + + def makePartitionsStmt = { partitionNum -> + def stmt = "" + for (int i = 0; i < partitionNum; i++) { + if (i == (partitionNum - 1)) { + stmt += """partition `p${i}` values [("${i}"), ("${i + 1}"))""" + } else { + stmt += """partition `p${i}` values [("${i}"), ("${i + 1}")),\n\t\t\t""" + } + } + return stmt + } + + // [start, end) + def makeInsertStmt = { start, end -> + def stmt = "" + for (int i = start; i < end; i++) { + if (i == (end - 1)) { + stmt += """(${i}, 1, false,null, "c5", "2023-09-01", null, "c8", 1.1234, "c10"),\n""" + stmt += """(${i}, 2, false, null, "c5", "2023-09-01", null, "c8", 1.1234, "c10"),\n""" + stmt += """(${i}, 3, false, null, "c5", "2023-09-01", null, "c8", 1.1234, "c10"),\n""" + stmt += """(${i}, 4, false, null, "c5", "2023-09-01", null, "c8", 1.1234, "c10")\n""" + } else { + stmt += """(${i}, 1, false, null, "c5", "2023-09-01", null, "c8", 1.1234, "c10"),\n""" + stmt += """(${i}, 2, false, null, "c5", "2023-09-01", null, "c8", 1.1234, "c10"),\n""" + stmt += """(${i}, 3, false, null, "c5", "2023-09-01", null, "c8", 1.1234, "c10"),\n""" + stmt += """(${i}, 4, false, null, "c5", "2023-09-01", null, "c8", 1.1234, "c10"),\n""" + } + } + return stmt + } + + sql """ DROP TABLE IF EXISTS ${tableName} FORCE""" + sql """ CREATE TABLE IF NOT EXISTS ${tableName} ( + `c1` bigint(20) not null, + `c2` int(6) not null, + `c3` boolean REPLACE_IF_NOT_NULL null, + `c4` decimalv3(9, 3) REPLACE_IF_NOT_NULL null, + `c5` char(36) REPLACE_IF_NOT_NULL null, + `c6` date REPLACE_IF_NOT_NULL null, + `c7` datetime REPLACE_IF_NOT_NULL null, + `c8` varchar(64) REPLACE_IF_NOT_NULL null, + `c9` double REPLACE_IF_NOT_NULL null, + `c10` string REPLACE_IF_NOT_NULL null + ) engine=olap + AGGREGATE KEY(`c1`, `c2`) + PARTITION BY RANGE(`c1`) + ( + ${makePartitionsStmt(100)} + ) + DISTRIBUTED BY HASH(`c2`) BUCKETS 16 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + sql """ insert into ${tableName} values ${makeInsertStmt(0, 100)};""" + qt_sql """ select count(*) from ${tableName};""" + sql """ DROP TABLE IF EXISTS ${tableName} FORCE;""" +} From 8510dfb67f3e01b822478fecf32b3cd833965bab Mon Sep 17 00:00:00 2001 From: Lei Zhang <27994433+SWJTU-ZhangLei@users.noreply.github.com> Date: Thu, 22 Aug 2024 16:11:42 +0800 Subject: [PATCH 2/5] fuzzy test --- cloud/src/common/config.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cloud/src/common/config.h b/cloud/src/common/config.h index 7a6dc04d95c172..04ac9cdfeec849 100644 --- a/cloud/src/common/config.h +++ b/cloud/src/common/config.h @@ -206,7 +206,7 @@ CONF_Validator(s3_client_http_scheme, [](const std::string& config) -> bool { // Max retry times for object storage request CONF_mInt64(max_s3_client_retry, "10"); -CONF_Bool(enable_txn_lazy_commit, "false"); +CONF_Bool(enable_txn_lazy_commit, "true"); CONF_Int32(txn_lazy_commit_rowsets_thresold, "2048"); CONF_Int32(txn_lazy_commit_num_threads, "8"); CONF_Int32(txn_lazy_max_rowsets_per_batch, "2048"); From 7d9fae3422e6f05bd0c63094fb95df27fc4ab5a7 Mon Sep 17 00:00:00 2001 From: Lei Zhang <27994433+SWJTU-ZhangLei@users.noreply.github.com> Date: Tue, 27 Aug 2024 11:05:25 +0800 Subject: [PATCH 3/5] fix some comments 20240827 --- be/src/cloud/cloud_meta_mgr.cpp | 2 +- be/src/cloud/config.cpp | 2 +- be/src/cloud/config.h | 2 +- cloud/src/common/config.h | 9 +- cloud/src/meta-service/meta_service.cpp | 51 +++++------ cloud/src/meta-service/meta_service_txn.cpp | 85 ++++++++++--------- cloud/src/meta-service/txn_lazy_committer.cpp | 38 ++++++--- cloud/src/recycler/recycler.cpp | 5 +- cloud/test/txn_lazy_commit_test.cpp | 4 +- .../java/org/apache/doris/common/Config.java | 2 +- .../CloudGlobalTransactionMgr.java | 4 +- gensrc/proto/cloud.proto | 5 +- 12 files changed, 114 insertions(+), 95 deletions(-) diff --git a/be/src/cloud/cloud_meta_mgr.cpp b/be/src/cloud/cloud_meta_mgr.cpp index 8b9b5a6360f1a2..bc6c3a0f098e7e 100644 --- a/be/src/cloud/cloud_meta_mgr.cpp +++ b/be/src/cloud/cloud_meta_mgr.cpp @@ -847,7 +847,7 @@ Status CloudMetaMgr::commit_txn(const StreamLoadContext& ctx, bool is_2pc) { req.set_db_id(ctx.db_id); req.set_txn_id(ctx.txn_id); req.set_is_2pc(is_2pc); - req.set_lazy_commit(config::cloud_txn_lazy_commit); + req.set_enable_txn_lazy_commit(config::enable_cloud_txn_lazy_commit); auto st = retry_rpc("commit txn", req, &res, &MetaService_Stub::commit_txn); if (st.ok()) { diff --git a/be/src/cloud/config.cpp b/be/src/cloud/config.cpp index c97a1da4b29db3..1b8256bf932135 100644 --- a/be/src/cloud/config.cpp +++ b/be/src/cloud/config.cpp @@ -62,6 +62,6 @@ DEFINE_mInt32(sync_load_for_tablets_thread, "32"); DEFINE_mBool(enable_new_tablet_do_compaction, "false"); -DEFINE_Bool(cloud_txn_lazy_commit, "false"); +DEFINE_Bool(enable_cloud_txn_lazy_commit, "false"); } // namespace doris::config diff --git a/be/src/cloud/config.h b/be/src/cloud/config.h index 8c6d2b441f858c..104ead04996dd2 100644 --- a/be/src/cloud/config.h +++ b/be/src/cloud/config.h @@ -96,6 +96,6 @@ DECLARE_mBool(save_load_error_log_to_s3); DECLARE_mInt32(sync_load_for_tablets_thread); // enable large txn lazy commit in meta-service `commit_txn` -DECLARE_mBool(cloud_txn_lazy_commit); +DECLARE_mBool(enable_cloud_txn_lazy_commit); } // namespace doris::config diff --git a/cloud/src/common/config.h b/cloud/src/common/config.h index 04ac9cdfeec849..d401caa4ad975f 100644 --- a/cloud/src/common/config.h +++ b/cloud/src/common/config.h @@ -206,8 +206,11 @@ CONF_Validator(s3_client_http_scheme, [](const std::string& config) -> bool { // Max retry times for object storage request CONF_mInt64(max_s3_client_retry, "10"); -CONF_Bool(enable_txn_lazy_commit, "true"); -CONF_Int32(txn_lazy_commit_rowsets_thresold, "2048"); +CONF_Bool(enable_cloud_txn_lazy_commit, "true"); +CONF_Int32(txn_lazy_commit_rowsets_thresold, "1000"); CONF_Int32(txn_lazy_commit_num_threads, "8"); -CONF_Int32(txn_lazy_max_rowsets_per_batch, "2048"); +CONF_Int32(txn_lazy_max_rowsets_per_batch, "1000"); + +// max TabletIndexPB num for batch get +CONF_Int32(max_tablet_index_num_per_batch, "1000"); } // namespace doris::cloud::config diff --git a/cloud/src/meta-service/meta_service.cpp b/cloud/src/meta-service/meta_service.cpp index 74fe5a562ba01d..12c13e2a183254 100644 --- a/cloud/src/meta-service/meta_service.cpp +++ b/cloud/src/meta-service/meta_service.cpp @@ -267,18 +267,10 @@ void MetaServiceImpl::get_version(::google::protobuf::RpcController* controller, msg = "malformed version value"; return; } - - if (version_pb.has_txn_id()) { - txn.reset(); - std::shared_ptr task = - txn_lazy_committer_->submit(instance_id, version_pb.txn_id()); - - std::tie(code, msg) = task->wait(); - if (code != MetaServiceCode::OK) { - LOG(WARNING) << "wait txn lazy commit failed, txn_id=" << version_pb.txn_id() - << " code=" << code << " msg=" << msg; - return; - } + if (!version_pb.has_version()) { + msg = "not found"; + code = MetaServiceCode::VERSION_NOT_FOUND; + return; } response->set_version(version_pb.version()); @@ -407,22 +399,15 @@ void MetaServiceImpl::batch_get_version(::google::protobuf::RpcController* contr msg = "malformed version value"; break; } - if (version_pb.has_txn_id()) { - txn.reset(); - std::shared_ptr task = - txn_lazy_committer_->submit(instance_id, version_pb.txn_id()); - std::tie(code, msg) = task->wait(); - if (code != MetaServiceCode::OK) { - LOG(WARNING) << "wait txn lazy commit failed, txn_id=" - << version_pb.txn_id(); - response->clear_partition_ids(); - response->clear_table_ids(); - response->clear_versions(); - return; - } + + if (!version_pb.has_version()) { + // return -1 if the target version is not exists. + response->add_versions(-1); + response->add_version_update_time_ms(-1); + } else { + response->add_versions(version_pb.version()); + response->add_version_update_time_ms(version_pb.update_time_ms()); } - response->add_versions(version_pb.version()); - response->add_version_update_time_ms(version_pb.update_time_ms()); } } } @@ -1473,6 +1458,8 @@ void MetaServiceImpl::get_rowset(::google::protobuf::RpcController* controller, DCHECK(request->has_idx()); if (idx.has_db_id()) { + // there is maybe a lazy commit txn when call get_rowset + // we need advance lazy commit txn here std::string ver_val; std::string ver_key = partition_version_key( {instance_id, idx.db_id(), idx.table_id(), idx.partition_id()}); @@ -1490,21 +1477,23 @@ void MetaServiceImpl::get_rowset(::google::protobuf::RpcController* controller, VersionPB version_pb; if (!version_pb.ParseFromString(ver_val)) { code = MetaServiceCode::PROTOBUF_PARSE_ERR; - ss << "failed to parse version pb tablet_id=" << tablet_id + ss << "failed to parse version pb db_id=" << idx.db_id() + << " table_id=" << idx.table_id() << " partition_id" << idx.partition_id() << " key=" << hex(ver_key); msg = ss.str(); LOG(WARNING) << msg; return; } - if (version_pb.has_txn_id()) { + if (version_pb.txn_ids_size() > 0) { + DCHECK(version_pb.txn_ids_size() == 1); txn.reset(); std::shared_ptr task = - txn_lazy_committer_->submit(instance_id, version_pb.txn_id()); + txn_lazy_committer_->submit(instance_id, version_pb.txn_ids(0)); std::tie(code, msg) = task->wait(); if (code != MetaServiceCode::OK) { - LOG(WARNING) << "advance_last_txn failed last_txn=" << version_pb.txn_id() + LOG(WARNING) << "advance_last_txn failed last_txn=" << version_pb.txn_ids(0) << " code=" << code << "msg=" << msg; return; } diff --git a/cloud/src/meta-service/meta_service_txn.cpp b/cloud/src/meta-service/meta_service_txn.cpp index 8a73b941f42d11..802571d93c3b71 100644 --- a/cloud/src/meta-service/meta_service_txn.cpp +++ b/cloud/src/meta-service/meta_service_txn.cpp @@ -916,9 +916,8 @@ void commit_txn_immediately( std::vector>& tmp_rowsets_meta) { std::stringstream ss; int64_t txn_id = request->txn_id(); - bool need_advance_last_txn = false; - int64_t last_pending_txn_id = 0; do { + int64_t last_pending_txn_id = 0; std::unique_ptr txn; TxnErrorCode err = txn_kv->create_txn(&txn); if (err != TxnErrorCode::TXN_OK) { @@ -1079,9 +1078,10 @@ void commit_txn_immediately( msg = ss.str(); return; } - if (version_pb.has_txn_id()) { - need_advance_last_txn = true; - last_pending_txn_id = version_pb.txn_id(); + if (version_pb.txn_ids_size() > 0) { + DCHECK(version_pb.txn_ids_size() == 1); + last_pending_txn_id = version_pb.txn_ids(0); + DCHECK(last_pending_txn_id > 0); break; } version = version_pb.version(); @@ -1089,26 +1089,23 @@ void commit_txn_immediately( version = 1; } new_versions[version_keys[i]] = version + 1; - need_advance_last_txn = false; last_pending_txn_id = 0; } version_keys.clear(); version_values.clear(); - if (need_advance_last_txn) { + if (last_pending_txn_id > 0) { txn.reset(); - DCHECK(last_pending_txn_id > 0); std::shared_ptr task = txn_lazy_committer->submit(instance_id, last_pending_txn_id); std::tie(code, msg) = task->wait(); - need_advance_last_txn = false; - last_pending_txn_id = 0; if (code != MetaServiceCode::OK) { LOG(WARNING) << "advance_last_txn failed last_txn=" << last_pending_txn_id << " code=" << code << "msg=" << msg; return; } + last_pending_txn_id = 0; continue; } @@ -1428,6 +1425,8 @@ void get_tablet_indexes( tablet_idx_values.clear(); } +// rewrite TabletIndexPB for fill db_id, in case of historical reasons +// TabletIndexPB missing db_id void repair_tablet_index( std::shared_ptr& txn_kv, MetaServiceCode& code, std::string& msg, const std::string& instance_id, int64_t db_id, int64_t txn_id, @@ -1438,12 +1437,13 @@ void repair_tablet_index( tablet_idx_keys.push_back(meta_tablet_idx_key({instance_id, i.tablet_id()})); } -#define MAX_TABLET_INDEX_NUM 2048 - for (size_t i = 0; i < tablet_idx_keys.size(); i += MAX_TABLET_INDEX_NUM) { - size_t end = (i + MAX_TABLET_INDEX_NUM) > tablet_idx_keys.size() ? tablet_idx_keys.size() - : i + MAX_TABLET_INDEX_NUM; + for (size_t i = 0; i < tablet_idx_keys.size(); i += config::max_tablet_index_num_per_batch) { + size_t end = (i + config::max_tablet_index_num_per_batch) > tablet_idx_keys.size() + ? tablet_idx_keys.size() + : i + config::max_tablet_index_num_per_batch; const std::vector sub_tablet_idx_keys(tablet_idx_keys.begin() + i, tablet_idx_keys.begin() + end); + std::unique_ptr txn; TxnErrorCode err = txn_kv->create_txn(&txn); if (err != TxnErrorCode::TXN_OK) { @@ -1456,7 +1456,7 @@ void repair_tablet_index( std::vector> tablet_idx_values; // batch get snapshot is false - err = txn->batch_get(&tablet_idx_values, tablet_idx_keys, + err = txn->batch_get(&tablet_idx_values, sub_tablet_idx_keys, Transaction::BatchGetOptions(false)); if (err != TxnErrorCode::TXN_OK) { code = cast_as(err); @@ -1465,6 +1465,7 @@ void repair_tablet_index( LOG(WARNING) << msg << " txn_id=" << txn_id; return; } + DCHECK(tablet_idx_values.size() <= config::max_tablet_index_num_per_batch); for (size_t j = 0; j < sub_tablet_idx_keys.size(); j++) { if (!tablet_idx_values[j].has_value()) [[unlikely]] { @@ -1498,6 +1499,9 @@ void repair_tablet_index( return; } txn->put(sub_tablet_idx_keys[j], idx_val); + LOG(INFO) << " repaire tablet index txn_id=" << txn_id + << " tablet_idx_pb:" << tablet_idx_pb.ShortDebugString() + << " key=" << hex(sub_tablet_idx_keys[j]); } } @@ -1520,10 +1524,9 @@ void commit_txn_eventually( std::stringstream ss; TxnErrorCode err = TxnErrorCode::TXN_OK; int64_t txn_id = request->txn_id(); - bool need_advance_last_txn = false; - int64_t last_pending_txn_id = 0; do { + int64_t last_pending_txn_id = 0; std::unique_ptr txn; err = txn_kv->create_txn(&txn); if (err != TxnErrorCode::TXN_OK) { @@ -1556,6 +1559,7 @@ void commit_txn_eventually( continue; } + // std::unordered_map new_versions; std::vector version_keys; for (auto& [_, i] : tmp_rowsets_meta) { @@ -1591,35 +1595,33 @@ void commit_txn_eventually( msg = ss.str(); return; } - if (version_pb.has_txn_id()) { - need_advance_last_txn = true; - last_pending_txn_id = version_pb.txn_id(); + if (version_pb.txn_ids_size() > 0) { + DCHECK(version_pb.txn_ids_size() == 1); + last_pending_txn_id = version_pb.txn_ids(0); + DCHECK(last_pending_txn_id > 0); break; } version = version_pb.version(); } else { version = 1; } - new_versions[version_keys[i]] = version + 1; - need_advance_last_txn = false; + new_versions[version_keys[i]] = version; last_pending_txn_id = 0; } - if (need_advance_last_txn) { + if (last_pending_txn_id > 0) { txn.reset(); - DCHECK(last_pending_txn_id > 0); std::shared_ptr task = txn_lazy_committer->submit(instance_id, last_pending_txn_id); std::tie(code, msg) = task->wait(); - need_advance_last_txn = false; - last_pending_txn_id = 0; if (code != MetaServiceCode::OK) { LOG(WARNING) << "advance_last_txn failed last_txn=" << last_pending_txn_id << " code=" << code << "msg=" << msg; return; } + last_pending_txn_id = 0; // there maybe concurrent commit_txn_eventually, so we need continue to make sure // partition versionPB has no txn_id continue; @@ -1646,6 +1648,7 @@ void commit_txn_eventually( LOG(WARNING) << msg; return; } + LOG(INFO) << "txn_id=" << txn_id << " txn_info=" << txn_info.ShortDebugString(); DCHECK(txn_info.txn_id() == txn_id); if (txn_info.status() == TxnStatusPB::TXN_STATUS_ABORTED) { @@ -1682,7 +1685,6 @@ void commit_txn_eventually( LOG(WARNING) << msg; return; } - LOG(INFO) << "txn_id=" << txn_id << " txn_info=" << txn_info.ShortDebugString(); auto now_time = system_clock::now(); uint64_t commit_time = duration_cast(now_time.time_since_epoch()).count(); @@ -1699,10 +1701,12 @@ void commit_txn_eventually( txn_info.mutable_commit_attachment()->CopyFrom(request->commit_attachment()); } DCHECK(txn_info.status() != TxnStatusPB::TXN_STATUS_COMMITTED); - // lazy commit set status TXN_STATUS_COMMITTED not TXN_STATUS_VISIBLE !!! + // set status TXN_STATUS_COMMITTED not TXN_STATUS_VISIBLE !!! + // lazy commit task will advance txn to make txn visible txn_info.set_status(TxnStatusPB::TXN_STATUS_COMMITTED); - LOG(INFO) << "after update txn_info=" << txn_info.ShortDebugString(); + LOG(INFO) << "after update txn_id= " << txn_id + << " txn_info=" << txn_info.ShortDebugString(); info_val.clear(); if (!txn_info.SerializeToString(&info_val)) { code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; @@ -1712,7 +1716,7 @@ void commit_txn_eventually( } txn->put(info_key, info_val); - LOG(INFO) << "xxx put info_key=" << hex(info_key) << " txn_id=" << txn_id; + LOG(INFO) << "put info_key=" << hex(info_key) << " txn_id=" << txn_id; if (txn_info.load_job_source_type() == LoadJobSourceTypePB::LOAD_JOB_SRC_TYPE_ROUTINE_LOAD_TASK) { @@ -1726,18 +1730,22 @@ void commit_txn_eventually( for (auto& i : new_versions) { std::string ver_val; VersionPB version_pb; - version_pb.set_version(i.second); - version_pb.set_txn_id(txn_id); + version_pb.add_txn_ids(txn_id); version_pb.set_update_time_ms(version_update_time_ms); + if (i.second > 1) { + version_pb.set_version(i.second); + } + if (!version_pb.SerializeToString(&ver_val)) { code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; - ss << "failed to serialize version_pb when saving, txn_id=" << txn_id; + ss << "failed to serialize version_pb when saving, txn_id=" << txn_id + << " partiton_key=" << hex(i.first); msg = ss.str(); return; } txn->put(i.first, ver_val); - LOG(INFO) << "xxx put partition_version_key=" << hex(i.first) << " version:" << i.second + LOG(INFO) << "put partition_version_key=" << hex(i.first) << " version:" << i.second << " txn_id=" << txn_id << " update_time=" << version_update_time_ms; std::string_view ver_key = i.first; @@ -1761,7 +1769,7 @@ void commit_txn_eventually( response->add_table_ids(table_id); response->add_partition_ids(partition_id); - response->add_versions(i.second); + response->add_versions(i.second + 1); } // process mow table, check lock and remove pending key @@ -1844,7 +1852,7 @@ void commit_txn_eventually( std::shared_ptr task = txn_lazy_committer->submit(instance_id, txn_id); std::pair ret = task->wait(); if (ret.first != MetaServiceCode::OK) { - LOG(WARNING) << "lazy commit txn failed txn_id=" << txn_id << " code=" << ret.first + LOG(WARNING) << "txn lazy commit failed txn_id=" << txn_id << " code=" << ret.first << "msg=" << ret.second; } @@ -1874,6 +1882,7 @@ void commit_txn_eventually( << " updated_row_count=" << stats_pb->updated_row_count(); } + // txn set visible for fe callback txn_info.set_status(TxnStatusPB::TXN_STATUS_VISIBLE); response->mutable_txn_info()->CopyFrom(txn_info); break; @@ -2503,8 +2512,8 @@ void MetaServiceImpl::commit_txn(::google::protobuf::RpcController* controller, return; } - if (request->has_is_2pc() && !request->is_2pc() && request->has_lazy_commit() && - request->lazy_commit() && config::enable_txn_lazy_commit && + if (request->has_is_2pc() && !request->is_2pc() && request->has_enable_txn_lazy_commit() && + request->enable_txn_lazy_commit() && config::enable_cloud_txn_lazy_commit && (tmp_rowsets_meta.size() >= config::txn_lazy_commit_rowsets_thresold)) { LOG(INFO) << "txn_id=" << txn_id << " commit_txn_eventually" << " size=" << tmp_rowsets_meta.size(); diff --git a/cloud/src/meta-service/txn_lazy_committer.cpp b/cloud/src/meta-service/txn_lazy_committer.cpp index 23eb61bc9b929f..05eebef229a51a 100644 --- a/cloud/src/meta-service/txn_lazy_committer.cpp +++ b/cloud/src/meta-service/txn_lazy_committer.cpp @@ -127,18 +127,20 @@ void convert_tmp_rowsets( } LOG(INFO) << "txn_id=" << txn_id << " key=" << hex(ver_key) << " version_pb:" << version_pb.ShortDebugString(); - DCHECK(version_pb.has_txn_id()); - DCHECK(version_pb.txn_id() == txn_id); partition_versions.emplace(tmp_rowset_pb.partition_id(), version_pb); } const VersionPB& version_pb = partition_versions[tmp_rowset_pb.partition_id()]; + DCHECK(version_pb.txn_ids_size() > 0); + DCHECK_EQ(version_pb.txn_ids(0), txn_id); - std::string rowset_key = - meta_rowset_key({instance_id, tmp_rowset_pb.tablet_id(), version_pb.version()}); + int64_t version = version_pb.has_version() ? (version_pb.version() + 1) : 2; + + std::string rowset_key = meta_rowset_key({instance_id, tmp_rowset_pb.tablet_id(), version}); std::string rowset_val; err = txn->get(rowset_key, &rowset_val); if (TxnErrorCode::TXN_OK == err) { + // tmp rowset key has been converted continue; } @@ -152,8 +154,8 @@ void convert_tmp_rowsets( } DCHECK(err == TxnErrorCode::TXN_KEY_NOT_FOUND); - tmp_rowset_pb.set_start_version(version_pb.version()); - tmp_rowset_pb.set_end_version(version_pb.version()); + tmp_rowset_pb.set_start_version(version); + tmp_rowset_pb.set_end_version(version); rowset_val.clear(); if (!tmp_rowset_pb.SerializeToString(&rowset_val)) { @@ -165,7 +167,7 @@ void convert_tmp_rowsets( } txn->put(rowset_key, rowset_val); - LOG(INFO) << "xxx put rowset_key=" << hex(rowset_key) << " txn_id=" << txn_id + LOG(INFO) << "put rowset_key=" << hex(rowset_key) << " txn_id=" << txn_id << " rowset_size=" << rowset_key.size() + rowset_val.size(); // Accumulate affected rows @@ -248,10 +250,10 @@ void make_committed_txn_visible(const std::string& instance_id, int64_t db_id, i return; } txn->put(info_key, info_val); - LOG(INFO) << "xxx put info_key=" << hex(info_key) << " txn_id=" << txn_id; + LOG(INFO) << "put info_key=" << hex(info_key) << " txn_id=" << txn_id; const std::string running_key = txn_running_key({instance_id, db_id, txn_id}); - LOG(INFO) << "xxx remove running_key=" << hex(running_key) << " txn_id=" << txn_id; + LOG(INFO) << "remove running_key=" << hex(running_key) << " txn_id=" << txn_id; txn->remove(running_key); std::string recycle_val; @@ -268,7 +270,9 @@ void make_committed_txn_visible(const std::string& instance_id, int64_t db_id, i msg = ss.str(); return; } + txn->put(recycle_key, recycle_val); + LOG(INFO) << "put recycle_key=" << hex(recycle_key) << " txn_id=" << txn_id; err = txn->commit(); if (err != TxnErrorCode::TXN_OK) { @@ -359,6 +363,8 @@ void TxnLazyCommitTask::commit() { table_id = tablet_ids[tmp_rowset_pb.tablet_id()].table_id(); } txn->remove(tmp_rowset_key); + LOG(INFO) << "remove tmp_rowset_key=" << hex(tmp_rowset_key) + << " txn_id=" << txn_id_; } DCHECK(table_id > 0); @@ -387,9 +393,17 @@ void TxnLazyCommitTask::commit() { break; } - if (version_pb.has_txn_id() && version_pb.txn_id() == txn_id_) { - version_pb.clear_txn_id(); + if (version_pb.txn_ids_size() > 0 && version_pb.txn_ids(0) == txn_id_) { + DCHECK(version_pb.txn_ids_size() == 1); + version_pb.clear_txn_ids(); ver_val.clear(); + + if (version_pb.has_version()) { + version_pb.set_version(version_pb.version() + 1); + } else { + // first commit txn version is 2 + version_pb.set_version(2); + } if (!version_pb.SerializeToString(&ver_val)) { code_ = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; ss << "failed to serialize version_pb when saving, txn_id=" << txn_id_; @@ -397,6 +411,8 @@ void TxnLazyCommitTask::commit() { return; } txn->put(ver_key, ver_val); + LOG(INFO) << "put ver_key=" << hex(ver_key) << " txn_id=" << txn_id_ + << " version_pb=" << version_pb.ShortDebugString(); err = txn->commit(); if (err != TxnErrorCode::TXN_OK) { diff --git a/cloud/src/recycler/recycler.cpp b/cloud/src/recycler/recycler.cpp index df60a827c0afcf..aee30a46ca5c00 100644 --- a/cloud/src/recycler/recycler.cpp +++ b/cloud/src/recycler/recycler.cpp @@ -891,12 +891,13 @@ bool check_lazy_txn_finished(std::shared_ptr txn_kv, const std::string in return false; } - if (version_pb.has_txn_id()) { + if (version_pb.txn_ids_size() > 0) { + DCHECK(version_pb.txn_ids_size() == 1); LOG(WARNING) << "lazy txn not finished, instance_id=" << instance_id << " db_id=" << tablet_idx_pb.db_id() << " table_id=" << tablet_idx_pb.table_id() << " partition_id=" << tablet_idx_pb.partition_id() - << " tablet_id=" << tablet_id << " txn_id=" << version_pb.txn_id() + << " tablet_id=" << tablet_id << " txn_id=" << version_pb.txn_ids(0) << " key=" << hex(ver_key); return false; } diff --git a/cloud/test/txn_lazy_commit_test.cpp b/cloud/test/txn_lazy_commit_test.cpp index 2fb842186fba13..6b4518c09ab6c4 100644 --- a/cloud/test/txn_lazy_commit_test.cpp +++ b/cloud/test/txn_lazy_commit_test.cpp @@ -51,7 +51,7 @@ int main(int argc, char** argv) { return -1; } - config::enable_txn_lazy_commit = true; + config::enable_cloud_txn_lazy_commit = true; config::txn_lazy_commit_rowsets_thresold = 1; config::txn_lazy_max_rowsets_per_batch = 1; config::txn_lazy_commit_num_threads = 2; @@ -190,7 +190,7 @@ TEST(TxnLazyCommitTest, CommitTxnEventuallyTest) { req.set_db_id(db_id); req.set_txn_id(txn_id); req.set_is_2pc(false); - req.set_lazy_commit(true); + req.set_enable_txn_lazy_commit(true); CommitTxnResponse res; meta_service->commit_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, &res, nullptr); diff --git a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java index c5698fecb4400e..2eaf6bff7dce64 100644 --- a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java +++ b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java @@ -3019,7 +3019,7 @@ public static int metaServiceRpcRetryTimes() { public static int try_commit_lock_timeout_seconds = 5; @ConfField(mutable = true, description = {"存算分离模式下是否开启大事务提交,默认false"}) - public static boolean cloud_txn_lazy_commit = false; + public static boolean enable_cloud_txn_lazy_commit = false; // ATTN: DONOT add any config not related to cloud mode here // ATTN: DONOT add any config not related to cloud mode here diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/CloudGlobalTransactionMgr.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/CloudGlobalTransactionMgr.java index 03057735cddbb5..f51454ad269c51 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/CloudGlobalTransactionMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/CloudGlobalTransactionMgr.java @@ -473,7 +473,7 @@ private void commitTransaction(long dbId, List
tableList, long transactio .setIs2Pc(is2PC) .setCloudUniqueId(Config.cloud_unique_id) .addAllBaseTabletIds(getBaseTabletsFromTables(tableList, tabletCommitInfos)) - .setLazyCommit(Config.cloud_txn_lazy_commit); + .setEnableTxnLazyCommit(Config.enable_cloud_txn_lazy_commit); // if tablet commit info is empty, no need to pass mowTableList to meta service. if (tabletCommitInfos != null && !tabletCommitInfos.isEmpty()) { @@ -935,7 +935,7 @@ public boolean commitAndPublishTransaction(DatabaseIf db, long transactionId, .setIs2Pc(false) .setCloudUniqueId(Config.cloud_unique_id) .setIsTxnLoad(true) - .setLazyCommit(Config.cloud_txn_lazy_commit); + .setEnableTxnLazyCommit(Config.enable_cloud_txn_lazy_commit); // add sub txn infos for (SubTransactionState subTransactionState : subTransactionStates) { builder.addSubTxnInfos(SubTxnInfo.newBuilder().setSubTxnId(subTransactionState.getSubTransactionId()) diff --git a/gensrc/proto/cloud.proto b/gensrc/proto/cloud.proto index 8035a3a96a3675..017d0509af3bb4 100644 --- a/gensrc/proto/cloud.proto +++ b/gensrc/proto/cloud.proto @@ -406,7 +406,8 @@ message TxnRunningPB { message VersionPB { optional int64 version = 1; optional int64 update_time_ms = 2; - optional int64 txn_id = 3; + // now only allow one element + repeated int64 txn_ids = 3; } message RecycleTxnPB { @@ -668,7 +669,7 @@ message CommitTxnRequest { // for transaction load optional bool is_txn_load = 9; repeated SubTxnInfo sub_txn_infos = 10; - optional bool lazy_commit = 11; + optional bool enable_txn_lazy_commit = 11; } message SubTxnInfo { From 83bd1832cc7d1e4976fe100e8d5b101099e20ce7 Mon Sep 17 00:00:00 2001 From: Lei Zhang <27994433+SWJTU-ZhangLei@users.noreply.github.com> Date: Tue, 27 Aug 2024 22:53:24 +0800 Subject: [PATCH 4/5] fuzzy test --- be/src/cloud/config.cpp | 2 +- cloud/src/common/config.h | 6 +++--- .../src/main/java/org/apache/doris/common/Config.java | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/be/src/cloud/config.cpp b/be/src/cloud/config.cpp index 1b8256bf932135..8318a7b6d9c0d9 100644 --- a/be/src/cloud/config.cpp +++ b/be/src/cloud/config.cpp @@ -62,6 +62,6 @@ DEFINE_mInt32(sync_load_for_tablets_thread, "32"); DEFINE_mBool(enable_new_tablet_do_compaction, "false"); -DEFINE_Bool(enable_cloud_txn_lazy_commit, "false"); +DEFINE_Bool(enable_cloud_txn_lazy_commit, "true"); } // namespace doris::config diff --git a/cloud/src/common/config.h b/cloud/src/common/config.h index d401caa4ad975f..e9394959be8429 100644 --- a/cloud/src/common/config.h +++ b/cloud/src/common/config.h @@ -207,10 +207,10 @@ CONF_Validator(s3_client_http_scheme, [](const std::string& config) -> bool { CONF_mInt64(max_s3_client_retry, "10"); CONF_Bool(enable_cloud_txn_lazy_commit, "true"); -CONF_Int32(txn_lazy_commit_rowsets_thresold, "1000"); +CONF_Int32(txn_lazy_commit_rowsets_thresold, "2"); CONF_Int32(txn_lazy_commit_num_threads, "8"); -CONF_Int32(txn_lazy_max_rowsets_per_batch, "1000"); +CONF_Int32(txn_lazy_max_rowsets_per_batch, "2"); // max TabletIndexPB num for batch get -CONF_Int32(max_tablet_index_num_per_batch, "1000"); +CONF_Int32(max_tablet_index_num_per_batch, "2"); } // namespace doris::cloud::config diff --git a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java index 2eaf6bff7dce64..eb5857d5e4db7b 100644 --- a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java +++ b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java @@ -3019,7 +3019,7 @@ public static int metaServiceRpcRetryTimes() { public static int try_commit_lock_timeout_seconds = 5; @ConfField(mutable = true, description = {"存算分离模式下是否开启大事务提交,默认false"}) - public static boolean enable_cloud_txn_lazy_commit = false; + public static boolean enable_cloud_txn_lazy_commit = true; // ATTN: DONOT add any config not related to cloud mode here // ATTN: DONOT add any config not related to cloud mode here From 0f6412f7fa9a90fc50e6af06a04d4e9b42655491 Mon Sep 17 00:00:00 2001 From: Lei Zhang <27994433+SWJTU-ZhangLei@users.noreply.github.com> Date: Wed, 28 Aug 2024 11:39:37 +0800 Subject: [PATCH 5/5] rename some field and function --- be/src/cloud/config.cpp | 2 +- cloud/src/common/config.h | 6 +++--- cloud/src/meta-service/meta_service.cpp | 11 ++++++----- cloud/src/meta-service/meta_service_txn.cpp | 14 +++++++------- cloud/src/meta-service/txn_lazy_committer.cpp | 11 ++++++----- cloud/src/recycler/recycler.cpp | 12 ++++++------ cloud/src/recycler/recycler.h | 2 +- cloud/test/recycler_test.cpp | 12 ++++++------ .../main/java/org/apache/doris/common/Config.java | 2 +- gensrc/proto/cloud.proto | 2 +- 10 files changed, 38 insertions(+), 36 deletions(-) diff --git a/be/src/cloud/config.cpp b/be/src/cloud/config.cpp index 8318a7b6d9c0d9..1b8256bf932135 100644 --- a/be/src/cloud/config.cpp +++ b/be/src/cloud/config.cpp @@ -62,6 +62,6 @@ DEFINE_mInt32(sync_load_for_tablets_thread, "32"); DEFINE_mBool(enable_new_tablet_do_compaction, "false"); -DEFINE_Bool(enable_cloud_txn_lazy_commit, "true"); +DEFINE_Bool(enable_cloud_txn_lazy_commit, "false"); } // namespace doris::config diff --git a/cloud/src/common/config.h b/cloud/src/common/config.h index e9394959be8429..d401caa4ad975f 100644 --- a/cloud/src/common/config.h +++ b/cloud/src/common/config.h @@ -207,10 +207,10 @@ CONF_Validator(s3_client_http_scheme, [](const std::string& config) -> bool { CONF_mInt64(max_s3_client_retry, "10"); CONF_Bool(enable_cloud_txn_lazy_commit, "true"); -CONF_Int32(txn_lazy_commit_rowsets_thresold, "2"); +CONF_Int32(txn_lazy_commit_rowsets_thresold, "1000"); CONF_Int32(txn_lazy_commit_num_threads, "8"); -CONF_Int32(txn_lazy_max_rowsets_per_batch, "2"); +CONF_Int32(txn_lazy_max_rowsets_per_batch, "1000"); // max TabletIndexPB num for batch get -CONF_Int32(max_tablet_index_num_per_batch, "2"); +CONF_Int32(max_tablet_index_num_per_batch, "1000"); } // namespace doris::cloud::config diff --git a/cloud/src/meta-service/meta_service.cpp b/cloud/src/meta-service/meta_service.cpp index 12c13e2a183254..84db38bfce1310 100644 --- a/cloud/src/meta-service/meta_service.cpp +++ b/cloud/src/meta-service/meta_service.cpp @@ -1485,16 +1485,17 @@ void MetaServiceImpl::get_rowset(::google::protobuf::RpcController* controller, return; } - if (version_pb.txn_ids_size() > 0) { - DCHECK(version_pb.txn_ids_size() == 1); + if (version_pb.pending_txn_ids_size() > 0) { + DCHECK(version_pb.pending_txn_ids_size() == 1); txn.reset(); std::shared_ptr task = - txn_lazy_committer_->submit(instance_id, version_pb.txn_ids(0)); + txn_lazy_committer_->submit(instance_id, version_pb.pending_txn_ids(0)); std::tie(code, msg) = task->wait(); if (code != MetaServiceCode::OK) { - LOG(WARNING) << "advance_last_txn failed last_txn=" << version_pb.txn_ids(0) - << " code=" << code << "msg=" << msg; + LOG(WARNING) << "advance_last_txn failed last_txn=" + << version_pb.pending_txn_ids(0) << " code=" << code + << "msg=" << msg; return; } continue; diff --git a/cloud/src/meta-service/meta_service_txn.cpp b/cloud/src/meta-service/meta_service_txn.cpp index 802571d93c3b71..3cd1bd798bbc84 100644 --- a/cloud/src/meta-service/meta_service_txn.cpp +++ b/cloud/src/meta-service/meta_service_txn.cpp @@ -1078,9 +1078,9 @@ void commit_txn_immediately( msg = ss.str(); return; } - if (version_pb.txn_ids_size() > 0) { - DCHECK(version_pb.txn_ids_size() == 1); - last_pending_txn_id = version_pb.txn_ids(0); + if (version_pb.pending_txn_ids_size() > 0) { + DCHECK(version_pb.pending_txn_ids_size() == 1); + last_pending_txn_id = version_pb.pending_txn_ids(0); DCHECK(last_pending_txn_id > 0); break; } @@ -1595,9 +1595,9 @@ void commit_txn_eventually( msg = ss.str(); return; } - if (version_pb.txn_ids_size() > 0) { - DCHECK(version_pb.txn_ids_size() == 1); - last_pending_txn_id = version_pb.txn_ids(0); + if (version_pb.pending_txn_ids_size() > 0) { + DCHECK(version_pb.pending_txn_ids_size() == 1); + last_pending_txn_id = version_pb.pending_txn_ids(0); DCHECK(last_pending_txn_id > 0); break; } @@ -1730,7 +1730,7 @@ void commit_txn_eventually( for (auto& i : new_versions) { std::string ver_val; VersionPB version_pb; - version_pb.add_txn_ids(txn_id); + version_pb.add_pending_txn_ids(txn_id); version_pb.set_update_time_ms(version_update_time_ms); if (i.second > 1) { version_pb.set_version(i.second); diff --git a/cloud/src/meta-service/txn_lazy_committer.cpp b/cloud/src/meta-service/txn_lazy_committer.cpp index 05eebef229a51a..5273a5c9683196 100644 --- a/cloud/src/meta-service/txn_lazy_committer.cpp +++ b/cloud/src/meta-service/txn_lazy_committer.cpp @@ -131,8 +131,8 @@ void convert_tmp_rowsets( } const VersionPB& version_pb = partition_versions[tmp_rowset_pb.partition_id()]; - DCHECK(version_pb.txn_ids_size() > 0); - DCHECK_EQ(version_pb.txn_ids(0), txn_id); + DCHECK(version_pb.pending_txn_ids_size() > 0); + DCHECK_EQ(version_pb.pending_txn_ids(0), txn_id); int64_t version = version_pb.has_version() ? (version_pb.version() + 1) : 2; @@ -393,9 +393,10 @@ void TxnLazyCommitTask::commit() { break; } - if (version_pb.txn_ids_size() > 0 && version_pb.txn_ids(0) == txn_id_) { - DCHECK(version_pb.txn_ids_size() == 1); - version_pb.clear_txn_ids(); + if (version_pb.pending_txn_ids_size() > 0 && + version_pb.pending_txn_ids(0) == txn_id_) { + DCHECK(version_pb.pending_txn_ids_size() == 1); + version_pb.clear_pending_txn_ids(); ver_val.clear(); if (version_pb.has_version()) { diff --git a/cloud/src/recycler/recycler.cpp b/cloud/src/recycler/recycler.cpp index aee30a46ca5c00..919b50358c77f5 100644 --- a/cloud/src/recycler/recycler.cpp +++ b/cloud/src/recycler/recycler.cpp @@ -593,7 +593,7 @@ int InstanceRecycler::do_recycle() { [this]() -> int { return InstanceRecycler::recycle_tmp_rowsets(); }, [this]() -> int { return InstanceRecycler::recycle_rowsets(); })) .add(task_wrapper( - [this]() { return InstanceRecycler::advance_pending_txn(); }, + [this]() { return InstanceRecycler::abort_timeout_txn(); }, [this]() { return InstanceRecycler::recycle_expired_txn_label(); })) .add(task_wrapper([this]() { return InstanceRecycler::recycle_copy_jobs(); })) .add(task_wrapper([this]() { return InstanceRecycler::recycle_stage(); })) @@ -891,13 +891,13 @@ bool check_lazy_txn_finished(std::shared_ptr txn_kv, const std::string in return false; } - if (version_pb.txn_ids_size() > 0) { - DCHECK(version_pb.txn_ids_size() == 1); + if (version_pb.pending_txn_ids_size() > 0) { + DCHECK(version_pb.pending_txn_ids_size() == 1); LOG(WARNING) << "lazy txn not finished, instance_id=" << instance_id << " db_id=" << tablet_idx_pb.db_id() << " table_id=" << tablet_idx_pb.table_id() << " partition_id=" << tablet_idx_pb.partition_id() - << " tablet_id=" << tablet_id << " txn_id=" << version_pb.txn_ids(0) + << " tablet_id=" << tablet_id << " txn_id=" << version_pb.pending_txn_ids(0) << " key=" << hex(ver_key); return false; } @@ -1998,8 +1998,8 @@ int InstanceRecycler::scan_and_recycle( return ret; } -int InstanceRecycler::advance_pending_txn() { - const std::string task_name = "advance_pending_txn"; +int InstanceRecycler::abort_timeout_txn() { + const std::string task_name = "abort_timeout_txn"; int num_scanned = 0; int num_timeout = 0; int num_abort = 0; diff --git a/cloud/src/recycler/recycler.h b/cloud/src/recycler/recycler.h index c37b8ca860b17f..91a461f474faed 100644 --- a/cloud/src/recycler/recycler.h +++ b/cloud/src/recycler/recycler.h @@ -166,7 +166,7 @@ class InstanceRecycler { // scan and abort timeout txn label // returns 0 for success otherwise error - int advance_pending_txn(); + int abort_timeout_txn(); //scan and recycle expire txn label // returns 0 for success otherwise error diff --git a/cloud/test/recycler_test.cpp b/cloud/test/recycler_test.cpp index 68dd1950ed08b0..d767c1bd8b7393 100644 --- a/cloud/test/recycler_test.cpp +++ b/cloud/test/recycler_test.cpp @@ -1270,7 +1270,7 @@ TEST(RecyclerTest, advance_pending_txn) { std::make_shared(txn_kv)); ASSERT_EQ(recycler.init(), 0); sleep(1); - ASSERT_EQ(recycler.advance_pending_txn(), 0); + ASSERT_EQ(recycler.abort_timeout_txn(), 0); TxnInfoPB txn_info_pb; get_txn_info(txn_kv, mock_instance, db_id, txn_id, txn_info_pb); ASSERT_EQ(txn_info_pb.status(), TxnStatusPB::TXN_STATUS_ABORTED); @@ -1314,7 +1314,7 @@ TEST(RecyclerTest, advance_pending_txn_and_rebegin) { std::make_shared(txn_kv)); ASSERT_EQ(recycler.init(), 0); sleep(1); - ASSERT_EQ(recycler.advance_pending_txn(), 0); + ASSERT_EQ(recycler.abort_timeout_txn(), 0); TxnInfoPB txn_info_pb; get_txn_info(txn_kv, mock_instance, db_id, txn_id, txn_info_pb); ASSERT_EQ(txn_info_pb.status(), TxnStatusPB::TXN_STATUS_ABORTED); @@ -1381,7 +1381,7 @@ TEST(RecyclerTest, recycle_expired_txn_label) { InstanceRecycler recycler(txn_kv, instance, thread_group, std::make_shared(txn_kv)); ASSERT_EQ(recycler.init(), 0); - recycler.advance_pending_txn(); + recycler.abort_timeout_txn(); TxnInfoPB txn_info_pb; ASSERT_EQ(get_txn_info(txn_kv, mock_instance, db_id, txn_id, txn_info_pb), 0); ASSERT_EQ(txn_info_pb.status(), TxnStatusPB::TXN_STATUS_PREPARED); @@ -1434,7 +1434,7 @@ TEST(RecyclerTest, recycle_expired_txn_label) { std::make_shared(txn_kv)); ASSERT_EQ(recycler.init(), 0); sleep(1); - recycler.advance_pending_txn(); + recycler.abort_timeout_txn(); TxnInfoPB txn_info_pb; get_txn_info(txn_kv, mock_instance, db_id, txn_id, txn_info_pb); ASSERT_EQ(txn_info_pb.status(), TxnStatusPB::TXN_STATUS_PREPARED); @@ -1487,7 +1487,7 @@ TEST(RecyclerTest, recycle_expired_txn_label) { std::make_shared(txn_kv)); ASSERT_EQ(recycler.init(), 0); sleep(1); - recycler.advance_pending_txn(); + recycler.abort_timeout_txn(); TxnInfoPB txn_info_pb; get_txn_info(txn_kv, mock_instance, db_id, txn_id, txn_info_pb); ASSERT_EQ(txn_info_pb.status(), TxnStatusPB::TXN_STATUS_PREPARED); @@ -1547,7 +1547,7 @@ TEST(RecyclerTest, recycle_expired_txn_label) { std::make_shared(txn_kv)); ASSERT_EQ(recycler.init(), 0); sleep(1); - recycler.advance_pending_txn(); + recycler.abort_timeout_txn(); TxnInfoPB txn_info_pb; get_txn_info(txn_kv, mock_instance, db_id, txn_id, txn_info_pb); ASSERT_EQ(txn_info_pb.status(), TxnStatusPB::TXN_STATUS_PREPARED); diff --git a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java index eb5857d5e4db7b..2eaf6bff7dce64 100644 --- a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java +++ b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java @@ -3019,7 +3019,7 @@ public static int metaServiceRpcRetryTimes() { public static int try_commit_lock_timeout_seconds = 5; @ConfField(mutable = true, description = {"存算分离模式下是否开启大事务提交,默认false"}) - public static boolean enable_cloud_txn_lazy_commit = true; + public static boolean enable_cloud_txn_lazy_commit = false; // ATTN: DONOT add any config not related to cloud mode here // ATTN: DONOT add any config not related to cloud mode here diff --git a/gensrc/proto/cloud.proto b/gensrc/proto/cloud.proto index 017d0509af3bb4..4fee41650b7fbb 100644 --- a/gensrc/proto/cloud.proto +++ b/gensrc/proto/cloud.proto @@ -407,7 +407,7 @@ message VersionPB { optional int64 version = 1; optional int64 update_time_ms = 2; // now only allow one element - repeated int64 txn_ids = 3; + repeated int64 pending_txn_ids = 3; } message RecycleTxnPB {