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