diff --git a/be/src/cloud/cloud_rowset_writer.cpp b/be/src/cloud/cloud_rowset_writer.cpp index 66067bb9428634..d4a9fbce7e26a7 100644 --- a/be/src/cloud/cloud_rowset_writer.cpp +++ b/be/src/cloud/cloud_rowset_writer.cpp @@ -61,6 +61,7 @@ Status CloudRowsetWriter::init(const RowsetWriterContext& rowset_writer_context) _rowset_meta->set_newest_write_timestamp(_context.newest_write_timestamp); } _rowset_meta->set_tablet_schema(_context.tablet_schema); + _rowset_meta->set_job_id(_context.job_id); _context.segment_collector = std::make_shared>(this); _context.file_writer_creator = std::make_shared>(this); if (_context.mow_context != nullptr) { diff --git a/be/src/cloud/cloud_schema_change_job.cpp b/be/src/cloud/cloud_schema_change_job.cpp index 8a4cd7bab132af..21633cb1a32135 100644 --- a/be/src/cloud/cloud_schema_change_job.cpp +++ b/be/src/cloud/cloud_schema_change_job.cpp @@ -326,6 +326,7 @@ Status CloudSchemaChangeJob::_convert_historical_rowsets(const SchemaChangeParam context.tablet_schema = _new_tablet->tablet_schema(); context.newest_write_timestamp = rs_reader->newest_write_timestamp(); context.storage_resource = _cloud_storage_engine.get_storage_resource(sc_params.vault_id); + context.job_id = _job_id; context.write_file_cache = sc_params.output_to_file_cache; context.tablet = _new_tablet; if (!context.storage_resource) { diff --git a/be/src/cloud/pb_convert.cpp b/be/src/cloud/pb_convert.cpp index e65a43e7a5cc6a..1af76d0797d635 100644 --- a/be/src/cloud/pb_convert.cpp +++ b/be/src/cloud/pb_convert.cpp @@ -105,6 +105,12 @@ void doris_rowset_meta_to_cloud(RowsetMetaCloudPB* out, const RowsetMetaPB& in) auto* slice_locations = out->mutable_packed_slice_locations(); slice_locations->clear(); slice_locations->insert(in.packed_slice_locations().begin(), in.packed_slice_locations().end()); + if (in.has_is_recycled()) { + out->set_is_recycled(in.is_recycled()); + } + if (in.has_job_id()) { + out->set_job_id(in.job_id()); + } } void doris_rowset_meta_to_cloud(RowsetMetaCloudPB* out, RowsetMetaPB&& in) { @@ -177,6 +183,12 @@ void doris_rowset_meta_to_cloud(RowsetMetaCloudPB* out, RowsetMetaPB&& in) { auto* slice_locations = out->mutable_packed_slice_locations(); slice_locations->clear(); slice_locations->insert(in.packed_slice_locations().begin(), in.packed_slice_locations().end()); + if (in.has_is_recycled()) { + out->set_is_recycled(in.is_recycled()); + } + if (in.has_job_id()) { + out->set_job_id(in.job_id()); + } } RowsetMetaPB cloud_rowset_meta_to_doris(const RowsetMetaCloudPB& in) { @@ -259,6 +271,12 @@ void cloud_rowset_meta_to_doris(RowsetMetaPB* out, const RowsetMetaCloudPB& in) auto* slice_locations = out->mutable_packed_slice_locations(); slice_locations->clear(); slice_locations->insert(in.packed_slice_locations().begin(), in.packed_slice_locations().end()); + if (in.has_is_recycled()) { + out->set_is_recycled(in.is_recycled()); + } + if (in.has_job_id()) { + out->set_job_id(in.job_id()); + } } void cloud_rowset_meta_to_doris(RowsetMetaPB* out, RowsetMetaCloudPB&& in) { @@ -330,6 +348,12 @@ void cloud_rowset_meta_to_doris(RowsetMetaPB* out, RowsetMetaCloudPB&& in) { auto* slice_locations = out->mutable_packed_slice_locations(); slice_locations->clear(); slice_locations->insert(in.packed_slice_locations().begin(), in.packed_slice_locations().end()); + if (in.has_is_recycled()) { + out->set_is_recycled(in.is_recycled()); + } + if (in.has_job_id()) { + out->set_job_id(in.job_id()); + } } TabletSchemaCloudPB doris_tablet_schema_to_cloud(const TabletSchemaPB& in) { diff --git a/be/src/olap/compaction.cpp b/be/src/olap/compaction.cpp index 6b0d214e25da02..63a9356f9fc38c 100644 --- a/be/src/olap/compaction.cpp +++ b/be/src/olap/compaction.cpp @@ -1720,6 +1720,7 @@ Status CloudCompactionMixin::construct_output_rowset_writer(RowsetWriterContext& ctx.file_cache_ttl_sec = _tablet->ttl_seconds(); ctx.approximate_bytes_to_write = _input_rowsets_total_size; ctx.tablet = _tablet; + ctx.job_id = _uuid; _output_rs_writer = DORIS_TRY(_tablet->create_rowset_writer(ctx, _is_vertical)); RETURN_IF_ERROR( diff --git a/be/src/olap/rowset/rowset_meta.h b/be/src/olap/rowset/rowset_meta.h index 48f55075ef1df7..bd16eb8e64443f 100644 --- a/be/src/olap/rowset/rowset_meta.h +++ b/be/src/olap/rowset/rowset_meta.h @@ -208,6 +208,10 @@ class RowsetMeta : public MetadataAdder { new_load_id->set_lo(load_id.lo()); } + void set_job_id(const std::string& job_id) { _rowset_meta_pb.set_job_id(job_id); } + + const std::string& job_id() const { return _rowset_meta_pb.job_id(); } + bool delete_flag() const { return _rowset_meta_pb.delete_flag(); } int64_t creation_time() const { return _rowset_meta_pb.creation_time(); } diff --git a/be/src/olap/rowset/rowset_writer_context.h b/be/src/olap/rowset/rowset_writer_context.h index 1d1b80d3984952..3728ba22687dfd 100644 --- a/be/src/olap/rowset/rowset_writer_context.h +++ b/be/src/olap/rowset/rowset_writer_context.h @@ -151,6 +151,8 @@ struct RowsetWriterContext { std::optional encrypt_algorithm; + std::string job_id; + bool is_local_rowset() const { return !storage_resource; } std::string segment_path(int seg_id) const { diff --git a/cloud/src/common/config.h b/cloud/src/common/config.h index e347fa764d1198..bf35f4243af0d3 100644 --- a/cloud/src/common/config.h +++ b/cloud/src/common/config.h @@ -273,6 +273,10 @@ CONF_mBool(enable_load_txn_status_check, "true"); CONF_mBool(enable_tablet_job_check, "true"); +CONF_mBool(enable_recycle_delete_rowset_key_check, "true"); +CONF_mBool(enable_mark_delete_rowset_before_recycle, "true"); +CONF_mBool(enable_abort_txn_and_job_for_delete_rowset_before_recycle, "true"); + // Declare a selection strategy for those servers have many ips. // Note that there should at most one ip match this list. // this is a list in semicolon-delimited format, in CIDR notation, diff --git a/cloud/src/meta-service/meta_service.cpp b/cloud/src/meta-service/meta_service.cpp index 129d1cd4e2d5d7..bca6eea8c09631 100644 --- a/cloud/src/meta-service/meta_service.cpp +++ b/cloud/src/meta-service/meta_service.cpp @@ -2589,6 +2589,75 @@ void MetaServiceImpl::prepare_rowset(::google::protobuf::RpcController* controll } } +// Check recycle_rowset_key to ensure idempotency for commit_rowset operation. +// The precondition for commit_rowset is that prepare_rowset has been successfully executed, +// which creates the recycle_rowset_key. Therefore, we only need to check if the +// recycle_rowset_key exists to determine if this is a duplicate request: +// - If key not found: commit_rowset has already been executed and remove the key, +// this is a duplicate request and should be rejected. +// - If key exists but marked as recycled: the rowset data has been recycled by recycler, +// this request should be rejected to prevent data inconsistency. +// - If key exists and not marked: this is a valid commit_rowset request, proceed normally. +// Note: We don't need to check txn/job status separately because prepare_rowset has already +// validated them, and the existence of recycle_rowset_key is sufficient to guarantee idempotency. +int check_idempotent_for_txn_or_job(Transaction* txn, const std::string& recycle_rs_key, + doris::RowsetMetaCloudPB& rowset_meta, + const std::string& instance_id, int64_t tablet_id, + const std::string& rowset_id, const std::string& tablet_job_id, + bool is_versioned_read, ResourceManager* resource_mgr, + MetaServiceCode& code, std::string& msg) { + if (!rowset_meta.has_delete_predicate() && config::enable_recycle_delete_rowset_key_check) { + std::string recycle_rs_val; + TxnErrorCode err = txn->get(recycle_rs_key, &recycle_rs_val); + if (err == TxnErrorCode::TXN_OK) { + RecycleRowsetPB recycle_rowset; + if (!recycle_rowset.ParseFromString(recycle_rs_val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = fmt::format("malformed recycle rowset value. key={}", hex(recycle_rs_key)); + return 1; + } + auto rs_meta = recycle_rowset.rowset_meta(); + if (rs_meta.has_is_recycled() && rs_meta.is_recycled()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = fmt::format("rowset has already been marked as recycled, key={}, rs_meta={}", + hex(recycle_rs_key), rs_meta.ShortDebugString()); + return 1; + } + } else if (err == TxnErrorCode::TXN_KEY_NOT_FOUND) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = fmt::format("recycle rowset key not found, key={}", hex(recycle_rs_key)); + return 1; + } else { + code = cast_as(err); + msg = fmt::format("failed to get recycle rowset, err={}, key={}", err, + hex(recycle_rs_key)); + return -1; + } + } else if (!config::enable_recycle_delete_rowset_key_check) { + if (config::enable_tablet_job_check && tablet_job_id.empty() && !tablet_job_id.empty()) { + if (!check_job_existed(txn, code, msg, instance_id, tablet_id, rowset_id, tablet_job_id, + is_versioned_read, resource_mgr)) { + return 1; + } + } + + // Check if the prepare rowset request is invalid. + // If the transaction has been finished, it means this prepare rowset is a timeout retry request. + // In this case, do not write the recycle key again, otherwise it may cause data loss. + // If the rowset had load id, it means it is a load request, otherwise it is a + // compaction/sc request. + if (config::enable_load_txn_status_check && rowset_meta.has_load_id() && + !check_transaction_status(TxnStatusPB::TXN_STATUS_PREPARED, txn, instance_id, + rowset_meta.txn_id(), code, msg)) { + LOG(WARNING) << "prepare rowset failed, txn_id=" << rowset_meta.txn_id() + << ", tablet_id=" << tablet_id << ", rowset_id=" << rowset_id + << ", rowset_state=" << rowset_meta.rowset_state() << ", msg=" << msg; + return 1; + } + } + return 0; +} + /** * 1. Check and confirm tmp rowset kv does not exist * a. if exist @@ -2635,27 +2704,12 @@ void MetaServiceImpl::commit_rowset(::google::protobuf::RpcController* controlle return; } - // Check if the compaction/sc tablet job has finished bool is_versioned_read = is_version_read_enabled(instance_id); - if (config::enable_tablet_job_check && request->has_tablet_job_id() && - !request->tablet_job_id().empty()) { - if (!check_job_existed(txn.get(), code, msg, instance_id, tablet_id, rowset_id, - request->tablet_job_id(), is_versioned_read, resource_mgr_.get())) { - return; - } - } + auto recycle_rs_key = recycle_rowset_key({instance_id, tablet_id, rowset_id}); - // Check if the commit rowset request is invalid. - // If the transaction has been finished, it means this commit rowset is a timeout retry request. - // In this case, do not write the recycle key again, otherwise it may cause data loss. - // If the rowset has load id, it means it is a load request, otherwise it is a - // compaction/sc request. - if (config::enable_load_txn_status_check && rowset_meta.has_load_id() && - !check_transaction_status(TxnStatusPB::TXN_STATUS_PREPARED, txn.get(), instance_id, - rowset_meta.txn_id(), code, msg)) { - LOG(WARNING) << "commit rowset failed, txn_id=" << rowset_meta.txn_id() - << ", tablet_id=" << tablet_id << ", rowset_id=" << rowset_id - << ", rowset_state=" << rowset_meta.rowset_state() << ", msg=" << msg; + if (check_idempotent_for_txn_or_job(txn.get(), recycle_rs_key, rowset_meta, instance_id, + tablet_id, rowset_id, request->tablet_job_id(), + is_versioned_read, resource_mgr_.get(), code, msg) != 0) { return; } @@ -2765,9 +2819,7 @@ void MetaServiceImpl::commit_rowset(::google::protobuf::RpcController* controlle rowset_meta.set_allocated_tablet_schema(nullptr); } - auto recycle_rs_key = recycle_rowset_key({instance_id, tablet_id, rowset_id}); txn->remove(recycle_rs_key); - DCHECK_GT(rowset_meta.txn_expiration(), 0); auto tmp_rs_val = rowset_meta.SerializeAsString(); txn->put(tmp_rs_key, tmp_rs_val); diff --git a/cloud/src/meta-service/meta_service.h b/cloud/src/meta-service/meta_service.h index 44027ed6316a0b..c801d438faf87a 100644 --- a/cloud/src/meta-service/meta_service.h +++ b/cloud/src/meta-service/meta_service.h @@ -68,6 +68,18 @@ static void* run_bthread_work(void* arg) { int64_t partition_count, int64_t tablet_count, int64_t txn_id); +[[maybe_unused]] void _abort_txn(const std::string& instance_id, const AbortTxnRequest* request, + Transaction* txn, TxnInfoPB& return_txn_info, + std::stringstream& ss, MetaServiceCode& code, std::string& msg); + +[[maybe_unused]] void _finish_tablet_job(const FinishTabletJobRequest* request, + FinishTabletJobResponse* response, + std::string& instance_id, + std::unique_ptr& txn, TxnKv* txn_kv, + DeleteBitmapLockWhiteList* delete_bitmap_lock_white_list, + ResourceManager* resource_mgr, MetaServiceCode& code, + std::string& msg, std::stringstream& ss); + class MetaServiceImpl : public cloud::MetaService { public: MetaServiceImpl(std::shared_ptr txn_kv, std::shared_ptr resource_mgr, diff --git a/cloud/src/meta-service/meta_service_job.cpp b/cloud/src/meta-service/meta_service_job.cpp index 6ffe6f1e506dae..2494e91c241e24 100644 --- a/cloud/src/meta-service/meta_service_job.cpp +++ b/cloud/src/meta-service/meta_service_job.cpp @@ -1220,6 +1220,14 @@ void process_compaction_job(MetaServiceCode& code, std::string& msg, std::string return; } + if (rs_meta.has_is_recycled() && rs_meta.is_recycled()) { + SS << "rowset has already been marked as recycled, tablet_id=" << tablet_id + << " txn_id=" << rs_meta.txn_id() << " rowset_id=" << rs_meta.rowset_id_v2(); + msg = ss.str(); + code = MetaServiceCode::TXN_ALREADY_ABORTED; + return; + } + txn->remove(tmp_rowset_key); INSTANCE_LOG(INFO) << "remove tmp rowset meta, tablet_id=" << tablet_id << " tmp_rowset_key=" << hex(tmp_rowset_key); @@ -1804,6 +1812,16 @@ void process_schema_change_job(MetaServiceCode& code, std::string& msg, std::str msg = ss.str(); return; } + + if (tmp_rowset_meta.has_is_recycled() && tmp_rowset_meta.is_recycled()) { + SS << "rowset has already been marked as recycled, tablet_id=" << new_tablet_id + << " txn_id=" << tmp_rowset_meta.txn_id() + << " rowset_id=" << tmp_rowset_meta.rowset_id_v2(); + msg = ss.str(); + code = MetaServiceCode::TXN_ALREADY_ABORTED; + return; + } + using namespace std::chrono; auto rowset_visible_time = duration_cast(system_clock::now().time_since_epoch()).count(); @@ -1896,34 +1914,16 @@ void process_schema_change_job(MetaServiceCode& code, std::string& msg, std::str } } -void MetaServiceImpl::finish_tablet_job(::google::protobuf::RpcController* controller, - const FinishTabletJobRequest* request, - FinishTabletJobResponse* response, - ::google::protobuf::Closure* done) { - RPC_PREPROCESS(finish_tablet_job, get, put, del); - std::string cloud_unique_id = request->cloud_unique_id(); - instance_id = get_instance_id(resource_mgr_, cloud_unique_id); - if (instance_id.empty()) { - code = MetaServiceCode::INVALID_ARGUMENT; - SS << "cannot find instance_id with cloud_unique_id=" - << (cloud_unique_id.empty() ? "(empty)" : cloud_unique_id); - msg = ss.str(); - LOG(INFO) << msg; - return; - } - RPC_RATE_LIMIT(finish_tablet_job) - if (!request->has_job() || - (request->job().compaction().empty() && !request->job().has_schema_change())) { - code = MetaServiceCode::INVALID_ARGUMENT; - msg = "no valid job specified"; - return; - } - - bool is_versioned_read = is_version_read_enabled(instance_id); - bool is_versioned_write = is_version_write_enabled(instance_id); +void _finish_tablet_job(const FinishTabletJobRequest* request, FinishTabletJobResponse* response, + std::string& instance_id, std::unique_ptr& txn, TxnKv* txn_kv, + DeleteBitmapLockWhiteList* delete_bitmap_lock_white_list, + ResourceManager* resource_mgr, MetaServiceCode& code, std::string& msg, + std::stringstream& ss) { + bool is_versioned_read = resource_mgr->is_version_read_enabled(instance_id); + bool is_versioned_write = resource_mgr->is_version_write_enabled(instance_id); for (int retry = 0; retry <= 1; retry++) { bool need_commit = false; - TxnErrorCode err = txn_kv_->create_txn(&txn); + TxnErrorCode err = txn_kv->create_txn(&txn); if (err != TxnErrorCode::TXN_OK) { code = cast_as(err); msg = "failed to create txn"; @@ -1943,7 +1943,7 @@ void MetaServiceImpl::finish_tablet_job(::google::protobuf::RpcController* contr get_tablet_idx(code, msg, txn.get(), instance_id, tablet_id, tablet_idx); if (code != MetaServiceCode::OK) return; } else { - CloneChainReader reader(instance_id, resource_mgr_.get()); + CloneChainReader reader(instance_id, resource_mgr); err = reader.get_tablet_index(txn.get(), tablet_id, &tablet_idx); if (err != TxnErrorCode::TXN_OK) { code = err == TxnErrorCode::TXN_KEY_NOT_FOUND @@ -1987,20 +1987,19 @@ void MetaServiceImpl::finish_tablet_job(::google::protobuf::RpcController* contr FinishTabletJobRequest_Action action = request->action(); std::string use_version = - delete_bitmap_lock_white_list_->get_delete_bitmap_lock_version(instance_id); + delete_bitmap_lock_white_list->get_delete_bitmap_lock_version(instance_id); LOG(INFO) << "finish_tablet_job instance_id=" << instance_id << " use_version=" << use_version; if (!request->job().compaction().empty()) { // Process compaction commit process_compaction_job(code, msg, ss, txn, request, response, recorded_job, instance_id, job_key, need_commit, use_version, is_versioned_read, - is_versioned_write, txn_kv_.get(), resource_mgr_.get()); + is_versioned_write, txn_kv, resource_mgr); } else if (request->job().has_schema_change()) { // Process schema change commit process_schema_change_job(code, msg, ss, txn, request, response, recorded_job, instance_id, job_key, need_commit, use_version, - is_versioned_read, is_versioned_write, txn_kv_.get(), - resource_mgr_.get()); + is_versioned_read, is_versioned_write, txn_kv, resource_mgr); } if (!need_commit) return; @@ -2037,6 +2036,32 @@ void MetaServiceImpl::finish_tablet_job(::google::protobuf::RpcController* contr } } +void MetaServiceImpl::finish_tablet_job(::google::protobuf::RpcController* controller, + const FinishTabletJobRequest* request, + FinishTabletJobResponse* response, + ::google::protobuf::Closure* done) { + RPC_PREPROCESS(finish_tablet_job, get, put, del); + std::string cloud_unique_id = request->cloud_unique_id(); + instance_id = get_instance_id(resource_mgr_, cloud_unique_id); + if (instance_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + SS << "cannot find instance_id with cloud_unique_id=" + << (cloud_unique_id.empty() ? "(empty)" : cloud_unique_id); + msg = ss.str(); + LOG(INFO) << msg; + return; + } + RPC_RATE_LIMIT(finish_tablet_job) + if (!request->has_job() || + (request->job().compaction().empty() && !request->job().has_schema_change())) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "no valid job specified"; + return; + } + _finish_tablet_job(request, response, instance_id, txn, txn_kv_.get(), + delete_bitmap_lock_white_list_.get(), resource_mgr_.get(), code, msg, ss); +} + #undef SS #undef INSTANCE_LOG } // namespace doris::cloud diff --git a/cloud/src/meta-service/meta_service_txn.cpp b/cloud/src/meta-service/meta_service_txn.cpp index ca4877dd82788f..ab6e2fedfe5319 100644 --- a/cloud/src/meta-service/meta_service_txn.cpp +++ b/cloud/src/meta-service/meta_service_txn.cpp @@ -1154,17 +1154,22 @@ void scan_tmp_rowset( while (it->has_next()) { auto [k, v] = it->next(); LOG(INFO) << "range_get rowset_tmp_key=" << hex(k) << " txn_id=" << txn_id; - tmp_rowsets_meta->emplace_back(); - if (!tmp_rowsets_meta->back().second.ParseFromArray(v.data(), v.size())) { + RowsetMetaCloudPB rs_meta; + if (!rs_meta.ParseFromArray(v.data(), v.size())) { code = MetaServiceCode::PROTOBUF_PARSE_ERR; ss << "malformed rowset meta, unable to initialize, txn_id=" << txn_id - << " key=" << hex(k); + << " key=" << hex(k) << " err=" << err; msg = ss.str(); LOG(WARNING) << msg; return; } - // Save keys that will be removed later - tmp_rowsets_meta->back().first = std::string(k.data(), k.size()); + if (rs_meta.has_is_recycled() && rs_meta.is_recycled()) { + code = MetaServiceCode::TXN_ALREADY_ABORTED; + msg = "rowset has already been marked as recycled"; + LOG(WARNING) << msg; + continue; + } + tmp_rowsets_meta->emplace_back(std::string(k.data(), k.size()), std::move(rs_meta)); ++num_rowsets; if (!it->has_next()) rs_tmp_key0 = k; } @@ -3046,9 +3051,9 @@ void MetaServiceImpl::commit_txn(::google::protobuf::RpcController* controller, stats); } -static void _abort_txn(const std::string& instance_id, const AbortTxnRequest* request, - Transaction* txn, TxnInfoPB& return_txn_info, std::stringstream& ss, - MetaServiceCode& code, std::string& msg) { +void _abort_txn(const std::string& instance_id, const AbortTxnRequest* request, Transaction* txn, + TxnInfoPB& return_txn_info, std::stringstream& ss, MetaServiceCode& code, + std::string& msg) { int64_t txn_id = request->txn_id(); std::string label = request->label(); int64_t db_id = request->db_id(); diff --git a/cloud/src/recycler/recycler.cpp b/cloud/src/recycler/recycler.cpp index 7959a72efa3942..1a4781144f2d0f 100644 --- a/cloud/src/recycler/recycler.cpp +++ b/cloud/src/recycler/recycler.cpp @@ -586,7 +586,11 @@ InstanceRecycler::InstanceRecycler(std::shared_ptr txn_kv, const Instance instance_info_(instance), inverted_index_id_cache_(std::make_unique(instance_id_, txn_kv_)), _thread_pool_group(std::move(thread_pool_group)), - txn_lazy_committer_(std::move(txn_lazy_committer)) { + txn_lazy_committer_(std::move(txn_lazy_committer)), + delete_bitmap_lock_white_list_(std::make_shared()), + resource_mgr_(std::make_shared(txn_kv_)) { + delete_bitmap_lock_white_list_->init(); + resource_mgr_->init(); snapshot_manager_ = std::make_shared(txn_kv_); // Since the recycler's resource manager could not be notified when instance info changes, @@ -1498,89 +1502,6 @@ int InstanceRecycler::handle_packed_file_kv(std::string_view key, std::string_vi } return 0; } -bool is_txn_finished(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) { - TEST_SYNC_POINT_CALLBACK("is_txn_finished::txn_has_been_recycled"); - // txn has been recycled; - LOG(INFO) << "txn index key has been recycled, txn_id=" << txn_id - << " instance_id=" << instance_id; - 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); - if (!index_pb.tablet_index().has_db_id()) { - // In the previous version, the db_id was not set in the index_pb. - // If updating to the version which enable txn lazy commit, the db_id will be set. - LOG(INFO) << "txn index has no db_id, txn_id=" << txn_id << " instance_id=" << instance_id - << " index=" << index_pb.ShortDebugString(); - return true; - } - - int64_t db_id = index_pb.tablet_index().db_id(); - DCHECK_GT(db_id, 0) << "db_id=" << db_id << " txn_id=" << txn_id - << " instance_id=" << instance_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) { - if (TxnErrorCode::TXN_KEY_NOT_FOUND == err) { - // txn info has been recycled; - LOG(INFO) << "txn info key has been recycled, db_id=" << db_id << " txn_id=" << txn_id - << " instance_id=" << instance_id; - return true; - } - - 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) << "txn_id=" << txn_id << " instance_id=" << instance_id - << " txn_info=" << txn_info.ShortDebugString(); - - if (TxnStatusPB::TXN_STATUS_ABORTED == txn_info.status() || - TxnStatusPB::TXN_STATUS_VISIBLE == txn_info.status()) { - TEST_SYNC_POINT_CALLBACK("is_txn_finished::txn_has_been_aborted", &txn_info); - return true; - } - - TEST_SYNC_POINT_CALLBACK("is_txn_finished::txn_not_finished", &txn_info); - return false; -} int64_t calculate_rowset_expired_time(const std::string& instance_id_, const RecycleRowsetPB& rs, int64_t* earlest_ts /* rowset earliest expiration ts */) { @@ -1728,6 +1649,275 @@ int get_meta_rowset_key(Transaction* txn, const std::string& instance_id, int64_ return 0; } +int InstanceRecycler::abort_txn_for_related_rowset(int64_t txn_id) { + AbortTxnRequest req; + TxnInfoPB txn_info; + MetaServiceCode code = MetaServiceCode::OK; + std::string msg; + std::stringstream ss; + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + LOG_WARNING("failed to create txn").tag("err", err); + return -1; + } + + // get txn index + TxnIndexPB txn_idx_pb; + auto index_key = txn_index_key({instance_id_, txn_id}); + std::string index_val; + err = txn->get(index_key, &index_val); + if (err != TxnErrorCode::TXN_OK) { + if (err == TxnErrorCode::TXN_KEY_NOT_FOUND) { + // maybe recycled + LOG_INFO("txn index not found, txn_id={} instance_id={}", txn_id, instance_id_) + .tag("key", hex(index_key)) + .tag("txn_id", txn_id); + return 0; + } + LOG_WARNING("failed to get txn index") + .tag("err", err) + .tag("key", hex(index_key)) + .tag("txn_id", txn_id); + return -1; + } + if (!txn_idx_pb.ParseFromString(index_val)) { + LOG_WARNING("failed to parse txn index") + .tag("err", err) + .tag("key", hex(index_key)) + .tag("txn_id", txn_id); + return -1; + } + + auto info_key = txn_info_key({instance_id_, txn_idx_pb.tablet_index().db_id(), txn_id}); + std::string info_val; + err = txn->get(info_key, &info_val); + if (err != TxnErrorCode::TXN_OK) { + if (err == TxnErrorCode::TXN_KEY_NOT_FOUND) { + // maybe recycled + LOG_INFO("txn info not found, txn_id={} instance_id={}", txn_id, instance_id_) + .tag("key", hex(info_key)) + .tag("txn_id", txn_id); + return 0; + } + LOG_WARNING("failed to get txn info") + .tag("err", err) + .tag("key", hex(info_key)) + .tag("txn_id", txn_id); + return -1; + } + if (!txn_info.ParseFromString(info_val)) { + LOG_WARNING("failed to parse txn info") + .tag("err", err) + .tag("key", hex(info_key)) + .tag("txn_id", txn_id); + return -1; + } + + if (txn_info.status() != TxnStatusPB::TXN_STATUS_PREPARED) { + LOG_INFO("txn is not prepared status, txn_id={} status={}", txn_id, txn_info.status()) + .tag("key", hex(info_key)) + .tag("txn_id", txn_id); + return 0; + } + + req.set_txn_id(txn_id); + + LOG(INFO) << "begin abort txn for related rowset, txn_id=" << txn_id + << " instance_id=" << instance_id_ << " txn_info=" << txn_info.ShortDebugString(); + + _abort_txn(instance_id_, &req, txn.get(), txn_info, ss, code, msg); + err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "failed to commit kv txn, txn_id=" << txn_info.txn_id() << " err=" << err; + msg = ss.str(); + return -1; + } + + LOG(INFO) << "finish abort txn for related rowset, txn_id=" << txn_id + << " instance_id=" << instance_id_ << " txn_info=" << txn_info.ShortDebugString() + << " code=" << code << " msg=" << msg; + + return 0; +} + +int InstanceRecycler::abort_job_for_related_rowset(const RowsetMetaCloudPB& rowset_meta) { + FinishTabletJobRequest req; + FinishTabletJobResponse res; + req.set_action(FinishTabletJobRequest::ABORT); + MetaServiceCode code = MetaServiceCode::OK; + std::string msg; + std::stringstream ss; + + TabletIndexPB tablet_idx; + int ret = get_tablet_idx(txn_kv_.get(), instance_id_, rowset_meta.tablet_id(), tablet_idx); + if (ret != 0) { + LOG(WARNING) << "failed to get tablet index, tablet_id=" << rowset_meta.tablet_id() + << " instance_id=" << instance_id_ << " ret=" << ret; + return ret; + } + + 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_ << " err=" << err; + return -1; + } + + std::string job_key = + job_tablet_key({instance_id_, tablet_idx.table_id(), tablet_idx.index_id(), + tablet_idx.partition_id(), tablet_idx.tablet_id()}); + std::string job_val; + err = txn->get(job_key, &job_val); + if (err != TxnErrorCode::TXN_OK) { + if (err == TxnErrorCode::TXN_KEY_NOT_FOUND) { + LOG(INFO) << "job not exists, instance_id=" << instance_id_ + << " tablet_id=" << tablet_idx.tablet_id() << " key=" << hex(job_key); + return 0; + } + LOG(WARNING) << "failed to get job, instance_id=" << instance_id_ + << " tablet_id=" << tablet_idx.tablet_id() << " err=" << err + << " key=" << hex(job_key); + return -1; + } + + TabletJobInfoPB job_pb; + if (!job_pb.ParseFromString(job_val)) { + LOG(WARNING) << "failed to parse job, instance_id=" << instance_id_ + << " tablet_id=" << tablet_idx.tablet_id() << " key=" << hex(job_key); + return -1; + } + + std::string job_id {}; + if (!job_pb.compaction().empty()) { + for (const auto& c : job_pb.compaction()) { + if (c.id() == rowset_meta.job_id()) { + job_id = c.id(); + break; + } + } + } else if (job_pb.has_schema_change()) { + job_id = job_pb.schema_change().id(); + } + + if (!job_id.empty() && rowset_meta.job_id() == job_id) { + LOG(INFO) << "begin to abort job for related rowset, job_id=" << rowset_meta.job_id() + << " instance_id=" << instance_id_ << " tablet_id=" << tablet_idx.tablet_id(); + req.mutable_job()->CopyFrom(job_pb); + req.set_action(FinishTabletJobRequest::ABORT); + _finish_tablet_job(&req, &res, instance_id_, txn, txn_kv_.get(), + delete_bitmap_lock_white_list_.get(), resource_mgr_.get(), code, msg, + ss); + if (code != MetaServiceCode::OK) { + LOG(WARNING) << "failed to abort job, instance_id=" << instance_id_ + << " tablet_id=" << tablet_idx.tablet_id() << " code=" << code + << " msg=" << msg; + return -1; + } + LOG(INFO) << "finish abort job for related rowset, job_id=" << rowset_meta.job_id() + << " instance_id=" << instance_id_ << " tablet_id=" << tablet_idx.tablet_id() + << " code=" << code << " msg=" << msg; + } else { + // clang-format off + LOG(INFO) << "there is no job for related rowset, directly recycle rowset data" + << ", instance_id=" << instance_id_ + << ", tablet_id=" << tablet_idx.tablet_id() + << ", job_id=" << job_id + << ", rowset_id=" << rowset_meta.rowset_id_v2(); + // clang-format on + } + + return 0; +} + +template +int InstanceRecycler::abort_txn_or_job_for_recycle(T& rowset_meta_pb) { + RowsetMetaCloudPB* rs_meta; + RecycleRowsetPB::Type rowset_type = RecycleRowsetPB::PREPARE; + + if constexpr (std::is_same_v) { + // For keys that are not in the RecycleRowsetPB::PREPARE state + // we do not need to check the job or txn state + // because tmp_rowset_key already exists when this key is generated. + rowset_type = rowset_meta_pb.type(); + rs_meta = rowset_meta_pb.mutable_rowset_meta(); + } else { + rs_meta = &rowset_meta_pb; + } + + DCHECK(rs_meta != nullptr); + + // compaction/sc will generate recycle_rowset_key for each input rowset with load_id + // we need skip them because the related txn has been finished + // load_rowset1 load_rowset2 => pick for compaction => compact_rowset + // compact_rowset1 compact_rowset2 => pick for compaction/sc job => new_rowset + if (rowset_type == RecycleRowsetPB::PREPARE) { + if (rs_meta->has_load_id()) { + // load + return abort_txn_for_related_rowset(rs_meta->txn_id()); + } else if (rs_meta->has_job_id()) { + // compaction / schema change + return abort_job_for_related_rowset(*rs_meta); + } + } + + return 0; +} + +template +int mark_rowset_as_recycled(TxnKv* txn_kv, const std::string& instance_id, std::string_view key, + T& rowset_meta_pb) { + RowsetMetaCloudPB* rs_meta; + + if constexpr (std::is_same_v) { + rs_meta = rowset_meta_pb.mutable_rowset_meta(); + } else { + rs_meta = &rowset_meta_pb; + } + + bool need_write_back = false; + if ((!rs_meta->has_is_recycled() || !rs_meta->is_recycled())) { + need_write_back = true; + rs_meta->set_is_recycled(true); + } + + if (need_write_back) { + 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; + return -1; + } + // double check becase of new transaction + T rowset_meta; + std::string val; + err = txn->get(key, &val); + if (!rowset_meta.ParseFromString(val)) { + LOG(WARNING) << "failed to parse rs_meta, instance_id=" << instance_id; + return -1; + } + if constexpr (std::is_same_v) { + rs_meta = rowset_meta.mutable_rowset_meta(); + } else { + rs_meta = &rowset_meta; + } + if ((rs_meta->has_is_recycled() && rs_meta->is_recycled())) { + return 0; + } + rs_meta->set_is_recycled(true); + val.clear(); + rowset_meta.SerializeToString(&val); + txn->put(key, val); + err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to commit txn, instance_id=" << instance_id; + return -1; + } + } + return need_write_back ? 1 : 0; +} + int InstanceRecycler::recycle_ref_rowsets(bool* has_unrecycled_rowsets) { const std::string task_name = "recycle_ref_rowsets"; int64_t num_scanned = 0; @@ -2144,8 +2334,8 @@ int InstanceRecycler::recycle_partitions() { return -1; } int64_t current_time = ::time(nullptr); - if (current_time < - calculate_partition_expired_time(instance_id_, part_pb, &earlest_ts)) { // not expired + if (current_time < calculate_partition_expired_time(instance_id_, part_pb, + &earlest_ts)) { // not expired return 0; } ++num_expired; @@ -3273,7 +3463,8 @@ int InstanceRecycler::delete_rowset_data( // If there are inverted indexes, some data might not be deleted, // but this is acceptable as we have made our best effort to delete the data. LOG_INFO( - "delete rowset data schema kv not found, need to delete again to double " + "delete rowset data schema kv not found, need to delete again to " + "double " "check") .tag("instance_id", instance_id_) .tag("tablet_id", tablet_id) @@ -4398,6 +4589,7 @@ int InstanceRecycler::recycle_rowsets() { } ++num_expired; expired_rowset_size += v.size(); + if (!rowset.has_type()) { // old version `RecycleRowsetPB` if (!rowset.has_resource_id()) [[unlikely]] { // impossible // in old version, keep this key-value pair and it needs to be checked manually @@ -4434,8 +4626,45 @@ int InstanceRecycler::recycle_rowsets() { metrics_context.report(); return 0; } + + auto* rowset_meta = rowset.mutable_rowset_meta(); + if (config::enable_mark_delete_rowset_before_recycle) { + int mark_ret = mark_rowset_as_recycled(txn_kv_.get(), instance_id_, k, rowset); + if (mark_ret == -1) { + LOG(WARNING) << "failed to mark rowset as recycled, instance_id=" << instance_id_ + << " tablet_id=" << rowset_meta->tablet_id() << " version=[" + << rowset_meta->start_version() << '-' << rowset_meta->end_version() + << "]"; + return -1; + } else if (mark_ret == 1) { + LOG(INFO) + << "rowset already marked as recycled, recycler will delete data and kv at " + "next turn, instance_id=" + << instance_id_ << " tablet_id=" << rowset_meta->tablet_id() << " version=[" + << rowset_meta->start_version() << '-' << rowset_meta->end_version() << "]"; + return 0; + } + } + + if (config::enable_abort_txn_and_job_for_delete_rowset_before_recycle) { + LOG(INFO) << "begin to abort txn or job for related rowset, instance_id=" + << instance_id_ << " tablet_id=" << rowset_meta->tablet_id() << " version=[" + << rowset_meta->start_version() << '-' << rowset_meta->end_version() << "]"; + + if (rowset_meta->end_version() != 1) { + int ret = abort_txn_or_job_for_recycle(rowset); + + if (ret != 0) { + LOG(WARNING) << "failed to abort txn or job for related rowset, instance_id=" + << instance_id_ << " tablet_id=" << rowset.tablet_id() + << " version=[" << rowset_meta->start_version() << '-' + << rowset_meta->end_version() << "]"; + return ret; + } + } + } + // TODO(plat1ko): check rowset not referenced - auto rowset_meta = rowset.mutable_rowset_meta(); if (!rowset_meta->has_resource_id()) [[unlikely]] { // impossible if (rowset.type() != RecycleRowsetPB::PREPARE && rowset_meta->num_segments() == 0) { LOG_INFO("recycle rowset that has empty resource id"); @@ -5126,17 +5355,35 @@ int InstanceRecycler::recycle_tmp_rowsets() { return 0; } - DCHECK_GT(rowset.txn_id(), 0) - << "txn_id=" << rowset.txn_id() << " rowset=" << rowset.ShortDebugString(); - if (!is_txn_finished(txn_kv_, instance_id_, rowset.txn_id())) { - LOG(INFO) << "txn is not finished, skip recycle tmp rowset, instance_id=" - << instance_id_ << " tablet_id=" << rowset.tablet_id() - << " rowset_id=" << rowset.rowset_id_v2() << " version=[" - << rowset.start_version() << '-' << rowset.end_version() - << "] txn_id=" << rowset.txn_id() - << " creation_time=" << rowset.creation_time() << " expiration=" << expiration - << " txn_expiration=" << rowset.txn_expiration(); - return 0; + if (config::enable_mark_delete_rowset_before_recycle) { + int mark_ret = mark_rowset_as_recycled(txn_kv_.get(), instance_id_, k, rowset); + if (mark_ret == -1) { + LOG(WARNING) << "failed to mark rowset as recycled, instance_id=" << instance_id_ + << " tablet_id=" << rowset.tablet_id() << " version=[" + << rowset.start_version() << '-' << rowset.end_version() << "]"; + return -1; + } else if (mark_ret == 1) { + LOG(INFO) + << "rowset already marked as recycled, recycler will delete data and kv at " + "next turn, instance_id=" + << instance_id_ << " tablet_id=" << rowset.tablet_id() << " version=[" + << rowset.start_version() << '-' << rowset.end_version() << "]"; + return 0; + } + } + + if (config::enable_abort_txn_and_job_for_delete_rowset_before_recycle) { + LOG(INFO) << "begin to abort txn or job for related rowset, instance_id=" + << instance_id_ << " tablet_id=" << rowset.tablet_id() << " version=[" + << rowset.start_version() << '-' << rowset.end_version() << "]"; + + int ret = abort_txn_or_job_for_recycle(rowset); + if (ret != 0) { + LOG(WARNING) << "failed to abort txn or job for related rowset, instance_id=" + << instance_id_ << " tablet_id=" << rowset.tablet_id() << " version=[" + << rowset.start_version() << '-' << rowset.end_version() << "]"; + return ret; + } } ++num_expired; @@ -6407,19 +6654,21 @@ int InstanceRecycler::scan_and_statistics_rowsets() { std::string recyc_rs_key0; std::string recyc_rs_key1; recycle_rowset_key(recyc_rs_key_info0, &recyc_rs_key0); - recycle_rowset_key(recyc_rs_key_info1, &recyc_rs_key1); - int64_t earlest_ts = std::numeric_limits::max(); + recycle_rowset_key(recyc_rs_key_info1, &recyc_rs_key1); + int64_t earlest_ts = std::numeric_limits::max(); auto handle_rowset_kv = [&, this](std::string_view k, std::string_view v) -> int { RecycleRowsetPB rowset; if (!rowset.ParseFromArray(v.data(), v.size())) { return 0; } + auto* rowset_meta = rowset.mutable_rowset_meta(); int64_t current_time = ::time(nullptr); if (current_time < calculate_rowset_expired_time(instance_id_, rowset, &earlest_ts)) { // not expired return 0; } + if (!rowset.has_type()) { if (!rowset.has_resource_id()) [[unlikely]] { return 0; @@ -6433,7 +6682,11 @@ int InstanceRecycler::scan_and_statistics_rowsets() { segment_metrics_context_.total_need_recycle_data_size += rowset.rowset_meta().total_disk_size(); return 0; } - auto* rowset_meta = rowset.mutable_rowset_meta(); + + if(!rowset_meta->has_is_recycled() || !rowset_meta->is_recycled()) { + return 0; + } + if (!rowset_meta->has_resource_id()) [[unlikely]] { if (rowset.type() == RecycleRowsetPB::PREPARE || rowset_meta->num_segments() != 0) { return 0; @@ -6476,7 +6729,8 @@ int InstanceRecycler::scan_and_statistics_tmp_rowsets() { DCHECK_GT(rowset.txn_id(), 0) << "txn_id=" << rowset.txn_id() << " rowset=" << rowset.ShortDebugString(); - if (!is_txn_finished(txn_kv_, instance_id_, rowset.txn_id())) { + + if(!rowset.has_is_recycled() || !rowset.is_recycled()) { return 0; } diff --git a/cloud/src/recycler/recycler.h b/cloud/src/recycler/recycler.h index 760b78aef8155c..a24a68516d1d8f 100644 --- a/cloud/src/recycler/recycler.h +++ b/cloud/src/recycler/recycler.h @@ -34,6 +34,7 @@ #include #include "common/bvars.h" +#include "meta-service/delete_bitmap_lock_white_list.h" #include "meta-service/txn_lazy_committer.h" #include "meta-store/versionstamp.h" #include "recycler/snapshot_chain_compactor.h" @@ -537,6 +538,34 @@ class InstanceRecycler { int handle_packed_file_kv(std::string_view key, std::string_view value, PackedFileRecycleStats* stats, int* ret); + // Abort the transaction/job associated with a rowset that is about to be recycled. + // This function is called during rowset recycling to prevent data loss by ensuring that + // the transaction/job cannot be committed after its rowset data has been deleted. + // + // Scenario: + // When recycler detects an expired prepared rowset (e.g., from a failed load transaction/job), + // it needs to recycle the rowset data. However, if the transaction/job is still active and gets + // committed after the data is deleted, it would lead to data loss - the transaction/job would + // reference non-existent data. + // + // Solution: + // Before recycling the rowset data, this function aborts the associated transaction/job to ensure + // it cannot be committed. This guarantees that: + // 1. The transaction/job state is marked as ABORTED + // 2. Any subsequent commit_rowset/commit_txn attempts will fail + // 3. The rowset data can be safely deleted without risk of data loss + // + // Parameters: + // txn_id: The transaction/job ID associated with the rowset to be recycled + // + // Returns: + // 0 on success, -1 on failure + int abort_txn_for_related_rowset(int64_t txn_id); + int abort_job_for_related_rowset(const RowsetMetaCloudPB& rowset_meta); + + template + int abort_txn_or_job_for_recycle(T& rowset_meta_pb); + private: std::atomic_bool stopped_ {false}; std::shared_ptr txn_kv_; @@ -563,6 +592,8 @@ class InstanceRecycler { std::shared_ptr txn_lazy_committer_; std::shared_ptr snapshot_manager_; + std::shared_ptr delete_bitmap_lock_white_list_; + std::shared_ptr resource_mgr_; TabletRecyclerMetricsContext tablet_metrics_context_; SegmentRecyclerMetricsContext segment_metrics_context_; diff --git a/cloud/test/meta_service_job_test.cpp b/cloud/test/meta_service_job_test.cpp index d3e2cea089e17e..65527e6a86973f 100644 --- a/cloud/test/meta_service_job_test.cpp +++ b/cloud/test/meta_service_job_test.cpp @@ -40,6 +40,11 @@ #include "meta-store/txn_kv.h" #include "meta-store/txn_kv_error.h" #include "meta-store/versioned_value.h" +#include "mock_accessor.h" +#include "recycler/recycler.h" + +extern doris::cloud::RecyclerThreadPoolGroup thread_group; +const std::string RESOURCE_ID = "mock_resource_id"; namespace doris::cloud { // External functions from meta_service_test.cpp @@ -115,6 +120,40 @@ void start_compaction_job(MetaService* meta_service, int64_t tablet_id, const st meta_service->start_tablet_job(&cntl, &req, &res, nullptr); }; +std::string next_rowset_id() { + static int cnt = 0; + return fmt::format("{:04}", ++cnt); +} + +void finish_compaction_job(MetaService* meta_service, int64_t tablet_id, const std::string& job_id, + const std::string& initiator, int base_compaction_cnt, + int cumu_compaction_cnt, TabletCompactionJobPB::CompactionType type, + FinishTabletJobResponse& res, + FinishTabletJobRequest::Action action = FinishTabletJobRequest::COMMIT, + std::pair input_version = {0, 0}, int64_t txn_id = 0) { + brpc::Controller cntl; + FinishTabletJobRequest req; + req.set_action(action); + req.mutable_job()->mutable_idx()->set_tablet_id(tablet_id); + auto* compaction = req.mutable_job()->add_compaction(); + compaction->set_id(job_id); + compaction->set_initiator(initiator); + compaction->set_base_compaction_cnt(base_compaction_cnt); + compaction->set_cumulative_compaction_cnt(cumu_compaction_cnt); + compaction->set_type(type); + if (input_version.second > 0) { + compaction->add_input_versions(input_version.first); + compaction->add_input_versions(input_version.second); + compaction->set_check_input_versions_range(true); + compaction->add_output_versions(input_version.second); + } + if (txn_id > 0) { + compaction->add_txn_id(txn_id); + } + compaction->add_output_rowset_ids(next_rowset_id()); + meta_service->finish_tablet_job(&cntl, &req, &res, nullptr); +} + void get_tablet_stats(MetaService* meta_service, int64_t tablet_id, TabletStatsPB& stats) { brpc::Controller cntl; GetTabletStatsRequest req; @@ -126,11 +165,6 @@ void get_tablet_stats(MetaService* meta_service, int64_t tablet_id, TabletStatsP stats = res.tablet_stats(0); } -std::string next_rowset_id() { - static int cnt = 0; - return fmt::format("{:04}", ++cnt); -} - doris::RowsetMetaCloudPB create_rowset(int64_t tablet_id, int64_t start_version, int64_t end_version, int num_rows = 100) { doris::RowsetMetaCloudPB rowset; @@ -159,6 +193,15 @@ void commit_rowset(MetaService* meta_service, const doris::RowsetMetaCloudPB& ro meta_service->commit_rowset(&cntl, &req, &res, nullptr); } +void prepare_rowset(MetaService* meta_service, const doris::RowsetMetaCloudPB& rowset, + CreateRowsetResponse& res, int txn_id = 1) { + brpc::Controller cntl; + CreateRowsetRequest req; + req.set_txn_id(txn_id); + req.mutable_rowset_meta()->CopyFrom(rowset); + meta_service->prepare_rowset(&cntl, &req, &res, nullptr); +} + void insert_rowsets(TxnKv* txn_kv, int64_t table_id, int64_t index_id, int64_t partition_id, int64_t tablet_id, const std::vector& rowsets) { std::unique_ptr txn; @@ -1863,6 +1906,8 @@ TEST(MetaServiceJobTest, DeleteBitmapUpdateLockCompatibilityTest) { for (int64_t i = 0; i < 5; ++i) { output_rowsets.push_back(create_rowset(new_tablet_id, i + 2, i + 2)); CreateRowsetResponse create_rowset_response; + prepare_rowset(meta_service.get(), output_rowsets.back(), create_rowset_response); + ASSERT_EQ(create_rowset_response.status().code(), MetaServiceCode::OK) << i; commit_rowset(meta_service.get(), output_rowsets.back(), create_rowset_response); ASSERT_EQ(create_rowset_response.status().code(), MetaServiceCode::OK) << i; } @@ -3507,6 +3552,8 @@ TEST(MetaServiceJobTest, SchemaChangeJobTest) { for (int64_t i = 0; i < 5; ++i) { output_rowsets.push_back(create_rowset(new_tablet_id, i + 2, i + 2)); CreateRowsetResponse res; + prepare_rowset(meta_service.get(), output_rowsets.back(), res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << i; commit_rowset(meta_service.get(), output_rowsets.back(), res); ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << i; } @@ -3584,6 +3631,8 @@ TEST(MetaServiceJobTest, SchemaChangeJobTest) { output_rowsets.push_back(create_rowset(new_tablet_id, 13, 13)); for (auto& rs : output_rowsets) { CreateRowsetResponse res; + prepare_rowset(meta_service.get(), rs, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << rs.end_version(); commit_rowset(meta_service.get(), rs, res); ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << rs.end_version(); } @@ -3727,6 +3776,8 @@ TEST(MetaServiceJobTest, RetrySchemaChangeJobTest) { be1_output_rowsets.push_back(create_rowset(new_tablet_id, 11, 11)); for (auto& rs : be1_output_rowsets) { CreateRowsetResponse res; + prepare_rowset(meta_service.get(), rs, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << rs.end_version(); commit_rowset(meta_service.get(), rs, res); ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << rs.end_version(); } @@ -3740,17 +3791,20 @@ TEST(MetaServiceJobTest, RetrySchemaChangeJobTest) { { CreateRowsetResponse res; // [2-8] has committed by BE1 - commit_rowset(meta_service.get(), create_rowset(new_tablet_id, 2, 8), res); + prepare_rowset(meta_service.get(), create_rowset(new_tablet_id, 2, 8), res); + // commit_rowset(meta_service.get(), create_rowset(new_tablet_id, 2, 8), res); ASSERT_EQ(res.status().code(), MetaServiceCode::ALREADY_EXISTED); ASSERT_TRUE(res.has_existed_rowset_meta()); ASSERT_EQ(res.existed_rowset_meta().rowset_id_v2(), be1_output_rowsets[0].rowset_id_v2()); be2_output_rowsets.push_back(res.existed_rowset_meta()); res.Clear(); be2_output_rowsets.push_back(create_rowset(new_tablet_id, 9, 12)); + prepare_rowset(meta_service.get(), be2_output_rowsets.back(), res); commit_rowset(meta_service.get(), be2_output_rowsets.back(), res); ASSERT_EQ(res.status().code(), MetaServiceCode::OK); res.Clear(); be2_output_rowsets.push_back(create_rowset(new_tablet_id, 13, 13)); + prepare_rowset(meta_service.get(), be2_output_rowsets.back(), res); commit_rowset(meta_service.get(), be2_output_rowsets.back(), res); ASSERT_EQ(res.status().code(), MetaServiceCode::OK); } @@ -3874,6 +3928,8 @@ void testSchemaChangeJobWithMoWTest(int lock_version) { for (int64_t i = 0; i < 5; ++i) { output_rowsets.push_back(create_rowset(new_tablet_id, i + 2, i + 2)); CreateRowsetResponse res; + prepare_rowset(meta_service.get(), output_rowsets.back(), res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << i; commit_rowset(meta_service.get(), output_rowsets.back(), res); ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << i; } @@ -3960,6 +4016,8 @@ void testSchemaChangeJobWithMoWTest(int lock_version) { for (int64_t i = 0; i < 5; ++i) { output_rowsets.push_back(create_rowset(new_tablet_id, i + 2, i + 2)); CreateRowsetResponse res; + prepare_rowset(meta_service.get(), output_rowsets.back(), res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << i; commit_rowset(meta_service.get(), output_rowsets.back(), res); ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << i; } @@ -4145,6 +4203,8 @@ TEST(MetaServiceJobTest, ConcurrentCompactionTest) { // Provide output rowset auto output_rowset = create_rowset(tablet_id, 5, 10); CreateRowsetResponse rowset_res; + prepare_rowset(meta_service.get(), output_rowset, rowset_res); + ASSERT_EQ(rowset_res.status().code(), MetaServiceCode::OK); commit_rowset(meta_service.get(), output_rowset, rowset_res); ASSERT_EQ(rowset_res.status().code(), MetaServiceCode::OK); @@ -4257,6 +4317,8 @@ TEST(MetaServiceJobTest, ConcurrentCompactionTest) { // Provide output rowset auto output_rowset = create_rowset(tablet_id, 2, 4); CreateRowsetResponse rowset_res; + prepare_rowset(meta_service.get(), output_rowset, rowset_res); + ASSERT_EQ(rowset_res.status().code(), MetaServiceCode::OK); commit_rowset(meta_service.get(), output_rowset, rowset_res); ASSERT_EQ(rowset_res.status().code(), MetaServiceCode::OK); @@ -5402,4 +5464,1308 @@ TEST(MetaServiceJobTest, ResetStreamingJobOffsetTest) { } } +// Test: Complete flow - begin_txn -> prepare_rowset -> recycle x 2 -> abort -> verify commit fails +TEST(MetaServiceJobTest, AbortTxnForRelatedRowsetTest1) { + auto meta_service = get_meta_service(); + auto* sp = SyncPoint::get_instance(); + DORIS_CLOUD_DEFER { + SyncPoint::get_instance()->clear_all_call_backs(); + config::enable_recycle_delete_rowset_key_check = false; + config::force_immediate_recycle = false; + }; + config::force_immediate_recycle = true; + config::enable_recycle_delete_rowset_key_check = true; + sp->set_call_back("get_instance_id", [&](auto&& args) { + auto* ret = try_any_cast_ret(args); + ret->first = instance_id; + ret->second = true; + }); + sp->enable_processing(); + + int64_t db_id = 1; + int64_t table_id = 12340; + int64_t index_id = 12350; + int64_t partition_id = 12360; + int64_t tablet_id = 12370; + + // Create tablet first + create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id); + + { + LOG(INFO) << "Test: Complete flow with abort_txn_for_related_rowset"; + + create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id); + + // Step 1: begin_txn + int64_t txn_id = -1; + std::string label = "test_abort_txn_label"; + { + brpc::Controller cntl; + BeginTxnRequest req; + BeginTxnResponse res; + req.set_cloud_unique_id("test_cloud_unique_id"); + auto* txn_info = req.mutable_txn_info(); + txn_info->set_db_id(db_id); + txn_info->set_label(label); + txn_info->add_table_ids(table_id); + txn_info->set_timeout_ms(36000); + meta_service->begin_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + txn_id = res.txn_id(); + LOG(INFO) << "Step 1: Transaction started, txn_id=" << txn_id; + } + + // Step 2: prepare_rowset + doris::RowsetMetaCloudPB rowset_meta; + { + rowset_meta = create_rowset(txn_id, tablet_id, partition_id, -1, 100); + rowset_meta.mutable_load_id()->CopyFrom(PUniqueId()); + rowset_meta.mutable_load_id()->set_hi(11111); + rowset_meta.mutable_load_id()->set_lo(22222); + rowset_meta.set_rowset_state(RowsetStatePB::PREPARED); + rowset_meta.set_resource_id(std::string(RESOURCE_ID)); + + brpc::Controller cntl; + CreateRowsetResponse res; + auto* arena = res.GetArena(); + auto* req = google::protobuf::Arena::CreateMessage(arena); + req->mutable_rowset_meta()->CopyFrom(rowset_meta); + meta_service->prepare_rowset(&cntl, req, &res, nullptr); + if (!arena) delete req; + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + LOG(INFO) << "Step 2: Rowset prepared"; + } + + // Step 4: Abort transaction via abort_txn_for_related_rowset + { + InstanceInfoPB instance_info; + instance_info.set_instance_id(instance_id); + + InstanceRecycler recycler(meta_service->txn_kv(), instance_info, thread_group, + std::make_shared(meta_service->txn_kv())); + ASSERT_EQ(recycler.init(), 0); + std::shared_ptr accessor = std::make_shared(); + recycler.TEST_add_accessor(RESOURCE_ID, accessor); + + // first for set is_recycled true + // second for abort txn and recycle data + for (size_t i = 0; i < 2; i++) { + int ret = recycler.recycle_rowsets(); + ASSERT_EQ(ret, 0); + } + + LOG(INFO) << "Step 4: Transaction aborted via recycler"; + } + + // Step 5: Try to commit_rowset (should fail because txn is aborted) + { + brpc::Controller cntl; + CreateRowsetResponse res; + auto* arena = res.GetArena(); + auto* req = google::protobuf::Arena::CreateMessage(arena); + req->mutable_rowset_meta()->CopyFrom(rowset_meta); + meta_service->commit_rowset(&cntl, req, &res, nullptr); + if (!arena) delete req; + // Should fail because recycle_rowset_key is recycled + ASSERT_NE(res.status().code(), MetaServiceCode::OK); + ASSERT_TRUE(res.status().msg().find("recycle rowset key not found") != + std::string::npos) + << res.status().msg(); + LOG(INFO) << "Step 5: commit_rowset correctly failed for aborted txn"; + } + + // Step 6: Try to commit_txn (should also fail) + { + brpc::Controller cntl; + CommitTxnRequest req; + CommitTxnResponse res; + req.set_db_id(db_id); + req.set_txn_id(txn_id); + meta_service->commit_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + // Should fail because transaction is already aborted + ASSERT_NE(res.status().code(), MetaServiceCode::OK); + ASSERT_TRUE(res.status().msg().find("is already aborted") != std::string::npos) + << res.status().msg(); + LOG(INFO) << "Step 6: commit_txn correctly failed for aborted txn"; + } + } +} + +// Test: Complete flow - start job -> prepare_rowset -> recycle x 2 -> abort job -> verify +TEST(MetaServiceJobTest, AbortJobForRelatedRowsetTest1) { + auto meta_service = get_meta_service(); + auto* sp = SyncPoint::get_instance(); + DORIS_CLOUD_DEFER { + SyncPoint::get_instance()->clear_all_call_backs(); + config::enable_recycle_delete_rowset_key_check = false; + config::force_immediate_recycle = false; + }; + config::enable_recycle_delete_rowset_key_check = true; + config::force_immediate_recycle = true; + sp->set_call_back("get_instance_id", [&](auto&& args) { + auto* ret = try_any_cast_ret(args); + ret->first = instance_id; + ret->second = true; + }); + sp->enable_processing(); + + int64_t table_id = 12440; + int64_t index_id = 12450; + int64_t partition_id = 12460; + int64_t tablet_id = 12470; + + // Create tablet first + create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id); + + { + LOG(INFO) << "Test: Complete flow with abort_job_for_related_rowset"; + + create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id); + + std::string job_id = "test_complete_flow_job_456"; + + // Step 0: Create input rowsets for compaction + { + std::vector input_rowsets; + input_rowsets.push_back(create_rowset(tablet_id, 2, 2, 100)); + input_rowsets.push_back(create_rowset(tablet_id, 3, 3, 100)); + input_rowsets.push_back(create_rowset(tablet_id, 4, 4, 100)); + input_rowsets[0].set_resource_id(std::string(RESOURCE_ID)); + input_rowsets[1].set_resource_id(std::string(RESOURCE_ID)); + input_rowsets[2].set_resource_id(std::string(RESOURCE_ID)); + insert_rowsets(meta_service->txn_kv().get(), table_id, index_id, partition_id, + tablet_id, input_rowsets); + LOG(INFO) << "Step 0: Input rowsets created (version 2-4)"; + } + + // Step 1: Start a compaction job + { + StartTabletJobResponse res; + start_compaction_job(meta_service.get(), tablet_id, job_id, "test_initiator", 0, 0, + TabletCompactionJobPB::CUMULATIVE, res, {2, 4}); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + LOG(INFO) << "Step 1: Compaction job started, job_id=" << job_id; + } + + // Step 2: prepare_rowset with job_id (output rowset for compaction) + doris::RowsetMetaCloudPB rowset_meta; + { + rowset_meta = create_rowset(tablet_id, 5, 5, 300); // Output rowset with merged data + rowset_meta.set_job_id(job_id); + rowset_meta.set_resource_id(std::string(RESOURCE_ID)); + + brpc::Controller cntl; + CreateRowsetResponse res; + auto* arena = res.GetArena(); + auto* req = google::protobuf::Arena::CreateMessage(arena); + req->mutable_rowset_meta()->CopyFrom(rowset_meta); + meta_service->prepare_rowset(&cntl, req, &res, nullptr); + if (!arena) delete req; + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + LOG(INFO) << "Step 2: Rowset with job_id prepared"; + } + + // Step 4: Abort job via abort_job_for_related_rowset + { + InstanceInfoPB instance_info; + instance_info.set_instance_id(instance_id); + + InstanceRecycler recycler(meta_service->txn_kv(), instance_info, thread_group, + std::make_shared(meta_service->txn_kv())); + ASSERT_EQ(recycler.init(), 0); + std::shared_ptr accessor = std::make_shared(); + recycler.TEST_add_accessor(RESOURCE_ID, accessor); + + // first for set is_recycled true + // second for abort txn and recycle data + for (size_t i = 0; i < 2; i++) { + int ret = recycler.recycle_rowsets(); + ASSERT_EQ(ret, 0); + } + + LOG(INFO) << "Step 4: Job aborted via recycler"; + } + + // Step 5: Try to commit_rowset (may succeed but job is aborted) + { + brpc::Controller cntl; + CreateRowsetResponse res; + auto* arena = res.GetArena(); + auto* req = google::protobuf::Arena::CreateMessage(arena); + req->mutable_rowset_meta()->CopyFrom(rowset_meta); + meta_service->commit_rowset(&cntl, req, &res, nullptr); + if (!arena) delete req; + // Should fail because recycle_rowset_key is recycled + ASSERT_NE(res.status().code(), MetaServiceCode::OK); + ASSERT_TRUE(res.status().msg().find("recycle rowset key not found") != + std::string::npos) + << res.status().msg(); + LOG(INFO) << "Step 5: commit_rowset executed with code=" << res.status().code(); + } + + // Step 6: Try to finish tablet job + { + FinishTabletJobResponse res; + finish_compaction_job(meta_service.get(), tablet_id, job_id, "test_initiator", 0, 0, + TabletCompactionJobPB::CUMULATIVE, res, + FinishTabletJobRequest::COMMIT, {2, 4}, rowset_meta.txn_id()); + ASSERT_NE(res.status().code(), MetaServiceCode::OK); + ASSERT_TRUE(res.status().msg().find("there is no running compaction") != + std::string::npos) + << res.status().msg(); + LOG(INFO) << "Step 6: Tablet job finished with code=" << res.status().code(); + } + } +} + +// Test: Complete flow - begin_txn -> prepare_rowset -> commit_rowset -> recycle x 2 -> abort -> verify commit fails +TEST(MetaServiceJobTest, AbortTxnForRelatedRowsetTest2) { + auto meta_service = get_meta_service(); + auto* sp = SyncPoint::get_instance(); + DORIS_CLOUD_DEFER { + SyncPoint::get_instance()->clear_all_call_backs(); + config::enable_recycle_delete_rowset_key_check = false; + config::force_immediate_recycle = false; + }; + config::force_immediate_recycle = true; + config::enable_recycle_delete_rowset_key_check = true; + sp->set_call_back("get_instance_id", [&](auto&& args) { + auto* ret = try_any_cast_ret(args); + ret->first = instance_id; + ret->second = true; + }); + sp->enable_processing(); + + int64_t db_id = 1; + int64_t table_id = 12340; + int64_t index_id = 12350; + int64_t partition_id = 12360; + int64_t tablet_id = 12370; + + // Create tablet first + create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id); + + { + LOG(INFO) << "Test: Complete flow with abort_txn_for_related_rowset"; + + create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id); + + // Step 1: begin_txn + int64_t txn_id = -1; + std::string label = "test_abort_txn_label"; + { + brpc::Controller cntl; + BeginTxnRequest req; + BeginTxnResponse res; + req.set_cloud_unique_id("test_cloud_unique_id"); + auto* txn_info = req.mutable_txn_info(); + txn_info->set_db_id(db_id); + txn_info->set_label(label); + txn_info->add_table_ids(table_id); + txn_info->set_timeout_ms(36000); + meta_service->begin_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + txn_id = res.txn_id(); + LOG(INFO) << "Step 1: Transaction started, txn_id=" << txn_id; + } + + // Step 2: prepare_rowset + doris::RowsetMetaCloudPB rowset_meta; + { + rowset_meta = create_rowset(txn_id, tablet_id, partition_id, -1, 100); + rowset_meta.mutable_load_id()->CopyFrom(PUniqueId()); + rowset_meta.mutable_load_id()->set_hi(11111); + rowset_meta.mutable_load_id()->set_lo(22222); + rowset_meta.set_rowset_state(RowsetStatePB::PREPARED); + rowset_meta.set_resource_id(std::string(RESOURCE_ID)); + + brpc::Controller cntl; + CreateRowsetResponse res; + auto* arena = res.GetArena(); + auto* req = google::protobuf::Arena::CreateMessage(arena); + req->mutable_rowset_meta()->CopyFrom(rowset_meta); + meta_service->prepare_rowset(&cntl, req, &res, nullptr); + if (!arena) delete req; + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + LOG(INFO) << "Step 2: Rowset prepared"; + } + + // Step 4: Try to commit_rowset + { + brpc::Controller cntl; + CreateRowsetResponse res; + auto* arena = res.GetArena(); + auto* req = google::protobuf::Arena::CreateMessage(arena); + req->mutable_rowset_meta()->CopyFrom(rowset_meta); + meta_service->commit_rowset(&cntl, req, &res, nullptr); + if (!arena) delete req; + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + LOG(INFO) << "Step 5: commit_rowset successfully"; + } + + // Step 5: Abort transaction via abort_txn_for_related_rowset + { + InstanceInfoPB instance_info; + instance_info.set_instance_id(instance_id); + + InstanceRecycler recycler(meta_service->txn_kv(), instance_info, thread_group, + std::make_shared(meta_service->txn_kv())); + ASSERT_EQ(recycler.init(), 0); + std::shared_ptr accessor = std::make_shared(); + recycler.TEST_add_accessor(RESOURCE_ID, accessor); + + // first for set is_recycled true + // second for abort txn and recycle data + for (size_t i = 0; i < 2; i++) { + int ret = recycler.recycle_tmp_rowsets(); + ASSERT_EQ(ret, 0); + } + + LOG(INFO) << "Step 4: Transaction aborted via recycler"; + } + + // Step 6: Try to commit_txn (should fail) + { + brpc::Controller cntl; + CommitTxnRequest req; + CommitTxnResponse res; + req.set_db_id(db_id); + req.set_txn_id(txn_id); + meta_service->commit_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + // Should fail because transaction is already aborted + ASSERT_NE(res.status().code(), MetaServiceCode::OK); + ASSERT_TRUE(res.status().msg().find("is already aborted") != std::string::npos) + << res.status().msg(); + LOG(INFO) << "Step 6: commit_txn correctly failed for aborted txn"; + } + } +} + +// Test: Complete flow - start compaction job -> prepare_rowset -> commit_rowset -> recycle x 2 -> abort job -> verify +TEST(MetaServiceJobTest, AbortCompactionJobForRelatedRowsetTest2) { + auto meta_service = get_meta_service(); + auto* sp = SyncPoint::get_instance(); + DORIS_CLOUD_DEFER { + SyncPoint::get_instance()->clear_all_call_backs(); + config::enable_recycle_delete_rowset_key_check = false; + config::force_immediate_recycle = false; + }; + config::enable_recycle_delete_rowset_key_check = true; + config::force_immediate_recycle = true; + sp->set_call_back("get_instance_id", [&](auto&& args) { + auto* ret = try_any_cast_ret(args); + ret->first = instance_id; + ret->second = true; + }); + sp->enable_processing(); + + // old version be don't set job id for rowset meta + std::array is_set_job_id = {false, true}; + for (const auto& is_set : is_set_job_id) { + int64_t table_id = 12440; + int64_t index_id = 12450; + int64_t partition_id = 12460; + int64_t tablet_id = 12470; + std::string job_id = "test_complete_flow_job_456"; + + // Create tablet first + create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id); + + { + LOG(INFO) << "Test: Complete flow with abort_job_for_related_rowset2"; + + create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id); + + // Step 0: Create input rowsets for compaction + { + std::vector input_rowsets; + input_rowsets.push_back(create_rowset(tablet_id, 2, 2, 100)); + input_rowsets.push_back(create_rowset(tablet_id, 3, 3, 100)); + input_rowsets.push_back(create_rowset(tablet_id, 4, 4, 100)); + input_rowsets[0].set_resource_id(std::string(RESOURCE_ID)); + input_rowsets[1].set_resource_id(std::string(RESOURCE_ID)); + input_rowsets[2].set_resource_id(std::string(RESOURCE_ID)); + insert_rowsets(meta_service->txn_kv().get(), table_id, index_id, partition_id, + tablet_id, input_rowsets); + LOG(INFO) << "Step 0: Input rowsets created (version 2-4)"; + } + + // Step 1: Start a compaction job + { + StartTabletJobResponse res; + start_compaction_job(meta_service.get(), tablet_id, job_id, "test_initiator", 0, 0, + TabletCompactionJobPB::CUMULATIVE, res, {2, 4}); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + LOG(INFO) << "Step 1: Compaction job started, job_id=" << job_id; + } + + // Step 2: prepare_rowset with job_id (output rowset for compaction) + doris::RowsetMetaCloudPB rowset_meta; + { + rowset_meta = create_rowset(tablet_id, 5, 5, 300); // Output rowset with merged data + if (is_set) { + rowset_meta.set_job_id(job_id); + } + rowset_meta.set_resource_id(std::string(RESOURCE_ID)); + + brpc::Controller cntl; + CreateRowsetResponse res; + auto* arena = res.GetArena(); + auto* req = google::protobuf::Arena::CreateMessage(arena); + req->mutable_rowset_meta()->CopyFrom(rowset_meta); + meta_service->prepare_rowset(&cntl, req, &res, nullptr); + if (!arena) delete req; + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + LOG(INFO) << "Step 2: Rowset with job_id prepared"; + } + + // Step 4: Try to commit_rowset (may succeed but job is aborted) + { + brpc::Controller cntl; + CreateRowsetResponse res; + auto* arena = res.GetArena(); + auto* req = google::protobuf::Arena::CreateMessage(arena); + req->mutable_rowset_meta()->CopyFrom(rowset_meta); + meta_service->commit_rowset(&cntl, req, &res, nullptr); + if (!arena) delete req; + // commit_rowset may succeed, but the job itself is aborted + LOG(INFO) << "Step 5: commit_rowset executed with code=" << res.status().code(); + } + + // Step 5: Abort job via abort_job_for_related_rowset + { + InstanceInfoPB instance_info; + instance_info.set_instance_id(instance_id); + + InstanceRecycler recycler( + meta_service->txn_kv(), instance_info, thread_group, + std::make_shared(meta_service->txn_kv())); + ASSERT_EQ(recycler.init(), 0); + std::shared_ptr accessor = std::make_shared(); + recycler.TEST_add_accessor(RESOURCE_ID, accessor); + + // first for set is_recycled true + // second for abort txn and recycle data + for (size_t i = 0; i < 2; i++) { + int ret = recycler.recycle_tmp_rowsets(); + ASSERT_EQ(ret, 0); + } + + LOG(INFO) << "Step 4: Job aborted via recycler"; + } + + // Step 6: Try to finish tablet job + { + FinishTabletJobResponse res; + finish_compaction_job(meta_service.get(), tablet_id, job_id, "test_initiator", 0, 0, + TabletCompactionJobPB::CUMULATIVE, res, + FinishTabletJobRequest::COMMIT, {2, 4}, rowset_meta.txn_id()); + ASSERT_NE(res.status().code(), MetaServiceCode::OK); + if (is_set) { + ASSERT_TRUE(res.status().msg().find("there is no running compaction") != + std::string::npos) + << res.status().msg(); + } else { + ASSERT_TRUE(res.status().msg().find("failed to get tmp rowset key") != + std::string::npos) + << res.status().msg(); + } + + LOG(INFO) << "Step 6: Tablet job finished with code=" << res.status().code(); + } + } + } +} + +// Test: Complete flow - start schema change job -> prepare_rowset -> commit_rowset -> recycle x 2 -> abort job -> verify +TEST(MetaServiceJobTest, AbortSchemaChangeJobForRelatedRowsetTest2) { + auto meta_service = get_meta_service(); + auto* sp = SyncPoint::get_instance(); + DORIS_CLOUD_DEFER { + SyncPoint::get_instance()->clear_all_call_backs(); + config::enable_recycle_delete_rowset_key_check = false; + config::force_immediate_recycle = false; + }; + config::enable_recycle_delete_rowset_key_check = true; + config::force_immediate_recycle = true; + sp->set_call_back("get_instance_id", [&](auto&& args) { + auto* ret = try_any_cast_ret(args); + ret->first = instance_id; + ret->second = true; + }); + sp->enable_processing(); + + // old version be don't set job id for rowset meta + std::array is_set_job_id = {0, 1}; + for (const auto& is_set : is_set_job_id) { + int64_t db_id = 12441; + int64_t table_id = 12440; + int64_t index_id = 12450; + int64_t partition_id = 12460; + int64_t tablet_id = 12470 + is_set * 2; + int64_t new_tablet_id = tablet_id + 10; + std::string job_id = "job_sc" + std::to_string(is_set); + + // Create tablet first + create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id, false, true); + create_tablet(meta_service.get(), table_id, index_id, partition_id, new_tablet_id, false, + true); + + { + LOG(INFO) << "Test: Complete flow with abort_job_for_related_rowset2"; + + create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id); + + // Step 1: begin_txn + int64_t txn_id = -1; + std::string label = "test_abort_txn_label" + std::to_string(is_set); + { + brpc::Controller cntl; + BeginTxnRequest req; + BeginTxnResponse res; + req.set_cloud_unique_id("test_cloud_unique_id"); + auto* txn_info = req.mutable_txn_info(); + txn_info->set_db_id(db_id); + txn_info->set_label(label); + txn_info->add_table_ids(table_id); + txn_info->set_timeout_ms(36000); + meta_service->begin_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + txn_id = res.txn_id(); + LOG(INFO) << "Step 1: Transaction started, txn_id=" << txn_id; + } + + // Step 2: prepare_rowset + doris::RowsetMetaCloudPB rowset_meta; + { + rowset_meta = create_rowset(tablet_id, 5, 5, 300); // Output rowset with merged data + rowset_meta.set_resource_id(std::string(RESOURCE_ID)); + + brpc::Controller cntl; + CreateRowsetResponse res; + auto* arena = res.GetArena(); + auto* req = google::protobuf::Arena::CreateMessage(arena); + req->mutable_rowset_meta()->CopyFrom(rowset_meta); + meta_service->prepare_rowset(&cntl, req, &res, nullptr); + if (!arena) delete req; + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + LOG(INFO) << "Step 2: Rowset with job_id prepared"; + } + + // Step 3: Try to commit_rowset + { + brpc::Controller cntl; + CreateRowsetResponse res; + auto* arena = res.GetArena(); + auto* req = google::protobuf::Arena::CreateMessage(arena); + req->mutable_rowset_meta()->CopyFrom(rowset_meta); + meta_service->commit_rowset(&cntl, req, &res, nullptr); + if (!arena) delete req; + // commit_rowset may succeed, but the job itself is aborted + LOG(INFO) << "Step 5: commit_rowset executed with code=" << res.status().code(); + } + + // Step 4: Try to commit_txn + { + brpc::Controller cntl; + CommitTxnRequest req; + CommitTxnResponse res; + req.set_db_id(db_id); + req.set_txn_id(txn_id); + meta_service->commit_txn( + reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, &res, + nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + LOG(INFO) << "Step 6: commit_txn correctly failed for aborted txn"; + } + + // Step 5: Start a compaction job + { + StartTabletJobResponse sc_res; + ASSERT_NO_FATAL_FAILURE(start_schema_change_job( + meta_service.get(), table_id, index_id, partition_id, tablet_id, + new_tablet_id, job_id, "BE1", sc_res, 8 + is_set)); + ASSERT_EQ(sc_res.status().code(), MetaServiceCode::OK); + LOG(INFO) << "Step 1: SC job started, job_id=" << job_id; + } + + // Step 6: prepare_rowset with job_id (output rowset for sc) + { + rowset_meta = + create_rowset(new_tablet_id, 5, 5, 300); // Output rowset with merged data + if (is_set) { + rowset_meta.set_job_id(job_id); + } + rowset_meta.set_resource_id(std::string(RESOURCE_ID)); + + brpc::Controller cntl; + CreateRowsetResponse res; + auto* arena = res.GetArena(); + auto* req = google::protobuf::Arena::CreateMessage(arena); + req->mutable_rowset_meta()->CopyFrom(rowset_meta); + meta_service->prepare_rowset(&cntl, req, &res, nullptr); + if (!arena) delete req; + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + LOG(INFO) << "Step 2: Rowset with job_id prepared"; + } + + // Step 7: commit_rowset with job_id (output rowset for sc) + { + brpc::Controller cntl; + CreateRowsetResponse res; + auto* arena = res.GetArena(); + auto* req = google::protobuf::Arena::CreateMessage(arena); + req->mutable_rowset_meta()->CopyFrom(rowset_meta); + meta_service->commit_rowset(&cntl, req, &res, nullptr); + if (!arena) delete req; + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + LOG(INFO) << "Step 2: Rowset with job_id prepared"; + } + + // Step 8: Abort job via abort_job_for_related_rowset + { + InstanceInfoPB instance_info; + instance_info.set_instance_id(instance_id); + + InstanceRecycler recycler( + meta_service->txn_kv(), instance_info, thread_group, + std::make_shared(meta_service->txn_kv())); + ASSERT_EQ(recycler.init(), 0); + std::shared_ptr accessor = std::make_shared(); + recycler.TEST_add_accessor(RESOURCE_ID, accessor); + + // first for set is_recycled true + // second for abort txn and recycle data + for (size_t i = 0; i < 2; i++) { + int ret = recycler.recycle_tmp_rowsets(); + ASSERT_EQ(ret, 0); + } + + LOG(INFO) << "Step 4: Job aborted via recycler"; + } + + // Step 9: Try to finish tablet job + { + FinishTabletJobResponse finish_res; + finish_schema_change_job(meta_service.get(), tablet_id, new_tablet_id, job_id, + "BE1", {}, finish_res, FinishTabletJobRequest::COMMIT); + if (is_set) { + ASSERT_NE(finish_res.status().code(), MetaServiceCode::OK); + } else { + ASSERT_EQ(finish_res.status().code(), MetaServiceCode::OK); + } + LOG(INFO) << "Step 6: SC job finished with code=" << finish_res.status().code() + << " " << finish_res.status().msg(); + } + } + } +} + +// Test: Complete flow - begin_txn -> prepare_rowset -> recycle x 1 -> commit_rowset -> commit_txn -> verify commit fails +TEST(MetaServiceJobTest, AbortTxnForRelatedRowsetTest3) { + auto meta_service = get_meta_service(); + auto* sp = SyncPoint::get_instance(); + DORIS_CLOUD_DEFER { + SyncPoint::get_instance()->clear_all_call_backs(); + config::enable_recycle_delete_rowset_key_check = false; + config::force_immediate_recycle = false; + }; + config::force_immediate_recycle = true; + config::enable_recycle_delete_rowset_key_check = true; + sp->set_call_back("get_instance_id", [&](auto&& args) { + auto* ret = try_any_cast_ret(args); + ret->first = instance_id; + ret->second = true; + }); + sp->enable_processing(); + + int64_t db_id = 1; + int64_t table_id = 12340; + int64_t index_id = 12350; + int64_t partition_id = 12360; + int64_t tablet_id = 12370; + + // Create tablet first + create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id); + + { + LOG(INFO) << "Test: Complete flow with abort_txn_for_related_rowset"; + + create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id); + + // Step 1: begin_txn + int64_t txn_id = -1; + std::string label = "test_abort_txn_label"; + { + brpc::Controller cntl; + BeginTxnRequest req; + BeginTxnResponse res; + req.set_cloud_unique_id("test_cloud_unique_id"); + auto* txn_info = req.mutable_txn_info(); + txn_info->set_db_id(db_id); + txn_info->set_label(label); + txn_info->add_table_ids(table_id); + txn_info->set_timeout_ms(36000); + meta_service->begin_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + txn_id = res.txn_id(); + LOG(INFO) << "Step 1: Transaction started, txn_id=" << txn_id; + } + + // Step 2: prepare_rowset + doris::RowsetMetaCloudPB rowset_meta; + { + rowset_meta = create_rowset(txn_id, tablet_id, partition_id, -1, 100); + rowset_meta.mutable_load_id()->CopyFrom(PUniqueId()); + rowset_meta.mutable_load_id()->set_hi(11111); + rowset_meta.mutable_load_id()->set_lo(22222); + rowset_meta.set_rowset_state(RowsetStatePB::PREPARED); + rowset_meta.set_resource_id(std::string(RESOURCE_ID)); + + brpc::Controller cntl; + CreateRowsetResponse res; + auto* arena = res.GetArena(); + auto* req = google::protobuf::Arena::CreateMessage(arena); + req->mutable_rowset_meta()->CopyFrom(rowset_meta); + meta_service->prepare_rowset(&cntl, req, &res, nullptr); + if (!arena) delete req; + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + LOG(INFO) << "Step 2: Rowset prepared"; + } + + // Step 4: Abort transaction via abort_txn_for_related_rowset + { + InstanceInfoPB instance_info; + instance_info.set_instance_id(instance_id); + + InstanceRecycler recycler(meta_service->txn_kv(), instance_info, thread_group, + std::make_shared(meta_service->txn_kv())); + ASSERT_EQ(recycler.init(), 0); + std::shared_ptr accessor = std::make_shared(); + recycler.TEST_add_accessor(RESOURCE_ID, accessor); + + // first for set is_recycled true + int ret = recycler.recycle_rowsets(); + ASSERT_EQ(ret, 0); + + LOG(INFO) << "Step 4: Transaction aborted via recycler"; + } + + // Step 5: Try to commit_rowset (should fail because txn is aborted) + { + brpc::Controller cntl; + CreateRowsetResponse res; + auto* arena = res.GetArena(); + auto* req = google::protobuf::Arena::CreateMessage(arena); + req->mutable_rowset_meta()->CopyFrom(rowset_meta); + meta_service->commit_rowset(&cntl, req, &res, nullptr); + if (!arena) delete req; + // Should fail because transaction is aborted + ASSERT_NE(res.status().code(), MetaServiceCode::OK); + ASSERT_TRUE(res.status().msg().find("rowset has already been marked as recycled") != + std::string::npos) + << res.status().msg(); + LOG(INFO) << "Step 5: commit_rowset correctly failed for aborted txn"; + } + } +} + +// Test: Complete flow - start job -> prepare_rowset -> recycle x 1 -> commit_rowset -> finish job -> verify +TEST(MetaServiceJobTest, AbortJobForRelatedRowsetTest3) { + auto meta_service = get_meta_service(); + auto* sp = SyncPoint::get_instance(); + DORIS_CLOUD_DEFER { + SyncPoint::get_instance()->clear_all_call_backs(); + config::enable_recycle_delete_rowset_key_check = false; + config::force_immediate_recycle = false; + }; + config::enable_recycle_delete_rowset_key_check = true; + config::force_immediate_recycle = true; + sp->set_call_back("get_instance_id", [&](auto&& args) { + auto* ret = try_any_cast_ret(args); + ret->first = instance_id; + ret->second = true; + }); + sp->enable_processing(); + + int64_t table_id = 12440; + int64_t index_id = 12450; + int64_t partition_id = 12460; + int64_t tablet_id = 12470; + + // Create tablet first + create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id); + + { + LOG(INFO) << "Test: Complete flow with abort_job_for_related_rowset"; + + create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id); + + std::string job_id = "test_complete_flow_job_456"; + + // Step 0: Create input rowsets for compaction + { + std::vector input_rowsets; + input_rowsets.push_back(create_rowset(tablet_id, 2, 2, 100)); + input_rowsets.push_back(create_rowset(tablet_id, 3, 3, 100)); + input_rowsets.push_back(create_rowset(tablet_id, 4, 4, 100)); + input_rowsets[0].set_resource_id(std::string(RESOURCE_ID)); + input_rowsets[1].set_resource_id(std::string(RESOURCE_ID)); + input_rowsets[2].set_resource_id(std::string(RESOURCE_ID)); + insert_rowsets(meta_service->txn_kv().get(), table_id, index_id, partition_id, + tablet_id, input_rowsets); + LOG(INFO) << "Step 0: Input rowsets created (version 2-4)"; + } + + // Step 1: Start a compaction job + { + StartTabletJobResponse res; + start_compaction_job(meta_service.get(), tablet_id, job_id, "test_initiator", 0, 0, + TabletCompactionJobPB::CUMULATIVE, res, {2, 4}); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + LOG(INFO) << "Step 1: Compaction job started, job_id=" << job_id; + } + + // Step 2: prepare_rowset with job_id (output rowset for compaction) + doris::RowsetMetaCloudPB rowset_meta; + { + rowset_meta = create_rowset(tablet_id, 5, 5, 300); // Output rowset with merged data + rowset_meta.set_job_id(job_id); + rowset_meta.set_resource_id(std::string(RESOURCE_ID)); + + brpc::Controller cntl; + CreateRowsetResponse res; + auto* arena = res.GetArena(); + auto* req = google::protobuf::Arena::CreateMessage(arena); + req->mutable_rowset_meta()->CopyFrom(rowset_meta); + meta_service->prepare_rowset(&cntl, req, &res, nullptr); + if (!arena) delete req; + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + LOG(INFO) << "Step 2: Rowset with job_id prepared"; + } + + // Step 4: Abort job via abort_job_for_related_rowset + { + InstanceInfoPB instance_info; + instance_info.set_instance_id(instance_id); + + InstanceRecycler recycler(meta_service->txn_kv(), instance_info, thread_group, + std::make_shared(meta_service->txn_kv())); + ASSERT_EQ(recycler.init(), 0); + std::shared_ptr accessor = std::make_shared(); + recycler.TEST_add_accessor(RESOURCE_ID, accessor); + + // first for set is_recycled true + int ret = recycler.recycle_rowsets(); + ASSERT_EQ(ret, 0); + + LOG(INFO) << "Step 4: Job aborted via recycler"; + } + + // Step 5: Try to commit_rowset (may succeed but job is aborted) + { + brpc::Controller cntl; + CreateRowsetResponse res; + auto* arena = res.GetArena(); + auto* req = google::protobuf::Arena::CreateMessage(arena); + req->mutable_rowset_meta()->CopyFrom(rowset_meta); + meta_service->commit_rowset(&cntl, req, &res, nullptr); + if (!arena) delete req; + // Should fail because recycle_rowset_key is marked recycled + ASSERT_NE(res.status().code(), MetaServiceCode::OK); + ASSERT_TRUE(res.status().msg().find("rowset has already been marked as recycled") != + std::string::npos) + << res.status().msg(); + LOG(INFO) << "Step 5: commit_rowset executed with code=" << res.status().code(); + } + + // Step 6: Try to finish tablet job + { + FinishTabletJobResponse res; + finish_compaction_job(meta_service.get(), tablet_id, job_id, "test_initiator", 0, 0, + TabletCompactionJobPB::CUMULATIVE, res, + FinishTabletJobRequest::COMMIT, {2, 4}, rowset_meta.txn_id()); + ASSERT_NE(res.status().code(), MetaServiceCode::OK); + ASSERT_TRUE(res.status().msg().find("failed to get tmp rowset key") != + std::string::npos) + << res.status().msg(); + LOG(INFO) << "Step 6: Tablet job finished with code=" << res.status().code(); + } + } +} + +// Test: Complete flow - begin_txn -> prepare_rowset -> commit_rowset -> recycle x 1 -> commit_txn -> verify commit fails +TEST(MetaServiceJobTest, AbortTxnForRelatedRowsetTest4) { + auto meta_service = get_meta_service(); + auto* sp = SyncPoint::get_instance(); + DORIS_CLOUD_DEFER { + SyncPoint::get_instance()->clear_all_call_backs(); + config::enable_recycle_delete_rowset_key_check = false; + config::force_immediate_recycle = false; + }; + config::force_immediate_recycle = true; + config::enable_recycle_delete_rowset_key_check = true; + sp->set_call_back("get_instance_id", [&](auto&& args) { + auto* ret = try_any_cast_ret(args); + ret->first = instance_id; + ret->second = true; + }); + sp->enable_processing(); + + int64_t db_id = 1; + int64_t table_id = 12340; + int64_t index_id = 12350; + int64_t partition_id = 12360; + int64_t tablet_id = 12370; + + // Create tablet first + create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id); + + { + LOG(INFO) << "Test: Complete flow with abort_txn_for_related_rowset"; + + create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id); + + // Step 1: begin_txn + int64_t txn_id = -1; + std::string label = "test_abort_txn_label"; + { + brpc::Controller cntl; + BeginTxnRequest req; + BeginTxnResponse res; + req.set_cloud_unique_id("test_cloud_unique_id"); + auto* txn_info = req.mutable_txn_info(); + txn_info->set_db_id(db_id); + txn_info->set_label(label); + txn_info->add_table_ids(table_id); + txn_info->set_timeout_ms(36000); + meta_service->begin_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + txn_id = res.txn_id(); + LOG(INFO) << "Step 1: Transaction started, txn_id=" << txn_id; + } + + // Step 2: prepare_rowset + doris::RowsetMetaCloudPB rowset_meta; + { + rowset_meta = create_rowset(txn_id, tablet_id, partition_id, -1, 100); + rowset_meta.mutable_load_id()->CopyFrom(PUniqueId()); + rowset_meta.mutable_load_id()->set_hi(11111); + rowset_meta.mutable_load_id()->set_lo(22222); + rowset_meta.set_rowset_state(RowsetStatePB::PREPARED); + rowset_meta.set_resource_id(std::string(RESOURCE_ID)); + + brpc::Controller cntl; + CreateRowsetResponse res; + auto* arena = res.GetArena(); + auto* req = google::protobuf::Arena::CreateMessage(arena); + req->mutable_rowset_meta()->CopyFrom(rowset_meta); + meta_service->prepare_rowset(&cntl, req, &res, nullptr); + if (!arena) delete req; + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + LOG(INFO) << "Step 2: Rowset prepared"; + } + + // Step 4: Try to commit_rowset + { + brpc::Controller cntl; + CreateRowsetResponse res; + auto* arena = res.GetArena(); + auto* req = google::protobuf::Arena::CreateMessage(arena); + req->mutable_rowset_meta()->CopyFrom(rowset_meta); + meta_service->commit_rowset(&cntl, req, &res, nullptr); + if (!arena) delete req; + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + LOG(INFO) << "Step 5: commit_rowset successfully"; + } + + // Step 5: Abort transaction via abort_txn_for_related_rowset + { + InstanceInfoPB instance_info; + instance_info.set_instance_id(instance_id); + + InstanceRecycler recycler(meta_service->txn_kv(), instance_info, thread_group, + std::make_shared(meta_service->txn_kv())); + ASSERT_EQ(recycler.init(), 0); + std::shared_ptr accessor = std::make_shared(); + recycler.TEST_add_accessor(RESOURCE_ID, accessor); + + // first for set is_recycled true + int ret = recycler.recycle_tmp_rowsets(); + ASSERT_EQ(ret, 0); + + LOG(INFO) << "Step 4: Transaction aborted via recycler"; + } + + // Step 6: Try to commit_txn (should also fail) + { + brpc::Controller cntl; + CommitTxnRequest req; + CommitTxnResponse res; + req.set_db_id(db_id); + req.set_txn_id(txn_id); + meta_service->commit_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + // Should fail because transaction is already aborted + ASSERT_NE(res.status().code(), MetaServiceCode::OK); + ASSERT_TRUE(res.status().msg().find("rowset has already been marked as recycled") != + std::string::npos) + << res.status().msg(); + LOG(INFO) << "Step 6: commit_txn correctly failed for aborted txn"; + } + } +} + +// Test: Complete flow - start job -> prepare_rowset -> commit_rowset -> recycle x 1 -> finish job -> verify +TEST(MetaServiceJobTest, AbortJobForRelatedRowsetTest4) { + auto meta_service = get_meta_service(); + auto* sp = SyncPoint::get_instance(); + DORIS_CLOUD_DEFER { + SyncPoint::get_instance()->clear_all_call_backs(); + config::enable_recycle_delete_rowset_key_check = false; + config::force_immediate_recycle = false; + }; + config::enable_recycle_delete_rowset_key_check = true; + config::force_immediate_recycle = true; + sp->set_call_back("get_instance_id", [&](auto&& args) { + auto* ret = try_any_cast_ret(args); + ret->first = instance_id; + ret->second = true; + }); + sp->enable_processing(); + + int64_t table_id = 12440; + int64_t index_id = 12450; + int64_t partition_id = 12460; + int64_t tablet_id = 12470; + + // Create tablet first + create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id); + + { + LOG(INFO) << "Test: Complete flow with abort_job_for_related_rowset2"; + + create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id); + + std::string job_id = "test_complete_flow_job_456"; + + // Step 0: Create input rowsets for compaction + { + std::vector input_rowsets; + input_rowsets.push_back(create_rowset(tablet_id, 2, 2, 100)); + input_rowsets.push_back(create_rowset(tablet_id, 3, 3, 100)); + input_rowsets.push_back(create_rowset(tablet_id, 4, 4, 100)); + input_rowsets[0].set_resource_id(std::string(RESOURCE_ID)); + input_rowsets[1].set_resource_id(std::string(RESOURCE_ID)); + input_rowsets[2].set_resource_id(std::string(RESOURCE_ID)); + insert_rowsets(meta_service->txn_kv().get(), table_id, index_id, partition_id, + tablet_id, input_rowsets); + LOG(INFO) << "Step 0: Input rowsets created (version 2-4)"; + } + + // Step 1: Start a compaction job + { + StartTabletJobResponse res; + start_compaction_job(meta_service.get(), tablet_id, job_id, "test_initiator", 0, 0, + TabletCompactionJobPB::CUMULATIVE, res, {2, 4}); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + LOG(INFO) << "Step 1: Compaction job started, job_id=" << job_id; + } + + // Step 2: prepare_rowset with job_id (output rowset for compaction) + doris::RowsetMetaCloudPB rowset_meta; + { + rowset_meta = create_rowset(tablet_id, 5, 5, 300); // Output rowset with merged data + rowset_meta.set_job_id(job_id); + rowset_meta.set_resource_id(std::string(RESOURCE_ID)); + + brpc::Controller cntl; + CreateRowsetResponse res; + auto* arena = res.GetArena(); + auto* req = google::protobuf::Arena::CreateMessage(arena); + req->mutable_rowset_meta()->CopyFrom(rowset_meta); + meta_service->prepare_rowset(&cntl, req, &res, nullptr); + if (!arena) delete req; + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + LOG(INFO) << "Step 2: Rowset with job_id prepared"; + } + + // Step 4: Try to commit_rowset (may succeed but job is aborted) + { + brpc::Controller cntl; + CreateRowsetResponse res; + auto* arena = res.GetArena(); + auto* req = google::protobuf::Arena::CreateMessage(arena); + req->mutable_rowset_meta()->CopyFrom(rowset_meta); + meta_service->commit_rowset(&cntl, req, &res, nullptr); + if (!arena) delete req; + // commit_rowset may succeed, but the job itself is aborted + LOG(INFO) << "Step 5: commit_rowset executed with code=" << res.status().code(); + } + + // Step 5: Abort job via abort_job_for_related_rowset + { + InstanceInfoPB instance_info; + instance_info.set_instance_id(instance_id); + + InstanceRecycler recycler(meta_service->txn_kv(), instance_info, thread_group, + std::make_shared(meta_service->txn_kv())); + ASSERT_EQ(recycler.init(), 0); + std::shared_ptr accessor = std::make_shared(); + recycler.TEST_add_accessor(RESOURCE_ID, accessor); + + // first for set is_recycled true + int ret = recycler.recycle_tmp_rowsets(); + ASSERT_EQ(ret, 0); + + LOG(INFO) << "Step 4: Job aborted via recycler"; + } + + // Step 6: Try to finish tablet job + { + FinishTabletJobResponse res; + finish_compaction_job(meta_service.get(), tablet_id, job_id, "test_initiator", 0, 0, + TabletCompactionJobPB::CUMULATIVE, res, + FinishTabletJobRequest::COMMIT, {2, 4}, rowset_meta.txn_id()); + ASSERT_NE(res.status().code(), MetaServiceCode::OK); + ASSERT_TRUE(res.status().msg().find("rowset has already been marked as recycled") != + std::string::npos) + << res.status().msg(); + LOG(INFO) << "Step 6: Tablet job finished with code=" << res.status().code(); + } + } +} + +// For old version be, delete job can't send prepare_rowset rpc. +// Test: Complete flow - begin_txn(delete job) -> prepare_rowset -> recycle x 1 -> commit_rowset -> commit txn -> verify +TEST(MetaServiceJobTest, DeleteJobForRelatedRowsetTest) { + auto meta_service = get_meta_service(); + auto* sp = SyncPoint::get_instance(); + DORIS_CLOUD_DEFER { + SyncPoint::get_instance()->clear_all_call_backs(); + config::enable_recycle_delete_rowset_key_check = false; + config::force_immediate_recycle = false; + }; + config::force_immediate_recycle = true; + config::enable_recycle_delete_rowset_key_check = true; + sp->set_call_back("get_instance_id", [&](auto&& args) { + auto* ret = try_any_cast_ret(args); + ret->first = instance_id; + ret->second = true; + }); + sp->enable_processing(); + + int64_t db_id = 1; + int64_t table_id = 12340; + int64_t index_id = 12350; + int64_t partition_id = 12360; + int64_t tablet_id = 12370; + + // Create tablet first + create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id); + + { + LOG(INFO) << "Test: Complete flow with abort_txn_for_related_rowset"; + + create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id); + + // Step 1: begin_txn + int64_t txn_id = -1; + std::string label = "test_abort_txn_label"; + { + brpc::Controller cntl; + BeginTxnRequest req; + BeginTxnResponse res; + req.set_cloud_unique_id("test_cloud_unique_id"); + auto* txn_info = req.mutable_txn_info(); + txn_info->set_db_id(db_id); + txn_info->set_label(label); + txn_info->add_table_ids(table_id); + txn_info->set_timeout_ms(36000); + meta_service->begin_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + txn_id = res.txn_id(); + LOG(INFO) << "Step 1: Transaction started, txn_id=" << txn_id; + } + + // Step 2: prepare_rowset with job_id (output rowset for compaction) + doris::RowsetMetaCloudPB rowset_meta; + { + rowset_meta = create_rowset(txn_id, tablet_id, partition_id, -1, 100); + rowset_meta.mutable_load_id()->CopyFrom(PUniqueId()); + rowset_meta.mutable_load_id()->set_hi(11111); + rowset_meta.mutable_load_id()->set_lo(22222); + rowset_meta.set_rowset_state(RowsetStatePB::PREPARED); + rowset_meta.set_resource_id(std::string(RESOURCE_ID)); + rowset_meta.set_txn_id(txn_id); + DeletePredicatePB delete_predicate; + delete_predicate.set_version(1); + rowset_meta.mutable_delete_predicate()->CopyFrom(delete_predicate); + + brpc::Controller cntl; + CreateRowsetResponse res; + auto* arena = res.GetArena(); + auto* req = google::protobuf::Arena::CreateMessage(arena); + req->mutable_rowset_meta()->CopyFrom(rowset_meta); + meta_service->prepare_rowset(&cntl, req, &res, nullptr); + if (!arena) delete req; + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + LOG(INFO) << "Step 2: Rowset with job_id prepared"; + } + + // Step 5: recycle recycle_rowset_key + { + InstanceInfoPB instance_info; + instance_info.set_instance_id(instance_id); + + InstanceRecycler recycler(meta_service->txn_kv(), instance_info, thread_group, + std::make_shared(meta_service->txn_kv())); + ASSERT_EQ(recycler.init(), 0); + std::shared_ptr accessor = std::make_shared(); + recycler.TEST_add_accessor(RESOURCE_ID, accessor); + + // first for set is_recycled true + int ret = recycler.recycle_rowsets(); + ASSERT_EQ(ret, 0); + + LOG(INFO) << "Step 4: recycle recycle_rowset_key"; + } + + // Step 4: Try to commit_rowset (may succeed but job is aborted) + { + brpc::Controller cntl; + CreateRowsetResponse res; + auto* arena = res.GetArena(); + auto* req = google::protobuf::Arena::CreateMessage(arena); + req->mutable_rowset_meta()->CopyFrom(rowset_meta); + meta_service->commit_rowset(&cntl, req, &res, nullptr); + if (!arena) delete req; + // commit_rowset may succeed, but the job itself is aborted + LOG(INFO) << "Step 5: commit_rowset executed with code=" << res.status().code(); + } + + // Step 6: Try to commit_txn (should also fail) + { + brpc::Controller cntl; + CommitTxnRequest req; + CommitTxnResponse res; + req.set_db_id(db_id); + req.set_txn_id(txn_id); + meta_service->commit_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + // Should fail because transaction is already aborted + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + LOG(INFO) << "Step 6: commit_txn correctly failed for aborted txn"; + } + } +} + } // namespace doris::cloud diff --git a/cloud/test/meta_service_test.cpp b/cloud/test/meta_service_test.cpp index f6fd0081cf6213..580427605c4c91 100644 --- a/cloud/test/meta_service_test.cpp +++ b/cloud/test/meta_service_test.cpp @@ -48,8 +48,11 @@ #include "meta-store/versioned_value.h" #include "mock_resource_manager.h" #include "rate-limiter/rate_limiter.h" +#include "recycler/recycler.h" #include "resource-manager/resource_manager.h" +doris::cloud::RecyclerThreadPoolGroup thread_group; + int main(int argc, char** argv) { const std::string conf_file = "doris_cloud.conf"; if (!doris::cloud::config::init(conf_file.c_str(), true)) { @@ -68,6 +71,18 @@ int main(int argc, char** argv) { return -1; } ::testing::InitGoogleTest(&argc, argv); + config::recycler_sleep_before_scheduling_seconds = 0; // we dont have to wait in UT + + 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); + recycle_tablet_pool->start(); + auto group_recycle_function_pool = + std::make_shared(config::recycle_pool_parallelism); + group_recycle_function_pool->start(); + thread_group = + RecyclerThreadPoolGroup(std::move(s3_producer_pool), std::move(recycle_tablet_pool), + std::move(group_recycle_function_pool)); return RUN_ALL_TESTS(); } @@ -1808,6 +1823,8 @@ TEST(MetaServiceTest, CommitTxnTest) { create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id_base + i); auto tmp_rowset = create_rowset(txn_id, tablet_id_base + i, partition_id); CreateRowsetResponse res; + prepare_rowset(meta_service.get(), tmp_rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); commit_rowset(meta_service.get(), tmp_rowset, res); ASSERT_EQ(res.status().code(), MetaServiceCode::OK); } @@ -1910,6 +1927,8 @@ TEST(MetaServiceTest, CommitTxnExpiredTest) { create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id_base + i); auto tmp_rowset = create_rowset(txn_id, tablet_id_base + i, partition_id); CreateRowsetResponse res; + prepare_rowset(meta_service.get(), tmp_rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); commit_rowset(meta_service.get(), tmp_rowset, res); ASSERT_EQ(res.status().code(), MetaServiceCode::OK); } @@ -1937,6 +1956,8 @@ void create_and_commit_rowset(MetaServiceProxy* meta_service, int64_t table_id, create_tablet(meta_service, table_id, index_id, partition_id, tablet_id); auto tmp_rowset = create_rowset(txn_id, tablet_id, partition_id); CreateRowsetResponse res; + prepare_rowset(meta_service, tmp_rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); commit_rowset(meta_service, tmp_rowset, res); ASSERT_EQ(res.status().code(), MetaServiceCode::OK); } @@ -2529,6 +2550,8 @@ TEST(MetaServiceTest, AbortTxnTest) { create_tablet(meta_service.get(), 12345, 1235, 1236, tablet_id_base + i); auto tmp_rowset = create_rowset(txn_id, tablet_id_base + i); CreateRowsetResponse res; + prepare_rowset(meta_service.get(), tmp_rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); commit_rowset(meta_service.get(), tmp_rowset, res); ASSERT_EQ(res.status().code(), MetaServiceCode::OK); } @@ -2579,6 +2602,8 @@ TEST(MetaServiceTest, AbortTxnTest) { 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; + prepare_rowset(meta_service.get(), tmp_rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); commit_rowset(meta_service.get(), tmp_rowset, res); ASSERT_EQ(res.status().code(), MetaServiceCode::OK); } @@ -2769,6 +2794,8 @@ TEST(MetaServiceTest, CheckTxnConflictTest) { 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; + prepare_rowset(meta_service.get(), tmp_rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); commit_rowset(meta_service.get(), tmp_rowset, res); ASSERT_EQ(res.status().code(), MetaServiceCode::OK); } @@ -3042,6 +3069,8 @@ TEST(MetaServiceTest, CleanTxnLabelTest) { create_tablet(meta_service.get(), 1234, 1235, 1236, tablet_id_base + i); auto tmp_rowset = create_rowset(txn_id, tablet_id_base + i); CreateRowsetResponse res; + prepare_rowset(meta_service.get(), tmp_rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); commit_rowset(meta_service.get(), tmp_rowset, res); ASSERT_EQ(res.status().code(), MetaServiceCode::OK); } @@ -3295,6 +3324,8 @@ TEST(MetaServiceTest, CleanTxnLabelTest) { create_tablet(meta_service.get(), 1234, 1235, 1236, tablet_id_base + i); auto tmp_rowset = create_rowset(txn_id, tablet_id_base + i); CreateRowsetResponse res; + prepare_rowset(meta_service.get(), tmp_rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); commit_rowset(meta_service.get(), tmp_rowset, res); ASSERT_EQ(res.status().code(), MetaServiceCode::OK); } @@ -3363,6 +3394,8 @@ TEST(MetaServiceTest, CleanTxnLabelTest) { create_tablet(meta_service.get(), 1234, 1235, 1236, tablet_id_base + i); auto tmp_rowset = create_rowset(txn_id, tablet_id_base + i); CreateRowsetResponse res; + prepare_rowset(meta_service.get(), tmp_rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); commit_rowset(meta_service.get(), tmp_rowset, res); ASSERT_EQ(res.status().code(), MetaServiceCode::OK); } @@ -6924,6 +6957,8 @@ TEST(MetaServiceTest, DeleteBimapCommitTxnTest) { auto tmp_rowset = create_rowset(txn_id, tablet_id_base + i); tmp_rowset.set_partition_id(partition_id); CreateRowsetResponse res; + prepare_rowset(meta_service.get(), tmp_rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); commit_rowset(meta_service.get(), tmp_rowset, res); ASSERT_EQ(res.status().code(), MetaServiceCode::OK); } @@ -10406,6 +10441,16 @@ TEST(MetaServiceTest, CheckJobExisted) { auto req = google::protobuf::Arena::CreateMessage(arena); req->set_tablet_job_id("compaction1"); req->mutable_rowset_meta()->CopyFrom(rowset); + meta_service->prepare_rowset(&cntl, req, &res, nullptr); + if (!arena) delete req; + + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << res.status().msg(); + res.Clear(); + + arena = res.GetArena(); + req = google::protobuf::Arena::CreateMessage(arena); + req->set_tablet_job_id("compaction1"); + req->mutable_rowset_meta()->CopyFrom(rowset); meta_service->commit_rowset(&cntl, req, &res, nullptr); if (!arena) delete req; @@ -10413,7 +10458,7 @@ TEST(MetaServiceTest, CheckJobExisted) { res.Clear(); } - // commit rowset, job does not exist, + // prepare rowset, job does not exist, { constexpr auto table_id = 953101, index_id = 953102, partition_id = 953103, tablet_id = 953104; @@ -10431,14 +10476,14 @@ TEST(MetaServiceTest, CheckJobExisted) { auto req = google::protobuf::Arena::CreateMessage(arena); req->set_tablet_job_id("compaction1"); req->mutable_rowset_meta()->CopyFrom(rowset); - meta_service->commit_rowset(&cntl, req, &res, nullptr); + meta_service->prepare_rowset(&cntl, req, &res, nullptr); if (!arena) delete req; ASSERT_EQ(res.status().code(), MetaServiceCode::STALE_PREPARE_ROWSET) << res.status().msg(); res.Clear(); } - // commit rowset, compaction job exists, job id not match + // prepare rowset, compaction job exists, job id not match { constexpr auto table_id = 953201, index_id = 953202, partition_id = 953203, tablet_id = 953204; @@ -10462,14 +10507,14 @@ TEST(MetaServiceTest, CheckJobExisted) { auto req = google::protobuf::Arena::CreateMessage(arena); req->set_tablet_job_id("compaction2"); req->mutable_rowset_meta()->CopyFrom(rowset); - meta_service->commit_rowset(&cntl, req, &res, nullptr); + meta_service->prepare_rowset(&cntl, req, &res, nullptr); if (!arena) delete req; ASSERT_EQ(res.status().code(), MetaServiceCode::STALE_PREPARE_ROWSET) << res.status().msg(); res.Clear(); } - // do not set job id when commit rowset + // do not set job id when prepare rowset { constexpr auto table_id = 953301, index_id = 953302, partition_id = 953303, tablet_id = 953304; @@ -10492,14 +10537,14 @@ TEST(MetaServiceTest, CheckJobExisted) { 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); + meta_service->prepare_rowset(&cntl, req, &res, nullptr); if (!arena) delete req; ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << res.status().msg(); res.Clear(); } - // job id is empty string when commit rowset + // job id is empty string when prepare rowset { constexpr auto table_id = 953401, index_id = 953402, partition_id = 953403, tablet_id = 953404; @@ -10523,7 +10568,7 @@ TEST(MetaServiceTest, CheckJobExisted) { auto req = google::protobuf::Arena::CreateMessage(arena); req->set_tablet_job_id(""); req->mutable_rowset_meta()->CopyFrom(rowset); - meta_service->commit_rowset(&cntl, req, &res, nullptr); + meta_service->prepare_rowset(&cntl, req, &res, nullptr); if (!arena) delete req; ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << res.status().msg(); @@ -10588,11 +10633,13 @@ TEST(MetaServiceTest, StaleCommitRowset) { ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << label; ASSERT_NO_FATAL_FAILURE(commit_rowset(meta_service.get(), rowset, res)); - ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << label; + ASSERT_TRUE(res.status().msg().find("recycle rowset key not found") != std::string::npos) + << res.status().msg(); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT) << res.status().code(); commit_txn(meta_service.get(), db_id, txn_id, label); ASSERT_NO_FATAL_FAILURE(commit_rowset(meta_service.get(), rowset, res)); - ASSERT_TRUE(res.status().msg().find("txn is not in") != std::string::npos) + ASSERT_TRUE(res.status().msg().find("recycle rowset key not found") != std::string::npos) << res.status().msg(); ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT) << res.status().code(); } @@ -12216,6 +12263,8 @@ TEST(MetaServiceTest, RowsetVisibleTimeTest) { create_tablet(meta_service.get(), 1234, 1235, 1236, tablet_id_base + i); auto tmp_rowset = create_rowset(txn_id, tablet_id_base + i); CreateRowsetResponse res; + prepare_rowset(meta_service.get(), tmp_rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); commit_rowset(meta_service.get(), tmp_rowset, res); ASSERT_EQ(res.status().code(), MetaServiceCode::OK); } @@ -12465,6 +12514,7 @@ TEST(MetaServiceTest, CleanTxnLabelVersionedWriteSkipWithoutRecycleKey) { auto tmp_rowset = create_rowset(txn_id, tablet_id); { CreateRowsetResponse res; + prepare_rowset(meta_service.get(), tmp_rowset, res); commit_rowset(meta_service.get(), tmp_rowset, res); ASSERT_EQ(res.status().code(), MetaServiceCode::OK); } @@ -12582,6 +12632,7 @@ TEST(MetaServiceTest, CleanTxnLabelVersionedWriteDeleteWithRecycleKey) { auto tmp_rowset = create_rowset(txn_id, tablet_id); { CreateRowsetResponse res; + prepare_rowset(meta_service.get(), tmp_rowset, res); commit_rowset(meta_service.get(), tmp_rowset, res); ASSERT_EQ(res.status().code(), MetaServiceCode::OK); } @@ -12731,6 +12782,7 @@ TEST(MetaServiceTest, CleanTxnLabelVersionedWriteMixedTxns) { auto tmp_rowset = create_rowset(txn_id, tablet_id); { CreateRowsetResponse res; + prepare_rowset(meta_service.get(), tmp_rowset, res); commit_rowset(meta_service.get(), tmp_rowset, res); ASSERT_EQ(res.status().code(), MetaServiceCode::OK); } diff --git a/cloud/test/recycler_test.cpp b/cloud/test/recycler_test.cpp index a308a1cfef8708..72e46c4ad2dcb4 100644 --- a/cloud/test/recycler_test.cpp +++ b/cloud/test/recycler_test.cpp @@ -1352,6 +1352,7 @@ TEST(RecyclerTest, recycle_rowsets) { check_delete_bitmap_keys_size(txn_kv.get(), tablet_id, 1000); check_delete_bitmap_file_size(accessor, tablet_id, 1000); + ASSERT_EQ(recycler.recycle_rowsets(), 0); ASSERT_EQ(recycler.recycle_rowsets(), 0); // check rowset does not exist on obj store @@ -1528,6 +1529,7 @@ TEST(RecyclerTest, bench_recycle_rowsets) { check_delete_bitmap_keys_size(txn_kv.get(), tablet_id, 1000); check_delete_bitmap_file_size(accessor, tablet_id, 1000); + ASSERT_EQ(recycler.recycle_rowsets(), 0); ASSERT_EQ(recycler.recycle_rowsets(), 0); ASSERT_EQ(recycler.check_recycle_tasks(), false); @@ -1618,6 +1620,7 @@ TEST(RecyclerTest, recycle_tmp_rowsets) { auto start = std::chrono::steady_clock::now(); ASSERT_EQ(recycler.recycle_tmp_rowsets(), 0); + ASSERT_EQ(recycler.recycle_tmp_rowsets(), 0); auto finish = std::chrono::steady_clock::now(); std::cout << "recycle tmp rowsets cost=" << std::chrono::duration_cast(finish - start).count() @@ -1701,6 +1704,7 @@ TEST(RecyclerTest, recycle_tmp_rowsets_partial_update) { check_delete_bitmap_keys_size(txn_kv.get(), tablet_id, 10); check_delete_bitmap_file_size(accessor, tablet_id, 10); + ASSERT_EQ(recycler.recycle_tmp_rowsets(), 0); ASSERT_EQ(recycler.recycle_tmp_rowsets(), 0); // check rowset does not exist on obj store std::unique_ptr list_iter; @@ -2002,8 +2006,9 @@ TEST(RecyclerTest, recycle_indexes) { check_delete_bitmap_file_size(accessor, tablet_id, 10); } ASSERT_EQ(recycler.recycle_indexes(), 0); + ASSERT_EQ(recycler.recycle_tmp_rowsets(), 0); ASSERT_EQ(recycler.recycle_tmp_rowsets(), 0); // Recycle tmp rowsets too, since - // recycle_indexes does not recycle tmp rowsets + // recycle_indexes does not recycle tmp rowsets // check rowset does not exist on s3 std::unique_ptr list_iter; diff --git a/cloud/test/rpc_kv_bvar_test.cpp b/cloud/test/rpc_kv_bvar_test.cpp index 48da02cca75239..6b8c5d7207d482 100644 --- a/cloud/test/rpc_kv_bvar_test.cpp +++ b/cloud/test/rpc_kv_bvar_test.cpp @@ -129,6 +129,18 @@ std::unique_ptr get_fdb_meta_service() { return std::make_unique(std::move(meta_service)); } +static void prepare_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->prepare_rowset(&cntl, req, &res, nullptr); + if (!arena) { + delete req; + } +} + static void commit_rowset(MetaServiceProxy* meta_service, const doris::RowsetMetaCloudPB& rowset, CreateRowsetResponse& res) { brpc::Controller cntl; @@ -332,22 +344,13 @@ static void create_and_commit_rowset(MetaServiceProxy* meta_service, int64_t tab create_tablet(meta_service, table_id, index_id, partition_id, tablet_id); auto tmp_rowset = create_rowset(txn_id, tablet_id, partition_id); CreateRowsetResponse res; + prepare_rowset(meta_service, tmp_rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + res.Clear(); commit_rowset(meta_service, tmp_rowset, res); ASSERT_EQ(res.status().code(), MetaServiceCode::OK); } -static void prepare_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->prepare_rowset(&cntl, req, &res, nullptr); - if (!arena) { - delete req; - } -} - static void get_tablet_stats(MetaService* meta_service, int64_t table_id, int64_t index_id, int64_t partition_id, int64_t tablet_id, GetTabletStatsResponse& res) { brpc::Controller cntl; @@ -1509,6 +1512,9 @@ TEST(RpcKvBvarTest, DropIndex) { auto tmp_rowset = create_rowset(txn_id, tablet_id_base + i, index_id, partition_id, -1, 100); CreateRowsetResponse res; + prepare_rowset(meta_service.get(), tmp_rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + res.Clear(); commit_rowset(meta_service.get(), tmp_rowset, res); ASSERT_EQ(res.status().code(), MetaServiceCode::OK); } diff --git a/cloud/test/txn_lazy_commit_test.cpp b/cloud/test/txn_lazy_commit_test.cpp index f647de4c974868..7899ea83ebbe6f 100644 --- a/cloud/test/txn_lazy_commit_test.cpp +++ b/cloud/test/txn_lazy_commit_test.cpp @@ -202,6 +202,18 @@ static doris::RowsetMetaCloudPB create_huge_rowset(int64_t txn_id, int64_t table return rowset; } +static void prepare_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->prepare_rowset(&cntl, req, &res, nullptr); + if (!arena) { + delete req; + } +} + static void commit_rowset(MetaServiceProxy* meta_service, const doris::RowsetMetaCloudPB& rowset, CreateRowsetResponse& res) { brpc::Controller cntl; @@ -481,6 +493,8 @@ TEST(TxnLazyCommitTest, RepairTabletIndexTest) { auto tmp_rowset = create_rowset(txn_id, tablet_id_base + i, index_id, partition_id); tmp_rowsets_meta.push_back(std::make_pair("mock_tmp_rowset_key", tmp_rowset)); CreateRowsetResponse res; + prepare_rowset(meta_service.get(), tmp_rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); commit_rowset(meta_service.get(), tmp_rowset, res); ASSERT_EQ(res.status().code(), MetaServiceCode::OK); } @@ -568,6 +582,8 @@ TEST(TxnLazyCommitTest, CommitTxnEventuallyWithoutDbIdTest) { tablet_id_base + i); auto tmp_rowset = create_rowset(txn_id, tablet_id_base + i, index_id, partition_id); CreateRowsetResponse res; + prepare_rowset(meta_service.get(), tmp_rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); commit_rowset(meta_service.get(), tmp_rowset, res); ASSERT_EQ(res.status().code(), MetaServiceCode::OK); } @@ -649,6 +665,8 @@ TEST(TxnLazyCommitTest, CommitTxnEventuallyWithAbortedTest) { tablet_id_base + i); auto tmp_rowset = create_rowset(txn_id, tablet_id_base + i, index_id, partition_id); CreateRowsetResponse res; + prepare_rowset(meta_service.get(), tmp_rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); commit_rowset(meta_service.get(), tmp_rowset, res); ASSERT_EQ(res.status().code(), MetaServiceCode::OK); } @@ -758,6 +776,8 @@ TEST(TxnLazyCommitTest, CommitTxnEventuallyWithDbIdTest) { tablet_id_base + i); auto tmp_rowset = create_rowset(txn_id, tablet_id_base + i, index_id, partition_id); CreateRowsetResponse res; + prepare_rowset(meta_service.get(), tmp_rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); commit_rowset(meta_service.get(), tmp_rowset, res); ASSERT_EQ(res.status().code(), MetaServiceCode::OK); } @@ -866,6 +886,8 @@ TEST(TxnLazyCommitVersionedReadTest, CommitTxnEventually) { tablet_id_base + i); auto tmp_rowset = create_rowset(txn_id, tablet_id_base + i, index_id, partition_id); CreateRowsetResponse res; + prepare_rowset(meta_service.get(), tmp_rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); commit_rowset(meta_service.get(), tmp_rowset, res); ASSERT_EQ(res.status().code(), MetaServiceCode::OK); } @@ -1001,6 +1023,8 @@ TEST(TxnLazyCommitVersionedReadTest, DISABLED_CommitTxnEventuallyWithoutDbIdTest tablet_id_base + i); auto tmp_rowset = create_rowset(txn_id, tablet_id_base + i, index_id, partition_id); CreateRowsetResponse res; + prepare_rowset(meta_service.get(), tmp_rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); commit_rowset(meta_service.get(), tmp_rowset, res); ASSERT_EQ(res.status().code(), MetaServiceCode::OK); } @@ -1101,6 +1125,8 @@ TEST(TxnLazyCommitTest, CommitTxnImmediatelyTest) { tablet_id_base + i); auto tmp_rowset = create_rowset(txn_id, tablet_id_base + i, index_id, partition_id); CreateRowsetResponse res; + prepare_rowset(meta_service.get(), tmp_rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); commit_rowset(meta_service.get(), tmp_rowset, res); ASSERT_EQ(res.status().code(), MetaServiceCode::OK); } @@ -1189,6 +1215,8 @@ TEST(TxnLazyCommitTest, NotFallThroughCommitTxnEventuallyTest) { tablet_id_base + i); auto tmp_rowset = create_rowset(txn_id, tablet_id_base + i, index_id, partition_id); CreateRowsetResponse res; + prepare_rowset(meta_service.get(), tmp_rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); commit_rowset(meta_service.get(), tmp_rowset, res); ASSERT_EQ(res.status().code(), MetaServiceCode::OK); } @@ -1277,6 +1305,8 @@ TEST(TxnLazyCommitTest, FallThroughCommitTxnEventuallyTest) { tablet_id_base + i); auto tmp_rowset = create_rowset(txn_id, tablet_id_base + i, index_id, partition_id); CreateRowsetResponse res; + prepare_rowset(meta_service.get(), tmp_rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); commit_rowset(meta_service.get(), tmp_rowset, res); ASSERT_EQ(res.status().code(), MetaServiceCode::OK); } @@ -1435,6 +1465,8 @@ TEST(TxnLazyCommitTest, ConcurrentCommitTxnEventuallyCase1Test) { auto tmp_rowset = create_rowset(txn_id1, tablet_id_base + i, index_id, partition_id); CreateRowsetResponse res; + prepare_rowset(meta_service.get(), tmp_rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); commit_rowset(meta_service.get(), tmp_rowset, res); ASSERT_EQ(res.status().code(), MetaServiceCode::OK); } @@ -1483,6 +1515,8 @@ TEST(TxnLazyCommitTest, ConcurrentCommitTxnEventuallyCase1Test) { auto tmp_rowset = create_rowset(txn_id2, tablet_id_base + i, index_id, partition_id); CreateRowsetResponse res; + prepare_rowset(meta_service.get(), tmp_rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); commit_rowset(meta_service.get(), tmp_rowset, res); ASSERT_EQ(res.status().code(), MetaServiceCode::OK); } @@ -1675,6 +1709,8 @@ TEST(TxnLazyCommitTest, ConcurrentCommitTxnEventuallyCase2Test) { auto tmp_rowset = create_rowset(txn_id1, tablet_id_base + i, index_id, partition_id); CreateRowsetResponse res; + prepare_rowset(meta_service.get(), tmp_rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); commit_rowset(meta_service.get(), tmp_rowset, res); ASSERT_EQ(res.status().code(), MetaServiceCode::OK); } @@ -1723,6 +1759,8 @@ TEST(TxnLazyCommitTest, ConcurrentCommitTxnEventuallyCase2Test) { auto tmp_rowset = create_rowset(txn_id2, tablet_id_base + i, index_id, partition_id); CreateRowsetResponse res; + prepare_rowset(meta_service.get(), tmp_rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); commit_rowset(meta_service.get(), tmp_rowset, res); ASSERT_EQ(res.status().code(), MetaServiceCode::OK); } @@ -1834,6 +1872,8 @@ TEST(TxnLazyCommitTest, ConcurrentCommitTxnEventuallyCase3Test) { auto tmp_rowset = create_rowset(tmp_txn_id, tablet_id_base + i, index_id, partition_id); CreateRowsetResponse res; + prepare_rowset(meta_service.get(), tmp_rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); commit_rowset(meta_service.get(), tmp_rowset, res); ASSERT_EQ(res.status().code(), MetaServiceCode::OK); } @@ -1943,6 +1983,8 @@ TEST(TxnLazyCommitTest, ConcurrentCommitTxnEventuallyCase3Test) { auto tmp_rowset = create_rowset(txn_id1, tablet_id_base + i, index_id, partition_id); CreateRowsetResponse res; + prepare_rowset(meta_service.get(), tmp_rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); commit_rowset(meta_service.get(), tmp_rowset, res); ASSERT_EQ(res.status().code(), MetaServiceCode::OK); } @@ -2094,6 +2136,8 @@ TEST(TxnLazyCommitTest, ConcurrentCommitTxnEventuallyCase4Test) { for (int i = 0; i < 2001; ++i) { auto tmp_rowset = create_rowset(txn_id, tablet_id_base + i, index_id, partition_id); CreateRowsetResponse res; + prepare_rowset(meta_service.get(), tmp_rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); commit_rowset(meta_service.get(), tmp_rowset, res); ASSERT_EQ(res.status().code(), MetaServiceCode::OK); } @@ -2272,6 +2316,8 @@ TEST(TxnLazyCommitTest, ConcurrentCommitTxnEventuallyCase5Test) { auto tmp_rowset = create_rowset(txn_id1, tablet_id_base + i, index_id, partition_id); CreateRowsetResponse res; + prepare_rowset(meta_service.get(), tmp_rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); commit_rowset(meta_service.get(), tmp_rowset, res); ASSERT_EQ(res.status().code(), MetaServiceCode::OK); } @@ -2320,6 +2366,8 @@ TEST(TxnLazyCommitTest, ConcurrentCommitTxnEventuallyCase5Test) { auto tmp_rowset = create_rowset(txn_id2, tablet_id_base + i, index_id, partition_id); CreateRowsetResponse res; + prepare_rowset(meta_service.get(), tmp_rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); commit_rowset(meta_service.get(), tmp_rowset, res); ASSERT_EQ(res.status().code(), MetaServiceCode::OK); } @@ -2353,6 +2401,8 @@ TEST(TxnLazyCommitTest, ConcurrentCommitTxnEventuallyCase5Test) { auto tmp_rowset = create_rowset(sub_txn_id2, tablet_id_base + i, index_id, partition_id); CreateRowsetResponse res; + prepare_rowset(meta_service.get(), tmp_rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); commit_rowset(meta_service.get(), tmp_rowset, res); ASSERT_EQ(res.status().code(), MetaServiceCode::OK); } @@ -2458,6 +2508,8 @@ TEST(TxnLazyCommitTest, RowsetMetaSizeExceedTest) { { auto tmp_rowset = create_rowset(tmp_txn_id, tablet_id, index_id, partition_id); CreateRowsetResponse res; + prepare_rowset(meta_service.get(), tmp_rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); commit_rowset(meta_service.get(), tmp_rowset, res); ASSERT_EQ(res.status().code(), MetaServiceCode::OK); } @@ -2530,363 +2582,6 @@ TEST(TxnLazyCommitTest, ForceTxnLazyCommit) { config::cloud_txn_lazy_commit_fuzzy_possibility = 0; } -TEST(TxnLazyCommitTest, RecycleTmpRowsetsCase1) { - // =========================================================================== - // threads concurrent execution flow: - // - // meta-service recycler - // | | - // begin | - // | | - // prepare/commit rowset | - // | | - // lazy commit submit | - // | | - // dead | - // | | - // | recyle_tmp_rowsets - // | | - // | abort_timeout_txn - // | | - // | | - // v v - - auto txn_kv = get_mem_txn_kv(); - - int64_t db_id = 988032131; - int64_t table_id = 5145043; - int64_t index_id = 273456; - int64_t partition_id = 44576544; - std::string mock_instance = "test_instance"; - const std::string label = "test_label_67ae2q1231"; - - bool commit_txn_eventullay_hit = false; - bool is_txn_finished_hit = false; - bool abort_timeout_txn_hit = false; - - auto sp = SyncPoint::get_instance(); - - sp->set_call_back("commit_txn_eventually::txn_lazy_committer_submit", [&](auto&& args) { - commit_txn_eventullay_hit = true; - bool* pred = try_any_cast(args.back()); - *pred = true; - }); - - TxnInfoPB txn_info_pb; - sp->set_call_back("is_txn_finished::txn_not_finished", [&](auto&& args) { - is_txn_finished_hit = true; - txn_info_pb = *try_any_cast(args[0]); - { - std::unique_ptr txn; - ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); - check_txn_committed(txn, db_id, txn_info_pb.txn_id(), label); - } - }); - - txn_info_pb.Clear(); - sp->set_call_back("abort_timeout_txn::advance_last_pending_txn_id", [&](auto&& args) { - abort_timeout_txn_hit = true; - txn_info_pb = *try_any_cast(args[0]); - { - std::unique_ptr txn; - ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); - check_txn_committed(txn, db_id, txn_info_pb.txn_id(), label); - } - }); - - sp->enable_processing(); - - auto meta_service = get_meta_service(txn_kv, true); - // mock rowset and tablet - int64_t tablet_id_base = 2313324; - for (int i = 0; i < 2001; ++i) { - create_tablet_with_db_id(meta_service.get(), db_id, table_id, index_id, partition_id, - tablet_id_base + i); - } - int txn_id = 0; - { - { - 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(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); - txn_id = res.txn_id(); - ASSERT_GT(txn_id, 0); - } - - { - for (int i = 0; i < 2001; ++i) { - auto tmp_rowset = create_rowset(txn_id, tablet_id_base + i, index_id, partition_id); - 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_enable_txn_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); - } - } - - InstanceInfoPB instance; - instance.set_instance_id(mock_instance); - InstanceRecycler recycler(txn_kv, instance, thread_group, - std::make_shared(txn_kv)); - - ASSERT_EQ(recycler.init(), 0); - ASSERT_EQ(recycler.recycle_tmp_rowsets(), 0); - ASSERT_TRUE(is_txn_finished_hit); - - ASSERT_EQ(recycler.abort_timeout_txn(), 0); - { - std::unique_ptr txn; - ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); - check_txn_visible(txn, db_id, txn_id, label); - } - sleep(1); - ASSERT_EQ(recycler.recycle_expired_txn_label(), 0); - { - std::unique_ptr txn; - ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); - check_txn_not_exist(txn, db_id, txn_id, label); - } - ASSERT_TRUE(abort_timeout_txn_hit); - - ASSERT_EQ(recycler.recycle_tmp_rowsets(), 0); - - sp->clear_all_call_backs(); - sp->clear_trace(); - sp->disable_processing(); - ASSERT_TRUE(commit_txn_eventullay_hit); -} - -TEST(TxnLazyCommitTest, RecycleTmpRowsetsCase2) { - // =========================================================================== - // threads concurrent execution flow: - // - // meta-service recycler - // | | - // begin txn | - // | | - // abort txn | - // | | - // | recyle_tmp_rowsets - // | | - // v v - - auto txn_kv = get_mem_txn_kv(); - - int64_t db_id = 41414; - int64_t table_id = 5454146; - int64_t index_id = 27656; - int64_t partition_id = 4423544; - std::string mock_instance = "test_instance"; - const std::string label = "test_label_67ae2q1231"; - - bool txn_has_been_aborted = false; - - auto sp = SyncPoint::get_instance(); - - TxnInfoPB txn_info_pb; - sp->set_call_back("is_txn_finished::txn_has_been_aborted", [&](auto&& args) { - txn_has_been_aborted = true; - txn_info_pb = *try_any_cast(args[0]); - ASSERT_EQ(txn_info_pb.status(), TxnStatusPB::TXN_STATUS_ABORTED); - }); - - sp->enable_processing(); - - auto meta_service = get_meta_service(txn_kv, true); - // mock rowset and tablet - int64_t tablet_id_base = 2313324; - for (int i = 0; i < 2001; ++i) { - create_tablet_with_db_id(meta_service.get(), db_id, table_id, index_id, partition_id, - tablet_id_base + i); - } - int txn_id = 0; - { - { - 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(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); - txn_id = res.txn_id(); - ASSERT_GT(txn_id, 0); - } - - { - for (int i = 0; i < 2001; ++i) { - auto tmp_rowset = create_rowset(txn_id, tablet_id_base + i, index_id, partition_id); - CreateRowsetResponse res; - commit_rowset(meta_service.get(), tmp_rowset, res); - ASSERT_EQ(res.status().code(), MetaServiceCode::OK); - } - } - { - brpc::Controller cntl; - AbortTxnRequest req; - req.set_cloud_unique_id("test_cloud_unique_id"); - ASSERT_GT(txn_id, 0); - req.set_txn_id(txn_id); - req.set_reason("test"); - AbortTxnResponse res; - meta_service->abort_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), - &req, &res, nullptr); - ASSERT_EQ(res.status().code(), MetaServiceCode::OK); - ASSERT_EQ(res.txn_info().status(), TxnStatusPB::TXN_STATUS_ABORTED); - } - } - - InstanceInfoPB instance; - instance.set_instance_id(mock_instance); - InstanceRecycler recycler(txn_kv, instance, thread_group, - std::make_shared(txn_kv)); - - ASSERT_EQ(recycler.init(), 0); - ASSERT_FALSE(txn_has_been_aborted); - ASSERT_EQ(recycler.recycle_tmp_rowsets(), 0); - ASSERT_TRUE(txn_has_been_aborted); - - sp->clear_all_call_backs(); - sp->clear_trace(); - sp->disable_processing(); -} - -TEST(TxnLazyCommitTest, RecycleTmpRowsetsCase3) { - // =========================================================================== - // threads concurrent execution flow: - // - // meta-service recycler - // | | - // begin txn | - // | | - // abort txn | - // | | - // recycle_expired_txn_label | - // | | - // | recyle_tmp_rowsets - // | | - // v v - - auto txn_kv = get_mem_txn_kv(); - - int64_t db_id = 42345236; - int64_t table_id = 3165524; - int64_t index_id = 89089; - int64_t partition_id = 434154; - std::string mock_instance = "test_instance"; - const std::string label = "test_label_67ae2q1231"; - - bool txn_has_been_recycled = false; - - auto sp = SyncPoint::get_instance(); - - TxnInfoPB txn_info_pb; - sp->set_call_back("is_txn_finished::txn_has_been_recycled", - [&](auto&& args) { txn_has_been_recycled = true; }); - - sp->enable_processing(); - - auto meta_service = get_meta_service(txn_kv, true); - // mock rowset and tablet - int64_t tablet_id_base = 2313324; - for (int i = 0; i < 2001; ++i) { - create_tablet_with_db_id(meta_service.get(), db_id, table_id, index_id, partition_id, - tablet_id_base + i); - } - int txn_id = 0; - { - { - 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(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); - txn_id = res.txn_id(); - ASSERT_GT(txn_id, 0); - } - - { - for (int i = 0; i < 2001; ++i) { - auto tmp_rowset = create_rowset(txn_id, tablet_id_base + i, index_id, partition_id); - CreateRowsetResponse res; - commit_rowset(meta_service.get(), tmp_rowset, res); - ASSERT_EQ(res.status().code(), MetaServiceCode::OK); - } - } - { - brpc::Controller cntl; - AbortTxnRequest req; - req.set_cloud_unique_id("test_cloud_unique_id"); - ASSERT_GT(txn_id, 0); - req.set_txn_id(txn_id); - req.set_reason("test"); - AbortTxnResponse res; - meta_service->abort_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), - &req, &res, nullptr); - ASSERT_EQ(res.status().code(), MetaServiceCode::OK); - ASSERT_EQ(res.txn_info().status(), TxnStatusPB::TXN_STATUS_ABORTED); - } - } - - InstanceInfoPB instance; - instance.set_instance_id(mock_instance); - InstanceRecycler recycler(txn_kv, instance, thread_group, - std::make_shared(txn_kv)); - - ASSERT_EQ(recycler.init(), 0); - - ASSERT_EQ(recycler.recycle_expired_txn_label(), 0); - { - std::unique_ptr txn; - ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); - check_txn_not_exist(txn, db_id, txn_id, label); - } - - ASSERT_FALSE(txn_has_been_recycled); - ASSERT_EQ(recycler.recycle_tmp_rowsets(), 0); - ASSERT_TRUE(txn_has_been_recycled); - - sp->clear_all_call_backs(); - sp->clear_trace(); - sp->disable_processing(); -} TEST(TxnLazyCommitTest, RecyclePartitions) { // =========================================================================== // threads concurrent execution flow: @@ -2972,6 +2667,8 @@ TEST(TxnLazyCommitTest, RecyclePartitions) { for (int i = 0; i < 2001; ++i) { auto tmp_rowset = create_rowset(txn_id, tablet_id_base + i, index_id, partition_id); CreateRowsetResponse res; + prepare_rowset(meta_service.get(), tmp_rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); commit_rowset(meta_service.get(), tmp_rowset, res); ASSERT_EQ(res.status().code(), MetaServiceCode::OK); } @@ -3118,6 +2815,8 @@ TEST(TxnLazyCommitTest, RecycleIndexes) { for (int i = 0; i < 2001; ++i) { auto tmp_rowset = create_rowset(txn_id, tablet_id_base + i, index_id, partition_id); CreateRowsetResponse res; + prepare_rowset(meta_service.get(), tmp_rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); commit_rowset(meta_service.get(), tmp_rowset, res); ASSERT_EQ(res.status().code(), MetaServiceCode::OK); } @@ -3221,6 +2920,8 @@ TEST(TxnLazyCommitTest, CommitTxnEventuallyWithMultiTableTest) { tablet_id_base + i); auto tmp_rowset = create_rowset(txn_id, tablet_id_base + i, index_id, partition_id); CreateRowsetResponse res; + prepare_rowset(meta_service.get(), tmp_rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); commit_rowset(meta_service.get(), tmp_rowset, res); ASSERT_EQ(res.status().code(), MetaServiceCode::OK); } @@ -3230,6 +2931,8 @@ TEST(TxnLazyCommitTest, CommitTxnEventuallyWithMultiTableTest) { tablet_id_base + i); auto tmp_rowset = create_rowset(txn_id, tablet_id_base + i, index_id2, partition_id2); CreateRowsetResponse res; + prepare_rowset(meta_service.get(), tmp_rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); commit_rowset(meta_service.get(), tmp_rowset, res); ASSERT_EQ(res.status().code(), MetaServiceCode::OK); } @@ -3319,6 +3022,8 @@ TEST(TxnLazyCommitTest, CommitTxnEventuallyWithHugeRowsetMetaTest) { tablet_id_base + i); auto tmp_rowset = create_huge_rowset(txn_id, tablet_id_base + i, index_id, partition_id); CreateRowsetResponse res; + prepare_rowset(meta_service.get(), tmp_rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); commit_rowset(meta_service.get(), tmp_rowset, res); ASSERT_EQ(res.status().code(), MetaServiceCode::OK); } @@ -3328,6 +3033,8 @@ TEST(TxnLazyCommitTest, CommitTxnEventuallyWithHugeRowsetMetaTest) { tablet_id_base + i); auto tmp_rowset = create_huge_rowset(txn_id, tablet_id_base + i, index_id2, partition_id2); CreateRowsetResponse res; + prepare_rowset(meta_service.get(), tmp_rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); commit_rowset(meta_service.get(), tmp_rowset, res); ASSERT_EQ(res.status().code(), MetaServiceCode::OK); } @@ -3456,6 +3163,8 @@ TEST(TxnLazyCommitTest, CommitTxnEventuallyWithSchemaChangeTest) { tablet_id_base + i); auto tmp_rowset = create_huge_rowset(txn_id, tablet_id_base + i, index_id, partition_id); CreateRowsetResponse res; + prepare_rowset(meta_service.get(), tmp_rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); commit_rowset(meta_service.get(), tmp_rowset, res); ASSERT_EQ(res.status().code(), MetaServiceCode::OK); } @@ -3501,6 +3210,8 @@ TEST(TxnLazyCommitTest, CommitTxnEventuallyWithSchemaChangeTest) { auto tmp_rowset = create_huge_rowset(txn_id, tablet_id_base + i, index_id, partition_id); CreateRowsetResponse res; + prepare_rowset(meta_service.get(), tmp_rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); commit_rowset(meta_service.get(), tmp_rowset, res); ASSERT_EQ(res.status().code(), MetaServiceCode::OK); } diff --git a/gensrc/proto/olap_file.proto b/gensrc/proto/olap_file.proto index cac152a6a55d8c..104c7102ea91f7 100644 --- a/gensrc/proto/olap_file.proto +++ b/gensrc/proto/olap_file.proto @@ -159,6 +159,9 @@ message RowsetMetaPB { optional int64 visible_ts_ms = 1010; optional string reference_instance_id = 1011; // For cluster snapshot map packed_slice_locations = 1012; + + optional bool is_recycled = 1013; // for recycler mark rowset as recycled + optional string job_id = 1014; } message SchemaDictKeyList { @@ -258,6 +261,9 @@ message RowsetMetaCloudPB { optional int64 visible_ts_ms = 109; optional string reference_instance_id = 110; // For cluster snapshot map packed_slice_locations = 111; + + optional bool is_recycled = 112; + optional string job_id = 113; } message SegmentStatisticsPB {