diff --git a/cloud/src/common/bvars.cpp b/cloud/src/common/bvars.cpp index d8d6d34127a8e8..3ca961afffb725 100644 --- a/cloud/src/common/bvars.cpp +++ b/cloud/src/common/bvars.cpp @@ -227,4 +227,171 @@ BvarStatusWithTag g_bvar_inverted_checker_abnormal_delete_bitmaps("chec BvarStatusWithTag g_bvar_inverted_checker_delete_bitmaps_scanned("checker", "delete_bitmap_keys_scanned"); BvarStatusWithTag g_bvar_max_rowsets_with_useless_delete_bitmap_version("checker", "max_rowsets_with_useless_delete_bitmap_version"); +// rpc kv rw count +// get_rowset +mBvarInt64Adder g_bvar_rpc_kv_get_rowset_get_counter("rpc_kv_get_rowset_get_counter",{"instance_id"}); +// get_version +mBvarInt64Adder g_bvar_rpc_kv_get_version_get_counter("rpc_kv_get_version_get_counter",{"instance_id"}); +// get_schema_dict +mBvarInt64Adder g_bvar_rpc_kv_get_schema_dict_get_counter("rpc_kv_get_schema_dict_get_counter",{"instance_id"}); +// create_tablets +mBvarInt64Adder g_bvar_rpc_kv_create_tablets_get_counter("rpc_kv_create_tablets_get_counter",{"instance_id"}); +mBvarInt64Adder g_bvar_rpc_kv_create_tablets_put_counter("rpc_kv_create_tablets_put_counter",{"instance_id"}); +// update_tablet +mBvarInt64Adder g_bvar_rpc_kv_update_tablet_get_counter("rpc_kv_update_tablet_get_counter",{"instance_id"}); +mBvarInt64Adder g_bvar_rpc_kv_update_tablet_put_counter("rpc_kv_update_tablet_put_counter",{"instance_id"}); +// update_tablet_schema +mBvarInt64Adder g_bvar_rpc_kv_update_tablet_schema_get_counter("rpc_kv_update_tablet_schema_get_counter",{"instance_id"}); +mBvarInt64Adder g_bvar_rpc_kv_update_tablet_schema_put_counter("rpc_kv_update_tablet_schema_put_counter",{"instance_id"}); +// get_tablet +mBvarInt64Adder g_bvar_rpc_kv_get_tablet_get_counter("rpc_kv_get_tablet_get_counter",{"instance_id"}); +// prepare_rowset +mBvarInt64Adder g_bvar_rpc_kv_prepare_rowset_get_counter("rpc_kv_prepare_rowset_get_counter",{"instance_id"}); +mBvarInt64Adder g_bvar_rpc_kv_prepare_rowset_put_counter("rpc_kv_prepare_rowset_put_counter",{"instance_id"}); +// commit_rowset +mBvarInt64Adder g_bvar_rpc_kv_commit_rowset_get_counter("rpc_kv_commit_rowset_get_counter",{"instance_id"}); +mBvarInt64Adder g_bvar_rpc_kv_commit_rowset_put_counter("rpc_kv_commit_rowset_put_counter",{"instance_id"}); +mBvarInt64Adder g_bvar_rpc_kv_commit_rowset_del_counter("rpc_kv_commit_rowset_del_counter",{"instance_id"}); +// update_tmp_rowset +mBvarInt64Adder g_bvar_rpc_kv_update_tmp_rowset_get_counter("rpc_kv_update_tmp_rowset_get_counter",{"instance_id"}); +mBvarInt64Adder g_bvar_rpc_kv_update_tmp_rowset_put_counter("rpc_kv_update_tmp_rowset_put_counter",{"instance_id"}); +// get_tablet_stats +mBvarInt64Adder g_bvar_rpc_kv_get_tablet_stats_get_counter("rpc_kv_get_tablet_stats_get_counter",{"instance_id"}); +// update_delete_bitmap +mBvarInt64Adder g_bvar_rpc_kv_update_delete_bitmap_get_counter("rpc_kv_update_delete_bitmap_get_counter",{"instance_id"}); +mBvarInt64Adder g_bvar_rpc_kv_update_delete_bitmap_put_counter("rpc_kv_update_delete_bitmap_put_counter",{"instance_id"}); +mBvarInt64Adder g_bvar_rpc_kv_update_delete_bitmap_del_counter("rpc_kv_update_delete_bitmap_del_counter",{"instance_id"}); +// get_delete_bitmap +mBvarInt64Adder g_bvar_rpc_kv_get_delete_bitmap_get_counter("rpc_kv_get_delete_bitmap_get_counter",{"instance_id"}); +// get_delete_bitmap_update_lock +mBvarInt64Adder g_bvar_rpc_kv_get_delete_bitmap_update_lock_get_counter("rpc_kv_get_delete_bitmap_update_lock_get_counter",{"instance_id"}); +mBvarInt64Adder g_bvar_rpc_kv_get_delete_bitmap_update_lock_put_counter("rpc_kv_get_delete_bitmap_update_lock_put_counter",{"instance_id"}); +mBvarInt64Adder g_bvar_rpc_kv_get_delete_bitmap_update_lock_del_counter("rpc_kv_get_delete_bitmap_update_lock_del_counter",{"instance_id"}); +// remove_delete_bitmap_update_lock +mBvarInt64Adder g_bvar_rpc_kv_remove_delete_bitmap_update_lock_get_counter("rpc_kv_remove_delete_bitmap_update_lock_get_counter",{"instance_id"}); +mBvarInt64Adder g_bvar_rpc_kv_remove_delete_bitmap_update_lock_put_counter("rpc_kv_remove_delete_bitmap_update_lock_put_counter",{"instance_id"}); +mBvarInt64Adder g_bvar_rpc_kv_remove_delete_bitmap_update_lock_del_counter("rpc_kv_remove_delete_bitmap_update_lock_del_counter",{"instance_id"}); +// remove_delete_bitmap +mBvarInt64Adder g_bvar_rpc_kv_remove_delete_bitmap_del_counter("rpc_kv_remove_delete_bitmap_del_counter",{"instance_id"}); +// start_tablet_job +mBvarInt64Adder g_bvar_rpc_kv_start_tablet_job_get_counter("rpc_kv_start_tablet_job_get_counter",{"instance_id"}); +mBvarInt64Adder g_bvar_rpc_kv_start_tablet_job_put_counter("rpc_kv_start_tablet_job_put_counter",{"instance_id"}); +// finish_tablet_job +mBvarInt64Adder g_bvar_rpc_kv_finish_tablet_job_get_counter("rpc_kv_finish_tablet_job_get_counter",{"instance_id"}); +mBvarInt64Adder g_bvar_rpc_kv_finish_tablet_job_put_counter("rpc_kv_finish_tablet_job_put_counter",{"instance_id"}); +mBvarInt64Adder g_bvar_rpc_kv_finish_tablet_job_del_counter("rpc_kv_finish_tablet_job_del_counter",{"instance_id"}); +// prepare_index +mBvarInt64Adder g_bvar_rpc_kv_prepare_index_get_counter("rpc_kv_prepare_index_get_counter",{"instance_id"}); +mBvarInt64Adder g_bvar_rpc_kv_prepare_index_put_counter("rpc_kv_prepare_index_put_counter",{"instance_id"}); +// commit_index +mBvarInt64Adder g_bvar_rpc_kv_commit_index_get_counter("rpc_kv_commit_index_get_counter",{"instance_id"}); +mBvarInt64Adder g_bvar_rpc_kv_commit_index_put_counter("rpc_kv_commit_index_put_counter",{"instance_id"}); +mBvarInt64Adder g_bvar_rpc_kv_commit_index_del_counter("rpc_kv_commit_index_del_counter",{"instance_id"}); +// drop_index +mBvarInt64Adder g_bvar_rpc_kv_drop_index_get_counter("rpc_kv_drop_index_get_counter",{"instance_id"}); +mBvarInt64Adder g_bvar_rpc_kv_drop_index_put_counter("rpc_kv_drop_index_put_counter",{"instance_id"}); +// prepare_partition +mBvarInt64Adder g_bvar_rpc_kv_prepare_partition_get_counter("rpc_kv_prepare_partition_get_counter",{"instance_id"}); +mBvarInt64Adder g_bvar_rpc_kv_prepare_partition_put_counter("rpc_kv_prepare_partition_put_counter",{"instance_id"}); +// commit_partition +mBvarInt64Adder g_bvar_rpc_kv_commit_partition_get_counter("rpc_kv_commit_partition_get_counter",{"instance_id"}); +mBvarInt64Adder g_bvar_rpc_kv_commit_partition_put_counter("rpc_kv_commit_partition_put_counter",{"instance_id"}); +mBvarInt64Adder g_bvar_rpc_kv_commit_partition_del_counter("rpc_kv_commit_partition_del_counter",{"instance_id"}); +// drop_partition +mBvarInt64Adder g_bvar_rpc_kv_drop_partition_get_counter("rpc_kv_drop_partition_get_counter",{"instance_id"}); +mBvarInt64Adder g_bvar_rpc_kv_drop_partition_put_counter("rpc_kv_drop_partition_put_counter",{"instance_id"}); +// check_kv +mBvarInt64Adder g_bvar_rpc_kv_check_kv_get_counter("rpc_kv_check_kv_get_counter",{"instance_id"}); +// get_obj_store_info +mBvarInt64Adder g_bvar_rpc_kv_get_obj_store_info_get_counter("rpc_kv_get_obj_store_info_get_counter",{"instance_id"}); +// alter_storage_vault +mBvarInt64Adder g_bvar_rpc_kv_alter_storage_vault_get_counter("rpc_kv_alter_storage_vault_get_counter",{"instance_id"}); +mBvarInt64Adder g_bvar_rpc_kv_alter_storage_vault_put_counter("rpc_kv_alter_storage_vault_put_counter",{"instance_id"}); +mBvarInt64Adder g_bvar_rpc_kv_alter_storage_vault_del_counter("rpc_kv_alter_storage_vault_del_counter",{"instance_id"}); +// alter_obj_store_info +mBvarInt64Adder g_bvar_rpc_kv_alter_obj_store_info_get_counter("rpc_kv_alter_obj_store_info_get_counter",{"instance_id"}); +mBvarInt64Adder g_bvar_rpc_kv_alter_obj_store_info_put_counter("rpc_kv_alter_obj_store_info_put_counter",{"instance_id"}); +// update_ak_sk +mBvarInt64Adder g_bvar_rpc_kv_update_ak_sk_get_counter("rpc_kv_update_ak_sk_get_counter",{"instance_id"}); +mBvarInt64Adder g_bvar_rpc_kv_update_ak_sk_put_counter("rpc_kv_update_ak_sk_put_counter",{"instance_id"}); +// create_instance +mBvarInt64Adder g_bvar_rpc_kv_create_instance_get_counter("rpc_kv_create_instance_get_counter",{"instance_id"}); +mBvarInt64Adder g_bvar_rpc_kv_create_instance_put_counter("rpc_kv_create_instance_put_counter",{"instance_id"}); +// get_instance +mBvarInt64Adder g_bvar_rpc_kv_get_instance_get_counter("rpc_kv_get_instance_get_counter",{"instance_id"}); +// alter_cluster +mBvarInt64Adder g_bvar_rpc_kv_alter_cluster_get_counter("rpc_kv_alter_cluster_get_counter",{"instance_id"}); +// get_cluster +mBvarInt64Adder g_bvar_rpc_kv_get_cluster_get_counter("rpc_kv_get_cluster_get_counter",{"instance_id"}); +mBvarInt64Adder g_bvar_rpc_kv_get_cluster_put_counter("rpc_kv_get_cluster_put_counter",{"instance_id"}); +// create_stage +mBvarInt64Adder g_bvar_rpc_kv_create_stage_get_counter("rpc_kv_create_stage_get_counter",{"instance_id"}); +mBvarInt64Adder g_bvar_rpc_kv_create_stage_put_counter("rpc_kv_create_stage_put_counter",{"instance_id"}); +// get_stage +mBvarInt64Adder g_bvar_rpc_kv_get_stage_get_counter("rpc_kv_get_stage_get_counter",{"instance_id"}); +// get_iam +mBvarInt64Adder g_bvar_rpc_kv_get_iam_get_counter("rpc_kv_get_iam_get_counter",{"instance_id"}); +// alter_iam +mBvarInt64Adder g_bvar_rpc_kv_alter_iam_get_counter("rpc_kv_alter_iam_get_counter",{"instance_id"}); +mBvarInt64Adder g_bvar_rpc_kv_alter_iam_put_counter("rpc_kv_alter_iam_put_counter",{"instance_id"}); +// alter_ram_user +mBvarInt64Adder g_bvar_rpc_kv_alter_ram_user_get_counter("rpc_kv_alter_ram_user_get_counter",{"instance_id"}); +mBvarInt64Adder g_bvar_rpc_kv_alter_ram_user_put_counter("rpc_kv_alter_ram_user_put_counter",{"instance_id"}); +// begin_copy +mBvarInt64Adder g_bvar_rpc_kv_begin_copy_get_counter("rpc_kv_begin_copy_get_counter",{"instance_id"}); +mBvarInt64Adder g_bvar_rpc_kv_begin_copy_put_counter("rpc_kv_begin_copy_put_counter",{"instance_id"}); +// finish_copy +mBvarInt64Adder g_bvar_rpc_kv_finish_copy_get_counter("rpc_kv_finish_copy_get_counter",{"instance_id"}); +mBvarInt64Adder g_bvar_rpc_kv_finish_copy_put_counter("rpc_kv_finish_copy_put_counter",{"instance_id"}); +mBvarInt64Adder g_bvar_rpc_kv_finish_copy_del_counter("rpc_kv_finish_copy_del_counter",{"instance_id"}); +// get_copy_job +mBvarInt64Adder g_bvar_rpc_kv_get_copy_job_get_counter("rpc_kv_get_copy_job_get_counter",{"instance_id"}); +// get_copy_files +mBvarInt64Adder g_bvar_rpc_kv_get_copy_files_get_counter("rpc_kv_get_copy_files_get_counter",{"instance_id"}); +// filter_copy_files +mBvarInt64Adder g_bvar_rpc_kv_filter_copy_files_get_counter("rpc_kv_filter_copy_files_get_counter",{"instance_id"}); +// get_cluster_status +mBvarInt64Adder g_bvar_rpc_kv_get_cluster_status_get_counter("rpc_kv_get_cluster_status_get_counter",{"instance_id"}); +// begin_txn +mBvarInt64Adder g_bvar_rpc_kv_begin_txn_get_counter("rpc_kv_begin_txn_get_counter",{"instance_id"}); +mBvarInt64Adder g_bvar_rpc_kv_begin_txn_put_counter("rpc_kv_begin_txn_put_counter",{"instance_id"}); +// precommit_txn +mBvarInt64Adder g_bvar_rpc_kv_precommit_txn_get_counter("rpc_kv_precommit_txn_get_counter",{"instance_id"}); +mBvarInt64Adder g_bvar_rpc_kv_precommit_txn_put_counter("rpc_kv_precommit_txn_put_counter",{"instance_id"}); +// get_rl_task_commit_attach +mBvarInt64Adder g_bvar_rpc_kv_get_rl_task_commit_attach_get_counter("rpc_kv_get_rl_task_commit_attach_get_counter",{"instance_id"}); +// reset_rl_progress +mBvarInt64Adder g_bvar_rpc_kv_reset_rl_progress_get_counter("rpc_kv_reset_rl_progress_get_counter",{"instance_id"}); +mBvarInt64Adder g_bvar_rpc_kv_reset_rl_progress_put_counter("rpc_kv_reset_rl_progress_put_counter",{"instance_id"}); +mBvarInt64Adder g_bvar_rpc_kv_reset_rl_progress_del_counter("rpc_kv_reset_rl_progress_del_counter",{"instance_id"}); +// commit_txn +mBvarInt64Adder g_bvar_rpc_kv_commit_txn_get_counter("rpc_kv_commit_txn_get_counter",{"instance_id"}); +mBvarInt64Adder g_bvar_rpc_kv_commit_txn_put_counter("rpc_kv_commit_txn_put_counter",{"instance_id"}); +mBvarInt64Adder g_bvar_rpc_kv_commit_txn_del_counter("rpc_kv_commit_txn_del_counter",{"instance_id"}); +// abort_txn +mBvarInt64Adder g_bvar_rpc_kv_abort_txn_get_counter("rpc_kv_abort_txn_get_counter",{"instance_id"}); +mBvarInt64Adder g_bvar_rpc_kv_abort_txn_put_counter("rpc_kv_abort_txn_put_counter",{"instance_id"}); +mBvarInt64Adder g_bvar_rpc_kv_abort_txn_del_counter("rpc_kv_abort_txn_del_counter",{"instance_id"}); +// get_txn +mBvarInt64Adder g_bvar_rpc_kv_get_txn_get_counter("rpc_kv_get_txn_get_counter",{"instance_id"}); +// get_current_max_txn_id +mBvarInt64Adder g_bvar_rpc_kv_get_current_max_txn_id_get_counter("rpc_kv_get_current_max_txn_id_get_counter",{"instance_id"}); +// begin_sub_txn +mBvarInt64Adder g_bvar_rpc_kv_begin_sub_txn_get_counter("rpc_kv_begin_sub_txn_get_counter",{"instance_id"}); +mBvarInt64Adder g_bvar_rpc_kv_begin_sub_txn_put_counter("rpc_kv_begin_sub_txn_put_counter",{"instance_id"}); +mBvarInt64Adder g_bvar_rpc_kv_begin_sub_txn_del_counter("rpc_kv_begin_sub_txn_del_counter",{"instance_id"}); +// abort_sub_txn +mBvarInt64Adder g_bvar_rpc_kv_abort_sub_txn_get_counter("rpc_kv_abort_sub_txn_get_counter",{"instance_id"}); +mBvarInt64Adder g_bvar_rpc_kv_abort_sub_txn_put_counter("rpc_kv_abort_sub_txn_put_counter",{"instance_id"}); +// abort_txn_with_coordinator +mBvarInt64Adder g_bvar_rpc_kv_abort_txn_with_coordinator_get_counter("rpc_kv_abort_txn_with_coordinator_get_counter",{"instance_id"}); +// check_txn_conflict +mBvarInt64Adder g_bvar_rpc_kv_check_txn_conflict_get_counter("rpc_kv_check_txn_conflict_get_counter",{"instance_id"}); +// clean_txn_label +mBvarInt64Adder g_bvar_rpc_kv_clean_txn_label_get_counter("rpc_kv_clean_txn_label_get_counter",{"instance_id"}); +mBvarInt64Adder g_bvar_rpc_kv_clean_txn_label_put_counter("rpc_kv_clean_txn_label_put_counter",{"instance_id"}); +mBvarInt64Adder g_bvar_rpc_kv_clean_txn_label_del_counter("rpc_kv_clean_txn_label_del_counter",{"instance_id"}); +// get_txn_id +mBvarInt64Adder g_bvar_rpc_kv_get_txn_id_get_counter("rpc_kv_get_txn_id_get_counter",{"instance_id"}); + // clang-format on diff --git a/cloud/src/common/bvars.h b/cloud/src/common/bvars.h index 9f4f3d1fccbd78..6034afe7112e32 100644 --- a/cloud/src/common/bvars.h +++ b/cloud/src/common/bvars.h @@ -150,10 +150,11 @@ class mBvarWrapper { auto get(const std::initializer_list& dim_values) { BvarType* stats = counter_.get_stats(std::list(dim_values)); + using ReturnType = decltype(stats->get_value()); if (stats) { return stats->get_value(); } - return std::declval(0); + return ReturnType {}; } private: @@ -174,6 +175,7 @@ class mBvarWrapper { }; using mBvarIntAdder = mBvarWrapper>; +using mBvarInt64Adder = mBvarWrapper>; using mBvarDoubleAdder = mBvarWrapper>; using mBvarIntRecorder = mBvarWrapper; using mBvarLatencyRecorder = mBvarWrapper; @@ -362,3 +364,112 @@ extern BvarStatusWithTag g_bvar_inverted_checker_leaked_delete_bitmaps; extern BvarStatusWithTag g_bvar_inverted_checker_abnormal_delete_bitmaps; extern BvarStatusWithTag g_bvar_inverted_checker_delete_bitmaps_scanned; extern BvarStatusWithTag g_bvar_max_rowsets_with_useless_delete_bitmap_version; + +// rpc kv +extern mBvarInt64Adder g_bvar_rpc_kv_get_rowset_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_get_version_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_get_schema_dict_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_create_tablets_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_create_tablets_put_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_update_tablet_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_update_tablet_put_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_update_tablet_schema_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_update_tablet_schema_put_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_get_tablet_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_prepare_rowset_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_prepare_rowset_put_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_commit_rowset_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_commit_rowset_put_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_commit_rowset_del_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_update_tmp_rowset_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_update_tmp_rowset_put_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_get_tablet_stats_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_update_delete_bitmap_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_update_delete_bitmap_put_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_update_delete_bitmap_del_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_get_delete_bitmap_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_get_delete_bitmap_update_lock_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_get_delete_bitmap_update_lock_put_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_get_delete_bitmap_update_lock_del_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_remove_delete_bitmap_update_lock_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_remove_delete_bitmap_update_lock_put_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_remove_delete_bitmap_update_lock_del_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_remove_delete_bitmap_del_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_start_tablet_job_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_start_tablet_job_put_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_finish_tablet_job_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_finish_tablet_job_put_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_finish_tablet_job_del_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_prepare_index_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_prepare_index_put_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_commit_index_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_commit_index_put_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_commit_index_del_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_drop_index_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_drop_index_put_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_prepare_partition_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_prepare_partition_put_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_commit_partition_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_commit_partition_put_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_commit_partition_del_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_drop_partition_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_drop_partition_put_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_check_kv_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_get_obj_store_info_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_alter_storage_vault_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_alter_storage_vault_put_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_alter_storage_vault_del_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_alter_obj_store_info_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_alter_obj_store_info_put_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_update_ak_sk_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_update_ak_sk_put_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_create_instance_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_create_instance_put_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_get_instance_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_alter_cluster_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_get_cluster_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_get_cluster_put_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_create_stage_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_create_stage_put_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_get_stage_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_get_iam_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_alter_iam_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_alter_iam_put_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_alter_ram_user_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_alter_ram_user_put_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_begin_copy_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_begin_copy_put_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_finish_copy_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_finish_copy_put_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_finish_copy_del_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_get_copy_job_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_get_copy_files_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_filter_copy_files_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_get_cluster_status_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_begin_txn_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_begin_txn_put_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_precommit_txn_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_precommit_txn_put_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_get_rl_task_commit_attach_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_reset_rl_progress_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_reset_rl_progress_put_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_reset_rl_progress_del_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_commit_txn_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_commit_txn_put_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_commit_txn_del_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_abort_txn_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_abort_txn_put_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_abort_txn_del_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_get_txn_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_get_current_max_txn_id_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_begin_sub_txn_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_begin_sub_txn_put_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_begin_sub_txn_del_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_abort_sub_txn_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_abort_sub_txn_put_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_abort_txn_with_coordinator_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_check_txn_conflict_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_clean_txn_label_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_clean_txn_label_put_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_clean_txn_label_del_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_get_txn_id_get_counter; diff --git a/cloud/src/common/stats.h b/cloud/src/common/stats.h new file mode 100644 index 00000000000000..3e9d9a5df1100f --- /dev/null +++ b/cloud/src/common/stats.h @@ -0,0 +1,31 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include + +#include + +namespace doris::cloud { +struct KVStats { + int64_t get_counter {}; + int64_t put_counter {}; + int64_t del_counter {}; +}; + +} // namespace doris::cloud \ No newline at end of file diff --git a/cloud/src/meta-service/mem_txn_kv.cpp b/cloud/src/meta-service/mem_txn_kv.cpp index 84593de3c200f8..ebd62fc12a9015 100644 --- a/cloud/src/meta-service/mem_txn_kv.cpp +++ b/cloud/src/meta-service/mem_txn_kv.cpp @@ -248,6 +248,7 @@ void Transaction::put(std::string_view key, std::string_view val) { writes_.insert_or_assign(k, v); op_list_.emplace_back(ModifyOpType::PUT, k, v); ++num_put_keys_; + kv_->put_count_++; put_bytes_ += key.size() + val.size(); approximate_bytes_ += key.size() + val.size(); } @@ -270,6 +271,8 @@ TxnErrorCode Transaction::get(std::string_view begin, std::string_view end, int limit) { TEST_SYNC_POINT_CALLBACK("memkv::Transaction::get", &limit); std::lock_guard l(lock_); + num_get_keys_++; + kv_->get_count_++; std::string begin_k(begin.data(), begin.size()); std::string end_k(end.data(), end.size()); // TODO: figure out what happen if range_get has part of unreadable_keys @@ -344,12 +347,15 @@ TxnErrorCode Transaction::inner_get(const std::string& begin, const std::string& } std::vector> kv_list(kv_map.begin(), kv_map.end()); + num_get_keys_ += kv_list.size(); + kv_->get_count_ += kv_list.size(); *iter = std::make_unique(std::move(kv_list), more); return TxnErrorCode::TXN_OK; } void Transaction::atomic_set_ver_key(std::string_view key_prefix, std::string_view val) { std::lock_guard l(lock_); + kv_->put_count_++; std::string k(key_prefix.data(), key_prefix.size()); std::string v(val.data(), val.size()); unreadable_keys_.insert(k); @@ -362,6 +368,7 @@ void Transaction::atomic_set_ver_key(std::string_view key_prefix, std::string_vi void Transaction::atomic_set_ver_value(std::string_view key, std::string_view value) { std::lock_guard l(lock_); + kv_->put_count_++; std::string k(key.data(), key.size()); std::string v(value.data(), value.size()); unreadable_keys_.insert(k); @@ -377,6 +384,7 @@ void Transaction::atomic_add(std::string_view key, int64_t to_add) { std::string v(sizeof(to_add), '\0'); memcpy(v.data(), &to_add, sizeof(to_add)); std::lock_guard l(lock_); + kv_->put_count_++; op_list_.emplace_back(ModifyOpType::ATOMIC_ADD, std::move(k), std::move(v)); ++num_put_keys_; @@ -395,6 +403,7 @@ bool Transaction::decode_atomic_int(std::string_view data, int64_t* val) { void Transaction::remove(std::string_view key) { std::lock_guard l(lock_); + kv_->del_count_++; std::string k(key.data(), key.size()); writes_.erase(k); std::string end_key = k; @@ -421,7 +430,9 @@ void Transaction::remove(std::string_view begin, std::string_view end) { remove_ranges_.emplace_back(begin_k, end_k); op_list_.emplace_back(ModifyOpType::REMOVE_RANGE, begin_k, end_k); } - ++num_del_keys_; + kv_->del_count_ += 2; + // same as normal txn + num_del_keys_ += 2; delete_bytes_ += begin.size() + end.size(); approximate_bytes_ += begin.size() + end.size(); } @@ -480,6 +491,8 @@ TxnErrorCode Transaction::batch_get(std::vector>* res auto ret = inner_get(k, &val, opts.snapshot); ret == TxnErrorCode::TXN_OK ? res->push_back(val) : res->push_back(std::nullopt); } + kv_->get_count_ += keys.size(); + num_get_keys_ += keys.size(); return TxnErrorCode::TXN_OK; } diff --git a/cloud/src/meta-service/mem_txn_kv.h b/cloud/src/meta-service/mem_txn_kv.h index 63fb008f586ed8..1046adcee1ce33 100644 --- a/cloud/src/meta-service/mem_txn_kv.h +++ b/cloud/src/meta-service/mem_txn_kv.h @@ -17,6 +17,7 @@ #pragma once +#include #include #include #include @@ -57,6 +58,10 @@ class MemTxnKv : public TxnKv, public std::enable_shared_from_this { TxnErrorCode get_kv(const std::string& begin, const std::string& end, int64_t version, int limit, bool* more, std::map* kv_list); + int64_t get_count_ {}; + int64_t put_count_ {}; + int64_t del_count_ {}; + private: using OpTuple = std::tuple; TxnErrorCode update(const std::set& read_set, const std::vector& op_list, @@ -189,6 +194,8 @@ class Transaction : public cloud::Transaction { size_t approximate_bytes() const override { return approximate_bytes_; } + size_t num_get_keys() const override { return num_get_keys_; } + size_t num_del_keys() const override { return num_del_keys_; } size_t num_put_keys() const override { return num_put_keys_; } @@ -218,6 +225,7 @@ class Transaction : public cloud::Transaction { int64_t read_version_ = -1; size_t approximate_bytes_ {0}; + size_t num_get_keys_ {0}; size_t num_del_keys_ {0}; size_t num_put_keys_ {0}; size_t delete_bytes_ {0}; diff --git a/cloud/src/meta-service/meta_service.cpp b/cloud/src/meta-service/meta_service.cpp index 962c4b0efcf11c..ba1eded05f5823 100644 --- a/cloud/src/meta-service/meta_service.cpp +++ b/cloud/src/meta-service/meta_service.cpp @@ -38,6 +38,7 @@ #include #include #include +#include #include #include #include @@ -50,6 +51,7 @@ #include "common/config.h" #include "common/encryption_util.h" #include "common/logging.h" +#include "common/stats.h" #include "common/stopwatch.h" #include "common/string_util.h" #include "common/util.h" @@ -211,7 +213,7 @@ void MetaServiceImpl::get_version(::google::protobuf::RpcController* controller, return; } - RPC_PREPROCESS(get_version); + RPC_PREPROCESS(get_version, get); std::string cloud_unique_id; if (request->has_cloud_unique_id()) { cloud_unique_id = request->cloud_unique_id(); @@ -251,7 +253,7 @@ void MetaServiceImpl::get_version(::google::protobuf::RpcController* controller, } code = MetaServiceCode::OK; - std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); if (err != TxnErrorCode::TXN_OK) { msg = "failed to create txn"; @@ -303,7 +305,7 @@ void MetaServiceImpl::batch_get_version(::google::protobuf::RpcController* contr const GetVersionRequest* request, GetVersionResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(get_version); + RPC_PREPROCESS(get_version, get); std::string cloud_unique_id; if (request->has_cloud_unique_id()) { @@ -359,7 +361,10 @@ void MetaServiceImpl::batch_get_version(::google::protobuf::RpcController* contr code = cast_as(err); break; } - + DORIS_CLOUD_DEFER { + if (txn == nullptr) return; + stats.get_counter += txn->num_get_keys(); + }; for (size_t i = response->versions_size(); i < num_acquired; i += BATCH_SIZE) { size_t limit = (i + BATCH_SIZE < num_acquired) ? i + BATCH_SIZE : num_acquired; version_keys.clear(); @@ -434,7 +439,7 @@ void MetaServiceImpl::batch_get_version(::google::protobuf::RpcController* contr void internal_create_tablet(const CreateTabletsRequest* request, MetaServiceCode& code, std::string& msg, const doris::TabletMetaCloudPB& meta, std::shared_ptr txn_kv, const std::string& instance_id, - std::set>& saved_schema) { + std::set>& saved_schema, KVStats& stats) { doris::TabletMetaCloudPB tablet_meta(meta); bool has_first_rowset = tablet_meta.rs_metas_size() > 0; @@ -457,6 +462,11 @@ void internal_create_tablet(const CreateTabletsRequest* request, MetaServiceCode msg = "failed to init txn"; return; } + DORIS_CLOUD_DEFER { + if (txn == nullptr) return; + stats.get_counter += txn->num_get_keys(); + stats.put_counter += txn->num_put_keys(); + }; std::string rs_key, rs_val; if (has_first_rowset) { @@ -570,7 +580,7 @@ void MetaServiceImpl::create_tablets(::google::protobuf::RpcController* controll const CreateTabletsRequest* request, CreateTabletsResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(create_tablets); + RPC_PREPROCESS(create_tablets, get, put); if (request->tablet_metas_size() == 0) { msg = "no tablet meta"; @@ -601,6 +611,7 @@ void MetaServiceImpl::create_tablets(::google::protobuf::RpcController* controll instance_key(key_info, &key); err = txn0->get(key, &val); + stats.get_counter++; LOG(INFO) << "get instance_key=" << hex(key); if (err != TxnErrorCode::TXN_OK) { @@ -609,7 +620,6 @@ void MetaServiceImpl::create_tablets(::google::protobuf::RpcController* controll msg = ss.str(); return; } - if (!instance.ParseFromString(val)) { code = MetaServiceCode::PROTOBUF_PARSE_ERR; msg = "failed to parse InstanceInfoPB"; @@ -653,7 +663,8 @@ void MetaServiceImpl::create_tablets(::google::protobuf::RpcController* controll std::set> saved_schema; TEST_SYNC_POINT_RETURN_WITH_VOID("create_tablets"); for (auto& tablet_meta : request->tablet_metas()) { - internal_create_tablet(request, code, msg, tablet_meta, txn_kv_, instance_id, saved_schema); + internal_create_tablet(request, code, msg, tablet_meta, txn_kv_, instance_id, saved_schema, + stats); if (code != MetaServiceCode::OK) { return; } @@ -725,7 +736,7 @@ void MetaServiceImpl::update_tablet(::google::protobuf::RpcController* controlle const UpdateTabletRequest* request, UpdateTabletResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(update_tablet); + RPC_PREPROCESS(update_tablet, get, put); instance_id = get_instance_id(resource_mgr_, request->cloud_unique_id()); if (instance_id.empty()) { code = MetaServiceCode::INVALID_ARGUMENT; @@ -734,7 +745,7 @@ void MetaServiceImpl::update_tablet(::google::protobuf::RpcController* controlle return; } RPC_RATE_LIMIT(update_tablet) - std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); if (err != TxnErrorCode::TXN_OK) { code = cast_as(err); @@ -834,7 +845,7 @@ void MetaServiceImpl::update_tablet_schema(::google::protobuf::RpcController* co UpdateTabletSchemaResponse* response, ::google::protobuf::Closure* done) { DCHECK(false) << "should not call update_tablet_schema"; - RPC_PREPROCESS(update_tablet_schema); + RPC_PREPROCESS(update_tablet_schema, get, put); instance_id = get_instance_id(resource_mgr_, request->cloud_unique_id()); if (instance_id.empty()) { code = MetaServiceCode::INVALID_ARGUMENT; @@ -845,7 +856,6 @@ void MetaServiceImpl::update_tablet_schema(::google::protobuf::RpcController* co RPC_RATE_LIMIT(update_tablet_schema) - std::unique_ptr txn; TxnErrorCode err = txn_kv_->create_txn(&txn); if (err != TxnErrorCode::TXN_OK) { code = cast_as(err); @@ -907,7 +917,7 @@ void MetaServiceImpl::update_tablet_schema(::google::protobuf::RpcController* co void MetaServiceImpl::get_tablet(::google::protobuf::RpcController* controller, const GetTabletRequest* request, GetTabletResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(get_tablet); + RPC_PREPROCESS(get_tablet, get); instance_id = get_instance_id(resource_mgr_, request->cloud_unique_id()); if (instance_id.empty()) { code = MetaServiceCode::INVALID_ARGUMENT; @@ -916,7 +926,7 @@ void MetaServiceImpl::get_tablet(::google::protobuf::RpcController* controller, return; } RPC_RATE_LIMIT(get_tablet) - std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); if (err != TxnErrorCode::TXN_OK) { code = cast_as(err); @@ -1125,7 +1135,7 @@ void MetaServiceImpl::prepare_rowset(::google::protobuf::RpcController* controll const CreateRowsetRequest* request, CreateRowsetResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(prepare_rowset); + RPC_PREPROCESS(prepare_rowset, get, put); if (!request->has_rowset_meta()) { code = MetaServiceCode::INVALID_ARGUMENT; msg = "no rowset meta"; @@ -1150,7 +1160,7 @@ void MetaServiceImpl::prepare_rowset(::google::protobuf::RpcController* controll int64_t tablet_id = rowset_meta.tablet_id(); const auto& rowset_id = rowset_meta.rowset_id_v2(); auto tmp_rs_key = meta_rowset_tmp_key({instance_id, rowset_meta.txn_id(), tablet_id}); - std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); if (err != TxnErrorCode::TXN_OK) { code = cast_as(err); @@ -1270,7 +1280,7 @@ void MetaServiceImpl::commit_rowset(::google::protobuf::RpcController* controlle const CreateRowsetRequest* request, CreateRowsetResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(commit_rowset); + RPC_PREPROCESS(commit_rowset, get, put, del); if (!request->has_rowset_meta()) { code = MetaServiceCode::INVALID_ARGUMENT; msg = "no rowset meta"; @@ -1296,7 +1306,6 @@ void MetaServiceImpl::commit_rowset(::google::protobuf::RpcController* controlle auto tmp_rs_key = meta_rowset_tmp_key({instance_id, rowset_meta.txn_id(), tablet_id}); - std::unique_ptr txn; TxnErrorCode err = txn_kv_->create_txn(&txn); if (err != TxnErrorCode::TXN_OK) { code = cast_as(err); @@ -1431,7 +1440,7 @@ void MetaServiceImpl::update_tmp_rowset(::google::protobuf::RpcController* contr const CreateRowsetRequest* request, CreateRowsetResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(update_tmp_rowset); + RPC_PREPROCESS(update_tmp_rowset, get, put); if (!request->has_rowset_meta()) { code = MetaServiceCode::INVALID_ARGUMENT; msg = "no rowset meta"; @@ -1460,7 +1469,6 @@ void MetaServiceImpl::update_tmp_rowset(::google::protobuf::RpcController* contr MetaRowsetTmpKeyInfo key_info {instance_id, txn_id, tablet_id}; meta_rowset_tmp_key(key_info, &update_key); - std::unique_ptr txn; TxnErrorCode err = txn_kv_->create_txn(&txn); if (err != TxnErrorCode::TXN_OK) { code = cast_as(err); @@ -1688,7 +1696,7 @@ static bool try_fetch_and_parse_schema(Transaction* txn, RowsetMetaCloudPB& rows void MetaServiceImpl::get_rowset(::google::protobuf::RpcController* controller, const GetRowsetRequest* request, GetRowsetResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(get_rowset); + RPC_PREPROCESS(get_rowset, get); instance_id = get_instance_id(resource_mgr_, request->cloud_unique_id()); if (instance_id.empty()) { code = MetaServiceCode::INVALID_ARGUMENT; @@ -1724,7 +1732,10 @@ void MetaServiceImpl::get_rowset(::google::protobuf::RpcController* controller, LOG(WARNING) << msg; return; } - + DORIS_CLOUD_DEFER { + if (txn == nullptr) return; + stats.get_counter += txn->num_get_keys(); + }; TabletIndexPB idx; // Get tablet id index from kv get_tablet_idx(code, msg, txn.get(), instance_id, tablet_id, idx); @@ -1763,6 +1774,7 @@ void MetaServiceImpl::get_rowset(::google::protobuf::RpcController* controller, if (version_pb.pending_txn_ids_size() > 0) { DCHECK(version_pb.pending_txn_ids_size() == 1); + stats.get_counter += txn->num_get_keys(); txn.reset(); TEST_SYNC_POINT_CALLBACK("get_rowset::advance_last_pending_txn_id", &version_pb); @@ -1876,7 +1888,7 @@ void MetaServiceImpl::get_tablet_stats(::google::protobuf::RpcController* contro const GetTabletStatsRequest* request, GetTabletStatsResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(get_tablet_stats); + RPC_PREPROCESS(get_tablet_stats, get); instance_id = get_instance_id(resource_mgr_, request->cloud_unique_id()); if (instance_id.empty()) { code = MetaServiceCode::INVALID_ARGUMENT; @@ -1886,7 +1898,6 @@ void MetaServiceImpl::get_tablet_stats(::google::protobuf::RpcController* contro } RPC_RATE_LIMIT(get_tablet_stats) - std::unique_ptr txn; for (auto& i : request->tablet_idx()) { TabletIndexPB idx(i); // FIXME(plat1ko): Get all tablet stats in one txn @@ -1896,6 +1907,11 @@ void MetaServiceImpl::get_tablet_stats(::google::protobuf::RpcController* contro msg = fmt::format("failed to create txn, tablet_id={}", idx.tablet_id()); return; } + DORIS_CLOUD_DEFER { + stats.get_counter += txn->num_get_keys(); + // the txn is not a local variable, if not reset will count last res twice + txn.reset(nullptr); + }; if (!(/* idx.has_db_id() && */ idx.has_table_id() && idx.has_index_id() && idx.has_partition_id() && i.has_tablet_id())) { get_tablet_idx(code, msg, txn.get(), instance_id, idx.tablet_id(), idx); @@ -2133,7 +2149,7 @@ void MetaServiceImpl::update_delete_bitmap(google::protobuf::RpcController* cont const UpdateDeleteBitmapRequest* request, UpdateDeleteBitmapResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(update_delete_bitmap); + RPC_PREPROCESS(update_delete_bitmap, get, put, del); std::string cloud_unique_id = request->has_cloud_unique_id() ? request->cloud_unique_id() : ""; if (cloud_unique_id.empty()) { code = MetaServiceCode::INVALID_ARGUMENT; @@ -2167,7 +2183,6 @@ void MetaServiceImpl::update_delete_bitmap(google::protobuf::RpcController* cont auto table_id = request->table_id(); auto tablet_id = request->tablet_id(); - std::unique_ptr txn; TxnErrorCode err = txn_kv_->create_txn(&txn); if (err != TxnErrorCode::TXN_OK) { code = cast_as(err); @@ -2275,6 +2290,9 @@ void MetaServiceImpl::update_delete_bitmap(google::protobuf::RpcController* cont g_bvar_update_delete_bitmap_fail_counter << 1; return; } + stats.get_counter += txn->num_get_keys(); + stats.put_counter += txn->num_put_keys(); + stats.del_counter += txn->num_del_keys(); current_key_count = 0; current_value_count = 0; TxnErrorCode err = txn_kv_->create_txn(&txn); @@ -2427,7 +2445,7 @@ void MetaServiceImpl::get_delete_bitmap(google::protobuf::RpcController* control const GetDeleteBitmapRequest* request, GetDeleteBitmapResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(get_delete_bitmap); + RPC_PREPROCESS(get_delete_bitmap, get); std::string cloud_unique_id = request->has_cloud_unique_id() ? request->cloud_unique_id() : ""; if (cloud_unique_id.empty()) { code = MetaServiceCode::INVALID_ARGUMENT; @@ -2473,6 +2491,10 @@ void MetaServiceImpl::get_delete_bitmap(google::protobuf::RpcController* control msg = "failed to init txn"; return; } + DORIS_CLOUD_DEFER { + if (txn == nullptr) return; + stats.get_counter += txn->num_get_keys(); + }; MetaDeleteBitmapInfo start_key_info {instance_id, tablet_id, rowset_ids[i], begin_versions[i], 0}; MetaDeleteBitmapInfo end_key_info {instance_id, tablet_id, rowset_ids[i], end_versions[i], @@ -2499,6 +2521,7 @@ void MetaServiceImpl::get_delete_bitmap(google::protobuf::RpcController* control TEST_SYNC_POINT_CALLBACK("get_delete_bitmap_err", &round, &err); int64_t retry = 0; while (err == TxnErrorCode::TXN_TOO_OLD && retry < 3) { + stats.get_counter += txn->num_get_keys(); txn = nullptr; err = txn_kv_->create_txn(&txn); if (err != TxnErrorCode::TXN_OK) { @@ -2590,6 +2613,10 @@ void MetaServiceImpl::get_delete_bitmap(google::protobuf::RpcController* control msg = "failed to init txn"; return; } + DORIS_CLOUD_DEFER { + if (txn == nullptr) return; + stats.get_counter += txn->num_get_keys(); + }; TabletIndexPB idx(request->idx()); TabletStatsPB tablet_stat; internal_get_tablet_stats(code, msg, txn.get(), instance_id, idx, tablet_stat, @@ -2656,7 +2683,7 @@ bool MetaServiceImpl::get_mow_tablet_stats_and_meta(MetaServiceCode& code, std:: const GetDeleteBitmapUpdateLockRequest* request, GetDeleteBitmapUpdateLockResponse* response, std::string& instance_id, std::string& lock_key, - std::string lock_use_version) { + std::string lock_use_version, KVStats& stats) { bool require_tablet_stats = request->has_require_compaction_stats() ? request->require_compaction_stats() : false; if (!require_tablet_stats) return true; @@ -2676,6 +2703,12 @@ bool MetaServiceImpl::get_mow_tablet_stats_and_meta(MetaServiceCode& code, std:: msg = "failed to init txn"; return false; } + DORIS_CLOUD_DEFER { + if (txn == nullptr) return; + stats.get_counter += txn->num_get_keys(); + stats.put_counter += txn->num_put_keys(); + stats.del_counter += txn->num_del_keys(); + }; auto table_id = request->table_id(); std::stringstream ss; if (!config::enable_batch_get_mow_tablet_stats_and_meta) { @@ -2850,7 +2883,8 @@ void MetaServiceImpl::get_delete_bitmap_update_lock_v2( google::protobuf::RpcController* controller, const GetDeleteBitmapUpdateLockRequest* request, GetDeleteBitmapUpdateLockResponse* response, ::google::protobuf::Closure* done, - std::string& instance_id, MetaServiceCode& code, std::string& msg, std::stringstream& ss) { + std::string& instance_id, MetaServiceCode& code, std::string& msg, std::stringstream& ss, + KVStats& stats) { VLOG_DEBUG << "get delete bitmap update lock in v2 for table=" << request->table_id() << ",lock id=" << request->lock_id() << ",initiator=" << request->initiator(); auto table_id = request->table_id(); @@ -2867,6 +2901,12 @@ void MetaServiceImpl::get_delete_bitmap_update_lock_v2( msg = "failed to init txn"; return; } + DORIS_CLOUD_DEFER { + if (txn == nullptr) return; + stats.get_counter += txn->num_get_keys(); + stats.put_counter += txn->num_put_keys(); + stats.del_counter += txn->num_del_keys(); + }; std::string lock_val; DeleteBitmapUpdateLockPB lock_info; err = txn->get(lock_key, &lock_val); @@ -3079,7 +3119,8 @@ void MetaServiceImpl::get_delete_bitmap_update_lock_v2( } } - if (!get_mow_tablet_stats_and_meta(code, msg, request, response, instance_id, lock_key, "v2")) { + if (!get_mow_tablet_stats_and_meta(code, msg, request, response, instance_id, lock_key, "v2", + stats)) { return; } } @@ -3088,7 +3129,8 @@ void MetaServiceImpl::get_delete_bitmap_update_lock_v1( google::protobuf::RpcController* controller, const GetDeleteBitmapUpdateLockRequest* request, GetDeleteBitmapUpdateLockResponse* response, ::google::protobuf::Closure* done, - std::string& instance_id, MetaServiceCode& code, std::string& msg, std::stringstream& ss) { + std::string& instance_id, MetaServiceCode& code, std::string& msg, std::stringstream& ss, + KVStats& stats) { VLOG_DEBUG << "get delete bitmap update lock in v1 for table=" << request->table_id() << ",lock id=" << request->lock_id() << ",initiator=" << request->initiator(); std::unique_ptr txn; @@ -3098,6 +3140,12 @@ void MetaServiceImpl::get_delete_bitmap_update_lock_v1( msg = "failed to init txn"; return; } + DORIS_CLOUD_DEFER { + if (txn == nullptr) return; + stats.get_counter += txn->num_get_keys(); + stats.put_counter += txn->num_put_keys(); + stats.del_counter += txn->num_del_keys(); + }; auto table_id = request->table_id(); std::string lock_key = meta_delete_bitmap_update_lock_key({instance_id, table_id, -1}); std::string lock_val; @@ -3163,7 +3211,8 @@ void MetaServiceImpl::get_delete_bitmap_update_lock_v1( return; } - if (!get_mow_tablet_stats_and_meta(code, msg, request, response, instance_id, lock_key, "v1")) { + if (!get_mow_tablet_stats_and_meta(code, msg, request, response, instance_id, lock_key, "v1", + stats)) { return; }; } @@ -3172,7 +3221,8 @@ void MetaServiceImpl::remove_delete_bitmap_update_lock_v2( google::protobuf::RpcController* controller, const RemoveDeleteBitmapUpdateLockRequest* request, RemoveDeleteBitmapUpdateLockResponse* response, ::google::protobuf::Closure* done, - std::string& instance_id, MetaServiceCode& code, std::string& msg, std::stringstream& ss) { + std::string& instance_id, MetaServiceCode& code, std::string& msg, std::stringstream& ss, + KVStats& stats) { VLOG_DEBUG << "remove delete bitmap update lock in v2 for table=" << request->table_id() << ",lock id=" << request->lock_id() << ",initiator=" << request->initiator(); std::unique_ptr txn; @@ -3182,6 +3232,12 @@ void MetaServiceImpl::remove_delete_bitmap_update_lock_v2( msg = "failed to init txn"; return; } + DORIS_CLOUD_DEFER { + if (txn == nullptr) return; + stats.get_counter += txn->num_get_keys(); + stats.put_counter += txn->num_put_keys(); + stats.del_counter += txn->num_del_keys(); + }; if (request->lock_id() == COMPACTION_DELETE_BITMAP_LOCK_ID) { std::string tablet_compaction_key = mow_tablet_compaction_key({instance_id, request->table_id(), request->initiator()}); @@ -3251,7 +3307,8 @@ void MetaServiceImpl::remove_delete_bitmap_update_lock_v1( google::protobuf::RpcController* controller, const RemoveDeleteBitmapUpdateLockRequest* request, RemoveDeleteBitmapUpdateLockResponse* response, ::google::protobuf::Closure* done, - std::string& instance_id, MetaServiceCode& code, std::string& msg, std::stringstream& ss) { + std::string& instance_id, MetaServiceCode& code, std::string& msg, std::stringstream& ss, + KVStats& stats) { VLOG_DEBUG << "remove delete bitmap update lock in v1 for table=" << request->table_id() << ",lock id=" << request->lock_id() << ",initiator=" << request->initiator(); std::unique_ptr txn; @@ -3261,6 +3318,12 @@ void MetaServiceImpl::remove_delete_bitmap_update_lock_v1( msg = "failed to init txn"; return; } + DORIS_CLOUD_DEFER { + if (txn == nullptr) return; + stats.get_counter += txn->num_get_keys(); + stats.put_counter += txn->num_put_keys(); + stats.del_counter += txn->num_del_keys(); + }; std::string lock_key = meta_delete_bitmap_update_lock_key({instance_id, request->table_id(), -1}); std::string lock_val; @@ -3317,7 +3380,7 @@ void MetaServiceImpl::get_delete_bitmap_update_lock(google::protobuf::RpcControl const GetDeleteBitmapUpdateLockRequest* request, GetDeleteBitmapUpdateLockResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(get_delete_bitmap_update_lock); + RPC_PREPROCESS(get_delete_bitmap_update_lock, get, put, del); std::string cloud_unique_id = request->has_cloud_unique_id() ? request->cloud_unique_id() : ""; if (cloud_unique_id.empty()) { code = MetaServiceCode::INVALID_ARGUMENT; @@ -3337,10 +3400,10 @@ void MetaServiceImpl::get_delete_bitmap_update_lock(google::protobuf::RpcControl << " use_version=" << use_version; if (use_version == "v2") { get_delete_bitmap_update_lock_v2(controller, request, response, done, instance_id, code, - msg, ss); + msg, ss, stats); } else { get_delete_bitmap_update_lock_v1(controller, request, response, done, instance_id, code, - msg, ss); + msg, ss, stats); } } @@ -3348,7 +3411,7 @@ void MetaServiceImpl::remove_delete_bitmap_update_lock( google::protobuf::RpcController* controller, const RemoveDeleteBitmapUpdateLockRequest* request, RemoveDeleteBitmapUpdateLockResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(remove_delete_bitmap_update_lock); + RPC_PREPROCESS(remove_delete_bitmap_update_lock, get, put, del); std::string cloud_unique_id = request->has_cloud_unique_id() ? request->cloud_unique_id() : ""; if (cloud_unique_id.empty()) { code = MetaServiceCode::INVALID_ARGUMENT; @@ -3368,10 +3431,10 @@ void MetaServiceImpl::remove_delete_bitmap_update_lock( << " use_version=" << use_version; if (use_version == "v2") { remove_delete_bitmap_update_lock_v2(controller, request, response, done, instance_id, code, - msg, ss); + msg, ss, stats); } else { remove_delete_bitmap_update_lock_v1(controller, request, response, done, instance_id, code, - msg, ss); + msg, ss, stats); } } @@ -3379,7 +3442,7 @@ void MetaServiceImpl::remove_delete_bitmap(google::protobuf::RpcController* cont const RemoveDeleteBitmapRequest* request, RemoveDeleteBitmapResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(remove_delete_bitmap); + RPC_PREPROCESS(remove_delete_bitmap, del); std::string cloud_unique_id = request->has_cloud_unique_id() ? request->cloud_unique_id() : ""; if (cloud_unique_id.empty()) { code = MetaServiceCode::INVALID_ARGUMENT; @@ -3408,7 +3471,6 @@ void MetaServiceImpl::remove_delete_bitmap(google::protobuf::RpcController* cont msg = ss.str(); return; } - std::unique_ptr txn; TxnErrorCode err = txn_kv_->create_txn(&txn); if (err != TxnErrorCode::TXN_OK) { LOG(WARNING) << "failed to init txn"; @@ -3530,7 +3592,7 @@ void MetaServiceImpl::get_schema_dict(::google::protobuf::RpcController* control const GetSchemaDictRequest* request, GetSchemaDictResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(get_schema_dict); + RPC_PREPROCESS(get_schema_dict, get); instance_id = get_instance_id(resource_mgr_, request->cloud_unique_id()); if (instance_id.empty()) { code = MetaServiceCode::INVALID_ARGUMENT; @@ -3547,7 +3609,6 @@ void MetaServiceImpl::get_schema_dict(::google::protobuf::RpcController* control RPC_RATE_LIMIT(get_schema_dict) - std::unique_ptr txn; TxnErrorCode err = txn_kv_->create_txn(&txn); if (err != TxnErrorCode::TXN_OK) { code = cast_as(err); diff --git a/cloud/src/meta-service/meta_service.h b/cloud/src/meta-service/meta_service.h index 7d22a14ad0371d..45c9a486d086bc 100644 --- a/cloud/src/meta-service/meta_service.h +++ b/cloud/src/meta-service/meta_service.h @@ -28,6 +28,7 @@ #include #include "common/config.h" +#include "common/stats.h" #include "cpp/sync_point.h" #include "meta-service/delete_bitmap_lock_white_list.h" #include "meta-service/txn_kv.h" @@ -337,35 +338,37 @@ class MetaServiceImpl : public cloud::MetaService { const GetDeleteBitmapUpdateLockRequest* request, GetDeleteBitmapUpdateLockResponse* response, std::string& instance_id, std::string& lock_key, - std::string lock_use_version); + std::string lock_use_version, KVStats& stats); void get_delete_bitmap_update_lock_v2(google::protobuf::RpcController* controller, const GetDeleteBitmapUpdateLockRequest* request, GetDeleteBitmapUpdateLockResponse* response, ::google::protobuf::Closure* done, std::string& instance_id, MetaServiceCode& code, - std::string& msg, std::stringstream& ss); + std::string& msg, std::stringstream& ss, KVStats& stats); void get_delete_bitmap_update_lock_v1(google::protobuf::RpcController* controller, const GetDeleteBitmapUpdateLockRequest* request, GetDeleteBitmapUpdateLockResponse* response, ::google::protobuf::Closure* done, std::string& instance_id, MetaServiceCode& code, - std::string& msg, std::stringstream& ss); + std::string& msg, std::stringstream& ss, KVStats& stats); void remove_delete_bitmap_update_lock_v2(google::protobuf::RpcController* controller, const RemoveDeleteBitmapUpdateLockRequest* request, RemoveDeleteBitmapUpdateLockResponse* response, ::google::protobuf::Closure* done, std::string& instance_id, MetaServiceCode& code, - std::string& msg, std::stringstream& ss); + std::string& msg, std::stringstream& ss, + KVStats& stats); void remove_delete_bitmap_update_lock_v1(google::protobuf::RpcController* controller, const RemoveDeleteBitmapUpdateLockRequest* request, RemoveDeleteBitmapUpdateLockResponse* response, ::google::protobuf::Closure* done, std::string& instance_id, MetaServiceCode& code, - std::string& msg, std::stringstream& ss); + std::string& msg, std::stringstream& ss, + KVStats& stats); std::shared_ptr txn_kv_; std::shared_ptr resource_mgr_; diff --git a/cloud/src/meta-service/meta_service_helper.h b/cloud/src/meta-service/meta_service_helper.h index 5e724eae14f619..256816b68bb3cf 100644 --- a/cloud/src/meta-service/meta_service_helper.h +++ b/cloud/src/meta-service/meta_service_helper.h @@ -30,6 +30,7 @@ #include "common/config.h" #include "common/defer.h" #include "common/logging.h" +#include "common/stats.h" #include "common/stopwatch.h" #include "common/util.h" #include "cpp/sync_point.h" @@ -226,24 +227,57 @@ inline MetaServiceCode cast_as(TxnErrorCode code) { } } -#define RPC_PREPROCESS(func_name) \ - StopWatch sw; \ - auto ctrl = static_cast(controller); \ - begin_rpc(#func_name, ctrl, request); \ - brpc::ClosureGuard closure_guard(done); \ - [[maybe_unused]] std::stringstream ss; \ - [[maybe_unused]] MetaServiceCode code = MetaServiceCode::OK; \ - [[maybe_unused]] std::string msg; \ - [[maybe_unused]] std::string instance_id; \ - [[maybe_unused]] bool drop_request = false; \ - DORIS_CLOUD_DEFER { \ - response->mutable_status()->set_code(code); \ - response->mutable_status()->set_msg(msg); \ - finish_rpc(#func_name, ctrl, response); \ - closure_guard.reset(nullptr); \ - if (config::use_detailed_metrics && !instance_id.empty() && !drop_request) { \ - g_bvar_ms_##func_name.put(instance_id, sw.elapsed_us()); \ - } \ +// don't use these macro it just for defer count, reduce useless variable(some rpc just need one of rw op) +// If we have to write separate code for each RPC, it would be quite troublesome +// After all, adding put, get, and del after the RPC_PREPROCESS macro is simpler than writing a long string of code +#define RPCKVCOUNTHELPER(func_name, op) \ + g_bvar_rpc_kv_##func_name##_##op##_counter.put({instance_id}, stats.op##_counter); +#define RPCKVCOUNT_0(func_name) +#define RPCKVCOUNT_1(func_name, op1) RPCKVCOUNTHELPER(func_name, op1) +#define RPCKVCOUNT_2(func_name, op1, op2) \ + RPCKVCOUNT_1(func_name, op1) RPCKVCOUNTHELPER(func_name, op2) +#define RPCKVCOUNT_3(func_name, op1, op2, op3) \ + RPCKVCOUNT_2(func_name, op1, op2) RPCKVCOUNTHELPER(func_name, op3) +#define GET_RPCKVCOUNT_MACRO(_0, _1, _2, _3, NAME, ...) NAME + +// input func_name, count type(get, put, del), make sure the counter is exist +// about defer_count: +// which means that these bvars will only be counted after stats has finished counting. +// why not cancle KVStats, count directly? +// 1. some RPC operations call functions and function reset txn it also need to be counted +// 2. some function such as `scan_tmp_rowset` it used by RPC(commit_txn) and non rpc +// maybe we can add a bool variable to judge weather we need count, but if have more complex situation +// `func1` used by RPC1, RPC2 and RPC3 judge it or just give func1 a pointer +#define RPC_PREPROCESS(func_name, ...) \ + StopWatch sw; \ + auto ctrl = static_cast(controller); \ + begin_rpc(#func_name, ctrl, request); \ + brpc::ClosureGuard closure_guard(done); \ + [[maybe_unused]] std::stringstream ss; \ + [[maybe_unused]] MetaServiceCode code = MetaServiceCode::OK; \ + [[maybe_unused]] std::unique_ptr txn; \ + [[maybe_unused]] std::string msg; \ + [[maybe_unused]] std::string instance_id; \ + [[maybe_unused]] bool drop_request = false; \ + [[maybe_unused]] KVStats stats; \ + DORIS_CLOUD_DEFER { \ + response->mutable_status()->set_code(code); \ + response->mutable_status()->set_msg(msg); \ + finish_rpc(#func_name, ctrl, response); \ + closure_guard.reset(nullptr); \ + if (txn != nullptr) { \ + stats.get_counter += txn->num_get_keys(); \ + stats.put_counter += txn->num_put_keys(); \ + stats.del_counter += txn->num_del_keys(); \ + } \ + if (config::use_detailed_metrics && !instance_id.empty()) { \ + if (!drop_request) { \ + g_bvar_ms_##func_name.put(instance_id, sw.elapsed_us()); \ + } \ + GET_RPCKVCOUNT_MACRO(_0, ##__VA_ARGS__, RPCKVCOUNT_3, RPCKVCOUNT_2, RPCKVCOUNT_1, \ + RPCKVCOUNT_0) \ + (func_name, ##__VA_ARGS__) \ + } \ }; #define RPC_RATE_LIMIT(func_name) \ @@ -272,7 +306,8 @@ int decrypt_instance_info(InstanceInfoPB& instance, const std::string& instance_ /** * Notifies other metaservice to refresh instance */ -void notify_refresh_instance(std::shared_ptr txn_kv, const std::string& instance_id); +void notify_refresh_instance(std::shared_ptr txn_kv, const std::string& instance_id, + KVStats* stats); void get_tablet_idx(MetaServiceCode& code, std::string& msg, Transaction* txn, const std::string& instance_id, int64_t tablet_id, TabletIndexPB& tablet_idx); diff --git a/cloud/src/meta-service/meta_service_job.cpp b/cloud/src/meta-service/meta_service_job.cpp index f62931f18ff81a..00d4dde51cec35 100644 --- a/cloud/src/meta-service/meta_service_job.cpp +++ b/cloud/src/meta-service/meta_service_job.cpp @@ -28,6 +28,7 @@ #include "common/bvars.h" #include "common/config.h" #include "common/logging.h" +#include "common/stats.h" #include "common/util.h" #include "cpp/sync_point.h" #include "keys.h" @@ -407,7 +408,7 @@ void MetaServiceImpl::start_tablet_job(::google::protobuf::RpcController* contro const StartTabletJobRequest* request, StartTabletJobResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(start_tablet_job); + RPC_PREPROCESS(start_tablet_job, get, put); std::string cloud_unique_id = request->cloud_unique_id(); instance_id = get_instance_id(resource_mgr_, cloud_unique_id); if (instance_id.empty()) { @@ -425,7 +426,6 @@ void MetaServiceImpl::start_tablet_job(::google::protobuf::RpcController* contro return; } - std::unique_ptr txn; TxnErrorCode err = txn_kv_->create_txn(&txn); if (err != TxnErrorCode::TXN_OK) { code = cast_as(err); @@ -1473,7 +1473,7 @@ void MetaServiceImpl::finish_tablet_job(::google::protobuf::RpcController* contr const FinishTabletJobRequest* request, FinishTabletJobResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(finish_tablet_job); + 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()) { @@ -1493,7 +1493,6 @@ void MetaServiceImpl::finish_tablet_job(::google::protobuf::RpcController* contr } bool need_commit = false; - std::unique_ptr txn; TxnErrorCode err = txn_kv_->create_txn(&txn); if (err != TxnErrorCode::TXN_OK) { code = cast_as(err); diff --git a/cloud/src/meta-service/meta_service_partition.cpp b/cloud/src/meta-service/meta_service_partition.cpp index d165b8b5e06bb2..5afb24bfe93aea 100644 --- a/cloud/src/meta-service/meta_service_partition.cpp +++ b/cloud/src/meta-service/meta_service_partition.cpp @@ -18,7 +18,10 @@ #include #include +#include + #include "common/logging.h" +#include "common/stats.h" #include "meta-service/keys.h" #include "meta-service/meta_service_helper.h" #include "meta-service/txn_kv_error.h" @@ -77,7 +80,7 @@ static TxnErrorCode check_recycle_key_exist(Transaction* txn, const std::string& void MetaServiceImpl::prepare_index(::google::protobuf::RpcController* controller, const IndexRequest* request, IndexResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(prepare_index); + RPC_PREPROCESS(prepare_index, get, put); instance_id = get_instance_id(resource_mgr_, request->cloud_unique_id()); if (instance_id.empty()) { code = MetaServiceCode::INVALID_ARGUMENT; @@ -94,7 +97,6 @@ void MetaServiceImpl::prepare_index(::google::protobuf::RpcController* controlle return; } - std::unique_ptr txn; TxnErrorCode err = txn_kv_->create_txn(&txn); if (err != TxnErrorCode::TXN_OK) { code = cast_as(err); @@ -164,7 +166,7 @@ void MetaServiceImpl::prepare_index(::google::protobuf::RpcController* controlle void MetaServiceImpl::commit_index(::google::protobuf::RpcController* controller, const IndexRequest* request, IndexResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(commit_index); + RPC_PREPROCESS(commit_index, get, put, del); instance_id = get_instance_id(resource_mgr_, request->cloud_unique_id()); if (instance_id.empty()) { code = MetaServiceCode::INVALID_ARGUMENT; @@ -179,7 +181,6 @@ void MetaServiceImpl::commit_index(::google::protobuf::RpcController* controller return; } - std::unique_ptr txn; TxnErrorCode err = txn_kv_->create_txn(&txn); if (err != TxnErrorCode::TXN_OK) { code = cast_as(err); @@ -248,7 +249,7 @@ void MetaServiceImpl::commit_index(::google::protobuf::RpcController* controller void MetaServiceImpl::drop_index(::google::protobuf::RpcController* controller, const IndexRequest* request, IndexResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(drop_index); + RPC_PREPROCESS(drop_index, get, put); instance_id = get_instance_id(resource_mgr_, request->cloud_unique_id()); if (instance_id.empty()) { code = MetaServiceCode::INVALID_ARGUMENT; @@ -263,7 +264,6 @@ void MetaServiceImpl::drop_index(::google::protobuf::RpcController* controller, return; } - std::unique_ptr txn; TxnErrorCode err = txn_kv_->create_txn(&txn); if (err != TxnErrorCode::TXN_OK) { code = cast_as(err); @@ -350,7 +350,7 @@ void MetaServiceImpl::prepare_partition(::google::protobuf::RpcController* contr const PartitionRequest* request, PartitionResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(prepare_partition); + RPC_PREPROCESS(prepare_partition, get, put); instance_id = get_instance_id(resource_mgr_, request->cloud_unique_id()); if (instance_id.empty()) { code = MetaServiceCode::INVALID_ARGUMENT; @@ -368,7 +368,6 @@ void MetaServiceImpl::prepare_partition(::google::protobuf::RpcController* contr return; } - std::unique_ptr txn; TxnErrorCode err = txn_kv_->create_txn(&txn); if (err != TxnErrorCode::TXN_OK) { code = cast_as(err); @@ -440,7 +439,7 @@ void MetaServiceImpl::prepare_partition(::google::protobuf::RpcController* contr void MetaServiceImpl::commit_partition(::google::protobuf::RpcController* controller, const PartitionRequest* request, PartitionResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(commit_partition); + RPC_PREPROCESS(commit_partition, get, put, del); instance_id = get_instance_id(resource_mgr_, request->cloud_unique_id()); if (instance_id.empty()) { code = MetaServiceCode::INVALID_ARGUMENT; @@ -455,7 +454,6 @@ void MetaServiceImpl::commit_partition(::google::protobuf::RpcController* contro return; } - std::unique_ptr txn; TxnErrorCode err = txn_kv_->create_txn(&txn); if (err != TxnErrorCode::TXN_OK) { code = cast_as(err); @@ -528,7 +526,7 @@ void MetaServiceImpl::commit_partition(::google::protobuf::RpcController* contro void MetaServiceImpl::drop_partition(::google::protobuf::RpcController* controller, const PartitionRequest* request, PartitionResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(drop_partition); + RPC_PREPROCESS(drop_partition, get, put); instance_id = get_instance_id(resource_mgr_, request->cloud_unique_id()); if (instance_id.empty()) { code = MetaServiceCode::INVALID_ARGUMENT; @@ -544,7 +542,6 @@ void MetaServiceImpl::drop_partition(::google::protobuf::RpcController* controll return; } - std::unique_ptr txn; TxnErrorCode err = txn_kv_->create_txn(&txn); if (err != TxnErrorCode::TXN_OK) { code = cast_as(err); @@ -623,7 +620,7 @@ void MetaServiceImpl::drop_partition(::google::protobuf::RpcController* controll void check_create_table(std::string instance_id, std::shared_ptr txn_kv, const CheckKVRequest* request, CheckKVResponse* response, - MetaServiceCode* code, std::string* msg, + MetaServiceCode* code, std::string* msg, KVStats& stats, check_create_table_type get_check_info) { StopWatch watch; std::unique_ptr txn; @@ -633,6 +630,10 @@ void check_create_table(std::string instance_id, std::shared_ptr txn_kv, *msg = "failed to create txn"; return; } + DORIS_CLOUD_DEFER { + if (txn == nullptr) return; + stats.get_counter += txn->num_get_keys(); + }; auto& [keys, hint, key_func] = get_check_info(request); if (keys.empty()) { *code = MetaServiceCode::INVALID_ARGUMENT; @@ -652,6 +653,7 @@ void check_create_table(std::string instance_id, std::shared_ptr txn_kv, *msg = "prepare and commit rpc not match, recycle key remained"; return; } else if (err == TxnErrorCode::TXN_TOO_OLD) { + stats.get_counter += txn->num_get_keys(); // separate it to several txn for rubustness txn.reset(); TxnErrorCode err = txn_kv->create_txn(&txn); @@ -677,7 +679,7 @@ void check_create_table(std::string instance_id, std::shared_ptr txn_kv, void MetaServiceImpl::check_kv(::google::protobuf::RpcController* controller, const CheckKVRequest* request, CheckKVResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(check_kv); + RPC_PREPROCESS(check_kv, get); instance_id = get_instance_id(resource_mgr_, request->cloud_unique_id()); if (instance_id.empty()) { code = MetaServiceCode::INVALID_ARGUMENT; @@ -697,7 +699,7 @@ void MetaServiceImpl::check_kv(::google::protobuf::RpcController* controller, RPC_RATE_LIMIT(check_kv); switch (request->op()) { case CheckKVRequest::CREATE_INDEX_AFTER_FE_COMMIT: { - check_create_table(instance_id, txn_kv_, request, response, &code, &msg, + check_create_table(instance_id, txn_kv_, request, response, &code, &msg, stats, [](const CheckKVRequest* request) { return std::make_tuple( request->check_keys().index_ids(), "index", @@ -709,7 +711,7 @@ void MetaServiceImpl::check_kv(::google::protobuf::RpcController* controller, } case CheckKVRequest::CREATE_PARTITION_AFTER_FE_COMMIT: { check_create_table( - instance_id, txn_kv_, request, response, &code, &msg, + instance_id, txn_kv_, request, response, &code, &msg, stats, [](const CheckKVRequest* request) { return std::make_tuple( request->check_keys().partition_ids(), "partition", diff --git a/cloud/src/meta-service/meta_service_resource.cpp b/cloud/src/meta-service/meta_service_resource.cpp index 44389a41e2b91f..642cc220ce7e4a 100644 --- a/cloud/src/meta-service/meta_service_resource.cpp +++ b/cloud/src/meta-service/meta_service_resource.cpp @@ -28,9 +28,12 @@ #include #include +#include "common/bvars.h" +#include "common/config.h" #include "common/encryption_util.h" #include "common/logging.h" #include "common/network_util.h" +#include "common/stats.h" #include "common/string_util.h" #include "cpp/sync_point.h" #include "meta-service/keys.h" @@ -195,7 +198,7 @@ void MetaServiceImpl::get_obj_store_info(google::protobuf::RpcController* contro const GetObjStoreInfoRequest* request, GetObjStoreInfoResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(get_obj_store_info); + RPC_PREPROCESS(get_obj_store_info, get); TEST_SYNC_POINT_CALLBACK("obj-store-info_sk_response", &response); TEST_SYNC_POINT_RETURN_WITH_VOID("obj-store-info_sk_response_return"); // Prepare data @@ -219,7 +222,6 @@ void MetaServiceImpl::get_obj_store_info(google::protobuf::RpcController* contro std::string val; instance_key(key_info, &key); - std::unique_ptr txn; TxnErrorCode err = txn_kv_->create_txn(&txn); if (err != TxnErrorCode::TXN_OK) { code = cast_as(err); @@ -929,7 +931,7 @@ void MetaServiceImpl::alter_storage_vault(google::protobuf::RpcController* contr bool use_path_style; EncryptionInfoPB encryption_info; AkSkPair cipher_ak_sk_pair; - RPC_PREPROCESS(alter_storage_vault); + RPC_PREPROCESS(alter_storage_vault, get, put, del); switch (request->op()) { case AlterObjStoreInfoRequest::ADD_S3_VAULT: case AlterObjStoreInfoRequest::DROP_S3_VAULT: { @@ -1008,7 +1010,6 @@ void MetaServiceImpl::alter_storage_vault(google::protobuf::RpcController* contr std::string val; instance_key(key_info, &key); - std::unique_ptr txn; TxnErrorCode err = txn_kv_->create_txn(&txn); if (err != TxnErrorCode::TXN_OK) { code = cast_as(err); @@ -1237,7 +1238,7 @@ void MetaServiceImpl::alter_obj_store_info(google::protobuf::RpcController* cont bool use_path_style; EncryptionInfoPB encryption_info; AkSkPair cipher_ak_sk_pair; - RPC_PREPROCESS(alter_obj_store_info); + RPC_PREPROCESS(alter_obj_store_info, get, put); switch (request->op()) { case AlterObjStoreInfoRequest::ADD_OBJ_INFO: case AlterObjStoreInfoRequest::LEGACY_UPDATE_AK_SK: @@ -1287,7 +1288,6 @@ void MetaServiceImpl::alter_obj_store_info(google::protobuf::RpcController* cont std::string val; instance_key(key_info, &key); - std::unique_ptr txn; TxnErrorCode err = txn_kv_->create_txn(&txn); if (err != TxnErrorCode::TXN_OK) { code = cast_as(err); @@ -1464,7 +1464,7 @@ void MetaServiceImpl::alter_obj_store_info(google::protobuf::RpcController* cont void MetaServiceImpl::update_ak_sk(google::protobuf::RpcController* controller, const UpdateAkSkRequest* request, UpdateAkSkResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(update_ak_sk); + RPC_PREPROCESS(update_ak_sk, get, put); instance_id = request->has_instance_id() ? request->instance_id() : ""; if (instance_id.empty()) { msg = "instance id not set"; @@ -1483,7 +1483,6 @@ void MetaServiceImpl::update_ak_sk(google::protobuf::RpcController* controller, std::string val; instance_key(key_info, &key); - std::unique_ptr txn; TxnErrorCode err = txn_kv_->create_txn(&txn); if (err != TxnErrorCode::TXN_OK) { code = cast_as(err); @@ -1662,7 +1661,7 @@ void MetaServiceImpl::create_instance(google::protobuf::RpcController* controlle const CreateInstanceRequest* request, CreateInstanceResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(create_instance); + RPC_PREPROCESS(create_instance, get, put); if (request->has_ram_user()) { auto& ram_user = request->ram_user(); std::string ram_user_id = ram_user.has_user_id() ? ram_user.user_id() : ""; @@ -1714,7 +1713,6 @@ void MetaServiceImpl::create_instance(google::protobuf::RpcController* controlle return; } - std::unique_ptr txn; TxnErrorCode err = txn_kv_->create_txn(&txn); if (err != TxnErrorCode::TXN_OK) { code = cast_as(err); @@ -1967,7 +1965,7 @@ void MetaServiceImpl::alter_instance(google::protobuf::RpcController* controller if (request->op() == AlterInstanceRequest::REFRESH) return; auto f = new std::function([instance_id = request->instance_id(), txn_kv = txn_kv_] { - notify_refresh_instance(txn_kv, instance_id); + notify_refresh_instance(txn_kv, instance_id, nullptr); }); bthread_t bid; if (bthread_start_background(&bid, nullptr, run_bthread_work, f) != 0) { @@ -1979,7 +1977,7 @@ void MetaServiceImpl::alter_instance(google::protobuf::RpcController* controller void MetaServiceImpl::get_instance(google::protobuf::RpcController* controller, const GetInstanceRequest* request, GetInstanceResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(get_instance); + RPC_PREPROCESS(get_instance, get); std::string cloud_unique_id = request->has_cloud_unique_id() ? request->cloud_unique_id() : ""; if (cloud_unique_id.empty()) { code = MetaServiceCode::INVALID_ARGUMENT; @@ -1999,7 +1997,6 @@ void MetaServiceImpl::get_instance(google::protobuf::RpcController* controller, std::string val; instance_key(key_info, &key); - std::unique_ptr txn; TxnErrorCode err = txn_kv_->create_txn(&txn); if (err != TxnErrorCode::TXN_OK) { code = cast_as(err); @@ -2093,7 +2090,7 @@ void MetaServiceImpl::alter_cluster(google::protobuf::RpcController* controller, const AlterClusterRequest* request, AlterClusterResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(alter_cluster); + RPC_PREPROCESS(alter_cluster, get); std::string cloud_unique_id = request->has_cloud_unique_id() ? request->cloud_unique_id() : ""; instance_id = request->has_instance_id() ? request->instance_id() : ""; if (!cloud_unique_id.empty() && instance_id.empty()) { @@ -2444,7 +2441,12 @@ void MetaServiceImpl::alter_cluster(google::protobuf::RpcController* controller, if (code != MetaServiceCode::OK) return; auto f = new std::function([instance_id = request->instance_id(), txn_kv = txn_kv_] { - notify_refresh_instance(txn_kv, instance_id); + // the func run with a thread, so if use macro proved stats, maybe cause stack-use-after-return error + KVStats stats; + notify_refresh_instance(txn_kv, instance_id, &stats); + if (config::use_detailed_metrics && !instance_id.empty()) { + g_bvar_rpc_kv_alter_cluster_get_counter.put({instance_id}, stats.get_counter); + } }); bthread_t bid; if (bthread_start_background(&bid, nullptr, run_bthread_work, f) != 0) { @@ -2456,7 +2458,7 @@ void MetaServiceImpl::alter_cluster(google::protobuf::RpcController* controller, void MetaServiceImpl::get_cluster(google::protobuf::RpcController* controller, const GetClusterRequest* request, GetClusterResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(get_cluster); + RPC_PREPROCESS(get_cluster, get, put); std::string cloud_unique_id = request->has_cloud_unique_id() ? request->cloud_unique_id() : ""; std::string cluster_id = request->has_cluster_id() ? request->cluster_id() : ""; std::string cluster_name = request->has_cluster_name() ? request->cluster_name() : ""; @@ -2510,7 +2512,6 @@ void MetaServiceImpl::get_cluster(google::protobuf::RpcController* controller, std::string val; instance_key(key_info, &key); - std::unique_ptr txn; TxnErrorCode err = txn_kv_->create_txn(&txn); if (err != TxnErrorCode::TXN_OK) { code = cast_as(err); @@ -2601,7 +2602,7 @@ void MetaServiceImpl::get_cluster(google::protobuf::RpcController* controller, void MetaServiceImpl::create_stage(::google::protobuf::RpcController* controller, const CreateStageRequest* request, CreateStageResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(create_stage); + RPC_PREPROCESS(create_stage, get, put); std::string cloud_unique_id = request->has_cloud_unique_id() ? request->cloud_unique_id() : ""; if (cloud_unique_id.empty()) { code = MetaServiceCode::INVALID_ARGUMENT; @@ -2658,7 +2659,6 @@ void MetaServiceImpl::create_stage(::google::protobuf::RpcController* controller std::string val; instance_key(key_info, &key); - std::unique_ptr txn; TxnErrorCode err = txn_kv_->create_txn(&txn); if (err != TxnErrorCode::TXN_OK) { code = cast_as(err); @@ -2788,7 +2788,7 @@ void MetaServiceImpl::create_stage(::google::protobuf::RpcController* controller void MetaServiceImpl::get_stage(google::protobuf::RpcController* controller, const GetStageRequest* request, GetStageResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(get_stage); + RPC_PREPROCESS(get_stage, get); TEST_SYNC_POINT_CALLBACK("stage_sk_response", &response); TEST_SYNC_POINT_RETURN_WITH_VOID("stage_sk_response_return"); std::string cloud_unique_id = request->has_cloud_unique_id() ? request->cloud_unique_id() : ""; @@ -2818,7 +2818,6 @@ void MetaServiceImpl::get_stage(google::protobuf::RpcController* controller, std::string val; instance_key(key_info, &key); - std::unique_ptr txn; TxnErrorCode err = txn_kv_->create_txn(&txn); if (err != TxnErrorCode::TXN_OK) { code = cast_as(err); @@ -3176,7 +3175,7 @@ void MetaServiceImpl::drop_stage(google::protobuf::RpcController* controller, void MetaServiceImpl::get_iam(google::protobuf::RpcController* controller, const GetIamRequest* request, GetIamResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(get_iam); + RPC_PREPROCESS(get_iam, get); std::string cloud_unique_id = request->has_cloud_unique_id() ? request->cloud_unique_id() : ""; if (cloud_unique_id.empty()) { code = MetaServiceCode::INVALID_ARGUMENT; @@ -3198,7 +3197,6 @@ void MetaServiceImpl::get_iam(google::protobuf::RpcController* controller, std::string val; instance_key(key_info, &key); - std::unique_ptr txn; TxnErrorCode err = txn_kv_->create_txn(&txn); if (err != TxnErrorCode::TXN_OK) { code = cast_as(err); @@ -3273,7 +3271,7 @@ void MetaServiceImpl::get_iam(google::protobuf::RpcController* controller, void MetaServiceImpl::alter_iam(google::protobuf::RpcController* controller, const AlterIamRequest* request, AlterIamResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(alter_iam); + RPC_PREPROCESS(alter_iam, get, put); std::string arn_id = request->has_account_id() ? request->account_id() : ""; std::string arn_ak = request->has_ak() ? request->ak() : ""; std::string arn_sk = request->has_sk() ? request->sk() : ""; @@ -3282,12 +3280,13 @@ void MetaServiceImpl::alter_iam(google::protobuf::RpcController* controller, msg = "invalid argument"; return; } - RPC_RATE_LIMIT(alter_iam) + // for metric, give it a common instance id + instance_id = "alter_iam_instance"; + std::string key = system_meta_service_arn_info_key(); std::string val; - std::unique_ptr txn; TxnErrorCode err = txn_kv_->create_txn(&txn); if (err != TxnErrorCode::TXN_OK) { code = cast_as(err); @@ -3363,7 +3362,7 @@ void MetaServiceImpl::alter_ram_user(google::protobuf::RpcController* controller const AlterRamUserRequest* request, AlterRamUserResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(alter_ram_user); + RPC_PREPROCESS(alter_ram_user, get, put); instance_id = request->has_instance_id() ? request->instance_id() : ""; if (instance_id.empty()) { code = MetaServiceCode::INVALID_ARGUMENT; @@ -3383,7 +3382,6 @@ void MetaServiceImpl::alter_ram_user(google::protobuf::RpcController* controller std::string val; instance_key(key_info, &key); - std::unique_ptr txn; TxnErrorCode err = txn_kv_->create_txn(&txn); if (err != TxnErrorCode::TXN_OK) { code = cast_as(err); @@ -3446,7 +3444,7 @@ void MetaServiceImpl::alter_ram_user(google::protobuf::RpcController* controller void MetaServiceImpl::begin_copy(google::protobuf::RpcController* controller, const BeginCopyRequest* request, BeginCopyResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(begin_copy); + RPC_PREPROCESS(begin_copy, get, put); std::string cloud_unique_id = request->has_cloud_unique_id() ? request->cloud_unique_id() : ""; if (cloud_unique_id.empty()) { code = MetaServiceCode::INVALID_ARGUMENT; @@ -3462,7 +3460,6 @@ void MetaServiceImpl::begin_copy(google::protobuf::RpcController* controller, return; } RPC_RATE_LIMIT(begin_copy) - std::unique_ptr txn; TxnErrorCode err = txn_kv_->create_txn(&txn); if (err != TxnErrorCode::TXN_OK) { code = cast_as(err); @@ -3563,7 +3560,7 @@ void MetaServiceImpl::begin_copy(google::protobuf::RpcController* controller, void MetaServiceImpl::finish_copy(google::protobuf::RpcController* controller, const FinishCopyRequest* request, FinishCopyResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(finish_copy); + RPC_PREPROCESS(finish_copy, get, put, del); std::string cloud_unique_id = request->has_cloud_unique_id() ? request->cloud_unique_id() : ""; if (cloud_unique_id.empty()) { code = MetaServiceCode::INVALID_ARGUMENT; @@ -3580,7 +3577,6 @@ void MetaServiceImpl::finish_copy(google::protobuf::RpcController* controller, } RPC_RATE_LIMIT(finish_copy) - std::unique_ptr txn; TxnErrorCode err = txn_kv_->create_txn(&txn); if (err != TxnErrorCode::TXN_OK) { code = cast_as(err); @@ -3668,7 +3664,7 @@ void MetaServiceImpl::finish_copy(google::protobuf::RpcController* controller, void MetaServiceImpl::get_copy_job(google::protobuf::RpcController* controller, const GetCopyJobRequest* request, GetCopyJobResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(get_copy_job); + RPC_PREPROCESS(get_copy_job, get); std::string cloud_unique_id = request->has_cloud_unique_id() ? request->cloud_unique_id() : ""; if (cloud_unique_id.empty()) { code = MetaServiceCode::INVALID_ARGUMENT; @@ -3684,7 +3680,6 @@ void MetaServiceImpl::get_copy_job(google::protobuf::RpcController* controller, return; } - std::unique_ptr txn; TxnErrorCode err = txn_kv_->create_txn(&txn); if (err != TxnErrorCode::TXN_OK) { code = cast_as(err); @@ -3720,7 +3715,7 @@ void MetaServiceImpl::get_copy_files(google::protobuf::RpcController* controller const GetCopyFilesRequest* request, GetCopyFilesResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(get_copy_files); + RPC_PREPROCESS(get_copy_files, get); std::string cloud_unique_id = request->has_cloud_unique_id() ? request->cloud_unique_id() : ""; if (cloud_unique_id.empty()) { code = MetaServiceCode::INVALID_ARGUMENT; @@ -3737,7 +3732,6 @@ void MetaServiceImpl::get_copy_files(google::protobuf::RpcController* controller } RPC_RATE_LIMIT(get_copy_files) - std::unique_ptr txn; TxnErrorCode err = txn_kv_->create_txn(&txn); if (err != TxnErrorCode::TXN_OK) { code = cast_as(err); @@ -3784,7 +3778,7 @@ void MetaServiceImpl::filter_copy_files(google::protobuf::RpcController* control const FilterCopyFilesRequest* request, FilterCopyFilesResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(filter_copy_files); + RPC_PREPROCESS(filter_copy_files, get); std::string cloud_unique_id = request->has_cloud_unique_id() ? request->cloud_unique_id() : ""; if (cloud_unique_id.empty()) { code = MetaServiceCode::INVALID_ARGUMENT; @@ -3801,7 +3795,6 @@ void MetaServiceImpl::filter_copy_files(google::protobuf::RpcController* control } RPC_RATE_LIMIT(filter_copy_files) - std::unique_ptr txn; TxnErrorCode err = txn_kv_->create_txn(&txn); if (err != TxnErrorCode::TXN_OK) { code = cast_as(err); @@ -3836,7 +3829,7 @@ void MetaServiceImpl::get_cluster_status(google::protobuf::RpcController* contro const GetClusterStatusRequest* request, GetClusterStatusResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(get_cluster_status); + RPC_PREPROCESS(get_cluster_status, get); if (request->instance_ids().empty() && request->cloud_unique_ids().empty()) { code = MetaServiceCode::INVALID_ARGUMENT; msg = "cloud_unique_ids or instance_ids must be given, instance_ids.size: " + @@ -3886,6 +3879,12 @@ void MetaServiceImpl::get_cluster_status(google::protobuf::RpcController* contro LOG(WARNING) << "failed to create txn err=" << err; return; } + DORIS_CLOUD_DEFER { + if (config::use_detailed_metrics && txn != nullptr) { + g_bvar_rpc_kv_get_cluster_status_get_counter.put({instance_id}, + txn->num_get_keys()); + } + }; err = txn->get(key, &val); LOG(INFO) << "get instance_key=" << hex(key); @@ -3893,7 +3892,6 @@ void MetaServiceImpl::get_cluster_status(google::protobuf::RpcController* contro LOG(WARNING) << "failed to get instance, instance_id=" << instance_id << " err=" << err; return; } - InstanceInfoPB instance; if (!instance.ParseFromString(val)) { LOG(WARNING) << "failed to parse InstanceInfoPB"; @@ -3930,7 +3928,8 @@ void MetaServiceImpl::get_cluster_status(google::protobuf::RpcController* contro msg = proto_to_json(*response); } -void notify_refresh_instance(std::shared_ptr txn_kv, const std::string& instance_id) { +void notify_refresh_instance(std::shared_ptr txn_kv, const std::string& instance_id, + KVStats* stats) { LOG(INFO) << "begin notify_refresh_instance"; std::unique_ptr txn; TxnErrorCode err = txn_kv->create_txn(&txn); @@ -3946,6 +3945,9 @@ void notify_refresh_instance(std::shared_ptr txn_kv, const std::string& i << " err=" << err; return; } + if (config::use_detailed_metrics && stats) { + stats->get_counter++; + } std::string self_endpoint = config::hostname.empty() ? get_local_ip(config::priority_networks) : config::hostname; self_endpoint = fmt::format("{}:{}", self_endpoint, config::brpc_listen_port); diff --git a/cloud/src/meta-service/meta_service_txn.cpp b/cloud/src/meta-service/meta_service_txn.cpp index 42da823bef0ded..a417ec84e4e2f5 100644 --- a/cloud/src/meta-service/meta_service_txn.cpp +++ b/cloud/src/meta-service/meta_service_txn.cpp @@ -23,6 +23,7 @@ #include "common/config.h" #include "common/logging.h" +#include "common/stats.h" #include "cpp/sync_point.h" #include "meta-service/doris_txn.h" #include "meta-service/keys.h" @@ -80,7 +81,7 @@ static void calc_table_stats(std::unordered_map& tablet_ void MetaServiceImpl::begin_txn(::google::protobuf::RpcController* controller, const BeginTxnRequest* request, BeginTxnResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(begin_txn); + RPC_PREPROCESS(begin_txn, get, put); if (!request->has_txn_info()) { code = MetaServiceCode::INVALID_ARGUMENT; msg = "invalid argument, missing txn info"; @@ -110,7 +111,6 @@ void MetaServiceImpl::begin_txn(::google::protobuf::RpcController* controller, RPC_RATE_LIMIT(begin_txn) //1. Generate version stamp for txn id - std::unique_ptr txn; TxnErrorCode err = txn_kv_->create_txn(&txn); if (err != TxnErrorCode::TXN_OK) { code = cast_as(err); @@ -150,6 +150,9 @@ void MetaServiceImpl::begin_txn(::google::protobuf::RpcController* controller, msg = ss.str(); return; } + // get count before txn reset, if not we will lose these count + stats.get_counter += txn->num_get_keys(); + stats.put_counter += txn->num_put_keys(); //2. Get txn id from version stamp txn.reset(); @@ -358,7 +361,7 @@ void MetaServiceImpl::precommit_txn(::google::protobuf::RpcController* controlle const PrecommitTxnRequest* request, PrecommitTxnResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(precommit_txn); + RPC_PREPROCESS(precommit_txn, get, put); int64_t txn_id = request->has_txn_id() ? request->txn_id() : -1; int64_t db_id = request->has_db_id() ? request->db_id() : -1; if ((txn_id < 0 && db_id < 0)) { @@ -379,7 +382,6 @@ void MetaServiceImpl::precommit_txn(::google::protobuf::RpcController* controlle return; } RPC_RATE_LIMIT(precommit_txn); - std::unique_ptr txn; TxnErrorCode err = txn_kv_->create_txn(&txn); if (err != TxnErrorCode::TXN_OK) { code = cast_as(err); @@ -599,7 +601,7 @@ void MetaServiceImpl::get_rl_task_commit_attach(::google::protobuf::RpcControlle const GetRLTaskCommitAttachRequest* request, GetRLTaskCommitAttachResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(get_rl_task_commit_attach); + RPC_PREPROCESS(get_rl_task_commit_attach, get); instance_id = get_instance_id(resource_mgr_, request->cloud_unique_id()); if (instance_id.empty()) { code = MetaServiceCode::INVALID_ARGUMENT; @@ -609,7 +611,6 @@ void MetaServiceImpl::get_rl_task_commit_attach(::google::protobuf::RpcControlle } RPC_RATE_LIMIT(get_rl_task_commit_attach) - std::unique_ptr txn; TxnErrorCode err = txn_kv_->create_txn(&txn); if (err != TxnErrorCode::TXN_OK) { code = cast_as(err); @@ -668,7 +669,7 @@ void MetaServiceImpl::reset_rl_progress(::google::protobuf::RpcController* contr const ResetRLProgressRequest* request, ResetRLProgressResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(reset_rl_progress); + RPC_PREPROCESS(reset_rl_progress, get, put, del); instance_id = get_instance_id(resource_mgr_, request->cloud_unique_id()); if (instance_id.empty()) { code = MetaServiceCode::INVALID_ARGUMENT; @@ -678,7 +679,6 @@ void MetaServiceImpl::reset_rl_progress(::google::protobuf::RpcController* contr } RPC_RATE_LIMIT(reset_rl_progress) - std::unique_ptr txn; TxnErrorCode err = txn_kv_->create_txn(&txn); if (err != TxnErrorCode::TXN_OK) { code = cast_as(err); @@ -769,7 +769,8 @@ void MetaServiceImpl::reset_rl_progress(::google::protobuf::RpcController* contr void scan_tmp_rowset( const std::string& instance_id, int64_t txn_id, std::shared_ptr txn_kv, MetaServiceCode& code, std::string& msg, int64_t* db_id, - std::vector>* tmp_rowsets_meta) { + std::vector>* tmp_rowsets_meta, + KVStats* stats) { // Create a readonly txn for scan tmp rowset std::stringstream ss; std::unique_ptr txn; @@ -781,6 +782,9 @@ void scan_tmp_rowset( LOG(WARNING) << msg; return; } + DORIS_CLOUD_DEFER { + if (stats && txn) stats->get_counter += txn->num_get_keys(); + }; // Get db id with txn id std::string index_val; @@ -937,7 +941,7 @@ void commit_txn_immediately( std::shared_ptr& txn_kv, std::shared_ptr& txn_lazy_committer, MetaServiceCode& code, std::string& msg, const std::string& instance_id, int64_t db_id, std::vector>& tmp_rowsets_meta, - TxnErrorCode& err) { + TxnErrorCode& err, KVStats& stats) { std::stringstream ss; int64_t txn_id = request->txn_id(); do { @@ -952,6 +956,12 @@ void commit_txn_immediately( LOG(WARNING) << msg; return; } + DORIS_CLOUD_DEFER { + if (txn == nullptr) return; + stats.get_counter += txn->num_get_keys(); + stats.put_counter += txn->num_put_keys(); + stats.del_counter += txn->num_del_keys(); + }; // Get txn info with db_id and txn_id std::string info_val; // Will be reused when saving updated txn @@ -1120,6 +1130,7 @@ void commit_txn_immediately( version_values.clear(); if (last_pending_txn_id > 0) { + stats.get_counter += txn->num_get_keys(); txn.reset(); TEST_SYNC_POINT_CALLBACK("commit_txn_immediately::advance_last_pending_txn_id", &last_pending_txn_id); @@ -1557,7 +1568,8 @@ void commit_txn_eventually( const CommitTxnRequest* request, CommitTxnResponse* response, std::shared_ptr& txn_kv, std::shared_ptr& txn_lazy_committer, MetaServiceCode& code, std::string& msg, const std::string& instance_id, int64_t db_id, - const std::vector>& tmp_rowsets_meta) { + const std::vector>& tmp_rowsets_meta, + KVStats& stats) { StopWatch sw; DORIS_CLOUD_DEFER { if (config::use_detailed_metrics && !instance_id.empty()) { @@ -1581,6 +1593,12 @@ void commit_txn_eventually( LOG(WARNING) << msg; return; } + DORIS_CLOUD_DEFER { + if (txn == nullptr) return; + stats.get_counter += txn->num_get_keys(); + stats.put_counter += txn->num_put_keys(); + stats.del_counter += txn->num_del_keys(); + }; // tablet_id -> {table/index/partition}_id std::unordered_map tablet_ids; @@ -1597,6 +1615,7 @@ void commit_txn_eventually( TEST_SYNC_POINT_CALLBACK("commit_txn_eventually::need_repair_tablet_idx", &need_repair_tablet_idx); if (need_repair_tablet_idx) { + stats.get_counter += txn->num_get_keys(); txn.reset(); repair_tablet_index(txn_kv, code, msg, instance_id, db_id, txn_id, tmp_rowsets_meta); if (code != MetaServiceCode::OK) { @@ -1661,6 +1680,7 @@ void commit_txn_eventually( if (last_pending_txn_id > 0) { TEST_SYNC_POINT_CALLBACK("commit_txn_eventually::advance_last_pending_txn_id", &last_pending_txn_id); + stats.get_counter += txn->num_get_keys(); txn.reset(); std::shared_ptr task = txn_lazy_committer->submit(instance_id, last_pending_txn_id); @@ -1979,7 +1999,7 @@ void commit_txn_eventually( */ void commit_txn_with_sub_txn(const CommitTxnRequest* request, CommitTxnResponse* response, std::shared_ptr& txn_kv, MetaServiceCode& code, - std::string& msg, const std::string& instance_id) { + std::string& msg, const std::string& instance_id, KVStats& stats) { std::stringstream ss; int64_t txn_id = request->txn_id(); auto sub_txn_infos = request->sub_txn_infos(); @@ -1993,6 +2013,12 @@ void commit_txn_with_sub_txn(const CommitTxnRequest* request, CommitTxnResponse* LOG(WARNING) << msg; return; } + DORIS_CLOUD_DEFER { + if (txn == nullptr) return; + stats.get_counter += txn->num_get_keys(); + stats.put_counter += txn->num_put_keys(); + stats.del_counter += txn->num_del_keys(); + }; // Get db id with txn id std::string index_val; @@ -2084,7 +2110,7 @@ void commit_txn_with_sub_txn(const CommitTxnRequest* request, CommitTxnResponse* << " tmp_rowsets_meta.size()=" << tmp_rowsets_meta.size(); sub_txn_to_tmp_rowsets_meta.emplace(sub_txn_id, std::move(tmp_rowsets_meta)); } - + stats.get_counter += txn->num_get_keys(); // Create a read/write txn for guarantee consistency txn.reset(); err = txn_kv->create_txn(&txn); @@ -2566,7 +2592,7 @@ static bool force_txn_lazy_commit() { void MetaServiceImpl::commit_txn(::google::protobuf::RpcController* controller, const CommitTxnRequest* request, CommitTxnResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(commit_txn); + RPC_PREPROCESS(commit_txn, get, put, del); if (!request->has_txn_id()) { code = MetaServiceCode::INVALID_ARGUMENT; msg = "invalid argument, missing txn id"; @@ -2585,13 +2611,13 @@ void MetaServiceImpl::commit_txn(::google::protobuf::RpcController* controller, RPC_RATE_LIMIT(commit_txn) if (request->has_is_txn_load() && request->is_txn_load()) { - commit_txn_with_sub_txn(request, response, txn_kv_, code, msg, instance_id); + commit_txn_with_sub_txn(request, response, txn_kv_, code, msg, instance_id, stats); return; } int64_t db_id; std::vector> tmp_rowsets_meta; - scan_tmp_rowset(instance_id, txn_id, txn_kv_, code, msg, &db_id, &tmp_rowsets_meta); + scan_tmp_rowset(instance_id, txn_id, txn_kv_, code, msg, &db_id, &tmp_rowsets_meta, &stats); if (code != MetaServiceCode::OK) { LOG(WARNING) << "scan_tmp_rowset failed, txn_id=" << txn_id << " code=" << code; return; @@ -2610,7 +2636,7 @@ void MetaServiceImpl::commit_txn(::google::protobuf::RpcController* controller, } commit_txn_immediately(request, response, txn_kv_, txn_lazy_committer_, code, msg, - instance_id, db_id, tmp_rowsets_meta, err); + instance_id, db_id, tmp_rowsets_meta, err, stats); if (MetaServiceCode::OK == code) { return; @@ -2640,7 +2666,7 @@ void MetaServiceImpl::commit_txn(::google::protobuf::RpcController* controller, code = MetaServiceCode::OK; msg.clear(); commit_txn_eventually(request, response, txn_kv_, txn_lazy_committer_, code, msg, instance_id, - db_id, tmp_rowsets_meta); + db_id, tmp_rowsets_meta, stats); } static void _abort_txn(const std::string& instance_id, const AbortTxnRequest* request, @@ -2850,7 +2876,7 @@ static void _abort_txn(const std::string& instance_id, const AbortTxnRequest* re void MetaServiceImpl::abort_txn(::google::protobuf::RpcController* controller, const AbortTxnRequest* request, AbortTxnResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(abort_txn); + RPC_PREPROCESS(abort_txn, get, put, del); // Get txn id int64_t txn_id = request->has_txn_id() ? request->txn_id() : -1; std::string label = request->has_label() ? request->label() : ""; @@ -2875,7 +2901,6 @@ void MetaServiceImpl::abort_txn(::google::protobuf::RpcController* controller, } RPC_RATE_LIMIT(abort_txn); - std::unique_ptr txn; TxnErrorCode err = txn_kv_->create_txn(&txn); if (err != TxnErrorCode::TXN_OK) { code = cast_as(err); @@ -2900,7 +2925,7 @@ void MetaServiceImpl::abort_txn(::google::protobuf::RpcController* controller, void MetaServiceImpl::get_txn(::google::protobuf::RpcController* controller, const GetTxnRequest* request, GetTxnResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(get_txn); + RPC_PREPROCESS(get_txn, get); int64_t txn_id = request->has_txn_id() ? request->txn_id() : -1; int64_t db_id = request->has_db_id() ? request->db_id() : -1; std::string label = request->has_label() ? request->label() : ""; @@ -2922,7 +2947,6 @@ void MetaServiceImpl::get_txn(::google::protobuf::RpcController* controller, } RPC_RATE_LIMIT(get_txn) - std::unique_ptr txn; TxnErrorCode err = txn_kv_->create_txn(&txn); if (err != TxnErrorCode::TXN_OK) { code = cast_as(err); @@ -3029,7 +3053,7 @@ void MetaServiceImpl::get_current_max_txn_id(::google::protobuf::RpcController* const GetCurrentMaxTxnRequest* request, GetCurrentMaxTxnResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(get_current_max_txn_id); + RPC_PREPROCESS(get_current_max_txn_id, get); // TODO: For auth instance_id = get_instance_id(resource_mgr_, request->cloud_unique_id()); if (instance_id.empty()) { @@ -3039,7 +3063,7 @@ void MetaServiceImpl::get_current_max_txn_id(::google::protobuf::RpcController* return; } RPC_RATE_LIMIT(get_current_max_txn_id) - std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); if (err != TxnErrorCode::TXN_OK) { msg = "failed to create txn"; @@ -3086,7 +3110,7 @@ void MetaServiceImpl::begin_sub_txn(::google::protobuf::RpcController* controlle const BeginSubTxnRequest* request, BeginSubTxnResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(begin_sub_txn); + RPC_PREPROCESS(begin_sub_txn, get, put, del); int64_t txn_id = request->has_txn_id() ? request->txn_id() : -1; int64_t sub_txn_num = request->has_sub_txn_num() ? request->sub_txn_num() : -1; int64_t db_id = request->has_db_id() ? request->db_id() : -1; @@ -3115,7 +3139,6 @@ void MetaServiceImpl::begin_sub_txn(::google::protobuf::RpcController* controlle } RPC_RATE_LIMIT(begin_sub_txn) - std::unique_ptr txn; TxnErrorCode err = txn_kv_->create_txn(&txn); if (err != TxnErrorCode::TXN_OK) { code = cast_as(err); @@ -3153,6 +3176,8 @@ void MetaServiceImpl::begin_sub_txn(::google::protobuf::RpcController* controlle msg = ss.str(); return; } + stats.get_counter += txn->num_get_keys(); + stats.put_counter += txn->num_put_keys(); // 2. Get sub txn id from version stamp txn.reset(); @@ -3284,7 +3309,7 @@ void MetaServiceImpl::abort_sub_txn(::google::protobuf::RpcController* controlle const AbortSubTxnRequest* request, AbortSubTxnResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(abort_sub_txn); + RPC_PREPROCESS(abort_sub_txn, get, put); int64_t txn_id = request->has_txn_id() ? request->txn_id() : -1; int64_t sub_txn_id = request->has_sub_txn_id() ? request->sub_txn_id() : -1; int64_t sub_txn_num = request->has_sub_txn_num() ? request->sub_txn_num() : -1; @@ -3313,7 +3338,6 @@ void MetaServiceImpl::abort_sub_txn(::google::protobuf::RpcController* controlle } RPC_RATE_LIMIT(abort_sub_txn) - std::unique_ptr txn; TxnErrorCode err = txn_kv_->create_txn(&txn); if (err != TxnErrorCode::TXN_OK) { code = cast_as(err); @@ -3399,7 +3423,7 @@ void MetaServiceImpl::abort_txn_with_coordinator(::google::protobuf::RpcControll const AbortTxnWithCoordinatorRequest* request, AbortTxnWithCoordinatorResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(abort_txn_with_coordinator); + RPC_PREPROCESS(abort_txn_with_coordinator, get); if (!request->has_id() || !request->has_ip() || !request->has_start_time()) { code = MetaServiceCode::INVALID_ARGUMENT; msg = "invalid coordinate id, coordinate ip or coordinate start time."; @@ -3420,7 +3444,6 @@ void MetaServiceImpl::abort_txn_with_coordinator(::google::protobuf::RpcControll std::string end_info_key = txn_info_key({instance_id, INT64_MAX, INT64_MAX}); LOG(INFO) << "begin_info_key:" << hex(begin_info_key) << " end_info_key:" << hex(end_info_key); - std::unique_ptr txn; TxnErrorCode err = txn_kv_->create_txn(&txn); if (err != TxnErrorCode::TXN_OK) { msg = "failed to create txn"; @@ -3512,7 +3535,7 @@ void MetaServiceImpl::check_txn_conflict(::google::protobuf::RpcController* cont const CheckTxnConflictRequest* request, CheckTxnConflictResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(check_txn_conflict); + RPC_PREPROCESS(check_txn_conflict, get); if (!request->has_db_id() || !request->has_end_txn_id() || (request->table_ids_size() <= 0)) { code = MetaServiceCode::INVALID_ARGUMENT; msg = "invalid db id, end txn id or table_ids."; @@ -3536,7 +3559,6 @@ void MetaServiceImpl::check_txn_conflict(::google::protobuf::RpcController* cont LOG(INFO) << "begin_running_key:" << hex(begin_running_key) << " end_running_key:" << hex(end_running_key); - std::unique_ptr txn; TxnErrorCode err = txn_kv_->create_txn(&txn); if (err != TxnErrorCode::TXN_OK) { msg = "failed to create txn"; @@ -3646,7 +3668,7 @@ void MetaServiceImpl::check_txn_conflict(::google::protobuf::RpcController* cont * @return TxnErrorCode */ TxnErrorCode internal_clean_label(std::shared_ptr txn_kv, const std::string_view instance_id, - int64_t db_id, const std::string_view label_key) { + int64_t db_id, const std::string_view label_key, KVStats& stats) { std::string label_val; TxnLabelPB label_pb; @@ -3662,6 +3684,12 @@ TxnErrorCode internal_clean_label(std::shared_ptr txn_kv, const std::stri << " label_key=" << hex(label_key); return err; } + DORIS_CLOUD_DEFER { + if (txn == nullptr) return; + stats.get_counter += txn->num_get_keys(); + stats.put_counter += txn->num_put_keys(); + stats.del_counter += txn->num_del_keys(); + }; err = txn->get(label_key, &label_val); if (err != TxnErrorCode::TXN_OK && err != TxnErrorCode::TXN_KEY_NOT_FOUND) { @@ -3777,7 +3805,7 @@ void MetaServiceImpl::clean_txn_label(::google::protobuf::RpcController* control const CleanTxnLabelRequest* request, CleanTxnLabelResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(clean_txn_label); + RPC_PREPROCESS(clean_txn_label, get, put, del); if (!request->has_db_id()) { code = MetaServiceCode::INVALID_ARGUMENT; msg = "missing db id"; @@ -3817,6 +3845,10 @@ void MetaServiceImpl::clean_txn_label(::google::protobuf::RpcController* control << " end=" << hex(end_label_key); return; } + DORIS_CLOUD_DEFER { + if (txn == nullptr) return; + stats.get_counter += txn->num_get_keys(); + }; err = txn->get(begin_label_key, end_label_key, &it, snapshot, limit); if (err != TxnErrorCode::TXN_OK) { @@ -3838,7 +3870,7 @@ void MetaServiceImpl::clean_txn_label(::google::protobuf::RpcController* control begin_label_key = k; LOG(INFO) << "iterator has no more kvs. key=" << hex(k); } - err = internal_clean_label(txn_kv_, instance_id, db_id, k); + err = internal_clean_label(txn_kv_, instance_id, db_id, k, stats); if (err != TxnErrorCode::TXN_OK) { code = cast_as(err); msg = fmt::format("failed to clean txn label. err={}", err); @@ -3851,7 +3883,7 @@ void MetaServiceImpl::clean_txn_label(::google::protobuf::RpcController* control } else { const std::string& label = request->labels(0); const std::string label_key = txn_label_key({instance_id, db_id, label}); - TxnErrorCode err = internal_clean_label(txn_kv_, instance_id, db_id, label_key); + TxnErrorCode err = internal_clean_label(txn_kv_, instance_id, db_id, label_key, stats); if (err != TxnErrorCode::TXN_OK) { code = cast_as(err); msg = fmt::format("failed to clean txn label. err={}", err); @@ -3871,7 +3903,7 @@ void MetaServiceImpl::clean_txn_label(::google::protobuf::RpcController* control void MetaServiceImpl::get_txn_id(::google::protobuf::RpcController* controller, const GetTxnIdRequest* request, GetTxnIdResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(get_txn_id); + RPC_PREPROCESS(get_txn_id, get); if (!request->has_db_id()) { code = MetaServiceCode::INVALID_ARGUMENT; msg = "missing db id"; @@ -3893,7 +3925,7 @@ void MetaServiceImpl::get_txn_id(::google::protobuf::RpcController* controller, const int64_t db_id = request->db_id(); std::string label = request->label(); const std::string label_key = txn_label_key({instance_id, db_id, label}); - std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); if (err != TxnErrorCode::TXN_OK) { LOG(WARNING) << "failed to create txn. err=" << err << " db_id=" << db_id diff --git a/cloud/src/meta-service/txn_kv.cpp b/cloud/src/meta-service/txn_kv.cpp index ad9b62bb0c14b7..56e5440c8fe9af 100644 --- a/cloud/src/meta-service/txn_kv.cpp +++ b/cloud/src/meta-service/txn_kv.cpp @@ -399,6 +399,7 @@ TxnErrorCode Transaction::get(std::string_view key, std::string* val, bool snaps const uint8_t* ret; int len; err = fdb_future_get_value(fut, &found, &ret, &len); + num_get_keys_++; if (err) { LOG(WARNING) << __PRETTY_FUNCTION__ @@ -438,6 +439,7 @@ TxnErrorCode Transaction::get(std::string_view begin, std::string_view end, std::unique_ptr ret(new RangeGetIterator(fut)); RETURN_IF_ERROR(ret->init()); + num_get_keys_ += ret->size(); g_bvar_txn_kv_get_count_normalized << ret->size(); *(iter) = std::move(ret); @@ -668,6 +670,7 @@ TxnErrorCode Transaction::batch_get(std::vector>* res futures.clear(); } DCHECK_EQ(res->size(), num_keys); + num_get_keys_ += num_keys; return TxnErrorCode::TXN_OK; } diff --git a/cloud/src/meta-service/txn_kv.h b/cloud/src/meta-service/txn_kv.h index 645982f76164ef..66b01c37752b8c 100644 --- a/cloud/src/meta-service/txn_kv.h +++ b/cloud/src/meta-service/txn_kv.h @@ -20,6 +20,7 @@ #include #include +#include #include #include #include @@ -233,6 +234,11 @@ class Transaction { **/ virtual size_t approximate_bytes() const = 0; + /** + * @brief return the num get keys submitted to this txn. + **/ + virtual size_t num_get_keys() const = 0; + /** * @brief return the num delete keys submitted to this txn. **/ @@ -565,6 +571,8 @@ class Transaction : public cloud::Transaction { size_t approximate_bytes() const override { return approximate_bytes_; } + size_t num_get_keys() const override { return num_get_keys_; } + size_t num_del_keys() const override { return num_del_keys_; } size_t num_put_keys() const override { return num_put_keys_; } @@ -579,6 +587,7 @@ class Transaction : public cloud::Transaction { bool aborted_ = false; FDBTransaction* txn_ = nullptr; + size_t num_get_keys_ {0}; size_t num_del_keys_ {0}; size_t num_put_keys_ {0}; size_t delete_bytes_ {0}; diff --git a/cloud/src/meta-service/txn_lazy_committer.cpp b/cloud/src/meta-service/txn_lazy_committer.cpp index 2260ea1ff23b7b..99aea5fed56fff 100644 --- a/cloud/src/meta-service/txn_lazy_committer.cpp +++ b/cloud/src/meta-service/txn_lazy_committer.cpp @@ -20,6 +20,7 @@ #include #include "common/logging.h" +#include "common/stats.h" #include "common/util.h" #include "cpp/sync_point.h" #include "meta-service/keys.h" @@ -33,7 +34,8 @@ namespace doris::cloud { void scan_tmp_rowset( const std::string& instance_id, int64_t txn_id, std::shared_ptr txn_kv, MetaServiceCode& code, std::string& msg, int64_t* db_id, - std::vector>* tmp_rowsets_meta); + std::vector>* tmp_rowsets_meta, + KVStats* stats); void update_tablet_stats(const StatsTabletKeyInfo& info, const TabletStats& stats, std::unique_ptr& txn, MetaServiceCode& code, @@ -308,7 +310,7 @@ void TxnLazyCommitTask::commit() { int64_t db_id; std::vector> all_tmp_rowset_metas; scan_tmp_rowset(instance_id_, txn_id_, txn_kv_, code_, msg_, &db_id, - &all_tmp_rowset_metas); + &all_tmp_rowset_metas, nullptr); if (code_ != MetaServiceCode::OK) { LOG(WARNING) << "scan_tmp_rowset failed, txn_id=" << txn_id_ << " code=" << code_; break; diff --git a/cloud/test/CMakeLists.txt b/cloud/test/CMakeLists.txt index 8378a33ddfd2dd..49e2178ac8e888 100644 --- a/cloud/test/CMakeLists.txt +++ b/cloud/test/CMakeLists.txt @@ -43,6 +43,8 @@ add_executable(log_test log_test.cpp) add_executable(resource_test resource_test.cpp) +add_executable(rpc_kv_bvar_test rpc_kv_bvar_test.cpp) + add_executable(http_encode_key_test http_encode_key_test.cpp) add_executable(fdb_injection_test fdb_injection_test.cpp) @@ -84,6 +86,8 @@ target_link_libraries(log_test ${TEST_LINK_LIBS}) target_link_libraries(resource_test ${TEST_LINK_LIBS}) +target_link_libraries(rpc_kv_bvar_test ${TEST_LINK_LIBS}) + target_link_libraries(http_encode_key_test ${TEST_LINK_LIBS}) target_link_libraries(s3_accessor_test ${TEST_LINK_LIBS}) diff --git a/cloud/test/meta_server_test.cpp b/cloud/test/meta_server_test.cpp index 8de04af4a8a19d..8cff9223498989 100644 --- a/cloud/test/meta_server_test.cpp +++ b/cloud/test/meta_server_test.cpp @@ -35,6 +35,7 @@ #include "common/config.h" #include "common/defer.h" #include "common/logging.h" +#include "common/stats.h" #include "cpp/sync_point.h" #include "meta-service/keys.h" #include "meta-service/mem_txn_kv.h" @@ -63,7 +64,8 @@ int main(int argc, char** argv) { } namespace doris::cloud { -void notify_refresh_instance(std::shared_ptr txn_kv, const std::string& instance_id); +void notify_refresh_instance(std::shared_ptr txn_kv, const std::string& instance_id, + KVStats* stats); } // namespace doris::cloud TEST(MetaServerTest, FQDNRefreshInstance) { @@ -137,7 +139,7 @@ TEST(MetaServerTest, FQDNRefreshInstance) { // Refresh instance with FQDN endpoint. config::hostname = ""; - notify_refresh_instance(txn_kv, "fqdn_instance_id"); + notify_refresh_instance(txn_kv, "fqdn_instance_id", nullptr); bool refreshed = false; for (size_t i = 0; i < 100; ++i) { diff --git a/cloud/test/rpc_kv_bvar_test.cpp b/cloud/test/rpc_kv_bvar_test.cpp new file mode 100644 index 00000000000000..6f6bf9d0d201ac --- /dev/null +++ b/cloud/test/rpc_kv_bvar_test.cpp @@ -0,0 +1,3141 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//#define private public +#include "common/bvars.h" +#include "meta-service/meta_service.h" +//#undef private + +#include +#include +#include +#include +#include +#include + +#include +#include +#include + +#include "common/config.h" +#include "common/logging.h" +#include "common/util.h" +#include "cpp/sync_point.h" +#include "meta-service/keys.h" +#include "meta-service/mem_txn_kv.h" +#include "meta-service/meta_service_helper.h" +#include "meta-service/txn_kv.h" +#include "meta-service/txn_kv_error.h" +#include "mock_resource_manager.h" +#include "rate-limiter/rate_limiter.h" + +int main(int argc, char** argv) { + const std::string conf_file = "doris_cloud.conf"; + if (!doris::cloud::config::init(conf_file.c_str(), true)) { + std::cerr << "failed to init config file, conf=" << conf_file << std::endl; + return -1; + } + + if (!doris::cloud::init_glog("rpc_kv_bvar_test")) { + std::cerr << "failed to init glog" << std::endl; + return -1; + } + + ::testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} + +namespace doris::cloud { +using Status = MetaServiceResponseStatus; + +void start_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, + StartTabletJobResponse& res, + std::pair input_version = {0, 0}) { + brpc::Controller cntl; + StartTabletJobRequest req; + 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); + long now = time(nullptr); + compaction->set_expiration(now + 12); + compaction->set_lease(now + 3); + 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); + } + meta_service->start_tablet_job(&cntl, &req, &res, nullptr); +}; + +std::unique_ptr get_meta_service() { + int ret = 0; + // MemKv + auto txn_kv = std::dynamic_pointer_cast(std::make_shared()); + if (txn_kv != nullptr) { + ret = txn_kv->init(); + [&] { ASSERT_EQ(ret, 0); }(); + } + [&] { ASSERT_NE(txn_kv.get(), nullptr); }(); + + std::unique_ptr txn; + EXPECT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->remove("\x00", "\xfe"); // This is dangerous if the fdb is not correctly set + EXPECT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + auto rs = std::make_shared(txn_kv); + auto rl = std::make_shared(); + auto meta_service = std::make_unique(txn_kv, rs, rl); + return std::make_unique(std::move(meta_service)); +} + +std::unique_ptr get_fdb_meta_service() { + config::fdb_cluster_file_path = "fdb.cluster"; + static auto txn_kv = std::dynamic_pointer_cast(std::make_shared()); + static std::atomic init {false}; + bool tmp = false; + if (init.compare_exchange_strong(tmp, true)) { + int ret = txn_kv->init(); + [&] { + ASSERT_EQ(ret, 0); + ASSERT_NE(txn_kv.get(), nullptr); + }(); + } + auto rs = std::make_shared(txn_kv); + auto rl = std::make_shared(); + auto meta_service = std::make_unique(txn_kv, rs, rl); + return std::make_unique(std::move(meta_service)); +} + +static void commit_rowset(MetaServiceProxy* meta_service, const doris::RowsetMetaCloudPB& rowset, + CreateRowsetResponse& res) { + brpc::Controller cntl; + auto* arena = res.GetArena(); + auto* req = google::protobuf::Arena::CreateMessage(arena); + req->mutable_rowset_meta()->CopyFrom(rowset); + meta_service->commit_rowset(&cntl, req, &res, nullptr); + if (!arena) { + delete req; + } +} + +static std::string next_rowset_id() { + static int cnt = 0; + return std::to_string(++cnt); +} + +static void fill_schema(doris::TabletSchemaCloudPB* schema, int32_t schema_version) { + schema->set_schema_version(schema_version); + for (int i = 0; i < 10; ++i) { + auto* column = schema->add_column(); + column->set_unique_id(20000 + i); + column->set_type("INT"); + } +} + +static void add_tablet(CreateTabletsRequest& req, int64_t table_id, int64_t index_id, + int64_t partition_id, int64_t tablet_id, const std::string& rowset_id, + int32_t schema_version) { + auto* tablet = req.add_tablet_metas(); + tablet->set_table_id(table_id); + tablet->set_index_id(index_id); + tablet->set_partition_id(partition_id); + tablet->set_tablet_id(tablet_id); + auto* schema = tablet->mutable_schema(); + fill_schema(schema, schema_version); + auto* first_rowset = tablet->add_rs_metas(); + first_rowset->set_rowset_id(0); // required + first_rowset->set_rowset_id_v2(rowset_id); + first_rowset->set_start_version(0); + first_rowset->set_end_version(1); + first_rowset->mutable_tablet_schema()->CopyFrom(*schema); +} + +static void add_tablet(CreateTabletsRequest& req, int64_t table_id, int64_t index_id, + int64_t partition_id, int64_t tablet_id) { + auto* tablet = req.add_tablet_metas(); + tablet->set_table_id(table_id); + tablet->set_index_id(index_id); + tablet->set_partition_id(partition_id); + tablet->set_tablet_id(tablet_id); + auto* schema = tablet->mutable_schema(); + schema->set_schema_version(0); + auto* first_rowset = tablet->add_rs_metas(); + first_rowset->set_rowset_id(0); // required + first_rowset->set_rowset_id_v2(next_rowset_id()); + first_rowset->set_start_version(0); + first_rowset->set_end_version(1); + first_rowset->mutable_tablet_schema()->CopyFrom(*schema); +} + +static void create_tablet_with_db_id(MetaServiceProxy* meta_service, int64_t db_id, + int64_t table_id, int64_t index_id, int64_t partition_id, + int64_t tablet_id) { + brpc::Controller cntl; + CreateTabletsRequest req; + CreateTabletsResponse res; + req.set_db_id(db_id); + add_tablet(req, table_id, index_id, partition_id, tablet_id); + meta_service->create_tablets(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << tablet_id; +} + +void create_tablet(MetaService* meta_service, int64_t table_id, int64_t index_id, + int64_t partition_id, int64_t tablet_id, bool enable_mow, + bool not_ready = false) { + brpc::Controller cntl; + CreateTabletsRequest req; + CreateTabletsResponse res; + auto* tablet = req.add_tablet_metas(); + tablet->set_tablet_state(not_ready ? doris::TabletStatePB::PB_NOTREADY + : doris::TabletStatePB::PB_RUNNING); + tablet->set_table_id(table_id); + tablet->set_index_id(index_id); + tablet->set_partition_id(partition_id); + tablet->set_tablet_id(tablet_id); + tablet->set_enable_unique_key_merge_on_write(enable_mow); + auto* schema = tablet->mutable_schema(); + schema->set_schema_version(0); + auto* first_rowset = tablet->add_rs_metas(); + first_rowset->set_rowset_id(0); // required + first_rowset->set_rowset_id_v2(next_rowset_id()); + first_rowset->set_start_version(0); + first_rowset->set_end_version(1); + first_rowset->mutable_tablet_schema()->CopyFrom(*schema); + meta_service->create_tablets(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << tablet_id; +} + +static void create_tablet(MetaServiceProxy* meta_service, int64_t table_id, int64_t index_id, + int64_t partition_id, int64_t tablet_id, const std::string& rowset_id, + int32_t schema_version) { + brpc::Controller cntl; + CreateTabletsRequest req; + CreateTabletsResponse res; + add_tablet(req, table_id, index_id, partition_id, tablet_id, rowset_id, schema_version); + meta_service->create_tablets(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << tablet_id; +} + +static void create_tablet(MetaServiceProxy* meta_service, int64_t table_id, int64_t index_id, + int64_t partition_id, int64_t tablet_id) { + brpc::Controller cntl; + CreateTabletsRequest req; + CreateTabletsResponse res; + add_tablet(req, table_id, index_id, partition_id, tablet_id); + meta_service->create_tablets(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << tablet_id; +} + +static void begin_txn(MetaServiceProxy* meta_service, int64_t db_id, const std::string& label, + int64_t table_id, int64_t& txn_id) { + brpc::Controller cntl; + BeginTxnRequest req; + BeginTxnResponse res; + 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(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << label; + ASSERT_TRUE(res.has_txn_id()) << label; + txn_id = res.txn_id(); +} + +static doris::RowsetMetaCloudPB create_rowset(int64_t txn_id, int64_t tablet_id, + int partition_id = 10, int64_t version = -1, + int num_rows = 100) { + doris::RowsetMetaCloudPB rowset; + rowset.set_rowset_id(0); // required + rowset.set_rowset_id_v2(next_rowset_id()); + rowset.set_tablet_id(tablet_id); + rowset.set_partition_id(partition_id); + rowset.set_txn_id(txn_id); + if (version > 0) { + rowset.set_start_version(version); + rowset.set_end_version(version); + } + rowset.set_num_segments(1); + rowset.set_num_rows(num_rows); + rowset.set_data_disk_size(num_rows * 100); + rowset.set_index_disk_size(num_rows * 10); + rowset.set_total_disk_size(num_rows * 110); + rowset.mutable_tablet_schema()->set_schema_version(0); + rowset.set_txn_expiration(::time(nullptr)); // Required by DCHECK + return rowset; +} + +static doris::RowsetMetaCloudPB create_rowset(int64_t txn_id, int64_t tablet_id, int index_id, + int partition_id, int64_t version = -1, + int num_rows = 100) { + doris::RowsetMetaCloudPB rowset; + rowset.set_rowset_id(0); // required + rowset.set_rowset_id_v2(next_rowset_id()); + rowset.set_tablet_id(tablet_id); + rowset.set_partition_id(partition_id); + rowset.set_index_id(index_id); + rowset.set_txn_id(txn_id); + if (version > 0) { + rowset.set_start_version(version); + rowset.set_end_version(version); + } + rowset.set_num_segments(0); + rowset.set_num_rows(0); + rowset.set_data_disk_size(0); + rowset.mutable_tablet_schema()->set_schema_version(0); + rowset.set_txn_expiration(::time(nullptr)); // Required by DCHECK + return rowset; +} + +static void check_get_tablet(MetaServiceProxy* meta_service, int64_t tablet_id, + int32_t schema_version) { + brpc::Controller cntl; + GetTabletRequest req; + GetTabletResponse res; + req.set_tablet_id(tablet_id); + meta_service->get_tablet(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << tablet_id; + ASSERT_TRUE(res.has_tablet_meta()) << tablet_id; + EXPECT_TRUE(res.tablet_meta().has_schema()) << tablet_id; + EXPECT_EQ(res.tablet_meta().schema_version(), schema_version) << tablet_id; +}; + +static void create_and_commit_rowset(MetaServiceProxy* meta_service, int64_t table_id, + int64_t index_id, int64_t partition_id, int64_t tablet_id, + int64_t txn_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; + 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; + GetTabletStatsRequest req; + auto* idx = req.add_tablet_idx(); + idx->set_table_id(table_id); + idx->set_index_id(index_id); + idx->set_partition_id(partition_id); + idx->set_tablet_id(tablet_id); + meta_service->get_tablet_stats(&cntl, &req, &res, nullptr); +} + +static void commit_txn(MetaServiceProxy* meta_service, int64_t db_id, int64_t txn_id, + const std::string& label) { + brpc::Controller cntl; + CommitTxnRequest req; + CommitTxnResponse res; + req.set_db_id(db_id); + req.set_txn_id(txn_id); + meta_service->commit_txn(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << label; +} + +static void begin_txn_and_commit_rowset(MetaServiceProxy* meta_service, const std::string& label, + int64_t db_id, int64_t table_id, int64_t partition_id, + int64_t tablet_id, int64_t* txn_id) { + begin_txn(meta_service, db_id, label, table_id, *txn_id); + CreateRowsetResponse res; + auto rowset = create_rowset(*txn_id, tablet_id, partition_id); + prepare_rowset(meta_service, rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + res.Clear(); + commit_rowset(meta_service, rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); +} + +static void insert_rowset(MetaServiceProxy* meta_service, int64_t db_id, const std::string& label, + int64_t table_id, int64_t partition_id, int64_t tablet_id) { + int64_t txn_id = 0; + ASSERT_NO_FATAL_FAILURE(begin_txn(meta_service, db_id, label, table_id, txn_id)); + CreateRowsetResponse res; + auto rowset = create_rowset(txn_id, tablet_id, partition_id); + prepare_rowset(meta_service, rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << label; + res.Clear(); + ASSERT_NO_FATAL_FAILURE(commit_rowset(meta_service, rowset, res)); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << label; + commit_txn(meta_service, db_id, txn_id, label); +} + +static void get_rowset(MetaServiceProxy* meta_service, int64_t table_id, int64_t index_id, + int64_t partition_id, int64_t tablet_id, GetRowsetResponse& res) { + brpc::Controller cntl; + GetRowsetRequest req; + auto* tablet_idx = req.mutable_idx(); + tablet_idx->set_table_id(table_id); + tablet_idx->set_index_id(index_id); + tablet_idx->set_partition_id(partition_id); + tablet_idx->set_tablet_id(tablet_id); + req.set_start_version(0); + req.set_end_version(-1); + req.set_cumulative_compaction_cnt(0); + req.set_base_compaction_cnt(0); + req.set_cumulative_point(2); + meta_service->get_rowset(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << tablet_id; +} + +static void get_delete_bitmap_update_lock(MetaServiceProxy* meta_service, int64_t table_id, + int64_t partition_id, int64_t lock_id, + int64_t initiator) { + brpc::Controller cntl; + GetDeleteBitmapUpdateLockRequest get_lock_req; + GetDeleteBitmapUpdateLockResponse get_lock_res; + get_lock_req.set_cloud_unique_id("test_cloud_unique_id"); + get_lock_req.set_table_id(table_id); + get_lock_req.add_partition_ids(partition_id); + get_lock_req.set_expiration(5); + get_lock_req.set_lock_id(lock_id); + get_lock_req.set_initiator(initiator); + meta_service->get_delete_bitmap_update_lock( + reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &get_lock_req, + &get_lock_res, nullptr); + ASSERT_EQ(get_lock_res.status().code(), MetaServiceCode::OK); +} + +static void update_delete_bitmap(MetaServiceProxy* meta_service, + UpdateDeleteBitmapRequest& update_delete_bitmap_req, + UpdateDeleteBitmapResponse& update_delete_bitmap_res, + int64_t table_id, int64_t partition_id, int64_t lock_id, + int64_t initiator, int64_t tablet_id, int64_t txn_id, + int64_t next_visible_version, std::string data = "1111") { + brpc::Controller cntl; + update_delete_bitmap_req.set_cloud_unique_id("test_cloud_unique_id"); + update_delete_bitmap_req.set_table_id(table_id); + update_delete_bitmap_req.set_partition_id(partition_id); + update_delete_bitmap_req.set_lock_id(lock_id); + update_delete_bitmap_req.set_initiator(initiator); + update_delete_bitmap_req.set_tablet_id(tablet_id); + update_delete_bitmap_req.set_txn_id(txn_id); + update_delete_bitmap_req.set_next_visible_version(next_visible_version); + update_delete_bitmap_req.add_rowset_ids("123"); + update_delete_bitmap_req.add_segment_ids(0); + update_delete_bitmap_req.add_versions(next_visible_version); + update_delete_bitmap_req.add_segment_delete_bitmaps(data); + meta_service->update_delete_bitmap(reinterpret_cast(&cntl), + &update_delete_bitmap_req, &update_delete_bitmap_res, + nullptr); +} + +void start_schema_change_job(MetaServiceProxy* meta_service, int64_t table_id, int64_t index_id, + int64_t partition_id, int64_t tablet_id, int64_t new_tablet_id, + const std::string& job_id, const std::string& initiator, + StartTabletJobResponse& res, int64_t alter_version = -1) { + brpc::Controller cntl; + StartTabletJobRequest req; + req.mutable_job()->mutable_idx()->set_tablet_id(tablet_id); + auto* sc = req.mutable_job()->mutable_schema_change(); + sc->set_id(job_id); + sc->set_initiator(initiator); + sc->mutable_new_tablet_idx()->set_tablet_id(new_tablet_id); + if (alter_version != -1) { + sc->set_alter_version(alter_version); + } + long now = time(nullptr); + sc->set_expiration(now + 12); + meta_service->start_tablet_job(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) + << job_id << ' ' << initiator << ' ' << res.status().msg(); +}; + +void finish_schema_change_job( + MetaService* meta_service, int64_t tablet_id, int64_t new_tablet_id, + const std::string& job_id, const std::string& initiator, + const std::vector& output_rowsets, FinishTabletJobResponse& res, + FinishTabletJobRequest_Action action = FinishTabletJobRequest::COMMIT) { + brpc::Controller cntl; + FinishTabletJobRequest req; + req.set_action(action); + req.mutable_job()->mutable_idx()->set_tablet_id(tablet_id); + auto* sc = req.mutable_job()->mutable_schema_change(); + sc->mutable_new_tablet_idx()->set_tablet_id(new_tablet_id); + if (output_rowsets.empty()) { + sc->set_alter_version(0); + } else { + sc->set_alter_version(output_rowsets.back().end_version()); + for (const auto& rowset : output_rowsets) { + sc->add_txn_ids(rowset.txn_id()); + sc->add_output_versions(rowset.end_version()); + sc->set_num_output_rows(sc->num_output_rows() + rowset.num_rows()); + sc->set_num_output_segments(sc->num_output_segments() + rowset.num_segments()); + sc->set_size_output_rowsets(sc->size_output_rowsets() + rowset.total_disk_size()); + sc->set_index_size_output_rowsets(sc->index_size_output_rowsets() + + rowset.index_disk_size()); + sc->set_segment_size_output_rowsets(sc->segment_size_output_rowsets() + + rowset.data_disk_size()); + } + sc->set_num_output_rowsets(output_rowsets.size()); + } + sc->set_id(job_id); + sc->set_initiator(initiator); + sc->set_delete_bitmap_lock_initiator(12345); + meta_service->finish_tablet_job(&cntl, &req, &res, nullptr); +} + +// create_tablets +TEST(RpcKvBvarTest, CreateTablets) { + auto meta_service = get_meta_service(); + auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); + constexpr auto table_id = 10021, index_id = 10022, partition_id = 10023, tablet_id = 10024; + + mem_kv->get_count_ = 0; + mem_kv->put_count_ = 0; + mem_kv->del_count_ = 0; + + create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id); + LOG(INFO) << "CreateTablets: " << mem_kv->get_count_ << ", " << mem_kv->put_count_ << ", " + << mem_kv->del_count_; + ASSERT_EQ(mem_kv->get_count_, g_bvar_rpc_kv_create_tablets_get_counter.get({mock_instance})); + ASSERT_EQ(mem_kv->put_count_, g_bvar_rpc_kv_create_tablets_put_counter.get({mock_instance})); +} + +// get_tablet +TEST(RpcKvBvarTest, GetTablet) { + std::string cloud_unique_id = "test_cloud_unique_id"; + auto meta_service = get_meta_service(); + auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); + constexpr auto table_id = 10021, index_id = 10022, partition_id = 10023, tablet_id = 10024; + create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id); + + mem_kv->get_count_ = 0; + mem_kv->put_count_ = 0; + mem_kv->del_count_ = 0; + + brpc::Controller cntl; + GetTabletRequest req; + req.set_cloud_unique_id(cloud_unique_id); + req.set_tablet_id(tablet_id); + GetTabletResponse resp; + + meta_service->get_tablet(&cntl, &req, &resp, nullptr); + LOG(INFO) << "GetTablet: " << mem_kv->get_count_ << ", " << mem_kv->put_count_ << ", " + << mem_kv->del_count_; + ASSERT_EQ(mem_kv->get_count_, g_bvar_rpc_kv_get_tablet_get_counter.get({mock_instance})); +} + +// get_tablet_stats +TEST(RpcKvBvarTest, GetTabletStats) { + std::string cloud_unique_id = "test_cloud_unique_id"; + auto meta_service = get_meta_service(); + auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); + constexpr auto table_id = 10021, index_id = 10022, partition_id = 10023, tablet_id = 10024; + create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id); + + mem_kv->get_count_ = 0; + mem_kv->put_count_ = 0; + mem_kv->del_count_ = 0; + + GetTabletStatsResponse res; + get_tablet_stats(meta_service.get(), table_id, index_id, partition_id, tablet_id, res); + + LOG(INFO) << "GetTabletStats: " << mem_kv->get_count_ << ", " << mem_kv->put_count_ << ", " + << mem_kv->del_count_; + ASSERT_EQ(mem_kv->get_count_, g_bvar_rpc_kv_get_tablet_stats_get_counter.get({mock_instance})); +} + +// update_tablet +TEST(RpcKvBvarTest, UpdateTablet) { + auto meta_service = get_meta_service(); + auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); + constexpr auto table_id = 10001, index_id = 10002, partition_id = 10003, tablet_id = 10004; + create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id); + + brpc::Controller cntl; + UpdateTabletRequest req; + UpdateTabletResponse resp; + req.set_cloud_unique_id("test_cloud_unique_id"); + TabletMetaInfoPB* tablet_meta_info = req.add_tablet_meta_infos(); + tablet_meta_info->set_tablet_id(tablet_id); + tablet_meta_info->set_is_in_memory(true); + + mem_kv->get_count_ = 0; + mem_kv->put_count_ = 0; + mem_kv->del_count_ = 0; + + meta_service->update_tablet(&cntl, &req, &resp, nullptr); + + LOG(INFO) << "UpdateTablet: " << mem_kv->get_count_ << ", " << mem_kv->put_count_ << ", " + << mem_kv->del_count_; + ASSERT_EQ(mem_kv->get_count_, g_bvar_rpc_kv_update_tablet_get_counter.get({mock_instance})); + ASSERT_EQ(mem_kv->put_count_, g_bvar_rpc_kv_update_tablet_put_counter.get({mock_instance})); +} + +// update_tablet_schema +// should not call update_tablet_schema +// TEST(RpcKvBvarTest, UpdateTabletSchema) { +// auto meta_service = get_meta_service(); +// auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); +// constexpr auto table_id = 10001, index_id = 10002, partition_id = 10003, tablet_id = 10004; +// create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id); + +// brpc::Controller cntl; +// UpdateTabletSchemaRequest req; +// UpdateTabletSchemaResponse resp; +// req.set_tablet_id(tablet_id); +// req.set_cloud_unique_id("test_cloud_unique_id"); + +// mem_kv->get_count_ = 0; +// mem_kv->put_count_ = 0; +// mem_kv->del_count_ = 0; + +// meta_service->update_tablet_schema(&cntl, &req, &resp, nullptr); + +// LOG(INFO) << "UpdateTabletSchema: " << mem_kv->get_count_ << ", " +// << mem_kv->put_count_ << ", " << mem_kv->del_count_; +// ASSERT_EQ(mem_kv->get_count_, +// g_bvar_rpc_kv_update_tablet_schema_get_counter.get({mock_instance})); +// ASSERT_EQ(mem_kv->put_count_, +// g_bvar_rpc_kv_update_tablet_schema_put_counter.get({mock_instance})); +// ASSERT_EQ(mem_kv->del_count_, +// g_bvar_rpc_kv_update_tablet_schema_del_counter.get({mock_instance})); +// } + +// begin_txn +TEST(RpcKvBvarTest, BeginTxn) { + auto meta_service = get_meta_service(); + auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); + int64_t db_id = 100201; + std::string label = "test_prepare_rowset"; + constexpr auto table_id = 10021, index_id = 10022, partition_id = 10023, tablet_id = 10024; + create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id); + + int64_t txn_id = 0; + + mem_kv->get_count_ = 0; + mem_kv->put_count_ = 0; + mem_kv->del_count_ = 0; + + ASSERT_NO_FATAL_FAILURE(begin_txn(meta_service.get(), db_id, label, table_id, txn_id)); + + LOG(INFO) << "BeginTxn: " << mem_kv->get_count_ << ", " << mem_kv->put_count_ << ", " + << mem_kv->del_count_; + ASSERT_EQ(mem_kv->get_count_, g_bvar_rpc_kv_begin_txn_get_counter.get({mock_instance})); + ASSERT_EQ(mem_kv->put_count_, g_bvar_rpc_kv_begin_txn_put_counter.get({mock_instance})); +} + +// commit_txn +TEST(RpcKvBvarTest, CommitTxn) { + auto meta_service = get_meta_service(); + auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); + int64_t db_id = 100201; + std::string label = "test_prepare_rowset"; + constexpr auto table_id = 10021, index_id = 10022, partition_id = 10023, tablet_id = 10024; + create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id); + int64_t txn_id = 0; + ASSERT_NO_FATAL_FAILURE(begin_txn(meta_service.get(), db_id, label, table_id, txn_id)); + + mem_kv->get_count_ = 0; + mem_kv->put_count_ = 0; + mem_kv->del_count_ = 0; + + commit_txn(meta_service.get(), db_id, txn_id, label); + + LOG(INFO) << "CommitTxn: " << mem_kv->get_count_ << ", " << mem_kv->put_count_ << ", " + << mem_kv->del_count_; + ASSERT_EQ(mem_kv->get_count_, g_bvar_rpc_kv_commit_txn_get_counter.get({mock_instance})); + ASSERT_EQ(mem_kv->put_count_, g_bvar_rpc_kv_commit_txn_put_counter.get({mock_instance})); + ASSERT_EQ(mem_kv->del_count_, g_bvar_rpc_kv_commit_txn_del_counter.get({mock_instance})); +} + +// precommit_txn +TEST(RpcKvBvarTest, PrecommitTxn) { + auto meta_service = get_meta_service(); + auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); + const int64_t db_id = 563413; + const int64_t table_id = 417417878; + const std::string& label = "label_123dae121das"; + int64_t txn_id = -1; + { + brpc::Controller cntl; + BeginTxnRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + TxnInfoPB 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); + req.mutable_txn_info()->CopyFrom(txn_info); + 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, -1); + } + + std::unique_ptr txn; + TxnErrorCode err = meta_service->txn_kv()->create_txn(&txn); + ASSERT_EQ(err, TxnErrorCode::TXN_OK); + + const std::string info_key = txn_info_key({mock_instance, db_id, txn_id}); + std::string info_val; + ASSERT_EQ(txn->get(info_key, &info_val), TxnErrorCode::TXN_OK); + TxnInfoPB txn_info; + txn_info.ParseFromString(info_val); + ASSERT_EQ(txn_info.status(), TxnStatusPB::TXN_STATUS_PREPARED); + + brpc::Controller cntl; + PrecommitTxnRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + req.set_txn_id(txn_id); + req.set_precommit_timeout_ms(36000); + PrecommitTxnResponse res; + + mem_kv->get_count_ = 0; + mem_kv->put_count_ = 0; + mem_kv->del_count_ = 0; + + meta_service->precommit_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, + &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + + LOG(INFO) << "PrecommitTxn: " << mem_kv->get_count_ << ", " << mem_kv->put_count_ << ", " + << mem_kv->del_count_; + ASSERT_EQ(mem_kv->get_count_, g_bvar_rpc_kv_precommit_txn_get_counter.get({mock_instance})); + ASSERT_EQ(mem_kv->put_count_, g_bvar_rpc_kv_precommit_txn_put_counter.get({mock_instance})); +} + +// abort_txn +TEST(RpcKvBvarTest, AbortTxn) { + auto meta_service = get_meta_service(); + auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); + int64_t db_id = 100201; + std::string label = "test_prepare_rowset"; + constexpr auto table_id = 10021, index_id = 10022, partition_id = 10023, tablet_id = 10024; + create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id); + int64_t txn_id = 0; + ASSERT_NO_FATAL_FAILURE(begin_txn(meta_service.get(), db_id, label, table_id, txn_id)); + + mem_kv->get_count_ = 0; + mem_kv->put_count_ = 0; + mem_kv->del_count_ = 0; + + brpc::Controller cntl; + AbortTxnRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + 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); + + LOG(INFO) << "AbortTxn: " << mem_kv->get_count_ << ", " << mem_kv->put_count_ << ", " + << mem_kv->del_count_; + ASSERT_EQ(mem_kv->get_count_, g_bvar_rpc_kv_abort_txn_get_counter.get({mock_instance})); + ASSERT_EQ(mem_kv->put_count_, g_bvar_rpc_kv_abort_txn_put_counter.get({mock_instance})); + ASSERT_EQ(mem_kv->del_count_, g_bvar_rpc_kv_abort_txn_del_counter.get({mock_instance})); +} + +// get_txn +TEST(RpcKvBvarTest, GetTxn) { + auto meta_service = get_meta_service(); + auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); + int64_t db_id = 100201; + std::string label = "test_prepare_rowset"; + constexpr auto table_id = 10021, index_id = 10022, partition_id = 10023, tablet_id = 10024; + create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id); + int64_t txn_id = 0; + ASSERT_NO_FATAL_FAILURE(begin_txn(meta_service.get(), db_id, label, table_id, txn_id)); + + mem_kv->get_count_ = 0; + mem_kv->put_count_ = 0; + mem_kv->del_count_ = 0; + + brpc::Controller cntl; + GetTxnRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + req.set_txn_id(txn_id); + req.set_db_id(db_id); + GetTxnResponse res; + meta_service->get_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, &res, + nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + + LOG(INFO) << "GetTxn: " << mem_kv->get_count_ << ", " << mem_kv->put_count_ << ", " + << mem_kv->del_count_; + ASSERT_EQ(mem_kv->get_count_, g_bvar_rpc_kv_get_txn_get_counter.get({mock_instance})); +} + +// get_txn_id +TEST(RpcKvBvarTest, GetTxnId) { + auto meta_service = get_meta_service(); + auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); + int64_t db_id = 100201; + std::string label = "test_prepare_rowset"; + constexpr auto table_id = 10021, index_id = 10022, partition_id = 10023, tablet_id = 10024; + create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id); + + int64_t txn_id = 0; + + ASSERT_NO_FATAL_FAILURE(begin_txn(meta_service.get(), db_id, label, table_id, txn_id)); + + brpc::Controller cntl; + GetTxnIdRequest req; + GetTxnIdResponse res; + req.set_cloud_unique_id("test_cloud_unique_id"); + req.set_label(label); + req.set_db_id(db_id); + + mem_kv->get_count_ = 0; + mem_kv->put_count_ = 0; + mem_kv->del_count_ = 0; + + meta_service->get_txn_id(&cntl, &req, &res, nullptr); + + LOG(INFO) << "GetTxnId: " << mem_kv->get_count_ << ", " << mem_kv->put_count_ << ", " + << mem_kv->del_count_; + ASSERT_EQ(mem_kv->get_count_, g_bvar_rpc_kv_get_txn_id_get_counter.get({mock_instance})); +} + +// prepare_rowset +TEST(RpcKvBvarTest, PrepareRowset) { + int64_t db_id = 100201; + std::string label = "test_prepare_rowset"; + auto meta_service = get_meta_service(); + auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); + constexpr auto table_id = 10021, index_id = 10022, partition_id = 10023, tablet_id = 10024; + create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id); + + int64_t txn_id = 0; + ASSERT_NO_FATAL_FAILURE(begin_txn(meta_service.get(), db_id, label, table_id, txn_id)); + CreateRowsetResponse res; + auto rowset = create_rowset(txn_id, tablet_id, partition_id); + rowset.mutable_load_id()->set_hi(123); + rowset.mutable_load_id()->set_lo(456); + + mem_kv->get_count_ = 0; + mem_kv->put_count_ = 0; + mem_kv->del_count_ = 0; + + prepare_rowset(meta_service.get(), rowset, res); + + LOG(INFO) << "PrepareRowset: " << mem_kv->get_count_ << ", " << mem_kv->put_count_ << ", " + << mem_kv->del_count_; + ASSERT_EQ(mem_kv->get_count_, g_bvar_rpc_kv_prepare_rowset_get_counter.get({mock_instance})); + ASSERT_EQ(mem_kv->put_count_, g_bvar_rpc_kv_prepare_rowset_put_counter.get({mock_instance})); +} + +// get_rowset +TEST(RpcKvBvarTest, GetRowset) { + auto meta_service = get_meta_service(); + auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); + + constexpr auto table_id = 10001, index_id = 10002, partition_id = 10003, tablet_id = 10004; + config::write_schema_kv = true; + ASSERT_NO_FATAL_FAILURE(create_tablet(meta_service.get(), table_id, index_id, partition_id, + tablet_id, next_rowset_id(), 1)); + // check get tablet response + check_get_tablet(meta_service.get(), tablet_id, 1); + + mem_kv->get_count_ = 0; + mem_kv->put_count_ = 0; + mem_kv->del_count_ = 0; + + // check get rowset response + GetRowsetResponse get_rowset_res; + get_rowset(meta_service.get(), table_id, index_id, partition_id, tablet_id, get_rowset_res); + + LOG(INFO) << "GetRowset: " << mem_kv->get_count_ << ", " << mem_kv->put_count_ << ", " + << mem_kv->del_count_; + ASSERT_EQ(mem_kv->get_count_, g_bvar_rpc_kv_get_rowset_get_counter.get({mock_instance})); +} + +// update_tmp_rowset +TEST(RpcKvBvarTest, UpdateTmpRowset) { + auto meta_service = get_meta_service(); + auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); + constexpr auto table_id = 10001, index_id = 10002, partition_id = 10003, tablet_id = 10004; + int64_t txn_id = -1; + create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id); + auto rowset = create_rowset(txn_id, tablet_id, partition_id); + rowset.set_num_segments(rowset.num_segments() + 3); + rowset.set_num_rows(rowset.num_rows() + 1000); + rowset.set_total_disk_size(rowset.total_disk_size() + 11000); + rowset.set_index_disk_size(rowset.index_disk_size() + 1000); + rowset.set_data_disk_size(rowset.data_disk_size() + 10000); + + std::unique_ptr txn; + std::string update_key; + brpc::Controller cntl; + CreateRowsetResponse res; + auto* arena = res.GetArena(); + auto* req = google::protobuf::Arena::CreateMessage(arena); + std::string instance_id = get_instance_id(meta_service->resource_mgr(), req->cloud_unique_id()); + MetaRowsetTmpKeyInfo key_info {instance_id, txn_id, tablet_id}; + meta_rowset_tmp_key(key_info, &update_key); + EXPECT_EQ(mem_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put(update_key, "update_tmp_rowset_val"); + EXPECT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + req->mutable_rowset_meta()->CopyFrom(rowset); + + mem_kv->get_count_ = 0; + mem_kv->put_count_ = 0; + mem_kv->del_count_ = 0; + + meta_service->update_tmp_rowset(&cntl, req, &res, nullptr); + + if (!arena) { + delete req; + } + + LOG(INFO) << "UpdateTmpRowset: " << mem_kv->get_count_ << ", " << mem_kv->put_count_ << ", " + << mem_kv->del_count_; + ASSERT_EQ(mem_kv->get_count_, g_bvar_rpc_kv_update_tmp_rowset_get_counter.get({mock_instance})); + ASSERT_EQ(mem_kv->put_count_, g_bvar_rpc_kv_update_tmp_rowset_put_counter.get({mock_instance})); +} + +// commit_rowset +TEST(RpcKvBvarTest, CommitRowset) { + auto meta_service = get_meta_service(); + auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); + + constexpr auto table_id = 10001, index_id = 10002, partition_id = 10003, tablet_id = 10004; + int64_t txn_id = -1; + create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id); + auto tmp_rowset = create_rowset(txn_id, tablet_id, partition_id); + + mem_kv->get_count_ = 0; + mem_kv->put_count_ = 0; + mem_kv->del_count_ = 0; + + CreateRowsetResponse res; + commit_rowset(meta_service.get(), tmp_rowset, res); + + LOG(INFO) << "CommitRowset: " << mem_kv->get_count_ << ", " << mem_kv->put_count_ << ", " + << mem_kv->del_count_; + ASSERT_EQ(mem_kv->get_count_, g_bvar_rpc_kv_commit_rowset_get_counter.get({mock_instance})); + ASSERT_EQ(mem_kv->put_count_, g_bvar_rpc_kv_commit_rowset_put_counter.get({mock_instance})); + ASSERT_EQ(mem_kv->del_count_, g_bvar_rpc_kv_commit_rowset_del_counter.get({mock_instance})); +} + +// get_version +TEST(RpcKvBvarTest, GetVersion) { + auto meta_service = get_meta_service(); + auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); + + constexpr auto table_id = 10001, partition_id = 10003, tablet_id = 10004; + create_tablet(meta_service.get(), table_id, 1, partition_id, tablet_id); + insert_rowset(meta_service.get(), 1, "get_version_label_1", table_id, partition_id, tablet_id); + + brpc::Controller ctrl; + GetVersionRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + req.set_db_id(1); + req.set_table_id(table_id); + req.set_partition_id(partition_id); + + mem_kv->get_count_ = 0; + mem_kv->put_count_ = 0; + mem_kv->del_count_ = 0; + + GetVersionResponse resp; + meta_service->get_version(&ctrl, &req, &resp, nullptr); + + LOG(INFO) << "GetVersion: " << mem_kv->get_count_ << ", " << mem_kv->put_count_ << ", " + << mem_kv->del_count_; + ASSERT_EQ(mem_kv->get_count_, g_bvar_rpc_kv_get_version_get_counter.get({mock_instance})); +} + +// get_schema_dict +TEST(RpcKvBvarTest, GetSchemaDict) { + auto meta_service = get_meta_service(); + auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); + constexpr auto table_id = 10001, index_id = 10002, partition_id = 10003, tablet_id = 10004; + create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id); + + brpc::Controller ctrl; + GetSchemaDictRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + req.set_index_id(index_id); + + std::unique_ptr txn; + std::string instance_id = get_instance_id(meta_service->resource_mgr(), req.cloud_unique_id()); + std::string dict_key = meta_schema_pb_dictionary_key({instance_id, req.index_id()}); + EXPECT_EQ(mem_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put(dict_key, "dict_val"); + EXPECT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + mem_kv->get_count_ = 0; + mem_kv->put_count_ = 0; + mem_kv->del_count_ = 0; + + GetSchemaDictResponse resp; + meta_service->get_schema_dict(&ctrl, &req, &resp, nullptr); + + LOG(INFO) << "GetSchemaDict: " << mem_kv->get_count_ << ", " << mem_kv->put_count_ << ", " + << mem_kv->del_count_; + ASSERT_EQ(mem_kv->get_count_, g_bvar_rpc_kv_get_schema_dict_get_counter.get({mock_instance})); +} + +// get_delete_bitmap_update_lock +TEST(RpcKvBvarTest, GetDeleteBitmapUpdateLock) { + auto meta_service = get_meta_service(); + auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); + + int64_t db_id = 99999; + int64_t table_id = 1801; + int64_t index_id = 4801; + int64_t t1p1 = 2001; + int64_t tablet_id = 3001; + int64_t txn_id; + ASSERT_NO_FATAL_FAILURE(create_tablet_with_db_id(meta_service.get(), db_id, table_id, index_id, + t1p1, tablet_id)); + begin_txn_and_commit_rowset(meta_service.get(), "label11", db_id, table_id, t1p1, tablet_id, + &txn_id); + int64_t lock_id = -2; + int64_t initiator = 1009; + + mem_kv->get_count_ = 0; + mem_kv->put_count_ = 0; + mem_kv->del_count_ = 0; + + get_delete_bitmap_update_lock(meta_service.get(), table_id, t1p1, lock_id, initiator); + + LOG(INFO) << "GetDeleteBitmapUpdateLock: " << mem_kv->get_count_ << ", " << mem_kv->put_count_ + << ", " << mem_kv->del_count_; + ASSERT_EQ(mem_kv->get_count_, + g_bvar_rpc_kv_get_delete_bitmap_update_lock_get_counter.get({mock_instance})); + ASSERT_EQ(mem_kv->put_count_, + g_bvar_rpc_kv_get_delete_bitmap_update_lock_put_counter.get({mock_instance})); + ASSERT_EQ(mem_kv->del_count_, + g_bvar_rpc_kv_get_delete_bitmap_update_lock_del_counter.get({mock_instance})); +} + +// update_delete_bitmap +TEST(RpcKvBvarTest, UpdateDeleteBitmap) { + auto meta_service = get_meta_service(); + auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); + + int64_t db_id = 99999; + int64_t table_id = 1801; + int64_t index_id = 4801; + int64_t t1p1 = 2001; + int64_t tablet_id = 3001; + int64_t txn_id; + size_t split_size = 90 * 1000; // see cloud/src/common/util.h + ASSERT_NO_FATAL_FAILURE(create_tablet_with_db_id(meta_service.get(), db_id, table_id, index_id, + t1p1, tablet_id)); + begin_txn_and_commit_rowset(meta_service.get(), "label11", db_id, table_id, t1p1, tablet_id, + &txn_id); + int64_t lock_id = -2; + int64_t initiator = 1009; + int64_t version = 100; + get_delete_bitmap_update_lock(meta_service.get(), table_id, t1p1, lock_id, initiator); + UpdateDeleteBitmapRequest update_delete_bitmap_req; + UpdateDeleteBitmapResponse update_delete_bitmap_res; + // will be splited and stored in 5 KVs + std::string data1(split_size * 5, 'c'); + + mem_kv->get_count_ = 0; + mem_kv->put_count_ = 0; + mem_kv->del_count_ = 0; + + update_delete_bitmap(meta_service.get(), update_delete_bitmap_req, update_delete_bitmap_res, + table_id, t1p1, lock_id, initiator, tablet_id, txn_id, version, data1); + + LOG(INFO) << "UpdateDeleteBitmap: " << mem_kv->get_count_ << ", " << mem_kv->put_count_ << ", " + << mem_kv->del_count_; + ASSERT_EQ(mem_kv->get_count_, + g_bvar_rpc_kv_update_delete_bitmap_get_counter.get({mock_instance})); + ASSERT_EQ(mem_kv->put_count_, + g_bvar_rpc_kv_update_delete_bitmap_put_counter.get({mock_instance})); + ASSERT_EQ(mem_kv->del_count_, + g_bvar_rpc_kv_update_delete_bitmap_del_counter.get({mock_instance})); +} + +// get_delete_bitmap +TEST(RpcKvBvarTest, GetDeleteBitmap) { + auto meta_service = get_meta_service(); + auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); + int64_t db_id = 99999; + int64_t table_id = 1801; + int64_t index_id = 4801; + int64_t t1p1 = 2001; + int64_t tablet_id = 3001; + int64_t txn_id; + size_t split_size = 90 * 1000; // see cloud/src/common/util.h + ASSERT_NO_FATAL_FAILURE(create_tablet_with_db_id(meta_service.get(), db_id, table_id, index_id, + t1p1, tablet_id)); + begin_txn_and_commit_rowset(meta_service.get(), "label11", db_id, table_id, t1p1, tablet_id, + &txn_id); + int64_t lock_id = -2; + int64_t initiator = 1009; + int64_t version = 100; + get_delete_bitmap_update_lock(meta_service.get(), table_id, t1p1, lock_id, initiator); + UpdateDeleteBitmapRequest update_delete_bitmap_req; + UpdateDeleteBitmapResponse update_delete_bitmap_res; + // will be splited and stored in 5 KVs + std::string data1(split_size * 5, 'c'); + update_delete_bitmap(meta_service.get(), update_delete_bitmap_req, update_delete_bitmap_res, + table_id, t1p1, lock_id, initiator, tablet_id, txn_id, version, data1); + + brpc::Controller ctrl; + GetDeleteBitmapRequest get_delete_bitmap_req; + GetDeleteBitmapResponse get_delete_bitmap_res; + get_delete_bitmap_req.set_cloud_unique_id("test_cloud_unique_id"); + get_delete_bitmap_req.set_tablet_id(tablet_id); + get_delete_bitmap_req.add_rowset_ids("123"); + get_delete_bitmap_req.add_begin_versions(0); + get_delete_bitmap_req.add_end_versions(version); + + mem_kv->get_count_ = 0; + mem_kv->put_count_ = 0; + mem_kv->del_count_ = 0; + + meta_service->get_delete_bitmap(reinterpret_cast(&ctrl), + &get_delete_bitmap_req, &get_delete_bitmap_res, nullptr); + + LOG(INFO) << "GetDeleteBitmap: " << mem_kv->get_count_ << ", " << mem_kv->put_count_ << ", " + << mem_kv->del_count_; + ASSERT_EQ(mem_kv->get_count_, g_bvar_rpc_kv_get_delete_bitmap_get_counter.get({mock_instance})); +} + +// remove_delete_bitmap_update_lock +TEST(RpcKvBvarTest, RemoveDeleteBitmapUpdateLock) { + auto meta_service = get_meta_service(); + auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); + + int64_t db_id = 99999; + int64_t table_id = 1801; + int64_t index_id = 4801; + int64_t t1p1 = 2001; + int64_t tablet_id = 3001; + int64_t txn_id; + ASSERT_NO_FATAL_FAILURE(create_tablet_with_db_id(meta_service.get(), db_id, table_id, index_id, + t1p1, tablet_id)); + begin_txn_and_commit_rowset(meta_service.get(), "label11", db_id, table_id, t1p1, tablet_id, + &txn_id); + int64_t lock_id = -2; + int64_t initiator = 1009; + + get_delete_bitmap_update_lock(meta_service.get(), table_id, t1p1, lock_id, initiator); + brpc::Controller cntl; + RemoveDeleteBitmapUpdateLockRequest remove_req; + RemoveDeleteBitmapUpdateLockResponse remove_res; + + remove_req.set_cloud_unique_id("test_cloud_unique_id"); + remove_req.set_table_id(table_id); + remove_req.set_lock_id(lock_id); + remove_req.set_initiator(initiator); + + mem_kv->get_count_ = 0; + mem_kv->put_count_ = 0; + mem_kv->del_count_ = 0; + + meta_service->remove_delete_bitmap_update_lock( + reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &remove_req, &remove_res, + nullptr); + + LOG(INFO) << "RemoveDeleteBitmapUpdateLock: " << mem_kv->get_count_ << ", " + << mem_kv->put_count_ << ", " << mem_kv->del_count_; + ASSERT_EQ(mem_kv->get_count_, + g_bvar_rpc_kv_remove_delete_bitmap_update_lock_get_counter.get({mock_instance})); + ASSERT_EQ(mem_kv->put_count_, + g_bvar_rpc_kv_remove_delete_bitmap_update_lock_put_counter.get({mock_instance})); + ASSERT_EQ(mem_kv->del_count_, + g_bvar_rpc_kv_remove_delete_bitmap_update_lock_del_counter.get({mock_instance})); +} + +// remove_delete_bitmap +TEST(RpcKvBvarTest, RemoveDeleteBitmap) { + auto meta_service = get_meta_service(); + auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); + int64_t db_id = 99999; + int64_t table_id = 1801; + int64_t index_id = 4801; + int64_t t1p1 = 2001; + int64_t tablet_id = 3001; + int64_t txn_id; + size_t split_size = 90 * 1000; // see cloud/src/common/util.h + ASSERT_NO_FATAL_FAILURE(create_tablet_with_db_id(meta_service.get(), db_id, table_id, index_id, + t1p1, tablet_id)); + begin_txn_and_commit_rowset(meta_service.get(), "label1", db_id, table_id, t1p1, tablet_id, + &txn_id); + int64_t lock_id = -2; + int64_t initiator = 1009; + get_delete_bitmap_update_lock(meta_service.get(), table_id, t1p1, lock_id, initiator); + int64_t version = 100; + UpdateDeleteBitmapRequest update_delete_bitmap_req; + UpdateDeleteBitmapResponse update_delete_bitmap_res; + // will be splited and stored in 5 KVs + std::string data1(split_size * 5, 'c'); + update_delete_bitmap(meta_service.get(), update_delete_bitmap_req, update_delete_bitmap_res, + table_id, t1p1, lock_id, initiator, tablet_id, txn_id, version, data1); + + brpc::Controller ctrl; + RemoveDeleteBitmapRequest req; + RemoveDeleteBitmapResponse resp; + req.add_begin_versions(version); + req.add_end_versions(version); + req.add_rowset_ids("rowset_ids"); + req.set_cloud_unique_id("test_cloud_unique_id"); + + mem_kv->get_count_ = 0; + mem_kv->put_count_ = 0; + mem_kv->del_count_ = 0; + + meta_service->remove_delete_bitmap(&ctrl, &req, &resp, nullptr); + + LOG(INFO) << "RemoveDeleteBitmap: " << mem_kv->get_count_ << ", " << mem_kv->put_count_ << ", " + << mem_kv->del_count_; + ASSERT_EQ(mem_kv->del_count_, + g_bvar_rpc_kv_remove_delete_bitmap_del_counter.get({mock_instance})); +} + +// start_tablet_job +TEST(RpcKvBvarTest, StartTabletJob) { + auto meta_service = get_meta_service(); + auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); + constexpr int64_t table_id = 10001; + constexpr int64_t index_id = 10002; + constexpr int64_t partition_id = 10003; + constexpr int64_t tablet_id = 10004; + create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id); + StartTabletJobResponse res; + + mem_kv->get_count_ = 0; + mem_kv->put_count_ = 0; + mem_kv->del_count_ = 0; + + start_compaction_job(meta_service.get(), tablet_id, "compaction1", "ip:port", 0, 0, + TabletCompactionJobPB::BASE, res); + + LOG(INFO) << "StartTabletJob: " << mem_kv->get_count_ << ", " << mem_kv->put_count_ << ", " + << mem_kv->del_count_; + ASSERT_EQ(mem_kv->get_count_, g_bvar_rpc_kv_start_tablet_job_get_counter.get({mock_instance})); + ASSERT_EQ(mem_kv->put_count_, g_bvar_rpc_kv_start_tablet_job_put_counter.get({mock_instance})); +} + +// finish_tablet_job +TEST(RpcKvBvarTest, FinishTabletJob) { + auto meta_service = get_meta_service(); + auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); + brpc::Controller cntl; + + int64_t table_id = 1; + int64_t index_id = 2; + int64_t partition_id = 3; + int64_t tablet_id = 4; + + ASSERT_NO_FATAL_FAILURE( + create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id, false)); + + StartTabletJobResponse res; + start_compaction_job(meta_service.get(), tablet_id, "job1", "BE1", 0, 0, + TabletCompactionJobPB::CUMULATIVE, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + res.Clear(); + + int64_t new_tablet_id = 11; + ASSERT_NO_FATAL_FAILURE(create_tablet(meta_service.get(), table_id, index_id, partition_id, + new_tablet_id, false, true)); + 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, "job2", + "BE1", sc_res)); + + long now = time(nullptr); + FinishTabletJobRequest req; + FinishTabletJobResponse finish_res_2; + req.set_action(FinishTabletJobRequest::LEASE); + auto* compaction = req.mutable_job()->add_compaction(); + compaction->set_id("job1"); + compaction->set_initiator("BE1"); + compaction->set_lease(now + 10); + req.mutable_job()->mutable_idx()->set_tablet_id(tablet_id); + + mem_kv->get_count_ = 0; + mem_kv->put_count_ = 0; + mem_kv->del_count_ = 0; + + meta_service->finish_tablet_job(&cntl, &req, &finish_res_2, nullptr); + + LOG(INFO) << "FinishTabletJob: " << mem_kv->get_count_ << ", " << mem_kv->put_count_ << ", " + << mem_kv->del_count_; + ASSERT_EQ(mem_kv->get_count_, g_bvar_rpc_kv_finish_tablet_job_get_counter.get({mock_instance})); + ASSERT_EQ(mem_kv->put_count_, g_bvar_rpc_kv_finish_tablet_job_put_counter.get({mock_instance})); + ASSERT_EQ(mem_kv->del_count_, g_bvar_rpc_kv_finish_tablet_job_del_counter.get({mock_instance})); +} + +// prepare_index +TEST(RpcKvBvarTest, PrepareIndex) { + auto meta_service = get_meta_service(); + auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); + std::string instance_id = "test_cloud_instance_id"; + + constexpr int64_t table_id = 10001; + constexpr int64_t index_id = 10002; + constexpr int64_t partition_id = 10003; + constexpr int64_t tablet_id = 10004; + + std::unique_ptr txn; + doris::TabletMetaCloudPB tablet_pb; + tablet_pb.set_table_id(table_id); + tablet_pb.set_index_id(index_id); + tablet_pb.set_partition_id(partition_id); + tablet_pb.set_tablet_id(tablet_id); + auto tablet_key = meta_tablet_key({instance_id, table_id, index_id, partition_id, tablet_id}); + auto tablet_val = tablet_pb.SerializeAsString(); + RecycleIndexPB index_pb; + auto index_key = recycle_index_key({instance_id, index_id}); + auto tbl_version_key = table_version_key({instance_id, 1, table_id}); + std::string val; + + brpc::Controller ctrl; + IndexRequest req; + req.set_db_id(1); + req.set_table_id(table_id); + req.add_index_ids(index_id); + req.set_is_new_table(true); + IndexResponse res; + + mem_kv->get_count_ = 0; + mem_kv->put_count_ = 0; + mem_kv->del_count_ = 0; + + meta_service->prepare_index(&ctrl, &req, &res, nullptr); + + LOG(INFO) << "PrepareIndex: " << mem_kv->get_count_ << ", " << mem_kv->put_count_ << ", " + << mem_kv->del_count_; + ASSERT_EQ(mem_kv->get_count_, g_bvar_rpc_kv_prepare_index_get_counter.get({mock_instance})); + ASSERT_EQ(mem_kv->put_count_, g_bvar_rpc_kv_prepare_index_put_counter.get({mock_instance})); +} + +// commit_index +TEST(RpcKvBvarTest, CommitIndex) { + auto meta_service = get_meta_service(); + auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); + std::string instance_id = "test_cloud_instance_id"; + + constexpr int64_t table_id = 10001; + constexpr int64_t index_id = 10002; + constexpr int64_t partition_id = 10003; + constexpr int64_t tablet_id = 10004; + + std::unique_ptr txn; + doris::TabletMetaCloudPB tablet_pb; + tablet_pb.set_table_id(table_id); + tablet_pb.set_index_id(index_id); + tablet_pb.set_partition_id(partition_id); + tablet_pb.set_tablet_id(tablet_id); + auto tablet_key = meta_tablet_key({instance_id, table_id, index_id, partition_id, tablet_id}); + auto tablet_val = tablet_pb.SerializeAsString(); + RecycleIndexPB index_pb; + auto index_key = recycle_index_key({instance_id, index_id}); + auto tbl_version_key = table_version_key({instance_id, 1, table_id}); + std::string val; + + brpc::Controller ctrl; + IndexRequest req; + req.set_db_id(1); + req.set_table_id(table_id); + req.add_index_ids(index_id); + req.set_is_new_table(true); + IndexResponse res; + meta_service->prepare_index(&ctrl, &req, &res, nullptr); + res.Clear(); + + mem_kv->get_count_ = 0; + mem_kv->put_count_ = 0; + mem_kv->del_count_ = 0; + + meta_service->commit_index(&ctrl, &req, &res, nullptr); + + LOG(INFO) << "CommitIndex: " << mem_kv->get_count_ << ", " << mem_kv->put_count_ << ", " + << mem_kv->del_count_; + ASSERT_EQ(mem_kv->get_count_, g_bvar_rpc_kv_commit_index_get_counter.get({mock_instance})); + ASSERT_EQ(mem_kv->put_count_, g_bvar_rpc_kv_commit_index_put_counter.get({mock_instance})); + ASSERT_EQ(mem_kv->del_count_, g_bvar_rpc_kv_commit_index_del_counter.get({mock_instance})); +} + +// drop_index +TEST(RpcKvBvarTest, DropIndex) { + auto meta_service = get_meta_service(); + auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); + int64_t db_id = 4524364; + int64_t table_id = 65354; + int64_t index_id = 658432; + int64_t partition_id = 76553; + std::string mock_instance = "test_instance"; + const std::string label = "test_label_67a34e2q1231"; + + int64_t tablet_id_base = 2313324; + for (int i = 0; i < 10; ++i) { + create_tablet_with_db_id(meta_service.get(), db_id, table_id, index_id, partition_id, + tablet_id_base + i); + } + int txn_id {}; + { + 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 < 10; ++i) { + auto tmp_rowset = + create_rowset(txn_id, tablet_id_base + i, index_id, partition_id, -1, 100); + CreateRowsetResponse res; + commit_rowset(meta_service.get(), tmp_rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } + } + + brpc::Controller cntl; + IndexRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + + req.set_db_id(1); + req.set_table_id(table_id); + req.add_index_ids(index_id); + IndexResponse res; + + mem_kv->get_count_ = 0; + mem_kv->put_count_ = 0; + mem_kv->del_count_ = 0; + + meta_service->drop_index(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, + &res, nullptr); + + LOG(INFO) << "DropIndex: " << mem_kv->get_count_ << ", " << mem_kv->put_count_ << ", " + << mem_kv->del_count_; + ASSERT_EQ(mem_kv->get_count_, g_bvar_rpc_kv_drop_index_get_counter.get({mock_instance})); + ASSERT_EQ(mem_kv->put_count_, g_bvar_rpc_kv_drop_index_put_counter.get({mock_instance})); +} + +// prepare_partition +TEST(RpcKvBvarTest, PreparePartition) { + auto meta_service = get_meta_service(); + auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); + std::string instance_id = "test_cloud_instance_id"; + constexpr int64_t table_id = 10001; + constexpr int64_t index_id = 10002; + constexpr int64_t partition_id = 10003; + constexpr int64_t tablet_id = 10004; + + std::unique_ptr txn; + doris::TabletMetaCloudPB tablet_pb; + tablet_pb.set_table_id(table_id); + tablet_pb.set_index_id(index_id); + tablet_pb.set_partition_id(partition_id); + tablet_pb.set_tablet_id(tablet_id); + auto tablet_key = meta_tablet_key({instance_id, table_id, index_id, partition_id, tablet_id}); + auto tablet_val = tablet_pb.SerializeAsString(); + RecyclePartitionPB partition_pb; + auto partition_key = recycle_partition_key({instance_id, partition_id}); + auto tbl_version_key = table_version_key({instance_id, 1, table_id}); + std::string val; + brpc::Controller ctrl; + PartitionRequest req; + PartitionResponse res; + req.set_db_id(1); + req.set_table_id(table_id); + req.add_index_ids(index_id); + req.add_partition_ids(partition_id); + res.Clear(); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->atomic_add(tbl_version_key, 1); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + mem_kv->get_count_ = 0; + mem_kv->put_count_ = 0; + mem_kv->del_count_ = 0; + + meta_service->prepare_partition(&ctrl, &req, &res, nullptr); + + LOG(INFO) << "PreparePartition: " << mem_kv->get_count_ << ", " << mem_kv->put_count_ << ", " + << mem_kv->del_count_; + ASSERT_EQ(mem_kv->get_count_, g_bvar_rpc_kv_prepare_partition_get_counter.get({mock_instance})); + ASSERT_EQ(mem_kv->put_count_, g_bvar_rpc_kv_prepare_partition_put_counter.get({mock_instance})); +} + +// commit_partition +TEST(RpcKvBvarTest, CommitPartition) { + auto meta_service = get_meta_service(); + auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); + std::string instance_id = "test_cloud_instance_id"; + constexpr int64_t table_id = 10001; + constexpr int64_t index_id = 10002; + constexpr int64_t partition_id = 10003; + constexpr int64_t tablet_id = 10004; + + std::unique_ptr txn; + doris::TabletMetaCloudPB tablet_pb; + tablet_pb.set_table_id(table_id); + tablet_pb.set_index_id(index_id); + tablet_pb.set_partition_id(partition_id); + tablet_pb.set_tablet_id(tablet_id); + auto tablet_key = meta_tablet_key({instance_id, table_id, index_id, partition_id, tablet_id}); + auto tablet_val = tablet_pb.SerializeAsString(); + RecyclePartitionPB partition_pb; + auto partition_key = recycle_partition_key({instance_id, partition_id}); + auto tbl_version_key = table_version_key({instance_id, 1, table_id}); + std::string val; + brpc::Controller ctrl; + PartitionRequest req; + PartitionResponse res; + req.set_db_id(1); + req.set_table_id(table_id); + req.add_index_ids(index_id); + req.add_partition_ids(partition_id); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->atomic_add(tbl_version_key, 1); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + meta_service->prepare_partition(&ctrl, &req, &res, nullptr); + + mem_kv->get_count_ = 0; + mem_kv->put_count_ = 0; + mem_kv->del_count_ = 0; + + meta_service->commit_partition(&ctrl, &req, &res, nullptr); + + LOG(INFO) << "CommitPartition: " << mem_kv->get_count_ << ", " << mem_kv->put_count_ << ", " + << mem_kv->del_count_; + ASSERT_EQ(mem_kv->get_count_, g_bvar_rpc_kv_commit_partition_get_counter.get({mock_instance})); + ASSERT_EQ(mem_kv->put_count_, g_bvar_rpc_kv_commit_partition_put_counter.get({mock_instance})); + ASSERT_EQ(mem_kv->del_count_, g_bvar_rpc_kv_commit_partition_del_counter.get({mock_instance})); +} + +// check_kv +TEST(RpcKvBvarTest, CheckKv) { + auto meta_service = get_meta_service(); + auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); + std::string instance_id = "test_instance"; + constexpr int64_t table_id = 10001; + constexpr int64_t index_id = 10002; + constexpr int64_t partition_id = 10003; + + std::unique_ptr txn; + RecyclePartitionPB partition_pb; + auto partition_key = recycle_partition_key({instance_id, 10004}); + auto tbl_version_key = table_version_key({instance_id, 1, table_id}); + brpc::Controller ctrl; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put(partition_key, "val"); + txn->atomic_add(tbl_version_key, 1); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + CheckKVRequest req_check; + CheckKVResponse res_check; + req_check.set_op(CheckKVRequest::CREATE_PARTITION_AFTER_FE_COMMIT); + CheckKeyInfos check_keys_pb; + check_keys_pb.add_table_ids(table_id + 1); + check_keys_pb.add_index_ids(index_id + 1); + check_keys_pb.add_partition_ids(partition_id + 1); + req_check.mutable_check_keys()->CopyFrom(check_keys_pb); + + mem_kv->get_count_ = 0; + mem_kv->put_count_ = 0; + mem_kv->del_count_ = 0; + + meta_service->check_kv(&ctrl, &req_check, &res_check, nullptr); + + LOG(INFO) << "CheckKv: " << mem_kv->get_count_ << ", " << mem_kv->put_count_ << ", " + << mem_kv->del_count_; + ASSERT_EQ(mem_kv->get_count_, g_bvar_rpc_kv_check_kv_get_counter.get({mock_instance})); +} + +// drop_partition +TEST(RpcKvBvarTest, DropPartition) { + auto meta_service = get_meta_service(); + auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); + std::string instance_id = "test_instance"; + constexpr int64_t table_id = 10001; + constexpr int64_t index_id = 10002; + constexpr int64_t partition_id = 10003; + auto tbl_version_key = table_version_key({instance_id, 1, table_id}); + + std::unique_ptr txn; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->atomic_add(tbl_version_key, 1); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + brpc::Controller ctrl; + PartitionRequest req; + PartitionResponse res; + req.set_db_id(1); + req.set_table_id(table_id); + req.add_index_ids(index_id); + req.add_partition_ids(partition_id); + req.set_need_update_table_version(true); + + mem_kv->get_count_ = 0; + mem_kv->put_count_ = 0; + mem_kv->del_count_ = 0; + + meta_service->drop_partition(&ctrl, &req, &res, nullptr); + + LOG(INFO) << "DropPartition: " << mem_kv->get_count_ << ", " << mem_kv->put_count_ << ", " + << mem_kv->del_count_; + ASSERT_EQ(mem_kv->get_count_, g_bvar_rpc_kv_drop_partition_get_counter.get({mock_instance})); + ASSERT_EQ(mem_kv->put_count_, g_bvar_rpc_kv_drop_partition_put_counter.get({mock_instance})); +} + +// get_obj_store_info +TEST(RpcKvBvarTest, GetObjStoreInfo) { + auto meta_service = get_meta_service(); + auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); + + auto rate_limiter = std::make_shared(); + + std::unique_ptr txn; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + InstanceKeyInfo key_info {"test_instance"}; + std::string key; + instance_key(key_info, &key); + txn->put(key, "val"); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + brpc::Controller cntl; + GetObjStoreInfoResponse res; + GetObjStoreInfoRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + + mem_kv->get_count_ = 0; + mem_kv->put_count_ = 0; + mem_kv->del_count_ = 0; + + meta_service->get_obj_store_info(&cntl, &req, &res, nullptr); + + LOG(INFO) << "GetObjStoreInfo: " << mem_kv->get_count_ << ", " << mem_kv->put_count_ << ", " + << mem_kv->del_count_; + ASSERT_EQ(mem_kv->get_count_, + g_bvar_rpc_kv_get_obj_store_info_get_counter.get({mock_instance})); +} + +// alter_storage_vault +TEST(RpcKvBvarTest, AlterStorageVault) { + auto meta_service = get_meta_service(); + auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); + + constexpr char vault_name[] = "test_alter_s3_vault"; + + InstanceKeyInfo key_info {"test_instance"}; + std::string key; + instance_key(key_info, &key); + std::unique_ptr txn; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put(key, "val"); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + AlterObjStoreInfoRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + req.set_op(AlterObjStoreInfoRequest::ALTER_S3_VAULT); + StorageVaultPB vault; + vault.mutable_obj_info()->set_ak("new_ak"); + vault.set_name(vault_name); + req.mutable_vault()->CopyFrom(vault); + + brpc::Controller cntl; + AlterObjStoreInfoResponse res; + + mem_kv->get_count_ = 0; + mem_kv->put_count_ = 0; + mem_kv->del_count_ = 0; + + meta_service->alter_storage_vault(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + + LOG(INFO) << "AlterStorageVault: " << mem_kv->get_count_ << ", " << mem_kv->put_count_ << ", " + << mem_kv->del_count_; + ASSERT_EQ(mem_kv->get_count_, + g_bvar_rpc_kv_alter_storage_vault_get_counter.get({mock_instance})); + ASSERT_EQ(mem_kv->put_count_, + g_bvar_rpc_kv_alter_storage_vault_put_counter.get({mock_instance})); + ASSERT_EQ(mem_kv->del_count_, + g_bvar_rpc_kv_alter_storage_vault_del_counter.get({mock_instance})); +} + +// alter_obj_store_info +TEST(RpcKvBvarTest, AlterObjStoreInfo) { + auto meta_service = get_meta_service(); + auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); + + auto* sp = SyncPoint::get_instance(); + sp->enable_processing(); + + sp->set_call_back("encrypt_ak_sk:get_encryption_key", [](auto&& args) { + auto* ret = try_any_cast(args[0]); + *ret = 0; + auto* key = try_any_cast(args[1]); + *key = "selectdbselectdbselectdbselectdb"; + auto* key_id = try_any_cast(args[2]); + *key_id = 1; + }); + + std::unique_ptr txn; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + std::string key; + std::string val; + InstanceKeyInfo key_info {"test_instance"}; + instance_key(key_info, &key); + + ObjectStoreInfoPB obj_info; + obj_info.set_id("1"); + obj_info.set_ak("ak"); + obj_info.set_sk("sk"); + InstanceInfoPB instance; + instance.add_obj_info()->CopyFrom(obj_info); + val = instance.SerializeAsString(); + txn->put(key, val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + std::string plain_sk = "Hx60p12123af234541nsVsffdfsdfghsdfhsdf34t"; + + AlterObjStoreInfoRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + req.set_op(AlterObjStoreInfoRequest::LEGACY_UPDATE_AK_SK); + req.mutable_obj()->set_id("1"); + req.mutable_obj()->set_ak("new_ak"); + req.mutable_obj()->set_sk(plain_sk); + + brpc::Controller cntl; + AlterObjStoreInfoResponse res; + + mem_kv->get_count_ = 0; + mem_kv->put_count_ = 0; + mem_kv->del_count_ = 0; + + meta_service->alter_obj_store_info(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + + LOG(INFO) << "AlterObjStoreInfo: " << mem_kv->get_count_ << ", " << mem_kv->put_count_ << ", " + << mem_kv->del_count_; + ASSERT_EQ(mem_kv->get_count_, + g_bvar_rpc_kv_alter_obj_store_info_get_counter.get({mock_instance})); + ASSERT_EQ(mem_kv->put_count_, + g_bvar_rpc_kv_alter_obj_store_info_put_counter.get({mock_instance})); + SyncPoint::get_instance()->disable_processing(); + SyncPoint::get_instance()->clear_all_call_backs(); +} + +// update_ak_sk +TEST(RpcKvBvarTest, UpdateAkSk) { + auto meta_service = get_meta_service(); + auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); + + auto* sp = SyncPoint::get_instance(); + sp->enable_processing(); + sp->set_call_back("encrypt_ak_sk:get_encryption_key", [](auto&& args) { + auto* ret = try_any_cast(args[0]); + *ret = 0; + auto* key = try_any_cast(args[1]); + *key = "selectdbselectdbselectdbselectdb"; + auto* key_id = try_any_cast(args[2]); + *key_id = 1; + }); + + std::string cipher_sk = "JUkuTDctR+ckJtnPkLScWaQZRcOtWBhsLLpnCRxQLxr734qB8cs6gNLH6grE1FxO"; + std::string plain_sk = "Hx60p12123af234541nsVsffdfsdfghsdfhsdf34t"; + + std::unique_ptr txn; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + std::string key; + std::string val; + InstanceKeyInfo key_info {"test_instance"}; + instance_key(key_info, &key); + + ObjectStoreInfoPB obj_info; + + obj_info.set_user_id("111"); + + obj_info.set_ak("ak"); + obj_info.set_sk("sk"); + InstanceInfoPB instance; + instance.add_obj_info()->CopyFrom(obj_info); + val = instance.SerializeAsString(); + txn->put(key, val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + UpdateAkSkRequest req; + req.set_instance_id("test_instance"); + RamUserPB ram_user; + ram_user.set_user_id("111"); + + ram_user.set_ak("new_ak"); + ram_user.set_sk(plain_sk); + req.add_internal_bucket_user()->CopyFrom(ram_user); + + brpc::Controller cntl; + UpdateAkSkResponse res; + + mem_kv->get_count_ = 0; + mem_kv->put_count_ = 0; + mem_kv->del_count_ = 0; + + meta_service->update_ak_sk(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, + &res, nullptr); + + LOG(INFO) << "UpdateAkSk: " << mem_kv->get_count_ << ", " << mem_kv->put_count_ << ", " + << mem_kv->del_count_; + ASSERT_EQ(mem_kv->get_count_, g_bvar_rpc_kv_update_ak_sk_get_counter.get({mock_instance})); + ASSERT_EQ(mem_kv->put_count_, g_bvar_rpc_kv_update_ak_sk_put_counter.get({mock_instance})); + + SyncPoint::get_instance()->disable_processing(); + SyncPoint::get_instance()->clear_all_call_backs(); +} + +// create_instance +TEST(RpcKvBvarTest, CreateInstance) { + auto meta_service = get_meta_service(); + auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); + + brpc::Controller cntl; + CreateInstanceRequest req; + req.set_instance_id("test_instance"); + req.set_user_id("test_user"); + req.set_name("test_name"); + ObjectStoreInfoPB obj; + obj.set_ak("123"); + obj.set_sk("321"); + obj.set_bucket("456"); + obj.set_prefix("654"); + obj.set_endpoint("789"); + obj.set_region("987"); + obj.set_external_endpoint("888"); + obj.set_provider(ObjectStoreInfoPB::BOS); + req.mutable_obj_info()->CopyFrom(obj); + + auto* sp = SyncPoint::get_instance(); + sp->set_call_back("encrypt_ak_sk:get_encryption_key", [](auto&& args) { + auto* ret = try_any_cast(args[0]); + *ret = 0; + auto* key = try_any_cast(args[1]); + *key = "test"; + auto* key_id = try_any_cast(args[2]); + *key_id = 1; + }); + sp->enable_processing(); + CreateInstanceResponse res; + + mem_kv->get_count_ = 0; + mem_kv->put_count_ = 0; + mem_kv->del_count_ = 0; + + meta_service->create_instance(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, + &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + + LOG(INFO) << "CreateInstance: " << mem_kv->get_count_ << ", " << mem_kv->put_count_ << ", " + << mem_kv->del_count_; + ASSERT_EQ(mem_kv->get_count_, g_bvar_rpc_kv_create_instance_get_counter.get({mock_instance})); + + sp->clear_all_call_backs(); + sp->clear_trace(); + sp->disable_processing(); +} + +// get_instance +TEST(RpcKvBvarTest, GetInstance) { + auto meta_service = get_meta_service(); + auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); + + auto* sp = SyncPoint::get_instance(); + sp->set_call_back("encrypt_ak_sk:get_encryption_key", [](auto&& args) { + auto* ret = try_any_cast(args[0]); + *ret = 0; + auto* key = try_any_cast(args[1]); + *key = "test"; + auto* key_id = try_any_cast(args[2]); + *key_id = 1; + }); + sp->enable_processing(); + brpc::Controller cntl; + { + CreateInstanceRequest req; + req.set_instance_id("test_instance"); + req.set_user_id("test_user"); + req.set_name("test_name"); + ObjectStoreInfoPB obj; + obj.set_ak("123"); + obj.set_sk("321"); + obj.set_bucket("456"); + obj.set_prefix("654"); + obj.set_endpoint("789"); + obj.set_region("987"); + obj.set_external_endpoint("888"); + obj.set_provider(ObjectStoreInfoPB::BOS); + req.mutable_obj_info()->CopyFrom(obj); + + CreateInstanceResponse res; + + meta_service->create_instance(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + } + GetInstanceRequest req; + GetInstanceResponse res; + + mem_kv->get_count_ = 0; + mem_kv->put_count_ = 0; + mem_kv->del_count_ = 0; + + req.set_cloud_unique_id("1:test_instance:m-n3qdpyal27rh8iprxx"); + meta_service->get_instance(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, + &res, nullptr); + + LOG(INFO) << "GetInstance: " << mem_kv->get_count_ << ", " << mem_kv->put_count_ << ", " + << mem_kv->del_count_; + ASSERT_EQ(mem_kv->get_count_, g_bvar_rpc_kv_get_instance_get_counter.get({mock_instance})); + sp->clear_all_call_backs(); + sp->clear_trace(); + sp->disable_processing(); +} + +// alter_cluster +// alter cluster have not do kv op +// TEST(RpcKvBvarTest, AlterCluster) { +// auto meta_service = get_meta_service(); +// auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); + +// brpc::Controller cntl; +// AlterClusterRequest req; +// req.set_instance_id(mock_instance); +// req.mutable_cluster()->set_cluster_name(mock_cluster_name); +// req.set_op(AlterClusterRequest::ADD_CLUSTER); +// AlterClusterResponse res; + +// mem_kv->get_count_ = 0; +// mem_kv->put_count_ = 0; +// mem_kv->del_count_ = 0; + +// meta_service->alter_cluster(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, +// &res, nullptr); + +// LOG(INFO) << "AlterCluster: " << mem_kv->get_count_ << ", " << mem_kv->put_count_ << ", " +// << mem_kv->del_count_; +// ASSERT_EQ(mem_kv->get_count_, g_bvar_rpc_kv_alter_cluster_get_counter.get({mock_instance})); +// } + +// get_cluster +TEST(RpcKvBvarTest, GetCluster) { + auto meta_service = get_meta_service(); + auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); + InstanceKeyInfo key_info {mock_instance}; + std::string key; + std::string val; + instance_key(key_info, &key); + + InstanceInfoPB instance; + instance.set_instance_id(mock_instance); + ClusterPB c1; + c1.set_cluster_name(mock_cluster_name); + c1.set_cluster_id(mock_cluster_id); + c1.add_mysql_user_name()->append("m1"); + instance.add_clusters()->CopyFrom(c1); + val = instance.SerializeAsString(); + + std::unique_ptr txn; + std::string get_val; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put(key, val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + brpc::Controller cntl; + GetClusterRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + req.set_cluster_id(mock_cluster_id); + req.set_cluster_name("test_cluster"); + GetClusterResponse res; + + mem_kv->get_count_ = 0; + mem_kv->put_count_ = 0; + mem_kv->del_count_ = 0; + + meta_service->get_cluster(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, + &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + + LOG(INFO) << "GetCluster: " << mem_kv->get_count_ << ", " << mem_kv->put_count_ << ", " + << mem_kv->del_count_; + ASSERT_EQ(mem_kv->get_count_, g_bvar_rpc_kv_get_cluster_get_counter.get({mock_instance})); + ASSERT_EQ(mem_kv->put_count_, g_bvar_rpc_kv_get_cluster_put_counter.get({mock_instance})); +} + +// create_stage +TEST(RpcKvBvarTest, CreateStage) { + auto meta_service = get_meta_service(); + auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); + + brpc::Controller cntl; + const auto* cloud_unique_id = "test_cloud_unique_id"; + std::string instance_id = "stage_test_instance_id"; + [[maybe_unused]] auto* sp = SyncPoint::get_instance(); + DORIS_CLOUD_DEFER { + SyncPoint::get_instance()->clear_all_call_backs(); + }; + sp->set_call_back("get_instance_id", [&](auto&& args) { + auto* ret = try_any_cast_ret(args); + ret->first = instance_id; + ret->second = true; + }); + sp->set_call_back("encrypt_ak_sk:get_encryption_key", [](auto&& args) { + auto* ret = try_any_cast(args[0]); + *ret = 0; + auto* key = try_any_cast(args[1]); + *key = "test"; + auto* key_id = try_any_cast(args[2]); + *key_id = 1; + }); + sp->set_call_back("decrypt_ak_sk:get_encryption_key", [](auto&& args) { + auto* key = try_any_cast(args[0]); + *key = "test"; + auto* ret = try_any_cast(args[1]); + *ret = 0; + }); + sp->enable_processing(); + + ObjectStoreInfoPB obj; + obj.set_ak("123"); + obj.set_sk("321"); + obj.set_bucket("456"); + obj.set_prefix("654"); + obj.set_endpoint("789"); + obj.set_region("987"); + obj.set_external_endpoint("888"); + obj.set_provider(ObjectStoreInfoPB::BOS); + + RamUserPB ram_user; + ram_user.set_user_id("test_user_id"); + ram_user.set_ak("test_ak"); + ram_user.set_sk("test_sk"); + EncryptionInfoPB encry_info; + encry_info.set_encryption_method("encry_method_test"); + encry_info.set_key_id(1111); + ram_user.mutable_encryption_info()->CopyFrom(encry_info); + + // create instance + { + CreateInstanceRequest req; + req.set_instance_id(instance_id); + req.set_user_id("test_user"); + req.set_name("test_name"); + req.mutable_ram_user()->CopyFrom(ram_user); + req.mutable_obj_info()->CopyFrom(obj); + + CreateInstanceResponse res; + meta_service->create_instance(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } + + // create an internal stage + CreateStageRequest create_stage_request; + StagePB stage; + stage.set_type(StagePB::INTERNAL); + stage.add_mysql_user_name("root"); + stage.add_mysql_user_id("root_id"); + stage.set_stage_id("internal_stage_id"); + create_stage_request.set_cloud_unique_id(cloud_unique_id); + create_stage_request.mutable_stage()->CopyFrom(stage); + CreateStageResponse create_stage_response; + mem_kv->get_count_ = 0; + mem_kv->put_count_ = 0; + mem_kv->del_count_ = 0; + + meta_service->create_stage(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &create_stage_request, &create_stage_response, nullptr); + ASSERT_EQ(create_stage_response.status().code(), MetaServiceCode::OK); + + LOG(INFO) << "CreateStage: " << mem_kv->get_count_ << ", " << mem_kv->put_count_ << ", " + << mem_kv->del_count_; + ASSERT_EQ(mem_kv->get_count_, g_bvar_rpc_kv_create_stage_get_counter.get({instance_id})); + ASSERT_EQ(mem_kv->put_count_, g_bvar_rpc_kv_create_stage_put_counter.get({instance_id})); + sp->clear_all_call_backs(); + sp->clear_trace(); + sp->disable_processing(); +} + +// get_stage +TEST(RpcKvBvarTest, GetStage) { + auto meta_service = get_meta_service(); + auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); + + brpc::Controller cntl; + const auto* cloud_unique_id = "test_cloud_unique_id"; + std::string instance_id = "stage_test_instance_id"; + [[maybe_unused]] auto* sp = SyncPoint::get_instance(); + DORIS_CLOUD_DEFER { + SyncPoint::get_instance()->clear_all_call_backs(); + }; + sp->set_call_back("get_instance_id", [&](auto&& args) { + auto* ret = try_any_cast_ret(args); + ret->first = instance_id; + ret->second = true; + }); + sp->set_call_back("encrypt_ak_sk:get_encryption_key", [](auto&& args) { + auto* ret = try_any_cast(args[0]); + *ret = 0; + auto* key = try_any_cast(args[1]); + *key = "test"; + auto* key_id = try_any_cast(args[2]); + *key_id = 1; + }); + sp->set_call_back("decrypt_ak_sk:get_encryption_key", [](auto&& args) { + auto* key = try_any_cast(args[0]); + *key = "test"; + auto* ret = try_any_cast(args[1]); + *ret = 0; + }); + sp->enable_processing(); + + ObjectStoreInfoPB obj; + obj.set_ak("123"); + obj.set_sk("321"); + obj.set_bucket("456"); + obj.set_prefix("654"); + obj.set_endpoint("789"); + obj.set_region("987"); + obj.set_external_endpoint("888"); + obj.set_provider(ObjectStoreInfoPB::BOS); + + RamUserPB ram_user; + ram_user.set_user_id("test_user_id"); + ram_user.set_ak("test_ak"); + ram_user.set_sk("test_sk"); + EncryptionInfoPB encry_info; + encry_info.set_encryption_method("encry_method_test"); + encry_info.set_key_id(1111); + ram_user.mutable_encryption_info()->CopyFrom(encry_info); + + // create instance + { + CreateInstanceRequest req; + req.set_instance_id(instance_id); + req.set_user_id("test_user"); + req.set_name("test_name"); + req.mutable_ram_user()->CopyFrom(ram_user); + req.mutable_obj_info()->CopyFrom(obj); + + CreateInstanceResponse res; + meta_service->create_instance(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } + + // create an internal stage + CreateStageRequest create_stage_request; + StagePB stage; + stage.set_type(StagePB::INTERNAL); + stage.add_mysql_user_name("root"); + stage.add_mysql_user_id("root_id"); + stage.set_stage_id("internal_stage_id"); + create_stage_request.set_cloud_unique_id(cloud_unique_id); + create_stage_request.mutable_stage()->CopyFrom(stage); + CreateStageResponse create_stage_response; + meta_service->create_stage(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &create_stage_request, &create_stage_response, nullptr); + ASSERT_EQ(create_stage_response.status().code(), MetaServiceCode::OK); + + GetStageRequest get_stage_req; + get_stage_req.set_type(StagePB::INTERNAL); + get_stage_req.set_cloud_unique_id(cloud_unique_id); + get_stage_req.set_mysql_user_name("root"); + get_stage_req.set_mysql_user_id("root_id"); + + // get existent internal stage + GetStageResponse res2; + + mem_kv->get_count_ = 0; + mem_kv->put_count_ = 0; + mem_kv->del_count_ = 0; + + meta_service->get_stage(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &get_stage_req, &res2, nullptr); + ASSERT_EQ(res2.status().code(), MetaServiceCode::OK); + ASSERT_EQ(1, res2.stage().size()); + + LOG(INFO) << "GetStage: " << mem_kv->get_count_ << ", " << mem_kv->put_count_ << ", " + << mem_kv->del_count_; + ASSERT_EQ(mem_kv->get_count_, g_bvar_rpc_kv_get_stage_get_counter.get({instance_id})); + sp->clear_all_call_backs(); + sp->clear_trace(); + sp->disable_processing(); +} + +// get_iam +TEST(RpcKvBvarTest, GetIam) { + auto meta_service = get_meta_service(); + auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); + + brpc::Controller cntl; + auto cloud_unique_id = "test_cloud_unique_id"; + std::string instance_id = "get_iam_test_instance_id"; + [[maybe_unused]] auto sp = SyncPoint::get_instance(); + DORIS_CLOUD_DEFER { + SyncPoint::get_instance()->clear_all_call_backs(); + }; + sp->set_call_back("get_instance_id", [&](auto&& args) { + auto* ret = try_any_cast_ret(args); + ret->first = instance_id; + ret->second = true; + }); + sp->set_call_back("encrypt_ak_sk:get_encryption_key", [](auto&& args) { + auto* ret = try_any_cast(args[0]); + *ret = 0; + auto* key = try_any_cast(args[1]); + *key = "test"; + auto* key_id = try_any_cast(args[2]); + *key_id = 1; + }); + sp->set_call_back("decrypt_ak_sk:get_encryption_key", [](auto&& args) { + auto* key = try_any_cast(args[0]); + *key = "test"; + auto* ret = try_any_cast(args[1]); + *ret = 0; + }); + sp->enable_processing(); + + config::arn_id = "iam_arn"; + config::arn_ak = "iam_ak"; + config::arn_sk = "iam_sk"; + + // create instance + { + ObjectStoreInfoPB obj; + obj.set_ak("123"); + obj.set_sk("321"); + obj.set_bucket("456"); + obj.set_prefix("654"); + obj.set_endpoint("789"); + obj.set_region("987"); + obj.set_external_endpoint("888"); + obj.set_provider(ObjectStoreInfoPB::BOS); + + RamUserPB ram_user; + ram_user.set_user_id("test_user_id"); + ram_user.set_ak("test_ak"); + ram_user.set_sk("test_sk"); + + CreateInstanceRequest req; + req.set_instance_id(instance_id); + req.set_user_id("test_user"); + req.set_name("test_name"); + req.mutable_ram_user()->CopyFrom(ram_user); + req.mutable_obj_info()->CopyFrom(obj); + + CreateInstanceResponse res; + meta_service->create_instance(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } + + GetIamRequest request; + request.set_cloud_unique_id(cloud_unique_id); + GetIamResponse response; + + mem_kv->get_count_ = 0; + mem_kv->put_count_ = 0; + mem_kv->del_count_ = 0; + + meta_service->get_iam(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &request, + &response, nullptr); + ASSERT_EQ(response.status().code(), MetaServiceCode::OK); + + LOG(INFO) << "GetIam: " << mem_kv->get_count_ << ", " << mem_kv->put_count_ << ", " + << mem_kv->del_count_; + ASSERT_EQ(mem_kv->get_count_, g_bvar_rpc_kv_get_iam_get_counter.get({instance_id})); + sp->clear_all_call_backs(); + sp->clear_trace(); + sp->disable_processing(); +} + +// alter_iam +TEST(RpcKvBvarTest, AlterIam) { + auto meta_service = get_meta_service(); + auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); + + auto* sp = SyncPoint::get_instance(); + sp->enable_processing(); + sp->set_call_back("encrypt_ak_sk:get_encryption_key", [](auto&& args) { + auto* ret = try_any_cast(args[0]); + *ret = 0; + auto* key = try_any_cast(args[1]); + *key = "selectdbselectdbselectdbselectdb"; + auto* key_id = try_any_cast(args[2]); + *key_id = 1; + }); + + std::string cipher_sk = "JUkuTDctR+ckJtnPkLScWaQZRcOtWBhsLLpnCRxQLxr734qB8cs6gNLH6grE1FxO"; + std::string plain_sk = "Hx60p12123af234541nsVsffdfsdfghsdfhsdf34t"; + + AlterIamRequest req; + req.set_account_id("123"); + req.set_ak("ak1"); + req.set_sk(plain_sk); + + brpc::Controller cntl; + AlterIamResponse res; + + mem_kv->get_count_ = 0; + mem_kv->put_count_ = 0; + mem_kv->del_count_ = 0; + + meta_service->alter_iam(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, &res, + nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + + LOG(INFO) << "AlterIam: " << mem_kv->get_count_ << ", " << mem_kv->put_count_ << ", " + << mem_kv->del_count_; + ASSERT_EQ(mem_kv->get_count_, g_bvar_rpc_kv_alter_iam_get_counter.get({"alter_iam_instance"})); + ASSERT_EQ(mem_kv->put_count_, g_bvar_rpc_kv_alter_iam_put_counter.get({"alter_iam_instance"})); +} + +// alter_ram_user +TEST(RpcKvBvarTest, AlterRamUser) { + auto meta_service = get_meta_service(); + auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); + brpc::Controller cntl; + std::string instance_id = "alter_ram_user_instance_id"; + [[maybe_unused]] auto* sp = SyncPoint::get_instance(); + DORIS_CLOUD_DEFER { + SyncPoint::get_instance()->clear_all_call_backs(); + }; + sp->set_call_back("get_instance_id", [&](auto&& args) { + auto* ret = try_any_cast_ret(args); + ret->first = instance_id; + ret->second = true; + }); + sp->set_call_back("encrypt_ak_sk:get_encryption_key", [](auto&& args) { + auto* ret = try_any_cast(args[0]); + *ret = 0; + auto* key = try_any_cast(args[1]); + *key = "test"; + auto* key_id = try_any_cast(args[2]); + *key_id = 1; + }); + sp->set_call_back("decrypt_ak_sk:get_encryption_key", [](auto&& args) { + auto* key = try_any_cast(args[0]); + *key = "test"; + auto* ret = try_any_cast(args[1]); + *ret = 0; + }); + sp->enable_processing(); + + config::arn_id = "iam_arn"; + config::arn_ak = "iam_ak"; + config::arn_sk = "iam_sk"; + + ObjectStoreInfoPB obj; + obj.set_ak("123"); + obj.set_sk("321"); + obj.set_bucket("456"); + obj.set_prefix("654"); + obj.set_endpoint("789"); + obj.set_region("987"); + obj.set_external_endpoint("888"); + obj.set_provider(ObjectStoreInfoPB::BOS); + + // create instance without ram user + CreateInstanceRequest create_instance_req; + create_instance_req.set_instance_id(instance_id); + create_instance_req.set_user_id("test_user"); + create_instance_req.set_name("test_name"); + create_instance_req.mutable_obj_info()->CopyFrom(obj); + CreateInstanceResponse create_instance_res; + meta_service->create_instance(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &create_instance_req, &create_instance_res, nullptr); + ASSERT_EQ(create_instance_res.status().code(), MetaServiceCode::OK); + + // alter ram user + RamUserPB ram_user; + ram_user.set_user_id("test_user_id"); + ram_user.set_ak("test_ak"); + ram_user.set_sk("test_sk"); + AlterRamUserRequest alter_ram_user_request; + alter_ram_user_request.set_instance_id(instance_id); + alter_ram_user_request.mutable_ram_user()->CopyFrom(ram_user); + AlterRamUserResponse alter_ram_user_response; + mem_kv->get_count_ = 0; + mem_kv->put_count_ = 0; + mem_kv->del_count_ = 0; + meta_service->alter_ram_user(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &alter_ram_user_request, &alter_ram_user_response, nullptr); + + LOG(INFO) << "AlterRamUser: " << mem_kv->get_count_ << ", " << mem_kv->put_count_ << ", " + << mem_kv->del_count_; + ASSERT_EQ(mem_kv->get_count_, g_bvar_rpc_kv_alter_ram_user_get_counter.get({instance_id})); + ASSERT_EQ(mem_kv->put_count_, g_bvar_rpc_kv_alter_ram_user_put_counter.get({instance_id})); + sp->clear_all_call_backs(); + sp->clear_trace(); + sp->disable_processing(); +} + +// begin_copy +TEST(RpcKvBvarTest, BeginCopy) { + auto meta_service = get_meta_service(); + auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); + brpc::Controller cntl; + auto cloud_unique_id = "test_cloud_unique_id"; + auto stage_id = "test_stage_id"; + int64_t table_id = 100; + std::string instance_id = "copy_job_test_instance_id"; + + [[maybe_unused]] auto sp = SyncPoint::get_instance(); + DORIS_CLOUD_DEFER { + SyncPoint::get_instance()->clear_all_call_backs(); + }; + 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(); + + // generate a begin copy request + BeginCopyRequest begin_copy_request; + begin_copy_request.set_cloud_unique_id(cloud_unique_id); + begin_copy_request.set_stage_id(stage_id); + begin_copy_request.set_stage_type(StagePB::EXTERNAL); + begin_copy_request.set_table_id(table_id); + begin_copy_request.set_copy_id("test_copy_id"); + begin_copy_request.set_group_id(0); + begin_copy_request.set_start_time_ms(200); + begin_copy_request.set_timeout_time_ms(300); + for (int i = 0; i < 20; ++i) { + ObjectFilePB object_file_pb; + object_file_pb.set_relative_path("obj_" + std::to_string(i)); + object_file_pb.set_etag("obj_" + std::to_string(i) + "_etag"); + begin_copy_request.add_object_files()->CopyFrom(object_file_pb); + } + BeginCopyResponse res; + + mem_kv->get_count_ = 0; + mem_kv->put_count_ = 0; + mem_kv->del_count_ = 0; + + meta_service->begin_copy(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &begin_copy_request, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + + LOG(INFO) << "BeginCopy: " << mem_kv->get_count_ << ", " << mem_kv->put_count_ << ", " + << mem_kv->del_count_; + ASSERT_EQ(mem_kv->get_count_, g_bvar_rpc_kv_begin_copy_get_counter.get({instance_id})); + ASSERT_EQ(mem_kv->put_count_, g_bvar_rpc_kv_begin_copy_put_counter.get({instance_id})); + sp->clear_all_call_backs(); + sp->clear_trace(); + sp->disable_processing(); +} + +// get_copy_job +TEST(RpcKvBvarTest, GetCopyJob) { + auto meta_service = get_meta_service(); + auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); + brpc::Controller cntl; + const char* cloud_unique_id = "test_cloud_unique_id"; + const char* stage_id = "test_stage_id"; + int64_t table_id = 100; + std::string instance_id = "copy_job_test_instance_id"; + + [[maybe_unused]] auto sp = SyncPoint::get_instance(); + DORIS_CLOUD_DEFER { + SyncPoint::get_instance()->clear_all_call_backs(); + }; + 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(); + { + // generate a begin copy request + BeginCopyRequest begin_copy_request; + begin_copy_request.set_cloud_unique_id(cloud_unique_id); + begin_copy_request.set_stage_id(stage_id); + begin_copy_request.set_stage_type(StagePB::EXTERNAL); + begin_copy_request.set_table_id(table_id); + begin_copy_request.set_copy_id("test_copy_id"); + begin_copy_request.set_group_id(0); + begin_copy_request.set_start_time_ms(200); + begin_copy_request.set_timeout_time_ms(300); + for (int i = 0; i < 20; ++i) { + ObjectFilePB object_file_pb; + object_file_pb.set_relative_path("obj_" + std::to_string(i)); + object_file_pb.set_etag("obj_" + std::to_string(i) + "_etag"); + begin_copy_request.add_object_files()->CopyFrom(object_file_pb); + } + BeginCopyResponse res; + meta_service->begin_copy(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &begin_copy_request, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } + + GetCopyJobRequest get_copy_job_request; + get_copy_job_request.set_cloud_unique_id(cloud_unique_id); + get_copy_job_request.set_stage_id(stage_id); + get_copy_job_request.set_table_id(table_id); + get_copy_job_request.set_copy_id("test_copy_id"); + get_copy_job_request.set_group_id(0); + + mem_kv->get_count_ = 0; + mem_kv->put_count_ = 0; + mem_kv->del_count_ = 0; + + GetCopyJobResponse res; + meta_service->get_copy_job(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &get_copy_job_request, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + + LOG(INFO) << "GetCopyJob: " << mem_kv->get_count_ << ", " << mem_kv->put_count_ << ", " + << mem_kv->del_count_; + ASSERT_EQ(mem_kv->get_count_, g_bvar_rpc_kv_get_copy_job_get_counter.get({instance_id})); + sp->clear_all_call_backs(); + sp->clear_trace(); + sp->disable_processing(); +} + +// finish_copy +TEST(RpcKvBvarTest, FinishCopy) { + auto meta_service = get_meta_service(); + auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); + brpc::Controller cntl; + const char* cloud_unique_id = "test_cloud_unique_id"; + const char* stage_id = "test_stage_id"; + int64_t table_id = 100; + std::string instance_id = "copy_job_test_instance_id"; + + [[maybe_unused]] auto sp = SyncPoint::get_instance(); + DORIS_CLOUD_DEFER { + SyncPoint::get_instance()->clear_all_call_backs(); + }; + 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(); + { + // generate a begin copy request + BeginCopyRequest begin_copy_request; + begin_copy_request.set_cloud_unique_id(cloud_unique_id); + begin_copy_request.set_stage_id(stage_id); + begin_copy_request.set_stage_type(StagePB::EXTERNAL); + begin_copy_request.set_table_id(table_id); + begin_copy_request.set_copy_id("test_copy_id"); + begin_copy_request.set_group_id(0); + begin_copy_request.set_start_time_ms(200); + begin_copy_request.set_timeout_time_ms(300); + for (int i = 0; i < 20; ++i) { + ObjectFilePB object_file_pb; + object_file_pb.set_relative_path("obj_" + std::to_string(i)); + object_file_pb.set_etag("obj_" + std::to_string(i) + "_etag"); + begin_copy_request.add_object_files()->CopyFrom(object_file_pb); + } + BeginCopyResponse res; + meta_service->begin_copy(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &begin_copy_request, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } + FinishCopyRequest finish_copy_request; + finish_copy_request.set_cloud_unique_id(cloud_unique_id); + finish_copy_request.set_stage_id(stage_id); + finish_copy_request.set_stage_type(StagePB::EXTERNAL); + finish_copy_request.set_table_id(table_id); + finish_copy_request.set_copy_id("test_copy_id"); + finish_copy_request.set_group_id(0); + finish_copy_request.set_action(FinishCopyRequest::COMMIT); + + mem_kv->get_count_ = 0; + mem_kv->put_count_ = 0; + mem_kv->del_count_ = 0; + + FinishCopyResponse res; + meta_service->finish_copy(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &finish_copy_request, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + + LOG(INFO) << "FinishCopy: " << mem_kv->get_count_ << ", " << mem_kv->put_count_ << ", " + << mem_kv->del_count_; + ASSERT_EQ(mem_kv->get_count_, g_bvar_rpc_kv_finish_copy_get_counter.get({instance_id})); + ASSERT_EQ(mem_kv->put_count_, g_bvar_rpc_kv_finish_copy_put_counter.get({instance_id})); + ASSERT_EQ(mem_kv->del_count_, g_bvar_rpc_kv_finish_copy_del_counter.get({instance_id})); + sp->clear_all_call_backs(); + sp->clear_trace(); + sp->disable_processing(); +} + +// get_copy_files +TEST(RpcKvBvarTest, GetCopyFiles) { + auto meta_service = get_meta_service(); + auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); + brpc::Controller cntl; + auto cloud_unique_id = "test_cloud_unique_id"; + auto stage_id = "test_stage_id"; + int64_t table_id = 100; + std::string instance_id = "copy_job_test_instance_id"; + + [[maybe_unused]] auto sp = SyncPoint::get_instance(); + DORIS_CLOUD_DEFER { + SyncPoint::get_instance()->clear_all_call_backs(); + }; + 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(); + { + // generate a begin copy request + BeginCopyRequest begin_copy_request; + begin_copy_request.set_cloud_unique_id(cloud_unique_id); + begin_copy_request.set_stage_id(stage_id); + begin_copy_request.set_stage_type(StagePB::EXTERNAL); + begin_copy_request.set_table_id(table_id); + begin_copy_request.set_copy_id("test_copy_id"); + begin_copy_request.set_group_id(0); + begin_copy_request.set_start_time_ms(200); + begin_copy_request.set_timeout_time_ms(300); + for (int i = 0; i < 20; ++i) { + ObjectFilePB object_file_pb; + object_file_pb.set_relative_path("obj_" + std::to_string(i)); + object_file_pb.set_etag("obj_" + std::to_string(i) + "_etag"); + begin_copy_request.add_object_files()->CopyFrom(object_file_pb); + } + BeginCopyResponse res; + meta_service->begin_copy(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &begin_copy_request, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } + GetCopyFilesRequest get_copy_file_req; + get_copy_file_req.set_cloud_unique_id(cloud_unique_id); + get_copy_file_req.set_stage_id(stage_id); + get_copy_file_req.set_table_id(table_id); + + mem_kv->get_count_ = 0; + mem_kv->put_count_ = 0; + mem_kv->del_count_ = 0; + + GetCopyFilesResponse res; + meta_service->get_copy_files(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &get_copy_file_req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + + LOG(INFO) << "GetCopyFiles: " << mem_kv->get_count_ << ", " << mem_kv->put_count_ << ", " + << mem_kv->del_count_; + ASSERT_EQ(mem_kv->get_count_, g_bvar_rpc_kv_get_copy_files_get_counter.get({instance_id})); + sp->clear_all_call_backs(); + sp->clear_trace(); + sp->disable_processing(); +} + +// filter_copy_files +TEST(RpcKvBvarTest, FilterCopyFiles) { + auto meta_service = get_meta_service(); + auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); + brpc::Controller cntl; + auto cloud_unique_id = "test_cloud_unique_id"; + auto stage_id = "test_stage_id"; + int64_t table_id = 100; + std::string instance_id = "copy_job_test_instance_id"; + + [[maybe_unused]] auto sp = SyncPoint::get_instance(); + DORIS_CLOUD_DEFER { + SyncPoint::get_instance()->clear_all_call_backs(); + }; + 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(); + { + // generate a begin copy request + BeginCopyRequest begin_copy_request; + begin_copy_request.set_cloud_unique_id(cloud_unique_id); + begin_copy_request.set_stage_id(stage_id); + begin_copy_request.set_stage_type(StagePB::EXTERNAL); + begin_copy_request.set_table_id(table_id); + begin_copy_request.set_copy_id("test_copy_id"); + begin_copy_request.set_group_id(0); + begin_copy_request.set_start_time_ms(200); + begin_copy_request.set_timeout_time_ms(300); + for (int i = 0; i < 20; ++i) { + ObjectFilePB object_file_pb; + object_file_pb.set_relative_path("obj_" + std::to_string(i)); + object_file_pb.set_etag("obj_" + std::to_string(i) + "_etag"); + begin_copy_request.add_object_files()->CopyFrom(object_file_pb); + } + BeginCopyResponse res; + meta_service->begin_copy(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &begin_copy_request, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } + + FilterCopyFilesRequest request; + request.set_cloud_unique_id(cloud_unique_id); + request.set_stage_id(stage_id); + request.set_table_id(table_id); + for (int i = 0; i < 10; ++i) { + ObjectFilePB object_file; + object_file.set_relative_path("file" + std::to_string(i)); + object_file.set_etag("etag" + std::to_string(i)); + request.add_object_files()->CopyFrom(object_file); + } + FilterCopyFilesResponse res; + mem_kv->get_count_ = 0; + mem_kv->put_count_ = 0; + mem_kv->del_count_ = 0; + + meta_service->filter_copy_files(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &request, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + + LOG(INFO) << "FilterCopyFiles: " << mem_kv->get_count_ << ", " << mem_kv->put_count_ << ", " + << mem_kv->del_count_; + ASSERT_EQ(mem_kv->get_count_, g_bvar_rpc_kv_filter_copy_files_get_counter.get({instance_id})); + sp->clear_all_call_backs(); + sp->clear_trace(); + sp->disable_processing(); +} + +// get_cluster_status +TEST(RpcKvBvarTest, GetClusterStatus) { + auto meta_service = get_meta_service(); + auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); + + InstanceKeyInfo key_info {mock_instance}; + std::string key; + std::string val; + instance_key(key_info, &key); + + InstanceInfoPB instance; + instance.set_instance_id(mock_instance); + ClusterPB c1; + c1.set_type(ClusterPB::COMPUTE); + c1.set_cluster_name(mock_cluster_name); + c1.set_cluster_id(mock_cluster_id); + c1.add_mysql_user_name()->append("m1"); + c1.set_cluster_status(ClusterStatus::NORMAL); + ClusterPB c2; + c2.set_type(ClusterPB::COMPUTE); + c2.set_cluster_name(mock_cluster_name + "2"); + c2.set_cluster_id(mock_cluster_id + "2"); + c2.add_mysql_user_name()->append("m2"); + c2.set_cluster_status(ClusterStatus::SUSPENDED); + ClusterPB c3; + c3.set_type(ClusterPB::COMPUTE); + c3.set_cluster_name(mock_cluster_name + "3"); + c3.set_cluster_id(mock_cluster_id + "3"); + c3.add_mysql_user_name()->append("m3"); + c3.set_cluster_status(ClusterStatus::TO_RESUME); + instance.add_clusters()->CopyFrom(c1); + instance.add_clusters()->CopyFrom(c2); + instance.add_clusters()->CopyFrom(c3); + val = instance.SerializeAsString(); + + std::unique_ptr txn; + std::string get_val; + TxnErrorCode err = meta_service->txn_kv()->create_txn(&txn); + ASSERT_EQ(err, TxnErrorCode::TXN_OK); + txn->put(key, val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + brpc::Controller cntl; + GetClusterStatusRequest req; + req.add_instance_ids(mock_instance); + GetClusterStatusResponse res; + + mem_kv->get_count_ = 0; + mem_kv->put_count_ = 0; + mem_kv->del_count_ = 0; + + meta_service->get_cluster_status(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + ASSERT_EQ(res.details().at(0).clusters().size(), 3); + + LOG(INFO) << "GetClusterStatus: " << mem_kv->get_count_ << ", " << mem_kv->put_count_ << ", " + << mem_kv->del_count_; + ASSERT_EQ(mem_kv->get_count_, + g_bvar_rpc_kv_get_cluster_status_get_counter.get({mock_instance})); +} + +// get_current_max_txn_id +TEST(RpcKvBvarTest, GetCurrentMaxTxnId) { + auto meta_service = get_meta_service(); + auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); + const int64_t db_id = 123; + const std::string label = "test_label123"; + const std::string cloud_unique_id = "test_cloud_unique_id"; + + brpc::Controller begin_txn_cntl; + BeginTxnRequest begin_txn_req; + BeginTxnResponse begin_txn_res; + TxnInfoPB txn_info_pb; + + begin_txn_req.set_cloud_unique_id(cloud_unique_id); + txn_info_pb.set_db_id(db_id); + txn_info_pb.set_label(label); + txn_info_pb.add_table_ids(12345); + txn_info_pb.set_timeout_ms(36000); + begin_txn_req.mutable_txn_info()->CopyFrom(txn_info_pb); + + meta_service->begin_txn(reinterpret_cast<::google::protobuf::RpcController*>(&begin_txn_cntl), + &begin_txn_req, &begin_txn_res, nullptr); + ASSERT_EQ(begin_txn_res.status().code(), MetaServiceCode::OK); + + brpc::Controller max_txn_id_cntl; + GetCurrentMaxTxnRequest max_txn_id_req; + GetCurrentMaxTxnResponse max_txn_id_res; + + std::unique_ptr txn; + EXPECT_EQ(mem_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put("schema change", "val"); + EXPECT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + max_txn_id_req.set_cloud_unique_id(cloud_unique_id); + + mem_kv->get_count_ = 0; + mem_kv->put_count_ = 0; + mem_kv->del_count_ = 0; + + meta_service->get_current_max_txn_id( + reinterpret_cast<::google::protobuf::RpcController*>(&max_txn_id_cntl), &max_txn_id_req, + &max_txn_id_res, nullptr); + + LOG(INFO) << "GetCurrentMaxTxnId: " << mem_kv->get_count_ << ", " << mem_kv->put_count_ << ", " + << mem_kv->del_count_; + ASSERT_EQ(mem_kv->get_count_, + g_bvar_rpc_kv_get_current_max_txn_id_get_counter.get({mock_instance})); +} + +// begin_sub_txn +TEST(RpcKvBvarTest, BeginSubTxn) { + auto meta_service = get_meta_service(); + auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); + int64_t db_id = 98131; + int64_t txn_id = -1; + int64_t t1 = 10; + int64_t t1_index = 100; + int64_t t1_p1 = 11; + int64_t t1_p1_t1 = 12; + int64_t t1_p1_t2 = 13; + int64_t t1_p2 = 14; + int64_t t1_p2_t1 = 15; + int64_t t2 = 16; + std::string label = "test_label"; + std::string label2 = "test_label_0"; + // begin txn + { + 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(t1); + 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(); + } + + // mock rowset and tablet: for sub_txn1 + int64_t sub_txn_id1 = txn_id; + create_and_commit_rowset(meta_service.get(), t1, t1_index, t1_p1, t1_p1_t1, sub_txn_id1); + create_and_commit_rowset(meta_service.get(), t1, t1_index, t1_p1, t1_p1_t2, sub_txn_id1); + create_and_commit_rowset(meta_service.get(), t1, t1_index, t1_p2, t1_p2_t1, sub_txn_id1); + + brpc::Controller cntl; + BeginSubTxnRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + req.set_txn_id(txn_id); + req.set_sub_txn_num(0); + req.set_db_id(db_id); + req.set_label(label2); + req.mutable_table_ids()->Add(t1); + req.mutable_table_ids()->Add(t2); + BeginSubTxnResponse res; + + mem_kv->get_count_ = 0; + mem_kv->put_count_ = 0; + mem_kv->del_count_ = 0; + + meta_service->begin_sub_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, + &res, nullptr); + + LOG(INFO) << "BeginSubTxn: " << mem_kv->get_count_ << ", " << mem_kv->put_count_ << ", " + << mem_kv->del_count_; + ASSERT_EQ(mem_kv->get_count_, g_bvar_rpc_kv_begin_sub_txn_get_counter.get({mock_instance})); + ASSERT_EQ(mem_kv->put_count_, g_bvar_rpc_kv_begin_sub_txn_put_counter.get({mock_instance})); + ASSERT_EQ(mem_kv->del_count_, g_bvar_rpc_kv_begin_sub_txn_del_counter.get({mock_instance})); +} + +// abort_sub_txn +TEST(RpcKvBvarTest, AbortSubTxn) { + auto meta_service = get_meta_service(); + auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); + int64_t db_id = 98131; + int64_t txn_id = -1; + int64_t t1 = 10; + int64_t t1_index = 100; + int64_t t1_p1 = 11; + int64_t t1_p1_t1 = 12; + int64_t t1_p1_t2 = 13; + int64_t t1_p2 = 14; + int64_t t1_p2_t1 = 15; + int64_t t2 = 16; + std::string label = "test_label"; + std::string label2 = "test_label_0"; + // begin txn + { + 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(t1); + 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(); + } + + // mock rowset and tablet: for sub_txn1 + int64_t sub_txn_id1 = txn_id; + create_and_commit_rowset(meta_service.get(), t1, t1_index, t1_p1, t1_p1_t1, sub_txn_id1); + create_and_commit_rowset(meta_service.get(), t1, t1_index, t1_p1, t1_p1_t2, sub_txn_id1); + create_and_commit_rowset(meta_service.get(), t1, t1_index, t1_p2, t1_p2_t1, sub_txn_id1); + brpc::Controller cntl; + { + BeginSubTxnRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + req.set_txn_id(txn_id); + req.set_sub_txn_num(0); + req.set_db_id(db_id); + req.set_label(label2); + req.mutable_table_ids()->Add(t1); + req.mutable_table_ids()->Add(t2); + BeginSubTxnResponse res; + + meta_service->begin_sub_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + } + + AbortSubTxnRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + req.set_txn_id(txn_id); + req.set_sub_txn_num(2); + req.set_sub_txn_id(sub_txn_id1); + req.set_db_id(db_id); + req.mutable_table_ids()->Add(t1); + req.mutable_table_ids()->Add(t2); + AbortSubTxnResponse res; + + mem_kv->get_count_ = 0; + mem_kv->put_count_ = 0; + mem_kv->del_count_ = 0; + + meta_service->abort_sub_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, + &res, nullptr); + + LOG(INFO) << "AbortSubTxn: " << mem_kv->get_count_ << ", " << mem_kv->put_count_ << ", " + << mem_kv->del_count_; + ASSERT_EQ(mem_kv->get_count_, g_bvar_rpc_kv_abort_sub_txn_get_counter.get({mock_instance})); + ASSERT_EQ(mem_kv->put_count_, g_bvar_rpc_kv_abort_sub_txn_put_counter.get({mock_instance})); +} + +// abort_txn_with_coordinator +TEST(RpcKvBvarTest, AbortTxnWithCoordinator) { + auto meta_service = get_meta_service(); + auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); + const int64_t db_id = 666; + const int64_t table_id = 777; + const std::string label = "test_label"; + const std::string cloud_unique_id = "test_cloud_unique_id"; + const int64_t coordinator_id = 15623; + int64_t cur_time = std::chrono::duration_cast( + std::chrono::steady_clock::now().time_since_epoch()) + .count(); + std::string host = "127.0.0.1:15586"; + int64_t txn_id = -1; + + brpc::Controller begin_txn_cntl; + BeginTxnRequest begin_txn_req; + BeginTxnResponse begin_txn_res; + TxnInfoPB txn_info_pb; + TxnCoordinatorPB coordinator; + + begin_txn_req.set_cloud_unique_id(cloud_unique_id); + 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); + coordinator.set_id(coordinator_id); + coordinator.set_ip(host); + coordinator.set_sourcetype(::doris::cloud::TxnSourceTypePB::TXN_SOURCE_TYPE_BE); + coordinator.set_start_time(cur_time); + txn_info_pb.mutable_coordinator()->CopyFrom(coordinator); + begin_txn_req.mutable_txn_info()->CopyFrom(txn_info_pb); + + meta_service->begin_txn(reinterpret_cast<::google::protobuf::RpcController*>(&begin_txn_cntl), + &begin_txn_req, &begin_txn_res, nullptr); + ASSERT_EQ(begin_txn_res.status().code(), MetaServiceCode::OK); + txn_id = begin_txn_res.txn_id(); + ASSERT_GT(txn_id, -1); + + brpc::Controller abort_txn_cntl; + AbortTxnWithCoordinatorRequest abort_txn_req; + AbortTxnWithCoordinatorResponse abort_txn_resp; + + abort_txn_req.set_id(coordinator_id); + abort_txn_req.set_ip(host); + abort_txn_req.set_start_time(cur_time + 3600); + + mem_kv->get_count_ = 0; + mem_kv->put_count_ = 0; + mem_kv->del_count_ = 0; + + meta_service->abort_txn_with_coordinator( + reinterpret_cast<::google::protobuf::RpcController*>(&begin_txn_cntl), &abort_txn_req, + &abort_txn_resp, nullptr); + + LOG(INFO) << "AbortTxnWithCoordinator: " << mem_kv->get_count_ << ", " << mem_kv->put_count_ + << ", " << mem_kv->del_count_; + ASSERT_EQ(mem_kv->get_count_, + g_bvar_rpc_kv_abort_txn_with_coordinator_get_counter.get({mock_instance})); +} + +// check_txn_conflict +TEST(RpcKvBvarTest, CheckTxnConflict) { + auto meta_service = get_meta_service(); + auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); + + const int64_t db_id = 666; + const int64_t table_id = 777; + const std::string label = "test_label"; + const std::string cloud_unique_id = "test_cloud_unique_id"; + int64_t txn_id = -1; + + brpc::Controller begin_txn_cntl; + BeginTxnRequest begin_txn_req; + BeginTxnResponse begin_txn_res; + TxnInfoPB txn_info_pb; + + begin_txn_req.set_cloud_unique_id(cloud_unique_id); + 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); + begin_txn_req.mutable_txn_info()->CopyFrom(txn_info_pb); + + meta_service->begin_txn(reinterpret_cast<::google::protobuf::RpcController*>(&begin_txn_cntl), + &begin_txn_req, &begin_txn_res, nullptr); + ASSERT_EQ(begin_txn_res.status().code(), MetaServiceCode::OK); + txn_id = begin_txn_res.txn_id(); + ASSERT_GT(txn_id, -1); + + brpc::Controller check_txn_conflict_cntl; + CheckTxnConflictRequest check_txn_conflict_req; + CheckTxnConflictResponse check_txn_conflict_res; + + check_txn_conflict_req.set_cloud_unique_id(cloud_unique_id); + check_txn_conflict_req.set_db_id(db_id); + check_txn_conflict_req.set_end_txn_id(txn_id + 1); + check_txn_conflict_req.add_table_ids(table_id); + + mem_kv->get_count_ = 0; + mem_kv->put_count_ = 0; + mem_kv->del_count_ = 0; + + meta_service->check_txn_conflict( + reinterpret_cast<::google::protobuf::RpcController*>(&begin_txn_cntl), + &check_txn_conflict_req, &check_txn_conflict_res, nullptr); + + LOG(INFO) << "CheckTxnConflict: " << mem_kv->get_count_ << ", " << mem_kv->put_count_ << ", " + << mem_kv->del_count_; + ASSERT_EQ(mem_kv->get_count_, + g_bvar_rpc_kv_check_txn_conflict_get_counter.get({mock_instance})); +} + +// clean_txn_label +TEST(RpcKvBvarTest, CleanTxnLabel) { + auto meta_service = get_meta_service(); + auto mem_kv = std::dynamic_pointer_cast(meta_service->txn_kv()); + int64_t db_id = 1987211; + const std::string& label = "test_clean_label"; + 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(1234); + 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); + } + CleanTxnLabelRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + req.set_db_id(db_id); + req.add_labels(label); + CleanTxnLabelResponse res; + + mem_kv->get_count_ = 0; + mem_kv->put_count_ = 0; + mem_kv->del_count_ = 0; + + meta_service->clean_txn_label(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, + &res, nullptr); + + LOG(INFO) << "CleanTxnLabel: " << mem_kv->get_count_ << ", " << mem_kv->put_count_ << ", " + << mem_kv->del_count_; + ASSERT_EQ(mem_kv->get_count_, g_bvar_rpc_kv_clean_txn_label_get_counter.get({mock_instance})); + ASSERT_EQ(mem_kv->put_count_, g_bvar_rpc_kv_clean_txn_label_put_counter.get({mock_instance})); + ASSERT_EQ(mem_kv->del_count_, g_bvar_rpc_kv_clean_txn_label_del_counter.get({mock_instance})); +} +} // namespace doris::cloud