diff --git a/be/src/cloud/cloud_base_compaction.cpp b/be/src/cloud/cloud_base_compaction.cpp index 81c1d47b7461e5..23d4850d0002e8 100644 --- a/be/src/cloud/cloud_base_compaction.cpp +++ b/be/src/cloud/cloud_base_compaction.cpp @@ -29,6 +29,7 @@ #include "service/backend_options.h" #include "util/thread.h" #include "util/uuid_generator.h" +#include "vec/runtime/vdatetime_value.h" namespace doris { using namespace ErrorCode; @@ -82,14 +83,18 @@ Status CloudBaseCompaction::prepare_compact() { compaction_job->set_type(cloud::TabletCompactionJobPB::BASE); compaction_job->set_base_compaction_cnt(_base_compaction_cnt); compaction_job->set_cumulative_compaction_cnt(_cumulative_compaction_cnt); + compaction_job->add_input_versions(_input_rowsets.front()->start_version()); + compaction_job->add_input_versions(_input_rowsets.back()->end_version()); using namespace std::chrono; int64_t now = duration_cast(system_clock::now().time_since_epoch()).count(); _expiration = now + config::compaction_timeout_seconds; compaction_job->set_expiration(_expiration); compaction_job->set_lease(now + config::lease_compaction_interval_seconds * 4); cloud::StartTabletJobResponse resp; - //auto st = cloud::meta_mgr()->prepare_tablet_job(job, &resp); auto st = _engine.meta_mgr().prepare_tablet_job(job, &resp); + if (resp.has_alter_version()) { + (static_cast(_tablet.get()))->set_alter_version(resp.alter_version()); + } if (!st.ok()) { if (resp.status().code() == cloud::STALE_TABLET_CACHE) { // set last_sync_time to 0 to force sync tablet next time @@ -97,6 +102,21 @@ Status CloudBaseCompaction::prepare_compact() { } else if (resp.status().code() == cloud::TABLET_NOT_FOUND) { // tablet not found cloud_tablet()->clear_cache(); + } else if (resp.status().code() == cloud::JOB_CHECK_ALTER_VERSION) { + auto* cloud_tablet = (static_cast(_tablet.get())); + std::stringstream ss; + ss << "failed to prepare cumu compaction. Check compaction input versions " + "failed in schema change. The input version end must " + "less than or equal to alter_version." + "current alter version in BE is not correct." + "input_version_start=" + << compaction_job->input_versions(0) + << " input_version_end=" << compaction_job->input_versions(1) + << " current alter_version=" << cloud_tablet->alter_version() + << " schema_change_alter_version=" << resp.alter_version(); + std::string msg = ss.str(); + LOG(WARNING) << msg; + return Status::InternalError(msg); } return st; } @@ -314,6 +334,22 @@ Status CloudBaseCompaction::modify_rowsets() { if (!st.ok()) { if (resp.status().code() == cloud::TABLET_NOT_FOUND) { cloud_tablet()->clear_cache(); + } else if (resp.status().code() == cloud::JOB_CHECK_ALTER_VERSION) { + auto* cloud_tablet = (static_cast(_tablet.get())); + std::stringstream ss; + ss << "failed to prepare cumu compaction. Check compaction input versions " + "failed in schema change. The input version end must " + "less than or equal to alter_version." + "current alter version in BE is not correct." + "input_version_start=" + << compaction_job->input_versions(0) + << " input_version_end=" << compaction_job->input_versions(1) + << " current alter_version=" << cloud_tablet->alter_version() + << " schema_change_alter_version=" << resp.alter_version(); + std::string msg = ss.str(); + LOG(WARNING) << msg; + cloud_tablet->set_alter_version(resp.alter_version()); + return Status::InternalError(msg); } return st; } diff --git a/be/src/cloud/cloud_cumulative_compaction.cpp b/be/src/cloud/cloud_cumulative_compaction.cpp index cc84dce1b58840..ea6062309f28c7 100644 --- a/be/src/cloud/cloud_cumulative_compaction.cpp +++ b/be/src/cloud/cloud_cumulative_compaction.cpp @@ -48,7 +48,9 @@ CloudCumulativeCompaction::CloudCumulativeCompaction(CloudStorageEngine& engine, CloudCumulativeCompaction::~CloudCumulativeCompaction() = default; Status CloudCumulativeCompaction::prepare_compact() { - if (_tablet->tablet_state() != TABLET_RUNNING) { + if (_tablet->tablet_state() != TABLET_RUNNING && + (!config::enable_new_tablet_do_compaction || + static_cast(_tablet.get())->alter_version() == -1)) { return Status::InternalError("invalid tablet state. tablet_id={}", _tablet->tablet_id()); } @@ -110,11 +112,11 @@ Status CloudCumulativeCompaction::prepare_compact() { _expiration = now + config::compaction_timeout_seconds; compaction_job->set_expiration(_expiration); compaction_job->set_lease(now + config::lease_compaction_interval_seconds * 4); - if (config::enable_parallel_cumu_compaction) { - // Set input version range to let meta-service judge version range conflict - compaction_job->add_input_versions(_input_rowsets.front()->start_version()); - compaction_job->add_input_versions(_input_rowsets.back()->end_version()); - } + + compaction_job->add_input_versions(_input_rowsets.front()->start_version()); + compaction_job->add_input_versions(_input_rowsets.back()->end_version()); + // Set input version range to let meta-service check version range conflict + compaction_job->set_check_input_versions_range(config::enable_parallel_cumu_compaction); cloud::StartTabletJobResponse resp; st = _engine.meta_mgr().prepare_tablet_job(job, &resp); if (!st.ok()) { @@ -141,6 +143,18 @@ Status CloudCumulativeCompaction::prepare_compact() { .tag("msg", resp.status().msg()); return Status::Error("no suitable versions"); } + } else if (resp.status().code() == cloud::JOB_CHECK_ALTER_VERSION) { + (static_cast(_tablet.get()))->set_alter_version(resp.alter_version()); + std::stringstream ss; + ss << "failed to prepare cumu compaction. Check compaction input versions " + "failed in schema change. " + "input_version_start=" + << compaction_job->input_versions(0) + << " input_version_end=" << compaction_job->input_versions(1) + << " schema_change_alter_version=" << resp.alter_version(); + std::string msg = ss.str(); + LOG(WARNING) << msg; + return Status::InternalError(msg); } return st; } @@ -256,12 +270,27 @@ Status CloudCumulativeCompaction::modify_rowsets() { cloud::FinishTabletJobResponse resp; auto st = _engine.meta_mgr().commit_tablet_job(job, &resp); + if (resp.has_alter_version()) { + (static_cast(_tablet.get()))->set_alter_version(resp.alter_version()); + } if (!st.ok()) { if (resp.status().code() == cloud::TABLET_NOT_FOUND) { cloud_tablet()->clear_cache(); + } else if (resp.status().code() == cloud::JOB_CHECK_ALTER_VERSION) { + std::stringstream ss; + ss << "failed to prepare cumu compaction. Check compaction input versions " + "failed in schema change. " + "input_version_start=" + << compaction_job->input_versions(0) + << " input_version_end=" << compaction_job->input_versions(1) + << " schema_change_alter_version=" << resp.alter_version(); + std::string msg = ss.str(); + LOG(WARNING) << msg; + return Status::InternalError(msg); } return st; } + auto& stats = resp.stats(); LOG(INFO) << "tablet stats=" << stats.ShortDebugString(); { @@ -344,8 +373,9 @@ Status CloudCumulativeCompaction::pick_rowsets_to_compact() { std::shared_lock rlock(_tablet->get_header_lock()); _base_compaction_cnt = cloud_tablet()->base_compaction_cnt(); _cumulative_compaction_cnt = cloud_tablet()->cumulative_compaction_cnt(); - int64_t candidate_version = - std::max(cloud_tablet()->cumulative_layer_point(), _max_conflict_version + 1); + int64_t candidate_version = std::max( + std::max(cloud_tablet()->cumulative_layer_point(), _max_conflict_version + 1), + cloud_tablet()->alter_version() + 1); // Get all rowsets whose version >= `candidate_version` as candidate rowsets cloud_tablet()->traverse_rowsets( [&candidate_rowsets, candidate_version](const RowsetSharedPtr& rs) { diff --git a/be/src/cloud/cloud_engine_calc_delete_bitmap_task.cpp b/be/src/cloud/cloud_engine_calc_delete_bitmap_task.cpp index 22f6689ff23782..b8acdb6bd38b9e 100644 --- a/be/src/cloud/cloud_engine_calc_delete_bitmap_task.cpp +++ b/be/src/cloud/cloud_engine_calc_delete_bitmap_task.cpp @@ -154,20 +154,22 @@ Status CloudTabletCalcDeleteBitmapTask::handle() const { }; if (_version != max_version + 1 || should_sync_rowsets_produced_by_compaction()) { auto sync_st = tablet->sync_rowsets(); - if (sync_st.is()) [[unlikely]] { - _engine_calc_delete_bitmap_task->add_succ_tablet_id(_tablet_id); - LOG(INFO) << "tablet is under alter process, delete bitmap will be calculated later, " - "tablet_id: " - << _tablet_id << " txn_id: " << _transaction_id - << ", request_version=" << _version; - return sync_st; - } if (!sync_st.ok()) { LOG(WARNING) << "failed to sync rowsets. tablet_id=" << _tablet_id << ", txn_id=" << _transaction_id << ", status=" << sync_st; _engine_calc_delete_bitmap_task->add_error_tablet_id(_tablet_id, sync_st); return sync_st; } + if (tablet->tablet_state() != TABLET_RUNNING) [[unlikely]] { + _engine_calc_delete_bitmap_task->add_succ_tablet_id(_tablet_id); + LOG(INFO) << "tablet is under alter process, delete bitmap will be calculated later, " + "tablet_id: " + << _tablet_id << " txn_id: " << _transaction_id + << ", request_version=" << _version; + return Status::Error( + "invalid tablet state {}. tablet_id={}", tablet->tablet_state(), + tablet->tablet_id()); + } } auto sync_rowset_time_us = MonotonicMicros() - t2; max_version = tablet->max_version_unlocked(); diff --git a/be/src/cloud/cloud_meta_mgr.cpp b/be/src/cloud/cloud_meta_mgr.cpp index e743ea9b12c8ce..d59a8b98764ba5 100644 --- a/be/src/cloud/cloud_meta_mgr.cpp +++ b/be/src/cloud/cloud_meta_mgr.cpp @@ -448,7 +448,10 @@ Status CloudMetaMgr::sync_tablet_rowsets(CloudTablet* tablet, bool warmup_delta_ int64_t now = duration_cast(system_clock::now().time_since_epoch()).count(); tablet->last_sync_time_s = now; - if (tablet->enable_unique_key_merge_on_write()) { + // If is mow, the tablet has no delete bitmap in base rowsets. + // So dont need to sync it. + if (tablet->enable_unique_key_merge_on_write() && + tablet->tablet_state() == TABLET_RUNNING) { DeleteBitmap delete_bitmap(tablet_id); int64_t old_max_version = req.start_version() - 1; auto st = sync_tablet_delete_bitmap(tablet, old_max_version, resp.rowset_meta(), diff --git a/be/src/cloud/cloud_schema_change_job.cpp b/be/src/cloud/cloud_schema_change_job.cpp index ed3e5f9433fcfd..9a9ce5ae8f0c20 100644 --- a/be/src/cloud/cloud_schema_change_job.cpp +++ b/be/src/cloud/cloud_schema_change_job.cpp @@ -19,7 +19,9 @@ #include +#include #include +#include #include "cloud/cloud_meta_mgr.h" #include "cloud/cloud_tablet_mgr.h" @@ -34,6 +36,7 @@ #include "olap/tablet_fwd.h" #include "olap/tablet_meta.h" #include "service/backend_options.h" +#include "util/debug_points.h" namespace doris { using namespace ErrorCode; @@ -59,10 +62,6 @@ CloudSchemaChangeJob::CloudSchemaChangeJob(CloudStorageEngine& cloud_storage_eng CloudSchemaChangeJob::~CloudSchemaChangeJob() = default; Status CloudSchemaChangeJob::process_alter_tablet(const TAlterTabletReqV2& request) { - LOG(INFO) << "Begin to alter tablet. base_tablet_id=" << request.base_tablet_id - << ", new_tablet_id=" << request.new_tablet_id - << ", alter_version=" << request.alter_version << ", job_id=" << _job_id; - // new tablet has to exist _new_tablet = DORIS_TRY(_cloud_storage_engine.tablet_mgr().get_tablet(request.new_tablet_id)); if (_new_tablet->tablet_state() == TABLET_RUNNING) { @@ -86,13 +85,54 @@ Status CloudSchemaChangeJob::process_alter_tablet(const TAlterTabletReqV2& reque RETURN_IF_ERROR(_base_tablet->sync_rowsets(request.alter_version)); // ATTN: Only convert rowsets of version larger than 1, MUST let the new tablet cache have rowset [0-1] _output_cumulative_point = _base_tablet->cumulative_layer_point(); - std::vector rs_splits; int64_t base_max_version = _base_tablet->max_version_unlocked(); + cloud::TabletJobInfoPB job; + auto* idx = job.mutable_idx(); + idx->set_tablet_id(_base_tablet->tablet_id()); + idx->set_table_id(_base_tablet->table_id()); + idx->set_index_id(_base_tablet->index_id()); + idx->set_partition_id(_base_tablet->partition_id()); + auto* sc_job = job.mutable_schema_change(); + sc_job->set_id(_job_id); + sc_job->set_initiator(BackendOptions::get_localhost() + ':' + + std::to_string(config::heartbeat_service_port)); + sc_job->set_alter_version(base_max_version); + auto* new_tablet_idx = sc_job->mutable_new_tablet_idx(); + new_tablet_idx->set_tablet_id(_new_tablet->tablet_id()); + new_tablet_idx->set_table_id(_new_tablet->table_id()); + new_tablet_idx->set_index_id(_new_tablet->index_id()); + new_tablet_idx->set_partition_id(_new_tablet->partition_id()); + cloud::StartTabletJobResponse start_resp; + auto st = _cloud_storage_engine.meta_mgr().prepare_tablet_job(job, &start_resp); + if (!st.ok()) { + if (start_resp.status().code() == cloud::JOB_ALREADY_SUCCESS) { + st = _new_tablet->sync_rowsets(); + if (!st.ok()) { + LOG_WARNING("failed to sync new tablet") + .tag("tablet_id", _new_tablet->tablet_id()) + .error(st); + } + return Status::OK(); + } + return st; + } if (request.alter_version > 1) { // [0-1] is a placeholder rowset, no need to convert - RETURN_IF_ERROR(_base_tablet->capture_rs_readers({2, base_max_version}, &rs_splits, false)); + RETURN_IF_ERROR(_base_tablet->capture_rs_readers({2, start_resp.alter_version()}, + &rs_splits, false)); } + Defer defer {[&]() { + _new_tablet->set_alter_version(-1); + _base_tablet->set_alter_version(-1); + }}; + _new_tablet->set_alter_version(start_resp.alter_version()); + _base_tablet->set_alter_version(start_resp.alter_version()); + LOG(INFO) << "Begin to alter tablet. base_tablet_id=" << request.base_tablet_id + << ", new_tablet_id=" << request.new_tablet_id + << ", alter_version=" << start_resp.alter_version() << ", job_id=" << _job_id; + sc_job->set_alter_version(start_resp.alter_version()); + // FIXME(cyx): Should trigger compaction on base_tablet if there are too many rowsets to convert. // Create a new tablet schema, should merge with dropped columns in light weight schema change @@ -110,7 +150,8 @@ Status CloudSchemaChangeJob::process_alter_tablet(const TAlterTabletReqV2& reque delete_predicates.push_back(rs_meta); } } - RETURN_IF_ERROR(delete_handler.init(_base_tablet_schema, delete_predicates, base_max_version)); + RETURN_IF_ERROR(delete_handler.init(_base_tablet_schema, delete_predicates, + start_resp.alter_version())); std::vector return_columns; return_columns.resize(_base_tablet_schema->num_columns()); @@ -127,7 +168,7 @@ Status CloudSchemaChangeJob::process_alter_tablet(const TAlterTabletReqV2& reque reader_context.is_unique = _base_tablet->keys_type() == UNIQUE_KEYS; reader_context.batch_size = ALTER_TABLE_BATCH_SIZE; reader_context.delete_bitmap = &_base_tablet->tablet_meta()->delete_bitmap(); - reader_context.version = Version(0, base_max_version); + reader_context.version = Version(0, start_resp.alter_version()); for (auto& split : rs_splits) { RETURN_IF_ERROR(split.rs_reader->init(&reader_context)); @@ -156,7 +197,7 @@ Status CloudSchemaChangeJob::process_alter_tablet(const TAlterTabletReqV2& reque } sc_params.vault_id = request.storage_vault_id; if (!request.__isset.materialized_view_params) { - return _convert_historical_rowsets(sc_params); + return _convert_historical_rowsets(sc_params, job); } for (auto item : request.materialized_view_params) { AlterMaterializedViewParam mv_param; @@ -176,10 +217,11 @@ Status CloudSchemaChangeJob::process_alter_tablet(const TAlterTabletReqV2& reque std::make_pair(to_lower(item.column_name), mv_param)); } sc_params.enable_unique_key_merge_on_write = _new_tablet->enable_unique_key_merge_on_write(); - return _convert_historical_rowsets(sc_params); + return _convert_historical_rowsets(sc_params, job); } -Status CloudSchemaChangeJob::_convert_historical_rowsets(const SchemaChangeParams& sc_params) { +Status CloudSchemaChangeJob::_convert_historical_rowsets(const SchemaChangeParams& sc_params, + cloud::TabletJobInfoPB& job) { LOG(INFO) << "Begin to convert historical rowsets for new_tablet from base_tablet. base_tablet=" << _base_tablet->tablet_id() << ", new_tablet=" << _new_tablet->tablet_id() << ", job_id=" << _job_id; @@ -210,36 +252,6 @@ Status CloudSchemaChangeJob::_convert_historical_rowsets(const SchemaChangeParam changer, sc_sorting, _cloud_storage_engine.memory_limitation_bytes_per_thread_for_schema_change()); - cloud::TabletJobInfoPB job; - auto* idx = job.mutable_idx(); - idx->set_tablet_id(_base_tablet->tablet_id()); - idx->set_table_id(_base_tablet->table_id()); - idx->set_index_id(_base_tablet->index_id()); - idx->set_partition_id(_base_tablet->partition_id()); - auto* sc_job = job.mutable_schema_change(); - sc_job->set_id(_job_id); - sc_job->set_initiator(BackendOptions::get_localhost() + ':' + - std::to_string(config::heartbeat_service_port)); - auto* new_tablet_idx = sc_job->mutable_new_tablet_idx(); - new_tablet_idx->set_tablet_id(_new_tablet->tablet_id()); - new_tablet_idx->set_table_id(_new_tablet->table_id()); - new_tablet_idx->set_index_id(_new_tablet->index_id()); - new_tablet_idx->set_partition_id(_new_tablet->partition_id()); - cloud::StartTabletJobResponse start_resp; - auto st = _cloud_storage_engine.meta_mgr().prepare_tablet_job(job, &start_resp); - if (!st.ok()) { - if (start_resp.status().code() == cloud::JOB_ALREADY_SUCCESS) { - st = _new_tablet->sync_rowsets(); - if (!st.ok()) { - LOG_WARNING("failed to sync new tablet") - .tag("tablet_id", _new_tablet->tablet_id()) - .error(st); - } - return Status::OK(); - } - return st; - } - // 3. Convert historical data bool already_exist_any_version = false; for (const auto& rs_reader : sc_params.ref_rowset_readers) { @@ -317,10 +329,8 @@ Status CloudSchemaChangeJob::_convert_historical_rowsets(const SchemaChangeParam VLOG_TRACE << "Successfully convert a history version " << rs_reader->version(); } - - if (sc_params.ref_rowset_readers.empty()) { - sc_job->set_alter_version(1); // no rowset to convert implies alter_version == 1 - } else { + auto* sc_job = job.mutable_schema_change(); + if (!sc_params.ref_rowset_readers.empty()) { int64_t num_output_rows = 0; int64_t size_output_rowsets = 0; int64_t num_output_segments = 0; @@ -335,11 +345,11 @@ Status CloudSchemaChangeJob::_convert_historical_rowsets(const SchemaChangeParam sc_job->set_size_output_rowsets(size_output_rowsets); sc_job->set_num_output_segments(num_output_segments); sc_job->set_num_output_rowsets(_output_rowsets.size()); - sc_job->set_alter_version(_output_rowsets.back()->end_version()); } _output_cumulative_point = std::min(_output_cumulative_point, sc_job->alter_version() + 1); sc_job->set_output_cumulative_point(_output_cumulative_point); + DBUG_EXECUTE_IF("CloudSchemaChangeJob.process_alter_tablet.sleep", DBUG_BLOCK); // process delete bitmap if the table is MOW if (_new_tablet->enable_unique_key_merge_on_write()) { int64_t initiator = boost::uuids::hash_value(UUIDGenerator::instance()->next_uuid()) & @@ -354,7 +364,7 @@ Status CloudSchemaChangeJob::_convert_historical_rowsets(const SchemaChangeParam } cloud::FinishTabletJobResponse finish_resp; - st = _cloud_storage_engine.meta_mgr().commit_tablet_job(job, &finish_resp); + auto st = _cloud_storage_engine.meta_mgr().commit_tablet_job(job, &finish_resp); if (!st.ok()) { if (finish_resp.status().code() == cloud::JOB_ALREADY_SUCCESS) { st = _new_tablet->sync_rowsets(); diff --git a/be/src/cloud/cloud_schema_change_job.h b/be/src/cloud/cloud_schema_change_job.h index d587111df717a3..c77aae4857049d 100644 --- a/be/src/cloud/cloud_schema_change_job.h +++ b/be/src/cloud/cloud_schema_change_job.h @@ -37,7 +37,8 @@ class CloudSchemaChangeJob { Status process_alter_tablet(const TAlterTabletReqV2& request); private: - Status _convert_historical_rowsets(const SchemaChangeParams& sc_params); + Status _convert_historical_rowsets(const SchemaChangeParams& sc_params, + cloud::TabletJobInfoPB& job); Status _process_delete_bitmap(int64_t alter_version, int64_t start_calc_delete_bitmap_version, int64_t initiator); diff --git a/be/src/cloud/cloud_storage_engine.cpp b/be/src/cloud/cloud_storage_engine.cpp index de4bbac7b3ef6c..2c3dc87fb81c5e 100644 --- a/be/src/cloud/cloud_storage_engine.cpp +++ b/be/src/cloud/cloud_storage_engine.cpp @@ -550,21 +550,21 @@ std::vector CloudStorageEngine::_generate_cloud_compaction_task std::function filter_out; if (compaction_type == CompactionType::BASE_COMPACTION) { filter_out = [&submitted_base_compactions, &submitted_full_compactions](CloudTablet* t) { - return !!submitted_base_compactions.count(t->tablet_id()) || - !!submitted_full_compactions.count(t->tablet_id()) || + return submitted_base_compactions.contains(t->tablet_id()) || + submitted_full_compactions.contains(t->tablet_id()) || t->tablet_state() != TABLET_RUNNING; }; } else if (config::enable_parallel_cumu_compaction) { filter_out = [&tablet_preparing_cumu_compaction](CloudTablet* t) { - return !!tablet_preparing_cumu_compaction.count(t->tablet_id()) || - t->tablet_state() != TABLET_RUNNING; + return tablet_preparing_cumu_compaction.contains(t->tablet_id()) || + (t->tablet_state() != TABLET_RUNNING && t->alter_version() == -1); }; } else { filter_out = [&tablet_preparing_cumu_compaction, &submitted_cumu_compactions](CloudTablet* t) { - return !!tablet_preparing_cumu_compaction.count(t->tablet_id()) || - !!submitted_cumu_compactions.count(t->tablet_id()) || - t->tablet_state() != TABLET_RUNNING; + return tablet_preparing_cumu_compaction.contains(t->tablet_id()) || + submitted_cumu_compactions.contains(t->tablet_id()) || + (t->tablet_state() != TABLET_RUNNING && t->alter_version() == -1); }; } diff --git a/be/src/cloud/cloud_tablet.cpp b/be/src/cloud/cloud_tablet.cpp index 17ec1fe22b0d85..67fd9dd0a8df5c 100644 --- a/be/src/cloud/cloud_tablet.cpp +++ b/be/src/cloud/cloud_tablet.cpp @@ -135,19 +135,6 @@ Status CloudTablet::sync_rowsets(int64_t query_version, bool warmup_delta_data) return st; } -TabletSchemaSPtr CloudTablet::merged_tablet_schema() const { - std::shared_lock rdlock(_meta_lock); - TabletSchemaSPtr target_schema; - std::vector schemas; - for (const auto& [_, rowset] : _rs_version_map) { - schemas.push_back(rowset->tablet_schema()); - } - // get the max version schema and merge all schema - static_cast( - vectorized::schema_util::get_least_common_schema(schemas, nullptr, target_schema)); - return target_schema; -} - // Sync tablet meta and all rowset meta if not running. // This could happen when BE didn't finish schema change job and another BE committed this schema change job. // It should be a quite rare situation. @@ -177,8 +164,7 @@ Status CloudTablet::sync_if_not_running() { if (tablet_meta->tablet_state() != TABLET_RUNNING) [[unlikely]] { // MoW may go to here when load while schema change - return Status::Error("invalid tablet state {}. tablet_id={}", - tablet_meta->tablet_state(), tablet_id()); + return Status::OK(); } TimestampedVersionTracker empty_tracker; @@ -200,6 +186,19 @@ Status CloudTablet::sync_if_not_running() { return st; } +TabletSchemaSPtr CloudTablet::merged_tablet_schema() const { + std::shared_lock rdlock(_meta_lock); + TabletSchemaSPtr target_schema; + std::vector schemas; + for (const auto& [_, rowset] : _rs_version_map) { + schemas.push_back(rowset->tablet_schema()); + } + // get the max version schema and merge all schema + static_cast( + vectorized::schema_util::get_least_common_schema(schemas, nullptr, target_schema)); + return target_schema; +} + void CloudTablet::add_rowsets(std::vector to_add, bool version_overlap, std::unique_lock& meta_lock, bool warmup_delta_data) { @@ -592,7 +591,8 @@ std::vector CloudTablet::pick_candidate_rowsets_to_base_compact { std::shared_lock rlock(_meta_lock); for (const auto& [version, rs] : _rs_version_map) { - if (version.first != 0 && version.first < _cumulative_point) { + if (version.first != 0 && version.first < _cumulative_point && + (_alter_version == -1 || version.second <= _alter_version)) { candidate_rowsets.push_back(rs); } } diff --git a/be/src/cloud/cloud_tablet.h b/be/src/cloud/cloud_tablet.h index 10ff1835e6c830..2bd1ce475028ab 100644 --- a/be/src/cloud/cloud_tablet.h +++ b/be/src/cloud/cloud_tablet.h @@ -145,6 +145,9 @@ class CloudTablet final : public BaseTablet { _last_base_compaction_schedule_millis = millis; } + int64_t alter_version() const { return _alter_version; } + void set_alter_version(int64_t alter_version) { _alter_version = alter_version; } + std::vector pick_candidate_rowsets_to_base_compaction(); inline Version max_version() const { @@ -238,6 +241,7 @@ class CloudTablet final : public BaseTablet { int64_t _cumulative_compaction_cnt = 0; int64_t _max_version = -1; int64_t _base_size = 0; + int64_t _alter_version = -1; std::mutex _base_compaction_lock; std::mutex _cumulative_compaction_lock; diff --git a/be/src/cloud/config.cpp b/be/src/cloud/config.cpp index 80522759b84b44..cf45df4dbc757e 100644 --- a/be/src/cloud/config.cpp +++ b/be/src/cloud/config.cpp @@ -59,4 +59,6 @@ DEFINE_mBool(save_load_error_log_to_s3, "false"); DEFINE_mInt32(sync_load_for_tablets_thread, "32"); +DEFINE_mBool(enable_new_tablet_do_compaction, "false"); + } // namespace doris::config diff --git a/be/src/cloud/config.h b/be/src/cloud/config.h index bf041ba0fa6fc5..4a5b3e0e16a208 100644 --- a/be/src/cloud/config.h +++ b/be/src/cloud/config.h @@ -65,6 +65,7 @@ DECLARE_mInt32(tablet_sync_interval_s); // Cloud compaction config DECLARE_mInt64(min_compaction_failure_interval_ms); +DECLARE_mBool(enable_new_tablet_do_compaction); // For cloud read/write separate mode DECLARE_mInt64(base_compaction_freeze_interval_s); DECLARE_mInt64(cu_compaction_freeze_interval_s); diff --git a/be/src/olap/base_tablet.cpp b/be/src/olap/base_tablet.cpp index 141e302af8c420..efc6db6debf3b2 100644 --- a/be/src/olap/base_tablet.cpp +++ b/be/src/olap/base_tablet.cpp @@ -1471,7 +1471,7 @@ Status BaseTablet::update_delete_bitmap_without_lock( << ", rnd:" << rnd << ", percent: " << percent; } }); - int64_t cur_version = rowset->end_version(); + int64_t cur_version = rowset->start_version(); std::vector segments; RETURN_IF_ERROR(std::dynamic_pointer_cast(rowset)->load_segments(&segments)); diff --git a/cloud/src/meta-service/meta_service_job.cpp b/cloud/src/meta-service/meta_service_job.cpp index 1886d4bdf53537..b2b9ec2531b3fb 100644 --- a/cloud/src/meta-service/meta_service_job.cpp +++ b/cloud/src/meta-service/meta_service_job.cpp @@ -49,6 +49,32 @@ namespace doris::cloud { static constexpr int COMPACTION_DELETE_BITMAP_LOCK_ID = -1; static constexpr int SCHEMA_CHANGE_DELETE_BITMAP_LOCK_ID = -2; +// check compaction input_versions are valid during schema change. +// If the schema change job doesnt have alter version, it dont need to check +// because the schema change job is come from old version BE. +// we will check they in prepare compaction and commit compaction. +// 1. When if base compaction, we need to guarantee the end version +// is less than or equal to alter_version. +// 2. When if cu compaction, we need to guarantee the start version +// is large than alter_version. +bool check_compaction_input_verions(const TabletCompactionJobPB& compaction, + const TabletJobInfoPB& job_pb) { + if (!job_pb.has_schema_change() || !job_pb.schema_change().has_alter_version()) return true; + if (compaction.input_versions_size() != 2 || + compaction.input_versions(0) > compaction.input_versions(1)) { + LOG(WARNING) << "The compaction need to know [start_version, end_version], and \ + the start_version should LE end_version. \n" + << proto_to_json(compaction); + return false; + } + + int64_t alter_version = job_pb.schema_change().alter_version(); + return (compaction.type() == TabletCompactionJobPB_CompactionType_BASE && + compaction.input_versions(1) <= alter_version) || + (compaction.type() == TabletCompactionJobPB_CompactionType_CUMULATIVE && + compaction.input_versions(0) > alter_version); +} + void start_compaction_job(MetaServiceCode& code, std::string& msg, std::stringstream& ss, std::unique_ptr& txn, const StartTabletJobRequest* request, StartTabletJobResponse* response, std::string& instance_id, @@ -124,6 +150,17 @@ void start_compaction_job(MetaServiceCode& code, std::string& msg, std::stringst } while (err == TxnErrorCode::TXN_OK) { job_pb.ParseFromString(job_val); + if (!check_compaction_input_verions(compaction, job_pb)) { + SS << "Check compaction input versions failed in schema change. input_version_start=" + << compaction.input_versions(0) + << " input_version_end=" << compaction.input_versions(1) + << " schema_change_alter_version=" << job_pb.schema_change().alter_version(); + msg = ss.str(); + INSTANCE_LOG(INFO) << msg; + code = MetaServiceCode::JOB_CHECK_ALTER_VERSION; + response->set_alter_version(job_pb.schema_change().alter_version()); + return; + } if (job_pb.compaction().empty()) { break; } @@ -153,7 +190,10 @@ void start_compaction_job(MetaServiceCode& code, std::string& msg, std::stringst // for MOW table, so priority should be given to performing full // compaction operations and canceling other types of compaction. compactions.Clear(); - } else if (compaction.input_versions().empty()) { + } else if ((!compaction.has_check_input_versions_range() && + compaction.input_versions().empty()) || + (compaction.has_check_input_versions_range() && + !compaction.check_input_versions_range())) { // Unknown input version range, doesn't support parallel compaction of same type for (auto& c : compactions) { if (c.type() != compaction.type() && c.type() != TabletCompactionJobPB::FULL) @@ -214,8 +254,8 @@ void start_compaction_job(MetaServiceCode& code, std::string& msg, std::stringst void start_schema_change_job(MetaServiceCode& code, std::string& msg, std::stringstream& ss, std::unique_ptr& txn, - const StartTabletJobRequest* request, std::string& instance_id, - bool& need_commit) { + const StartTabletJobRequest* request, StartTabletJobResponse* response, + std::string& instance_id, bool& need_commit) { auto& schema_change = request->job().schema_change(); if (!schema_change.has_id() || schema_change.id().empty()) { code = MetaServiceCode::INVALID_ARGUMENT; @@ -302,6 +342,13 @@ void start_schema_change_job(MetaServiceCode& code, std::string& msg, std::strin msg = "pb deserialization failed"; return; } + if (job_pb.has_schema_change() && job_pb.schema_change().has_alter_version() && + job_pb.schema_change().id() == schema_change.id() && + job_pb.schema_change().initiator() == schema_change.initiator()) { + TEST_SYNC_POINT_CALLBACK("restart_compaction_job"); + response->set_alter_version(job_pb.schema_change().alter_version()); + return; + } job_pb.mutable_idx()->CopyFrom(request->job().idx()); // FE can ensure that a tablet does not have more than one schema_change job at the same time, // so we can directly preempt previous schema_change job. @@ -314,6 +361,11 @@ void start_schema_change_job(MetaServiceCode& code, std::string& msg, std::strin } INSTANCE_LOG(INFO) << "schema_change job to save job=" << proto_to_json(schema_change); txn->put(job_key, job_val); + auto new_tablet_job_key = + job_tablet_key({instance_id, new_tablet_idx.table_id(), new_tablet_idx.index_id(), + new_tablet_idx.partition_id(), new_tablet_id}); + txn->put(new_tablet_job_key, job_val); + response->set_alter_version(job_pb.schema_change().alter_version()); need_commit = true; } @@ -386,7 +438,7 @@ void MetaServiceImpl::start_tablet_job(::google::protobuf::RpcController* contro } if (request->job().has_schema_change()) { - start_schema_change_job(code, msg, ss, txn, request, instance_id, need_commit); + start_schema_change_job(code, msg, ss, txn, request, response, instance_id, need_commit); return; } } @@ -558,30 +610,23 @@ void process_compaction_job(MetaServiceCode& code, std::string& msg, std::string return; } - //========================================================================== - // Lease - //========================================================================== - if (request->action() == FinishTabletJobRequest::LEASE) { - if (compaction.lease() <= 0 || recorded_compaction->lease() > compaction.lease()) { - ss << "invalid lease. recoreded_lease=" << recorded_compaction->lease() - << " req_lease=" << compaction.lease(); - msg = ss.str(); - code = MetaServiceCode::INVALID_ARGUMENT; - return; - } - recorded_compaction->set_lease(compaction.lease()); - auto job_val = recorded_job.SerializeAsString(); - txn->put(job_key, job_val); - INSTANCE_LOG(INFO) << "lease tablet compaction job, tablet_id=" << tablet_id - << " key=" << hex(job_key); - need_commit = true; - return; + bool abort_compaction = false; + if (request->action() == FinishTabletJobRequest::COMMIT && + !check_compaction_input_verions(compaction, recorded_job)) { + SS << "Check compaction input versions failed in schema change. input_version_start=" + << compaction.input_versions(0) << " input_version_end=" << compaction.input_versions(1) + << " schema_change_alter_version=" << recorded_job.schema_change().alter_version(); + msg = ss.str(); + INSTANCE_LOG(INFO) << msg; + abort_compaction = true; + response->set_alter_version(recorded_job.schema_change().alter_version()); + code = MetaServiceCode::JOB_CHECK_ALTER_VERSION; } //========================================================================== // Abort //========================================================================== - if (request->action() == FinishTabletJobRequest::ABORT) { + if (request->action() == FinishTabletJobRequest::ABORT || abort_compaction) { // TODO(gavin): mv tmp rowsets to recycle or remove them directly recorded_job.mutable_compaction()->erase(recorded_compaction); auto job_val = recorded_job.SerializeAsString(); @@ -597,6 +642,26 @@ void process_compaction_job(MetaServiceCode& code, std::string& msg, std::string return; } + //========================================================================== + // Lease + //========================================================================== + if (request->action() == FinishTabletJobRequest::LEASE) { + if (compaction.lease() <= 0 || recorded_compaction->lease() > compaction.lease()) { + ss << "invalid lease. recoreded_lease=" << recorded_compaction->lease() + << " req_lease=" << compaction.lease(); + msg = ss.str(); + code = MetaServiceCode::INVALID_ARGUMENT; + return; + } + recorded_compaction->set_lease(compaction.lease()); + auto job_val = recorded_job.SerializeAsString(); + txn->put(job_key, job_val); + INSTANCE_LOG(INFO) << "lease tablet compaction job, tablet_id=" << tablet_id + << " key=" << hex(job_key); + need_commit = true; + return; + } + //========================================================================== // Commit //========================================================================== @@ -854,7 +919,10 @@ void process_compaction_job(MetaServiceCode& code, std::string& msg, std::string txn->put(job_key, job_val); INSTANCE_LOG(INFO) << "remove compaction job tabelt_id=" << tablet_id << " key=" << hex(job_key); - + response->set_alter_version(recorded_job.has_schema_change() && + recorded_job.schema_change().has_alter_version() + ? recorded_job.schema_change().alter_version() + : -1); need_commit = true; } @@ -961,11 +1029,50 @@ void process_schema_change_job(MetaServiceCode& code, std::string& msg, std::str return; } + auto new_tablet_job_key = job_tablet_key( + {instance_id, new_table_id, new_index_id, new_partition_id, new_tablet_id}); + + std::string new_tablet_job_val; + TabletJobInfoPB new_recorded_job; + err = txn->get(new_tablet_job_key, &new_tablet_job_val); + if (err != TxnErrorCode::TXN_OK && err != TxnErrorCode::TXN_KEY_NOT_FOUND) { + SS << "internal error," + << " instance_id=" << instance_id << " tablet_id=" << new_tablet_id + << " job=" << proto_to_json(request->job()) << " err=" << err; + msg = ss.str(); + code = err == TxnErrorCode::TXN_KEY_NOT_FOUND ? MetaServiceCode::INVALID_ARGUMENT + : cast_as(err); + return; + } else if (err == TxnErrorCode::TXN_OK) { + if (!new_recorded_job.ParseFromString(new_tablet_job_val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = "malformed new tablet recorded job"; + return; + } + } + //========================================================================== // Abort //========================================================================== if (request->action() == FinishTabletJobRequest::ABORT) { - // TODO(cyx) + if (schema_change.new_tablet_idx().index_id() == + recorded_schema_change.new_tablet_idx().index_id() && + schema_change.new_tablet_idx().tablet_id() == + recorded_schema_change.new_tablet_idx().tablet_id()) { + // remove schema change + recorded_job.clear_schema_change(); + auto job_val = recorded_job.SerializeAsString(); + txn->put(job_key, job_val); + if (!new_tablet_job_val.empty()) { + new_recorded_job.clear_schema_change(); + new_tablet_job_val = new_recorded_job.SerializeAsString(); + txn->put(new_tablet_job_key, new_tablet_job_val); + } + INSTANCE_LOG(INFO) << "remove schema_change job tablet_id=" << tablet_id + << " key=" << hex(job_key); + + need_commit = true; + } return; } @@ -977,7 +1084,7 @@ void process_schema_change_job(MetaServiceCode& code, std::string& msg, std::str // 2. move rowsets [2-alter_version] in new_tablet to recycle // 3. update new_tablet stats // 4. change tmp rowset to formal rowset - // 5. remove schema_change job (unnecessary) + // 5. remove schema_change job // //========================================================================== // update tablet meta @@ -1126,6 +1233,11 @@ void process_schema_change_job(MetaServiceCode& code, std::string& msg, std::str recorded_job.clear_schema_change(); auto job_val = recorded_job.SerializeAsString(); txn->put(job_key, job_val); + if (!new_tablet_job_val.empty()) { + new_recorded_job.clear_schema_change(); + new_tablet_job_val = new_recorded_job.SerializeAsString(); + txn->put(new_tablet_job_key, new_tablet_job_val); + } INSTANCE_LOG(INFO) << "remove schema_change job tablet_id=" << tablet_id << " key=" << hex(job_key); diff --git a/cloud/test/meta_service_job_test.cpp b/cloud/test/meta_service_job_test.cpp index 250cf43ea98684..f0323eebb790be 100644 --- a/cloud/test/meta_service_job_test.cpp +++ b/cloud/test/meta_service_job_test.cpp @@ -60,9 +60,10 @@ void start_compaction_job(MetaService* meta_service, int64_t tablet_id, const st long now = time(nullptr); compaction->set_expiration(now + 12); compaction->set_lease(now + 3); - if (input_version.first > 0 && input_version.second > 0) { + 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); }; @@ -191,15 +192,18 @@ void create_tablet(MetaService* meta_service, int64_t table_id, int64_t index_id 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) { + const std::string& job_id, const std::string& initiator, + StartTabletJobResponse& res, int64_t alter_version = -1) { brpc::Controller cntl; StartTabletJobRequest req; - StartTabletJobResponse res; 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); @@ -217,13 +221,14 @@ void start_schema_change_job(MetaServiceProxy* meta_service, int64_t table_id, i EXPECT_EQ(job_pb.schema_change().id(), job_id) << ' ' << initiator; }; -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) { +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(FinishTabletJobRequest::COMMIT); + 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); @@ -682,8 +687,11 @@ TEST(MetaServiceJobTest, ProcessSchemaChangeArguments) { recorded_sc->set_id("sc1"); recorded_sc->set_initiator("BE1"); job_val = recorded_job.SerializeAsString(); + auto new_job_key = + job_tablet_key({instance_id, table_id, new_index_id, partition_id, new_tablet_id}); ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); txn->put(job_key, job_val); + txn->put(new_job_key, job_val); ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); meta_service->finish_tablet_job(&cntl, &req, &res, nullptr); ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT) << res.status().msg(); @@ -1332,9 +1340,10 @@ TEST(MetaServiceJobTest, SchemaChangeJobTest) { int64_t new_tablet_id = 14; 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, - "job1", "be1")); + "job1", "be1", sc_res)); FinishTabletJobResponse res; finish_schema_change_job(meta_service.get(), tablet_id, new_tablet_id, "job2", "be1", {}, res); @@ -1360,9 +1369,10 @@ TEST(MetaServiceJobTest, SchemaChangeJobTest) { int64_t new_tablet_id = 24; 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")); + "job2", "be1", sc_res)); std::vector output_rowsets; for (int64_t i = 0; i < 5; ++i) { @@ -1423,9 +1433,10 @@ TEST(MetaServiceJobTest, SchemaChangeJobTest) { int64_t new_tablet_id = 34; 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, - "job3", "be1")); + "job3", "be1", sc_res)); // provide existed rowsets std::vector existed_rowsets; for (int i = 0; i < 5; ++i) { @@ -1543,9 +1554,10 @@ TEST(MetaServiceJobTest, RetrySchemaChangeJobTest) { // start "job1" on BE1 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, "job1", - "be1")); + "be1", sc_res)); // provide existed rowsets std::vector existed_rowsets; for (int i = 0; i < 5; ++i) { @@ -1553,15 +1565,16 @@ TEST(MetaServiceJobTest, RetrySchemaChangeJobTest) { } ASSERT_NO_FATAL_FAILURE(insert_rowsets(meta_service->txn_kv().get(), table_id, index_id, partition_id, new_tablet_id, existed_rowsets)); - + sc_res.Clear(); // FE canceled "job1" and starts "job2" on BE1, should preempt previous "job1" ASSERT_NO_FATAL_FAILURE(start_schema_change_job(meta_service.get(), table_id, index_id, partition_id, tablet_id, new_tablet_id, "job2", - "be1")); + "be1", sc_res)); + sc_res.Clear(); // retry "job2" on BE1 ASSERT_NO_FATAL_FAILURE(start_schema_change_job(meta_service.get(), table_id, index_id, partition_id, tablet_id, new_tablet_id, "job2", - "be1")); + "be1", sc_res)); // BE1 output_versions=[2-8][9-9][10-10][11-11] std::vector be1_output_rowsets; be1_output_rowsets.push_back(create_rowset(new_tablet_id, 2, 8)); @@ -1573,11 +1586,11 @@ TEST(MetaServiceJobTest, RetrySchemaChangeJobTest) { commit_rowset(meta_service.get(), rs, res); ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << rs.end_version(); } - + sc_res.Clear(); // FE thinks BE1 is not alive and retries "job2" on BE2, should preempt "job2" created by BE1 ASSERT_NO_FATAL_FAILURE(start_schema_change_job(meta_service.get(), table_id, index_id, partition_id, tablet_id, new_tablet_id, "job2", - "be2")); + "be2", sc_res)); // BE2 output_versions=[2-8][9-12][13-13] std::vector be2_output_rowsets; { @@ -1703,9 +1716,10 @@ TEST(MetaServiceJobTest, SchemaChangeJobWithMoWTest) { int64_t new_tablet_id = 14; ASSERT_NO_FATAL_FAILURE(create_tablet(meta_service.get(), table_id, index_id, partition_id, new_tablet_id, true, 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, - "job1", "be1")); + "job1", "be1", sc_res)); std::vector output_rowsets; for (int64_t i = 0; i < 5; ++i) { output_rowsets.push_back(create_rowset(new_tablet_id, i + 2, i + 2)); @@ -1750,9 +1764,10 @@ TEST(MetaServiceJobTest, SchemaChangeJobWithMoWTest) { int64_t new_tablet_id = 15; ASSERT_NO_FATAL_FAILURE(create_tablet(meta_service.get(), table_id, index_id, partition_id, new_tablet_id, true, 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")); + "job2", "be1", sc_res)); std::vector output_rowsets; for (int64_t i = 0; i < 5; ++i) { output_rowsets.push_back(create_rowset(new_tablet_id, i + 2, i + 2)); @@ -2272,9 +2287,10 @@ TEST(MetaServiceJobTest, SchemaChangeJobPersistTest) { 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")); + "BE1", sc_res)); long now = time(nullptr); FinishTabletJobRequest req; @@ -2294,4 +2310,212 @@ TEST(MetaServiceJobTest, SchemaChangeJobPersistTest) { ASSERT_EQ(res.status().code(), MetaServiceCode::OK); } +TEST(MetaServiceJobTest, DoCompactionWhenSC) { + auto meta_service = get_meta_service(); + + auto* sp = SyncPoint::get_instance(); + std::unique_ptr> defer( + (int*)0x01, [](int*) { 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(); + + brpc::Controller cntl; + + int64_t table_id = 5421; + int64_t index_id = 5422; + int64_t partition_id = 5423; + int64_t tablet_id = 5424; + int64_t new_tablet_id = 54211; + + ASSERT_NO_FATAL_FAILURE( + create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id, false)); + + 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, + "job_sc", "BE1", sc_res, 8)); + + StartTabletJobResponse res; + start_compaction_job(meta_service.get(), tablet_id, "job1", "BE1", 0, 7, + TabletCompactionJobPB::CUMULATIVE, res, {7, 10}); + ASSERT_EQ(res.status().code(), MetaServiceCode::JOB_CHECK_ALTER_VERSION); + res.Clear(); + + start_compaction_job(meta_service.get(), tablet_id, "job1", "BE1", 0, 7, + TabletCompactionJobPB::BASE, res, {0, 10}); + ASSERT_EQ(res.status().code(), MetaServiceCode::JOB_CHECK_ALTER_VERSION); + res.Clear(); + + start_compaction_job(meta_service.get(), tablet_id, "job1", "BE1", 0, 7, + TabletCompactionJobPB::BASE, res, {0, 7}); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + res.Clear(); + + start_compaction_job(meta_service.get(), new_tablet_id, "job2", "BE1", 0, 7, + TabletCompactionJobPB::CUMULATIVE, res, {9, 10}); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + res.Clear(); + + std::unique_ptr txn; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + std::string job_key = + job_tablet_key({instance_id, table_id, index_id, partition_id, tablet_id}); + std::string job_val; + TabletJobInfoPB job_pb; + ASSERT_EQ(txn->get(job_key, &job_val), TxnErrorCode::TXN_OK); + ASSERT_TRUE(job_pb.ParseFromString(job_val)); + ASSERT_EQ(job_pb.compaction_size(), 1); + ASSERT_EQ(job_pb.compaction(0).id(), "job1"); + ASSERT_EQ(job_pb.compaction(0).initiator(), "BE1"); + ASSERT_EQ(job_pb.compaction(0).input_versions(0), 0); + ASSERT_EQ(job_pb.compaction(0).input_versions(1), 7); + ASSERT_EQ(job_pb.schema_change().alter_version(), 8); + + std::string new_job_key = + job_tablet_key({instance_id, table_id, index_id, partition_id, new_tablet_id}); + std::string new_job_val; + TabletJobInfoPB new_job_pb; + ASSERT_EQ(txn->get(new_job_key, &new_job_val), TxnErrorCode::TXN_OK); + ASSERT_TRUE(new_job_pb.ParseFromString(new_job_val)); + ASSERT_EQ(new_job_pb.compaction_size(), 1); + ASSERT_EQ(new_job_pb.compaction(0).id(), "job2"); + ASSERT_EQ(new_job_pb.compaction(0).initiator(), "BE1"); + ASSERT_EQ(new_job_pb.compaction(0).input_versions(0), 9); + ASSERT_EQ(new_job_pb.compaction(0).input_versions(1), 10); + ASSERT_EQ(new_job_pb.schema_change().alter_version(), 8); + + FinishTabletJobResponse finish_res; + finish_schema_change_job(meta_service.get(), tablet_id, new_tablet_id, "job_sc", "BE1", {}, + finish_res); + ASSERT_EQ(finish_res.status().code(), MetaServiceCode::OK); +} + +TEST(MetaServiceJobTest, ReStartSC) { + auto meta_service = get_meta_service(); + + auto* sp = SyncPoint::get_instance(); + std::unique_ptr> defer( + (int*)0x01, [](int*) { 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; + }); + bool use_origin_job = false; + sp->set_call_back("restart_compaction_job", [&](auto&&) { use_origin_job = true; }); + sp->enable_processing(); + + brpc::Controller cntl; + + int64_t table_id = 5331; + int64_t index_id = 5332; + int64_t partition_id = 5333; + int64_t tablet_id = 5334; + int64_t new_tablet_id = 53311; + + ASSERT_NO_FATAL_FAILURE( + create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id, false)); + + 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, + "job_sc", "BE1", sc_res, 8)); + sc_res.Clear(); + ASSERT_NO_FATAL_FAILURE(start_schema_change_job(meta_service.get(), table_id, index_id, + partition_id, tablet_id, new_tablet_id, + "job_sc", "BE1", sc_res, 8)); + ASSERT_TRUE(use_origin_job); + ASSERT_EQ(sc_res.alter_version(), 8); + FinishTabletJobResponse finish_res; + finish_schema_change_job(meta_service.get(), tablet_id, new_tablet_id, "job_sc", "BE1", {}, + finish_res); + ASSERT_EQ(finish_res.status().code(), MetaServiceCode::OK); +} + +TEST(MetaServiceJobTest, CancelSC) { + auto meta_service = get_meta_service(); + + auto* sp = SyncPoint::get_instance(); + std::unique_ptr> defer( + (int*)0x01, [](int*) { 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(); + + brpc::Controller cntl; + + int64_t table_id = 5731; + int64_t index_id = 5732; + int64_t partition_id = 5733; + int64_t tablet_id = 5734; + int64_t new_tablet_id = 57311; + int64_t new_tablet_id1 = 57322; + + ASSERT_NO_FATAL_FAILURE( + create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id, false)); + + ASSERT_NO_FATAL_FAILURE(create_tablet(meta_service.get(), table_id, index_id, partition_id, + new_tablet_id, false, true)); + + ASSERT_NO_FATAL_FAILURE(create_tablet(meta_service.get(), table_id, index_id, partition_id, + new_tablet_id1, 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, + "job_sc", "BE1", sc_res, 8)); + { + FinishTabletJobResponse finish_res; + finish_schema_change_job(meta_service.get(), tablet_id, new_tablet_id, "job_sc", "BE1", {}, + finish_res, FinishTabletJobRequest::ABORT); + ASSERT_EQ(finish_res.status().code(), MetaServiceCode::OK); + } + { + std::unique_ptr txn; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + std::string job_key = + job_tablet_key({instance_id, table_id, index_id, partition_id, tablet_id}); + std::string job_val; + TabletJobInfoPB job_pb; + ASSERT_EQ(txn->get(job_key, &job_val), TxnErrorCode::TXN_OK); + ASSERT_TRUE(job_pb.ParseFromString(job_val)); + ASSERT_FALSE(job_pb.has_schema_change()); + } + sc_res.Clear(); + ASSERT_NO_FATAL_FAILURE(start_schema_change_job(meta_service.get(), table_id, index_id, + partition_id, tablet_id, new_tablet_id1, + "job_sc1", "BE1", sc_res, 8)); + { + FinishTabletJobResponse finish_res; + finish_schema_change_job(meta_service.get(), tablet_id, new_tablet_id, "job_sc", "BE1", {}, + finish_res, FinishTabletJobRequest::ABORT); + ASSERT_NE(finish_res.status().msg().find("unmatched job id or initiator"), + std::string::npos); + } + { + std::unique_ptr txn; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + std::string job_key = + job_tablet_key({instance_id, table_id, index_id, partition_id, tablet_id}); + std::string job_val; + TabletJobInfoPB job_pb; + ASSERT_EQ(txn->get(job_key, &job_val), TxnErrorCode::TXN_OK); + ASSERT_TRUE(job_pb.ParseFromString(job_val)); + ASSERT_TRUE(job_pb.has_schema_change()); + } +} + } // namespace doris::cloud diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/CloudRollupJobV2.java b/fe/fe-core/src/main/java/org/apache/doris/alter/CloudRollupJobV2.java index 1c31d74d98630b..5c5dd1972ed323 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/CloudRollupJobV2.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/CloudRollupJobV2.java @@ -124,6 +124,20 @@ protected void onCancel() { try { ((CloudInternalCatalog) Env.getCurrentInternalCatalog()) .dropMaterializedIndex(tableId, rollupIndexList, false); + for (Map.Entry> partitionEntry : partitionIdToBaseRollupTabletIdMap.entrySet()) { + Long partitionId = partitionEntry.getKey(); + Map rollupTabletIdToBaseTabletId = partitionEntry.getValue(); + for (Map.Entry tabletEntry : rollupTabletIdToBaseTabletId.entrySet()) { + Long rollupTabletId = tabletEntry.getKey(); + Long baseTabletId = tabletEntry.getValue(); + ((CloudInternalCatalog) Env.getCurrentInternalCatalog()) + .removeSchemaChangeJob(dbId, tableId, baseIndexId, rollupIndexId, + partitionId, baseTabletId, rollupTabletId); + } + LOG.info("Cancel RollupJob. Remove SchemaChangeJob in ms." + + "dbId:{}, tableId:{}, rollupIndexId: {} partitionId:{}. tabletSize:{}", + dbId, tableId, rollupIndexId, partitionId, rollupTabletIdToBaseTabletId.size()); + } break; } catch (Exception e) { LOG.warn("tryTimes:{}, onCancel exception:", tryTimes, e); diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/CloudSchemaChangeJobV2.java b/fe/fe-core/src/main/java/org/apache/doris/alter/CloudSchemaChangeJobV2.java index a8bcc546de33e6..3d79863addb8ac 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/CloudSchemaChangeJobV2.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/CloudSchemaChangeJobV2.java @@ -41,6 +41,8 @@ import com.google.common.base.Preconditions; import com.google.common.base.Strings; +import com.google.common.collect.Table; +import com.google.common.collect.Table.Cell; import com.google.gson.annotations.SerializedName; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -49,8 +51,10 @@ import java.lang.reflect.Field; import java.util.ArrayList; import java.util.Arrays; +import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.stream.Collectors; public class CloudSchemaChangeJobV2 extends SchemaChangeJobV2 { @@ -106,7 +110,7 @@ protected void commitShadowIndex() throws AlterCancelException { } @Override - protected void postProcessShadowIndex() { + protected void onCancel() { if (Config.enable_check_compatibility_mode) { LOG.info("skip drop shadown indexes in checking compatibility mode"); return; @@ -114,6 +118,36 @@ protected void postProcessShadowIndex() { List shadowIdxList = indexIdMap.keySet().stream().collect(Collectors.toList()); dropIndex(shadowIdxList); + + long tryTimes = 1; + while (true) { + try { + Set>> tableSet = partitionIndexTabletMap.cellSet(); + Iterator>> it = tableSet.iterator(); + while (it.hasNext()) { + Table.Cell> data = it.next(); + Long partitionId = data.getRowKey(); + Long shadowIndexId = data.getColumnKey(); + Long originIndexId = indexIdMap.get(shadowIndexId); + Map shadowTabletIdToOriginTabletId = data.getValue(); + for (Map.Entry entry : shadowTabletIdToOriginTabletId.entrySet()) { + Long shadowTabletId = entry.getKey(); + Long originTabletId = entry.getValue(); + ((CloudInternalCatalog) Env.getCurrentInternalCatalog()) + .removeSchemaChangeJob(dbId, tableId, originIndexId, shadowIndexId, + partitionId, originTabletId, shadowTabletId); + } + LOG.info("Cancel SchemaChange. Remove SchemaChangeJob in ms." + + "dbId:{}, tableId:{}, originIndexId:{}, partitionId:{}. tabletSize:{}", + dbId, tableId, originIndexId, partitionId, shadowTabletIdToOriginTabletId.size()); + } + break; + } catch (Exception e) { + LOG.warn("tryTimes:{}, onCancel exception:", tryTimes, e); + } + sleepSeveralSeconds(); + tryTimes++; + } } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java b/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java index e38c91d296fed4..d22b1bff9fc30e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java @@ -104,7 +104,7 @@ public class RollupJobV2 extends AlterJobV2 implements GsonPostProcessable { // partition id -> (rollup tablet id -> base tablet id) @SerializedName(value = "partitionIdToBaseRollupTabletIdMap") - private Map> partitionIdToBaseRollupTabletIdMap = Maps.newHashMap(); + protected Map> partitionIdToBaseRollupTabletIdMap = Maps.newHashMap(); @SerializedName(value = "partitionIdToRollupIndex") protected Map partitionIdToRollupIndex = Maps.newHashMap(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java index 527c8620c6cf77..94dd2421f843ad 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java @@ -94,7 +94,7 @@ public class SchemaChangeJobV2 extends AlterJobV2 { // partition id -> (shadow index id -> (shadow tablet id -> origin tablet id)) @SerializedName(value = "partitionIndexTabletMap") - private Table> partitionIndexTabletMap = HashBasedTable.create(); + protected Table> partitionIndexTabletMap = HashBasedTable.create(); // partition id -> (shadow index id -> shadow index)) @SerializedName(value = "partitionIndexMap") protected Table partitionIndexMap = HashBasedTable.create(); @@ -755,7 +755,7 @@ protected synchronized boolean cancelImpl(String errMsg) { changeTableState(dbId, tableId, OlapTableState.NORMAL); LOG.info("set table's state to NORMAL when cancel, table id: {}, job id: {}", tableId, jobId); - postProcessShadowIndex(); + onCancel(); return true; } @@ -903,7 +903,7 @@ private void replayRunningJob(SchemaChangeJobV2 replayedJob) { private void replayCancelled(SchemaChangeJobV2 replayedJob) { cancelInternal(); // try best to drop shadow index - postProcessShadowIndex(); + onCancel(); this.jobState = JobState.CANCELLED; this.finishedTimeMs = replayedJob.finishedTimeMs; this.errMsg = replayedJob.errMsg; @@ -1006,7 +1006,7 @@ private void changeTableState(long dbId, long tableId, OlapTableState olapTableS protected void commitShadowIndex() throws AlterCancelException {} // try best to drop shadow index, when job is cancelled in cloud mode - protected void postProcessShadowIndex() {} + protected void onCancel() {} // try best to drop origin index in cloud mode protected void postProcessOriginIndex() {} diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/datasource/CloudInternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/datasource/CloudInternalCatalog.java index e9d4e3c33acedc..f0c9278562d437 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/cloud/datasource/CloudInternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/datasource/CloudInternalCatalog.java @@ -845,6 +845,67 @@ private void dropCloudPartition(long dbId, long tableId, List partitionIds } } + public void removeSchemaChangeJob(long dbId, long tableId, long indexId, long newIndexId, + long partitionId, long tabletId, long newTabletId) + throws DdlException { + Cloud.FinishTabletJobRequest.Builder finishTabletJobRequestBuilder = Cloud.FinishTabletJobRequest.newBuilder(); + finishTabletJobRequestBuilder.setCloudUniqueId(Config.cloud_unique_id); + finishTabletJobRequestBuilder.setAction(Cloud.FinishTabletJobRequest.Action.ABORT); + Cloud.TabletJobInfoPB.Builder tabletJobInfoPBBuilder = Cloud.TabletJobInfoPB.newBuilder(); + + // set origin tablet + Cloud.TabletIndexPB.Builder tabletIndexPBBuilder = Cloud.TabletIndexPB.newBuilder(); + tabletIndexPBBuilder.setDbId(dbId); + tabletIndexPBBuilder.setTableId(tableId); + tabletIndexPBBuilder.setIndexId(indexId); + tabletIndexPBBuilder.setPartitionId(partitionId); + tabletIndexPBBuilder.setTabletId(tabletId); + final Cloud.TabletIndexPB tabletIndex = tabletIndexPBBuilder.build(); + tabletJobInfoPBBuilder.setIdx(tabletIndex); + + // set new tablet + Cloud.TabletSchemaChangeJobPB.Builder schemaChangeJobPBBuilder = + Cloud.TabletSchemaChangeJobPB.newBuilder(); + Cloud.TabletIndexPB.Builder newtabletIndexPBBuilder = Cloud.TabletIndexPB.newBuilder(); + newtabletIndexPBBuilder.setDbId(dbId); + newtabletIndexPBBuilder.setTableId(tableId); + newtabletIndexPBBuilder.setIndexId(newIndexId); + newtabletIndexPBBuilder.setPartitionId(partitionId); + newtabletIndexPBBuilder.setTabletId(newTabletId); + final Cloud.TabletIndexPB newtabletIndex = newtabletIndexPBBuilder.build(); + schemaChangeJobPBBuilder.setNewTabletIdx(newtabletIndex); + final Cloud.TabletSchemaChangeJobPB tabletSchemaChangeJobPb = + schemaChangeJobPBBuilder.build(); + + tabletJobInfoPBBuilder.setSchemaChange(tabletSchemaChangeJobPb); + + final Cloud.TabletJobInfoPB tabletJobInfoPB = tabletJobInfoPBBuilder.build(); + finishTabletJobRequestBuilder.setJob(tabletJobInfoPB); + + final Cloud.FinishTabletJobRequest request = finishTabletJobRequestBuilder.build(); + + Cloud.FinishTabletJobResponse response = null; + int tryTimes = 0; + while (tryTimes++ < Config.metaServiceRpcRetryTimes()) { + try { + response = MetaServiceProxy.getInstance().finishTabletJob(request); + if (response.getStatus().getCode() != Cloud.MetaServiceCode.KV_TXN_CONFLICT) { + break; + } + } catch (RpcException e) { + LOG.warn("tryTimes:{}, finishTabletJob RpcException", tryTimes, e); + if (tryTimes + 1 >= Config.metaServiceRpcRetryTimes()) { + throw new DdlException(e.getMessage()); + } + } + sleepSeveralMs(); + } + + if (response.getStatus().getCode() != Cloud.MetaServiceCode.OK) { + LOG.warn("finishTabletJob response: {} ", response); + } + } + public void dropMaterializedIndex(long tableId, List indexIds, boolean dropTable) throws DdlException { if (Config.enable_check_compatibility_mode) { LOG.info("skip dropping materialized index in compatibility checking mode"); diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/rpc/MetaServiceClient.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/rpc/MetaServiceClient.java index 5fd42a31f92459..1216d0a684692b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/cloud/rpc/MetaServiceClient.java +++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/rpc/MetaServiceClient.java @@ -382,4 +382,15 @@ public Cloud.GetInstanceResponse getInstance(Cloud.GetInstanceRequest request) { } return blockingStub.abortTxnWithCoordinator(request); } + + public Cloud.FinishTabletJobResponse + finishTabletJob(Cloud.FinishTabletJobRequest request) { + if (!request.hasCloudUniqueId()) { + Cloud.FinishTabletJobRequest.Builder builder = + Cloud.FinishTabletJobRequest.newBuilder(); + builder.mergeFrom(request); + return blockingStub.finishTabletJob(builder.setCloudUniqueId(Config.cloud_unique_id).build()); + } + return blockingStub.finishTabletJob(request); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/rpc/MetaServiceProxy.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/rpc/MetaServiceProxy.java index 5f17692180b353..9f944f0ddf9cda 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/cloud/rpc/MetaServiceProxy.java +++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/rpc/MetaServiceProxy.java @@ -497,6 +497,16 @@ public Cloud.AlterObjStoreInfoResponse alterStorageVault(Cloud.AlterObjStoreInfo } } + public Cloud.FinishTabletJobResponse finishTabletJob(Cloud.FinishTabletJobRequest request) + throws RpcException { + try { + final MetaServiceClient client = getProxy(); + return client.finishTabletJob(request); + } catch (Exception e) { + throw new RpcException("", e.getMessage(), e); + } + } + public Cloud.GetRLTaskCommitAttachResponse getRLTaskCommitAttach(Cloud.GetRLTaskCommitAttachRequest request) throws RpcException { diff --git a/gensrc/proto/cloud.proto b/gensrc/proto/cloud.proto index 7d9c9c19169f40..f8acd97d05fb38 100644 --- a/gensrc/proto/cloud.proto +++ b/gensrc/proto/cloud.proto @@ -536,6 +536,7 @@ message TabletCompactionJobPB { optional int64 lease = 23; // prepare optional int64 delete_bitmap_lock_initiator = 24; optional int64 full_compaction_cnt = 25; // prepare + optional bool check_input_versions_range = 26; } message TabletSchemaChangeJobPB { @@ -1182,6 +1183,7 @@ message StartTabletJobRequest { message StartTabletJobResponse { optional MetaServiceResponseStatus status = 1; repeated int64 version_in_compaction = 2; + optional int64 alter_version = 3; } message FinishTabletJobRequest { @@ -1199,6 +1201,7 @@ message FinishTabletJobRequest { message FinishTabletJobResponse { optional MetaServiceResponseStatus status = 1; optional TabletStatsPB stats = 2; + optional int64 alter_version = 3; } message BeginCopyRequest { @@ -1337,6 +1340,7 @@ enum MetaServiceCode { JOB_ALREADY_SUCCESS = 5002; ROUTINE_LOAD_DATA_INCONSISTENT = 5003; ROUTINE_LOAD_PROGRESS_NOT_FOUND = 5004; + JOB_CHECK_ALTER_VERSION = 5005; // Rate limit MAX_QPS_LIMIT = 6001; diff --git a/regression-test/pipeline/cloud_p0/conf/be_custom.conf b/regression-test/pipeline/cloud_p0/conf/be_custom.conf index 9f2967b1972c11..1da9c9992d5f93 100644 --- a/regression-test/pipeline/cloud_p0/conf/be_custom.conf +++ b/regression-test/pipeline/cloud_p0/conf/be_custom.conf @@ -33,3 +33,4 @@ save_load_error_log_to_s3 = true enable_stream_load_record = true stream_load_record_batch_size = 500 webserver_num_workers = 128 +enable_new_tablet_do_compaction = true diff --git a/regression-test/plugins/plugin_curl_requester.groovy b/regression-test/plugins/plugin_curl_requester.groovy index 1e097a045a76d5..4be7f4b2e0ff6a 100644 --- a/regression-test/plugins/plugin_curl_requester.groovy +++ b/regression-test/plugins/plugin_curl_requester.groovy @@ -173,6 +173,10 @@ Suite.metaClass.be_get_overall_compaction_status{ String ip, String port /* par return curl("GET", String.format("http://%s:%s/api/compaction/run_status", ip, port)) } +Suite.metaClass.be_show_tablet_status{ String ip, String port, String tablet_id /* param */-> + return curl("GET", String.format("http://%s:%s/api/compaction/show?tablet_id=%s", ip, port, tablet_id)) +} + logger.info("Added 'be_get_compaction_status' function to Suite") Suite.metaClass._be_run_compaction = { String ip, String port, String tablet_id, String compact_type -> diff --git a/regression-test/suites/cloud_p0/schema_change/compaction1/test_schema_change_with_compaction1.groovy b/regression-test/suites/cloud_p0/schema_change/compaction1/test_schema_change_with_compaction1.groovy new file mode 100644 index 00000000000000..93bb6254303301 --- /dev/null +++ b/regression-test/suites/cloud_p0/schema_change/compaction1/test_schema_change_with_compaction1.groovy @@ -0,0 +1,257 @@ +// 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. + +// Most of the cases are copied from https://github.com/trinodb/trino/tree/master +// /testing/trino-product-tests/src/main/resources/sql-tests/testcases +// and modified by Doris. + +// Note: To filter out tables from sql files, use the following one-liner comamnd +// sed -nr 's/.*tables: (.*)$/\1/gp' /path/to/*.sql | sed -nr 's/,/\n/gp' | sort | uniq + +import org.apache.doris.regression.util.DebugPoint + +import org.apache.doris.regression.util.NodeType + +suite('test_schema_change_with_compaction1', 'nonConcurrent') { + def getJobState = { tableName -> + def jobStateResult = sql """ SHOW ALTER TABLE COLUMN WHERE IndexName='${tableName}' ORDER BY createtime DESC LIMIT 1 """ + return jobStateResult[0][9] + } + + def s3BucketName = getS3BucketName() + def s3WithProperties = """WITH S3 ( + |"AWS_ACCESS_KEY" = "${getS3AK()}", + |"AWS_SECRET_KEY" = "${getS3SK()}", + |"AWS_ENDPOINT" = "${getS3Endpoint()}", + |"AWS_REGION" = "${getS3Region()}", + |"provider" = "${getS3Provider()}") + |PROPERTIES( + |"exec_mem_limit" = "8589934592", + |"load_parallelism" = "3")""".stripMargin() + + // set fe configuration + sql "ADMIN SET FRONTEND CONFIG ('max_bytes_per_broker_scanner' = '161061273600')" + sql new File("""${context.file.parent}/../ddl/date_delete.sql""").text + def load_date_once = { String table -> + def uniqueID = Math.abs(UUID.randomUUID().hashCode()).toString() + def loadLabel = table + "_" + uniqueID + // load data from cos + def loadSql = new File("""${context.file.parent}/../ddl/${table}_load.sql""").text.replaceAll("\\\$\\{s3BucketName\\}", s3BucketName) + loadSql = loadSql.replaceAll("\\\$\\{loadLabel\\}", loadLabel) + s3WithProperties + sql loadSql + + // check load state + while (true) { + def stateResult = sql "show load where Label = '${loadLabel}'" + def loadState = stateResult[stateResult.size() - 1][2].toString() + if ("CANCELLED".equalsIgnoreCase(loadState)) { + throw new IllegalStateException("load ${loadLabel} failed.") + } else if ("FINISHED".equalsIgnoreCase(loadState)) { + break + } + sleep(5000) + } + } + + sql new File("""${context.file.parent}/../ddl/date_create.sql""").text + def injectName = 'CloudSchemaChangeJob.process_alter_tablet.sleep' + def injectBe = null + def backends = sql_return_maparray('show backends') + def array = sql_return_maparray("SHOW TABLETS FROM date") + def injectBeId = array[0].BackendId + def originTabletId = array[0].TabletId + injectBe = backends.stream().filter(be -> be.BackendId == injectBeId).findFirst().orElse(null) + assertNotNull(injectBe) + + def load_delete_compaction = { + load_date_once("date"); + sql "delete from date where d_datekey < 19900000" + sql "select count(*) from date" + // cu compaction + logger.info("run compaction:" + originTabletId) + (code, out, err) = be_run_cumulative_compaction(injectBe.Host, injectBe.HttpPort, originTabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + boolean running = true + do { + Thread.sleep(100) + (code, out, err) = be_get_compaction_status(injectBe.Host, injectBe.HttpPort, originTabletId) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } while (running) + } + + try { + load_delete_compaction() + load_delete_compaction() + load_delete_compaction() + + load_date_once("date"); + + sleep(1000) + + DebugPoint.enableDebugPoint(injectBe.Host, injectBe.HttpPort.toInteger(), NodeType.BE, injectName) + sql "ALTER TABLE date MODIFY COLUMN d_holidayfl bigint(11)" + sleep(5000) + array = sql_return_maparray("SHOW TABLETS FROM date") + + for (int i = 0; i < 5; i++) { + load_date_once("date"); + } + + // base compaction + logger.info("run compaction:" + originTabletId) + (code, out, err) = be_run_base_compaction(injectBe.Host, injectBe.HttpPort, originTabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + + + // wait for all compactions done + boolean running = true + while (running) { + Thread.sleep(100) + (code, out, err) = be_get_compaction_status(injectBe.Host, injectBe.HttpPort, originTabletId) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } + def newTabletId = array[1].TabletId + logger.info("run compaction:" + newTabletId) + (code, out, err) = be_run_base_compaction(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + assertTrue(out.contains("invalid tablet state.")) + + + // cu compaction + for (int i = 0; i < array.size(); i++) { + tabletId = array[i].TabletId + logger.info("run compaction:" + tabletId) + (code, out, err) = be_run_cumulative_compaction(injectBe.Host, injectBe.HttpPort, tabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + } + + for (int i = 0; i < array.size(); i++) { + running = true + do { + Thread.sleep(100) + tabletId = array[i].TabletId + (code, out, err) = be_get_compaction_status(injectBe.Host, injectBe.HttpPort, tabletId) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } while (running) + } + } finally { + if (injectBe != null) { + DebugPoint.disableDebugPoint(injectBe.Host, injectBe.HttpPort.toInteger(), NodeType.BE, injectName) + } + int max_try_time = 3000 + while (max_try_time--){ + result = getJobState("date") + if (result == "FINISHED" || result == "CANCELLED") { + sleep(3000) + break + } else { + sleep(100) + if (max_try_time < 1){ + assertEquals(1,2) + } + } + } + assertEquals(result, "FINISHED"); + def count = sql """ select count(*) from date; """ + assertEquals(count[0][0], 23004); + // check rowsets + logger.info("run show:" + originTabletId) + (code, out, err) = be_show_tablet_status(injectBe.Host, injectBe.HttpPort, originTabletId) + logger.info("Run show: code=" + code + ", out=" + out + ", err=" + err) + assertTrue(out.contains("[0-1]")) + assertTrue(out.contains("[2-7]")) + assertTrue(out.contains("[8-8]")) + assertTrue(out.contains("[9-13]")) + + logger.info("run show:" + newTabletId) + (code, out, err) = be_show_tablet_status(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Run show: code=" + code + ", out=" + out + ", err=" + err) + assertTrue(out.contains("[0-1]")) + assertTrue(out.contains("[2-2]")) + assertTrue(out.contains("[7-7]")) + assertTrue(out.contains("[8-8]")) + assertTrue(out.contains("[9-13]")) + + // base compaction + logger.info("run compaction:" + newTabletId) + (code, out, err) = be_run_base_compaction(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + + + // wait for all compactions done + boolean running = true + while (running) { + Thread.sleep(100) + (code, out, err) = be_get_compaction_status(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } + + logger.info("run show:" + newTabletId) + (code, out, err) = be_show_tablet_status(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Run show: code=" + code + ", out=" + out + ", err=" + err) + assertTrue(out.contains("[0-1]")) + assertTrue(out.contains("[2-7]")) + assertTrue(out.contains("[8-8]")) + assertTrue(out.contains("[9-13]")) + + for (int i = 0; i < 3; i++) { + load_date_once("date"); + } + + sql """ select count(*) from date """ + + logger.info("run compaction:" + newTabletId) + (code, out, err) = be_run_cumulative_compaction(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + + // wait for all compactions done + running = true + while (running) { + Thread.sleep(100) + (code, out, err) = be_get_compaction_status(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } + + logger.info("run show:" + newTabletId) + (code, out, err) = be_show_tablet_status(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Run show: code=" + code + ", out=" + out + ", err=" + err) + assertTrue(out.contains("[0-1]")) + assertTrue(out.contains("[2-7]")) + assertTrue(out.contains("[8-16]")) + } + +} \ No newline at end of file diff --git a/regression-test/suites/cloud_p0/schema_change/compaction10/test_schema_change_with_compaction10.groovy b/regression-test/suites/cloud_p0/schema_change/compaction10/test_schema_change_with_compaction10.groovy new file mode 100644 index 00000000000000..b393979d44218a --- /dev/null +++ b/regression-test/suites/cloud_p0/schema_change/compaction10/test_schema_change_with_compaction10.groovy @@ -0,0 +1,263 @@ +// 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. + +import org.apache.doris.regression.suite.ClusterOptions +import org.apache.http.NoHttpResponseException +import org.apache.doris.regression.util.DebugPoint +import org.apache.doris.regression.util.NodeType + +suite('test_schema_change_with_compaction10') { + def options = new ClusterOptions() + options.cloudMode = true + options.enableDebugPoints() + options.beConfigs += [ "enable_java_support=false" ] + options.beConfigs += [ "enable_new_tablet_do_compaction=true" ] + options.beConfigs += [ "disable_auto_compaction=true" ] + options.beNum = 1 + docker(options) { + def getJobState = { tableName -> + def jobStateResult = sql """ SHOW ALTER TABLE COLUMN WHERE IndexName='${tableName}' ORDER BY createtime DESC LIMIT 1 """ + return jobStateResult[0][9] + } + + def s3BucketName = getS3BucketName() + def s3WithProperties = """WITH S3 ( + |"AWS_ACCESS_KEY" = "${getS3AK()}", + |"AWS_SECRET_KEY" = "${getS3SK()}", + |"AWS_ENDPOINT" = "${getS3Endpoint()}", + |"AWS_REGION" = "${getS3Region()}", + |"provider" = "${getS3Provider()}") + |PROPERTIES( + |"exec_mem_limit" = "8589934592", + |"load_parallelism" = "3")""".stripMargin() + + // set fe configuration + sql "ADMIN SET FRONTEND CONFIG ('max_bytes_per_broker_scanner' = '161061273600')" + sql new File("""${context.file.parent}/../ddl/date_delete.sql""").text + def load_date_once = { String table -> + def uniqueID = Math.abs(UUID.randomUUID().hashCode()).toString() + def loadLabel = table + "_" + uniqueID + // load data from cos + def loadSql = new File("""${context.file.parent}/../ddl/${table}_load.sql""").text.replaceAll("\\\$\\{s3BucketName\\}", s3BucketName) + loadSql = loadSql.replaceAll("\\\$\\{loadLabel\\}", loadLabel) + s3WithProperties + sql loadSql + + // check load state + while (true) { + def stateResult = sql "show load where Label = '${loadLabel}'" + def loadState = stateResult[stateResult.size() - 1][2].toString() + if ("CANCELLED".equalsIgnoreCase(loadState)) { + throw new IllegalStateException("load ${loadLabel} failed.") + } else if ("FINISHED".equalsIgnoreCase(loadState)) { + break + } + sleep(5000) + } + } + + sql new File("""${context.file.parent}/../ddl/date_unique_create.sql""").text + def injectName = 'CloudSchemaChangeJob.process_alter_tablet.sleep' + def injectBe = null + def backends = sql_return_maparray('show backends') + def array = sql_return_maparray("SHOW TABLETS FROM date") + def injectBeId = array[0].BackendId + def originTabletId = array[0].TabletId + injectBe = backends.stream().filter(be -> be.BackendId == injectBeId).findFirst().orElse(null) + assertNotNull(injectBe) + + def load_delete_compaction = { + load_date_once("date"); + sql "delete from date where d_datekey < 19900000" + sql "select count(*) from date" + // cu compaction + logger.info("run compaction:" + originTabletId) + (code, out, err) = be_run_cumulative_compaction(injectBe.Host, injectBe.HttpPort, originTabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + boolean running = true + do { + Thread.sleep(100) + (code, out, err) = be_get_compaction_status(injectBe.Host, injectBe.HttpPort, originTabletId) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } while (running) + } + + try { + load_delete_compaction() + load_delete_compaction() + load_delete_compaction() + + load_date_once("date"); + + sleep(1000) + GetDebugPoint().enableDebugPointForAllBEs(injectName) + sql "ALTER TABLE date MODIFY COLUMN d_holidayfl bigint(11)" + sleep(5000) + array = sql_return_maparray("SHOW TABLETS FROM date") + + for (int i = 0; i < 5; i++) { + load_date_once("date"); + } + + cluster.restartBackends() + GetDebugPoint().enableDebugPointForAllBEs(injectName) + sleep(30000) + + // base compaction + logger.info("run compaction:" + originTabletId) + (code, out, err) = be_run_base_compaction(injectBe.Host, injectBe.HttpPort, originTabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + + + // wait for all compactions done + boolean running = true + while (running) { + Thread.sleep(100) + (code, out, err) = be_get_compaction_status(injectBe.Host, injectBe.HttpPort, originTabletId) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } + def newTabletId = array[1].TabletId + logger.info("run compaction:" + newTabletId) + (code, out, err) = be_run_base_compaction(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + assertTrue(out.contains("invalid tablet state.")) + + + // cu compaction + for (int i = 0; i < array.size(); i++) { + tabletId = array[i].TabletId + logger.info("run compaction:" + tabletId) + (code, out, err) = be_run_cumulative_compaction(injectBe.Host, injectBe.HttpPort, tabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + } + + for (int i = 0; i < array.size(); i++) { + running = true + do { + Thread.sleep(100) + tabletId = array[i].TabletId + (code, out, err) = be_get_compaction_status(injectBe.Host, injectBe.HttpPort, tabletId) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } while (running) + } + + } finally { + if (injectBe != null) { + GetDebugPoint().disableDebugPointForAllBEs(injectName) + } + int max_try_time = 3000 + while (max_try_time--){ + result = getJobState("date") + if (result == "FINISHED" || result == "CANCELLED") { + sleep(3000) + break + } else { + sleep(100) + if (max_try_time < 1){ + assertEquals(1,2) + } + } + } + assertEquals(result, "FINISHED"); + def count = sql """ select count(*) from date; """ + assertEquals(count[0][0], 2556); + // check rowsets + logger.info("run show:" + originTabletId) + (code, out, err) = be_show_tablet_status(injectBe.Host, injectBe.HttpPort, originTabletId) + logger.info("Run show: code=" + code + ", out=" + out + ", err=" + err) + assertTrue(out.contains("[0-1]")) + assertTrue(out.contains("[2-7]")) + assertTrue(out.contains("[8-8]")) + assertTrue(out.contains("[9-13]")) + + logger.info("run show:" + newTabletId) + (code, out, err) = be_show_tablet_status(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Run show: code=" + code + ", out=" + out + ", err=" + err) + assertTrue(out.contains("[0-1]")) + assertTrue(out.contains("[2-2]")) + assertTrue(out.contains("[7-7]")) + assertTrue(out.contains("[8-8]")) + assertTrue(out.contains("[9-13]")) + + // base compaction + logger.info("run compaction:" + newTabletId) + (code, out, err) = be_run_base_compaction(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + + + // wait for all compactions done + boolean running = true + while (running) { + Thread.sleep(100) + (code, out, err) = be_get_compaction_status(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } + + logger.info("run show:" + newTabletId) + (code, out, err) = be_show_tablet_status(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Run show: code=" + code + ", out=" + out + ", err=" + err) + assertTrue(out.contains("[0-1]")) + assertTrue(out.contains("[2-7]")) + assertTrue(out.contains("[8-8]")) + assertTrue(out.contains("[9-13]")) + + for (int i = 0; i < 3; i++) { + load_date_once("date"); + } + + sql """ select count(*) from date """ + + logger.info("run compaction:" + newTabletId) + (code, out, err) = be_run_cumulative_compaction(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + + // wait for all compactions done + running = true + while (running) { + Thread.sleep(100) + (code, out, err) = be_get_compaction_status(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } + + logger.info("run show:" + newTabletId) + (code, out, err) = be_show_tablet_status(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Run show: code=" + code + ", out=" + out + ", err=" + err) + assertTrue(out.contains("[0-1]")) + assertTrue(out.contains("[2-7]")) + assertTrue(out.contains("[8-16]")) + } + } +} \ No newline at end of file diff --git a/regression-test/suites/cloud_p0/schema_change/compaction11/test_schema_change_with_compaction11.groovy b/regression-test/suites/cloud_p0/schema_change/compaction11/test_schema_change_with_compaction11.groovy new file mode 100644 index 00000000000000..fd257fcb7ea950 --- /dev/null +++ b/regression-test/suites/cloud_p0/schema_change/compaction11/test_schema_change_with_compaction11.groovy @@ -0,0 +1,280 @@ +// 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. + +import org.apache.doris.regression.suite.ClusterOptions +import org.apache.http.NoHttpResponseException +import org.apache.doris.regression.util.DebugPoint +import org.apache.doris.regression.util.NodeType + +suite('test_schema_change_with_compaction11') { + def options = new ClusterOptions() + options.cloudMode = true + options.enableDebugPoints() + options.beConfigs += [ "enable_java_support=false" ] + options.beConfigs += [ "enable_new_tablet_do_compaction=false" ] + options.beConfigs += [ "disable_auto_compaction=true" ] + options.beNum = 1 + docker(options) { + def getJobState = { tableName -> + def jobStateResult = sql """ SHOW ALTER TABLE COLUMN WHERE IndexName='${tableName}' ORDER BY createtime DESC LIMIT 1 """ + return jobStateResult[0][9] + } + + def s3BucketName = getS3BucketName() + def s3WithProperties = """WITH S3 ( + |"AWS_ACCESS_KEY" = "${getS3AK()}", + |"AWS_SECRET_KEY" = "${getS3SK()}", + |"AWS_ENDPOINT" = "${getS3Endpoint()}", + |"AWS_REGION" = "${getS3Region()}", + |"provider" = "${getS3Provider()}") + |PROPERTIES( + |"exec_mem_limit" = "8589934592", + |"load_parallelism" = "3")""".stripMargin() + + // set fe configuration + sql "ADMIN SET FRONTEND CONFIG ('max_bytes_per_broker_scanner' = '161061273600')" + sql new File("""${context.file.parent}/../ddl/date_delete.sql""").text + def load_date_once = { String table -> + def uniqueID = Math.abs(UUID.randomUUID().hashCode()).toString() + def loadLabel = table + "_" + uniqueID + // load data from cos + def loadSql = new File("""${context.file.parent}/../ddl/${table}_load.sql""").text.replaceAll("\\\$\\{s3BucketName\\}", s3BucketName) + loadSql = loadSql.replaceAll("\\\$\\{loadLabel\\}", loadLabel) + s3WithProperties + sql loadSql + + // check load state + while (true) { + def stateResult = sql "show load where Label = '${loadLabel}'" + def loadState = stateResult[stateResult.size() - 1][2].toString() + if ("CANCELLED".equalsIgnoreCase(loadState)) { + throw new IllegalStateException("load ${loadLabel} failed.") + } else if ("FINISHED".equalsIgnoreCase(loadState)) { + break + } + sleep(5000) + } + } + + sql new File("""${context.file.parent}/../ddl/date_unique_create.sql""").text + def injectName = 'CloudSchemaChangeJob.process_alter_tablet.sleep' + def injectBe = null + def backends = sql_return_maparray('show backends') + def array = sql_return_maparray("SHOW TABLETS FROM date") + def injectBeId = array[0].BackendId + def originTabletId = array[0].TabletId + injectBe = backends.stream().filter(be -> be.BackendId == injectBeId).findFirst().orElse(null) + assertNotNull(injectBe) + + def load_delete_compaction = { + load_date_once("date"); + sql "delete from date where d_datekey < 19900000" + sql "select count(*) from date" + // cu compaction + logger.info("run compaction:" + originTabletId) + (code, out, err) = be_run_cumulative_compaction(injectBe.Host, injectBe.HttpPort, originTabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + boolean running = true + do { + Thread.sleep(100) + (code, out, err) = be_get_compaction_status(injectBe.Host, injectBe.HttpPort, originTabletId) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } while (running) + } + + try { + load_delete_compaction() + load_delete_compaction() + load_delete_compaction() + + load_date_once("date"); + + sleep(1000) + GetDebugPoint().enableDebugPointForAllBEs(injectName) + sql "ALTER TABLE date MODIFY COLUMN d_holidayfl bigint(11)" + sleep(5000) + array = sql_return_maparray("SHOW TABLETS FROM date") + + for (int i = 0; i < 5; i++) { + load_date_once("date"); + } + + // base compaction + logger.info("run compaction:" + originTabletId) + (code, out, err) = be_run_base_compaction(injectBe.Host, injectBe.HttpPort, originTabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + + + // wait for all compactions done + boolean running = true + while (running) { + Thread.sleep(100) + (code, out, err) = be_get_compaction_status(injectBe.Host, injectBe.HttpPort, originTabletId) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } + def newTabletId = array[1].TabletId + logger.info("run compaction:" + newTabletId) + (code, out, err) = be_run_base_compaction(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + assertTrue(out.contains("invalid tablet state.")) + + + // cu compaction + tabletId = array[0].TabletId + logger.info("run compaction:" + tabletId) + (code, out, err) = be_run_cumulative_compaction(injectBe.Host, injectBe.HttpPort, tabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + + running = true + do { + Thread.sleep(100) + tabletId = array[0].TabletId + (code, out, err) = be_get_compaction_status(injectBe.Host, injectBe.HttpPort, tabletId) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } while (running) + + // new tablet cannot do cu compaction + tabletId = array[1].TabletId + logger.info("run compaction:" + tabletId) + (code, out, err) = be_run_cumulative_compaction(injectBe.Host, injectBe.HttpPort, tabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + assertTrue(out.contains("invalid tablet state.")) + + } finally { + if (injectBe != null) { + GetDebugPoint().disableDebugPointForAllBEs(injectName) + } + int max_try_time = 3000 + while (max_try_time--){ + result = getJobState("date") + if (result == "FINISHED" || result == "CANCELLED") { + sleep(3000) + break + } else { + sleep(100) + if (max_try_time < 1){ + assertEquals(1,2) + } + } + } + assertEquals(result, "FINISHED"); + def count = sql """ select count(*) from date; """ + assertEquals(count[0][0], 2556); + // check rowsets + logger.info("run show:" + originTabletId) + (code, out, err) = be_show_tablet_status(injectBe.Host, injectBe.HttpPort, originTabletId) + logger.info("Run show: code=" + code + ", out=" + out + ", err=" + err) + assertTrue(out.contains("[0-1]")) + assertTrue(out.contains("[2-7]")) + assertTrue(out.contains("[8-8]")) + assertTrue(out.contains("[9-13]")) + + logger.info("run show:" + newTabletId) + (code, out, err) = be_show_tablet_status(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Run show: code=" + code + ", out=" + out + ", err=" + err) + assertTrue(out.contains("[0-1]")) + assertTrue(out.contains("[2-2]")) + assertTrue(out.contains("[7-7]")) + assertTrue(out.contains("[8-8]")) + assertTrue(out.contains("[9-9]")) + assertTrue(out.contains("[13-13]")) + + // base compaction + logger.info("run compaction:" + newTabletId) + (code, out, err) = be_run_base_compaction(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + + + // wait for all compactions done + boolean running = true + while (running) { + Thread.sleep(100) + (code, out, err) = be_get_compaction_status(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } + + // cu compaction + logger.info("run compaction:" + newTabletId) + (code, out, err) = be_run_cumulative_compaction(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + + + // wait for all compactions done + running = true + while (running) { + Thread.sleep(100) + (code, out, err) = be_get_compaction_status(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } + + logger.info("run show:" + newTabletId) + (code, out, err) = be_show_tablet_status(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Run show: code=" + code + ", out=" + out + ", err=" + err) + assertTrue(out.contains("[0-1]")) + assertTrue(out.contains("[2-7]")) + assertTrue(out.contains("[8-13]")) + + for (int i = 0; i < 4; i++) { + load_date_once("date"); + } + + sql """ select count(*) from date """ + + logger.info("run compaction:" + newTabletId) + (code, out, err) = be_run_cumulative_compaction(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + + // wait for all compactions done + running = true + while (running) { + Thread.sleep(100) + (code, out, err) = be_get_compaction_status(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } + + logger.info("run show:" + newTabletId) + (code, out, err) = be_show_tablet_status(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Run show: code=" + code + ", out=" + out + ", err=" + err) + assertTrue(out.contains("[0-1]")) + assertTrue(out.contains("[2-7]")) + assertTrue(out.contains("[8-17]")) + } + } +} diff --git a/regression-test/suites/cloud_p0/schema_change/compaction2/test_schema_change_with_compaction2.groovy b/regression-test/suites/cloud_p0/schema_change/compaction2/test_schema_change_with_compaction2.groovy new file mode 100644 index 00000000000000..c8ca8a54109824 --- /dev/null +++ b/regression-test/suites/cloud_p0/schema_change/compaction2/test_schema_change_with_compaction2.groovy @@ -0,0 +1,214 @@ +// 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. + +// Most of the cases are copied from https://github.com/trinodb/trino/tree/master +// /testing/trino-product-tests/src/main/resources/sql-tests/testcases +// and modified by Doris. + +// Note: To filter out tables from sql files, use the following one-liner comamnd +// sed -nr 's/.*tables: (.*)$/\1/gp' /path/to/*.sql | sed -nr 's/,/\n/gp' | sort | uniq + +import org.apache.doris.regression.util.DebugPoint + +import org.apache.doris.regression.util.NodeType + +suite('test_schema_change_with_compaction2', 'nonConcurrent') { + def getJobState = { tableName -> + def jobStateResult = sql """ SHOW ALTER TABLE COLUMN WHERE IndexName='${tableName}' ORDER BY createtime DESC LIMIT 1 """ + return jobStateResult[0][9] + } + + def s3BucketName = getS3BucketName() + def s3WithProperties = """WITH S3 ( + |"AWS_ACCESS_KEY" = "${getS3AK()}", + |"AWS_SECRET_KEY" = "${getS3SK()}", + |"AWS_ENDPOINT" = "${getS3Endpoint()}", + |"AWS_REGION" = "${getS3Region()}", + |"provider" = "${getS3Provider()}") + |PROPERTIES( + |"exec_mem_limit" = "8589934592", + |"load_parallelism" = "3")""".stripMargin() + + // set fe configuration + sql "ADMIN SET FRONTEND CONFIG ('max_bytes_per_broker_scanner' = '161061273600')" + sql new File("""${context.file.parent}/../ddl/date_delete.sql""").text + def load_date_once = { String table -> + def uniqueID = Math.abs(UUID.randomUUID().hashCode()).toString() + def loadLabel = table + "_" + uniqueID + // load data from cos + def loadSql = new File("""${context.file.parent}/../ddl/${table}_load.sql""").text.replaceAll("\\\$\\{s3BucketName\\}", s3BucketName) + loadSql = loadSql.replaceAll("\\\$\\{loadLabel\\}", loadLabel) + s3WithProperties + sql loadSql + + // check load state + while (true) { + def stateResult = sql "show load where Label = '${loadLabel}'" + def loadState = stateResult[stateResult.size() - 1][2].toString() + if ("CANCELLED".equalsIgnoreCase(loadState)) { + throw new IllegalStateException("load ${loadLabel} failed.") + } else if ("FINISHED".equalsIgnoreCase(loadState)) { + break + } + sleep(5000) + } + } + + sql new File("""${context.file.parent}/../ddl/date_create.sql""").text + def injectName = 'CloudSchemaChangeJob.process_alter_tablet.sleep' + def injectBe = null + def backends = sql_return_maparray('show backends') + def array = sql_return_maparray("SHOW TABLETS FROM date") + def injectBeId = array[0].BackendId + def originTabletId = array[0].TabletId + injectBe = backends.stream().filter(be -> be.BackendId == injectBeId).findFirst().orElse(null) + assertNotNull(injectBe) + + def load_delete_compaction = { + load_date_once("date"); + sql "delete from date where d_datekey < 19900000" + sql "select count(*) from date" + // cu compaction + logger.info("run compaction:" + originTabletId) + (code, out, err) = be_run_cumulative_compaction(injectBe.Host, injectBe.HttpPort, originTabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + boolean running = true + do { + Thread.sleep(100) + (code, out, err) = be_get_compaction_status(injectBe.Host, injectBe.HttpPort, originTabletId) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } while (running) + } + + try { + load_delete_compaction() + load_delete_compaction() + load_delete_compaction() + + + sleep(1000) + + DebugPoint.enableDebugPoint(injectBe.Host, injectBe.HttpPort.toInteger(), NodeType.BE, injectName) + sql "ALTER TABLE date MODIFY COLUMN d_holidayfl bigint(11)" + sleep(5000) + array = sql_return_maparray("SHOW TABLETS FROM date") + + + // base compaction + logger.info("run compaction:" + originTabletId) + (code, out, err) = be_run_base_compaction(injectBe.Host, injectBe.HttpPort, originTabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + + + // wait for all compactions done + boolean running = true + while (running) { + Thread.sleep(100) + (code, out, err) = be_get_compaction_status(injectBe.Host, injectBe.HttpPort, originTabletId) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } + def newTabletId = array[1].TabletId + logger.info("run compaction:" + newTabletId) + (code, out, err) = be_run_base_compaction(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + assertTrue(out.contains("invalid tablet state.")) + + } finally { + if (injectBe != null) { + DebugPoint.disableDebugPoint(injectBe.Host, injectBe.HttpPort.toInteger(), NodeType.BE, injectName) + } + int max_try_time = 3000 + while (max_try_time--){ + result = getJobState("date") + if (result == "FINISHED") { + sleep(3000) + break + } else { + sleep(100) + if (max_try_time < 1){ + assertEquals(1,2) + } + } + } + for (int i = 0; i < 5; i++) { + load_date_once("date"); + } + def count = sql """ select count(*) from date; """ + assertEquals(count[0][0], 20448); + // check rowsets + logger.info("run show:" + originTabletId) + (code, out, err) = be_show_tablet_status(injectBe.Host, injectBe.HttpPort, originTabletId) + logger.info("Run show: code=" + code + ", out=" + out + ", err=" + err) + assertTrue(out.contains("[0-1]")) + assertTrue(out.contains("[2-7]")) + + logger.info("run show:" + newTabletId) + (code, out, err) = be_show_tablet_status(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Run show: code=" + code + ", out=" + out + ", err=" + err) + assertTrue(out.contains("[0-1]")) + assertTrue(out.contains("[2-2]")) + assertTrue(out.contains("[7-7]")) + + // base compaction + logger.info("run compaction:" + newTabletId) + (code, out, err) = be_run_base_compaction(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + + + // wait for all compactions done + boolean running = true + while (running) { + Thread.sleep(100) + (code, out, err) = be_get_compaction_status(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } + + logger.info("run compaction:" + newTabletId) + (code, out, err) = be_run_cumulative_compaction(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + + running = true + while (running) { + Thread.sleep(100) + (code, out, err) = be_get_compaction_status(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } + + logger.info("run show:" + newTabletId) + (code, out, err) = be_show_tablet_status(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Run show: code=" + code + ", out=" + out + ", err=" + err) + assertTrue(out.contains("[0-1]")) + assertTrue(out.contains("[2-7]")) + assertTrue(out.contains("[8-12]")) + } + +} \ No newline at end of file diff --git a/regression-test/suites/cloud_p0/schema_change/compaction3/test_schema_change_with_compaction3.groovy b/regression-test/suites/cloud_p0/schema_change/compaction3/test_schema_change_with_compaction3.groovy new file mode 100644 index 00000000000000..b2aab9f2dc7c84 --- /dev/null +++ b/regression-test/suites/cloud_p0/schema_change/compaction3/test_schema_change_with_compaction3.groovy @@ -0,0 +1,194 @@ +// 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. + +// Most of the cases are copied from https://github.com/trinodb/trino/tree/master +// /testing/trino-product-tests/src/main/resources/sql-tests/testcases +// and modified by Doris. + +// Note: To filter out tables from sql files, use the following one-liner comamnd +// sed -nr 's/.*tables: (.*)$/\1/gp' /path/to/*.sql | sed -nr 's/,/\n/gp' | sort | uniq + +import org.apache.doris.regression.util.DebugPoint + +import org.apache.doris.regression.util.NodeType + +suite('test_schema_change_with_compaction3', 'nonConcurrent') { + def getJobState = { tableName -> + def jobStateResult = sql """ SHOW ALTER TABLE COLUMN WHERE IndexName='${tableName}' ORDER BY createtime DESC LIMIT 1 """ + return jobStateResult[0][9] + } + + def s3BucketName = getS3BucketName() + def s3WithProperties = """WITH S3 ( + |"AWS_ACCESS_KEY" = "${getS3AK()}", + |"AWS_SECRET_KEY" = "${getS3SK()}", + |"AWS_ENDPOINT" = "${getS3Endpoint()}", + |"AWS_REGION" = "${getS3Region()}", + |"provider" = "${getS3Provider()}") + |PROPERTIES( + |"exec_mem_limit" = "8589934592", + |"load_parallelism" = "3")""".stripMargin() + + // set fe configuration + sql "ADMIN SET FRONTEND CONFIG ('max_bytes_per_broker_scanner' = '161061273600')" + sql new File("""${context.file.parent}/../ddl/date_delete.sql""").text + def load_date_once = { String table -> + def uniqueID = Math.abs(UUID.randomUUID().hashCode()).toString() + def loadLabel = table + "_" + uniqueID + // load data from cos + def loadSql = new File("""${context.file.parent}/../ddl/${table}_load.sql""").text.replaceAll("\\\$\\{s3BucketName\\}", s3BucketName) + loadSql = loadSql.replaceAll("\\\$\\{loadLabel\\}", loadLabel) + s3WithProperties + sql loadSql + + // check load state + while (true) { + def stateResult = sql "show load where Label = '${loadLabel}'" + def loadState = stateResult[stateResult.size() - 1][2].toString() + if ("CANCELLED".equalsIgnoreCase(loadState)) { + throw new IllegalStateException("load ${loadLabel} failed.") + } else if ("FINISHED".equalsIgnoreCase(loadState)) { + break + } + sleep(5000) + } + } + + sql new File("""${context.file.parent}/../ddl/date_create.sql""").text + def injectName = 'CloudSchemaChangeJob.process_alter_tablet.sleep' + def injectBe = null + def backends = sql_return_maparray('show backends') + def array = sql_return_maparray("SHOW TABLETS FROM date") + def injectBeId = array[0].BackendId + def originTabletId = array[0].TabletId + injectBe = backends.stream().filter(be -> be.BackendId == injectBeId).findFirst().orElse(null) + assertNotNull(injectBe) + + def load_delete_compaction = { + load_date_once("date"); + sql "delete from date where d_datekey < 19900000" + sql "select count(*) from date" + // cu compaction + logger.info("run compaction:" + originTabletId) + (code, out, err) = be_run_cumulative_compaction(injectBe.Host, injectBe.HttpPort, originTabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + boolean running = true + do { + Thread.sleep(100) + (code, out, err) = be_get_compaction_status(injectBe.Host, injectBe.HttpPort, originTabletId) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } while (running) + } + + try { + load_delete_compaction() + load_delete_compaction() + load_delete_compaction() + + load_date_once("date"); + + sleep(1000) + + DebugPoint.enableDebugPoint(injectBe.Host, injectBe.HttpPort.toInteger(), NodeType.BE, injectName) + sql "ALTER TABLE date MODIFY COLUMN d_holidayfl bigint(11)" + sleep(5000) + array = sql_return_maparray("SHOW TABLETS FROM date") + + for (int i = 0; i < 5; i++) { + load_date_once("date"); + } + + // base compaction + logger.info("run compaction:" + originTabletId) + (code, out, err) = be_run_base_compaction(injectBe.Host, injectBe.HttpPort, originTabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + + + // wait for all compactions done + boolean running = true + while (running) { + Thread.sleep(100) + (code, out, err) = be_get_compaction_status(injectBe.Host, injectBe.HttpPort, originTabletId) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } + def newTabletId = array[1].TabletId + logger.info("run compaction:" + newTabletId) + (code, out, err) = be_run_base_compaction(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + assertTrue(out.contains("invalid tablet state.")) + + + // cu compaction + for (int i = 0; i < array.size(); i++) { + tabletId = array[i].TabletId + logger.info("run compaction:" + tabletId) + (code, out, err) = be_run_cumulative_compaction(injectBe.Host, injectBe.HttpPort, tabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + } + + for (int i = 0; i < array.size(); i++) { + running = true + do { + Thread.sleep(100) + tabletId = array[i].TabletId + (code, out, err) = be_get_compaction_status(injectBe.Host, injectBe.HttpPort, tabletId) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } while (running) + } + } finally { + sql """ CANCEL ALTER TABLE COLUMN FROM date """ + if (injectBe != null) { + DebugPoint.disableDebugPoint(injectBe.Host, injectBe.HttpPort.toInteger(), NodeType.BE, injectName) + } + int max_try_time = 3000 + while (max_try_time--){ + result = getJobState("date") + if (result == "FINISHED" || result == "CANCELLED") { + sleep(3000) + break + } else { + sleep(100) + if (max_try_time < 1){ + assertEquals(1,2) + } + } + } + assertEquals(result, "CANCELLED"); + def count = sql """ select count(*) from date; """ + assertEquals(count[0][0], 23004); + // check rowsets + logger.info("run show:" + originTabletId) + (code, out, err) = be_show_tablet_status(injectBe.Host, injectBe.HttpPort, originTabletId) + logger.info("Run show: code=" + code + ", out=" + out + ", err=" + err) + assertTrue(out.contains("[0-1]")) + assertTrue(out.contains("[2-7]")) + assertTrue(out.contains("[8-8]")) + assertTrue(out.contains("[9-13]")) + } + +} \ No newline at end of file diff --git a/regression-test/suites/cloud_p0/schema_change/compaction4/test_schema_change_with_compaction4.groovy b/regression-test/suites/cloud_p0/schema_change/compaction4/test_schema_change_with_compaction4.groovy new file mode 100644 index 00000000000000..4b53dbdd998104 --- /dev/null +++ b/regression-test/suites/cloud_p0/schema_change/compaction4/test_schema_change_with_compaction4.groovy @@ -0,0 +1,194 @@ +// 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. + +// Most of the cases are copied from https://github.com/trinodb/trino/tree/master +// /testing/trino-product-tests/src/main/resources/sql-tests/testcases +// and modified by Doris. + +// Note: To filter out tables from sql files, use the following one-liner comamnd +// sed -nr 's/.*tables: (.*)$/\1/gp' /path/to/*.sql | sed -nr 's/,/\n/gp' | sort | uniq + +import org.apache.doris.regression.util.DebugPoint + +import org.apache.doris.regression.util.NodeType + +suite('test_schema_change_with_compaction4', 'nonConcurrent') { + def getJobState = { tableName -> + def jobStateResult = sql """ SHOW ALTER TABLE MATERIALIZED VIEW WHERE IndexName='${tableName}' ORDER BY createtime DESC LIMIT 1 """ + return jobStateResult[0][8] + } + + def s3BucketName = getS3BucketName() + def s3WithProperties = """WITH S3 ( + |"AWS_ACCESS_KEY" = "${getS3AK()}", + |"AWS_SECRET_KEY" = "${getS3SK()}", + |"AWS_ENDPOINT" = "${getS3Endpoint()}", + |"AWS_REGION" = "${getS3Region()}", + |"provider" = "${getS3Provider()}") + |PROPERTIES( + |"exec_mem_limit" = "8589934592", + |"load_parallelism" = "3")""".stripMargin() + + // set fe configuration + sql "ADMIN SET FRONTEND CONFIG ('max_bytes_per_broker_scanner' = '161061273600')" + sql new File("""${context.file.parent}/../ddl/date_delete.sql""").text + def load_date_once = { String table -> + def uniqueID = Math.abs(UUID.randomUUID().hashCode()).toString() + def loadLabel = table + "_" + uniqueID + // load data from cos + def loadSql = new File("""${context.file.parent}/../ddl/${table}_load.sql""").text.replaceAll("\\\$\\{s3BucketName\\}", s3BucketName) + loadSql = loadSql.replaceAll("\\\$\\{loadLabel\\}", loadLabel) + s3WithProperties + sql loadSql + + // check load state + while (true) { + def stateResult = sql "show load where Label = '${loadLabel}'" + def loadState = stateResult[stateResult.size() - 1][2].toString() + if ("CANCELLED".equalsIgnoreCase(loadState)) { + throw new IllegalStateException("load ${loadLabel} failed.") + } else if ("FINISHED".equalsIgnoreCase(loadState)) { + break + } + sleep(5000) + } + } + + sql new File("""${context.file.parent}/../ddl/date_create.sql""").text + def injectName = 'CloudSchemaChangeJob.process_alter_tablet.sleep' + def injectBe = null + def backends = sql_return_maparray('show backends') + def array = sql_return_maparray("SHOW TABLETS FROM date") + def injectBeId = array[0].BackendId + def originTabletId = array[0].TabletId + injectBe = backends.stream().filter(be -> be.BackendId == injectBeId).findFirst().orElse(null) + assertNotNull(injectBe) + + def load_delete_compaction = { + load_date_once("date"); + sql "delete from date where d_datekey < 19900000" + sql "select count(*) from date" + // cu compaction + logger.info("run compaction:" + originTabletId) + (code, out, err) = be_run_cumulative_compaction(injectBe.Host, injectBe.HttpPort, originTabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + boolean running = true + do { + Thread.sleep(100) + (code, out, err) = be_get_compaction_status(injectBe.Host, injectBe.HttpPort, originTabletId) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } while (running) + } + + try { + load_delete_compaction() + load_delete_compaction() + load_delete_compaction() + + load_date_once("date"); + + sleep(1000) + + DebugPoint.enableDebugPoint(injectBe.Host, injectBe.HttpPort.toInteger(), NodeType.BE, injectName) + sql " CREATE MATERIALIZED VIEW date_view as select d_datekey, sum(d_daynuminweek) from date group by d_datekey;" + sleep(5000) + array = sql_return_maparray("SHOW TABLETS FROM date") + + for (int i = 0; i < 5; i++) { + load_date_once("date"); + } + + // base compaction + logger.info("run compaction:" + originTabletId) + (code, out, err) = be_run_base_compaction(injectBe.Host, injectBe.HttpPort, originTabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + + + // wait for all compactions done + boolean running = true + while (running) { + Thread.sleep(100) + (code, out, err) = be_get_compaction_status(injectBe.Host, injectBe.HttpPort, originTabletId) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } + def newTabletId = array[1].TabletId + logger.info("run compaction:" + newTabletId) + (code, out, err) = be_run_base_compaction(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + assertTrue(out.contains("invalid tablet state.")) + + + // cu compaction + for (int i = 0; i < array.size(); i++) { + tabletId = array[i].TabletId + logger.info("run compaction:" + tabletId) + (code, out, err) = be_run_cumulative_compaction(injectBe.Host, injectBe.HttpPort, tabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + } + + for (int i = 0; i < array.size(); i++) { + running = true + do { + Thread.sleep(100) + tabletId = array[i].TabletId + (code, out, err) = be_get_compaction_status(injectBe.Host, injectBe.HttpPort, tabletId) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } while (running) + } + } finally { + sql """ CANCEL ALTER TABLE MATERIALIZED VIEW FROM date """ + if (injectBe != null) { + DebugPoint.disableDebugPoint(injectBe.Host, injectBe.HttpPort.toInteger(), NodeType.BE, injectName) + } + int max_try_time = 3000 + while (max_try_time--){ + result = getJobState("date") + if (result == "FINISHED" || result == "CANCELLED") { + sleep(3000) + break + } else { + sleep(100) + if (max_try_time < 1){ + assertEquals(1,2) + } + } + } + assertEquals(result, "CANCELLED"); + def count = sql """ select count(*) from date; """ + assertEquals(count[0][0], 23004); + // check rowsets + logger.info("run show:" + originTabletId) + (code, out, err) = be_show_tablet_status(injectBe.Host, injectBe.HttpPort, originTabletId) + logger.info("Run show: code=" + code + ", out=" + out + ", err=" + err) + assertTrue(out.contains("[0-1]")) + assertTrue(out.contains("[2-7]")) + assertTrue(out.contains("[8-8]")) + assertTrue(out.contains("[9-13]")) + } + +} \ No newline at end of file diff --git a/regression-test/suites/cloud_p0/schema_change/compaction5/test_schema_change_with_compaction5.groovy b/regression-test/suites/cloud_p0/schema_change/compaction5/test_schema_change_with_compaction5.groovy new file mode 100644 index 00000000000000..f5028ff9e818c3 --- /dev/null +++ b/regression-test/suites/cloud_p0/schema_change/compaction5/test_schema_change_with_compaction5.groovy @@ -0,0 +1,260 @@ +// 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. + +import org.apache.doris.regression.suite.ClusterOptions +import org.apache.http.NoHttpResponseException +import org.apache.doris.regression.util.DebugPoint +import org.apache.doris.regression.util.NodeType + +suite('test_schema_change_with_compaction5', 'nonConcurrent') { + def options = new ClusterOptions() + options.cloudMode = true + options.enableDebugPoints() + options.beConfigs += [ "enable_java_support=false" ] + options.beConfigs += [ "disable_auto_compaction=true" ] + options.beConfigs += [ "enable_new_tablet_do_compaction=true" ] + options.beNum = 1 + docker(options) { + def getJobState = { tableName -> + def jobStateResult = sql """ SHOW ALTER TABLE COLUMN WHERE IndexName='${tableName}' ORDER BY createtime DESC LIMIT 1 """ + return jobStateResult[0][9] + } + + def s3BucketName = getS3BucketName() + def s3WithProperties = """WITH S3 ( + |"AWS_ACCESS_KEY" = "${getS3AK()}", + |"AWS_SECRET_KEY" = "${getS3SK()}", + |"AWS_ENDPOINT" = "${getS3Endpoint()}", + |"AWS_REGION" = "${getS3Region()}", + |"provider" = "${getS3Provider()}") + |PROPERTIES( + |"exec_mem_limit" = "8589934592", + |"load_parallelism" = "3")""".stripMargin() + + // set fe configuration + sql "ADMIN SET FRONTEND CONFIG ('max_bytes_per_broker_scanner' = '161061273600')" + sql new File("""${context.file.parent}/../ddl/date_delete.sql""").text + def load_date_once = { String table -> + def uniqueID = Math.abs(UUID.randomUUID().hashCode()).toString() + def loadLabel = table + "_" + uniqueID + // load data from cos + def loadSql = new File("""${context.file.parent}/../ddl/${table}_load.sql""").text.replaceAll("\\\$\\{s3BucketName\\}", s3BucketName) + loadSql = loadSql.replaceAll("\\\$\\{loadLabel\\}", loadLabel) + s3WithProperties + sql loadSql + + // check load state + while (true) { + def stateResult = sql "show load where Label = '${loadLabel}'" + def loadState = stateResult[stateResult.size() - 1][2].toString() + if ("CANCELLED".equalsIgnoreCase(loadState)) { + throw new IllegalStateException("load ${loadLabel} failed.") + } else if ("FINISHED".equalsIgnoreCase(loadState)) { + break + } + sleep(5000) + } + } + + sql new File("""${context.file.parent}/../ddl/date_create.sql""").text + def injectName = 'CloudSchemaChangeJob.process_alter_tablet.sleep' + def injectBe = null + def backends = sql_return_maparray('show backends') + def array = sql_return_maparray("SHOW TABLETS FROM date") + def injectBeId = array[0].BackendId + def originTabletId = array[0].TabletId + injectBe = backends.stream().filter(be -> be.BackendId == injectBeId).findFirst().orElse(null) + assertNotNull(injectBe) + + def load_delete_compaction = { + load_date_once("date"); + sql "delete from date where d_datekey < 19900000" + sql "select count(*) from date" + // cu compaction + logger.info("run compaction:" + originTabletId) + (code, out, err) = be_run_cumulative_compaction(injectBe.Host, injectBe.HttpPort, originTabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + boolean running = true + do { + Thread.sleep(100) + (code, out, err) = be_get_compaction_status(injectBe.Host, injectBe.HttpPort, originTabletId) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } while (running) + } + + try { + load_delete_compaction() + load_delete_compaction() + load_delete_compaction() + + load_date_once("date"); + + sleep(1000) + GetDebugPoint().enableDebugPointForAllBEs(injectName) + sql "ALTER TABLE date MODIFY COLUMN d_holidayfl bigint(11)" + sleep(5000) + array = sql_return_maparray("SHOW TABLETS FROM date") + + for (int i = 0; i < 5; i++) { + load_date_once("date"); + } + // base compaction + logger.info("run compaction:" + originTabletId) + (code, out, err) = be_run_base_compaction(injectBe.Host, injectBe.HttpPort, originTabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + + + // wait for all compactions done + boolean running = true + while (running) { + Thread.sleep(100) + (code, out, err) = be_get_compaction_status(injectBe.Host, injectBe.HttpPort, originTabletId) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } + def newTabletId = array[1].TabletId + logger.info("run compaction:" + newTabletId) + (code, out, err) = be_run_base_compaction(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + assertTrue(out.contains("invalid tablet state.")) + + + // cu compaction + for (int i = 0; i < array.size(); i++) { + tabletId = array[i].TabletId + logger.info("run compaction:" + tabletId) + (code, out, err) = be_run_cumulative_compaction(injectBe.Host, injectBe.HttpPort, tabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + } + + for (int i = 0; i < array.size(); i++) { + running = true + do { + Thread.sleep(100) + tabletId = array[i].TabletId + (code, out, err) = be_get_compaction_status(injectBe.Host, injectBe.HttpPort, tabletId) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } while (running) + } + cluster.restartFrontends() + sleep(30000) + context.reconnectFe() + } finally { + if (injectBe != null) { + GetDebugPoint().disableDebugPointForAllBEs(injectName) + } + int max_try_time = 3000 + while (max_try_time--){ + result = getJobState("date") + if (result == "FINISHED" || result == "CANCELLED") { + sleep(3000) + break + } else { + sleep(100) + if (max_try_time < 1){ + assertEquals(1,2) + } + } + } + assertEquals(result, "FINISHED"); + def count = sql """ select count(*) from date; """ + assertEquals(count[0][0], 23004); + // check rowsets + logger.info("run show:" + originTabletId) + (code, out, err) = be_show_tablet_status(injectBe.Host, injectBe.HttpPort, originTabletId) + logger.info("Run show: code=" + code + ", out=" + out + ", err=" + err) + assertTrue(out.contains("[0-1]")) + assertTrue(out.contains("[2-7]")) + assertTrue(out.contains("[8-8]")) + assertTrue(out.contains("[9-13]")) + + logger.info("run show:" + newTabletId) + (code, out, err) = be_show_tablet_status(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Run show: code=" + code + ", out=" + out + ", err=" + err) + assertTrue(out.contains("[0-1]")) + assertTrue(out.contains("[2-2]")) + assertTrue(out.contains("[7-7]")) + assertTrue(out.contains("[8-8]")) + assertTrue(out.contains("[9-13]")) + + // base compaction + logger.info("run compaction:" + newTabletId) + (code, out, err) = be_run_base_compaction(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + + + // wait for all compactions done + boolean running = true + while (running) { + Thread.sleep(100) + (code, out, err) = be_get_compaction_status(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } + + logger.info("run show:" + newTabletId) + (code, out, err) = be_show_tablet_status(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Run show: code=" + code + ", out=" + out + ", err=" + err) + assertTrue(out.contains("[0-1]")) + assertTrue(out.contains("[2-7]")) + assertTrue(out.contains("[8-8]")) + assertTrue(out.contains("[9-13]")) + + for (int i = 0; i < 3; i++) { + load_date_once("date"); + } + + sql """ select count(*) from date """ + + logger.info("run compaction:" + newTabletId) + (code, out, err) = be_run_cumulative_compaction(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + + // wait for all compactions done + running = true + while (running) { + Thread.sleep(100) + (code, out, err) = be_get_compaction_status(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } + + logger.info("run show:" + newTabletId) + (code, out, err) = be_show_tablet_status(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Run show: code=" + code + ", out=" + out + ", err=" + err) + assertTrue(out.contains("[0-1]")) + assertTrue(out.contains("[2-7]")) + assertTrue(out.contains("[8-16]")) + } + } +} \ No newline at end of file diff --git a/regression-test/suites/cloud_p0/schema_change/compaction6/test_schema_change_with_compaction6.groovy b/regression-test/suites/cloud_p0/schema_change/compaction6/test_schema_change_with_compaction6.groovy new file mode 100644 index 00000000000000..951535433d1362 --- /dev/null +++ b/regression-test/suites/cloud_p0/schema_change/compaction6/test_schema_change_with_compaction6.groovy @@ -0,0 +1,263 @@ +// 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. + +import org.apache.doris.regression.suite.ClusterOptions +import org.apache.http.NoHttpResponseException +import org.apache.doris.regression.util.DebugPoint +import org.apache.doris.regression.util.NodeType + +suite('test_schema_change_with_compaction6', 'nonConcurrent') { + def options = new ClusterOptions() + options.cloudMode = true + options.enableDebugPoints() + options.beConfigs += [ "enable_java_support=false" ] + options.beConfigs += [ "disable_auto_compaction=true" ] + options.beConfigs += [ "enable_new_tablet_do_compaction=true" ] + options.beNum = 1 + docker(options) { + def getJobState = { tableName -> + def jobStateResult = sql """ SHOW ALTER TABLE COLUMN WHERE IndexName='${tableName}' ORDER BY createtime DESC LIMIT 1 """ + return jobStateResult[0][9] + } + + def s3BucketName = getS3BucketName() + def s3WithProperties = """WITH S3 ( + |"AWS_ACCESS_KEY" = "${getS3AK()}", + |"AWS_SECRET_KEY" = "${getS3SK()}", + |"AWS_ENDPOINT" = "${getS3Endpoint()}", + |"AWS_REGION" = "${getS3Region()}", + |"provider" = "${getS3Provider()}") + |PROPERTIES( + |"exec_mem_limit" = "8589934592", + |"load_parallelism" = "3")""".stripMargin() + + // set fe configuration + sql "ADMIN SET FRONTEND CONFIG ('max_bytes_per_broker_scanner' = '161061273600')" + sql new File("""${context.file.parent}/../ddl/date_delete.sql""").text + def load_date_once = { String table -> + def uniqueID = Math.abs(UUID.randomUUID().hashCode()).toString() + def loadLabel = table + "_" + uniqueID + // load data from cos + def loadSql = new File("""${context.file.parent}/../ddl/${table}_load.sql""").text.replaceAll("\\\$\\{s3BucketName\\}", s3BucketName) + loadSql = loadSql.replaceAll("\\\$\\{loadLabel\\}", loadLabel) + s3WithProperties + sql loadSql + + // check load state + while (true) { + def stateResult = sql "show load where Label = '${loadLabel}'" + def loadState = stateResult[stateResult.size() - 1][2].toString() + if ("CANCELLED".equalsIgnoreCase(loadState)) { + throw new IllegalStateException("load ${loadLabel} failed.") + } else if ("FINISHED".equalsIgnoreCase(loadState)) { + break + } + sleep(5000) + } + } + + sql new File("""${context.file.parent}/../ddl/date_create.sql""").text + def injectName = 'CloudSchemaChangeJob.process_alter_tablet.sleep' + def injectBe = null + def backends = sql_return_maparray('show backends') + def array = sql_return_maparray("SHOW TABLETS FROM date") + def injectBeId = array[0].BackendId + def originTabletId = array[0].TabletId + injectBe = backends.stream().filter(be -> be.BackendId == injectBeId).findFirst().orElse(null) + assertNotNull(injectBe) + + def load_delete_compaction = { + load_date_once("date"); + sql "delete from date where d_datekey < 19900000" + sql "select count(*) from date" + // cu compaction + logger.info("run compaction:" + originTabletId) + (code, out, err) = be_run_cumulative_compaction(injectBe.Host, injectBe.HttpPort, originTabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + boolean running = true + do { + Thread.sleep(100) + (code, out, err) = be_get_compaction_status(injectBe.Host, injectBe.HttpPort, originTabletId) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } while (running) + } + + try { + load_delete_compaction() + load_delete_compaction() + load_delete_compaction() + + load_date_once("date"); + + sleep(1000) + GetDebugPoint().enableDebugPointForAllBEs(injectName) + sql "ALTER TABLE date MODIFY COLUMN d_holidayfl bigint(11)" + sleep(5000) + array = sql_return_maparray("SHOW TABLETS FROM date") + + for (int i = 0; i < 5; i++) { + load_date_once("date"); + } + + cluster.restartBackends() + GetDebugPoint().enableDebugPointForAllBEs(injectName) + sleep(30000) + + // base compaction + logger.info("run compaction:" + originTabletId) + (code, out, err) = be_run_base_compaction(injectBe.Host, injectBe.HttpPort, originTabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + + + // wait for all compactions done + boolean running = true + while (running) { + Thread.sleep(100) + (code, out, err) = be_get_compaction_status(injectBe.Host, injectBe.HttpPort, originTabletId) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } + def newTabletId = array[1].TabletId + logger.info("run compaction:" + newTabletId) + (code, out, err) = be_run_base_compaction(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + assertTrue(out.contains("invalid tablet state.")) + + + // cu compaction + for (int i = 0; i < array.size(); i++) { + tabletId = array[i].TabletId + logger.info("run compaction:" + tabletId) + (code, out, err) = be_run_cumulative_compaction(injectBe.Host, injectBe.HttpPort, tabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + } + + for (int i = 0; i < array.size(); i++) { + running = true + do { + Thread.sleep(100) + tabletId = array[i].TabletId + (code, out, err) = be_get_compaction_status(injectBe.Host, injectBe.HttpPort, tabletId) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } while (running) + } + + } finally { + if (injectBe != null) { + GetDebugPoint().disableDebugPointForAllBEs(injectName) + } + int max_try_time = 3000 + while (max_try_time--){ + result = getJobState("date") + if (result == "FINISHED" || result == "CANCELLED") { + sleep(3000) + break + } else { + sleep(100) + if (max_try_time < 1){ + assertEquals(1,2) + } + } + } + assertEquals(result, "FINISHED"); + def count = sql """ select count(*) from date; """ + assertEquals(count[0][0], 23004); + // check rowsets + logger.info("run show:" + originTabletId) + (code, out, err) = be_show_tablet_status(injectBe.Host, injectBe.HttpPort, originTabletId) + logger.info("Run show: code=" + code + ", out=" + out + ", err=" + err) + assertTrue(out.contains("[0-1]")) + assertTrue(out.contains("[2-7]")) + assertTrue(out.contains("[8-8]")) + assertTrue(out.contains("[9-13]")) + + logger.info("run show:" + newTabletId) + (code, out, err) = be_show_tablet_status(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Run show: code=" + code + ", out=" + out + ", err=" + err) + assertTrue(out.contains("[0-1]")) + assertTrue(out.contains("[2-2]")) + assertTrue(out.contains("[7-7]")) + assertTrue(out.contains("[8-8]")) + assertTrue(out.contains("[9-13]")) + + // base compaction + logger.info("run compaction:" + newTabletId) + (code, out, err) = be_run_base_compaction(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + + + // wait for all compactions done + boolean running = true + while (running) { + Thread.sleep(100) + (code, out, err) = be_get_compaction_status(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } + + logger.info("run show:" + newTabletId) + (code, out, err) = be_show_tablet_status(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Run show: code=" + code + ", out=" + out + ", err=" + err) + assertTrue(out.contains("[0-1]")) + assertTrue(out.contains("[2-7]")) + assertTrue(out.contains("[8-8]")) + assertTrue(out.contains("[9-13]")) + + for (int i = 0; i < 3; i++) { + load_date_once("date"); + } + + sql """ select count(*) from date """ + + logger.info("run compaction:" + newTabletId) + (code, out, err) = be_run_cumulative_compaction(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + + // wait for all compactions done + running = true + while (running) { + Thread.sleep(100) + (code, out, err) = be_get_compaction_status(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } + + logger.info("run show:" + newTabletId) + (code, out, err) = be_show_tablet_status(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Run show: code=" + code + ", out=" + out + ", err=" + err) + assertTrue(out.contains("[0-1]")) + assertTrue(out.contains("[2-7]")) + assertTrue(out.contains("[8-16]")) + } + } +} \ No newline at end of file diff --git a/regression-test/suites/cloud_p0/schema_change/compaction7/test_schema_change_with_compaction7.groovy b/regression-test/suites/cloud_p0/schema_change/compaction7/test_schema_change_with_compaction7.groovy new file mode 100644 index 00000000000000..7291ea3a341e44 --- /dev/null +++ b/regression-test/suites/cloud_p0/schema_change/compaction7/test_schema_change_with_compaction7.groovy @@ -0,0 +1,256 @@ +// 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. + +// Most of the cases are copied from https://github.com/trinodb/trino/tree/master +// /testing/trino-product-tests/src/main/resources/sql-tests/testcases +// and modified by Doris. + +// Note: To filter out tables from sql files, use the following one-liner comamnd +// sed -nr 's/.*tables: (.*)$/\1/gp' /path/to/*.sql | sed -nr 's/,/\n/gp' | sort | uniq + +import org.apache.doris.regression.util.DebugPoint + +import org.apache.doris.regression.util.NodeType + +suite('test_schema_change_with_compaction7', 'nonConcurrent') { + def getJobState = { tableName -> + def jobStateResult = sql """ SHOW ALTER TABLE COLUMN WHERE IndexName='${tableName}' ORDER BY createtime DESC LIMIT 1 """ + return jobStateResult[0][9] + } + + def s3BucketName = getS3BucketName() + def s3WithProperties = """WITH S3 ( + |"AWS_ACCESS_KEY" = "${getS3AK()}", + |"AWS_SECRET_KEY" = "${getS3SK()}", + |"AWS_ENDPOINT" = "${getS3Endpoint()}", + |"AWS_REGION" = "${getS3Region()}", + |"provider" = "${getS3Provider()}") + |PROPERTIES( + |"exec_mem_limit" = "8589934592", + |"load_parallelism" = "3")""".stripMargin() + + // set fe configuration + sql "ADMIN SET FRONTEND CONFIG ('max_bytes_per_broker_scanner' = '161061273600')" + sql new File("""${context.file.parent}/../ddl/date_delete.sql""").text + def load_date_once = { String table -> + def uniqueID = Math.abs(UUID.randomUUID().hashCode()).toString() + def loadLabel = table + "_" + uniqueID + // load data from cos + def loadSql = new File("""${context.file.parent}/../ddl/${table}_load.sql""").text.replaceAll("\\\$\\{s3BucketName\\}", s3BucketName) + loadSql = loadSql.replaceAll("\\\$\\{loadLabel\\}", loadLabel) + s3WithProperties + sql loadSql + + // check load state + while (true) { + def stateResult = sql "show load where Label = '${loadLabel}'" + def loadState = stateResult[stateResult.size() - 1][2].toString() + if ("CANCELLED".equalsIgnoreCase(loadState)) { + throw new IllegalStateException("load ${loadLabel} failed.") + } else if ("FINISHED".equalsIgnoreCase(loadState)) { + break + } + sleep(5000) + } + } + + sql new File("""${context.file.parent}/../ddl/date_unique_create.sql""").text + def injectName = 'CloudSchemaChangeJob.process_alter_tablet.sleep' + def injectBe = null + def backends = sql_return_maparray('show backends') + def array = sql_return_maparray("SHOW TABLETS FROM date") + def injectBeId = array[0].BackendId + def originTabletId = array[0].TabletId + injectBe = backends.stream().filter(be -> be.BackendId == injectBeId).findFirst().orElse(null) + assertNotNull(injectBe) + + def load_delete_compaction = { + load_date_once("date"); + sql "delete from date where d_datekey < 19900000" + sql "select count(*) from date" + // cu compaction + logger.info("run compaction:" + originTabletId) + (code, out, err) = be_run_cumulative_compaction(injectBe.Host, injectBe.HttpPort, originTabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + boolean running = true + do { + Thread.sleep(100) + (code, out, err) = be_get_compaction_status(injectBe.Host, injectBe.HttpPort, originTabletId) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } while (running) + } + + try { + load_delete_compaction() + load_delete_compaction() + load_delete_compaction() + + load_date_once("date"); + + sleep(1000) + + DebugPoint.enableDebugPoint(injectBe.Host, injectBe.HttpPort.toInteger(), NodeType.BE, injectName) + sql "ALTER TABLE date MODIFY COLUMN d_holidayfl bigint(11)" + sleep(15000) + array = sql_return_maparray("SHOW TABLETS FROM date") + + for (int i = 0; i < 5; i++) { + load_date_once("date"); + } + // base compaction + logger.info("run compaction:" + originTabletId) + (code, out, err) = be_run_base_compaction(injectBe.Host, injectBe.HttpPort, originTabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + + + // wait for all compactions done + boolean running = true + while (running) { + Thread.sleep(100) + (code, out, err) = be_get_compaction_status(injectBe.Host, injectBe.HttpPort, originTabletId) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } + def newTabletId = array[1].TabletId + logger.info("run compaction:" + newTabletId) + (code, out, err) = be_run_base_compaction(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + assertTrue(out.contains("invalid tablet state.")) + + + // cu compaction + for (int i = 0; i < array.size(); i++) { + tabletId = array[i].TabletId + logger.info("run compaction:" + tabletId) + (code, out, err) = be_run_cumulative_compaction(injectBe.Host, injectBe.HttpPort, tabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + } + + for (int i = 0; i < array.size(); i++) { + running = true + do { + Thread.sleep(100) + tabletId = array[i].TabletId + (code, out, err) = be_get_compaction_status(injectBe.Host, injectBe.HttpPort, tabletId) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } while (running) + } + } finally { + if (injectBe != null) { + DebugPoint.disableDebugPoint(injectBe.Host, injectBe.HttpPort.toInteger(), NodeType.BE, injectName) + } + int max_try_time = 3000 + while (max_try_time--){ + result = getJobState("date") + if (result == "FINISHED" || result == "CANCELLED") { + sleep(3000) + break + } else { + sleep(100) + if (max_try_time < 1){ + assertEquals(1,2) + } + } + } + assertEquals(result, "FINISHED"); + def count = sql """ select count(*) from date; """ + assertEquals(count[0][0], 2556); + // check rowsets + logger.info("run show:" + originTabletId) + (code, out, err) = be_show_tablet_status(injectBe.Host, injectBe.HttpPort, originTabletId) + logger.info("Run show: code=" + code + ", out=" + out + ", err=" + err) + assertTrue(out.contains("[0-1]")) + assertTrue(out.contains("[2-7]")) + assertTrue(out.contains("[8-8]")) + assertTrue(out.contains("[9-13]")) + + logger.info("run show:" + newTabletId) + (code, out, err) = be_show_tablet_status(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Run show: code=" + code + ", out=" + out + ", err=" + err) + assertTrue(out.contains("[0-1]")) + assertTrue(out.contains("[2-2]")) + assertTrue(out.contains("[7-7]")) + assertTrue(out.contains("[8-8]")) + assertTrue(out.contains("[9-13]")) + + // base compaction + logger.info("run compaction:" + newTabletId) + (code, out, err) = be_run_base_compaction(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + + + // wait for all compactions done + boolean running = true + while (running) { + Thread.sleep(100) + (code, out, err) = be_get_compaction_status(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } + + logger.info("run show:" + newTabletId) + (code, out, err) = be_show_tablet_status(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Run show: code=" + code + ", out=" + out + ", err=" + err) + assertTrue(out.contains("[0-1]")) + assertTrue(out.contains("[2-7]")) + assertTrue(out.contains("[8-8]")) + assertTrue(out.contains("[9-13]")) + + for (int i = 0; i < 3; i++) { + load_date_once("date"); + } + + sql """ select count(*) from date """ + + logger.info("run compaction:" + newTabletId) + (code, out, err) = be_run_cumulative_compaction(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + + // wait for all compactions done + running = true + while (running) { + Thread.sleep(100) + (code, out, err) = be_get_compaction_status(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } + + logger.info("run show:" + newTabletId) + (code, out, err) = be_show_tablet_status(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Run show: code=" + code + ", out=" + out + ", err=" + err) + assertTrue(out.contains("[0-1]")) + assertTrue(out.contains("[2-7]")) + assertTrue(out.contains("[8-16]")) + } + +} \ No newline at end of file diff --git a/regression-test/suites/cloud_p0/schema_change/compaction8/test_schema_change_with_compaction8.groovy b/regression-test/suites/cloud_p0/schema_change/compaction8/test_schema_change_with_compaction8.groovy new file mode 100644 index 00000000000000..1017e1d50f235f --- /dev/null +++ b/regression-test/suites/cloud_p0/schema_change/compaction8/test_schema_change_with_compaction8.groovy @@ -0,0 +1,214 @@ +// 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. + +// Most of the cases are copied from https://github.com/trinodb/trino/tree/master +// /testing/trino-product-tests/src/main/resources/sql-tests/testcases +// and modified by Doris. + +// Note: To filter out tables from sql files, use the following one-liner comamnd +// sed -nr 's/.*tables: (.*)$/\1/gp' /path/to/*.sql | sed -nr 's/,/\n/gp' | sort | uniq + +import org.apache.doris.regression.util.DebugPoint + +import org.apache.doris.regression.util.NodeType + +suite('test_schema_change_with_compaction8', 'nonConcurrent') { + def getJobState = { tableName -> + def jobStateResult = sql """ SHOW ALTER TABLE COLUMN WHERE IndexName='${tableName}' ORDER BY createtime DESC LIMIT 1 """ + return jobStateResult[0][9] + } + + def s3BucketName = getS3BucketName() + def s3WithProperties = """WITH S3 ( + |"AWS_ACCESS_KEY" = "${getS3AK()}", + |"AWS_SECRET_KEY" = "${getS3SK()}", + |"AWS_ENDPOINT" = "${getS3Endpoint()}", + |"AWS_REGION" = "${getS3Region()}", + |"provider" = "${getS3Provider()}") + |PROPERTIES( + |"exec_mem_limit" = "8589934592", + |"load_parallelism" = "3")""".stripMargin() + + // set fe configuration + sql "ADMIN SET FRONTEND CONFIG ('max_bytes_per_broker_scanner' = '161061273600')" + sql new File("""${context.file.parent}/../ddl/date_delete.sql""").text + def load_date_once = { String table -> + def uniqueID = Math.abs(UUID.randomUUID().hashCode()).toString() + def loadLabel = table + "_" + uniqueID + // load data from cos + def loadSql = new File("""${context.file.parent}/../ddl/${table}_load.sql""").text.replaceAll("\\\$\\{s3BucketName\\}", s3BucketName) + loadSql = loadSql.replaceAll("\\\$\\{loadLabel\\}", loadLabel) + s3WithProperties + sql loadSql + + // check load state + while (true) { + def stateResult = sql "show load where Label = '${loadLabel}'" + def loadState = stateResult[stateResult.size() - 1][2].toString() + if ("CANCELLED".equalsIgnoreCase(loadState)) { + throw new IllegalStateException("load ${loadLabel} failed.") + } else if ("FINISHED".equalsIgnoreCase(loadState)) { + break + } + sleep(5000) + } + } + + sql new File("""${context.file.parent}/../ddl/date_unique_create.sql""").text + def injectName = 'CloudSchemaChangeJob.process_alter_tablet.sleep' + def injectBe = null + def backends = sql_return_maparray('show backends') + def array = sql_return_maparray("SHOW TABLETS FROM date") + def injectBeId = array[0].BackendId + def originTabletId = array[0].TabletId + injectBe = backends.stream().filter(be -> be.BackendId == injectBeId).findFirst().orElse(null) + assertNotNull(injectBe) + + def load_delete_compaction = { + load_date_once("date"); + sql "delete from date where d_datekey < 19900000" + sql "select count(*) from date" + // cu compaction + logger.info("run compaction:" + originTabletId) + (code, out, err) = be_run_cumulative_compaction(injectBe.Host, injectBe.HttpPort, originTabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + boolean running = true + do { + Thread.sleep(100) + (code, out, err) = be_get_compaction_status(injectBe.Host, injectBe.HttpPort, originTabletId) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } while (running) + } + + try { + load_delete_compaction() + load_delete_compaction() + load_delete_compaction() + + + sleep(1000) + + DebugPoint.enableDebugPoint(injectBe.Host, injectBe.HttpPort.toInteger(), NodeType.BE, injectName) + sql "ALTER TABLE date MODIFY COLUMN d_holidayfl bigint(11)" + sleep(5000) + array = sql_return_maparray("SHOW TABLETS FROM date") + + + // base compaction + logger.info("run compaction:" + originTabletId) + (code, out, err) = be_run_base_compaction(injectBe.Host, injectBe.HttpPort, originTabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + + + // wait for all compactions done + boolean running = true + while (running) { + Thread.sleep(100) + (code, out, err) = be_get_compaction_status(injectBe.Host, injectBe.HttpPort, originTabletId) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } + def newTabletId = array[1].TabletId + logger.info("run compaction:" + newTabletId) + (code, out, err) = be_run_base_compaction(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + assertTrue(out.contains("invalid tablet state.")) + + } finally { + if (injectBe != null) { + DebugPoint.disableDebugPoint(injectBe.Host, injectBe.HttpPort.toInteger(), NodeType.BE, injectName) + } + int max_try_time = 3000 + while (max_try_time--){ + result = getJobState("date") + if (result == "FINISHED") { + sleep(3000) + break + } else { + sleep(100) + if (max_try_time < 1){ + assertEquals(1,2) + } + } + } + for (int i = 0; i < 5; i++) { + load_date_once("date"); + } + def count = sql """ select count(*) from date; """ + assertEquals(count[0][0], 2556); + // check rowsets + logger.info("run show:" + originTabletId) + (code, out, err) = be_show_tablet_status(injectBe.Host, injectBe.HttpPort, originTabletId) + logger.info("Run show: code=" + code + ", out=" + out + ", err=" + err) + assertTrue(out.contains("[0-1]")) + assertTrue(out.contains("[2-7]")) + + logger.info("run show:" + newTabletId) + (code, out, err) = be_show_tablet_status(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Run show: code=" + code + ", out=" + out + ", err=" + err) + assertTrue(out.contains("[0-1]")) + assertTrue(out.contains("[2-2]")) + assertTrue(out.contains("[7-7]")) + + // base compaction + logger.info("run compaction:" + newTabletId) + (code, out, err) = be_run_base_compaction(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + + + // wait for all compactions done + boolean running = true + while (running) { + Thread.sleep(100) + (code, out, err) = be_get_compaction_status(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } + + logger.info("run compaction:" + newTabletId) + (code, out, err) = be_run_cumulative_compaction(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + + running = true + while (running) { + Thread.sleep(100) + (code, out, err) = be_get_compaction_status(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } + + logger.info("run show:" + newTabletId) + (code, out, err) = be_show_tablet_status(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Run show: code=" + code + ", out=" + out + ", err=" + err) + assertTrue(out.contains("[0-1]")) + assertTrue(out.contains("[2-7]")) + assertTrue(out.contains("[8-12]")) + } + +} \ No newline at end of file diff --git a/regression-test/suites/cloud_p0/schema_change/compaction9/test_schema_change_with_compaction9.groovy b/regression-test/suites/cloud_p0/schema_change/compaction9/test_schema_change_with_compaction9.groovy new file mode 100644 index 00000000000000..83c549eefc5abd --- /dev/null +++ b/regression-test/suites/cloud_p0/schema_change/compaction9/test_schema_change_with_compaction9.groovy @@ -0,0 +1,260 @@ +// 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. + +import org.apache.doris.regression.suite.ClusterOptions +import org.apache.http.NoHttpResponseException +import org.apache.doris.regression.util.DebugPoint +import org.apache.doris.regression.util.NodeType + +suite('test_schema_change_with_compaction9') { + def options = new ClusterOptions() + options.cloudMode = true + options.enableDebugPoints() + options.beConfigs += [ "enable_java_support=false" ] + options.beConfigs += [ "disable_auto_compaction=true" ] + options.beConfigs += [ "enable_new_tablet_do_compaction=true" ] + options.beNum = 1 + docker(options) { + def getJobState = { tableName -> + def jobStateResult = sql """ SHOW ALTER TABLE COLUMN WHERE IndexName='${tableName}' ORDER BY createtime DESC LIMIT 1 """ + return jobStateResult[0][9] + } + + def s3BucketName = getS3BucketName() + def s3WithProperties = """WITH S3 ( + |"AWS_ACCESS_KEY" = "${getS3AK()}", + |"AWS_SECRET_KEY" = "${getS3SK()}", + |"AWS_ENDPOINT" = "${getS3Endpoint()}", + |"AWS_REGION" = "${getS3Region()}", + |"provider" = "${getS3Provider()}") + |PROPERTIES( + |"exec_mem_limit" = "8589934592", + |"load_parallelism" = "3")""".stripMargin() + + // set fe configuration + sql "ADMIN SET FRONTEND CONFIG ('max_bytes_per_broker_scanner' = '161061273600')" + sql new File("""${context.file.parent}/../ddl/date_delete.sql""").text + def load_date_once = { String table -> + def uniqueID = Math.abs(UUID.randomUUID().hashCode()).toString() + def loadLabel = table + "_" + uniqueID + // load data from cos + def loadSql = new File("""${context.file.parent}/../ddl/${table}_load.sql""").text.replaceAll("\\\$\\{s3BucketName\\}", s3BucketName) + loadSql = loadSql.replaceAll("\\\$\\{loadLabel\\}", loadLabel) + s3WithProperties + sql loadSql + + // check load state + while (true) { + def stateResult = sql "show load where Label = '${loadLabel}'" + def loadState = stateResult[stateResult.size() - 1][2].toString() + if ("CANCELLED".equalsIgnoreCase(loadState)) { + throw new IllegalStateException("load ${loadLabel} failed.") + } else if ("FINISHED".equalsIgnoreCase(loadState)) { + break + } + sleep(5000) + } + } + + sql new File("""${context.file.parent}/../ddl/date_unique_create.sql""").text + def injectName = 'CloudSchemaChangeJob.process_alter_tablet.sleep' + def injectBe = null + def backends = sql_return_maparray('show backends') + def array = sql_return_maparray("SHOW TABLETS FROM date") + def injectBeId = array[0].BackendId + def originTabletId = array[0].TabletId + injectBe = backends.stream().filter(be -> be.BackendId == injectBeId).findFirst().orElse(null) + assertNotNull(injectBe) + + def load_delete_compaction = { + load_date_once("date"); + sql "delete from date where d_datekey < 19900000" + sql "select count(*) from date" + // cu compaction + logger.info("run compaction:" + originTabletId) + (code, out, err) = be_run_cumulative_compaction(injectBe.Host, injectBe.HttpPort, originTabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + boolean running = true + do { + Thread.sleep(100) + (code, out, err) = be_get_compaction_status(injectBe.Host, injectBe.HttpPort, originTabletId) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } while (running) + } + + try { + load_delete_compaction() + load_delete_compaction() + load_delete_compaction() + + load_date_once("date"); + + sleep(1000) + GetDebugPoint().enableDebugPointForAllBEs(injectName) + sql "ALTER TABLE date MODIFY COLUMN d_holidayfl bigint(11)" + sleep(5000) + array = sql_return_maparray("SHOW TABLETS FROM date") + + for (int i = 0; i < 5; i++) { + load_date_once("date"); + } + // base compaction + logger.info("run compaction:" + originTabletId) + (code, out, err) = be_run_base_compaction(injectBe.Host, injectBe.HttpPort, originTabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + + + // wait for all compactions done + boolean running = true + while (running) { + Thread.sleep(100) + (code, out, err) = be_get_compaction_status(injectBe.Host, injectBe.HttpPort, originTabletId) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } + def newTabletId = array[1].TabletId + logger.info("run compaction:" + newTabletId) + (code, out, err) = be_run_base_compaction(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + assertTrue(out.contains("invalid tablet state.")) + + + // cu compaction + for (int i = 0; i < array.size(); i++) { + tabletId = array[i].TabletId + logger.info("run compaction:" + tabletId) + (code, out, err) = be_run_cumulative_compaction(injectBe.Host, injectBe.HttpPort, tabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + } + + for (int i = 0; i < array.size(); i++) { + running = true + do { + Thread.sleep(100) + tabletId = array[i].TabletId + (code, out, err) = be_get_compaction_status(injectBe.Host, injectBe.HttpPort, tabletId) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } while (running) + } + cluster.restartFrontends() + sleep(30000) + context.reconnectFe() + } finally { + if (injectBe != null) { + GetDebugPoint().disableDebugPointForAllBEs(injectName) + } + int max_try_time = 3000 + while (max_try_time--){ + result = getJobState("date") + if (result == "FINISHED" || result == "CANCELLED") { + sleep(3000) + break + } else { + sleep(100) + if (max_try_time < 1){ + assertEquals(1,2) + } + } + } + assertEquals(result, "FINISHED"); + def count = sql """ select count(*) from date; """ + assertEquals(count[0][0], 2556); + // check rowsets + logger.info("run show:" + originTabletId) + (code, out, err) = be_show_tablet_status(injectBe.Host, injectBe.HttpPort, originTabletId) + logger.info("Run show: code=" + code + ", out=" + out + ", err=" + err) + assertTrue(out.contains("[0-1]")) + assertTrue(out.contains("[2-7]")) + assertTrue(out.contains("[8-8]")) + assertTrue(out.contains("[9-13]")) + + logger.info("run show:" + newTabletId) + (code, out, err) = be_show_tablet_status(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Run show: code=" + code + ", out=" + out + ", err=" + err) + assertTrue(out.contains("[0-1]")) + assertTrue(out.contains("[2-2]")) + assertTrue(out.contains("[7-7]")) + assertTrue(out.contains("[8-8]")) + assertTrue(out.contains("[9-13]")) + + // base compaction + logger.info("run compaction:" + newTabletId) + (code, out, err) = be_run_base_compaction(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + + + // wait for all compactions done + boolean running = true + while (running) { + Thread.sleep(100) + (code, out, err) = be_get_compaction_status(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } + + logger.info("run show:" + newTabletId) + (code, out, err) = be_show_tablet_status(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Run show: code=" + code + ", out=" + out + ", err=" + err) + assertTrue(out.contains("[0-1]")) + assertTrue(out.contains("[2-7]")) + assertTrue(out.contains("[8-8]")) + assertTrue(out.contains("[9-13]")) + + for (int i = 0; i < 3; i++) { + load_date_once("date"); + } + + sql """ select count(*) from date """ + + logger.info("run compaction:" + newTabletId) + (code, out, err) = be_run_cumulative_compaction(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + + // wait for all compactions done + running = true + while (running) { + Thread.sleep(100) + (code, out, err) = be_get_compaction_status(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } + + logger.info("run show:" + newTabletId) + (code, out, err) = be_show_tablet_status(injectBe.Host, injectBe.HttpPort, newTabletId) + logger.info("Run show: code=" + code + ", out=" + out + ", err=" + err) + assertTrue(out.contains("[0-1]")) + assertTrue(out.contains("[2-7]")) + assertTrue(out.contains("[8-16]")) + } + } +} \ No newline at end of file diff --git a/regression-test/suites/cloud_p0/schema_change/ddl/date_create.sql b/regression-test/suites/cloud_p0/schema_change/ddl/date_create.sql new file mode 100644 index 00000000000000..99c85399c123b2 --- /dev/null +++ b/regression-test/suites/cloud_p0/schema_change/ddl/date_create.sql @@ -0,0 +1,24 @@ +CREATE TABLE IF NOT EXISTS `date` ( + `d_datekey` int(11) NOT NULL COMMENT "", + `d_date` varchar(20) NOT NULL COMMENT "", + `d_dayofweek` varchar(10) NOT NULL COMMENT "", + `d_month` varchar(11) NOT NULL COMMENT "", + `d_year` int(11) NOT NULL COMMENT "", + `d_yearmonthnum` int(11) NOT NULL COMMENT "", + `d_yearmonth` varchar(9) NOT NULL COMMENT "", + `d_daynuminweek` int(11) NOT NULL COMMENT "", + `d_daynuminmonth` int(11) NOT NULL COMMENT "", + `d_daynuminyear` int(11) NOT NULL COMMENT "", + `d_monthnuminyear` int(11) NOT NULL COMMENT "", + `d_weeknuminyear` int(11) NOT NULL COMMENT "", + `d_sellingseason` varchar(14) NOT NULL COMMENT "", + `d_lastdayinweekfl` int(11) NOT NULL COMMENT "", + `d_lastdayinmonthfl` int(11) NOT NULL COMMENT "", + `d_holidayfl` int(11) NOT NULL COMMENT "", + `d_weekdayfl` int(11) NOT NULL COMMENT "" +) +DISTRIBUTED BY HASH(`d_datekey`) BUCKETS 1 +PROPERTIES ( +"replication_num" = "1", +"disable_auto_compaction" = "true" +); diff --git a/regression-test/suites/cloud_p0/schema_change/ddl/date_delete.sql b/regression-test/suites/cloud_p0/schema_change/ddl/date_delete.sql new file mode 100644 index 00000000000000..41702d336d7e7f --- /dev/null +++ b/regression-test/suites/cloud_p0/schema_change/ddl/date_delete.sql @@ -0,0 +1 @@ +drop table if exists `date` FORCE; \ No newline at end of file diff --git a/regression-test/suites/cloud_p0/schema_change/ddl/date_load.sql b/regression-test/suites/cloud_p0/schema_change/ddl/date_load.sql new file mode 100644 index 00000000000000..3e1511ca69a67a --- /dev/null +++ b/regression-test/suites/cloud_p0/schema_change/ddl/date_load.sql @@ -0,0 +1,6 @@ +LOAD LABEL ${loadLabel} ( + DATA INFILE("s3://${s3BucketName}/regression/ssb/sf100/date.tbl.gz") + INTO TABLE date + COLUMNS TERMINATED BY "|" + (d_datekey,d_date,d_dayofweek,d_month,d_year,d_yearmonthnum,d_yearmonth, d_daynuminweek,d_daynuminmonth,d_daynuminyear,d_monthnuminyear,d_weeknuminyear, d_sellingseason,d_lastdayinweekfl,d_lastdayinmonthfl,d_holidayfl,d_weekdayfl,temp) +) diff --git a/regression-test/suites/cloud_p0/schema_change/ddl/date_unique_create.sql b/regression-test/suites/cloud_p0/schema_change/ddl/date_unique_create.sql new file mode 100644 index 00000000000000..6138cb213a2063 --- /dev/null +++ b/regression-test/suites/cloud_p0/schema_change/ddl/date_unique_create.sql @@ -0,0 +1,27 @@ +CREATE TABLE IF NOT EXISTS `date` ( + `d_datekey` int(11) NOT NULL COMMENT "", + `d_date` varchar(20) NOT NULL COMMENT "", + `d_dayofweek` varchar(10) NOT NULL COMMENT "", + `d_month` varchar(11) NOT NULL COMMENT "", + `d_year` int(11) NOT NULL COMMENT "", + `d_yearmonthnum` int(11) NOT NULL COMMENT "", + `d_yearmonth` varchar(9) NOT NULL COMMENT "", + `d_daynuminweek` int(11) NOT NULL COMMENT "", + `d_daynuminmonth` int(11) NOT NULL COMMENT "", + `d_daynuminyear` int(11) NOT NULL COMMENT "", + `d_monthnuminyear` int(11) NOT NULL COMMENT "", + `d_weeknuminyear` int(11) NOT NULL COMMENT "", + `d_sellingseason` varchar(14) NOT NULL COMMENT "", + `d_lastdayinweekfl` int(11) NOT NULL COMMENT "", + `d_lastdayinmonthfl` int(11) NOT NULL COMMENT "", + `d_holidayfl` int(11) NOT NULL COMMENT "", + `d_weekdayfl` int(11) NOT NULL COMMENT "" +) +UNIQUE KEY (`d_datekey`) +DISTRIBUTED BY HASH(`d_datekey`) BUCKETS 1 +PROPERTIES ( +"replication_num" = "1", +"enable_unique_key_merge_on_write" = "true", +"enable_mow_light_delete" = "true", +"disable_auto_compaction" = "true" +);