diff --git a/be/src/cloud/cloud_base_compaction.cpp b/be/src/cloud/cloud_base_compaction.cpp index 23d4850d0002e8..81c1d47b7461e5 100644 --- a/be/src/cloud/cloud_base_compaction.cpp +++ b/be/src/cloud/cloud_base_compaction.cpp @@ -29,7 +29,6 @@ #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; @@ -83,18 +82,14 @@ 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 @@ -102,21 +97,6 @@ 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; } @@ -334,22 +314,6 @@ 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 ea6062309f28c7..cc84dce1b58840 100644 --- a/be/src/cloud/cloud_cumulative_compaction.cpp +++ b/be/src/cloud/cloud_cumulative_compaction.cpp @@ -48,9 +48,7 @@ CloudCumulativeCompaction::CloudCumulativeCompaction(CloudStorageEngine& engine, CloudCumulativeCompaction::~CloudCumulativeCompaction() = default; Status CloudCumulativeCompaction::prepare_compact() { - if (_tablet->tablet_state() != TABLET_RUNNING && - (!config::enable_new_tablet_do_compaction || - static_cast(_tablet.get())->alter_version() == -1)) { + if (_tablet->tablet_state() != TABLET_RUNNING) { return Status::InternalError("invalid tablet state. tablet_id={}", _tablet->tablet_id()); } @@ -112,11 +110,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); - - 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); + 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()); + } cloud::StartTabletJobResponse resp; st = _engine.meta_mgr().prepare_tablet_job(job, &resp); if (!st.ok()) { @@ -143,18 +141,6 @@ 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; } @@ -270,27 +256,12 @@ 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(); { @@ -373,9 +344,8 @@ 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( - std::max(cloud_tablet()->cumulative_layer_point(), _max_conflict_version + 1), - cloud_tablet()->alter_version() + 1); + int64_t candidate_version = + std::max(cloud_tablet()->cumulative_layer_point(), _max_conflict_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 b8acdb6bd38b9e..22f6689ff23782 100644 --- a/be/src/cloud/cloud_engine_calc_delete_bitmap_task.cpp +++ b/be/src/cloud/cloud_engine_calc_delete_bitmap_task.cpp @@ -154,21 +154,19 @@ Status CloudTabletCalcDeleteBitmapTask::handle() const { }; if (_version != max_version + 1 || should_sync_rowsets_produced_by_compaction()) { auto sync_st = tablet->sync_rowsets(); - 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]] { + 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 Status::Error( - "invalid tablet state {}. tablet_id={}", tablet->tablet_state(), - tablet->tablet_id()); + 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; } } auto sync_rowset_time_us = MonotonicMicros() - t2; diff --git a/be/src/cloud/cloud_meta_mgr.cpp b/be/src/cloud/cloud_meta_mgr.cpp index d59a8b98764ba5..e743ea9b12c8ce 100644 --- a/be/src/cloud/cloud_meta_mgr.cpp +++ b/be/src/cloud/cloud_meta_mgr.cpp @@ -448,10 +448,7 @@ 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 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) { + if (tablet->enable_unique_key_merge_on_write()) { 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 9a9ce5ae8f0c20..ed3e5f9433fcfd 100644 --- a/be/src/cloud/cloud_schema_change_job.cpp +++ b/be/src/cloud/cloud_schema_change_job.cpp @@ -19,9 +19,7 @@ #include -#include #include -#include #include "cloud/cloud_meta_mgr.h" #include "cloud/cloud_tablet_mgr.h" @@ -36,7 +34,6 @@ #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; @@ -62,6 +59,10 @@ 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) { @@ -85,54 +86,13 @@ 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, start_resp.alter_version()}, - &rs_splits, false)); + RETURN_IF_ERROR(_base_tablet->capture_rs_readers({2, base_max_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 @@ -150,8 +110,7 @@ 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, - start_resp.alter_version())); + RETURN_IF_ERROR(delete_handler.init(_base_tablet_schema, delete_predicates, base_max_version)); std::vector return_columns; return_columns.resize(_base_tablet_schema->num_columns()); @@ -168,7 +127,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, start_resp.alter_version()); + reader_context.version = Version(0, base_max_version); for (auto& split : rs_splits) { RETURN_IF_ERROR(split.rs_reader->init(&reader_context)); @@ -197,7 +156,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, job); + return _convert_historical_rowsets(sc_params); } for (auto item : request.materialized_view_params) { AlterMaterializedViewParam mv_param; @@ -217,11 +176,10 @@ 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, job); + return _convert_historical_rowsets(sc_params); } -Status CloudSchemaChangeJob::_convert_historical_rowsets(const SchemaChangeParams& sc_params, - cloud::TabletJobInfoPB& job) { +Status CloudSchemaChangeJob::_convert_historical_rowsets(const SchemaChangeParams& sc_params) { 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; @@ -252,6 +210,36 @@ 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) { @@ -329,8 +317,10 @@ Status CloudSchemaChangeJob::_convert_historical_rowsets(const SchemaChangeParam VLOG_TRACE << "Successfully convert a history version " << rs_reader->version(); } - auto* sc_job = job.mutable_schema_change(); - if (!sc_params.ref_rowset_readers.empty()) { + + if (sc_params.ref_rowset_readers.empty()) { + sc_job->set_alter_version(1); // no rowset to convert implies alter_version == 1 + } else { int64_t num_output_rows = 0; int64_t size_output_rowsets = 0; int64_t num_output_segments = 0; @@ -345,11 +335,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()) & @@ -364,7 +354,7 @@ Status CloudSchemaChangeJob::_convert_historical_rowsets(const SchemaChangeParam } cloud::FinishTabletJobResponse finish_resp; - auto st = _cloud_storage_engine.meta_mgr().commit_tablet_job(job, &finish_resp); + 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 c77aae4857049d..d587111df717a3 100644 --- a/be/src/cloud/cloud_schema_change_job.h +++ b/be/src/cloud/cloud_schema_change_job.h @@ -37,8 +37,7 @@ class CloudSchemaChangeJob { Status process_alter_tablet(const TAlterTabletReqV2& request); private: - Status _convert_historical_rowsets(const SchemaChangeParams& sc_params, - cloud::TabletJobInfoPB& job); + Status _convert_historical_rowsets(const SchemaChangeParams& sc_params); 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 bd88aee3109cad..8062631af1fc29 100644 --- a/be/src/cloud/cloud_storage_engine.cpp +++ b/be/src/cloud/cloud_storage_engine.cpp @@ -558,21 +558,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.contains(t->tablet_id()) || - submitted_full_compactions.contains(t->tablet_id()) || + return !!submitted_base_compactions.count(t->tablet_id()) || + !!submitted_full_compactions.count(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.contains(t->tablet_id()) || - (t->tablet_state() != TABLET_RUNNING && t->alter_version() == -1); + return !!tablet_preparing_cumu_compaction.count(t->tablet_id()) || + t->tablet_state() != TABLET_RUNNING; }; } else { filter_out = [&tablet_preparing_cumu_compaction, &submitted_cumu_compactions](CloudTablet* t) { - 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); + return !!tablet_preparing_cumu_compaction.count(t->tablet_id()) || + !!submitted_cumu_compactions.count(t->tablet_id()) || + t->tablet_state() != TABLET_RUNNING; }; } diff --git a/be/src/cloud/cloud_tablet.cpp b/be/src/cloud/cloud_tablet.cpp index 67fd9dd0a8df5c..17ec1fe22b0d85 100644 --- a/be/src/cloud/cloud_tablet.cpp +++ b/be/src/cloud/cloud_tablet.cpp @@ -135,6 +135,19 @@ 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. @@ -164,7 +177,8 @@ 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::OK(); + return Status::Error("invalid tablet state {}. tablet_id={}", + tablet_meta->tablet_state(), tablet_id()); } TimestampedVersionTracker empty_tracker; @@ -186,19 +200,6 @@ 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) { @@ -591,8 +592,7 @@ 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 && - (_alter_version == -1 || version.second <= _alter_version)) { + if (version.first != 0 && version.first < _cumulative_point) { candidate_rowsets.push_back(rs); } } diff --git a/be/src/cloud/cloud_tablet.h b/be/src/cloud/cloud_tablet.h index 2bd1ce475028ab..10ff1835e6c830 100644 --- a/be/src/cloud/cloud_tablet.h +++ b/be/src/cloud/cloud_tablet.h @@ -145,9 +145,6 @@ 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 { @@ -241,7 +238,6 @@ 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 3d48441080cc01..e589025423ec1f 100644 --- a/be/src/cloud/config.cpp +++ b/be/src/cloud/config.cpp @@ -59,6 +59,4 @@ 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 4a5b3e0e16a208..bf041ba0fa6fc5 100644 --- a/be/src/cloud/config.h +++ b/be/src/cloud/config.h @@ -65,7 +65,6 @@ 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 efc6db6debf3b2..141e302af8c420 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->start_version(); + int64_t cur_version = rowset->end_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 b2b9ec2531b3fb..1886d4bdf53537 100644 --- a/cloud/src/meta-service/meta_service_job.cpp +++ b/cloud/src/meta-service/meta_service_job.cpp @@ -49,32 +49,6 @@ 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, @@ -150,17 +124,6 @@ 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; } @@ -190,10 +153,7 @@ 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.has_check_input_versions_range() && - compaction.input_versions().empty()) || - (compaction.has_check_input_versions_range() && - !compaction.check_input_versions_range())) { + } else if (compaction.input_versions().empty()) { // 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) @@ -254,8 +214,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, StartTabletJobResponse* response, - std::string& instance_id, bool& need_commit) { + const StartTabletJobRequest* request, 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; @@ -342,13 +302,6 @@ 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. @@ -361,11 +314,6 @@ 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; } @@ -438,7 +386,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, response, instance_id, need_commit); + start_schema_change_job(code, msg, ss, txn, request, instance_id, need_commit); return; } } @@ -610,38 +558,6 @@ void process_compaction_job(MetaServiceCode& code, std::string& msg, std::string 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 || 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(); - txn->put(job_key, job_val); - INSTANCE_LOG(INFO) << "abort tablet compaction job, tablet_id=" << tablet_id - << " key=" << hex(job_key); - if (compaction.has_delete_bitmap_lock_initiator()) { - remove_delete_bitmap_update_lock(txn, instance_id, table_id, - COMPACTION_DELETE_BITMAP_LOCK_ID, - compaction.delete_bitmap_lock_initiator()); - } - need_commit = true; - return; - } - //========================================================================== // Lease //========================================================================== @@ -662,6 +578,25 @@ void process_compaction_job(MetaServiceCode& code, std::string& msg, std::string return; } + //========================================================================== + // Abort + //========================================================================== + if (request->action() == FinishTabletJobRequest::ABORT) { + // TODO(gavin): mv tmp rowsets to recycle or remove them directly + recorded_job.mutable_compaction()->erase(recorded_compaction); + auto job_val = recorded_job.SerializeAsString(); + txn->put(job_key, job_val); + INSTANCE_LOG(INFO) << "abort tablet compaction job, tablet_id=" << tablet_id + << " key=" << hex(job_key); + if (compaction.has_delete_bitmap_lock_initiator()) { + remove_delete_bitmap_update_lock(txn, instance_id, table_id, + COMPACTION_DELETE_BITMAP_LOCK_ID, + compaction.delete_bitmap_lock_initiator()); + } + need_commit = true; + return; + } + //========================================================================== // Commit //========================================================================== @@ -919,10 +854,7 @@ 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; } @@ -1029,50 +961,11 @@ 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) { - 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; - } + // TODO(cyx) return; } @@ -1084,7 +977,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 + // 5. remove schema_change job (unnecessary) // //========================================================================== // update tablet meta @@ -1233,11 +1126,6 @@ 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 f0323eebb790be..250cf43ea98684 100644 --- a/cloud/test/meta_service_job_test.cpp +++ b/cloud/test/meta_service_job_test.cpp @@ -60,10 +60,9 @@ 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.second > 0) { + if (input_version.first > 0 && 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); }; @@ -192,18 +191,15 @@ 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, - StartTabletJobResponse& res, int64_t alter_version = -1) { + const std::string& job_id, const std::string& initiator) { 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); @@ -221,14 +217,13 @@ 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, - FinishTabletJobRequest_Action action = FinishTabletJobRequest::COMMIT) { +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) { brpc::Controller cntl; FinishTabletJobRequest req; - req.set_action(action); + req.set_action(FinishTabletJobRequest::COMMIT); 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); @@ -687,11 +682,8 @@ 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(); @@ -1340,10 +1332,9 @@ 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", sc_res)); + "job1", "be1")); FinishTabletJobResponse res; finish_schema_change_job(meta_service.get(), tablet_id, new_tablet_id, "job2", "be1", {}, res); @@ -1369,10 +1360,9 @@ 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", sc_res)); + "job2", "be1")); std::vector output_rowsets; for (int64_t i = 0; i < 5; ++i) { @@ -1433,10 +1423,9 @@ 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", sc_res)); + "job3", "be1")); // provide existed rowsets std::vector existed_rowsets; for (int i = 0; i < 5; ++i) { @@ -1554,10 +1543,9 @@ 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", sc_res)); + "be1")); // provide existed rowsets std::vector existed_rowsets; for (int i = 0; i < 5; ++i) { @@ -1565,16 +1553,15 @@ 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", sc_res)); - sc_res.Clear(); + "be1")); // 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", sc_res)); + "be1")); // 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)); @@ -1586,11 +1573,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", sc_res)); + "be2")); // BE2 output_versions=[2-8][9-12][13-13] std::vector be2_output_rowsets; { @@ -1716,10 +1703,9 @@ 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", sc_res)); + "job1", "be1")); 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)); @@ -1764,10 +1750,9 @@ 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", sc_res)); + "job2", "be1")); 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)); @@ -2287,10 +2272,9 @@ 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", sc_res)); + "BE1")); long now = time(nullptr); FinishTabletJobRequest req; @@ -2310,212 +2294,4 @@ 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 5c5dd1972ed323..1c31d74d98630b 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,20 +124,6 @@ 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 3d79863addb8ac..a8bcc546de33e6 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,8 +41,6 @@ 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; @@ -51,10 +49,8 @@ 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 { @@ -110,7 +106,7 @@ protected void commitShadowIndex() throws AlterCancelException { } @Override - protected void onCancel() { + protected void postProcessShadowIndex() { if (Config.enable_check_compatibility_mode) { LOG.info("skip drop shadown indexes in checking compatibility mode"); return; @@ -118,36 +114,6 @@ protected void onCancel() { 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 d22b1bff9fc30e..e38c91d296fed4 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") - protected Map> partitionIdToBaseRollupTabletIdMap = Maps.newHashMap(); + private 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 c2c0aa108f4062..111bfbce1f7fab 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 @@ -95,7 +95,7 @@ public class SchemaChangeJobV2 extends AlterJobV2 { // partition id -> (shadow index id -> (shadow tablet id -> origin tablet id)) @SerializedName(value = "partitionIndexTabletMap") - protected Table> partitionIndexTabletMap = HashBasedTable.create(); + private Table> partitionIndexTabletMap = HashBasedTable.create(); // partition id -> (shadow index id -> shadow index)) @SerializedName(value = "partitionIndexMap") protected Table partitionIndexMap = HashBasedTable.create(); @@ -754,7 +754,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); - onCancel(); + postProcessShadowIndex(); return true; } @@ -902,7 +902,7 @@ private void replayRunningJob(SchemaChangeJobV2 replayedJob) { private void replayCancelled(SchemaChangeJobV2 replayedJob) { cancelInternal(); // try best to drop shadow index - onCancel(); + postProcessShadowIndex(); this.jobState = JobState.CANCELLED; this.finishedTimeMs = replayedJob.finishedTimeMs; this.errMsg = replayedJob.errMsg; @@ -1005,7 +1005,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 onCancel() {} + protected void postProcessShadowIndex() {} // 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 f0c9278562d437..e9d4e3c33acedc 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,67 +845,6 @@ 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 1216d0a684692b..5fd42a31f92459 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,15 +382,4 @@ 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 9f944f0ddf9cda..5f17692180b353 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,16 +497,6 @@ 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/fe/fe-core/src/main/java/org/apache/doris/master/MetaHelper.java b/fe/fe-core/src/main/java/org/apache/doris/master/MetaHelper.java index fe516b02bfd360..e4fd5cacf71262 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/master/MetaHelper.java +++ b/fe/fe-core/src/main/java/org/apache/doris/master/MetaHelper.java @@ -25,6 +25,8 @@ import org.apache.doris.persist.gson.GsonUtils; import org.apache.commons.codec.digest.DigestUtils; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.BufferedInputStream; import java.io.BufferedReader; @@ -36,8 +38,10 @@ import java.io.InputStreamReader; import java.io.OutputStream; import java.net.HttpURLConnection; +import java.util.Map; public class MetaHelper { + public static final Logger LOG = LogManager.getLogger(MetaHelper.class); private static final String PART_SUFFIX = ".part"; public static final String X_IMAGE_SIZE = "X-Image-Size"; public static final String X_IMAGE_MD5 = "X-Image-Md5"; @@ -74,7 +78,9 @@ public static File getFile(String filename, File dir) { } public static ResponseBody doGet(String url, int timeout, Class clazz) throws IOException { - String response = HttpUtils.doGet(url, HttpURLUtil.getNodeIdentHeaders(), timeout); + Map headers = HttpURLUtil.getNodeIdentHeaders(); + LOG.info("meta helper, url: {}, timeout{}, headers: {}", url, timeout, headers); + String response = HttpUtils.doGet(url, headers, timeout); return parseResponse(response, clazz); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/meta/MetaWriter.java b/fe/fe-core/src/main/java/org/apache/doris/persist/meta/MetaWriter.java index 56bbbc196e551b..06136f509b5712 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/meta/MetaWriter.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/meta/MetaWriter.java @@ -127,8 +127,8 @@ public static void write(File imageFile, Env env) throws IOException { MetaFooter.write(imageFile, metaIndices, checksum.getRef()); long saveImageEndTime = System.currentTimeMillis(); - LOG.info("finished save image {} in {} ms. checksum is {}", imageFile.getAbsolutePath(), - (saveImageEndTime - saveImageStartTime), checksum.getRef()); + LOG.info("finished save image {} in {} ms. checksum is {}, size is {}", imageFile.getAbsolutePath(), + (saveImageEndTime - saveImageStartTime), checksum.getRef(), imageFile.length()); } } diff --git a/gensrc/proto/cloud.proto b/gensrc/proto/cloud.proto index f8acd97d05fb38..7d9c9c19169f40 100644 --- a/gensrc/proto/cloud.proto +++ b/gensrc/proto/cloud.proto @@ -536,7 +536,6 @@ 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 { @@ -1183,7 +1182,6 @@ message StartTabletJobRequest { message StartTabletJobResponse { optional MetaServiceResponseStatus status = 1; repeated int64 version_in_compaction = 2; - optional int64 alter_version = 3; } message FinishTabletJobRequest { @@ -1201,7 +1199,6 @@ message FinishTabletJobRequest { message FinishTabletJobResponse { optional MetaServiceResponseStatus status = 1; optional TabletStatsPB stats = 2; - optional int64 alter_version = 3; } message BeginCopyRequest { @@ -1340,7 +1337,6 @@ 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 1da9c9992d5f93..9f2967b1972c11 100644 --- a/regression-test/pipeline/cloud_p0/conf/be_custom.conf +++ b/regression-test/pipeline/cloud_p0/conf/be_custom.conf @@ -33,4 +33,3 @@ 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 4be7f4b2e0ff6a..1e097a045a76d5 100644 --- a/regression-test/plugins/plugin_curl_requester.groovy +++ b/regression-test/plugins/plugin_curl_requester.groovy @@ -173,10 +173,6 @@ 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 deleted file mode 100644 index 93bb6254303301..00000000000000 --- a/regression-test/suites/cloud_p0/schema_change/compaction1/test_schema_change_with_compaction1.groovy +++ /dev/null @@ -1,257 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -// 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 deleted file mode 100644 index b393979d44218a..00000000000000 --- a/regression-test/suites/cloud_p0/schema_change/compaction10/test_schema_change_with_compaction10.groovy +++ /dev/null @@ -1,263 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -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 deleted file mode 100644 index fd257fcb7ea950..00000000000000 --- a/regression-test/suites/cloud_p0/schema_change/compaction11/test_schema_change_with_compaction11.groovy +++ /dev/null @@ -1,280 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -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 deleted file mode 100644 index c8ca8a54109824..00000000000000 --- a/regression-test/suites/cloud_p0/schema_change/compaction2/test_schema_change_with_compaction2.groovy +++ /dev/null @@ -1,214 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -// 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 deleted file mode 100644 index b2aab9f2dc7c84..00000000000000 --- a/regression-test/suites/cloud_p0/schema_change/compaction3/test_schema_change_with_compaction3.groovy +++ /dev/null @@ -1,194 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -// 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 deleted file mode 100644 index 4b53dbdd998104..00000000000000 --- a/regression-test/suites/cloud_p0/schema_change/compaction4/test_schema_change_with_compaction4.groovy +++ /dev/null @@ -1,194 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -// 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 deleted file mode 100644 index f5028ff9e818c3..00000000000000 --- a/regression-test/suites/cloud_p0/schema_change/compaction5/test_schema_change_with_compaction5.groovy +++ /dev/null @@ -1,260 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -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 deleted file mode 100644 index 951535433d1362..00000000000000 --- a/regression-test/suites/cloud_p0/schema_change/compaction6/test_schema_change_with_compaction6.groovy +++ /dev/null @@ -1,263 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -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 deleted file mode 100644 index 7291ea3a341e44..00000000000000 --- a/regression-test/suites/cloud_p0/schema_change/compaction7/test_schema_change_with_compaction7.groovy +++ /dev/null @@ -1,256 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -// 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 deleted file mode 100644 index 1017e1d50f235f..00000000000000 --- a/regression-test/suites/cloud_p0/schema_change/compaction8/test_schema_change_with_compaction8.groovy +++ /dev/null @@ -1,214 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -// 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 deleted file mode 100644 index 83c549eefc5abd..00000000000000 --- a/regression-test/suites/cloud_p0/schema_change/compaction9/test_schema_change_with_compaction9.groovy +++ /dev/null @@ -1,260 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -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 deleted file mode 100644 index 99c85399c123b2..00000000000000 --- a/regression-test/suites/cloud_p0/schema_change/ddl/date_create.sql +++ /dev/null @@ -1,24 +0,0 @@ -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 deleted file mode 100644 index 41702d336d7e7f..00000000000000 --- a/regression-test/suites/cloud_p0/schema_change/ddl/date_delete.sql +++ /dev/null @@ -1 +0,0 @@ -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 deleted file mode 100644 index 3e1511ca69a67a..00000000000000 --- a/regression-test/suites/cloud_p0/schema_change/ddl/date_load.sql +++ /dev/null @@ -1,6 +0,0 @@ -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 deleted file mode 100644 index 6138cb213a2063..00000000000000 --- a/regression-test/suites/cloud_p0/schema_change/ddl/date_unique_create.sql +++ /dev/null @@ -1,27 +0,0 @@ -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" -);