diff --git a/be/src/agent/agent_server.cpp b/be/src/agent/agent_server.cpp index 269ee49022465d..fc63c1ff837217 100644 --- a/be/src/agent/agent_server.cpp +++ b/be/src/agent/agent_server.cpp @@ -193,12 +193,15 @@ void AgentServer::cloud_start_workers(CloudStorageEngine& engine, ExecEnv* exec_ "PUSH", config::delete_worker_count, [&engine](auto&& task) { cloud_push_callback(engine, task); }); // TODO(plat1ko): SUBMIT_TABLE_COMPACTION worker - // TODO(plat1ko): CALCULATE_DELETE_BITMAP worker _workers[TTaskType::ALTER] = std::make_unique( "ALTER_TABLE", config::alter_tablet_worker_count, [&engine](auto&& task) { return alter_cloud_tablet_callback(engine, task); }); + _workers[TTaskType::CALCULATE_DELETE_BITMAP] = std::make_unique( + "CALC_DBM_TASK", config::calc_delete_bitmap_worker_count, + [&engine](auto&& task) { return calc_delete_bimtap_callback(engine, task); }); + _report_workers.push_back(std::make_unique( "REPORT_TASK", _master_info, config::report_task_interval_seconds, [&master_info = _master_info] { report_task_callback(master_info); })); diff --git a/be/src/agent/task_worker_pool.cpp b/be/src/agent/task_worker_pool.cpp index c4fc258552b9b4..af4eade1b3c0df 100644 --- a/be/src/agent/task_worker_pool.cpp +++ b/be/src/agent/task_worker_pool.cpp @@ -45,6 +45,7 @@ #include "agent/utils.h" #include "cloud/cloud_delete_task.h" +#include "cloud/cloud_engine_calc_delete_bitmap_task.h" #include "cloud/cloud_schema_change_job.h" #include "common/config.h" #include "common/logging.h" @@ -1864,4 +1865,35 @@ void storage_medium_migrate_callback(StorageEngine& engine, const TAgentTaskRequ remove_task_info(req.task_type, req.signature); } +void calc_delete_bimtap_callback(CloudStorageEngine& engine, const TAgentTaskRequest& req) { + std::vector error_tablet_ids; + std::vector succ_tablet_ids; + Status status; + error_tablet_ids.clear(); + const auto& calc_delete_bitmap_req = req.calc_delete_bitmap_req; + CloudEngineCalcDeleteBitmapTask engine_task(engine, calc_delete_bitmap_req, &error_tablet_ids, + &succ_tablet_ids); + status = engine_task.execute(); + + TFinishTaskRequest finish_task_request; + if (!status) { + DorisMetrics::instance()->publish_task_failed_total->increment(1); + LOG_WARNING("failed to calculate delete bitmap") + .tag("signature", req.signature) + .tag("transaction_id", calc_delete_bitmap_req.transaction_id) + .tag("error_tablets_num", error_tablet_ids.size()) + .error(status); + } + + status.to_thrift(&finish_task_request.task_status); + finish_task_request.__set_backend(BackendOptions::get_local_backend()); + finish_task_request.__set_task_type(req.task_type); + finish_task_request.__set_signature(req.signature); + finish_task_request.__set_report_version(s_report_version); + finish_task_request.__set_error_tablet_ids(error_tablet_ids); + + finish_task(finish_task_request); + remove_task_info(req.task_type, req.signature); +} + } // namespace doris diff --git a/be/src/agent/task_worker_pool.h b/be/src/agent/task_worker_pool.h index 531a64655a1d84..5c546582576c89 100644 --- a/be/src/agent/task_worker_pool.h +++ b/be/src/agent/task_worker_pool.h @@ -180,4 +180,6 @@ void report_disk_callback(StorageEngine& engine, const TMasterInfo& master_info) void report_tablet_callback(StorageEngine& engine, const TMasterInfo& master_info); +void calc_delete_bimtap_callback(CloudStorageEngine& engine, const TAgentTaskRequest& req); + } // namespace doris diff --git a/be/src/cloud/cloud_base_compaction.cpp b/be/src/cloud/cloud_base_compaction.cpp new file mode 100644 index 00000000000000..12f1fa6aa9d058 --- /dev/null +++ b/be/src/cloud/cloud_base_compaction.cpp @@ -0,0 +1,388 @@ +// 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. + +#include "cloud/cloud_base_compaction.h" + +#include + +#include "cloud/cloud_meta_mgr.h" +#include "cloud/config.h" +#include "common/config.h" +#include "common/sync_point.h" +#include "gen_cpp/cloud.pb.h" +#include "olap/compaction.h" +#include "olap/task/engine_checksum_task.h" +#include "service/backend_options.h" +#include "util/thread.h" +#include "util/uuid_generator.h" + +namespace doris { +using namespace ErrorCode; + +bvar::Adder base_output_size("base_compaction", "output_size"); + +CloudBaseCompaction::CloudBaseCompaction(CloudStorageEngine& engine, CloudTabletSPtr tablet) + : CloudCompactionMixin(engine, tablet, + "BaseCompaction:" + std::to_string(tablet->tablet_id())) { + auto uuid = UUIDGenerator::instance()->next_uuid(); + std::stringstream ss; + ss << uuid; + _uuid = ss.str(); +} + +CloudBaseCompaction::~CloudBaseCompaction() = default; + +Status CloudBaseCompaction::prepare_compact() { + if (_tablet->tablet_state() != TABLET_RUNNING) { + return Status::InternalError("invalid tablet state. tablet_id={}", _tablet->tablet_id()); + } + + bool need_sync_tablet = true; + { + std::shared_lock rlock(_tablet->get_header_lock()); + // If number of rowsets is equal to approximate_num_rowsets, it is very likely that this tablet has been + // synchronized with meta-service. + if (_tablet->tablet_meta()->all_rs_metas().size() >= + cloud_tablet()->fetch_add_approximate_num_rowsets(0) && + cloud_tablet()->last_sync_time_s > 0) { + need_sync_tablet = false; + } + } + if (need_sync_tablet) { + RETURN_IF_ERROR(cloud_tablet()->sync_rowsets()); + } + + RETURN_IF_ERROR(pick_rowsets_to_compact()); + + // prepare compaction job + cloud::TabletJobInfoPB job; + auto idx = job.mutable_idx(); + idx->set_tablet_id(_tablet->tablet_id()); + idx->set_table_id(_tablet->table_id()); + idx->set_index_id(_tablet->index_id()); + idx->set_partition_id(_tablet->partition_id()); + auto compaction_job = job.add_compaction(); + compaction_job->set_id(_uuid); + compaction_job->set_initiator(BackendOptions::get_localhost() + ':' + + std::to_string(config::heartbeat_service_port)); + 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); + 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 (!st.ok()) { + if (resp.status().code() == cloud::STALE_TABLET_CACHE) { + // set last_sync_time to 0 to force sync tablet next time + cloud_tablet()->last_sync_time_s = 0; + } else if (resp.status().code() == cloud::TABLET_NOT_FOUND) { + // tablet not found + cloud_tablet()->recycle_cached_data(); + } + return st; + } + + for (auto& rs : _input_rowsets) { + _input_row_num += rs->num_rows(); + _input_segments += rs->num_segments(); + _input_rowsets_size += rs->data_disk_size(); + } + LOG_INFO("start CloudBaseCompaction, tablet_id={}, range=[{}-{}]", _tablet->tablet_id(), + _input_rowsets.front()->start_version(), _input_rowsets.back()->end_version()) + .tag("job_id", _uuid) + .tag("input_rowsets", _input_rowsets.size()) + .tag("input_rows", _input_row_num) + .tag("input_segments", _input_segments) + .tag("input_data_size", _input_rowsets_size); + return st; +} + +void CloudBaseCompaction::_filter_input_rowset() { + // if dup_key and no delete predicate + // we skip big files to save resources + if (_tablet->keys_type() != KeysType::DUP_KEYS) { + return; + } + for (auto& rs : _input_rowsets) { + if (rs->rowset_meta()->has_delete_predicate()) { + return; + } + } + int64_t max_size = config::base_compaction_dup_key_max_file_size_mbytes * 1024 * 1024; + // first find a proper rowset for start + auto rs_iter = _input_rowsets.begin(); + while (rs_iter != _input_rowsets.end()) { + if ((*rs_iter)->rowset_meta()->total_disk_size() >= max_size) { + rs_iter = _input_rowsets.erase(rs_iter); + } else { + break; + } + } +} + +Status CloudBaseCompaction::pick_rowsets_to_compact() { + _input_rowsets.clear(); + { + std::shared_lock rlock(_tablet->get_header_lock()); + _base_compaction_cnt = cloud_tablet()->base_compaction_cnt(); + _cumulative_compaction_cnt = cloud_tablet()->cumulative_compaction_cnt(); + _input_rowsets = cloud_tablet()->pick_candidate_rowsets_to_base_compaction(); + } + if (auto st = check_version_continuity(_input_rowsets); !st.ok()) { + DCHECK(false) << st; + return st; + } + _filter_input_rowset(); + if (_input_rowsets.size() <= 1) { + return Status::Error( + "insuffient compation input rowset, #rowsets={}", _input_rowsets.size()); + } + + if (_input_rowsets.size() == 2 && _input_rowsets[0]->end_version() == 1) { + // the tablet is with rowset: [0-1], [2-y] + // and [0-1] has no data. in this situation, no need to do base compaction. + return Status::Error("no suitable versions for compaction"); + } + + // 1. cumulative rowset must reach base_compaction_min_rowset_num threshold + if (_input_rowsets.size() > config::base_compaction_min_rowset_num) { + VLOG_NOTICE << "satisfy the base compaction policy. tablet=" << _tablet->tablet_id() + << ", num_cumulative_rowsets=" << _input_rowsets.size() - 1 + << ", base_compaction_num_cumulative_rowsets=" + << config::base_compaction_min_rowset_num; + return Status::OK(); + } + + // 2. the ratio between base rowset and all input cumulative rowsets reaches the threshold + // `_input_rowsets` has been sorted by end version, so we consider `_input_rowsets[0]` is the base rowset. + int64_t base_size = _input_rowsets.front()->data_disk_size(); + int64_t cumulative_total_size = 0; + for (auto it = _input_rowsets.begin() + 1; it != _input_rowsets.end(); ++it) { + cumulative_total_size += (*it)->data_disk_size(); + } + + double base_cumulative_delta_ratio = config::base_compaction_min_data_ratio; + if (base_size == 0) { + // base_size == 0 means this may be a base version [0-1], which has no data. + // set to 1 to void divide by zero + base_size = 1; + } + double cumulative_base_ratio = static_cast(cumulative_total_size) / base_size; + + if (cumulative_base_ratio > base_cumulative_delta_ratio) { + VLOG_NOTICE << "satisfy the base compaction policy. tablet=" << _tablet->tablet_id() + << ", cumulative_total_size=" << cumulative_total_size + << ", base_size=" << base_size + << ", cumulative_base_ratio=" << cumulative_base_ratio + << ", policy_ratio=" << base_cumulative_delta_ratio; + return Status::OK(); + } + + // 3. the interval since last base compaction reaches the threshold + int64_t base_creation_time = _input_rowsets[0]->creation_time(); + int64_t interval_threshold = config::base_compaction_interval_seconds_since_last_operation; + int64_t interval_since_last_base_compaction = time(nullptr) - base_creation_time; + if (interval_since_last_base_compaction > interval_threshold) { + VLOG_NOTICE << "satisfy the base compaction policy. tablet=" << _tablet->tablet_id() + << ", interval_since_last_base_compaction=" + << interval_since_last_base_compaction + << ", interval_threshold=" << interval_threshold; + return Status::OK(); + } + + VLOG_NOTICE << "don't satisfy the base compaction policy. tablet=" << _tablet->tablet_id() + << ", num_cumulative_rowsets=" << _input_rowsets.size() - 1 + << ", cumulative_base_ratio=" << cumulative_base_ratio + << ", interval_since_last_base_compaction=" << interval_since_last_base_compaction; + return Status::Error("no suitable versions for compaction"); +} + +Status CloudBaseCompaction::execute_compact() { + if (config::enable_base_compaction_idle_sched) { + Thread::set_idle_sched(); + } + + SCOPED_ATTACH_TASK(_mem_tracker); + + using namespace std::chrono; + auto start = steady_clock::now(); + RETURN_IF_ERROR(CloudCompactionMixin::execute_compact()); + LOG_INFO("finish CloudBaseCompaction, tablet_id={}, cost={}ms", _tablet->tablet_id(), + duration_cast(steady_clock::now() - start).count()) + .tag("job_id", _uuid) + .tag("input_rowsets", _input_rowsets.size()) + .tag("input_rows", _input_row_num) + .tag("input_segments", _input_segments) + .tag("input_data_size", _input_rowsets_size) + .tag("output_rows", _output_rowset->num_rows()) + .tag("output_segments", _output_rowset->num_segments()) + .tag("output_data_size", _output_rowset->data_disk_size()); + + //_compaction_succeed = true; + _state = CompactionState::SUCCESS; + + DorisMetrics::instance()->base_compaction_deltas_total->increment(_input_rowsets.size()); + DorisMetrics::instance()->base_compaction_bytes_total->increment(_input_rowsets_size); + base_output_size << _output_rowset->data_disk_size(); + + return Status::OK(); +} + +Status CloudBaseCompaction::modify_rowsets() { + // commit compaction job + cloud::TabletJobInfoPB job; + auto idx = job.mutable_idx(); + idx->set_tablet_id(_tablet->tablet_id()); + idx->set_table_id(_tablet->table_id()); + idx->set_index_id(_tablet->index_id()); + idx->set_partition_id(_tablet->partition_id()); + auto compaction_job = job.add_compaction(); + compaction_job->set_id(_uuid); + compaction_job->set_initiator(BackendOptions::get_localhost() + ':' + + std::to_string(config::heartbeat_service_port)); + compaction_job->set_type(cloud::TabletCompactionJobPB::BASE); + compaction_job->set_input_cumulative_point(cloud_tablet()->cumulative_layer_point()); + compaction_job->set_output_cumulative_point(cloud_tablet()->cumulative_layer_point()); + compaction_job->set_num_input_rows(_input_row_num); + compaction_job->set_num_output_rows(_output_rowset->num_rows()); + compaction_job->set_size_input_rowsets(_input_rowsets_size); + compaction_job->set_size_output_rowsets(_output_rowset->data_disk_size()); + compaction_job->set_num_input_segments(_input_segments); + compaction_job->set_num_output_segments(_output_rowset->num_segments()); + compaction_job->set_num_input_rowsets(_input_rowsets.size()); + compaction_job->set_num_output_rowsets(1); + compaction_job->add_input_versions(_input_rowsets.front()->start_version()); + compaction_job->add_input_versions(_input_rowsets.back()->end_version()); + compaction_job->add_output_versions(_output_rowset->end_version()); + compaction_job->add_txn_id(_output_rowset->txn_id()); + compaction_job->add_output_rowset_ids(_output_rowset->rowset_id().to_string()); + + /* + DeleteBitmapPtr output_rowset_delete_bitmap = nullptr; + if (_tablet->keys_type() == KeysType::UNIQUE_KEYS && + _tablet->enable_unique_key_merge_on_write()) { + int64_t missed_rows = _stats.merged_rows; + int64_t initiator = + boost::hash_range(_uuid.begin(), _uuid.end()) & std::numeric_limits::max(); + RETURN_IF_ERROR(_tablet->cloud_calc_delete_bitmap_for_compaciton( + _input_rowsets, _output_rowset, _rowid_conversion, compaction_type(), missed_rows, + initiator, output_rowset_delete_bitmap)); + compaction_job->set_delete_bitmap_lock_initiator(initiator); + } + */ + + cloud::FinishTabletJobResponse resp; + auto st = _engine.meta_mgr().commit_tablet_job(job, &resp); + if (!st.ok()) { + if (resp.status().code() == cloud::TABLET_NOT_FOUND) { + cloud_tablet()->recycle_cached_data(); + } + return st; + } + auto& stats = resp.stats(); + LOG(INFO) << "tablet stats=" << stats.ShortDebugString(); + + { + std::unique_lock wrlock(_tablet->get_header_lock()); + // clang-format off + cloud_tablet()->set_last_base_compaction_success_time(std::max(cloud_tablet()->last_base_compaction_success_time(), stats.last_base_compaction_time_ms())); + cloud_tablet()->set_last_cumu_compaction_success_time(std::max(cloud_tablet()->last_cumu_compaction_success_time(), stats.last_cumu_compaction_time_ms())); + cloud_tablet()->set_last_full_compaction_success_time(std::max(cloud_tablet()->last_full_compaction_success_time(), stats.last_full_compaction_time_ms())); + // clang-format on + if (cloud_tablet()->base_compaction_cnt() >= stats.base_compaction_cnt()) { + // This could happen while calling `sync_tablet_rowsets` during `commit_tablet_job` + return Status::OK(); + } + // Try to make output rowset visible immediately in tablet cache, instead of waiting for next synchronization from meta-service. + cloud_tablet()->delete_rowsets(_input_rowsets, wrlock); + cloud_tablet()->add_rowsets({_output_rowset}, false, wrlock); + // ATTN: MUST NOT update `cumu_compaction_cnt` or `cumu_point` which are used when sync rowsets, otherwise may cause + // the tablet to be unable to synchronize the rowset meta changes generated by cumu compaction. + cloud_tablet()->set_base_compaction_cnt(cloud_tablet()->base_compaction_cnt() + 1); + /* + if (output_rowset_delete_bitmap) { + _tablet->tablet_meta()->delete_bitmap().merge(*output_rowset_delete_bitmap); + } + */ + if (stats.cumulative_compaction_cnt() >= cloud_tablet()->cumulative_compaction_cnt()) { + cloud_tablet()->reset_approximate_stats(stats.num_rowsets(), stats.num_segments(), + stats.num_rows(), stats.data_size()); + } + } + return Status::OK(); +} + +void CloudBaseCompaction::garbage_collection() { + // TODO(luwei) implement file_cache_garbage_collection + //file_cache_garbage_collection(); + cloud::TabletJobInfoPB job; + auto idx = job.mutable_idx(); + idx->set_tablet_id(_tablet->tablet_id()); + idx->set_table_id(_tablet->table_id()); + idx->set_index_id(_tablet->index_id()); + idx->set_partition_id(_tablet->partition_id()); + auto compaction_job = job.add_compaction(); + compaction_job->set_id(_uuid); + compaction_job->set_initiator(BackendOptions::get_localhost() + ':' + + std::to_string(config::heartbeat_service_port)); + compaction_job->set_type(cloud::TabletCompactionJobPB::BASE); + if (_tablet->keys_type() == KeysType::UNIQUE_KEYS && + _tablet->enable_unique_key_merge_on_write()) { + int64_t initiator = + boost::hash_range(_uuid.begin(), _uuid.end()) & std::numeric_limits::max(); + compaction_job->set_delete_bitmap_lock_initiator(initiator); + } + auto st = _engine.meta_mgr().abort_tablet_job(job); + if (!st.ok()) { + LOG_WARNING("failed to abort compaction job") + .tag("job_id", _uuid) + .tag("tablet_id", _tablet->tablet_id()) + .error(st); + } +} + +void CloudBaseCompaction::do_lease() { + cloud::TabletJobInfoPB job; + if (_state == CompactionState::SUCCESS) { + return; + } + auto idx = job.mutable_idx(); + idx->set_tablet_id(_tablet->tablet_id()); + idx->set_table_id(_tablet->table_id()); + idx->set_index_id(_tablet->index_id()); + idx->set_partition_id(_tablet->partition_id()); + auto compaction_job = job.add_compaction(); + compaction_job->set_id(_uuid); + using namespace std::chrono; + int64_t lease_time = duration_cast(system_clock::now().time_since_epoch()).count() + + config::lease_compaction_interval_seconds * 4; + compaction_job->set_lease(lease_time); + auto st = _engine.meta_mgr().lease_tablet_job(job); + if (!st.ok()) { + LOG_WARNING("failed to lease compaction job") + .tag("job_id", _uuid) + .tag("tablet_id", _tablet->tablet_id()) + .error(st); + } +} + +} // namespace doris diff --git a/be/src/cloud/cloud_base_compaction.h b/be/src/cloud/cloud_base_compaction.h new file mode 100644 index 00000000000000..4240458f21ba87 --- /dev/null +++ b/be/src/cloud/cloud_base_compaction.h @@ -0,0 +1,59 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include + +#include "cloud/cloud_storage_engine.h" +#include "cloud/cloud_tablet.h" +#include "olap/compaction.h" + +namespace doris { + +class CloudBaseCompaction : public CloudCompactionMixin { +public: + CloudBaseCompaction(CloudStorageEngine& engine, CloudTabletSPtr tablet); + ~CloudBaseCompaction() override; + + Status prepare_compact() override; + Status execute_compact() override; + + void do_lease(); + +private: + Status pick_rowsets_to_compact(); + + std::string_view compaction_name() const override { return "CloudBaseCompaction"; } + + Status modify_rowsets() override; + + void garbage_collection() override; + + void _filter_input_rowset(); + + void build_basic_info(); + + ReaderType compaction_type() const override { return ReaderType::READER_BASE_COMPACTION; } + + std::string _uuid; + int64_t _input_segments = 0; + int64_t _base_compaction_cnt = 0; + int64_t _cumulative_compaction_cnt = 0; +}; + +} // namespace doris diff --git a/be/src/cloud/cloud_compaction_action.cpp b/be/src/cloud/cloud_compaction_action.cpp new file mode 100644 index 00000000000000..fbc05e608b8602 --- /dev/null +++ b/be/src/cloud/cloud_compaction_action.cpp @@ -0,0 +1,338 @@ +// 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. + +#include "cloud/cloud_compaction_action.h" + +// IWYU pragma: no_include +#include // IWYU pragma: keep +#include +#include +#include +#include +#include +#include +#include +#include + +#include "cloud/cloud_base_compaction.h" +#include "cloud/cloud_compaction_action.h" +#include "cloud/cloud_cumulative_compaction.h" +#include "cloud/cloud_full_compaction.h" +#include "cloud/cloud_tablet.h" +#include "cloud/cloud_tablet_mgr.h" +#include "common/logging.h" +#include "common/status.h" +#include "gutil/strings/substitute.h" +#include "http/http_channel.h" +#include "http/http_headers.h" +#include "http/http_request.h" +#include "http/http_status.h" +#include "olap/base_compaction.h" +#include "olap/cumulative_compaction.h" +#include "olap/cumulative_compaction_policy.h" +#include "olap/cumulative_compaction_time_series_policy.h" +#include "olap/full_compaction.h" +#include "olap/olap_define.h" +#include "olap/storage_engine.h" +#include "olap/tablet_manager.h" +#include "util/doris_metrics.h" +#include "util/stopwatch.hpp" + +namespace doris { +using namespace ErrorCode; + +namespace {} + +const static std::string HEADER_JSON = "application/json"; + +CloudCompactionAction::CloudCompactionAction(CompactionActionType ctype, ExecEnv* exec_env, + CloudStorageEngine& engine, TPrivilegeHier::type hier, + TPrivilegeType::type ptype) + : HttpHandlerWithAuth(exec_env, hier, ptype), _engine(engine), _type(ctype) {} + +/// check param and fetch tablet_id from req +static Status _check_param(HttpRequest* req, uint64_t* tablet_id, uint64_t* table_id) { + // req tablet id and table id, we have to set only one of them. + std::string req_tablet_id = req->param(TABLET_ID_KEY); + std::string req_table_id = req->param(TABLE_ID_KEY); + if (req_tablet_id == "") { + if (req_table_id == "") { + // both tablet id and table id are empty, return error. + return Status::InternalError( + "tablet id and table id can not be empty at the same time!"); + } else { + try { + *table_id = std::stoull(req_table_id); + } catch (const std::exception& e) { + return Status::InternalError("convert tablet_id or table_id failed, {}", e.what()); + } + return Status::OK(); + } + } else { + if (req_table_id == "") { + try { + *tablet_id = std::stoull(req_tablet_id); + } catch (const std::exception& e) { + return Status::InternalError("convert tablet_id or table_id failed, {}", e.what()); + } + return Status::OK(); + } else { + // both tablet id and table id are not empty, return err. + return Status::InternalError("tablet id and table id can not be set at the same time!"); + } + } +} + +// for viewing the compaction status +Status CloudCompactionAction::_handle_show_compaction(HttpRequest* req, std::string* json_result) { + uint64_t tablet_id = 0; + uint64_t table_id = 0; + RETURN_NOT_OK_STATUS_WITH_WARN(_check_param(req, &tablet_id, &table_id), "check param failed"); + + //TabletSharedPtr tablet = _engine.tablet_manager()->get_tablet(tablet_id); + CloudTabletSPtr tablet = DORIS_TRY(_engine.tablet_mgr().get_tablet(tablet_id)); + if (tablet == nullptr) { + return Status::NotFound("Tablet not found. tablet_id={}", tablet_id); + } + + tablet->get_compaction_status(json_result); + return Status::OK(); +} + +Status CloudCompactionAction::_handle_run_compaction(HttpRequest* req, std::string* json_result) { + // 1. param check + // check req_tablet_id or req_table_id is not empty and can not be set together. + uint64_t tablet_id = 0; + uint64_t table_id = 0; + RETURN_NOT_OK_STATUS_WITH_WARN(_check_param(req, &tablet_id, &table_id), "check param failed"); + + // check compaction_type equals 'base' or 'cumulative' + std::string compaction_type = req->param(PARAM_COMPACTION_TYPE); + if (compaction_type != PARAM_COMPACTION_BASE && + compaction_type != PARAM_COMPACTION_CUMULATIVE && + compaction_type != PARAM_COMPACTION_FULL) { + return Status::NotSupported("The compaction type '{}' is not supported", compaction_type); + } + + if (tablet_id == 0 && table_id != 0) { + /* + std::vector tablet_vec = _engine.tablet_manager()->get_all_tablet( + [table_id](Tablet* tablet) -> bool { return tablet->get_table_id() == table_id; }); + */ + auto tablet_vec = _engine.tablet_mgr().get_weak_tablets(); + for (const auto& weak_tablet : tablet_vec) { + if (auto tablet = weak_tablet.lock()) { + if (tablet->table_id() != table_id) { + continue; + } + RETURN_IF_ERROR( + _engine.submit_compaction_task(tablet, CompactionType::FULL_COMPACTION)); + } + } + } else { + // 2. fetch the tablet by tablet_id + CloudTabletSPtr tablet = DORIS_TRY(_engine.tablet_mgr().get_tablet(tablet_id)); + if (tablet == nullptr) { + return Status::NotFound("Tablet not found. tablet_id={}", tablet_id); + } + + // 3. execute compaction task + std::packaged_task task([this, tablet, compaction_type]() { + return _execute_compaction_callback(tablet, compaction_type); + }); + std::future future_obj = task.get_future(); + std::thread(std::move(task)).detach(); + + // 4. wait for result for 2 seconds by async + std::future_status status = future_obj.wait_for(std::chrono::seconds(2)); + if (status == std::future_status::ready) { + // fetch execute result + Status olap_status = future_obj.get(); + if (!olap_status.ok()) { + return olap_status; + } + } else { + LOG(INFO) << "Manual compaction task is timeout for waiting " + << (status == std::future_status::timeout); + } + } + LOG(INFO) << "Manual compaction task is successfully triggered"; + *json_result = + R"({"status": "Success", "msg": "compaction task is successfully triggered. Table id: )" + + std::to_string(table_id) + ". Tablet id: " + std::to_string(tablet_id) + "\"}"; + return Status::OK(); +} + +Status CloudCompactionAction::_handle_run_status_compaction(HttpRequest* req, + std::string* json_result) { + uint64_t tablet_id = 0; + uint64_t table_id = 0; + + // check req_tablet_id is not empty + RETURN_NOT_OK_STATUS_WITH_WARN(_check_param(req, &tablet_id, &table_id), "check param failed"); + + if (tablet_id == 0) { + // overall compaction status + RETURN_IF_ERROR(_engine.get_compaction_status_json(json_result)); + return Status::OK(); + } else { + // fetch the tablet by tablet_id + //TabletSharedPtr tablet = _engine.tablet_manager()->get_tablet(tablet_id); + CloudTabletSPtr tablet = DORIS_TRY(_engine.tablet_mgr().get_tablet(tablet_id)); + if (tablet == nullptr) { + LOG(WARNING) << "invalid argument.tablet_id:" << tablet_id; + return Status::InternalError("fail to get {}", tablet_id); + } + + std::string json_template = R"({ + "status" : "Success", + "run_status" : $0, + "msg" : "$1", + "tablet_id" : $2, + "compact_type" : "$3" + })"; + + std::string msg = "compaction task for this tablet is not running"; + std::string compaction_type; + bool run_status = false; + + { + // use try lock to check this tablet is running cumulative compaction + std::unique_lock lock_cumulative(tablet->get_cumulative_compaction_lock(), + std::try_to_lock); + if (!lock_cumulative.owns_lock()) { + msg = "compaction task for this tablet is running"; + compaction_type = "cumulative"; + run_status = true; + *json_result = strings::Substitute(json_template, run_status, msg, tablet_id, + compaction_type); + return Status::OK(); + } + } + + { + // use try lock to check this tablet is running base compaction + std::unique_lock lock_base(tablet->get_base_compaction_lock(), + std::try_to_lock); + if (!lock_base.owns_lock()) { + msg = "compaction task for this tablet is running"; + compaction_type = "base"; + run_status = true; + *json_result = strings::Substitute(json_template, run_status, msg, tablet_id, + compaction_type); + return Status::OK(); + } + } + // not running any compaction + *json_result = + strings::Substitute(json_template, run_status, msg, tablet_id, compaction_type); + return Status::OK(); + } +} + +Status CloudCompactionAction::_execute_compaction_callback(CloudTabletSPtr tablet, + const std::string& compaction_type) { + MonotonicStopWatch timer; + timer.start(); + + std::shared_ptr cumulative_compaction_policy = + CumulativeCompactionPolicyFactory::create_cumulative_compaction_policy( + tablet->tablet_meta()->compaction_policy()); + /* + if (tablet->get_cumulative_compaction_policy() == nullptr) { + tablet->set_cumulative_compaction_policy(cumulative_compaction_policy); + } + */ + Status res = Status::OK(); + auto do_compact = [](Compaction& compaction) { + RETURN_IF_ERROR(compaction.prepare_compact()); + return compaction.execute_compact(); + }; + if (compaction_type == PARAM_COMPACTION_BASE) { + CloudBaseCompaction base_compaction(_engine, tablet); + res = do_compact(base_compaction); + if (!res) { + if (!res.is()) { + DorisMetrics::instance()->base_compaction_request_failed->increment(1); + } + } + } else if (compaction_type == PARAM_COMPACTION_CUMULATIVE) { + CloudCumulativeCompaction cumulative_compaction(_engine, tablet); + res = do_compact(cumulative_compaction); + if (!res) { + if (res.is()) { + // Ignore this error code. + VLOG_NOTICE << "failed to init cumulative compaction due to no suitable version," + << "tablet=" << tablet->tablet_id(); + } else { + DorisMetrics::instance()->cumulative_compaction_request_failed->increment(1); + LOG(WARNING) << "failed to do cumulative compaction. res=" << res + << ", table=" << tablet->tablet_id(); + } + } + } else if (compaction_type == PARAM_COMPACTION_FULL) { + CloudFullCompaction full_compaction(_engine, tablet); + res = do_compact(full_compaction); + if (!res) { + if (res.is()) { + // Ignore this error code. + VLOG_NOTICE << "failed to init full compaction due to no suitable version," + << "tablet=" << tablet->tablet_id(); + } else { + LOG(WARNING) << "failed to do full compaction. res=" << res + << ", table=" << tablet->tablet_id(); + } + } + } + + timer.stop(); + LOG(INFO) << "Manual compaction task finish, status=" << res + << ", compaction_use_time=" << timer.elapsed_time() / 1000000 << "ms"; + return res; +} + +void CloudCompactionAction::handle(HttpRequest* req) { + req->add_output_header(HttpHeaders::CONTENT_TYPE, HEADER_JSON.c_str()); + + if (_type == CompactionActionType::SHOW_INFO) { + std::string json_result; + Status st = _handle_show_compaction(req, &json_result); + if (!st.ok()) { + HttpChannel::send_reply(req, HttpStatus::OK, st.to_json()); + } else { + HttpChannel::send_reply(req, HttpStatus::OK, json_result); + } + } else if (_type == CompactionActionType::RUN_COMPACTION) { + std::string json_result; + Status st = _handle_run_compaction(req, &json_result); + if (!st.ok()) { + HttpChannel::send_reply(req, HttpStatus::OK, st.to_json()); + } else { + HttpChannel::send_reply(req, HttpStatus::OK, json_result); + } + } else { + std::string json_result; + Status st = _handle_run_status_compaction(req, &json_result); + if (!st.ok()) { + HttpChannel::send_reply(req, HttpStatus::OK, st.to_json()); + } else { + HttpChannel::send_reply(req, HttpStatus::OK, json_result); + } + } +} + +} // end namespace doris diff --git a/be/src/http/action/cloud_compaction_action.h b/be/src/cloud/cloud_compaction_action.h similarity index 56% rename from be/src/http/action/cloud_compaction_action.h rename to be/src/cloud/cloud_compaction_action.h index b9b9bed472b8c5..dbee3a61c6e9a3 100644 --- a/be/src/http/action/cloud_compaction_action.h +++ b/be/src/cloud/cloud_compaction_action.h @@ -17,14 +17,36 @@ #pragma once +#include + +#include + #include "cloud/cloud_storage_engine.h" #include "common/status.h" #include "http/action/compaction_action.h" #include "http/http_handler_with_auth.h" +#include "olap/tablet.h" namespace doris { class HttpRequest; +class ExecEnv; + +/* +enum class CompactionActionType { + SHOW_INFO = 1, + RUN_COMPACTION = 2, + RUN_COMPACTION_STATUS = 3, +}; +*/ + +/* +const std::string PARAM_COMPACTION_TYPE = "compact_type"; +const std::string PARAM_COMPACTION_BASE = "base"; +const std::string PARAM_COMPACTION_CUMULATIVE = "cumulative"; +const std::string PARAM_COMPACTION_FULL = "full"; +*/ + /// This action is used for viewing the compaction status. /// See compaction-action.md for details. class CloudCompactionAction : public HttpHandlerWithAuth { @@ -37,8 +59,21 @@ class CloudCompactionAction : public HttpHandlerWithAuth { void handle(HttpRequest* req) override; private: - [[maybe_unused]] CloudStorageEngine& _engine; - [[maybe_unused]] CompactionActionType _type; + Status _handle_show_compaction(HttpRequest* req, std::string* json_result); + + /// execute compaction request to run compaction task + /// param compact_type in req to distinguish the task type, base or cumulative + Status _handle_run_compaction(HttpRequest* req, std::string* json_result); + + /// thread callback function for the tablet to do compaction + Status _execute_compaction_callback(CloudTabletSPtr tablet, const std::string& compaction_type); + + /// fetch compaction running status + Status _handle_run_status_compaction(HttpRequest* req, std::string* json_result); + +private: + CloudStorageEngine& _engine; + CompactionActionType _type; }; -} // namespace doris \ No newline at end of file +} // end namespace doris diff --git a/be/src/cloud/cloud_cumulative_compaction.cpp b/be/src/cloud/cloud_cumulative_compaction.cpp new file mode 100644 index 00000000000000..422350c9bce246 --- /dev/null +++ b/be/src/cloud/cloud_cumulative_compaction.cpp @@ -0,0 +1,477 @@ +// 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. + +#include "cloud/cloud_cumulative_compaction.h" + +#include "cloud/cloud_meta_mgr.h" +#include "cloud/config.h" +#include "common/config.h" +#include "common/logging.h" +#include "common/status.h" +#include "common/sync_point.h" +#include "gen_cpp/cloud.pb.h" +#include "olap/compaction.h" +#include "olap/cumulative_compaction_policy.h" +#include "service/backend_options.h" +#include "util/trace.h" +#include "util/uuid_generator.h" + +namespace doris { +using namespace ErrorCode; + +bvar::Adder cumu_output_size("cumu_compaction", "output_size"); + +CloudCumulativeCompaction::CloudCumulativeCompaction(CloudStorageEngine& engine, + CloudTabletSPtr tablet) + : CloudCompactionMixin(engine, tablet, + "BaseCompaction:" + std::to_string(tablet->tablet_id())) { + auto uuid = UUIDGenerator::instance()->next_uuid(); + std::stringstream ss; + ss << uuid; + _uuid = ss.str(); +} + +CloudCumulativeCompaction::~CloudCumulativeCompaction() = default; + +Status CloudCumulativeCompaction::prepare_compact() { + if (_tablet->tablet_state() != TABLET_RUNNING) { + return Status::InternalError("invalid tablet state. tablet_id={}", _tablet->tablet_id()); + } + + std::vector> cumu_compactions; + _engine.get_cumu_compaction(_tablet->tablet_id(), cumu_compactions); + if (!cumu_compactions.empty()) { + for (auto& cumu : cumu_compactions) { + _max_conflict_version = + std::max(_max_conflict_version, cumu->_input_rowsets.back()->end_version()); + } + } + + int tried = 0; +PREPARE_TRY_AGAIN: + + bool need_sync_tablet = true; + { + std::shared_lock rlock(_tablet->get_header_lock()); + // If number of rowsets is equal to approximate_num_rowsets, it is very likely that this tablet has been + // synchronized with meta-service. + if (_tablet->tablet_meta()->all_rs_metas().size() >= + cloud_tablet()->fetch_add_approximate_num_rowsets(0) && + cloud_tablet()->last_sync_time_s > 0) { + need_sync_tablet = false; + } + } + if (need_sync_tablet) { + RETURN_IF_ERROR(cloud_tablet()->sync_rowsets()); + } + + // pick rowsets to compact + auto st = pick_rowsets_to_compact(); + if (!st.ok()) { + if (tried == 0 && _last_delete_version.first != -1) { + // we meet a delete version, should increase the cumulative point to let base compaction handle the delete version. + // plus 1 to skip the delete version. + // NOTICE: after that, the cumulative point may be larger than max version of this tablet, but it doesn't matter. + update_cumulative_point(); + } + return st; + } + + // prepare compaction job + cloud::TabletJobInfoPB job; + auto idx = job.mutable_idx(); + idx->set_tablet_id(_tablet->tablet_id()); + idx->set_table_id(_tablet->table_id()); + idx->set_index_id(_tablet->index_id()); + idx->set_partition_id(_tablet->partition_id()); + auto compaction_job = job.add_compaction(); + compaction_job->set_id(_uuid); + compaction_job->set_initiator(BackendOptions::get_localhost() + ':' + + std::to_string(config::heartbeat_service_port)); + compaction_job->set_type(cloud::TabletCompactionJobPB::CUMULATIVE); + compaction_job->set_base_compaction_cnt(_base_compaction_cnt); + compaction_job->set_cumulative_compaction_cnt(_cumulative_compaction_cnt); + 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); + if (config::enable_parallel_cumu_compaction) { + // Set input version range to let meta-service judge version range conflict + compaction_job->add_input_versions(_input_rowsets.front()->start_version()); + compaction_job->add_input_versions(_input_rowsets.back()->end_version()); + } + cloud::StartTabletJobResponse resp; + st = _engine.meta_mgr().prepare_tablet_job(job, &resp); + if (!st.ok()) { + if (resp.status().code() == cloud::STALE_TABLET_CACHE) { + // set last_sync_time to 0 to force sync tablet next time + cloud_tablet()->last_sync_time_s = 0; + } else if (resp.status().code() == cloud::TABLET_NOT_FOUND) { + // tablet not found + cloud_tablet()->recycle_cached_data(); + } else if (resp.status().code() == cloud::JOB_TABLET_BUSY) { + if (config::enable_parallel_cumu_compaction && resp.version_in_compaction_size() > 0 && + ++tried <= 2) { + _max_conflict_version = *std::max_element(resp.version_in_compaction().begin(), + resp.version_in_compaction().end()); + LOG_INFO("retry pick input rowsets") + .tag("job_id", _uuid) + .tag("max_conflict_version", _max_conflict_version) + .tag("tried", tried) + .tag("msg", resp.status().msg()); + goto PREPARE_TRY_AGAIN; + } else { + LOG_WARNING("failed to prepare cumu compaction") + .tag("job_id", _uuid) + .tag("msg", resp.status().msg()); + return Status::Error("no suitable versions"); + } + } + return st; + } + + for (auto& rs : _input_rowsets) { + _input_row_num += rs->num_rows(); + _input_segments += rs->num_segments(); + _input_rowsets_size += rs->data_disk_size(); + } + LOG_INFO("start CloudCumulativeCompaction, tablet_id={}, range=[{}-{}]", _tablet->tablet_id(), + _input_rowsets.front()->start_version(), _input_rowsets.back()->end_version()) + .tag("job_id", _uuid) + .tag("input_rowsets", _input_rowsets.size()) + .tag("input_rows", _input_row_num) + .tag("input_segments", _input_segments) + .tag("input_data_size", _input_rowsets_size) + .tag("tablet_max_version", cloud_tablet()->max_version_unlocked()) + .tag("cumulative_point", cloud_tablet()->cumulative_layer_point()) + .tag("num_rowsets", cloud_tablet()->fetch_add_approximate_num_rowsets(0)) + .tag("cumu_num_rowsets", cloud_tablet()->fetch_add_approximate_cumu_num_rowsets(0)); + return st; +} + +Status CloudCumulativeCompaction::execute_compact() { + TEST_SYNC_POINT_RETURN_WITH_VALUE("CloudCumulativeCompaction::execute_compact_impl", + Status::OK(), this); + using namespace std::chrono; + auto start = steady_clock::now(); + RETURN_IF_ERROR(CloudCompactionMixin::execute_compact()); + LOG_INFO("finish CloudCumulativeCompaction, tablet_id={}, cost={}ms", _tablet->tablet_id(), + duration_cast(steady_clock::now() - start).count()) + .tag("job_id", _uuid) + .tag("input_rowsets", _input_rowsets.size()) + .tag("input_rows", _input_row_num) + .tag("input_segments", _input_segments) + .tag("input_data_size", _input_rowsets_size) + .tag("output_rows", _output_rowset->num_rows()) + .tag("output_segments", _output_rowset->num_segments()) + .tag("output_data_size", _output_rowset->data_disk_size()) + .tag("tablet_max_version", _tablet->max_version_unlocked()) + .tag("cumulative_point", cloud_tablet()->cumulative_layer_point()) + .tag("num_rowsets", cloud_tablet()->fetch_add_approximate_num_rowsets(0)) + .tag("cumu_num_rowsets", cloud_tablet()->fetch_add_approximate_cumu_num_rowsets(0)); + + _state = CompactionState::SUCCESS; + + DorisMetrics::instance()->cumulative_compaction_deltas_total->increment(_input_rowsets.size()); + DorisMetrics::instance()->cumulative_compaction_bytes_total->increment(_input_rowsets_size); + cumu_output_size << _output_rowset->data_disk_size(); + + return Status::OK(); +} + +Status CloudCumulativeCompaction::modify_rowsets() { + // calculate new cumulative point + int64_t input_cumulative_point = cloud_tablet()->cumulative_layer_point(); + int64_t new_cumulative_point = _engine.cumu_compaction_policy()->new_cumulative_point( + cloud_tablet(), _output_rowset, _last_delete_version, input_cumulative_point); + // commit compaction job + cloud::TabletJobInfoPB job; + auto idx = job.mutable_idx(); + idx->set_tablet_id(_tablet->tablet_id()); + idx->set_table_id(_tablet->table_id()); + idx->set_index_id(_tablet->index_id()); + idx->set_partition_id(_tablet->partition_id()); + auto compaction_job = job.add_compaction(); + compaction_job->set_id(_uuid); + compaction_job->set_initiator(BackendOptions::get_localhost() + ':' + + std::to_string(config::heartbeat_service_port)); + compaction_job->set_type(cloud::TabletCompactionJobPB::CUMULATIVE); + compaction_job->set_input_cumulative_point(input_cumulative_point); + compaction_job->set_output_cumulative_point(new_cumulative_point); + compaction_job->set_num_input_rows(_input_row_num); + compaction_job->set_num_output_rows(_output_rowset->num_rows()); + compaction_job->set_size_input_rowsets(_input_rowsets_size); + compaction_job->set_size_output_rowsets(_output_rowset->data_disk_size()); + compaction_job->set_num_input_segments(_input_segments); + compaction_job->set_num_output_segments(_output_rowset->num_segments()); + compaction_job->set_num_input_rowsets(_input_rowsets.size()); + compaction_job->set_num_output_rowsets(1); + compaction_job->add_input_versions(_input_rowsets.front()->start_version()); + compaction_job->add_input_versions(_input_rowsets.back()->end_version()); + compaction_job->add_output_versions(_output_rowset->end_version()); + compaction_job->add_txn_id(_output_rowset->txn_id()); + compaction_job->add_output_rowset_ids(_output_rowset->rowset_id().to_string()); + + /* + DeleteBitmapPtr output_rowset_delete_bitmap = nullptr; + if (_tablet->keys_type() == KeysType::UNIQUE_KEYS && + _tablet->enable_unique_key_merge_on_write()) { + int64_t missed_rows = merger_stats ? merger_stats->merged_rows : -1; + int64_t initiator = + boost::hash_range(_uuid.begin(), _uuid.end()) & std::numeric_limits::max(); + RETURN_IF_ERROR(_tablet->cloud_calc_delete_bitmap_for_compaciton( + _input_rowsets, _output_rowset, _rowid_conversion, compaction_type(), missed_rows, + initiator, output_rowset_delete_bitmap)); + compaction_job->set_delete_bitmap_lock_initiator(initiator); + } + */ + + cloud::FinishTabletJobResponse resp; + auto st = _engine.meta_mgr().commit_tablet_job(job, &resp); + if (!st.ok()) { + if (resp.status().code() == cloud::TABLET_NOT_FOUND) { + cloud_tablet()->recycle_cached_data(); + } + return st; + } + auto& stats = resp.stats(); + LOG(INFO) << "tablet stats=" << stats.ShortDebugString(); + { + std::unique_lock wrlock(_tablet->get_header_lock()); + // clang-format off + cloud_tablet()->set_last_base_compaction_success_time(std::max(cloud_tablet()->last_base_compaction_success_time(), stats.last_base_compaction_time_ms())); + cloud_tablet()->set_last_cumu_compaction_success_time(std::max(cloud_tablet()->last_cumu_compaction_success_time(), stats.last_cumu_compaction_time_ms())); + cloud_tablet()->set_last_full_compaction_success_time(std::max(cloud_tablet()->last_full_compaction_success_time(), stats.last_full_compaction_time_ms())); + // clang-format on + if (cloud_tablet()->cumulative_compaction_cnt() >= stats.cumulative_compaction_cnt()) { + // This could happen while calling `sync_tablet_rowsets` during `commit_tablet_job`, or parallel cumu compactions which are + // committed later increase tablet.cumulative_compaction_cnt (see CloudCompactionTest.parallel_cumu_compaction) + return Status::OK(); + } + // Try to make output rowset visible immediately in tablet cache, instead of waiting for next synchronization from meta-service. + if (stats.cumulative_point() > cloud_tablet()->cumulative_layer_point() && + stats.cumulative_compaction_cnt() != cloud_tablet()->cumulative_compaction_cnt() + 1) { + // This could happen when there are multiple parallel cumu compaction committed, tablet cache lags several + // cumu compactions behind meta-service (stats.cumulative_compaction_cnt > tablet.cumulative_compaction_cnt + 1). + // If `cumu_point` of the tablet cache also falls behind, MUST ONLY synchronize tablet cache from meta-service, + // otherwise may cause the tablet to be unable to synchronize the rowset meta changes generated by other cumu compaction. + return Status::OK(); + } + if (_input_rowsets.size() == 1) { + DCHECK_EQ(_output_rowset->version(), _input_rowsets[0]->version()); + // MUST NOT move input rowset to stale path + cloud_tablet()->add_rowsets({_output_rowset}, true, wrlock); + } else { + cloud_tablet()->delete_rowsets(_input_rowsets, wrlock); + cloud_tablet()->add_rowsets({_output_rowset}, false, wrlock); + } + // ATTN: MUST NOT update `base_compaction_cnt` which are used when sync rowsets, otherwise may cause + // the tablet to be unable to synchronize the rowset meta changes generated by base compaction. + cloud_tablet()->set_cumulative_compaction_cnt(cloud_tablet()->cumulative_compaction_cnt() + + 1); + cloud_tablet()->set_cumulative_layer_point(stats.cumulative_point()); + /* + if (output_rowset_delete_bitmap) { + _tablet->tablet_meta()->delete_bitmap().merge(*output_rowset_delete_bitmap); + } + */ + if (stats.base_compaction_cnt() >= cloud_tablet()->base_compaction_cnt()) { + cloud_tablet()->reset_approximate_stats(stats.num_rowsets(), stats.num_segments(), + stats.num_rows(), stats.data_size()); + } + } + return Status::OK(); +} + +void CloudCumulativeCompaction::garbage_collection() { + //file_cache_garbage_collection(); + cloud::TabletJobInfoPB job; + auto idx = job.mutable_idx(); + idx->set_tablet_id(_tablet->tablet_id()); + idx->set_table_id(_tablet->table_id()); + idx->set_index_id(_tablet->index_id()); + idx->set_partition_id(_tablet->partition_id()); + auto compaction_job = job.add_compaction(); + compaction_job->set_id(_uuid); + compaction_job->set_initiator(BackendOptions::get_localhost() + ':' + + std::to_string(config::heartbeat_service_port)); + compaction_job->set_type(cloud::TabletCompactionJobPB::CUMULATIVE); + if (_tablet->keys_type() == KeysType::UNIQUE_KEYS && + _tablet->enable_unique_key_merge_on_write()) { + int64_t initiator = + boost::hash_range(_uuid.begin(), _uuid.end()) & std::numeric_limits::max(); + compaction_job->set_delete_bitmap_lock_initiator(initiator); + } + auto st = _engine.meta_mgr().abort_tablet_job(job); + if (!st.ok()) { + LOG_WARNING("failed to abort compaction job") + .tag("job_id", _uuid) + .tag("tablet_id", _tablet->tablet_id()) + .error(st); + } +} + +Status CloudCumulativeCompaction::pick_rowsets_to_compact() { + _input_rowsets.clear(); + + std::vector candidate_rowsets; + { + 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); + // Get all rowsets whose version >= `candidate_version` as candidate rowsets + cloud_tablet()->traverse_rowsets( + [&candidate_rowsets, candidate_version](const RowsetSharedPtr& rs) { + if (rs->start_version() >= candidate_version) { + candidate_rowsets.push_back(rs); + } + }); + } + if (candidate_rowsets.empty()) { + return Status::Error("no suitable versions"); + } + std::sort(candidate_rowsets.begin(), candidate_rowsets.end(), Rowset::comparator); + if (auto st = check_version_continuity(candidate_rowsets); !st.ok()) { + DCHECK(false) << st; + return st; + } + + size_t compaction_score = 0; + _engine.cumu_compaction_policy()->pick_input_rowsets( + cloud_tablet(), candidate_rowsets, config::cumulative_compaction_max_deltas, + config::cumulative_compaction_min_deltas, &_input_rowsets, &_last_delete_version, + &compaction_score); + + if (_input_rowsets.empty()) { + return Status::Error("no suitable versions"); + } else if (_input_rowsets.size() == 1 && + !_input_rowsets.front()->rowset_meta()->is_segments_overlapping()) { + VLOG_DEBUG << "there is only one rowset and not overlapping. tablet_id=" + << _tablet->tablet_id() << ", version=" << _input_rowsets.front()->version(); + return Status::Error("no suitable versions"); + } + return Status::OK(); +} + +void CloudCumulativeCompaction::update_cumulative_point() { + cloud::TabletJobInfoPB job; + auto idx = job.mutable_idx(); + idx->set_tablet_id(_tablet->tablet_id()); + idx->set_table_id(_tablet->table_id()); + idx->set_index_id(_tablet->index_id()); + idx->set_partition_id(_tablet->partition_id()); + auto compaction_job = job.add_compaction(); + compaction_job->set_id(_uuid); + compaction_job->set_initiator(BackendOptions::get_localhost() + ':' + + std::to_string(config::heartbeat_service_port)); + compaction_job->set_type(cloud::TabletCompactionJobPB::EMPTY_CUMULATIVE); + compaction_job->set_base_compaction_cnt(_base_compaction_cnt); + compaction_job->set_cumulative_compaction_cnt(_cumulative_compaction_cnt); + int64_t now = time(nullptr); + compaction_job->set_lease(now + config::lease_compaction_interval_seconds); + // No need to set expiration time, since there is no output rowset + cloud::StartTabletJobResponse start_resp; + auto st = _engine.meta_mgr().prepare_tablet_job(job, &start_resp); + if (!st.ok()) { + if (start_resp.status().code() == cloud::STALE_TABLET_CACHE) { + // set last_sync_time to 0 to force sync tablet next time + cloud_tablet()->last_sync_time_s = 0; + } else if (start_resp.status().code() == cloud::TABLET_NOT_FOUND) { + // tablet not found + cloud_tablet()->recycle_cached_data(); + } + LOG_WARNING("failed to update cumulative point to meta srv") + .tag("job_id", _uuid) + .tag("tablet_id", _tablet->tablet_id()) + .error(st); + return; + } + int64_t input_cumulative_point = cloud_tablet()->cumulative_layer_point(); + int64_t output_cumulative_point = _last_delete_version.first + 1; + compaction_job->set_input_cumulative_point(input_cumulative_point); + compaction_job->set_output_cumulative_point(output_cumulative_point); + cloud::FinishTabletJobResponse finish_resp; + st = _engine.meta_mgr().commit_tablet_job(job, &finish_resp); + if (!st.ok()) { + if (finish_resp.status().code() == cloud::TABLET_NOT_FOUND) { + cloud_tablet()->recycle_cached_data(); + } + LOG_WARNING("failed to update cumulative point to meta srv") + .tag("job_id", _uuid) + .tag("tablet_id", _tablet->tablet_id()) + .error(st); + return; + } + LOG_INFO("do empty cumulative compaction to update cumulative point") + .tag("job_id", _uuid) + .tag("tablet_id", _tablet->tablet_id()) + .tag("input_cumulative_point", input_cumulative_point) + .tag("output_cumulative_point", output_cumulative_point); + auto& stats = finish_resp.stats(); + LOG(INFO) << "tablet stats=" << stats.ShortDebugString(); + { + std::lock_guard wrlock(_tablet->get_header_lock()); + // clang-format off + cloud_tablet()->set_last_base_compaction_success_time(std::max(cloud_tablet()->last_base_compaction_success_time(), stats.last_base_compaction_time_ms())); + cloud_tablet()->set_last_cumu_compaction_success_time(std::max(cloud_tablet()->last_cumu_compaction_success_time(), stats.last_cumu_compaction_time_ms())); + // clang-format on + if (cloud_tablet()->cumulative_compaction_cnt() >= stats.cumulative_compaction_cnt()) { + // This could happen while calling `sync_tablet_rowsets` during `commit_tablet_job` + return; + } + // ATTN: MUST NOT update `base_compaction_cnt` which are used when sync rowsets, otherwise may cause + // the tablet to be unable to synchronize the rowset meta changes generated by base compaction. + cloud_tablet()->set_cumulative_compaction_cnt(cloud_tablet()->cumulative_compaction_cnt() + + 1); + cloud_tablet()->set_cumulative_layer_point(stats.cumulative_point()); + if (stats.base_compaction_cnt() >= cloud_tablet()->base_compaction_cnt()) { + cloud_tablet()->reset_approximate_stats(stats.num_rowsets(), stats.num_segments(), + stats.num_rows(), stats.data_size()); + } + } +} + +void CloudCumulativeCompaction::do_lease() { + TEST_INJECTION_POINT_RETURN_WITH_VOID("CloudCumulativeCompaction::do_lease"); + if (_state == CompactionState::SUCCESS) { + return; + } + cloud::TabletJobInfoPB job; + auto idx = job.mutable_idx(); + idx->set_tablet_id(_tablet->tablet_id()); + idx->set_table_id(_tablet->table_id()); + idx->set_index_id(_tablet->index_id()); + idx->set_partition_id(_tablet->partition_id()); + auto compaction_job = job.add_compaction(); + compaction_job->set_id(_uuid); + using namespace std::chrono; + int64_t lease_time = duration_cast(system_clock::now().time_since_epoch()).count() + + config::lease_compaction_interval_seconds * 4; + compaction_job->set_lease(lease_time); + auto st = _engine.meta_mgr().lease_tablet_job(job); + if (!st.ok()) { + LOG_WARNING("failed to lease compaction job") + .tag("job_id", _uuid) + .tag("tablet_id", _tablet->tablet_id()) + .error(st); + } +} + +} // namespace doris diff --git a/be/src/cloud/cloud_cumulative_compaction.h b/be/src/cloud/cloud_cumulative_compaction.h new file mode 100644 index 00000000000000..f353d0f51891fb --- /dev/null +++ b/be/src/cloud/cloud_cumulative_compaction.h @@ -0,0 +1,61 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include + +#include "cloud/cloud_storage_engine.h" +#include "cloud/cloud_tablet.h" +#include "olap/compaction.h" + +namespace doris { + +class CloudCumulativeCompaction : public CloudCompactionMixin { +public: + CloudCumulativeCompaction(CloudStorageEngine& engine, CloudTabletSPtr tablet); + + ~CloudCumulativeCompaction() override; + + Status prepare_compact() override; + Status execute_compact() override; + + void do_lease(); + +private: + Status pick_rowsets_to_compact(); + + std::string_view compaction_name() const override { return "CloudCumulativeCompaction"; } + + Status modify_rowsets() override; + + void garbage_collection() override; + + void update_cumulative_point(); + + ReaderType compaction_type() const override { return ReaderType::READER_CUMULATIVE_COMPACTION; } + + std::string _uuid; + int64_t _input_segments = 0; + int64_t _max_conflict_version = 0; + // Snapshot values when pick input rowsets + int64_t _base_compaction_cnt = 0; + int64_t _cumulative_compaction_cnt = 0; + Version _last_delete_version {-1, -1}; +}; + +} // namespace doris diff --git a/be/src/cloud/cloud_cumulative_compaction_policy.cpp b/be/src/cloud/cloud_cumulative_compaction_policy.cpp new file mode 100644 index 00000000000000..cb4ae1619f2d3c --- /dev/null +++ b/be/src/cloud/cloud_cumulative_compaction_policy.cpp @@ -0,0 +1,206 @@ +// 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. + +#include "cloud/cloud_cumulative_compaction_policy.h" + +#include +#include +#include +#include + +#include "cloud/config.h" +#include "common/config.h" +#include "common/logging.h" +#include "common/sync_point.h" +#include "olap/olap_common.h" +#include "olap/tablet.h" +#include "olap/tablet_meta.h" + +namespace doris { + +CloudSizeBasedCumulativeCompactionPolicy::CloudSizeBasedCumulativeCompactionPolicy( + int64_t promotion_size, double promotion_ratio, int64_t promotion_min_size, + int64_t compaction_min_size) + : _promotion_size(promotion_size), + _promotion_ratio(promotion_ratio), + _promotion_min_size(promotion_min_size), + _compaction_min_size(compaction_min_size) {} + +int64_t CloudSizeBasedCumulativeCompactionPolicy::_level_size(const int64_t size) { + if (size < 1024) return 0; + int64_t max_level = (int64_t)1 + << (sizeof(_promotion_size) * 8 - 1 - __builtin_clzl(_promotion_size / 2)); + if (size >= max_level) return max_level; + return (int64_t)1 << (sizeof(size) * 8 - 1 - __builtin_clzl(size)); +} + +int CloudSizeBasedCumulativeCompactionPolicy::pick_input_rowsets( + CloudTablet* tablet, const std::vector& candidate_rowsets, + const int64_t max_compaction_score, const int64_t min_compaction_score, + std::vector* input_rowsets, Version* last_delete_version, + size_t* compaction_score, bool allow_delete) { + //size_t promotion_size = tablet->cumulative_promotion_size(); + auto max_version = tablet->max_version().first; + int transient_size = 0; + *compaction_score = 0; + int64_t total_size = 0; + for (auto& rowset : candidate_rowsets) { + // check whether this rowset is delete version + if (!allow_delete && rowset->rowset_meta()->has_delete_predicate()) { + *last_delete_version = rowset->version(); + if (!input_rowsets->empty()) { + // we meet a delete version, and there were other versions before. + // we should compact those version before handling them over to base compaction + break; + } else { + // we meet a delete version, and no other versions before, skip it and continue + input_rowsets->clear(); + *compaction_score = 0; + transient_size = 0; + continue; + } + } + if (tablet->tablet_state() == TABLET_NOTREADY) { + // If tablet under alter, keep latest 10 version so that base tablet max version + // not merged in new tablet, and then we can copy data from base tablet + if (rowset->version().second < max_version - 10) { + continue; + } + } + if (*compaction_score >= max_compaction_score) { + // got enough segments + break; + } + *compaction_score += rowset->rowset_meta()->get_compaction_score(); + total_size += rowset->rowset_meta()->total_disk_size(); + + transient_size += 1; + input_rowsets->push_back(rowset); + } + + // if there is delete version, do compaction directly + if (last_delete_version->first != -1) { + if (input_rowsets->size() == 1) { + auto rs_meta = input_rowsets->front()->rowset_meta(); + // if there is only one rowset and not overlapping, + // we do not need to do cumulative compaction + if (!rs_meta->is_segments_overlapping()) { + input_rowsets->clear(); + *compaction_score = 0; + } + } + return transient_size; + } + + auto rs_begin = input_rowsets->begin(); + size_t new_compaction_score = *compaction_score; + while (rs_begin != input_rowsets->end()) { + auto& rs_meta = (*rs_begin)->rowset_meta(); + int current_level = _level_size(rs_meta->total_disk_size()); + int remain_level = _level_size(total_size - rs_meta->total_disk_size()); + // if current level less then remain level, input rowsets contain current rowset + // and process return; otherwise, input rowsets do not contain current rowset. + if (current_level <= remain_level) { + break; + } + total_size -= rs_meta->total_disk_size(); + new_compaction_score -= rs_meta->get_compaction_score(); + ++rs_begin; + } + if (rs_begin == input_rowsets->end()) { // No suitable level size found in `input_rowsets` + if (config::prioritize_query_perf_in_compaction && tablet->keys_type() != DUP_KEYS) { + // While tablet's key type is not `DUP_KEYS`, compacting rowset in such tablets has a significant + // positive impact on queries and reduces space amplification, so we ignore level limitation and + // pick candidate rowsets as input rowsets. + return transient_size; + } else if (*compaction_score >= max_compaction_score) { + // Score of `input_rowsets` exceed max compaction score, which means `input_rowsets` will never change and + // this tablet will never execute cumulative compaction. MUST execute compaction on these `input_rowsets` + // to reduce compaction score. + RowsetSharedPtr rs_with_max_score; + uint32_t max_score = 1; + for (auto& rs : *input_rowsets) { + if (rs->rowset_meta()->get_compaction_score() > max_score) { + max_score = rs->rowset_meta()->get_compaction_score(); + rs_with_max_score = rs; + } + } + if (rs_with_max_score) { + input_rowsets->clear(); + input_rowsets->push_back(std::move(rs_with_max_score)); + *compaction_score = max_score; + return transient_size; + } + // Exceeding max compaction score, do compaction on all candidate rowsets anyway + return transient_size; + } + } + input_rowsets->erase(input_rowsets->begin(), rs_begin); + *compaction_score = new_compaction_score; + + VLOG_CRITICAL << "cumulative compaction size_based policy, compaction_score = " + << *compaction_score << ", total_size = " + << total_size + //<< ", calc promotion size value = " << promotion_size + << ", tablet = " << tablet->tablet_id() << ", input_rowset size " + << input_rowsets->size(); + + // empty return + if (input_rowsets->empty()) { + return transient_size; + } + + // if we have a sufficient number of segments, we should process the compaction. + // otherwise, we check number of segments and total_size whether can do compaction. + if (total_size < _compaction_min_size && *compaction_score < min_compaction_score) { + input_rowsets->clear(); + *compaction_score = 0; + } else if (total_size >= _compaction_min_size && input_rowsets->size() == 1) { + auto rs_meta = input_rowsets->front()->rowset_meta(); + // if there is only one rowset and not overlapping, + // we do not need to do compaction + if (!rs_meta->is_segments_overlapping()) { + input_rowsets->clear(); + *compaction_score = 0; + } + } + return transient_size; +} + +int64_t CloudSizeBasedCumulativeCompactionPolicy::cloud_promotion_size(CloudTablet* t) const { + int64_t promotion_size = t->base_size() * _promotion_ratio; + // promotion_size is between _size_based_promotion_size and _size_based_promotion_min_size + return promotion_size > _promotion_size ? _promotion_size + : promotion_size < _promotion_min_size ? _promotion_min_size + : promotion_size; +} + +int64_t CloudSizeBasedCumulativeCompactionPolicy::new_cumulative_point( + CloudTablet* tablet, const RowsetSharedPtr& output_rowset, Version& last_delete_version, + int64_t last_cumulative_point) { + TEST_INJECTION_POINT_RETURN_WITH_VALUE("new_cumulative_point", int64_t(0), output_rowset.get(), + last_cumulative_point); + // if rowsets have delete version, move to the last directly. + // if rowsets have no delete version, check output_rowset total disk size satisfies promotion size. + return output_rowset->start_version() == last_cumulative_point && + (last_delete_version.first != -1 || + output_rowset->data_disk_size() >= cloud_promotion_size(tablet)) + ? output_rowset->end_version() + 1 + : last_cumulative_point; +} + +} // namespace doris diff --git a/be/src/cloud/cloud_cumulative_compaction_policy.h b/be/src/cloud/cloud_cumulative_compaction_policy.h new file mode 100644 index 00000000000000..9ca9a207b9f0a4 --- /dev/null +++ b/be/src/cloud/cloud_cumulative_compaction_policy.h @@ -0,0 +1,73 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include +#include + +#include +#include +#include + +#include "cloud/cloud_tablet.h" +#include "common/config.h" +#include "olap/rowset/rowset.h" +#include "olap/rowset/rowset_meta.h" + +namespace doris { + +class Tablet; +struct Version; + +class CloudSizeBasedCumulativeCompactionPolicy { +public: + CloudSizeBasedCumulativeCompactionPolicy( + int64_t promotion_size = config::compaction_promotion_size_mbytes * 1024 * 1024, + double promotion_ratio = config::compaction_promotion_ratio, + int64_t promotion_min_size = config::compaction_promotion_min_size_mbytes * 1024 * 1024, + int64_t compaction_min_size = config::compaction_min_size_mbytes * 1024 * 1024); + + ~CloudSizeBasedCumulativeCompactionPolicy() {} + + int64_t new_cumulative_point(CloudTablet* tablet, const RowsetSharedPtr& output_rowset, + Version& last_delete_version, int64_t last_cumulative_point); + + int pick_input_rowsets(CloudTablet* tablet, + const std::vector& candidate_rowsets, + const int64_t max_compaction_score, const int64_t min_compaction_score, + std::vector* input_rowsets, + Version* last_delete_version, size_t* compaction_score, + bool allow_delete = false); + +private: + int64_t _level_size(const int64_t size); + + int64_t cloud_promotion_size(CloudTablet* tablet) const; + +private: + /// cumulative compaction promotion size, unit is byte. + int64_t _promotion_size; + /// cumulative compaction promotion ratio of base rowset total disk size. + double _promotion_ratio; + /// cumulative compaction promotion min size, unit is byte. + int64_t _promotion_min_size; + /// lower bound size to do compaction compaction. + int64_t _compaction_min_size; +}; + +} // namespace doris diff --git a/be/src/cloud/cloud_delete_task.cpp b/be/src/cloud/cloud_delete_task.cpp index 2e3913533c336a..7895799f9be9d0 100644 --- a/be/src/cloud/cloud_delete_task.cpp +++ b/be/src/cloud/cloud_delete_task.cpp @@ -83,7 +83,15 @@ Status CloudDeleteTask::execute(CloudStorageEngine& engine, const TPushReq& requ tablet->fetch_add_approximate_num_rowsets(1); tablet->fetch_add_approximate_cumu_num_rowsets(1); - // TODO(liaoxin): set_tablet_txn_info if enable_unique_key_merge_on_write + // TODO(liaoxin) delete operator don't send calculate delete bitmap task from fe, + // then we don't need to set_txn_related_delete_bitmap here. + if (tablet->enable_unique_key_merge_on_write()) { + DeleteBitmapPtr delete_bitmap = std::make_shared(tablet->tablet_id()); + RowsetIdUnorderedSet rowset_ids; + engine.txn_delete_bitmap_cache().set_tablet_txn_info( + request.transaction_id, tablet->tablet_id(), delete_bitmap, rowset_ids, rowset, + request.timeout, nullptr); + } return Status::OK(); } diff --git a/be/src/cloud/cloud_delta_writer.cpp b/be/src/cloud/cloud_delta_writer.cpp index 9d549997cd7975..f0b148448f74a8 100644 --- a/be/src/cloud/cloud_delta_writer.cpp +++ b/be/src/cloud/cloud_delta_writer.cpp @@ -114,4 +114,8 @@ Status CloudDeltaWriter::commit_rowset() { return _engine.meta_mgr().commit_rowset(*rowset_meta(), true); } +Status CloudDeltaWriter::set_txn_related_delete_bitmap() { + return rowset_builder()->set_txn_related_delete_bitmap(); +} + } // namespace doris diff --git a/be/src/cloud/cloud_delta_writer.h b/be/src/cloud/cloud_delta_writer.h index 61a025130f0b0b..c4153a075db32e 100644 --- a/be/src/cloud/cloud_delta_writer.h +++ b/be/src/cloud/cloud_delta_writer.h @@ -51,6 +51,8 @@ class CloudDeltaWriter final : public BaseDeltaWriter { Status commit_rowset(); + Status set_txn_related_delete_bitmap(); + private: // Convert `_rowset_builder` from `BaseRowsetBuilder` to `CloudRowsetBuilder` CloudRowsetBuilder* rowset_builder(); diff --git a/be/src/cloud/cloud_engine_calc_delete_bitmap_task.cpp b/be/src/cloud/cloud_engine_calc_delete_bitmap_task.cpp new file mode 100644 index 00000000000000..2ff9b8e91ba9e8 --- /dev/null +++ b/be/src/cloud/cloud_engine_calc_delete_bitmap_task.cpp @@ -0,0 +1,168 @@ +// 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. + +#include "cloud/cloud_engine_calc_delete_bitmap_task.h" + +#include + +#include "cloud/cloud_meta_mgr.h" +#include "cloud/cloud_tablet.h" +#include "common/status.h" +#include "olap/base_tablet.h" +#include "olap/olap_common.h" +#include "olap/rowset/rowset.h" +#include "olap/tablet_fwd.h" +#include "olap/tablet_meta.h" +#include "olap/txn_manager.h" +#include "olap/utils.h" + +namespace doris { + +CloudEngineCalcDeleteBitmapTask::CloudEngineCalcDeleteBitmapTask( + CloudStorageEngine& engine, const TCalcDeleteBitmapRequest& cal_delete_bitmap_req, + std::vector* error_tablet_ids, std::vector* succ_tablet_ids) + : _engine(engine), + _cal_delete_bitmap_req(cal_delete_bitmap_req), + _error_tablet_ids(error_tablet_ids), + _succ_tablet_ids(succ_tablet_ids) {} + +void CloudEngineCalcDeleteBitmapTask::add_error_tablet_id(int64_t tablet_id, const Status& err) { + std::lock_guard lck(_mutex); + _error_tablet_ids->push_back(tablet_id); + if (_res.ok() || _res.is()) { + _res = err; + } +} + +void CloudEngineCalcDeleteBitmapTask::add_succ_tablet_id(int64_t tablet_id) { + std::lock_guard lck(_mutex); + _succ_tablet_ids->push_back(tablet_id); +} + +Status CloudEngineCalcDeleteBitmapTask::execute() { + int64_t transaction_id = _cal_delete_bitmap_req.transaction_id; + OlapStopWatch watch; + VLOG_NOTICE << "begin to calculate delete bitmap. transaction_id=" << transaction_id; + std::unique_ptr token = + _engine.calc_tablet_delete_bitmap_task_thread_pool()->new_token( + ThreadPool::ExecutionMode::CONCURRENT); + + for (const auto& partition : _cal_delete_bitmap_req.partitions) { + int64_t version = partition.version; + for (auto tablet_id : partition.tablet_ids) { + auto base_tablet = DORIS_TRY(_engine.get_tablet(tablet_id)); + std::shared_ptr tablet = + std::dynamic_pointer_cast(base_tablet); + if (tablet == nullptr) { + LOG(WARNING) << "can't get tablet when calculate delete bitmap. tablet_id=" + << tablet_id; + _error_tablet_ids->push_back(tablet_id); + _res = Status::Error( + "can't get tablet when calculate delete bitmap. tablet_id={}", tablet_id); + break; + } + + Status st = tablet->sync_rowsets(); + if (!st.ok() && !st.is()) { + return st; + } + if (st.is()) [[unlikely]] { + add_succ_tablet_id(tablet->tablet_id()); + LOG(INFO) + << "tablet is under alter process, delete bitmap will be calculated later, " + "tablet_id: " + << tablet->tablet_id() << " txn_id: " << transaction_id + << ", request_version=" << version; + continue; + } + int64_t max_version = tablet->max_version_unlocked(); + if (version != max_version + 1) { + _error_tablet_ids->push_back(tablet_id); + _res = Status::Error( + "version not continuous"); + LOG(WARNING) << "version not continuous, current max version=" << max_version + << ", request_version=" << version + << " tablet_id=" << tablet->tablet_id(); + break; + } + + auto tablet_calc_delete_bitmap_ptr = std::make_shared( + _engine, this, tablet, transaction_id, version); + auto submit_st = token->submit_func([=]() { tablet_calc_delete_bitmap_ptr->handle(); }); + if (!submit_st.ok()) { + _res = submit_st; + break; + } + } + } + // wait for all finished + token->wait(); + + LOG(INFO) << "finish to calculate delete bitmap on transaction." + << "transaction_id=" << transaction_id << ", cost(us): " << watch.get_elapse_time_us() + << ", error_tablet_size=" << _error_tablet_ids->size() + << ", res=" << _res.to_string(); + return _res; +} + +CloudTabletCalcDeleteBitmapTask::CloudTabletCalcDeleteBitmapTask( + CloudStorageEngine& engine, CloudEngineCalcDeleteBitmapTask* engine_task, + std::shared_ptr tablet, int64_t transaction_id, int64_t version) + : _engine(engine), + _engine_calc_delete_bitmap_task(engine_task), + _tablet(tablet), + _transaction_id(transaction_id), + _version(version) {} + +void CloudTabletCalcDeleteBitmapTask::handle() const { + RowsetSharedPtr rowset; + DeleteBitmapPtr delete_bitmap; + RowsetIdUnorderedSet rowset_ids; + std::shared_ptr partial_update_info; + int64_t txn_expiration; + Status status = _engine.txn_delete_bitmap_cache().get_tablet_txn_info( + _transaction_id, _tablet->tablet_id(), &rowset, &delete_bitmap, &rowset_ids, + &txn_expiration, &partial_update_info); + if (status != Status::OK()) { + LOG(WARNING) << "failed to get tablet txn info. tablet_id=" << _tablet->tablet_id() + << ", txn_id=" << _transaction_id << ", status=" << status; + _engine_calc_delete_bitmap_task->add_error_tablet_id(_tablet->tablet_id(), status); + return; + } + + rowset->set_version(Version(_version, _version)); + TabletTxnInfo txn_info; + txn_info.rowset = rowset; + txn_info.delete_bitmap = delete_bitmap; + txn_info.rowset_ids = rowset_ids; + txn_info.partial_update_info = partial_update_info; + status = CloudTablet::update_delete_bitmap(_tablet, &txn_info, _transaction_id, txn_expiration); + if (status != Status::OK()) { + LOG(WARNING) << "failed to calculate delete bitmap. rowset_id=" << rowset->rowset_id() + << ", tablet_id=" << _tablet->tablet_id() << ", txn_id=" << _transaction_id + << ", status=" << status; + _engine_calc_delete_bitmap_task->add_error_tablet_id(_tablet->tablet_id(), status); + return; + } + + _engine_calc_delete_bitmap_task->add_succ_tablet_id(_tablet->tablet_id()); + LOG(INFO) << "calculate delete bitmap successfully on tablet" + << ", table_id=" << _tablet->table_id() << ", transaction_id=" << _transaction_id + << ", num_rows=" << rowset->num_rows() << ", res=" << status; +} + +} // namespace doris diff --git a/be/src/cloud/cloud_engine_calc_delete_bitmap_task.h b/be/src/cloud/cloud_engine_calc_delete_bitmap_task.h new file mode 100644 index 00000000000000..514f1b059d5be6 --- /dev/null +++ b/be/src/cloud/cloud_engine_calc_delete_bitmap_task.h @@ -0,0 +1,72 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include + +#include "cloud/cloud_storage_engine.h" +#include "cloud/cloud_tablet.h" +#include "gen_cpp/AgentService_types.h" +#include "olap/tablet_fwd.h" +#include "olap/task/engine_task.h" + +namespace doris { + +class CloudEngineCalcDeleteBitmapTask; + +class CloudTabletCalcDeleteBitmapTask { +public: + CloudTabletCalcDeleteBitmapTask(CloudStorageEngine& engine, + CloudEngineCalcDeleteBitmapTask* engine_task, + std::shared_ptr tablet, int64_t transaction_id, + int64_t version); + ~CloudTabletCalcDeleteBitmapTask() = default; + + void handle() const; + +private: + CloudStorageEngine& _engine; + CloudEngineCalcDeleteBitmapTask* _engine_calc_delete_bitmap_task; + + std::shared_ptr _tablet; + int64_t _transaction_id; + int64_t _version; +}; + +class CloudEngineCalcDeleteBitmapTask : public EngineTask { +public: + CloudEngineCalcDeleteBitmapTask(CloudStorageEngine& engine, + const TCalcDeleteBitmapRequest& cal_delete_bitmap_req, + std::vector* error_tablet_ids, + std::vector* succ_tablet_ids = nullptr); + Status execute() override; + + void add_error_tablet_id(int64_t tablet_id, const Status& err); + void add_succ_tablet_id(int64_t tablet_id); + +private: + CloudStorageEngine& _engine; + const TCalcDeleteBitmapRequest& _cal_delete_bitmap_req; + std::mutex _mutex; + std::vector* _error_tablet_ids; + std::vector* _succ_tablet_ids; + + Status _res; +}; + +} // namespace doris diff --git a/be/src/cloud/cloud_full_compaction.cpp b/be/src/cloud/cloud_full_compaction.cpp new file mode 100644 index 00000000000000..189a101fa03c70 --- /dev/null +++ b/be/src/cloud/cloud_full_compaction.cpp @@ -0,0 +1,356 @@ +// 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. + +#include "cloud/cloud_full_compaction.h" + +#include + +#include "cloud/cloud_meta_mgr.h" +#include "cloud/cloud_storage_engine.h" +#include "cloud/config.h" +#include "common/config.h" +#include "common/sync_point.h" +#include "gen_cpp/cloud.pb.h" +#include "olap/compaction.h" +#include "olap/rowset/beta_rowset.h" +#include "olap/tablet_meta.h" +#include "service/backend_options.h" +#include "util/thread.h" +#include "util/uuid_generator.h" +#include "vec/columns/column.h" + +namespace doris { +using namespace ErrorCode; + +bvar::Adder full_output_size("full_compaction", "output_size"); + +CloudFullCompaction::CloudFullCompaction(CloudStorageEngine& engine, CloudTabletSPtr tablet) + : CloudCompactionMixin(engine, std::move(tablet), + "BaseCompaction:" + std::to_string(tablet->tablet_id())) { + auto uuid = UUIDGenerator::instance()->next_uuid(); + std::stringstream ss; + ss << uuid; + _uuid = ss.str(); +} + +CloudFullCompaction::~CloudFullCompaction() = default; + +Status CloudFullCompaction::prepare_compact() { + if (_tablet->tablet_state() != TABLET_RUNNING) { + return Status::InternalError("invalid tablet state. tablet_id={}", _tablet->tablet_id()); + } + + // always sync lastest rowset for full compaction + RETURN_IF_ERROR(cloud_tablet()->sync_rowsets()); + + RETURN_IF_ERROR(pick_rowsets_to_compact()); + + // prepare compaction job + cloud::TabletJobInfoPB job; + auto idx = job.mutable_idx(); + idx->set_tablet_id(_tablet->tablet_id()); + idx->set_table_id(_tablet->table_id()); + idx->set_index_id(_tablet->index_id()); + idx->set_partition_id(_tablet->partition_id()); + auto compaction_job = job.add_compaction(); + compaction_job->set_id(_uuid); + compaction_job->set_initiator(BackendOptions::get_localhost() + ':' + + std::to_string(config::heartbeat_service_port)); + compaction_job->set_type(cloud::TabletCompactionJobPB::FULL); + compaction_job->set_base_compaction_cnt(_base_compaction_cnt); + compaction_job->set_cumulative_compaction_cnt(_cumulative_compaction_cnt); + 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); + // Set input version range to let meta-service judge version range conflict + compaction_job->add_input_versions(_input_rowsets.front()->start_version()); + compaction_job->add_input_versions(_input_rowsets.back()->end_version()); + cloud::StartTabletJobResponse resp; + auto st = _engine.meta_mgr().prepare_tablet_job(job, &resp); + if (!st.ok()) { + if (resp.status().code() == cloud::STALE_TABLET_CACHE) { + // set last_sync_time to 0 to force sync tablet next time + cloud_tablet()->last_sync_time_s = 0; + } else if (resp.status().code() == cloud::TABLET_NOT_FOUND) { + // tablet not found + cloud_tablet()->recycle_cached_data(); + } + return st; + } + + for (auto& rs : _input_rowsets) { + _input_row_num += rs->num_rows(); + _input_segments += rs->num_segments(); + _input_rowsets_size += rs->data_disk_size(); + } + LOG_INFO("start CloudFullCompaction, tablet_id={}, range=[{}-{}]", _tablet->tablet_id(), + _input_rowsets.front()->start_version(), _input_rowsets.back()->end_version()) + .tag("job_id", _uuid) + .tag("input_rowsets", _input_rowsets.size()) + .tag("input_rows", _input_row_num) + .tag("input_segments", _input_segments) + .tag("input_data_size", _input_rowsets_size); + return st; +} + +Status CloudFullCompaction::pick_rowsets_to_compact() { + _input_rowsets.clear(); + { + std::shared_lock rlock(_tablet->get_header_lock()); + _base_compaction_cnt = cloud_tablet()->base_compaction_cnt(); + _cumulative_compaction_cnt = cloud_tablet()->cumulative_compaction_cnt(); + _input_rowsets = cloud_tablet()->pick_candidate_rowsets_to_full_compaction(); + } + if (auto st = check_version_continuity(_input_rowsets); !st.ok()) { + DCHECK(false) << st; + return st; + } + if (_input_rowsets.size() <= 1) { + return Status::Error( + "insuffient compation input rowset, #rowsets={}", _input_rowsets.size()); + } + + if (_input_rowsets.size() == 2 && _input_rowsets[0]->end_version() == 1) { + // the tablet is with rowset: [0-1], [2-y] + // and [0-1] has no data. in this situation, no need to do full compaction. + return Status::Error("no suitable versions for compaction"); + } + + return Status::OK(); +} + +Status CloudFullCompaction::execute_compact() { + TEST_SYNC_POINT_RETURN_WITH_VALUE("CloudFullCompaction::execute_compact_impl", Status::OK(), + this); + if (config::enable_base_compaction_idle_sched) { + Thread::set_idle_sched(); + } + + SCOPED_ATTACH_TASK(_mem_tracker); + + using namespace std::chrono; + auto start = steady_clock::now(); + RETURN_IF_ERROR(CloudCompactionMixin::execute_compact()); + LOG_INFO("finish CloudFullCompaction, tablet_id={}, cost={}ms", _tablet->tablet_id(), + duration_cast(steady_clock::now() - start).count()) + .tag("job_id", _uuid) + .tag("input_rowsets", _input_rowsets.size()) + .tag("input_rows", _input_row_num) + .tag("input_segments", _input_segments) + .tag("input_data_size", _input_rowsets_size) + .tag("output_rows", _output_rowset->num_rows()) + .tag("output_segments", _output_rowset->num_segments()) + .tag("output_data_size", _output_rowset->data_disk_size()); + + _state = CompactionState::SUCCESS; + + DorisMetrics::instance()->full_compaction_deltas_total->increment(_input_rowsets.size()); + DorisMetrics::instance()->full_compaction_bytes_total->increment(_input_rowsets_size); + full_output_size << _output_rowset->data_disk_size(); + + return Status::OK(); +} + +Status CloudFullCompaction::modify_rowsets() { + // commit compaction job + cloud::TabletJobInfoPB job; + auto idx = job.mutable_idx(); + idx->set_tablet_id(_tablet->tablet_id()); + idx->set_table_id(_tablet->table_id()); + idx->set_index_id(_tablet->index_id()); + idx->set_partition_id(_tablet->partition_id()); + auto compaction_job = job.add_compaction(); + compaction_job->set_id(_uuid); + compaction_job->set_initiator(BackendOptions::get_localhost() + ':' + + std::to_string(config::heartbeat_service_port)); + compaction_job->set_type(cloud::TabletCompactionJobPB::FULL); + compaction_job->set_input_cumulative_point(cloud_tablet()->cumulative_layer_point()); + compaction_job->set_output_cumulative_point(_output_rowset->end_version() + 1); + compaction_job->set_num_input_rows(_input_row_num); + compaction_job->set_num_output_rows(_output_rowset->num_rows()); + compaction_job->set_size_input_rowsets(_input_rowsets_size); + compaction_job->set_size_output_rowsets(_output_rowset->data_disk_size()); + compaction_job->set_num_input_segments(_input_segments); + compaction_job->set_num_output_segments(_output_rowset->num_segments()); + compaction_job->set_num_input_rowsets(_input_rowsets.size()); + compaction_job->set_num_output_rowsets(1); + compaction_job->add_input_versions(_input_rowsets.front()->start_version()); + compaction_job->add_input_versions(_input_rowsets.back()->end_version()); + compaction_job->add_output_versions(_output_rowset->end_version()); + compaction_job->add_txn_id(_output_rowset->txn_id()); + compaction_job->add_output_rowset_ids(_output_rowset->rowset_id().to_string()); + + DeleteBitmapPtr output_rowset_delete_bitmap = nullptr; + int64_t initiator = + boost::hash_range(_uuid.begin(), _uuid.end()) & std::numeric_limits::max(); + // TODO(luwei) update delete bitmap + //RETURN_IF_ERROR(_cloud_full_compaction_update_delete_bitmap(initiator)); + compaction_job->set_delete_bitmap_lock_initiator(initiator); + + cloud::FinishTabletJobResponse resp; + auto st = _engine.meta_mgr().commit_tablet_job(job, &resp); + if (!st.ok()) { + if (resp.status().code() == cloud::TABLET_NOT_FOUND) { + cloud_tablet()->recycle_cached_data(); + } + return st; + } + auto& stats = resp.stats(); + LOG(INFO) << "tablet stats=" << stats.ShortDebugString(); + + { + std::unique_lock wrlock(_tablet->get_header_lock()); + // clang-format off + cloud_tablet()->set_last_base_compaction_success_time(std::max(cloud_tablet()->last_base_compaction_success_time(), stats.last_base_compaction_time_ms())); + cloud_tablet()->set_last_cumu_compaction_success_time(std::max(cloud_tablet()->last_cumu_compaction_success_time(), stats.last_cumu_compaction_time_ms())); + cloud_tablet()->set_last_full_compaction_success_time(std::max(cloud_tablet()->last_full_compaction_success_time(), stats.last_full_compaction_time_ms())); + // clang-format on + if (cloud_tablet()->base_compaction_cnt() >= stats.base_compaction_cnt()) { + // This could happen while calling `sync_tablet_rowsets` during `commit_tablet_job` + return Status::OK(); + } + // Try to make output rowset visible immediately in tablet cache, instead of waiting for next synchronization from meta-service. + cloud_tablet()->delete_rowsets(_input_rowsets, wrlock); + cloud_tablet()->add_rowsets({_output_rowset}, false, wrlock); + cloud_tablet()->set_base_compaction_cnt(cloud_tablet()->base_compaction_cnt() + 1); + cloud_tablet()->set_cumulative_layer_point(stats.cumulative_point()); + if (output_rowset_delete_bitmap) { + _tablet->tablet_meta()->delete_bitmap().merge(*output_rowset_delete_bitmap); + } + if (stats.cumulative_compaction_cnt() >= cloud_tablet()->cumulative_compaction_cnt()) { + cloud_tablet()->reset_approximate_stats(stats.num_rowsets(), stats.num_segments(), + stats.num_rows(), stats.data_size()); + } + } + return Status::OK(); +} + +void CloudFullCompaction::garbage_collection() { + //file_cache_garbage_collection(); + cloud::TabletJobInfoPB job; + auto idx = job.mutable_idx(); + idx->set_tablet_id(_tablet->tablet_id()); + idx->set_table_id(_tablet->table_id()); + idx->set_index_id(_tablet->index_id()); + idx->set_partition_id(_tablet->partition_id()); + auto compaction_job = job.add_compaction(); + compaction_job->set_id(_uuid); + compaction_job->set_initiator(BackendOptions::get_localhost() + ':' + + std::to_string(config::heartbeat_service_port)); + compaction_job->set_type(cloud::TabletCompactionJobPB::FULL); + if (_tablet->keys_type() == KeysType::UNIQUE_KEYS && + _tablet->enable_unique_key_merge_on_write()) { + int64_t initiator = + boost::hash_range(_uuid.begin(), _uuid.end()) & std::numeric_limits::max(); + compaction_job->set_delete_bitmap_lock_initiator(initiator); + } + auto st = _engine.meta_mgr().abort_tablet_job(job); + if (!st.ok()) { + LOG_WARNING("failed to abort compaction job") + .tag("job_id", _uuid) + .tag("tablet_id", _tablet->tablet_id()) + .error(st); + } +} + +void CloudFullCompaction::do_lease() { + cloud::TabletJobInfoPB job; + auto idx = job.mutable_idx(); + idx->set_tablet_id(_tablet->tablet_id()); + idx->set_table_id(_tablet->table_id()); + idx->set_index_id(_tablet->index_id()); + idx->set_partition_id(_tablet->partition_id()); + auto compaction_job = job.add_compaction(); + compaction_job->set_id(_uuid); + using namespace std::chrono; + int64_t lease_time = duration_cast(system_clock::now().time_since_epoch()).count() + + config::lease_compaction_interval_seconds * 4; + compaction_job->set_lease(lease_time); + auto st = _engine.meta_mgr().lease_tablet_job(job); + if (!st.ok()) { + LOG_WARNING("failed to lease compaction job") + .tag("job_id", _uuid) + .tag("tablet_id", _tablet->tablet_id()) + .error(st); + } +} + +/* +Status CloudFullCompaction::_cloud_full_compaction_update_delete_bitmap(int64_t initiator) { + std::vector tmp_rowsets {}; + DeleteBitmapPtr delete_bitmap = + std::make_shared(_tablet->tablet_meta()->tablet_id()); + RETURN_IF_ERROR(_engine.meta_mgr().sync_tablet_rowsets(cloud_tablet())); + int64_t max_version = cloud_tablet()->max_version().second; + DCHECK(max_version >= _output_rowset->version().second); + if (max_version > _output_rowset->version().second) { + cloud_tablet()->capture_consistent_rowsets({_output_rowset->version().second + 1, max_version}, + &tmp_rowsets); + } + for (const auto& it : tmp_rowsets) { + const int64_t& cur_version = it->rowset_meta()->start_version(); + RETURN_IF_ERROR(_cloud_full_compaction_calc_delete_bitmap(it, cur_version, delete_bitmap)); + } + + RETURN_IF_ERROR(_engine.meta_mgr().get_delete_bitmap_update_lock(*cloud_tablet(), -1, initiator)); + RETURN_IF_ERROR(_engine.meta_mgr().sync_tablet_rowsets(cloud_tablet())); + std::lock_guard rowset_update_lock(_tablet->get_rowset_update_lock()); + std::lock_guard header_lock(_tablet->get_header_lock()); + for (const auto& it : _tablet->rowset_map()) { + const int64_t& cur_version = it.first.first; + const RowsetSharedPtr& published_rowset = it.second; + if (cur_version > max_version) { + RETURN_IF_ERROR(_cloud_full_compaction_calc_delete_bitmap(published_rowset, cur_version, + delete_bitmap)); + } + } + RETURN_IF_ERROR(_engine.meta_mgr().update_delete_bitmap(*cloud_tablet(), -1, initiator, + delete_bitmap.get())); + _tablet->tablet_meta()->delete_bitmap().merge(*delete_bitmap); + return Status::OK(); +} + +Status CloudFullCompaction::_cloud_full_compaction_calc_delete_bitmap( + const RowsetSharedPtr& published_rowset, const int64_t& cur_version, + const DeleteBitmapPtr& delete_bitmap) { + std::vector segments; + auto beta_rowset = reinterpret_cast(published_rowset.get()); + RETURN_IF_ERROR(beta_rowset->load_segments(&segments)); + std::vector specified_rowsets(1, _output_rowset); + + OlapStopWatch watch; + auto token = _engine->calc_delete_bitmap_executor()->create_token(); + RETURN_IF_ERROR(_tablet->calc_delete_bitmap(published_rowset, segments, specified_rowsets, + delete_bitmap, cur_version, token.get(), + _output_rs_writer.get())); + RETURN_IF_ERROR(token->wait()); + size_t total_rows = std::accumulate( + segments.begin(), segments.end(), 0, + [](size_t sum, const segment_v2::SegmentSharedPtr& s) { return sum += s->num_rows(); }); + VLOG_DEBUG << "[Full compaction] construct delete bitmap tablet: " << _tablet->tablet_id() + << ", published rowset version: [" << published_rowset->version().first << "-" + << published_rowset->version().second << "]" + << ", full compaction rowset version: [" << _output_rowset->version().first << "-" + << _output_rowset->version().second << "]" + << ", cost: " << watch.get_elapse_time_us() << "(us), total rows: " << total_rows; + return Status::OK(); +} +*/ + +} // namespace doris diff --git a/be/src/cloud/cloud_full_compaction.h b/be/src/cloud/cloud_full_compaction.h new file mode 100644 index 00000000000000..eb6016779d37ce --- /dev/null +++ b/be/src/cloud/cloud_full_compaction.h @@ -0,0 +1,64 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include + +#include "cloud/cloud_storage_engine.h" +#include "cloud/cloud_tablet.h" +#include "olap/compaction.h" + +namespace doris { + +class CloudFullCompaction : public CloudCompactionMixin { +public: + CloudFullCompaction(CloudStorageEngine& engine, CloudTabletSPtr tablet); + + ~CloudFullCompaction() override; + + Status prepare_compact() override; + Status execute_compact() override; + + void do_lease(); + +protected: + Status pick_rowsets_to_compact(); + + std::string_view compaction_name() const override { return "CloudFullCompaction"; } + + Status modify_rowsets() override; + void garbage_collection() override; + +private: + /* + Status _cloud_full_compaction_update_delete_bitmap(int64_t initiator); + Status _cloud_full_compaction_calc_delete_bitmap(const RowsetSharedPtr& rowset, + const int64_t& cur_version, + const DeleteBitmapPtr& delete_bitmap); + */ + + ReaderType compaction_type() const override { return ReaderType::READER_FULL_COMPACTION; } + + std::string _uuid; + int64_t _input_segments = 0; + // Snapshot values when pick input rowsets + int64_t _base_compaction_cnt = 0; + int64_t _cumulative_compaction_cnt = 0; +}; + +} // namespace doris diff --git a/be/src/cloud/cloud_rowset_builder.cpp b/be/src/cloud/cloud_rowset_builder.cpp index 2ac7f2e3337295..13f49ae35a526e 100644 --- a/be/src/cloud/cloud_rowset_builder.cpp +++ b/be/src/cloud/cloud_rowset_builder.cpp @@ -34,8 +34,11 @@ CloudRowsetBuilder::~CloudRowsetBuilder() = default; Status CloudRowsetBuilder::init() { _tablet = DORIS_TRY(_engine.get_tablet(_req.tablet_id)); - // TODO(plat1ko): get rowset ids snapshot to calculate delete bitmap - + std::shared_ptr mow_context; + if (_tablet->enable_unique_key_merge_on_write()) { + RETURN_IF_ERROR(std::dynamic_pointer_cast(_tablet)->sync_rowsets()); + RETURN_IF_ERROR(init_mow_context(mow_context)); + } RETURN_IF_ERROR(check_tablet_version_count()); // build tablet schema in request level @@ -55,8 +58,7 @@ Status CloudRowsetBuilder::init() { context.index_id = _req.index_id; context.tablet = _tablet; context.write_type = DataWriteType::TYPE_DIRECT; - // TODO(plat1ko): - // context.mow_context = mow_context; + context.mow_context = mow_context; context.write_file_cache = _req.write_file_cache; context.partial_update_info = _partial_update_info; // New loaded data is always written to latest shared storage @@ -104,4 +106,24 @@ const RowsetMetaSharedPtr& CloudRowsetBuilder::rowset_meta() { return _rowset_writer->rowset_meta(); } +Status CloudRowsetBuilder::set_txn_related_delete_bitmap() { + if (_tablet->enable_unique_key_merge_on_write()) { + if (config::enable_merge_on_write_correctness_check && _rowset->num_rows() != 0) { + auto st = _tablet->check_delete_bitmap_correctness( + _delete_bitmap, _rowset->end_version() - 1, _req.txn_id, _rowset_ids); + if (!st.ok()) { + LOG(WARNING) << fmt::format( + "[tablet_id:{}][txn_id:{}][load_id:{}][partition_id:{}] " + "delete bitmap correctness check failed in commit phase!", + _req.tablet_id, _req.txn_id, UniqueId(_req.load_id).to_string(), + _req.partition_id); + return st; + } + } + _engine.txn_delete_bitmap_cache().set_tablet_txn_info( + _req.txn_id, _tablet->tablet_id(), _delete_bitmap, _rowset_ids, _rowset, + _req.txn_expiration, _partial_update_info); + } + return Status::OK(); +} } // namespace doris diff --git a/be/src/cloud/cloud_rowset_builder.h b/be/src/cloud/cloud_rowset_builder.h index 0ce2cfa514ddc1..05e24e663825fc 100644 --- a/be/src/cloud/cloud_rowset_builder.h +++ b/be/src/cloud/cloud_rowset_builder.h @@ -37,6 +37,8 @@ class CloudRowsetBuilder final : public BaseRowsetBuilder { const RowsetMetaSharedPtr& rowset_meta(); + Status set_txn_related_delete_bitmap(); + private: // Convert `_tablet` from `BaseTablet` to `CloudTablet` CloudTablet* cloud_tablet(); diff --git a/be/src/cloud/cloud_rowset_writer.cpp b/be/src/cloud/cloud_rowset_writer.cpp index f7e4f006194a3d..98f7752b660020 100644 --- a/be/src/cloud/cloud_rowset_writer.cpp +++ b/be/src/cloud/cloud_rowset_writer.cpp @@ -115,9 +115,4 @@ Status CloudRowsetWriter::build(RowsetSharedPtr& rowset) { return Status::OK(); } -Status CloudRowsetWriter::_generate_delete_bitmap(int32_t segment_id) { - // TODO(plat1ko) - return Status::NotSupported("CloudRowsetWriter::_generate_delete_bitmap is not implemented"); -} - } // namespace doris diff --git a/be/src/cloud/cloud_rowset_writer.h b/be/src/cloud/cloud_rowset_writer.h index ba258f9e34864a..1bb2b3d38b8064 100644 --- a/be/src/cloud/cloud_rowset_writer.h +++ b/be/src/cloud/cloud_rowset_writer.h @@ -21,7 +21,7 @@ namespace doris { -class CloudRowsetWriter final : public BaseBetaRowsetWriter { +class CloudRowsetWriter : public BaseBetaRowsetWriter { public: CloudRowsetWriter(); @@ -30,9 +30,6 @@ class CloudRowsetWriter final : public BaseBetaRowsetWriter { Status init(const RowsetWriterContext& rowset_writer_context) override; Status build(RowsetSharedPtr& rowset) override; - -private: - Status _generate_delete_bitmap(int32_t segment_id) override; }; } // namespace doris diff --git a/be/src/cloud/cloud_storage_engine.cpp b/be/src/cloud/cloud_storage_engine.cpp index 439c767211f179..d82e6503c668e7 100644 --- a/be/src/cloud/cloud_storage_engine.cpp +++ b/be/src/cloud/cloud_storage_engine.cpp @@ -17,11 +17,21 @@ #include "cloud/cloud_storage_engine.h" +#include +#include +#include +#include + +#include "cloud/cloud_base_compaction.h" +#include "cloud/cloud_cumulative_compaction.h" +#include "cloud/cloud_cumulative_compaction_policy.h" +#include "cloud/cloud_full_compaction.h" #include "cloud/cloud_meta_mgr.h" -#include "cloud/cloud_tablet.h" #include "cloud/cloud_tablet_mgr.h" +#include "cloud/cloud_txn_delete_bitmap_cache.h" #include "cloud/config.h" #include "io/fs/s3_file_system.h" +#include "olap/cumulative_compaction_policy.h" #include "olap/memtable_flush_executor.h" #include "olap/storage_policy.h" #include "runtime/memory/cache_manager.h" @@ -30,10 +40,30 @@ namespace doris { using namespace std::literals; +int get_cumu_thread_num() { + if (config::max_cumu_compaction_threads > 0) { + return config::max_cumu_compaction_threads; + } + + int num_cores = doris::CpuInfo::num_cores(); + return std::min(std::max(int(num_cores * config::cumu_compaction_thread_num_factor), 2), 20); +} + +int get_base_thread_num() { + if (config::max_base_compaction_threads > 0) { + return config::max_base_compaction_threads; + } + + int num_cores = doris::CpuInfo::num_cores(); + return std::min(std::max(int(num_cores * config::base_compaction_thread_num_factor), 1), 10); +} + CloudStorageEngine::CloudStorageEngine(const UniqueId& backend_uid) : BaseStorageEngine(Type::CLOUD, backend_uid), _meta_mgr(std::make_unique()), - _tablet_mgr(std::make_unique(*this)) {} + _tablet_mgr(std::make_unique(*this)), + _cumulative_compaction_policy( + std::make_shared()) {} CloudStorageEngine::~CloudStorageEngine() = default; @@ -70,6 +100,10 @@ Status CloudStorageEngine::open() { _calc_delete_bitmap_executor = std::make_unique(); _calc_delete_bitmap_executor->init(); + _txn_delete_bitmap_cache = + std::make_unique(config::delete_bitmap_agg_cache_capacity); + RETURN_IF_ERROR(_txn_delete_bitmap_cache->init()); + return Status::OK(); } @@ -121,10 +155,37 @@ Status CloudStorageEngine::start_bg_threads() { &_evict_quering_rowset_thread)); LOG(INFO) << "evict quering thread started"; - // TODO(plat1ko): lease_compaction_thread + // add calculate tablet delete bitmap task thread pool + RETURN_IF_ERROR(ThreadPoolBuilder("TabletCalDeleteBitmapThreadPool") + .set_min_threads(1) + .set_max_threads(config::calc_tablet_delete_bitmap_task_max_thread) + .build(&_calc_tablet_delete_bitmap_task_thread_pool)); // TODO(plat1ko): check_bucket_enable_versioning_thread + // compaction tasks producer thread + int base_thread_num = get_base_thread_num(); + int cumu_thread_num = get_cumu_thread_num(); + RETURN_IF_ERROR(ThreadPoolBuilder("BaseCompactionTaskThreadPool") + .set_min_threads(base_thread_num) + .set_max_threads(base_thread_num) + .build(&_base_compaction_thread_pool)); + RETURN_IF_ERROR(ThreadPoolBuilder("CumuCompactionTaskThreadPool") + .set_min_threads(cumu_thread_num) + .set_max_threads(cumu_thread_num) + .build(&_cumu_compaction_thread_pool)); + RETURN_IF_ERROR(Thread::create( + "StorageEngine", "compaction_tasks_producer_thread", + [this]() { this->_compaction_tasks_producer_callback(); }, + &_bg_threads.emplace_back())); + LOG(INFO) << "compaction tasks producer thread started," + << " base thread num " << base_thread_num << " cumu thread num " << cumu_thread_num; + + RETURN_IF_ERROR(Thread::create( + "StorageEngine", "lease_compaction_thread", + [this]() { this->_lease_compaction_thread_callback(); }, &_bg_threads.emplace_back())); + LOG(INFO) << "lease compaction thread started"; + return Status::OK(); } @@ -196,4 +257,454 @@ void CloudStorageEngine::_sync_tablets_thread_callback() { } } +void CloudStorageEngine::get_cumu_compaction( + int64_t tablet_id, std::vector>& res) { + std::lock_guard lock(_compaction_mtx); + if (auto it = _submitted_cumu_compactions.find(tablet_id); + it != _submitted_cumu_compactions.end()) { + res = it->second; + } +} + +void CloudStorageEngine::_adjust_compaction_thread_num() { + int base_thread_num = get_base_thread_num(); + if (_base_compaction_thread_pool->max_threads() != base_thread_num) { + int old_max_threads = _base_compaction_thread_pool->max_threads(); + Status status = _base_compaction_thread_pool->set_max_threads(base_thread_num); + if (status.ok()) { + VLOG_NOTICE << "update base compaction thread pool max_threads from " << old_max_threads + << " to " << base_thread_num; + } + } + if (_base_compaction_thread_pool->min_threads() != base_thread_num) { + int old_min_threads = _base_compaction_thread_pool->min_threads(); + Status status = _base_compaction_thread_pool->set_min_threads(base_thread_num); + if (status.ok()) { + VLOG_NOTICE << "update base compaction thread pool min_threads from " << old_min_threads + << " to " << base_thread_num; + } + } + + int cumu_thread_num = get_cumu_thread_num(); + if (_cumu_compaction_thread_pool->max_threads() != cumu_thread_num) { + int old_max_threads = _cumu_compaction_thread_pool->max_threads(); + Status status = _cumu_compaction_thread_pool->set_max_threads(cumu_thread_num); + if (status.ok()) { + VLOG_NOTICE << "update cumu compaction thread pool max_threads from " << old_max_threads + << " to " << cumu_thread_num; + } + } + if (_cumu_compaction_thread_pool->min_threads() != cumu_thread_num) { + int old_min_threads = _cumu_compaction_thread_pool->min_threads(); + Status status = _cumu_compaction_thread_pool->set_min_threads(cumu_thread_num); + if (status.ok()) { + VLOG_NOTICE << "update cumu compaction thread pool min_threads from " << old_min_threads + << " to " << cumu_thread_num; + } + } +} + +void CloudStorageEngine::_compaction_tasks_producer_callback() { + LOG(INFO) << "try to start compaction producer process!"; + + int round = 0; + CompactionType compaction_type; + + // Used to record the time when the score metric was last updated. + // The update of the score metric is accompanied by the logic of selecting the tablet. + // If there is no slot available, the logic of selecting the tablet will be terminated, + // which causes the score metric update to be terminated. + // In order to avoid this situation, we need to update the score regularly. + int64_t last_cumulative_score_update_time = 0; + int64_t last_base_score_update_time = 0; + static const int64_t check_score_interval_ms = 5000; // 5 secs + + int64_t interval = config::generate_compaction_tasks_interval_ms; + do { + if (!config::disable_auto_compaction) { + _adjust_compaction_thread_num(); + + bool check_score = false; + int64_t cur_time = UnixMillis(); + if (round < config::cumulative_compaction_rounds_for_each_base_compaction_round) { + compaction_type = CompactionType::CUMULATIVE_COMPACTION; + round++; + if (cur_time - last_cumulative_score_update_time >= check_score_interval_ms) { + check_score = true; + last_cumulative_score_update_time = cur_time; + } + } else { + compaction_type = CompactionType::BASE_COMPACTION; + round = 0; + if (cur_time - last_base_score_update_time >= check_score_interval_ms) { + check_score = true; + last_base_score_update_time = cur_time; + } + } + std::unique_ptr& thread_pool = + (compaction_type == CompactionType::CUMULATIVE_COMPACTION) + ? _cumu_compaction_thread_pool + : _base_compaction_thread_pool; + VLOG_CRITICAL << "compaction thread pool. type: " + << (compaction_type == CompactionType::CUMULATIVE_COMPACTION ? "CUMU" + : "BASE") + << ", num_threads: " << thread_pool->num_threads() + << ", num_threads_pending_start: " + << thread_pool->num_threads_pending_start() + << ", num_active_threads: " << thread_pool->num_active_threads() + << ", max_threads: " << thread_pool->max_threads() + << ", min_threads: " << thread_pool->min_threads() + << ", num_total_queued_tasks: " << thread_pool->get_queue_size(); + std::vector tablets_compaction = + _generate_cloud_compaction_tasks(compaction_type, check_score); + + /// Regardless of whether the tablet is submitted for compaction or not, + /// we need to call 'reset_compaction' to clean up the base_compaction or cumulative_compaction objects + /// in the tablet, because these two objects store the tablet's own shared_ptr. + /// If it is not cleaned up, the reference count of the tablet will always be greater than 1, + /// thus cannot be collected by the garbage collector. (TabletManager::start_trash_sweep) + for (const auto& tablet : tablets_compaction) { + Status st = submit_compaction_task(tablet, compaction_type); + if (st.ok()) continue; + if ((!st.is() && + !st.is()) || + VLOG_DEBUG_IS_ON) { + LOG(WARNING) << "failed to submit compaction task for tablet: " + << tablet->tablet_id() << ", err: " << st; + } + } + interval = config::generate_compaction_tasks_interval_ms; + } else { + interval = config::check_auto_compaction_interval_seconds * 1000; + } + } while (!_stop_background_threads_latch.wait_for(std::chrono::milliseconds(interval))); +} + +std::vector CloudStorageEngine::_generate_cloud_compaction_tasks( + CompactionType compaction_type, bool check_score) { + std::vector> tablets_compaction; + + int64_t max_compaction_score = 0; + std::unordered_set tablet_preparing_cumu_compaction; + std::unordered_map>> + submitted_cumu_compactions; + std::unordered_map> submitted_base_compactions; + std::unordered_map> submitted_full_compactions; + { + std::lock_guard lock(_compaction_mtx); + tablet_preparing_cumu_compaction = _tablet_preparing_cumu_compaction; + submitted_cumu_compactions = _submitted_cumu_compactions; + submitted_base_compactions = _submitted_base_compactions; + submitted_full_compactions = _submitted_full_compactions; + } + + bool need_pick_tablet = true; + int thread_per_disk = + config::compaction_task_num_per_fast_disk; // all disks are fast in cloud mode + int num_cumu = + std::accumulate(submitted_cumu_compactions.begin(), submitted_cumu_compactions.end(), 0, + [](int a, auto& b) { return a + b.second.size(); }); + int num_base = submitted_base_compactions.size() + submitted_full_compactions.size(); + int n = thread_per_disk - num_cumu - num_base; + if (compaction_type == CompactionType::BASE_COMPACTION) { + // We need to reserve at least one thread for cumulative compaction, + // because base compactions may take too long to complete, which may + // leads to "too many rowsets" error. + int base_n = std::min(config::max_base_compaction_task_num_per_disk, thread_per_disk - 1) - + num_base; + n = std::min(base_n, n); + } + if (n <= 0) { // No threads available + if (!check_score) return tablets_compaction; + need_pick_tablet = false; + n = 0; + } + + // Return true for skipping compaction + 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()) || + 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; + }; + } 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; + }; + } + + // Even if need_pick_tablet is false, we still need to call find_best_tablet_to_compaction(), + // So that we can update the max_compaction_score metric. + do { + std::vector tablets; + auto st = tablet_mgr().get_topn_tablets_to_compact(n, compaction_type, filter_out, &tablets, + &max_compaction_score); + if (!st.ok()) { + LOG(WARNING) << "failed to get tablets to compact, err=" << st; + break; + } + if (!need_pick_tablet) break; + tablets_compaction = std::move(tablets); + } while (false); + + if (max_compaction_score > 0) { + if (compaction_type == CompactionType::BASE_COMPACTION) { + DorisMetrics::instance()->tablet_base_max_compaction_score->set_value( + max_compaction_score); + } else { + DorisMetrics::instance()->tablet_cumulative_max_compaction_score->set_value( + max_compaction_score); + } + } + + return tablets_compaction; +} + +Status CloudStorageEngine::_submit_base_compaction_task(const CloudTabletSPtr& tablet) { + using namespace std::chrono; + { + std::lock_guard lock(_compaction_mtx); + // Take a placeholder for base compaction + auto [_, success] = _submitted_base_compactions.emplace(tablet->tablet_id(), nullptr); + if (!success) { + return Status::AlreadyExist( + "other base compaction or full compaction is submitted, tablet_id={}", + tablet->tablet_id()); + } + } + auto compaction = std::make_shared(*this, tablet); + auto st = compaction->prepare_compact(); + if (!st.ok()) { + long now = duration_cast(system_clock::now().time_since_epoch()).count(); + tablet->set_last_base_compaction_failure_time(now); + std::lock_guard lock(_compaction_mtx); + _submitted_base_compactions.erase(tablet->tablet_id()); + return st; + } + { + std::lock_guard lock(_compaction_mtx); + _submitted_base_compactions[tablet->tablet_id()] = compaction; + } + st = _base_compaction_thread_pool->submit_func([=, this, compaction = std::move(compaction)]() { + auto st = compaction->execute_compact(); + if (!st.ok()) { + // Error log has been output in `execute_compact` + long now = duration_cast(system_clock::now().time_since_epoch()).count(); + tablet->set_last_base_compaction_failure_time(now); + } + std::lock_guard lock(_compaction_mtx); + _submitted_base_compactions.erase(tablet->tablet_id()); + }); + if (!st.ok()) { + std::lock_guard lock(_compaction_mtx); + _submitted_base_compactions.erase(tablet->tablet_id()); + return Status::InternalError("failed to submit base compaction, tablet_id={}", + tablet->tablet_id()); + } + return st; +} + +Status CloudStorageEngine::_submit_cumulative_compaction_task(const CloudTabletSPtr& tablet) { + using namespace std::chrono; + { + std::lock_guard lock(_compaction_mtx); + if (!config::enable_parallel_cumu_compaction && + _submitted_cumu_compactions.count(tablet->tablet_id())) { + return Status::AlreadyExist("other cumu compaction is submitted, tablet_id={}", + tablet->tablet_id()); + } + auto [_, success] = _tablet_preparing_cumu_compaction.insert(tablet->tablet_id()); + if (!success) { + return Status::AlreadyExist("other cumu compaction is preparing, tablet_id={}", + tablet->tablet_id()); + } + } + auto compaction = std::make_shared(*this, tablet); + auto st = compaction->prepare_compact(); + if (!st.ok()) { + long now = duration_cast(system_clock::now().time_since_epoch()).count(); + if (st.is()) { + // Backoff strategy if no suitable version + tablet->last_cumu_no_suitable_version_ms = now; + } + tablet->set_last_cumu_compaction_failure_time(now); + std::lock_guard lock(_compaction_mtx); + _tablet_preparing_cumu_compaction.erase(tablet->tablet_id()); + return st; + } + { + std::lock_guard lock(_compaction_mtx); + _tablet_preparing_cumu_compaction.erase(tablet->tablet_id()); + _submitted_cumu_compactions[tablet->tablet_id()].push_back(compaction); + } + auto erase_submitted_cumu_compaction = [=, this]() { + std::lock_guard lock(_compaction_mtx); + auto it = _submitted_cumu_compactions.find(tablet->tablet_id()); + DCHECK(it != _submitted_cumu_compactions.end()); + auto& compactions = it->second; + auto it1 = std::find(compactions.begin(), compactions.end(), compaction); + DCHECK(it1 != compactions.end()); + compactions.erase(it1); + if (compactions.empty()) { // No compactions on this tablet, erase key + _submitted_cumu_compactions.erase(it); + // No cumu compaction on this tablet, reset `last_cumu_no_suitable_version_ms` to enable this tablet to + // enter the compaction scheduling candidate set. The purpose of doing this is to have at least one BE perform + // cumu compaction on tablet which has suitable versions for cumu compaction. + tablet->last_cumu_no_suitable_version_ms = 0; + } + }; + st = _cumu_compaction_thread_pool->submit_func([=, compaction = std::move(compaction)]() { + auto st = compaction->execute_compact(); + if (!st.ok()) { + // Error log has been output in `execute_compact` + long now = duration_cast(system_clock::now().time_since_epoch()).count(); + tablet->set_last_cumu_compaction_failure_time(now); + } + erase_submitted_cumu_compaction(); + }); + if (!st.ok()) { + erase_submitted_cumu_compaction(); + return Status::InternalError("failed to submit cumu compaction, tablet_id={}", + tablet->tablet_id()); + } + return st; +} + +Status CloudStorageEngine::_submit_full_compaction_task(const CloudTabletSPtr& tablet) { + using namespace std::chrono; + { + std::lock_guard lock(_compaction_mtx); + // Take a placeholder for full compaction + auto [_, success] = _submitted_full_compactions.emplace(tablet->tablet_id(), nullptr); + if (!success) { + return Status::AlreadyExist( + "other full compaction or base compaction is submitted, tablet_id={}", + tablet->tablet_id()); + } + } + //auto compaction = std::make_shared(tablet); + auto compaction = std::make_shared(*this, tablet); + auto st = compaction->prepare_compact(); + if (!st.ok()) { + long now = duration_cast(system_clock::now().time_since_epoch()).count(); + tablet->set_last_full_compaction_failure_time(now); + std::lock_guard lock(_compaction_mtx); + _submitted_full_compactions.erase(tablet->tablet_id()); + return st; + } + { + std::lock_guard lock(_compaction_mtx); + _submitted_full_compactions[tablet->tablet_id()] = compaction; + } + st = _base_compaction_thread_pool->submit_func([=, this, compaction = std::move(compaction)]() { + auto st = compaction->execute_compact(); + if (!st.ok()) { + // Error log has been output in `execute_compact` + long now = duration_cast(system_clock::now().time_since_epoch()).count(); + tablet->set_last_full_compaction_failure_time(now); + } + std::lock_guard lock(_compaction_mtx); + _submitted_full_compactions.erase(tablet->tablet_id()); + }); + if (!st.ok()) { + std::lock_guard lock(_compaction_mtx); + _submitted_full_compactions.erase(tablet->tablet_id()); + return Status::InternalError("failed to submit full compaction, tablet_id={}", + tablet->tablet_id()); + } + return st; +} + +Status CloudStorageEngine::submit_compaction_task(const CloudTabletSPtr& tablet, + CompactionType compaction_type) { + DCHECK(compaction_type == CompactionType::CUMULATIVE_COMPACTION || + compaction_type == CompactionType::BASE_COMPACTION || + compaction_type == CompactionType::FULL_COMPACTION); + switch (compaction_type) { + case CompactionType::BASE_COMPACTION: + RETURN_IF_ERROR(_submit_base_compaction_task(tablet)); + return Status::OK(); + case CompactionType::CUMULATIVE_COMPACTION: + RETURN_IF_ERROR(_submit_cumulative_compaction_task(tablet)); + return Status::OK(); + case CompactionType::FULL_COMPACTION: + RETURN_IF_ERROR(_submit_full_compaction_task(tablet)); + return Status::OK(); + default: + return Status::InternalError("unknown compaction type!"); + } +} + +void CloudStorageEngine::_lease_compaction_thread_callback() { + while (!_stop_background_threads_latch.wait_for( + std::chrono::seconds(config::lease_compaction_interval_seconds))) { + std::vector> base_compactions; + std::vector> cumu_compactions; + { + std::lock_guard lock(_compaction_mtx); + for (auto& [_, base] : _submitted_base_compactions) { + if (base) { // `base` might be a nullptr placeholder + base_compactions.push_back(base); + } + } + for (auto& [_, cumus] : _submitted_cumu_compactions) { + for (auto& cumu : cumus) { + cumu_compactions.push_back(cumu); + } + } + } + // TODO(plat1ko): Support batch lease rpc + for (auto& comp : cumu_compactions) { + comp->do_lease(); + } + for (auto& comp : base_compactions) { + comp->do_lease(); + } + } +} + +Status CloudStorageEngine::get_compaction_status_json(std::string* result) { + rapidjson::Document root; + root.SetObject(); + + std::lock_guard lock(_compaction_mtx); + // cumu + std::string_view cumu = "CumulativeCompaction"; + rapidjson::Value cumu_key; + cumu_key.SetString(cumu.data(), cumu.length(), root.GetAllocator()); + rapidjson::Document cumu_arr; + cumu_arr.SetArray(); + for (auto& [tablet_id, v] : _submitted_cumu_compactions) { + for (int i = 0; i < v.size(); ++i) { + cumu_arr.PushBack(tablet_id, root.GetAllocator()); + } + } + root.AddMember(cumu_key, cumu_arr, root.GetAllocator()); + // base + std::string_view base = "BaseCompaction"; + rapidjson::Value base_key; + base_key.SetString(base.data(), base.length(), root.GetAllocator()); + rapidjson::Document base_arr; + base_arr.SetArray(); + for (auto& [tablet_id, _] : _submitted_base_compactions) { + base_arr.PushBack(tablet_id, root.GetAllocator()); + } + root.AddMember(base_key, base_arr, root.GetAllocator()); + + rapidjson::StringBuffer strbuf; + rapidjson::PrettyWriter writer(strbuf); + root.Accept(writer); + *result = std::string(strbuf.GetString()); + return Status::OK(); +} + } // namespace doris diff --git a/be/src/cloud/cloud_storage_engine.h b/be/src/cloud/cloud_storage_engine.h index 16b1cc5a1e04a8..61e4ca9859befa 100644 --- a/be/src/cloud/cloud_storage_engine.h +++ b/be/src/cloud/cloud_storage_engine.h @@ -19,7 +19,14 @@ #include +//#include "cloud/cloud_cumulative_compaction.h" +//#include "cloud/cloud_base_compaction.h" +//#include "cloud/cloud_full_compaction.h" +#include "cloud/cloud_cumulative_compaction_policy.h" +#include "cloud/cloud_tablet.h" +#include "cloud_txn_delete_bitmap_cache.h" #include "olap/storage_engine.h" +#include "util/threadpool.h" namespace doris { namespace cloud { @@ -27,6 +34,9 @@ class CloudMetaMgr; } class CloudTabletMgr; +class CloudCumulativeCompaction; +class CloudBaseCompaction; +class CloudFullCompaction; class CloudStorageEngine final : public BaseStorageEngine { public: @@ -51,6 +61,11 @@ class CloudStorageEngine final : public BaseStorageEngine { CloudTabletMgr& tablet_mgr() { return *_tablet_mgr; } + CloudTxnDeleteBitmapCache& txn_delete_bitmap_cache() { return *_txn_delete_bitmap_cache; } + std::unique_ptr& calc_tablet_delete_bitmap_task_thread_pool() { + return _calc_tablet_delete_bitmap_task_thread_pool; + } + io::FileSystemSPtr latest_fs() const { std::lock_guard lock(_latest_fs_mtx); return _latest_fs; @@ -61,21 +76,59 @@ class CloudStorageEngine final : public BaseStorageEngine { _latest_fs = fs; } + void get_cumu_compaction(int64_t tablet_id, + std::vector>& res); + + CloudSizeBasedCumulativeCompactionPolicy* cumu_compaction_policy() const { + return _cumulative_compaction_policy.get(); + } + + Status submit_compaction_task(const CloudTabletSPtr& tablet, CompactionType compaction_type); + + Status get_compaction_status_json(std::string* result); + private: void _refresh_s3_info_thread_callback(); void _vacuum_stale_rowsets_thread_callback(); void _sync_tablets_thread_callback(); + void _compaction_tasks_producer_callback(); + std::vector _generate_cloud_compaction_tasks(CompactionType compaction_type, + bool check_score); + void _adjust_compaction_thread_num(); + Status _submit_base_compaction_task(const CloudTabletSPtr& tablet); + Status _submit_cumulative_compaction_task(const CloudTabletSPtr& tablet); + Status _submit_full_compaction_task(const CloudTabletSPtr& tablet); + void _lease_compaction_thread_callback(); std::atomic_bool _stopped {false}; std::unique_ptr _meta_mgr; std::unique_ptr _tablet_mgr; + std::unique_ptr _txn_delete_bitmap_cache; + std::unique_ptr _calc_tablet_delete_bitmap_task_thread_pool; // FileSystem with latest shared storage info, new data will be written to this fs. mutable std::mutex _latest_fs_mtx; io::FileSystemSPtr _latest_fs; std::vector> _bg_threads; + + // ATTN: Compactions in maps depend on `CloudTabletMgr` and `CloudMetaMgr` + mutable std::mutex _compaction_mtx; + // tablet_id -> submitted base compaction, guarded by `_compaction_mtx` + std::unordered_map> _submitted_base_compactions; + // tablet_id -> submitted full compaction, guarded by `_compaction_mtx` + std::unordered_map> _submitted_full_compactions; + // Store tablets which are preparing cumu compaction, guarded by `_compaction_mtx` + std::unordered_set _tablet_preparing_cumu_compaction; + // tablet_id -> submitted cumu compactions, guarded by `_compaction_mtx` + std::unordered_map>> + _submitted_cumu_compactions; + + std::unique_ptr _base_compaction_thread_pool; + std::unique_ptr _cumu_compaction_thread_pool; + + std::shared_ptr _cumulative_compaction_policy; }; } // namespace doris diff --git a/be/src/cloud/cloud_tablet.cpp b/be/src/cloud/cloud_tablet.cpp index 6c0b50807c6737..8e3932f7d24914 100644 --- a/be/src/cloud/cloud_tablet.cpp +++ b/be/src/cloud/cloud_tablet.cpp @@ -24,20 +24,25 @@ #include #include +#include #include "cloud/cloud_meta_mgr.h" #include "cloud/cloud_storage_engine.h" #include "io/cache/block/block_file_cache_factory.h" #include "olap/olap_define.h" +#include "olap/rowset/beta_rowset.h" #include "olap/rowset/rowset.h" #include "olap/rowset/rowset_factory.h" #include "olap/rowset/rowset_fwd.h" #include "olap/rowset/rowset_writer.h" #include "olap/rowset/segment_v2/inverted_index_desc.h" +#include "olap/txn_manager.h" namespace doris { using namespace ErrorCode; +static constexpr int COMPACTION_DELETE_BITMAP_LOCK_ID = -1; + CloudTablet::CloudTablet(CloudStorageEngine& engine, TabletMetaSharedPtr tablet_meta) : BaseTablet(std::move(tablet_meta)), _engine(engine) { _tablet_path = remote_tablet_path(_tablet_meta->tablet_id()); @@ -351,6 +356,36 @@ Result> CloudTablet::create_rowset_writer( return RowsetFactory::create_rowset_writer(_engine, context, vertical); } +// create a rowset writer with rowset_id and seg_id +// after writer, merge this transient rowset with original rowset +Result> CloudTablet::create_transient_rowset_writer( + const Rowset& rowset, std::shared_ptr partial_update_info, + int64_t txn_expiration) { + RowsetWriterContext context; + context.rowset_state = PREPARED; + context.segments_overlap = OVERLAPPING; + context.tablet_schema = std::make_shared(); + context.tablet_schema->copy_from(*(rowset.tablet_schema())); + context.newest_write_timestamp = UnixSeconds(); + context.tablet_id = table_id(); + context.enable_segcompaction = false; + context.write_type = DataWriteType::TYPE_DIRECT; + context.partial_update_info = std::move(partial_update_info); + context.is_transient_rowset_writer = true; + context.rowset_id = rowset.rowset_id(); + context.tablet_id = tablet_id(); + context.index_id = index_id(); + context.partition_id = partition_id(); + context.rowset_dir = remote_tablet_path(tablet_id()); + context.enable_unique_key_merge_on_write = enable_unique_key_merge_on_write(); + context.txn_expiration = txn_expiration; + return RowsetFactory::create_rowset_writer(_engine, context, false) + .transform([&](auto&& writer) { + writer->set_segment_start_id(rowset.num_segments()); + return writer; + }); +} + int64_t CloudTablet::get_cloud_base_compaction_score() const { return _approximate_num_rowsets.load(std::memory_order_relaxed) - _approximate_cumu_num_rowsets.load(std::memory_order_relaxed); @@ -443,4 +478,69 @@ void CloudTablet::set_cumulative_layer_point(int64_t new_point) { _cumulative_point = new_point; } +std::vector CloudTablet::pick_candidate_rowsets_to_base_compaction() { + std::vector candidate_rowsets; + { + std::shared_lock rlock(_meta_lock); + for (const auto& [version, rs] : _rs_version_map) { + // Do compaction on local rowsets only. + if (version.first < _cumulative_point && rs->is_local()) { + candidate_rowsets.push_back(rs); + } + } + } + std::sort(candidate_rowsets.begin(), candidate_rowsets.end(), Rowset::comparator); + return candidate_rowsets; +} + +std::vector CloudTablet::pick_candidate_rowsets_to_single_replica_compaction() { + std::vector candidate_rowsets; + { + std::shared_lock rlock(_meta_lock); + for (const auto& [version, rs] : _rs_version_map) { + if (rs->is_local()) { + candidate_rowsets.push_back(rs); + } + } + } + std::sort(candidate_rowsets.begin(), candidate_rowsets.end(), Rowset::comparator); + return candidate_rowsets; +} + +std::vector CloudTablet::pick_candidate_rowsets_to_full_compaction() { + return pick_candidate_rowsets_to_single_replica_compaction(); +} + +CalcDeleteBitmapExecutor* CloudTablet::calc_delete_bitmap_executor() { + return _engine.calc_delete_bitmap_executor(); +} + +Status CloudTablet::save_delete_bitmap(const TabletTxnInfo* txn_info, int64_t txn_id, + DeleteBitmapPtr delete_bitmap, RowsetWriter* rowset_writer, + const RowsetIdUnorderedSet& cur_rowset_ids) { + RowsetSharedPtr rowset = txn_info->rowset; + int64_t cur_version = rowset->start_version(); + // update delete bitmap info, in order to avoid recalculation when trying again + _engine.txn_delete_bitmap_cache().update_tablet_txn_info(txn_id, tablet_id(), delete_bitmap, + cur_rowset_ids); + + if (txn_info->partial_update_info && txn_info->partial_update_info->is_partial_update && + rowset_writer->num_rows() > 0) { + const auto& rowset_meta = rowset->rowset_meta(); + RETURN_IF_ERROR(_engine.meta_mgr().update_tmp_rowset(*rowset_meta)); + } + + DeleteBitmapPtr new_delete_bitmap = std::make_shared(tablet_id()); + for (auto iter = delete_bitmap->delete_bitmap.begin(); + iter != delete_bitmap->delete_bitmap.end(); ++iter) { + new_delete_bitmap->merge({std::get<0>(iter->first), std::get<1>(iter->first), cur_version}, + iter->second); + } + + RETURN_IF_ERROR(_engine.meta_mgr().update_delete_bitmap( + *this, txn_id, COMPACTION_DELETE_BITMAP_LOCK_ID, new_delete_bitmap.get())); + + return Status::OK(); +} + } // namespace doris diff --git a/be/src/cloud/cloud_tablet.h b/be/src/cloud/cloud_tablet.h index 10e1390496140a..37ba38c6db33e8 100644 --- a/be/src/cloud/cloud_tablet.h +++ b/be/src/cloud/cloud_tablet.h @@ -17,7 +17,10 @@ #pragma once +#include + #include "olap/base_tablet.h" +#include "olap/partial_update_info.h" namespace doris { @@ -109,6 +112,79 @@ class CloudTablet final : public BaseTablet { void set_cumulative_compaction_cnt(int64_t cnt) { _cumulative_compaction_cnt = cnt; } void set_cumulative_layer_point(int64_t new_point); + int64_t last_cumu_compaction_failure_time() { return _last_cumu_compaction_failure_millis; } + void set_last_cumu_compaction_failure_time(int64_t millis) { + _last_cumu_compaction_failure_millis = millis; + } + + int64_t last_base_compaction_failure_time() { return _last_base_compaction_failure_millis; } + void set_last_base_compaction_failure_time(int64_t millis) { + _last_base_compaction_failure_millis = millis; + } + + int64_t last_full_compaction_failure_time() { return _last_full_compaction_failure_millis; } + void set_last_full_compaction_failure_time(int64_t millis) { + _last_full_compaction_failure_millis = millis; + } + + int64_t last_cumu_compaction_success_time() { return _last_cumu_compaction_success_millis; } + void set_last_cumu_compaction_success_time(int64_t millis) { + _last_cumu_compaction_success_millis = millis; + } + + int64_t last_base_compaction_success_time() { return _last_base_compaction_success_millis; } + void set_last_base_compaction_success_time(int64_t millis) { + _last_base_compaction_success_millis = millis; + } + + int64_t last_full_compaction_success_time() { return _last_full_compaction_success_millis; } + void set_last_full_compaction_success_time(int64_t millis) { + _last_full_compaction_success_millis = millis; + } + + int64_t last_base_compaction_schedule_time() { return _last_base_compaction_schedule_millis; } + void set_last_base_compaction_schedule_time(int64_t millis) { + _last_base_compaction_schedule_millis = millis; + } + + std::vector pick_candidate_rowsets_to_base_compaction(); + + void traverse_rowsets(std::function visitor, + bool include_stale = false) { + std::shared_lock rlock(_meta_lock); + for (auto& [v, rs] : _rs_version_map) { + visitor(rs); + } + if (!include_stale) return; + for (auto& [v, rs] : _stale_rs_version_map) { + visitor(rs); + } + } + + inline Version max_version() const { + std::shared_lock rdlock(_meta_lock); + return _tablet_meta->max_version(); + } + + int64_t base_size() const { return _base_size; } + + std::vector pick_candidate_rowsets_to_single_replica_compaction(); + + std::vector pick_candidate_rowsets_to_full_compaction(); + + std::mutex& get_base_compaction_lock() { return _base_compaction_lock; } + std::mutex& get_cumulative_compaction_lock() { return _cumulative_compaction_lock; } + + Result> create_transient_rowset_writer( + const Rowset& rowset, std::shared_ptr partial_update_info, + int64_t txn_expiration = 0) override; + + CalcDeleteBitmapExecutor* calc_delete_bitmap_executor() override; + + Status save_delete_bitmap(const TabletTxnInfo* txn_info, int64_t txn_id, + DeleteBitmapPtr delete_bitmap, RowsetWriter* rowset_writer, + const RowsetIdUnorderedSet& cur_rowset_ids) override; + int64_t last_sync_time_s = 0; int64_t last_load_time_ms = 0; int64_t last_base_compaction_success_time_ms = 0; @@ -135,10 +211,30 @@ class CloudTablet final : public BaseTablet { // Number of sorted arrays (e.g. for rowset with N segments, if rowset is overlapping, delta is N, otherwise 1) after cumu point std::atomic _approximate_cumu_num_deltas {-1}; + // timestamp of last cumu compaction failure + std::atomic _last_cumu_compaction_failure_millis; + // timestamp of last base compaction failure + std::atomic _last_base_compaction_failure_millis; + // timestamp of last full compaction failure + std::atomic _last_full_compaction_failure_millis; + // timestamp of last cumu compaction success + std::atomic _last_cumu_compaction_success_millis; + // timestamp of last base compaction success + std::atomic _last_base_compaction_success_millis; + // timestamp of last full compaction success + std::atomic _last_full_compaction_success_millis; + // timestamp of last base compaction schedule time + std::atomic _last_base_compaction_schedule_millis; + int64_t _base_compaction_cnt = 0; int64_t _cumulative_compaction_cnt = 0; int64_t _max_version = -1; int64_t _base_size = 0; + + std::mutex _base_compaction_lock; + std::mutex _cumulative_compaction_lock; }; +using CloudTabletSPtr = std::shared_ptr; + } // namespace doris diff --git a/be/src/cloud/cloud_tablets_channel.cpp b/be/src/cloud/cloud_tablets_channel.cpp index 07c247ac8097c4..046916aa9a078b 100644 --- a/be/src/cloud/cloud_tablets_channel.cpp +++ b/be/src/cloud/cloud_tablets_channel.cpp @@ -235,7 +235,16 @@ Status CloudTabletsChannel::close(LoadChannel* parent, const PTabletWriterAddBlo } } - // TODO(plat1ko): 6. set txn related delete bitmap if necessary + // 6. set txn related delete bitmap if necessary + for (auto it = writers_to_commit.begin(); it != writers_to_commit.end();) { + auto st = (*it)->set_txn_related_delete_bitmap(); + if (!st.ok()) { + _add_error_tablet(tablet_errors, (*it)->tablet_id(), st); + _close_status = std::move(st); + return _close_status; + } + it++; + } tablet_vec->Reserve(writers_to_commit.size()); for (auto* writer : writers_to_commit) { diff --git a/be/src/cloud/cloud_txn_delete_bitmap_cache.cpp b/be/src/cloud/cloud_txn_delete_bitmap_cache.cpp new file mode 100644 index 00000000000000..08bc035770a398 --- /dev/null +++ b/be/src/cloud/cloud_txn_delete_bitmap_cache.cpp @@ -0,0 +1,190 @@ +// 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. + +#include "cloud/cloud_txn_delete_bitmap_cache.h" + +#include + +#include +#include +#include + +#include "common/status.h" +#include "common/sync_point.h" +#include "olap/olap_common.h" +#include "olap/tablet_meta.h" + +namespace doris { + +CloudTxnDeleteBitmapCache::CloudTxnDeleteBitmapCache(size_t size_in_bytes) + : LRUCachePolicy(CachePolicy::CacheType::CLOUD_TXN_DELETE_BITMAP_CACHE, size_in_bytes, + LRUCacheType::SIZE, 86400, 4), + _stop_latch(1) {} + +CloudTxnDeleteBitmapCache::~CloudTxnDeleteBitmapCache() { + _stop_latch.count_down(); + _clean_thread->join(); +} + +Status CloudTxnDeleteBitmapCache::init() { + auto st = Thread::create( + "CloudTxnDeleteBitmapCache", "clean_txn_dbm_thread", + [this]() { this->_clean_thread_callback(); }, &_clean_thread); + if (!st.ok()) { + LOG(WARNING) << "failed to create thread for CloudTxnDeleteBitmapCache, error: " << st; + } + return st; +} + +Status CloudTxnDeleteBitmapCache::get_tablet_txn_info( + TTransactionId transaction_id, int64_t tablet_id, RowsetSharedPtr* rowset, + DeleteBitmapPtr* delete_bitmap, RowsetIdUnorderedSet* rowset_ids, int64_t* txn_expiration, + std::shared_ptr* partial_update_info) { + { + std::shared_lock rlock(_rwlock); + TxnKey key(transaction_id, tablet_id); + auto iter = _txn_map.find(key); + if (iter == _txn_map.end()) { + return Status::Error( + "not found txn info, tablet_id={}, transaction_id={}", tablet_id, + transaction_id); + } + *rowset = iter->second.rowset; + *txn_expiration = iter->second.txn_expiration; + *partial_update_info = iter->second.partial_update_info; + } + std::string key_str = fmt::format("{}/{}", transaction_id, tablet_id); + CacheKey key(key_str); + Cache::Handle* handle = cache()->lookup(key); + + DeleteBitmapCacheValue* val = + handle == nullptr ? nullptr + : reinterpret_cast(cache()->value(handle)); + if (val) { + *delete_bitmap = val->delete_bitmap; + *rowset_ids = val->rowset_ids; + // must call release handle to reduce the reference count, + // otherwise there will be memory leak + cache()->release(handle); + } else { + LOG_INFO("cache missed when get delete bitmap") + .tag("txn_id", transaction_id) + .tag("tablet_id", tablet_id); + // Becasue of the rowset_ids become empty, all delete bitmap + // will be recalculate in CalcDeleteBitmapTask + *delete_bitmap = std::make_shared(tablet_id); + } + return Status::OK(); +} + +void CloudTxnDeleteBitmapCache::set_tablet_txn_info( + TTransactionId transaction_id, int64_t tablet_id, DeleteBitmapPtr delete_bitmap, + const RowsetIdUnorderedSet& rowset_ids, RowsetSharedPtr rowset, int64_t txn_expiration, + std::shared_ptr partial_update_info) { + if (txn_expiration <= 0) { + txn_expiration = duration_cast( + std::chrono::system_clock::now().time_since_epoch()) + .count() + + 120; + } + { + std::unique_lock wlock(_rwlock); + TxnKey txn_key(transaction_id, tablet_id); + _txn_map[txn_key] = TxnVal(rowset, txn_expiration, std::move(partial_update_info)); + _expiration_txn.emplace(txn_expiration, txn_key); + } + std::string key_str = fmt::format("{}/{}", transaction_id, tablet_id); + CacheKey key(key_str); + + auto val = new DeleteBitmapCacheValue(delete_bitmap, rowset_ids); + auto deleter = [](const CacheKey&, void* value) { + delete (DeleteBitmapCacheValue*)value; // Just delete to reclaim + }; + size_t charge = sizeof(DeleteBitmapCacheValue); + for (auto& [k, v] : val->delete_bitmap->delete_bitmap) { + charge += v.getSizeInBytes(); + } + auto handle = cache()->insert(key, val, charge, deleter, CachePriority::NORMAL); + // must call release handle to reduce the reference count, + // otherwise there will be memory leak + cache()->release(handle); + LOG_INFO("set txn related delete bitmap") + .tag("txn_id", transaction_id) + .tag("expiration", txn_expiration) + .tag("tablet_id", tablet_id) + .tag("delete_bitmap_size", charge); +} + +void CloudTxnDeleteBitmapCache::update_tablet_txn_info(TTransactionId transaction_id, + int64_t tablet_id, + DeleteBitmapPtr delete_bitmap, + const RowsetIdUnorderedSet& rowset_ids) { + std::string key_str = fmt::format("{}/{}", transaction_id, tablet_id); + CacheKey key(key_str); + + auto val = new DeleteBitmapCacheValue(delete_bitmap, rowset_ids); + auto deleter = [](const CacheKey&, void* value) { + delete (DeleteBitmapCacheValue*)value; // Just delete to reclaim + }; + size_t charge = sizeof(DeleteBitmapCacheValue); + for (auto& [k, v] : val->delete_bitmap->delete_bitmap) { + charge += v.getSizeInBytes(); + } + auto handle = cache()->insert(key, val, charge, deleter, CachePriority::NORMAL); + // must call release handle to reduce the reference count, + // otherwise there will be memory leak + cache()->release(handle); + LOG_INFO("update txn related delete bitmap") + .tag("txn_id", transaction_id) + .tag("tablt_id", tablet_id) + .tag("delete_bitmap_size", charge); +} + +void CloudTxnDeleteBitmapCache::remove_expired_tablet_txn_info() { + TEST_SYNC_POINT_RETURN_WITH_VOID("CloudTxnDeleteBitmapCache::remove_expired_tablet_txn_info"); + std::unique_lock wlock(_rwlock); + while (!_expiration_txn.empty()) { + auto iter = _expiration_txn.begin(); + int64_t current_time = duration_cast( + std::chrono::system_clock::now().time_since_epoch()) + .count(); + if (iter->first > current_time) { + break; + } + auto txn_iter = _txn_map.find(iter->second); + if (iter->first == txn_iter->second.txn_expiration) { + LOG_INFO("clean expired delete bitmap") + .tag("txn_id", txn_iter->first.txn_id) + .tag("expiration", txn_iter->second.txn_expiration) + .tag("tablt_id", txn_iter->first.tablet_id); + std::string key_str = std::to_string(txn_iter->first.txn_id) + "/" + + std::to_string(txn_iter->first.tablet_id); // Cache key container + CacheKey cache_key(key_str); + cache()->erase(cache_key); + _txn_map.erase(iter->second); + } + _expiration_txn.erase(iter); + } +} + +void CloudTxnDeleteBitmapCache::_clean_thread_callback() { + do { + remove_expired_tablet_txn_info(); + } while (!_stop_latch.wait_for(std::chrono::seconds(300))); +} + +} // namespace doris \ No newline at end of file diff --git a/be/src/cloud/cloud_txn_delete_bitmap_cache.h b/be/src/cloud/cloud_txn_delete_bitmap_cache.h new file mode 100644 index 00000000000000..c84e19a765f2d9 --- /dev/null +++ b/be/src/cloud/cloud_txn_delete_bitmap_cache.h @@ -0,0 +1,95 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include + +#include "olap/lru_cache.h" +#include "olap/olap_common.h" +#include "olap/partial_update_info.h" +#include "olap/rowset/rowset.h" +#include "olap/tablet_meta.h" +#include "util/countdown_latch.h" + +namespace doris { + +// Record transaction related delete bitmaps using a lru cache. +class CloudTxnDeleteBitmapCache : public LRUCachePolicy { +public: + CloudTxnDeleteBitmapCache(size_t size_in_bytes); + + ~CloudTxnDeleteBitmapCache() override; + + Status init(); + + Status get_tablet_txn_info(TTransactionId transaction_id, int64_t tablet_id, + RowsetSharedPtr* rowset, DeleteBitmapPtr* delete_bitmap, + RowsetIdUnorderedSet* rowset_ids, int64_t* txn_expiration, + std::shared_ptr* partial_update_info); + + void set_tablet_txn_info(TTransactionId transaction_id, int64_t tablet_id, + DeleteBitmapPtr delete_bitmap, const RowsetIdUnorderedSet& rowset_ids, + RowsetSharedPtr rowset, int64_t txn_expirationm, + std::shared_ptr partial_update_info); + + void update_tablet_txn_info(TTransactionId transaction_id, int64_t tablet_id, + DeleteBitmapPtr delete_bitmap, + const RowsetIdUnorderedSet& rowset_ids); + + void remove_expired_tablet_txn_info(); + +private: + void _clean_thread_callback(); + + struct DeleteBitmapCacheValue { + DeleteBitmapPtr delete_bitmap; + // records rowsets calc in commit txn + RowsetIdUnorderedSet rowset_ids; + + DeleteBitmapCacheValue(DeleteBitmapPtr delete_bitmap_, const RowsetIdUnorderedSet& ids_) + : delete_bitmap(std::move(delete_bitmap_)), rowset_ids(ids_) {} + }; + + struct TxnKey { + TTransactionId txn_id; + int64_t tablet_id; + TxnKey(TTransactionId txn_id_, int64_t tablet_id_) + : txn_id(txn_id_), tablet_id(tablet_id_) {} + auto operator<=>(const TxnKey&) const = default; + }; + + struct TxnVal { + RowsetSharedPtr rowset; + int64_t txn_expiration; + std::shared_ptr partial_update_info; + TxnVal() : txn_expiration(0) {}; + TxnVal(RowsetSharedPtr rowset_, int64_t txn_expiration_, + std::shared_ptr partial_update_info_) + : rowset(std::move(rowset_)), + txn_expiration(txn_expiration_), + partial_update_info(std::move(partial_update_info_)) {} + }; + + std::map _txn_map; + std::multimap _expiration_txn; + std::shared_mutex _rwlock; + scoped_refptr _clean_thread; + CountDownLatch _stop_latch; +}; + +} // namespace doris diff --git a/be/src/cloud/cloud_vertical_rowset_writer.cpp b/be/src/cloud/cloud_vertical_rowset_writer.cpp new file mode 100644 index 00000000000000..861670abd7537d --- /dev/null +++ b/be/src/cloud/cloud_vertical_rowset_writer.cpp @@ -0,0 +1,71 @@ +// 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. + +#include "cloud/cloud_vertical_rowset_writer.h" + +#include +#include + +#include +#include +#include +#include +#include +#include +#include + +#include "common/compiler_util.h" // IWYU pragma: keep +#include "common/logging.h" +#include "io/fs/file_system.h" +#include "io/fs/file_writer.h" +#include "olap/rowset/beta_rowset.h" +#include "olap/rowset/rowset_meta.h" +#include "olap/rowset/rowset_writer_context.h" +#include "olap/rowset/vertical_beta_rowset_writer_helper.h" +#include "util/slice.h" +#include "util/spinlock.h" +#include "vec/core/block.h" + +namespace doris { +using namespace ErrorCode; + +CloudVerticalRowsetWriter::CloudVerticalRowsetWriter() : CloudRowsetWriter() { + _helper = std::make_shared( + &_segment_writers, _already_built, _rowset_meta, &_num_segment, _context, + &_num_rows_written, &_segments_encoded_key_bounds, &_segment_num_rows, + &_total_index_size, &_file_writers, &_total_data_size, &_lock); +} + +CloudVerticalRowsetWriter::~CloudVerticalRowsetWriter() { + _helper->destruct_writer(); +} + +Status CloudVerticalRowsetWriter::add_columns(const vectorized::Block* block, + const std::vector& col_ids, bool is_key, + uint32_t max_rows_per_segment) { + return _helper->add_columns(block, col_ids, is_key, max_rows_per_segment); +} + +Status CloudVerticalRowsetWriter::flush_columns(bool is_key) { + return _helper->flush_columns(is_key); +} + +Status CloudVerticalRowsetWriter::final_flush() { + return _helper->final_flush(); +} + +} // namespace doris diff --git a/be/src/cloud/cloud_vertical_rowset_writer.h b/be/src/cloud/cloud_vertical_rowset_writer.h new file mode 100644 index 00000000000000..f6804c011f6ea8 --- /dev/null +++ b/be/src/cloud/cloud_vertical_rowset_writer.h @@ -0,0 +1,60 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include +#include + +#include +#include + +#include "cloud/cloud_rowset_writer.h" +#include "cloud/cloud_storage_engine.h" +#include "common/status.h" +#include "olap/rowset/segment_v2/segment_writer.h" +#include "olap/rowset/vertical_beta_rowset_writer.h" +#include "olap/rowset/vertical_beta_rowset_writer_helper.h" + +namespace doris { +namespace vectorized { +class Block; +} // namespace vectorized + +class CloudVerticalRowsetWriter final : public CloudRowsetWriter { +public: + CloudVerticalRowsetWriter(); + ~CloudVerticalRowsetWriter() override; + + Status add_columns(const vectorized::Block* block, const std::vector& col_ids, + bool is_key, uint32_t max_rows_per_segment) override; + + // flush last segment's column + Status flush_columns(bool is_key) override; + + // flush when all column finished, flush column footer + Status final_flush() override; + + int64_t num_rows() const override { return _total_key_group_rows; } + +private: + std::vector> _segment_writers; + size_t _total_key_group_rows = 0; + std::shared_ptr _helper; +}; + +} // namespace doris \ No newline at end of file diff --git a/be/src/cloud/config.cpp b/be/src/cloud/config.cpp index 932425e266ba04..6dac337ea65db8 100644 --- a/be/src/cloud/config.cpp +++ b/be/src/cloud/config.cpp @@ -39,6 +39,16 @@ DEFINE_mInt64(base_compaction_freeze_interval_s, "86400"); DEFINE_mInt64(cu_compaction_freeze_interval_s, "1200"); DEFINE_mInt64(cumu_compaction_interval_s, "1800"); +DEFINE_mInt32(compaction_timeout_seconds, "86400"); +DEFINE_mInt32(lease_compaction_interval_seconds, "20"); +DEFINE_mInt64(base_compaction_interval_seconds_since_last_operation, "86400"); +DEFINE_mBool(enable_parallel_cumu_compaction, "false"); +DEFINE_mDouble(base_compaction_thread_num_factor, "0.25"); +DEFINE_mDouble(cumu_compaction_thread_num_factor, "0.5"); +DEFINE_mInt32(check_auto_compaction_interval_seconds, "5"); +DEFINE_mInt32(max_base_compaction_task_num_per_disk, "2"); +DEFINE_mBool(prioritize_query_perf_in_compaction, "false"); + DEFINE_mInt32(refresh_s3_info_interval_s, "60"); DEFINE_mInt32(vacuum_stale_rowsets_interval_s, "300"); DEFINE_mInt32(schedule_sync_tablets_interval_s, "600"); diff --git a/be/src/cloud/config.h b/be/src/cloud/config.h index 2cbf213320dcd0..22666164ace9ec 100644 --- a/be/src/cloud/config.h +++ b/be/src/cloud/config.h @@ -53,6 +53,16 @@ DECLARE_mInt64(base_compaction_freeze_interval_s); DECLARE_mInt64(cu_compaction_freeze_interval_s); DECLARE_mInt64(cumu_compaction_interval_s); +DECLARE_mInt32(compaction_timeout_seconds); +DECLARE_mInt32(lease_compaction_interval_seconds); +DECLARE_mInt64(base_compaction_interval_seconds_since_last_operation); +DECLARE_mBool(enable_parallel_cumu_compaction); +DECLARE_mDouble(base_compaction_thread_num_factor); +DECLARE_mDouble(cumu_compaction_thread_num_factor); +DECLARE_mInt32(check_auto_compaction_interval_seconds); +DECLARE_mInt32(max_base_compaction_task_num_per_disk); +DECLARE_mBool(prioritize_query_perf_in_compaction); + // CloudStorageEngine config DECLARE_mInt32(refresh_s3_info_interval_s); DECLARE_mInt32(vacuum_stale_rowsets_interval_s); diff --git a/be/src/common/config.cpp b/be/src/common/config.cpp index dc3b7ae04492f2..a08fdd5a6cc391 100644 --- a/be/src/common/config.cpp +++ b/be/src/common/config.cpp @@ -156,6 +156,10 @@ DEFINE_Int32(tablet_publish_txn_max_thread, "32"); DEFINE_Int32(publish_version_task_timeout_s, "8"); // the count of thread to calc delete bitmap DEFINE_Int32(calc_delete_bitmap_max_thread, "32"); +// the count of thread to calc delete bitmap worker, only used for cloud +DEFINE_Int32(calc_delete_bitmap_worker_count, "8"); +// the count of thread to calc tablet delete bitmap task, only used for cloud +DEFINE_Int32(calc_tablet_delete_bitmap_task_max_thread, "32"); // the count of thread to clear transaction task DEFINE_Int32(clear_transaction_task_worker_count, "1"); // the count of thread to delete @@ -908,6 +912,8 @@ DEFINE_Validator(file_cache_type, [](std::string_view config) -> bool { return config.empty() || config == "file_block_cache"; }); +DEFINE_String(tmp_file_dir, "tmp"); + DEFINE_Int32(s3_transfer_executor_pool_size, "2"); DEFINE_Bool(enable_time_lut, "true"); diff --git a/be/src/common/config.h b/be/src/common/config.h index b95969b761bf65..81685bc1e5cf7c 100644 --- a/be/src/common/config.h +++ b/be/src/common/config.h @@ -204,6 +204,10 @@ DECLARE_Int32(tablet_publish_txn_max_thread); DECLARE_Int32(publish_version_task_timeout_s); // the count of thread to calc delete bitmap DECLARE_Int32(calc_delete_bitmap_max_thread); +// the count of thread to calc delete bitmap worker, only used for cloud +DECLARE_Int32(calc_delete_bitmap_worker_count); +// the count of thread to calc tablet delete bitmap task, only used for cloud +DECLARE_Int32(calc_tablet_delete_bitmap_task_max_thread); // the count of thread to clear transaction task DECLARE_Int32(clear_transaction_task_worker_count); // the count of thread to delete @@ -1257,6 +1261,9 @@ DECLARE_mBool(check_segment_when_build_rowset_meta); // max s3 client retry times DECLARE_mInt32(max_s3_client_retry); +// write as inverted index tmp directory +DECLARE_String(tmp_file_dir); + #ifdef BE_TEST // test s3 DECLARE_String(test_s3_resource); diff --git a/be/src/exprs/bitmapfilter_predicate.h b/be/src/exprs/bitmapfilter_predicate.h index 8df488cf875fce..376453c06817b0 100644 --- a/be/src/exprs/bitmapfilter_predicate.h +++ b/be/src/exprs/bitmapfilter_predicate.h @@ -28,7 +28,7 @@ namespace doris { // only used in Runtime Filter -class BitmapFilterFuncBase : public FilterFuncBase { +class BitmapFilterFuncBase : public RuntimeFilterFuncBase { public: virtual void insert(const void* data) = 0; virtual void insert_many(const std::vector& bitmaps) = 0; diff --git a/be/src/exprs/bloom_filter_func.h b/be/src/exprs/bloom_filter_func.h index 84e6eba1e44e89..ce1ceb6f8f7e50 100644 --- a/be/src/exprs/bloom_filter_func.h +++ b/be/src/exprs/bloom_filter_func.h @@ -27,7 +27,10 @@ namespace doris { class BloomFilterAdaptor { public: - BloomFilterAdaptor() { _bloom_filter = std::make_shared(); } + BloomFilterAdaptor(bool null_aware = false) : _null_aware(null_aware) { + _bloom_filter = std::make_shared(); + } + static int64_t optimal_bit_num(int64_t expect_num, double fpp) { return doris::segment_v2::BloomFilter::optimal_bit_num(expect_num, fpp) / 8; } @@ -74,12 +77,18 @@ class BloomFilterAdaptor { } } + void set_contain_null() { _contain_null = true; } + + bool contain_null() const { return _null_aware && _contain_null; } + private: + bool _null_aware = false; + bool _contain_null = false; std::shared_ptr _bloom_filter; }; // Only Used In RuntimeFilter -class BloomFilterFuncBase : public FilterFuncBase { +class BloomFilterFuncBase : public RuntimeFilterFuncBase { public: virtual ~BloomFilterFuncBase() = default; @@ -236,10 +245,13 @@ uint16_t find_batch_olap(const BloomFilterAdaptor& bloom_filter, const char* dat for (int i = 0; i < number; i++) { uint16_t idx = offsets[i]; if (nullmap[idx]) { - continue; - } - if (!bloom_filter.test_element(get_element(data, idx))) { - continue; + if (!bloom_filter.contain_null()) { + continue; + } + } else { + if (!bloom_filter.test_element(get_element(data, idx))) { + continue; + } } offsets[new_size++] = idx; } @@ -255,10 +267,13 @@ uint16_t find_batch_olap(const BloomFilterAdaptor& bloom_filter, const char* dat } else { for (int i = 0; i < number; i++) { if (nullmap[i]) { - continue; - } - if (!bloom_filter.test_element(get_element(data, i))) { - continue; + if (!bloom_filter.contain_null()) { + continue; + } + } else { + if (!bloom_filter.test_element(get_element(data, i))) { + continue; + } } offsets[new_size++] = i; } @@ -277,6 +292,7 @@ struct CommonFindOp { void insert_batch(BloomFilterAdaptor& bloom_filter, const vectorized::ColumnPtr& column, size_t start) const { + const auto size = column->size(); if (column->is_nullable()) { const auto* nullable = assert_cast(column.get()); const auto& col = nullable->get_nested_column(); @@ -285,14 +301,16 @@ struct CommonFindOp { .get_data(); const T* data = (T*)col.get_raw_data().data; - for (size_t i = start; i < column->size(); i++) { + for (size_t i = start; i < size; i++) { if (!nullmap[i]) { bloom_filter.add_element(*(data + i)); + } else { + bloom_filter.set_contain_null(); } } } else { const T* data = (T*)column->get_raw_data().data; - for (size_t i = start; i < column->size(); i++) { + for (size_t i = start; i < size; i++) { bloom_filter.add_element(*(data + i)); } } @@ -315,16 +333,17 @@ struct CommonFindOp { data = (T*)column->get_raw_data().data; } + const auto size = column->size(); if (nullmap) { - for (size_t i = 0; i < column->size(); i++) { + for (size_t i = 0; i < size; i++) { if (!nullmap[i]) { results[i] = bloom_filter.test_element(data[i]); } else { - results[i] = false; + results[i] = bloom_filter.contain_null(); } } } else { - for (size_t i = 0; i < column->size(); i++) { + for (size_t i = 0; i < size; i++) { results[i] = bloom_filter.test_element(data[i]); } } @@ -346,14 +365,16 @@ struct StringFindOp : CommonFindOp { assert_cast(nullable->get_null_map_column()) .get_data(); - for (size_t i = start; i < column->size(); i++) { + for (size_t i = start; i < col.size(); i++) { if (!nullmap[i]) { bloom_filter.add_element(col.get_data_at(i)); + } else { + bloom_filter.set_contain_null(); } } } else { const auto& col = assert_cast(column.get()); - for (size_t i = start; i < column->size(); i++) { + for (size_t i = start; i < col->size(); i++) { bloom_filter.add_element(col->get_data_at(i)); } } @@ -368,22 +389,23 @@ struct StringFindOp : CommonFindOp { const auto& nullmap = assert_cast(nullable->get_null_map_column()) .get_data(); + if (nullable->has_null()) { - for (size_t i = 0; i < column->size(); i++) { + for (size_t i = 0; i < col.size(); i++) { if (!nullmap[i]) { results[i] = bloom_filter.test_element(col.get_data_at(i)); } else { - results[i] = false; + results[i] = bloom_filter.contain_null(); } } } else { - for (size_t i = 0; i < column->size(); i++) { + for (size_t i = 0; i < col.size(); i++) { results[i] = bloom_filter.test_element(col.get_data_at(i)); } } } else { const auto& col = assert_cast(column.get()); - for (size_t i = 0; i < column->size(); i++) { + for (size_t i = 0; i < col->size(); i++) { results[i] = bloom_filter.test_element(col->get_data_at(i)); } } @@ -451,6 +473,7 @@ class BloomFilterFunc final : public BloomFilterFuncBase { uint16_t idx = offsets[i]; offsets[new_size] = idx; if constexpr (is_nullable) { + new_size += nullmap[idx] && _bloom_filter->contain_null(); new_size += !nullmap[idx] && _bloom_filter->test(column->get_hash_value(idx)); } else { new_size += _bloom_filter->test(column->get_hash_value(idx)); diff --git a/be/src/exprs/hybrid_set.h b/be/src/exprs/hybrid_set.h index 9151dc7d3bd550..96e0c3f879af08 100644 --- a/be/src/exprs/hybrid_set.h +++ b/be/src/exprs/hybrid_set.h @@ -175,7 +175,7 @@ class DynamicContainer { }; // TODO Maybe change void* parameter to template parameter better. -class HybridSetBase : public FilterFuncBase { +class HybridSetBase : public RuntimeFilterFuncBase { public: HybridSetBase() = default; virtual ~HybridSetBase() = default; @@ -275,6 +275,8 @@ class HybridSet : public HybridSetBase { void insert(void* data, size_t /*unused*/) override { insert(data); } void insert_fixed_len(const vectorized::ColumnPtr& column, size_t start) override { + const auto size = column->size(); + if (column->is_nullable()) { const auto* nullable = assert_cast(column.get()); const auto& col = nullable->get_nested_column(); @@ -283,14 +285,14 @@ class HybridSet : public HybridSetBase { .get_data(); const ElementType* data = (ElementType*)col.get_raw_data().data; - for (size_t i = start; i < column->size(); i++) { + for (size_t i = start; i < size; i++) { if (!nullmap[i]) { _set.insert(*(data + i)); } } } else { const ElementType* data = (ElementType*)column->get_raw_data().data; - for (size_t i = start; i < column->size(); i++) { + for (size_t i = start; i < size; i++) { _set.insert(*(data + i)); } } @@ -412,14 +414,14 @@ class StringSet : public HybridSetBase { assert_cast(nullable->get_null_map_column()) .get_data(); - for (size_t i = start; i < column->size(); i++) { + for (size_t i = start; i < nullable->size(); i++) { if (!nullmap[i]) { _set.insert(col.get_data_at(i).to_string()); } } } else { const auto& col = assert_cast(column.get()); - for (size_t i = start; i < column->size(); i++) { + for (size_t i = start; i < col->size(); i++) { _set.insert(col->get_data_at(i).to_string()); } } @@ -554,14 +556,14 @@ class StringValueSet : public HybridSetBase { assert_cast(nullable->get_null_map_column()) .get_data(); - for (size_t i = start; i < column->size(); i++) { + for (size_t i = start; i < nullable->size(); i++) { if (!nullmap[i]) { _set.insert(col.get_data_at(i)); } } } else { const auto& col = assert_cast(column.get()); - for (size_t i = start; i < column->size(); i++) { + for (size_t i = start; i < col->size(); i++) { _set.insert(col->get_data_at(i)); } } diff --git a/be/src/exprs/minmax_predicate.h b/be/src/exprs/minmax_predicate.h index b9ee56a8dc1148..297530dbd84591 100644 --- a/be/src/exprs/minmax_predicate.h +++ b/be/src/exprs/minmax_predicate.h @@ -65,9 +65,10 @@ class MinMaxNumFunc : public MinMaxFuncBase { } void update_batch(const vectorized::ColumnPtr& column, size_t start) { + const auto size = column->size(); if constexpr (std::is_same_v) { const auto& column_string = assert_cast(*column); - for (size_t i = start; i < column->size(); i++) { + for (size_t i = start; i < size; i++) { if constexpr (NeedMin) { _min = std::min(_min, column_string.get_data_at(i)); } @@ -77,7 +78,7 @@ class MinMaxNumFunc : public MinMaxFuncBase { } } else { const T* data = (T*)column->get_raw_data().data; - for (size_t i = start; i < column->size(); i++) { + for (size_t i = start; i < size; i++) { if constexpr (NeedMin) { _min = std::min(_min, *(data + i)); } @@ -90,9 +91,10 @@ class MinMaxNumFunc : public MinMaxFuncBase { void update_batch(const vectorized::ColumnPtr& column, const vectorized::NullMap& nullmap, size_t start) { + const auto size = column->size(); if constexpr (std::is_same_v) { const auto& column_string = assert_cast(*column); - for (size_t i = start; i < column->size(); i++) { + for (size_t i = start; i < size; i++) { if (!nullmap[i]) { if constexpr (NeedMin) { _min = std::min(_min, column_string.get_data_at(i)); @@ -104,7 +106,7 @@ class MinMaxNumFunc : public MinMaxFuncBase { } } else { const T* data = (T*)column->get_raw_data().data; - for (size_t i = start; i < column->size(); i++) { + for (size_t i = start; i < size; i++) { if (!nullmap[i]) { if constexpr (NeedMin) { _min = std::min(_min, *(data + i)); diff --git a/be/src/exprs/runtime_filter.cpp b/be/src/exprs/runtime_filter.cpp index 24c41613be4e51..84c82d89b388dc 100644 --- a/be/src/exprs/runtime_filter.cpp +++ b/be/src/exprs/runtime_filter.cpp @@ -1026,6 +1026,8 @@ Status IRuntimeFilter::push_to_remote(const TNetworkAddress* addr, bool opt_remo merge_filter_request->set_filter_id(_filter_id); merge_filter_request->set_opt_remote_rf(opt_remote_rf); merge_filter_request->set_is_pipeline(_state->enable_pipeline_exec); + auto column_type = _wrapper->column_type(); + merge_filter_request->set_column_type(to_proto(column_type)); merge_filter_callback->cntl_->set_timeout_ms(wait_time_ms()); Status serialize_status = serialize(merge_filter_request.get(), &data, &len); @@ -1325,6 +1327,9 @@ Status IRuntimeFilter::_create_wrapper(const T* param, ObjectPool* pool, if (param->request->has_in_filter()) { column_type = to_primitive_type(param->request->in_filter().column_type()); } + if (param->request->has_column_type()) { + column_type = to_primitive_type(param->request->column_type()); + } wrapper->reset(new RuntimePredicateWrapper(pool, column_type, get_type(filter_type), param->request->filter_id())); switch (filter_type) { diff --git a/be/src/exprs/runtime_filter.h b/be/src/exprs/runtime_filter.h index 91456cccced924..c83758f38ba395 100644 --- a/be/src/exprs/runtime_filter.h +++ b/be/src/exprs/runtime_filter.h @@ -132,7 +132,8 @@ struct RuntimeFilterParams { bool bitmap_filter_not_in; bool build_bf_exactly; }; -struct FilterFuncBase { + +struct RuntimeFilterFuncBase { public: void set_filter_id(int filter_id) { if (_filter_id == -1) { @@ -147,6 +148,7 @@ struct FilterFuncBase { private: int _filter_id = -1; }; + struct UpdateRuntimeFilterParams { UpdateRuntimeFilterParams(const PPublishFilterRequest* req, butil::IOBufAsZeroCopyInputStream* data_stream, ObjectPool* obj_pool) diff --git a/be/src/http/action/cloud_compaction_action.cpp b/be/src/http/action/cloud_compaction_action.cpp deleted file mode 100644 index f8ed8e089a264a..00000000000000 --- a/be/src/http/action/cloud_compaction_action.cpp +++ /dev/null @@ -1,41 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#include "http/action/cloud_compaction_action.h" - -#include "http/http_channel.h" -#include "http/http_headers.h" -#include "http/http_request.h" -#include "http/http_status.h" - -namespace doris { -using namespace ErrorCode; - -const static std::string HEADER_JSON = "application/json"; - -CloudCompactionAction::CloudCompactionAction(CompactionActionType ctype, ExecEnv* exec_env, - CloudStorageEngine& engine, TPrivilegeHier::type hier, - TPrivilegeType::type ptype) - : HttpHandlerWithAuth(exec_env, hier, ptype), _engine(engine), _type(ctype) {} - -void CloudCompactionAction::handle(HttpRequest* req) { - std::string json_str = R"({"status" : "Success"})"; - req->add_output_header(HttpHeaders::CONTENT_TYPE, HEADER_JSON.c_str()); - HttpChannel::send_reply(req, HttpStatus::OK, json_str); -} - -} // end namespace doris diff --git a/be/src/olap/accept_null_predicate.h b/be/src/olap/accept_null_predicate.h index 35d2546582b0f4..1f6f9f6ec3523b 100644 --- a/be/src/olap/accept_null_predicate.h +++ b/be/src/olap/accept_null_predicate.h @@ -65,15 +65,15 @@ class AcceptNullPredicate : public ColumnPredicate { bool* flags) const override { if (column.has_null()) { // copy original flags - auto original_flags_buf = std::make_unique(size); - auto* original_flags = original_flags_buf.get(); + bool original_flags[size]; memcpy(original_flags, flags, size * sizeof(bool)); + const auto& nullable_col = assert_cast(column); // call evaluate_and and restore true for NULL rows - _nested->evaluate_and(column, sel, size, flags); + _nested->evaluate_and(nullable_col.get_nested_column(), sel, size, flags); for (uint16_t i = 0; i < size; ++i) { uint16_t idx = sel[i]; - if (original_flags[idx] && !flags[idx] && column.is_null_at(idx)) { + if (original_flags[i] && !flags[i] && nullable_col.is_null_at(idx)) { flags[i] = true; } } @@ -85,11 +85,13 @@ class AcceptNullPredicate : public ColumnPredicate { void evaluate_or(const vectorized::IColumn& column, const uint16_t* sel, uint16_t size, bool* flags) const override { if (column.has_null()) { + const auto& nullable_col = assert_cast(column); + _nested->evaluate_or(nullable_col.get_nested_column(), sel, size, flags); + // call evaluate_or and set true for NULL rows - _nested->evaluate_or(column, sel, size, flags); for (uint16_t i = 0; i < size; ++i) { uint16_t idx = sel[i]; - if (!flags[idx] && column.is_null_at(idx)) { + if (!flags[i] && nullable_col.is_null_at(idx)) { flags[i] = true; } } @@ -118,14 +120,17 @@ class AcceptNullPredicate : public ColumnPredicate { void evaluate_vec(const vectorized::IColumn& column, uint16_t size, bool* flags) const override { - _nested->evaluate_vec(column, size, flags); if (column.has_null()) { + const auto& nullable_col = assert_cast(column); + _nested->evaluate_vec(nullable_col.get_nested_column(), size, flags); for (uint16_t i = 0; i < size; ++i) { - if (!flags[i] && column.is_null_at(i)) { + if (!flags[i] && nullable_col.is_null_at(i)) { // set true for NULL rows flags[i] = true; } } + } else { + _nested->evaluate_vec(column, size, flags); } } @@ -133,14 +138,14 @@ class AcceptNullPredicate : public ColumnPredicate { bool* flags) const override { if (column.has_null()) { // copy original flags - auto original_flags_buf = std::make_unique(size); - auto* original_flags = original_flags_buf.get(); + bool original_flags[size]; memcpy(original_flags, flags, size * sizeof(bool)); + const auto& nullable_col = assert_cast(column); // call evaluate_and_vec and restore true for NULL rows - _nested->evaluate_and_vec(column, size, flags); + _nested->evaluate_and_vec(nullable_col.get_nested_column(), size, flags); for (uint16_t i = 0; i < size; ++i) { - if (original_flags[i] && !flags[i] && column.is_null_at(i)) { + if (original_flags[i] && !flags[i] && nullable_col.is_null_at(i)) { flags[i] = true; } } @@ -155,44 +160,35 @@ class AcceptNullPredicate : public ColumnPredicate { uint16_t _evaluate_inner(const vectorized::IColumn& column, uint16_t* sel, uint16_t size) const override { if (column.has_null()) { + if (size == 0) return 0; // create selected_flags - uint16_t max_idx = *std::max_element(sel, sel + size); - auto selected_flags_ptr = std::make_unique(max_idx + 1); - auto* selected_flags = selected_flags_ptr.get(); - // init to 0 / false - memset(selected_flags, 0, (max_idx + 1) * sizeof(bool)); - for (uint16_t i = 0; i < size; ++i) { - uint16_t row_idx = sel[i]; - if (column.is_null_at(row_idx)) { - // set selected flag true for NULL value - selected_flags[row_idx] = true; - } - } + uint16_t max_idx = sel[size - 1]; + bool selected[max_idx + 1]; + const auto& nullable_col = assert_cast(column); + memcpy(selected, nullable_col.get_null_map_data().data(), (max_idx + 1) * sizeof(bool)); // call nested predicate evaluate - uint16_t new_size = _nested->evaluate(column, sel, size); + uint16_t new_size = _nested->evaluate(nullable_col.get_nested_column(), sel, size); // process NULL values if (new_size < size) { // add rows selected by _nested->evaluate for (uint16_t i = 0; i < new_size; ++i) { uint16_t row_idx = sel[i]; - selected_flags[row_idx] = true; + selected[row_idx] = true; } // recaculate new_size and sel array new_size = 0; for (uint16_t row_idx = 0; row_idx < max_idx + 1; ++row_idx) { - if (selected_flags[row_idx]) { + if (selected[row_idx]) { sel[new_size++] = row_idx; } } } - return new_size; - } else { - return _nested->evaluate(column, sel, size); } + return _nested->evaluate(column, sel, size); } std::string _debug_string() const override { diff --git a/be/src/olap/base_tablet.cpp b/be/src/olap/base_tablet.cpp index 09960fd0c0368c..c4ce206d8c5ffd 100644 --- a/be/src/olap/base_tablet.cpp +++ b/be/src/olap/base_tablet.cpp @@ -18,7 +18,9 @@ #include "olap/base_tablet.h" #include +#include +#include "common/status.h" #include "olap/calc_delete_bitmap_executor.h" #include "olap/delete_bitmap_calculator.h" #include "olap/memtable.h" @@ -27,8 +29,10 @@ #include "olap/rowset/rowset.h" #include "olap/rowset/rowset_reader.h" #include "olap/tablet_fwd.h" +#include "olap/txn_manager.h" #include "service/point_query_executor.h" #include "util/bvar_helper.h" +#include "util/debug_points.h" #include "util/doris_metrics.h" #include "vec/common/schema_util.h" #include "vec/data_types/data_type_factory.hpp" @@ -45,6 +49,7 @@ bvar::LatencyRecorder g_tablet_lookup_rowkey_latency("doris_pk", "tablet_lookup_ bvar::Adder g_tablet_pk_not_found("doris_pk", "lookup_not_found"); bvar::PerSecond> g_tablet_pk_not_found_per_second( "doris_pk", "lookup_not_found_per_second", &g_tablet_pk_not_found, 60); +bvar::LatencyRecorder g_tablet_update_delete_bitmap_latency("doris_pk", "update_delete_bitmap"); // read columns by read plan // read_index: ori_pos-> block_idx @@ -1059,4 +1064,200 @@ Status BaseTablet::_capture_consistent_rowsets_unlocked( return Status::OK(); } +Status BaseTablet::check_delete_bitmap_correctness(DeleteBitmapPtr delete_bitmap, + int64_t max_version, int64_t txn_id, + const RowsetIdUnorderedSet& rowset_ids, + std::vector* rowsets) { + RowsetIdUnorderedSet missing_ids; + for (const auto& rowsetid : rowset_ids) { + if (!delete_bitmap->delete_bitmap.contains({rowsetid, DeleteBitmap::INVALID_SEGMENT_ID, + DeleteBitmap::TEMP_VERSION_COMMON})) { + missing_ids.insert(rowsetid); + } + } + + if (!missing_ids.empty()) { + LOG(WARNING) << "[txn_id:" << txn_id << "][tablet_id:" << tablet_id() + << "][max_version: " << max_version + << "] check delete bitmap correctness failed!"; + rapidjson::Document root; + root.SetObject(); + rapidjson::Document required_rowsets_arr; + required_rowsets_arr.SetArray(); + rapidjson::Document missing_rowsets_arr; + missing_rowsets_arr.SetArray(); + + if (rowsets != nullptr) { + for (const auto& rowset : *rowsets) { + rapidjson::Value value; + std::string version_str = rowset->get_rowset_info_str(); + value.SetString(version_str.c_str(), version_str.length(), + required_rowsets_arr.GetAllocator()); + required_rowsets_arr.PushBack(value, required_rowsets_arr.GetAllocator()); + } + } else { + std::vector rowsets; + { + std::shared_lock meta_rlock(_meta_lock); + rowsets = get_rowset_by_ids(&rowset_ids); + } + for (const auto& rowset : rowsets) { + rapidjson::Value value; + std::string version_str = rowset->get_rowset_info_str(); + value.SetString(version_str.c_str(), version_str.length(), + required_rowsets_arr.GetAllocator()); + required_rowsets_arr.PushBack(value, required_rowsets_arr.GetAllocator()); + } + } + for (const auto& missing_rowset_id : missing_ids) { + rapidjson::Value miss_value; + std::string rowset_id_str = missing_rowset_id.to_string(); + miss_value.SetString(rowset_id_str.c_str(), rowset_id_str.length(), + missing_rowsets_arr.GetAllocator()); + missing_rowsets_arr.PushBack(miss_value, missing_rowsets_arr.GetAllocator()); + } + + root.AddMember("required_rowsets", required_rowsets_arr, root.GetAllocator()); + root.AddMember("missing_rowsets", missing_rowsets_arr, root.GetAllocator()); + rapidjson::StringBuffer strbuf; + rapidjson::PrettyWriter writer(strbuf); + root.Accept(writer); + std::string rowset_status_string = std::string(strbuf.GetString()); + LOG_EVERY_SECOND(WARNING) << rowset_status_string; + // let it crash if correctness check failed in Debug mode + DCHECK(false) << "delete bitmap correctness check failed in publish phase!"; + return Status::InternalError("check delete bitmap failed!"); + } + return Status::OK(); +} + +void BaseTablet::_remove_sentinel_mark_from_delete_bitmap(DeleteBitmapPtr delete_bitmap) { + for (auto it = delete_bitmap->delete_bitmap.begin(), end = delete_bitmap->delete_bitmap.end(); + it != end;) { + if (std::get<1>(it->first) == DeleteBitmap::INVALID_SEGMENT_ID) { + it = delete_bitmap->delete_bitmap.erase(it); + } else { + ++it; + } + } +} + +Status BaseTablet::update_delete_bitmap(const BaseTabletSPtr& self, const TabletTxnInfo* txn_info, + int64_t txn_id, int64_t txn_expiration) { + SCOPED_BVAR_LATENCY(g_tablet_update_delete_bitmap_latency); + RowsetIdUnorderedSet cur_rowset_ids; + RowsetIdUnorderedSet rowset_ids_to_add; + RowsetIdUnorderedSet rowset_ids_to_del; + RowsetSharedPtr rowset = txn_info->rowset; + int64_t cur_version = rowset->start_version(); + + auto rowset_writer = DORIS_TRY(self->create_transient_rowset_writer( + *rowset, txn_info->partial_update_info, txn_expiration)); + + DeleteBitmapPtr delete_bitmap = txn_info->delete_bitmap; + // Partial update might generate new segments when there is conflicts while publish, and mark + // the same key in original segments as delete. + // When the new segment flush fails or the rowset build fails, the deletion marker for the + // duplicate key of the original segment should not remain in `txn_info->delete_bitmap`, + // so we need to make a copy of `txn_info->delete_bitmap` and make changes on it. + if (txn_info->partial_update_info && txn_info->partial_update_info->is_partial_update) { + delete_bitmap = std::make_shared(*(txn_info->delete_bitmap)); + } + + OlapStopWatch watch; + std::vector segments; + RETURN_IF_ERROR(std::dynamic_pointer_cast(rowset)->load_segments(&segments)); + auto t1 = watch.get_elapse_time_us(); + + { + std::shared_lock meta_rlock(self->_meta_lock); + // tablet is under alter process. The delete bitmap will be calculated after conversion. + if (self->tablet_state() == TABLET_NOTREADY) { + LOG(INFO) << "tablet is under alter process, update delete bitmap later, tablet_id=" + << self->tablet_id(); + return Status::OK(); + } + RETURN_IF_ERROR(self->get_all_rs_id_unlocked(cur_version - 1, &cur_rowset_ids)); + } + auto t2 = watch.get_elapse_time_us(); + + _rowset_ids_difference(cur_rowset_ids, txn_info->rowset_ids, &rowset_ids_to_add, + &rowset_ids_to_del); + for (const auto& to_del : rowset_ids_to_del) { + delete_bitmap->remove({to_del, 0, 0}, {to_del, UINT32_MAX, INT64_MAX}); + } + + std::vector specified_rowsets; + { + std::shared_lock meta_rlock(self->_meta_lock); + specified_rowsets = self->get_rowset_by_ids(&rowset_ids_to_add); + } + auto t3 = watch.get_elapse_time_us(); + + // When there is only one segment, it will be calculated in the current thread. + // Otherwise, it will be submitted to the thread pool for calculation. + if (segments.size() <= 1) { + RETURN_IF_ERROR(calc_delete_bitmap(self, rowset, segments, specified_rowsets, delete_bitmap, + cur_version - 1, nullptr, rowset_writer.get())); + + } else { + auto token = self->calc_delete_bitmap_executor()->create_token(); + RETURN_IF_ERROR(calc_delete_bitmap(self, rowset, segments, specified_rowsets, delete_bitmap, + cur_version - 1, token.get(), rowset_writer.get())); + RETURN_IF_ERROR(token->wait()); + } + + std::stringstream ss; + if (watch.get_elapse_time_us() < 1 * 1000 * 1000) { + ss << "cost: " << watch.get_elapse_time_us() - t3 << "(us)"; + } else { + ss << "cost(us): (load segments: " << t1 << ", get all rsid: " << t2 - t1 + << ", get rowsets: " << t3 - t2 + << ", calc delete bitmap: " << watch.get_elapse_time_us() - t3 << ")"; + } + + size_t total_rows = std::accumulate( + segments.begin(), segments.end(), 0, + [](size_t sum, const segment_v2::SegmentSharedPtr& s) { return sum += s->num_rows(); }); + LOG(INFO) << "[Publish] construct delete bitmap tablet: " << self->tablet_id() + << ", rowset_ids to add: " << rowset_ids_to_add.size() + << ", rowset_ids to del: " << rowset_ids_to_del.size() + << ", cur version: " << cur_version << ", transaction_id: " << txn_id << "," + << ss.str() << " , total rows: " << total_rows; + + if (config::enable_merge_on_write_correctness_check && rowset->num_rows() != 0) { + // only do correctness check if the rowset has at least one row written + // check if all the rowset has ROWSET_SENTINEL_MARK + auto st = self->check_delete_bitmap_correctness(delete_bitmap, cur_version - 1, -1, + cur_rowset_ids, &specified_rowsets); + if (!st.ok()) { + LOG(WARNING) << fmt::format("delete bitmap correctness check failed in publish phase!"); + } + self->_remove_sentinel_mark_from_delete_bitmap(delete_bitmap); + } + + if (txn_info->partial_update_info && txn_info->partial_update_info->is_partial_update) { + DBUG_EXECUTE_IF("Tablet.update_delete_bitmap.partial_update_write_rowset_fail", { + if (rand() % 100 < (100 * dp->param("percent", 0.5))) { + LOG_WARNING("Tablet.update_delete_bitmap.partial_update_write_rowset random failed") + .tag("txn_id", txn_id); + return Status::InternalError( + "debug update_delete_bitmap partial update write rowset random failed"); + } + }); + // build rowset writer and merge transient rowset + RETURN_IF_ERROR(rowset_writer->flush()); + RowsetSharedPtr transient_rowset; + RETURN_IF_ERROR(rowset_writer->build(transient_rowset)); + rowset->merge_rowset_meta(transient_rowset->rowset_meta()); + + // erase segment cache cause we will add a segment to rowset + SegmentLoader::instance()->erase_segments(rowset->rowset_id(), rowset->num_segments()); + } + + RETURN_IF_ERROR(self->save_delete_bitmap(txn_info, txn_id, delete_bitmap, rowset_writer.get(), + cur_rowset_ids)); + return Status::OK(); +} + } // namespace doris diff --git a/be/src/olap/base_tablet.h b/be/src/olap/base_tablet.h index 943c257de645e3..837ff28d461205 100644 --- a/be/src/olap/base_tablet.h +++ b/be/src/olap/base_tablet.h @@ -22,6 +22,7 @@ #include #include "common/status.h" +#include "olap/partial_update_info.h" #include "olap/rowset/segment_v2/segment.h" #include "olap/tablet_fwd.h" #include "olap/tablet_meta.h" @@ -178,6 +179,10 @@ class BaseTablet { static void add_sentinel_mark_to_delete_bitmap(DeleteBitmap* delete_bitmap, const RowsetIdUnorderedSet& rowsetids); + Status check_delete_bitmap_correctness(DeleteBitmapPtr delete_bitmap, int64_t max_version, + int64_t txn_id, const RowsetIdUnorderedSet& rowset_ids, + std::vector* rowsets = nullptr); + static Status generate_new_block_for_partial_update( TabletSchemaSPtr rowset_schema, const std::vector& missing_cids, const std::vector& update_cids, const PartialUpdateReadPlan& read_plan_ori, @@ -198,6 +203,18 @@ class BaseTablet { const std::vector& rowids, const TabletColumn& tablet_column, vectorized::MutableColumnPtr& dst); + + virtual Result> create_transient_rowset_writer( + const Rowset& rowset, std::shared_ptr partial_update_info, + int64_t txn_expiration = 0) = 0; + + static Status update_delete_bitmap(const BaseTabletSPtr& self, const TabletTxnInfo* txn_info, + int64_t txn_id, int64_t txn_expiration = 0); + + virtual Status save_delete_bitmap(const TabletTxnInfo* txn_info, int64_t txn_id, + DeleteBitmapPtr delete_bitmap, RowsetWriter* rowset_writer, + const RowsetIdUnorderedSet& cur_rowset_ids) = 0; + virtual CalcDeleteBitmapExecutor* calc_delete_bitmap_executor() = 0; //////////////////////////////////////////////////////////////////////////// // end MoW functions //////////////////////////////////////////////////////////////////////////// @@ -216,6 +233,7 @@ class BaseTablet { static void _rowset_ids_difference(const RowsetIdUnorderedSet& cur, const RowsetIdUnorderedSet& pre, RowsetIdUnorderedSet* to_add, RowsetIdUnorderedSet* to_del); + static void _remove_sentinel_mark_from_delete_bitmap(DeleteBitmapPtr delete_bitmap); Status _capture_consistent_rowsets_unlocked(const std::vector& version_path, std::vector* rowsets) const; diff --git a/be/src/olap/compaction.cpp b/be/src/olap/compaction.cpp index 7c46435f336c7b..a512d89be58ae5 100644 --- a/be/src/olap/compaction.cpp +++ b/be/src/olap/compaction.cpp @@ -33,8 +33,11 @@ #include #include +#include "cloud/cloud_meta_mgr.h" +#include "cloud/cloud_storage_engine.h" #include "common/config.h" #include "common/status.h" +#include "common/sync_point.h" #include "io/fs/file_system.h" #include "io/fs/remote_file_system.h" #include "olap/cumulative_compaction_policy.h" @@ -178,6 +181,8 @@ Status Compaction::merge_input_rowsets() { fmt::format("rowset writer build failed. output_version: {}", _output_version.to_string())); + //RETURN_IF_ERROR(_engine.meta_mgr().commit_rowset(*_output_rowset->rowset_meta().get(), true)); + // Now we support delete in cumu compaction, to make all data in rowsets whose version // is below output_version to be delete in the future base compaction, we should carry // all delete predicate in the output rowset. @@ -846,4 +851,80 @@ void Compaction::_load_segment_to_cache() { } } +void CloudCompactionMixin::build_basic_info() { + _output_version = + Version(_input_rowsets.front()->start_version(), _input_rowsets.back()->end_version()); + + _newest_write_timestamp = _input_rowsets.back()->newest_write_timestamp(); + + std::vector rowset_metas(_input_rowsets.size()); + std::transform(_input_rowsets.begin(), _input_rowsets.end(), rowset_metas.begin(), + [](const RowsetSharedPtr& rowset) { return rowset->rowset_meta(); }); + _cur_tablet_schema = _tablet->tablet_schema_with_merged_max_schema_version(rowset_metas); +} + +int64_t CloudCompactionMixin::get_compaction_permits() { + int64_t permits = 0; + for (auto&& rowset : _input_rowsets) { + permits += rowset->rowset_meta()->get_compaction_score(); + } + return permits; +} + +CloudCompactionMixin::CloudCompactionMixin(CloudStorageEngine& engine, CloudTabletSPtr tablet, + const std::string& label) + : Compaction(tablet, label), _engine(engine) {} + +Status CloudCompactionMixin::execute_compact_impl(int64_t permits) { + OlapStopWatch watch; + + build_basic_info(); + + LOG(INFO) << "start " << compaction_name() << ". tablet=" << _tablet->tablet_id() + << ", output_version=" << _output_version << ", permits: " << permits; + + RETURN_IF_ERROR(merge_input_rowsets()); + + RETURN_IF_ERROR(_engine.meta_mgr().commit_rowset(*_output_rowset->rowset_meta().get(), true)); + + // 4. modify rowsets in memory + RETURN_IF_ERROR(modify_rowsets()); + + return Status::OK(); +} + +Status CloudCompactionMixin::execute_compact() { + TEST_INJECTION_POINT("Compaction::do_compaction"); + int64_t permits = get_compaction_permits(); + Status st = execute_compact_impl(permits); + if (!st.ok()) { + garbage_collection(); + return st; + } + _load_segment_to_cache(); + return st; +} + +Status CloudCompactionMixin::modify_rowsets() { + return Status::OK(); +} + +Status CloudCompactionMixin::construct_output_rowset_writer(RowsetWriterContext& ctx) { + ctx.fs = _engine.latest_fs(); + ctx.txn_id = boost::uuids::hash_value(UUIDGenerator::instance()->next_uuid()) & + std::numeric_limits::max(); // MUST be positive + ctx.txn_expiration = _expiration; + + ctx.version = _output_version; + ctx.rowset_state = VISIBLE; + ctx.segments_overlap = NONOVERLAPPING; + ctx.tablet_schema = _cur_tablet_schema; + ctx.newest_write_timestamp = _newest_write_timestamp; + ctx.write_type = DataWriteType::TYPE_COMPACTION; + _output_rs_writer = DORIS_TRY(_tablet->create_rowset_writer(ctx, _is_vertical)); + RETURN_IF_ERROR( + _engine.meta_mgr().prepare_rowset(*_output_rs_writer->rowset_meta().get(), true)); + return Status::OK(); +} + } // namespace doris diff --git a/be/src/olap/compaction.h b/be/src/olap/compaction.h index 6998856ee94c80..75837e83b0eb93 100644 --- a/be/src/olap/compaction.h +++ b/be/src/olap/compaction.h @@ -22,6 +22,7 @@ #include #include +#include "cloud/cloud_tablet.h" #include "common/status.h" #include "io/io_common.h" #include "olap/merger.h" @@ -38,6 +39,7 @@ class MemTrackerLimiter; class RowsetWriter; struct RowsetWriterContext; class StorageEngine; +class CloudStorageEngine; // This class is a base class for compaction. // The entrance of this class is compact() @@ -156,4 +158,34 @@ class CompactionMixin : public Compaction { PendingRowsetGuard _pending_rs_guard; }; +class CloudCompactionMixin : public Compaction { +public: + CloudCompactionMixin(CloudStorageEngine& engine, CloudTabletSPtr tablet, + const std::string& label); + + ~CloudCompactionMixin() override = default; + + Status execute_compact() override; + +protected: + CloudTablet* cloud_tablet() { return static_cast(_tablet.get()); } + + CloudStorageEngine& _engine; + + int64_t _expiration = 0; + +private: + Status construct_output_rowset_writer(RowsetWriterContext& ctx) override; + + virtual void garbage_collection() {}; + + Status execute_compact_impl(int64_t permits); + + void build_basic_info(); + + virtual Status modify_rowsets(); + + int64_t get_compaction_permits(); +}; + } // namespace doris diff --git a/be/src/olap/comparison_predicate.h b/be/src/olap/comparison_predicate.h index 826b1414b2afb1..3673b89a1d0077 100644 --- a/be/src/olap/comparison_predicate.h +++ b/be/src/olap/comparison_predicate.h @@ -578,24 +578,26 @@ class ComparisonPredicateBase : public ColumnPredicate { __attribute__((flatten)) int32_t _find_code_from_dictionary_column( const vectorized::ColumnDictI32& column) const { - if (!_segment_id_to_cached_code.contains(column.get_rowset_segment_id())) { - int32_t code = _is_range() ? column.find_code_by_bound(_value, _is_greater(), _is_eq()) - : column.find_code(_value); - - // Sometimes the dict is not initialized when run comparison predicate here, for example, - // the full page is null, then the reader will skip read, so that the dictionary is not - // inited. The cached code is wrong during this case, because the following page maybe not - // null, and the dict should have items in the future. - // - // Cached code may have problems, so that add a config here, if not opened, then - // we will return the code and not cache it. - if (column.is_dict_empty() || !config::enable_low_cardinality_cache_code) { - return code; - } - // If the dict is not empty, then the dict is inited and we could cache the value. - _segment_id_to_cached_code[column.get_rowset_segment_id()] = code; + int32_t code = 0; + if (_segment_id_to_cached_code.if_contains( + column.get_rowset_segment_id(), + [&code](const auto& pair) { code = pair.second; })) { + return code; + } + code = _is_range() ? column.find_code_by_bound(_value, _is_greater(), _is_eq()) + : column.find_code(_value); + // Sometimes the dict is not initialized when run comparison predicate here, for example, + // the full page is null, then the reader will skip read, so that the dictionary is not + // inited. The cached code is wrong during this case, because the following page maybe not + // null, and the dict should have items in the future. + // + // Cached code may have problems, so that add a config here, if not opened, then + // we will return the code and not cache it. + if (!column.is_dict_empty() && config::enable_low_cardinality_cache_code) { + _segment_id_to_cached_code.emplace(std::pair {column.get_rowset_segment_id(), code}); } - return _segment_id_to_cached_code[column.get_rowset_segment_id()]; + + return code; } std::string _debug_string() const override { @@ -604,7 +606,13 @@ class ComparisonPredicateBase : public ColumnPredicate { return info; } - mutable std::map, int32_t> _segment_id_to_cached_code; + mutable phmap::parallel_flat_hash_map< + std::pair, int32_t, + phmap::priv::hash_default_hash>, + phmap::priv::hash_default_eq>, + std::allocator, int32_t>>, 4, + std::shared_mutex> + _segment_id_to_cached_code; T _value; }; diff --git a/be/src/olap/data_dir.cpp b/be/src/olap/data_dir.cpp index 9a7d1ad5cdcadd..443cf1b12a3ac2 100644 --- a/be/src/olap/data_dir.cpp +++ b/be/src/olap/data_dir.cpp @@ -776,7 +776,7 @@ void DataDir::_perform_path_gc_by_rowset(const std::vector& tablet_ }; // rowset_id -> is_garbage - std::unordered_map checked_rowsets; + std::unordered_map checked_rowsets; for (auto&& [rowset_id, filename] : rowsets_not_pending) { if (auto it = checked_rowsets.find(rowset_id); it != checked_rowsets.end()) { if (it->second) { // Is checked garbage rowset diff --git a/be/src/olap/olap_common.h b/be/src/olap/olap_common.h index 42bad24dfed5b1..c08705861df7ad 100644 --- a/be/src/olap/olap_common.h +++ b/be/src/olap/olap_common.h @@ -458,18 +458,7 @@ struct RowsetId { } }; -// used for hash-struct of hash_map. -struct HashOfRowsetId { - size_t operator()(const RowsetId& rowset_id) const { - size_t seed = 0; - seed = HashUtil::hash64(&rowset_id.hi, sizeof(rowset_id.hi), seed); - seed = HashUtil::hash64(&rowset_id.mi, sizeof(rowset_id.mi), seed); - seed = HashUtil::hash64(&rowset_id.lo, sizeof(rowset_id.lo), seed); - return seed; - } -}; - -using RowsetIdUnorderedSet = std::unordered_set; +using RowsetIdUnorderedSet = std::unordered_set; // Extract rowset id from filename, return uninitialized rowset id if filename is invalid inline RowsetId extract_rowset_id(std::string_view filename) { @@ -517,3 +506,18 @@ struct RidAndPos { using PartialUpdateReadPlan = std::map>>; } // namespace doris + +// This intended to be a "good" hash function. It may change from time to time. +template <> +struct std::hash { + size_t operator()(const doris::RowsetId& rowset_id) const { + size_t seed = 0; + seed = doris::HashUtil::xxHash64WithSeed((const char*)&rowset_id.hi, sizeof(rowset_id.hi), + seed); + seed = doris::HashUtil::xxHash64WithSeed((const char*)&rowset_id.mi, sizeof(rowset_id.mi), + seed); + seed = doris::HashUtil::xxHash64WithSeed((const char*)&rowset_id.lo, sizeof(rowset_id.lo), + seed); + return seed; + } +}; diff --git a/be/src/olap/rowset/beta_rowset_writer.cpp b/be/src/olap/rowset/beta_rowset_writer.cpp index 8556e19483fc8a..f58daf6816af31 100644 --- a/be/src/olap/rowset/beta_rowset_writer.cpp +++ b/be/src/olap/rowset/beta_rowset_writer.cpp @@ -166,7 +166,7 @@ Status BaseBetaRowsetWriter::add_block(const vectorized::Block* block) { return _segment_creator.add_block(block); } -Status BetaRowsetWriter::_generate_delete_bitmap(int32_t segment_id) { +Status BaseBetaRowsetWriter::_generate_delete_bitmap(int32_t segment_id) { SCOPED_RAW_TIMER(&_delete_bitmap_ns); if (!_context.tablet->enable_unique_key_merge_on_write() || (_context.partial_update_info && _context.partial_update_info->is_partial_update)) { diff --git a/be/src/olap/rowset/beta_rowset_writer.h b/be/src/olap/rowset/beta_rowset_writer.h index 3681e0fe12030b..cffb951451e1ea 100644 --- a/be/src/olap/rowset/beta_rowset_writer.h +++ b/be/src/olap/rowset/beta_rowset_writer.h @@ -125,12 +125,11 @@ class BaseBetaRowsetWriter : public RowsetWriter { } private: - virtual Status _generate_delete_bitmap(int32_t segment_id) = 0; - void update_rowset_schema(TabletSchemaSPtr flush_schema); // build a tmp rowset for load segment to calc delete_bitmap // for this segment protected: + Status _generate_delete_bitmap(int32_t segment_id); Status _build_rowset_meta(RowsetMeta* rowset_meta, bool check_segment_num = false); Status _create_file_writer(std::string path, io::FileWriterPtr& file_writer); virtual Status _close_file_writers(); @@ -193,8 +192,6 @@ class BetaRowsetWriter : public BaseBetaRowsetWriter { KeyBoundsPB& key_bounds); private: - Status _generate_delete_bitmap(int32_t segment_id) override; - // segment compaction friend class SegcompactionWorker; Status _close_file_writers() override; diff --git a/be/src/olap/rowset/rowset.cpp b/be/src/olap/rowset/rowset.cpp index bc946b3e06e1e8..781510f2a21d56 100644 --- a/be/src/olap/rowset/rowset.cpp +++ b/be/src/olap/rowset/rowset.cpp @@ -75,6 +75,10 @@ void Rowset::make_visible(Version version) { } } +void Rowset::set_version(Version version) { + _rowset_meta->set_version(version); +} + bool Rowset::check_rowset_segment() { std::lock_guard load_lock(_lock); return check_current_rowset_segment(); diff --git a/be/src/olap/rowset/rowset.h b/be/src/olap/rowset/rowset.h index d9da990740566e..b95aad700aec4c 100644 --- a/be/src/olap/rowset/rowset.h +++ b/be/src/olap/rowset/rowset.h @@ -138,6 +138,7 @@ class Rowset : public std::enable_shared_from_this { // publish rowset to make it visible to read void make_visible(Version version); + void set_version(Version version); const TabletSchemaSPtr& tablet_schema() const { return _schema; } // helper class to access RowsetMeta diff --git a/be/src/olap/rowset/rowset_factory.cpp b/be/src/olap/rowset/rowset_factory.cpp index 5991deced63e7a..9d8b253b7f3bdd 100644 --- a/be/src/olap/rowset/rowset_factory.cpp +++ b/be/src/olap/rowset/rowset_factory.cpp @@ -23,6 +23,7 @@ #include "beta_rowset.h" #include "cloud/cloud_rowset_writer.h" +#include "cloud/cloud_vertical_rowset_writer.h" #include "cloud/config.h" #include "io/fs/file_writer.h" // IWYU pragma: keep #include "olap/rowset/beta_rowset_writer.h" @@ -72,7 +73,13 @@ Result> RowsetFactory::create_rowset_writer( CloudStorageEngine& engine, const RowsetWriterContext& context, bool is_vertical) { DCHECK_EQ(context.rowset_type, BETA_ROWSET); // TODO(plat1ko): cloud vertical rowset writer - auto writer = std::make_unique(); + std::unique_ptr writer; + if (is_vertical) { + writer = std::make_unique(); + } else { + writer = std::make_unique(); + } + RETURN_IF_ERROR_RESULT(writer->init(context)); return writer; } diff --git a/be/src/olap/rowset/segment_v2/inverted_index_compound_directory.cpp b/be/src/olap/rowset/segment_v2/inverted_index_compound_directory.cpp index 4ed3258d74da12..1a0a3e9ab1d47d 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index_compound_directory.cpp +++ b/be/src/olap/rowset/segment_v2/inverted_index_compound_directory.cpp @@ -399,7 +399,8 @@ void DorisCompoundDirectory::FSIndexInput::readInternal(uint8_t* b, const int32_ void DorisCompoundDirectory::FSIndexOutput::init(const io::FileSystemSPtr& fileSystem, const char* path) { - Status status = fileSystem->create_file(path, &_writer); + io::FileWriterOptions opts {.create_empty_file = false}; + Status status = fileSystem->create_file(path, &_writer, &opts); DBUG_EXECUTE_IF( "DorisCompoundDirectory::FSIndexOutput._throw_clucene_error_in_fsindexoutput_" "init", diff --git a/be/src/olap/rowset/segment_v2/inverted_index_writer.cpp b/be/src/olap/rowset/segment_v2/inverted_index_writer.cpp index 07bea0c83f378c..9cd9897f6f5d3e 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index_writer.cpp +++ b/be/src/olap/rowset/segment_v2/inverted_index_writer.cpp @@ -32,11 +32,13 @@ #pragma clang diagnostic push #pragma clang diagnostic ignored "-Wshadow-field" #endif + #include "CLucene/analysis/standard95/StandardAnalyzer.h" + #ifdef __clang__ #pragma clang diagnostic pop #endif -#include "common/config.h" + #include "olap/field.h" #include "olap/inverted_index_parser.h" #include "olap/key_coder.h" @@ -50,6 +52,7 @@ #include "olap/tablet_schema.h" #include "olap/types.h" #include "runtime/collection_value.h" +#include "runtime/exec_env.h" #include "util/debug_points.h" #include "util/faststring.h" #include "util/slice.h" @@ -209,8 +212,14 @@ class InvertedIndexColumnWriterImpl : public InvertedIndexColumnWriter { return Status::InternalError("init_fulltext_index directory already exists"); } + auto tmp_file_dir = ExecEnv::GetInstance()->get_tmp_file_dirs()->get_tmp_file_dir(); + _lfs = io::global_local_filesystem(); + auto lfs_index_path = InvertedIndexDescriptor::get_temporary_index_path( + tmp_file_dir / _segment_file_name, _index_meta->index_id(), + _index_meta->get_index_suffix()); dir = std::unique_ptr(DorisCompoundDirectoryFactory::getDirectory( - _fs, index_path.c_str(), use_compound_file_writer, can_use_ram_dir)); + _lfs, lfs_index_path.c_str(), use_compound_file_writer, can_use_ram_dir, nullptr, + _fs, index_path.c_str())); return Status::OK(); } @@ -451,6 +460,7 @@ class InvertedIndexColumnWriterImpl : public InvertedIndexColumnWriter { } return Status::OK(); } + Status add_array_values(size_t field_size, const CollectionValue* values, size_t count) override { if constexpr (field_is_slice_type(field_type)) { @@ -620,6 +630,7 @@ class InvertedIndexColumnWriterImpl : public InvertedIndexColumnWriter { std::string _segment_file_name; std::string _directory; io::FileSystemSPtr _fs; + io::FileSystemSPtr _lfs; const KeyCoder* _value_key_coder; const TabletIndex* _index_meta; InvertedIndexParserType _parser_type; diff --git a/be/src/olap/rowset/segment_v2/inverted_index_writer.h b/be/src/olap/rowset/segment_v2/inverted_index_writer.h index 44cc41789b5ed6..fee81f8235a763 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index_writer.h +++ b/be/src/olap/rowset/segment_v2/inverted_index_writer.h @@ -21,15 +21,22 @@ #include #include +#include #include #include +#include "common/config.h" #include "common/status.h" +#include "gutil/strings/split.h" #include "io/fs/file_system.h" +#include "io/fs/local_file_system.h" +#include "olap/options.h" namespace doris { class CollectionValue; + class Field; + class TabletIndex; namespace segment_v2 { @@ -66,5 +73,34 @@ class InvertedIndexColumnWriter { DISALLOW_COPY_AND_ASSIGN(InvertedIndexColumnWriter); }; +class TmpFileDirs { +public: + TmpFileDirs(const std::vector& store_paths) { + for (const auto& store_path : store_paths) { + _tmp_file_dirs.emplace_back(store_path.path + "/" + config::tmp_file_dir); + } + }; + + Status init() { + for (auto& tmp_file_dir : _tmp_file_dirs) { + bool exists = true; + RETURN_IF_ERROR(io::global_local_filesystem()->exists(tmp_file_dir, &exists)); + if (!exists) { + RETURN_IF_ERROR(io::global_local_filesystem()->create_directory(tmp_file_dir)); + } + } + return Status::OK(); + }; + + io::Path get_tmp_file_dir() { + size_t cur_index = _next_index.fetch_add(1); + return _tmp_file_dirs[cur_index % _tmp_file_dirs.size()]; + }; + +private: + std::vector _tmp_file_dirs; + std::atomic_size_t _next_index {0}; // use for round-robin +}; + } // namespace segment_v2 } // namespace doris diff --git a/be/src/olap/rowset/segment_v2/segment_writer.cpp b/be/src/olap/rowset/segment_v2/segment_writer.cpp index 0dfaf4b7466df3..235a698021bea3 100644 --- a/be/src/olap/rowset/segment_v2/segment_writer.cpp +++ b/be/src/olap/rowset/segment_v2/segment_writer.cpp @@ -344,11 +344,6 @@ void SegmentWriter::_serialize_block_to_row_column(vectorized::Block& block) { // 3. set columns to data convertor and then write all columns Status SegmentWriter::append_block_with_partial_content(const vectorized::Block* block, size_t row_pos, size_t num_rows) { - if (config::is_cloud_mode()) { - // TODO(plat1ko): cloud mode - return Status::NotSupported("append_block_with_partial_content"); - } - auto* tablet = static_cast(_tablet.get()); if (block->columns() <= _tablet_schema->num_key_columns() || block->columns() >= _tablet_schema->num_columns()) { diff --git a/be/src/olap/rowset/segment_v2/vertical_segment_writer.cpp b/be/src/olap/rowset/segment_v2/vertical_segment_writer.cpp index 1e3ff116783626..16d5e60b09a92b 100644 --- a/be/src/olap/rowset/segment_v2/vertical_segment_writer.cpp +++ b/be/src/olap/rowset/segment_v2/vertical_segment_writer.cpp @@ -290,11 +290,6 @@ void VerticalSegmentWriter::_serialize_block_to_row_column(vectorized::Block& bl // 2.3 fill block // 3. set columns to data convertor and then write all columns Status VerticalSegmentWriter::_append_block_with_partial_content(RowsInBlock& data) { - if (config::is_cloud_mode()) { - // TODO(plat1ko): CloudStorageEngine - return Status::NotSupported("append_block_with_partial_content"); - } - DCHECK(_tablet_schema->keys_type() == UNIQUE_KEYS && _opts.enable_unique_key_merge_on_write); DCHECK(_opts.rowset_ctx->partial_update_info != nullptr); @@ -527,10 +522,6 @@ Status VerticalSegmentWriter::_fill_missing_columns( vectorized::MutableColumns& mutable_full_columns, const std::vector& use_default_or_null_flag, bool has_default_or_nullable, const size_t& segment_start_pos) { - if (config::is_cloud_mode()) { - // TODO(plat1ko): CloudStorageEngine - return Status::NotSupported("fill_missing_columns"); - } auto tablet = static_cast(_tablet.get()); // create old value columns const auto& missing_cids = _opts.rowset_ctx->partial_update_info->missing_cids; diff --git a/be/src/olap/rowset/vertical_beta_rowset_writer.cpp b/be/src/olap/rowset/vertical_beta_rowset_writer.cpp index 35f878dd17ccbe..5fdbdcc49afd16 100644 --- a/be/src/olap/rowset/vertical_beta_rowset_writer.cpp +++ b/be/src/olap/rowset/vertical_beta_rowset_writer.cpp @@ -42,183 +42,29 @@ namespace doris { using namespace ErrorCode; VerticalBetaRowsetWriter::VerticalBetaRowsetWriter(StorageEngine& engine) - : BetaRowsetWriter(engine) {} + : BetaRowsetWriter(engine) { + _helper = std::make_shared( + &_segment_writers, _already_built, _rowset_meta, &_num_segment, _context, + &_num_rows_written, &_segments_encoded_key_bounds, &_segment_num_rows, + &_total_index_size, &_file_writers, &_total_data_size, &_lock); +} VerticalBetaRowsetWriter::~VerticalBetaRowsetWriter() { - if (!_already_built) { - const auto& fs = _rowset_meta->fs(); - if (!fs || !_rowset_meta->is_local()) { // Remote fs will delete them asynchronously - return; - } - for (auto& segment_writer : _segment_writers) { - segment_writer.reset(); - } - for (int i = 0; i < _num_segment; ++i) { - auto path = BetaRowset::segment_file_path(_context.rowset_dir, _context.rowset_id, i); - // Even if an error is encountered, these files that have not been cleaned up - // will be cleaned up by the GC background. So here we only print the error - // message when we encounter an error. - WARN_IF_ERROR(fs->delete_file(path), fmt::format("Failed to delete file={}", path)); - } - } + _helper->destruct_writer(); } Status VerticalBetaRowsetWriter::add_columns(const vectorized::Block* block, const std::vector& col_ids, bool is_key, uint32_t max_rows_per_segment) { - VLOG_NOTICE << "VerticalBetaRowsetWriter::add_columns, columns: " << block->columns(); - size_t num_rows = block->rows(); - if (num_rows == 0) { - return Status::OK(); - } - if (UNLIKELY(max_rows_per_segment > _context.max_rows_per_segment)) { - max_rows_per_segment = _context.max_rows_per_segment; - } - - if (_segment_writers.empty()) { - // it must be key columns - DCHECK(is_key); - std::unique_ptr writer; - RETURN_IF_ERROR(_create_segment_writer(col_ids, is_key, &writer)); - _segment_writers.emplace_back(std::move(writer)); - _cur_writer_idx = 0; - RETURN_IF_ERROR(_segment_writers[_cur_writer_idx]->append_block(block, 0, num_rows)); - } else if (is_key) { - if (_segment_writers[_cur_writer_idx]->num_rows_written() > max_rows_per_segment) { - // segment is full, need flush columns and create new segment writer - RETURN_IF_ERROR(_flush_columns(&_segment_writers[_cur_writer_idx], true)); - - std::unique_ptr writer; - RETURN_IF_ERROR(_create_segment_writer(col_ids, is_key, &writer)); - _segment_writers.emplace_back(std::move(writer)); - ++_cur_writer_idx; - } - RETURN_IF_ERROR(_segment_writers[_cur_writer_idx]->append_block(block, 0, num_rows)); - } else { - // value columns - uint32_t num_rows_written = _segment_writers[_cur_writer_idx]->num_rows_written(); - VLOG_NOTICE << "num_rows_written: " << num_rows_written - << ", _cur_writer_idx: " << _cur_writer_idx; - uint32_t num_rows_key_group = _segment_writers[_cur_writer_idx]->row_count(); - // init if it's first value column write in current segment - if (_cur_writer_idx == 0 && num_rows_written == 0) { - VLOG_NOTICE << "init first value column segment writer"; - RETURN_IF_ERROR(_segment_writers[_cur_writer_idx]->init(col_ids, is_key)); - } - // when splitting segment, need to make rows align between key columns and value columns - size_t start_offset = 0; - size_t limit = num_rows; - if (num_rows_written + num_rows >= num_rows_key_group && - _cur_writer_idx < _segment_writers.size() - 1) { - RETURN_IF_ERROR(_segment_writers[_cur_writer_idx]->append_block( - block, 0, num_rows_key_group - num_rows_written)); - RETURN_IF_ERROR(_flush_columns(&_segment_writers[_cur_writer_idx])); - start_offset = num_rows_key_group - num_rows_written; - limit = num_rows - start_offset; - ++_cur_writer_idx; - // switch to next writer - RETURN_IF_ERROR(_segment_writers[_cur_writer_idx]->init(col_ids, is_key)); - num_rows_written = 0; - num_rows_key_group = _segment_writers[_cur_writer_idx]->row_count(); - } - if (limit > 0) { - RETURN_IF_ERROR( - _segment_writers[_cur_writer_idx]->append_block(block, start_offset, limit)); - DCHECK(_segment_writers[_cur_writer_idx]->num_rows_written() <= - _segment_writers[_cur_writer_idx]->row_count()); - } - } - if (is_key) { - _num_rows_written += num_rows; - } - return Status::OK(); -} - -Status VerticalBetaRowsetWriter::_flush_columns( - std::unique_ptr* segment_writer, bool is_key) { - uint64_t index_size = 0; - VLOG_NOTICE << "flush columns index: " << _cur_writer_idx; - RETURN_IF_ERROR((*segment_writer)->finalize_columns_data()); - RETURN_IF_ERROR((*segment_writer)->finalize_columns_index(&index_size)); - if (is_key) { - _total_key_group_rows += (*segment_writer)->row_count(); - // record segment key bound - KeyBoundsPB key_bounds; - Slice min_key = (*segment_writer)->min_encoded_key(); - Slice max_key = (*segment_writer)->max_encoded_key(); - DCHECK_LE(min_key.compare(max_key), 0); - key_bounds.set_min_key(min_key.to_string()); - key_bounds.set_max_key(max_key.to_string()); - _segments_encoded_key_bounds.emplace_back(key_bounds); - _segment_num_rows.resize(_cur_writer_idx + 1); - _segment_num_rows[_cur_writer_idx] = _segment_writers[_cur_writer_idx]->row_count(); - } - _total_index_size += - static_cast(index_size) + (*segment_writer)->get_inverted_index_file_size(); - return Status::OK(); + return _helper->add_columns(block, col_ids, is_key, max_rows_per_segment); } Status VerticalBetaRowsetWriter::flush_columns(bool is_key) { - if (_segment_writers.empty()) { - return Status::OK(); - } - - DCHECK(_cur_writer_idx < _segment_writers.size() && _segment_writers[_cur_writer_idx]); - RETURN_IF_ERROR(_flush_columns(&_segment_writers[_cur_writer_idx], is_key)); - _cur_writer_idx = 0; - return Status::OK(); -} - -Status VerticalBetaRowsetWriter::_create_segment_writer( - const std::vector& column_ids, bool is_key, - std::unique_ptr* writer) { - auto path = - BetaRowset::segment_file_path(_context.rowset_dir, _context.rowset_id, _num_segment++); - auto fs = _rowset_meta->fs(); - if (!fs) { - return Status::Error("get fs failed"); - } - io::FileWriterPtr file_writer; - Status st = fs->create_file(path, &file_writer); - if (!st.ok()) { - LOG(WARNING) << "failed to create writable file. path=" << path << ", err: " << st; - return st; - } - - DCHECK(file_writer != nullptr); - segment_v2::SegmentWriterOptions writer_options; - writer_options.enable_unique_key_merge_on_write = _context.enable_unique_key_merge_on_write; - writer_options.rowset_ctx = &_context; - writer->reset(new segment_v2::SegmentWriter( - file_writer.get(), _num_segment, _context.tablet_schema, _context.tablet, - _context.data_dir, _context.max_rows_per_segment, writer_options, nullptr)); - { - std::lock_guard l(_lock); - _file_writers.push_back(std::move(file_writer)); - } - - auto s = (*writer)->init(column_ids, is_key); - if (!s.ok()) { - LOG(WARNING) << "failed to init segment writer: " << s.to_string(); - writer->reset(nullptr); - return s; - } - return Status::OK(); + return _helper->flush_columns(is_key); } Status VerticalBetaRowsetWriter::final_flush() { - for (auto& segment_writer : _segment_writers) { - uint64_t segment_size = 0; - //uint64_t footer_position = 0; - auto st = segment_writer->finalize_footer(&segment_size); - if (!st.ok()) { - LOG(WARNING) << "Fail to finalize segment footer, " << st; - return st; - } - _total_data_size += segment_size + segment_writer->get_inverted_index_file_size(); - segment_writer.reset(); - } - return Status::OK(); + return _helper->final_flush(); } } // namespace doris diff --git a/be/src/olap/rowset/vertical_beta_rowset_writer.h b/be/src/olap/rowset/vertical_beta_rowset_writer.h index 6393726cb0f14e..b2056cc8fba438 100644 --- a/be/src/olap/rowset/vertical_beta_rowset_writer.h +++ b/be/src/olap/rowset/vertical_beta_rowset_writer.h @@ -26,6 +26,7 @@ #include "common/status.h" #include "olap/rowset/beta_rowset_writer.h" #include "olap/rowset/segment_v2/segment_writer.h" +#include "olap/rowset/vertical_beta_rowset_writer_helper.h" namespace doris { namespace vectorized { @@ -50,18 +51,10 @@ class VerticalBetaRowsetWriter final : public BetaRowsetWriter { int64_t num_rows() const override { return _total_key_group_rows; } -private: - // only key group will create segment writer - Status _create_segment_writer(const std::vector& column_ids, bool is_key, - std::unique_ptr* writer); - - Status _flush_columns(std::unique_ptr* segment_writer, - bool is_key = false); - private: std::vector> _segment_writers; - size_t _cur_writer_idx = 0; size_t _total_key_group_rows = 0; + std::shared_ptr _helper; }; } // namespace doris \ No newline at end of file diff --git a/be/src/olap/rowset/vertical_beta_rowset_writer_helper.cpp b/be/src/olap/rowset/vertical_beta_rowset_writer_helper.cpp new file mode 100644 index 00000000000000..e8d03530662545 --- /dev/null +++ b/be/src/olap/rowset/vertical_beta_rowset_writer_helper.cpp @@ -0,0 +1,244 @@ +// 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. + +#include "olap/rowset/vertical_beta_rowset_writer_helper.h" + +#include +#include + +#include +#include +#include +#include +#include +#include + +#include "common/compiler_util.h" // IWYU pragma: keep +#include "common/logging.h" +#include "io/fs/file_system.h" +#include "io/fs/file_writer.h" +#include "olap/rowset/beta_rowset.h" +#include "olap/rowset/rowset_meta.h" +#include "olap/rowset/rowset_writer_context.h" +#include "util/slice.h" +#include "util/spinlock.h" +#include "vec/core/block.h" + +namespace doris { +using namespace ErrorCode; + +VerticalBetaRowsetWriterHelper::VerticalBetaRowsetWriterHelper( + std::vector>* segment_writers, + bool& already_built, RowsetMetaSharedPtr& rowset_meta, std::atomic* num_segment, + RowsetWriterContext& context, std::atomic* num_rows_written, + std::vector* segments_encoded_key_bounds, + std::vector* segment_num_rows, std::atomic* total_index_size, + std::vector* file_writers, std::atomic* total_data_size, + SpinLock* lock) + : _segment_writers(segment_writers), + _already_built(already_built), + _rowset_meta(rowset_meta), + _num_segment(num_segment), + _context(context), + _num_rows_written(num_rows_written), + _segments_encoded_key_bounds(segments_encoded_key_bounds), + _segment_num_rows(segment_num_rows), + _total_index_size(total_index_size), + _file_writers(file_writers), + _total_data_size(total_data_size), + _lock(lock) {} + +void VerticalBetaRowsetWriterHelper::destruct_writer() { + if (!_already_built) { + const auto& fs = _rowset_meta->fs(); + if (!fs || !_rowset_meta->is_local()) { // Remote fs will delete them asynchronously + return; + } + for (auto& segment_writer : *_segment_writers) { + segment_writer.reset(); + } + for (int i = 0; i < *_num_segment; ++i) { + auto path = BetaRowset::segment_file_path(_context.rowset_dir, _context.rowset_id, i); + // Even if an error is encountered, these files that have not been cleaned up + // will be cleaned up by the GC background. So here we only print the error + // message when we encounter an error. + WARN_IF_ERROR(fs->delete_file(path), fmt::format("Failed to delete file={}", path)); + } + } +} + +Status VerticalBetaRowsetWriterHelper::add_columns(const vectorized::Block* block, + const std::vector& col_ids, + bool is_key, uint32_t max_rows_per_segment) { + VLOG_NOTICE << "VerticalBetaRowsetWriter::add_columns, columns: " << block->columns(); + size_t num_rows = block->rows(); + if (num_rows == 0) { + return Status::OK(); + } + if (UNLIKELY(max_rows_per_segment > _context.max_rows_per_segment)) { + max_rows_per_segment = _context.max_rows_per_segment; + } + + if (_segment_writers->empty()) { + // it must be key columns + DCHECK(is_key); + std::unique_ptr writer; + RETURN_IF_ERROR(_create_segment_writer(col_ids, is_key, &writer)); + _segment_writers->emplace_back(std::move(writer)); + _cur_writer_idx = 0; + RETURN_IF_ERROR((*_segment_writers)[_cur_writer_idx]->append_block(block, 0, num_rows)); + } else if (is_key) { + if ((*_segment_writers)[_cur_writer_idx]->num_rows_written() > max_rows_per_segment) { + // segment is full, need flush columns and create new segment writer + RETURN_IF_ERROR(_flush_columns(&(*_segment_writers)[_cur_writer_idx], true)); + + std::unique_ptr writer; + RETURN_IF_ERROR(_create_segment_writer(col_ids, is_key, &writer)); + _segment_writers->emplace_back(std::move(writer)); + ++_cur_writer_idx; + } + RETURN_IF_ERROR((*_segment_writers)[_cur_writer_idx]->append_block(block, 0, num_rows)); + } else { + // value columns + uint32_t num_rows_written = (*_segment_writers)[_cur_writer_idx]->num_rows_written(); + VLOG_NOTICE << "num_rows_written: " << num_rows_written + << ", _cur_writer_idx: " << _cur_writer_idx; + uint32_t num_rows_key_group = (*_segment_writers)[_cur_writer_idx]->row_count(); + // init if it's first value column write in current segment + if (_cur_writer_idx == 0 && num_rows_written == 0) { + VLOG_NOTICE << "init first value column segment writer"; + RETURN_IF_ERROR((*_segment_writers)[_cur_writer_idx]->init(col_ids, is_key)); + } + // when splitting segment, need to make rows align between key columns and value columns + size_t start_offset = 0; + size_t limit = num_rows; + if (num_rows_written + num_rows >= num_rows_key_group && + _cur_writer_idx < _segment_writers->size() - 1) { + RETURN_IF_ERROR((*_segment_writers)[_cur_writer_idx]->append_block( + block, 0, num_rows_key_group - num_rows_written)); + RETURN_IF_ERROR(_flush_columns(&(*_segment_writers)[_cur_writer_idx])); + start_offset = num_rows_key_group - num_rows_written; + limit = num_rows - start_offset; + ++_cur_writer_idx; + // switch to next writer + RETURN_IF_ERROR((*_segment_writers)[_cur_writer_idx]->init(col_ids, is_key)); + num_rows_written = 0; + num_rows_key_group = (*_segment_writers)[_cur_writer_idx]->row_count(); + } + if (limit > 0) { + RETURN_IF_ERROR( + (*_segment_writers)[_cur_writer_idx]->append_block(block, start_offset, limit)); + DCHECK((*_segment_writers)[_cur_writer_idx]->num_rows_written() <= + (*_segment_writers)[_cur_writer_idx]->row_count()); + } + } + if (is_key) { + *_num_rows_written += num_rows; + } + return Status::OK(); +} + +Status VerticalBetaRowsetWriterHelper::_flush_columns( + std::unique_ptr* segment_writer, bool is_key) { + uint64_t index_size = 0; + VLOG_NOTICE << "flush columns index: " << _cur_writer_idx; + RETURN_IF_ERROR((*segment_writer)->finalize_columns_data()); + RETURN_IF_ERROR((*segment_writer)->finalize_columns_index(&index_size)); + if (is_key) { + _total_key_group_rows += (*segment_writer)->row_count(); + // record segment key bound + KeyBoundsPB key_bounds; + Slice min_key = (*segment_writer)->min_encoded_key(); + Slice max_key = (*segment_writer)->max_encoded_key(); + DCHECK_LE(min_key.compare(max_key), 0); + key_bounds.set_min_key(min_key.to_string()); + key_bounds.set_max_key(max_key.to_string()); + _segments_encoded_key_bounds->emplace_back(key_bounds); + _segment_num_rows->resize(_cur_writer_idx + 1); + (*_segment_num_rows)[_cur_writer_idx] = (*_segment_writers)[_cur_writer_idx]->row_count(); + } + *_total_index_size += + static_cast(index_size) + (*segment_writer)->get_inverted_index_file_size(); + return Status::OK(); +} + +Status VerticalBetaRowsetWriterHelper::flush_columns(bool is_key) { + if (_segment_writers->empty()) { + return Status::OK(); + } + + DCHECK(_cur_writer_idx < _segment_writers->size() && (*_segment_writers)[_cur_writer_idx]); + RETURN_IF_ERROR(_flush_columns(&(*_segment_writers)[_cur_writer_idx], is_key)); + _cur_writer_idx = 0; + return Status::OK(); +} + +Status VerticalBetaRowsetWriterHelper::_create_segment_writer( + const std::vector& column_ids, bool is_key, + std::unique_ptr* writer) { + auto path = BetaRowset::segment_file_path(_context.rowset_dir, _context.rowset_id, + (*_num_segment)++); + auto fs = _rowset_meta->fs(); + if (!fs) { + return Status::Error("get fs failed"); + } + io::FileWriterPtr file_writer; + io::FileWriterOptions opts; + opts.create_empty_file = false; + Status st = fs->create_file(path, &file_writer, &opts); + if (!st.ok()) { + LOG(WARNING) << "failed to create writable file. path=" << path << ", err: " << st; + return st; + } + + DCHECK(file_writer != nullptr); + segment_v2::SegmentWriterOptions writer_options; + writer_options.enable_unique_key_merge_on_write = _context.enable_unique_key_merge_on_write; + writer_options.rowset_ctx = &_context; + writer->reset(new segment_v2::SegmentWriter( + file_writer.get(), *_num_segment, _context.tablet_schema, _context.tablet, + _context.data_dir, _context.max_rows_per_segment, writer_options, nullptr)); + { + std::lock_guard l(*_lock); + _file_writers->push_back(std::move(file_writer)); + } + + auto s = (*writer)->init(column_ids, is_key); + if (!s.ok()) { + LOG(WARNING) << "failed to init segment writer: " << s.to_string(); + writer->reset(nullptr); + return s; + } + return Status::OK(); +} + +Status VerticalBetaRowsetWriterHelper::final_flush() { + for (auto& segment_writer : *_segment_writers) { + uint64_t segment_size = 0; + //uint64_t footer_position = 0; + auto st = segment_writer->finalize_footer(&segment_size); + if (!st.ok()) { + LOG(WARNING) << "Fail to finalize segment footer, " << st; + return st; + } + *_total_data_size += segment_size + segment_writer->get_inverted_index_file_size(); + segment_writer.reset(); + } + return Status::OK(); +} + +} // namespace doris diff --git a/be/src/olap/rowset/vertical_beta_rowset_writer_helper.h b/be/src/olap/rowset/vertical_beta_rowset_writer_helper.h new file mode 100644 index 00000000000000..10982bf6b2af47 --- /dev/null +++ b/be/src/olap/rowset/vertical_beta_rowset_writer_helper.h @@ -0,0 +1,83 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include +#include + +#include +#include + +#include "common/status.h" +#include "olap/rowset/beta_rowset_writer.h" +#include "olap/rowset/segment_v2/segment_writer.h" + +namespace doris { +namespace vectorized { +class Block; +} // namespace vectorized + +class VerticalBetaRowsetWriterHelper { +public: + VerticalBetaRowsetWriterHelper( + std::vector>* segment_writers, + bool& already_built, RowsetMetaSharedPtr& rowset_meta, + std::atomic* num_segment, RowsetWriterContext& context, + std::atomic* _num_rows_written, + std::vector* _segments_encoded_key_bounds, + std::vector* _segment_num_rows, std::atomic* _total_index_size, + std::vector* _file_writers, std::atomic* _total_data_size, + SpinLock* _lock); + ~VerticalBetaRowsetWriterHelper() = default; + + Status add_columns(const vectorized::Block* block, const std::vector& col_ids, + bool is_key, uint32_t max_rows_per_segment); + + Status flush_columns(bool is_key); + + Status final_flush(); + + int64_t num_rows() const { return _total_key_group_rows; } + + void destruct_writer(); + +private: + Status _flush_columns(std::unique_ptr* segment_writer, + bool is_key = false); + Status _create_segment_writer(const std::vector& column_ids, bool is_key, + std::unique_ptr* writer); + +private: + std::vector>* _segment_writers; + size_t _cur_writer_idx = 0; + size_t _total_key_group_rows = 0; + + bool& _already_built; + RowsetMetaSharedPtr& _rowset_meta; + std::atomic* _num_segment; + RowsetWriterContext& _context; + std::atomic* _num_rows_written; + std::vector* _segments_encoded_key_bounds; + std::vector* _segment_num_rows; + std::atomic* _total_index_size; + std::vector* _file_writers; + std::atomic* _total_data_size; + SpinLock* _lock; +}; + +} // namespace doris \ No newline at end of file diff --git a/be/src/olap/rowset_builder.cpp b/be/src/olap/rowset_builder.cpp index 62263b8ae3c828..6a6da21ac6219b 100644 --- a/be/src/olap/rowset_builder.cpp +++ b/be/src/olap/rowset_builder.cpp @@ -119,7 +119,7 @@ void RowsetBuilder::_garbage_collection() { } } -Status RowsetBuilder::init_mow_context(std::shared_ptr& mow_context) { +Status BaseRowsetBuilder::init_mow_context(std::shared_ptr& mow_context) { std::lock_guard lck(tablet()->get_header_lock()); int64_t cur_max_version = tablet()->max_version_unlocked(); // tablet is under alter process. The delete bitmap will be calculated after conversion. diff --git a/be/src/olap/rowset_builder.h b/be/src/olap/rowset_builder.h index 5563ece1b389e9..e54faee3435c79 100644 --- a/be/src/olap/rowset_builder.h +++ b/be/src/olap/rowset_builder.h @@ -82,6 +82,8 @@ class BaseRowsetBuilder { return _partial_update_info; } + Status init_mow_context(std::shared_ptr& mow_context); + protected: void _build_current_tablet_schema(int64_t index_id, const OlapTableSchemaParam* table_schema_param, @@ -133,8 +135,6 @@ class RowsetBuilder final : public BaseRowsetBuilder { void _garbage_collection(); - Status init_mow_context(std::shared_ptr& mow_context); - // Cast `BaseTablet` to `Tablet` Tablet* tablet(); TabletSharedPtr tablet_sptr(); diff --git a/be/src/olap/snapshot_manager.cpp b/be/src/olap/snapshot_manager.cpp index c5bf5a785db4b5..71e39e60d0f658 100644 --- a/be/src/olap/snapshot_manager.cpp +++ b/be/src/olap/snapshot_manager.cpp @@ -156,7 +156,7 @@ Result> SnapshotManager::convert_rowset_ids( tablet_schema->init_from_pb(new_tablet_meta_pb.schema()); std::unordered_map rs_version_map; - std::unordered_map rowset_id_mapping; + std::unordered_map rowset_id_mapping; guards.reserve(cloned_tablet_meta_pb.rs_metas_size() + cloned_tablet_meta_pb.stale_rs_metas_size()); for (auto&& visible_rowset : cloned_tablet_meta_pb.rs_metas()) { diff --git a/be/src/olap/storage_engine.h b/be/src/olap/storage_engine.h index 6ebfb9479f4873..b0e9ed6000e827 100644 --- a/be/src/olap/storage_engine.h +++ b/be/src/olap/storage_engine.h @@ -158,7 +158,7 @@ class BaseStorageEngine { // Hold reference of quering rowsets std::mutex _quering_rowsets_mutex; - std::unordered_map _querying_rowsets; + std::unordered_map _querying_rowsets; scoped_refptr _evict_quering_rowset_thread; }; @@ -409,7 +409,7 @@ class StorageEngine final : public BaseStorageEngine { std::atomic_bool _stopped {false}; std::mutex _gc_mutex; - std::unordered_map _unused_rowsets; + std::unordered_map _unused_rowsets; PendingRowsetSet _pending_local_rowsets; PendingRowsetSet _pending_remote_rowsets; diff --git a/be/src/olap/tablet.cpp b/be/src/olap/tablet.cpp index efe63591e445aa..c217505eaaad92 100644 --- a/be/src/olap/tablet.cpp +++ b/be/src/olap/tablet.cpp @@ -146,8 +146,6 @@ using io::FileSystemSPtr; namespace { -bvar::LatencyRecorder g_tablet_update_delete_bitmap_latency("doris_pk", "update_delete_bitmap"); - bvar::Adder exceed_version_limit_counter; bvar::Window> exceed_version_limit_counter_minute( &exceed_version_limit_counter, 60); @@ -169,14 +167,6 @@ void set_last_failure_time(Tablet* tablet, const Compaction& compaction, int64_t } }; -// load segment may do io so it should out lock -Status _load_rowset_segments(const RowsetSharedPtr& rowset, - std::vector* segments) { - auto* beta_rowset = reinterpret_cast(rowset.get()); - RETURN_IF_ERROR(beta_rowset->load_segments(segments)); - return Status::OK(); -} - } // namespace struct WriteCooldownMetaExecutors { @@ -1757,7 +1747,8 @@ Result> Tablet::create_rowset_writer(RowsetWriterC // create a rowset writer with rowset_id and seg_id // after writer, merge this transient rowset with original rowset Result> Tablet::create_transient_rowset_writer( - const Rowset& rowset, std::shared_ptr partial_update_info) { + const Rowset& rowset, std::shared_ptr partial_update_info, + int64_t txn_expiration) { RowsetWriterContext context; context.rowset_state = PREPARED; context.segments_overlap = OVERLAPPING; @@ -2322,7 +2313,7 @@ Status Tablet::update_delete_bitmap_without_lock(const TabletSharedPtr& self, }); int64_t cur_version = rowset->end_version(); std::vector segments; - RETURN_IF_ERROR(_load_rowset_segments(rowset, &segments)); + RETURN_IF_ERROR(std::dynamic_pointer_cast(rowset)->load_segments(&segments)); // If this rowset does not have a segment, there is no need for an update. if (segments.empty()) { @@ -2365,117 +2356,22 @@ Status Tablet::update_delete_bitmap_without_lock(const TabletSharedPtr& self, return Status::OK(); } -Status Tablet::update_delete_bitmap(const TabletSharedPtr& self, const TabletTxnInfo* txn_info, - int64_t txn_id) { - SCOPED_BVAR_LATENCY(g_tablet_update_delete_bitmap_latency); - RowsetIdUnorderedSet cur_rowset_ids; - RowsetIdUnorderedSet rowset_ids_to_add; - RowsetIdUnorderedSet rowset_ids_to_del; +CalcDeleteBitmapExecutor* Tablet::calc_delete_bitmap_executor() { + return _engine.calc_delete_bitmap_executor(); +} + +Status Tablet::save_delete_bitmap(const TabletTxnInfo* txn_info, int64_t txn_id, + DeleteBitmapPtr delete_bitmap, RowsetWriter* rowset_writer, + const RowsetIdUnorderedSet& cur_rowset_ids) { RowsetSharedPtr rowset = txn_info->rowset; int64_t cur_version = rowset->start_version(); - auto rowset_writer = - DORIS_TRY(self->create_transient_rowset_writer(*rowset, txn_info->partial_update_info)); - - DeleteBitmapPtr delete_bitmap = txn_info->delete_bitmap; - // Partial update might generate new segments when there is conflicts while publish, and mark - // the same key in original segments as delete. - // When the new segment flush fails or the rowset build fails, the deletion marker for the - // duplicate key of the original segment should not remain in `txn_info->delete_bitmap`, - // so we need to make a copy of `txn_info->delete_bitmap` and make changes on it. - if (txn_info->partial_update_info && txn_info->partial_update_info->is_partial_update) { - delete_bitmap = std::make_shared(*(txn_info->delete_bitmap)); - } - - OlapStopWatch watch; - std::vector segments; - RETURN_IF_ERROR(_load_rowset_segments(rowset, &segments)); - auto t1 = watch.get_elapse_time_us(); - - { - std::shared_lock meta_rlock(self->_meta_lock); - // tablet is under alter process. The delete bitmap will be calculated after conversion. - if (self->tablet_state() == TABLET_NOTREADY) { - LOG(INFO) << "tablet is under alter process, update delete bitmap later, tablet_id=" - << self->tablet_id(); - return Status::OK(); - } - RETURN_IF_ERROR(self->get_all_rs_id_unlocked(cur_version - 1, &cur_rowset_ids)); - } - auto t2 = watch.get_elapse_time_us(); - - _rowset_ids_difference(cur_rowset_ids, txn_info->rowset_ids, &rowset_ids_to_add, - &rowset_ids_to_del); - for (const auto& to_del : rowset_ids_to_del) { - delete_bitmap->remove({to_del, 0, 0}, {to_del, UINT32_MAX, INT64_MAX}); - } - - std::vector specified_rowsets; - { - std::shared_lock meta_rlock(self->_meta_lock); - specified_rowsets = self->get_rowset_by_ids(&rowset_ids_to_add); - } - auto t3 = watch.get_elapse_time_us(); - - auto token = self->_engine.calc_delete_bitmap_executor()->create_token(); - RETURN_IF_ERROR(calc_delete_bitmap(self, rowset, segments, specified_rowsets, delete_bitmap, - cur_version - 1, token.get(), rowset_writer.get())); - RETURN_IF_ERROR(token->wait()); - - std::stringstream ss; - if (watch.get_elapse_time_us() < 1 * 1000 * 1000) { - ss << "cost: " << watch.get_elapse_time_us() - t3 << "(us)"; - } else { - ss << "cost(us): (load segments: " << t1 << ", get all rsid: " << t2 - t1 - << ", get rowsets: " << t3 - t2 - << ", calc delete bitmap: " << watch.get_elapse_time_us() - t3 << ")"; - } - - size_t total_rows = std::accumulate( - segments.begin(), segments.end(), 0, - [](size_t sum, const segment_v2::SegmentSharedPtr& s) { return sum += s->num_rows(); }); - LOG(INFO) << "[Publish] construct delete bitmap tablet: " << self->tablet_id() - << ", rowset_ids to add: " << rowset_ids_to_add.size() - << ", rowset_ids to del: " << rowset_ids_to_del.size() - << ", cur version: " << cur_version << ", transaction_id: " << txn_id << "," - << ss.str() << " , total rows: " << total_rows; - - if (config::enable_merge_on_write_correctness_check && rowset->num_rows() != 0) { - // only do correctness check if the rowset has at least one row written - // check if all the rowset has ROWSET_SENTINEL_MARK - auto st = self->check_delete_bitmap_correctness(delete_bitmap, cur_version - 1, -1, - cur_rowset_ids, &specified_rowsets); - if (!st.ok()) { - LOG(WARNING) << fmt::format("delete bitmap correctness check failed in publish phase!"); - } - self->_remove_sentinel_mark_from_delete_bitmap(delete_bitmap); - } - - if (txn_info->partial_update_info && txn_info->partial_update_info->is_partial_update) { - DBUG_EXECUTE_IF("Tablet.update_delete_bitmap.partial_update_write_rowset_fail", { - if (rand() % 100 < (100 * dp->param("percent", 0.5))) { - LOG_WARNING("Tablet.update_delete_bitmap.partial_update_write_rowset random failed") - .tag("txn_id", txn_id); - return Status::InternalError( - "debug update_delete_bitmap partial update write rowset random failed"); - } - }); - // build rowset writer and merge transient rowset - RETURN_IF_ERROR(rowset_writer->flush()); - RowsetSharedPtr transient_rowset; - RETURN_IF_ERROR(rowset_writer->build(transient_rowset)); - rowset->merge_rowset_meta(transient_rowset->rowset_meta()); - - // erase segment cache cause we will add a segment to rowset - SegmentLoader::instance()->erase_segments(rowset->rowset_id(), rowset->num_segments()); - } - // update version without write lock, compaction and publish_txn // will update delete bitmap, handle compaction with _rowset_update_lock // and publish_txn runs sequential so no need to lock here for (auto& [key, bitmap] : delete_bitmap->delete_bitmap) { - self->_tablet_meta->delete_bitmap().merge({std::get<0>(key), std::get<1>(key), cur_version}, - bitmap); + _tablet_meta->delete_bitmap().merge({std::get<0>(key), std::get<1>(key), cur_version}, + bitmap); } return Status::OK(); @@ -2536,16 +2432,18 @@ Status Tablet::check_rowid_conversion( return Status::OK(); } std::vector dst_segments; - RETURN_IF_ERROR(_load_rowset_segments(dst_rowset, &dst_segments)); - std::unordered_map, HashOfRowsetId> - input_rowsets_segment; + + RETURN_IF_ERROR( + std::dynamic_pointer_cast(dst_rowset)->load_segments(&dst_segments)); + std::unordered_map> input_rowsets_segment; VLOG_DEBUG << "check_rowid_conversion, dst_segments size: " << dst_segments.size(); for (auto [src_rowset, locations] : location_map) { std::vector& segments = input_rowsets_segment[src_rowset->rowset_id()]; if (segments.empty()) { - RETURN_IF_ERROR(_load_rowset_segments(src_rowset, &segments)); + RETURN_IF_ERROR( + std::dynamic_pointer_cast(src_rowset)->load_segments(&segments)); } for (auto& [src, dst] : locations) { std::string src_key; @@ -2760,81 +2658,4 @@ Status Tablet::ingest_binlog_metas(RowsetBinlogMetasPB* metas_pb) { return RowsetMetaManager::ingest_binlog_metas(_data_dir->get_meta(), tablet_uid(), metas_pb); } -void Tablet::_remove_sentinel_mark_from_delete_bitmap(DeleteBitmapPtr delete_bitmap) { - for (auto it = delete_bitmap->delete_bitmap.begin(), end = delete_bitmap->delete_bitmap.end(); - it != end;) { - if (std::get<1>(it->first) == DeleteBitmap::INVALID_SEGMENT_ID) { - it = delete_bitmap->delete_bitmap.erase(it); - } else { - ++it; - } - } -} - -Status Tablet::check_delete_bitmap_correctness(DeleteBitmapPtr delete_bitmap, int64_t max_version, - int64_t txn_id, - const RowsetIdUnorderedSet& rowset_ids, - std::vector* rowsets) { - RowsetIdUnorderedSet missing_ids; - for (const auto& rowsetid : rowset_ids) { - if (!delete_bitmap->delete_bitmap.contains({rowsetid, DeleteBitmap::INVALID_SEGMENT_ID, - DeleteBitmap::TEMP_VERSION_COMMON})) { - missing_ids.insert(rowsetid); - } - } - - if (!missing_ids.empty()) { - LOG(WARNING) << "[txn_id:" << txn_id << "][tablet_id:" << tablet_id() - << "][max_version: " << max_version - << "] check delete bitmap correctness failed!"; - rapidjson::Document root; - root.SetObject(); - rapidjson::Document required_rowsets_arr; - required_rowsets_arr.SetArray(); - rapidjson::Document missing_rowsets_arr; - missing_rowsets_arr.SetArray(); - - if (rowsets != nullptr) { - for (const auto& rowset : *rowsets) { - rapidjson::Value value; - std::string version_str = rowset->get_rowset_info_str(); - value.SetString(version_str.c_str(), version_str.length(), - required_rowsets_arr.GetAllocator()); - required_rowsets_arr.PushBack(value, required_rowsets_arr.GetAllocator()); - } - } else { - std::vector rowsets; - { - std::shared_lock meta_rlock(_meta_lock); - rowsets = get_rowset_by_ids(&rowset_ids); - } - for (const auto& rowset : rowsets) { - rapidjson::Value value; - std::string version_str = rowset->get_rowset_info_str(); - value.SetString(version_str.c_str(), version_str.length(), - required_rowsets_arr.GetAllocator()); - required_rowsets_arr.PushBack(value, required_rowsets_arr.GetAllocator()); - } - } - for (const auto& missing_rowset_id : missing_ids) { - rapidjson::Value miss_value; - std::string rowset_id_str = missing_rowset_id.to_string(); - miss_value.SetString(rowset_id_str.c_str(), rowset_id_str.length(), - missing_rowsets_arr.GetAllocator()); - missing_rowsets_arr.PushBack(miss_value, missing_rowsets_arr.GetAllocator()); - } - - root.AddMember("required_rowsets", required_rowsets_arr, root.GetAllocator()); - root.AddMember("missing_rowsets", missing_rowsets_arr, root.GetAllocator()); - rapidjson::StringBuffer strbuf; - rapidjson::PrettyWriter writer(strbuf); - root.Accept(writer); - std::string rowset_status_string = std::string(strbuf.GetString()); - LOG_EVERY_SECOND(WARNING) << rowset_status_string; - // let it crash if correctness check failed in Debug mode - DCHECK(false) << "delete bitmap correctness check failed in publish phase!"; - return Status::InternalError("check delete bitmap failed!"); - } - return Status::OK(); -} } // namespace doris diff --git a/be/src/olap/tablet.h b/be/src/olap/tablet.h index aea9dade3109f3..873f22bff7401a 100644 --- a/be/src/olap/tablet.h +++ b/be/src/olap/tablet.h @@ -70,7 +70,6 @@ class TupleDescriptor; class CalcDeleteBitmapToken; enum CompressKind : int; class RowsetBinlogMetasPB; -struct TabletTxnInfo; namespace io { class RemoteFileSystem; @@ -306,7 +305,8 @@ class Tablet final : public BaseTablet { bool vertical) override; Result> create_transient_rowset_writer( - const Rowset& rowset, std::shared_ptr partial_update_info); + const Rowset& rowset, std::shared_ptr partial_update_info, + int64_t txn_expiration = 0) override; Result> create_transient_rowset_writer( RowsetWriterContext& context, const RowsetId& rowset_id); @@ -370,12 +370,14 @@ class Tablet final : public BaseTablet { // end cooldown functions //////////////////////////////////////////////////////////////////////////// - static Status update_delete_bitmap(const TabletSharedPtr& self, const TabletTxnInfo* txn_info, - int64_t txn_id); - static Status update_delete_bitmap_without_lock(const TabletSharedPtr& self, const RowsetSharedPtr& rowset); + CalcDeleteBitmapExecutor* calc_delete_bitmap_executor() override; + Status save_delete_bitmap(const TabletTxnInfo* txn_info, int64_t txn_id, + DeleteBitmapPtr delete_bitmap, RowsetWriter* rowset_writer, + const RowsetIdUnorderedSet& cur_rowset_ids) override; + void calc_compaction_output_rowset_delete_bitmap( const std::vector& input_rowsets, const RowIdConversion& rowid_conversion, uint64_t start_version, uint64_t end_version, @@ -448,9 +450,6 @@ class Tablet final : public BaseTablet { void set_binlog_config(BinlogConfig binlog_config); - Status check_delete_bitmap_correctness(DeleteBitmapPtr delete_bitmap, int64_t max_version, - int64_t txn_id, const RowsetIdUnorderedSet& rowset_ids, - std::vector* rowsets = nullptr); void set_alter_failed(bool alter_failed) { _alter_failed = alter_failed; } bool is_alter_failed() { return _alter_failed; } @@ -488,8 +487,6 @@ class Tablet final : public BaseTablet { // end cooldown functions //////////////////////////////////////////////////////////////////////////// - void _remove_sentinel_mark_from_delete_bitmap(DeleteBitmapPtr delete_bitmap); - public: static const int64_t K_INVALID_CUMULATIVE_POINT = -1; diff --git a/be/src/olap/tablet_fwd.h b/be/src/olap/tablet_fwd.h index c54164c57cb805..3583a28664301c 100644 --- a/be/src/olap/tablet_fwd.h +++ b/be/src/olap/tablet_fwd.h @@ -26,6 +26,8 @@ class Tablet; class TabletSchema; class TabletMeta; class DeleteBitmap; +class CalcDeleteBitmapExecutor; +struct TabletTxnInfo; using BaseTabletSPtr = std::shared_ptr; using TabletSharedPtr = std::shared_ptr; diff --git a/be/src/pipeline/exec/file_scan_operator.cpp b/be/src/pipeline/exec/file_scan_operator.cpp index ac193147dfbd05..9d48fce2552239 100644 --- a/be/src/pipeline/exec/file_scan_operator.cpp +++ b/be/src/pipeline/exec/file_scan_operator.cpp @@ -73,20 +73,53 @@ void FileScanLocalState::set_scan_ranges(RuntimeState* state, _scan_ranges = scan_ranges; } else { // There is no need for the number of scanners to exceed the number of threads in thread pool. - _scan_ranges.clear(); - auto range_iter = scan_ranges.begin(); - for (int i = 0; i < max_scanners && range_iter != scan_ranges.end(); ++i, ++range_iter) { - _scan_ranges.push_back(*range_iter); + _scan_ranges.resize(max_scanners); + std::vector& scan_ranges_ = + const_cast&>(scan_ranges); + auto& first_ranges = scan_ranges_[0].scan_range.ext_scan_range.file_scan_range.ranges; + if (first_ranges[0].__isset.columns_from_path_keys && + !first_ranges[0].columns_from_path_keys.empty()) { + int num_keys = first_ranges[0].columns_from_path_keys.size(); + // In the insert statement, reading data in partition order can reduce the memory usage of BE + // and prevent the generation of smaller tables. + std::sort(scan_ranges_.begin(), scan_ranges_.end(), + [&num_keys](TScanRangeParams r1, TScanRangeParams r2) { + auto& path1 = r1.scan_range.ext_scan_range.file_scan_range.ranges[0] + .columns_from_path; + auto& path2 = r2.scan_range.ext_scan_range.file_scan_range.ranges[0] + .columns_from_path; + for (int i = 0; i < num_keys; ++i) { + if (path1[i] < path2[i]) { + return true; + } + } + return false; + }); } - for (int i = 0; range_iter != scan_ranges.end(); ++i, ++range_iter) { - if (i == max_scanners) { - i = 0; + int num_ranges = scan_ranges.size() / max_scanners; + int num_add_one = scan_ranges.size() - num_ranges * max_scanners; + int scan_index = 0; + int range_index = 0; + for (int i = 0; i < num_add_one; ++i) { + _scan_ranges[scan_index] = scan_ranges_[range_index++]; + auto& ranges = + _scan_ranges[scan_index++].scan_range.ext_scan_range.file_scan_range.ranges; + for (int j = 0; j < num_ranges; j++) { + auto& merged_ranges = scan_ranges_[range_index++] + .scan_range.ext_scan_range.file_scan_range.ranges; + ranges.insert(ranges.end(), merged_ranges.begin(), merged_ranges.end()); + } + } + for (int i = num_add_one; i < max_scanners; ++i) { + _scan_ranges[scan_index] = scan_ranges_[range_index++]; + auto& ranges = + _scan_ranges[scan_index++].scan_range.ext_scan_range.file_scan_range.ranges; + for (int j = 0; j < num_ranges - 1; j++) { + auto& merged_ranges = scan_ranges_[range_index++] + .scan_range.ext_scan_range.file_scan_range.ranges; + ranges.insert(ranges.end(), merged_ranges.begin(), merged_ranges.end()); } - auto& ranges = _scan_ranges[i].scan_range.ext_scan_range.file_scan_range.ranges; - auto& merged_ranges = range_iter->scan_range.ext_scan_range.file_scan_range.ranges; - ranges.insert(ranges.end(), merged_ranges.begin(), merged_ranges.end()); } - _scan_ranges.shrink_to_fit(); LOG(INFO) << "Merge " << scan_ranges.size() << " scan ranges to " << _scan_ranges.size(); } if (scan_ranges.size() > 0 && diff --git a/be/src/pipeline/exec/union_source_operator.cpp b/be/src/pipeline/exec/union_source_operator.cpp index 213b9bc0d3fa56..9e15c2bd9f6f1b 100644 --- a/be/src/pipeline/exec/union_source_operator.cpp +++ b/be/src/pipeline/exec/union_source_operator.cpp @@ -26,6 +26,7 @@ #include "pipeline/exec/union_sink_operator.h" #include "pipeline/pipeline_x/dependency.h" #include "runtime/descriptors.h" +#include "util/defer_op.h" #include "vec/core/block.h" namespace doris { @@ -161,6 +162,13 @@ std::string UnionSourceLocalState::debug_string(int indentation_level) const { Status UnionSourceOperatorX::get_block(RuntimeState* state, vectorized::Block* block, bool* eos) { auto& local_state = get_local_state(state); + Defer set_eos {[&]() { + //have executing const expr, queue have no data anymore, and child could be closed + *eos = (_child_size == 0 && !local_state._need_read_for_const_expr) || + (_child_size > 0 && local_state._shared_state->data_queue.is_all_finish() && + !_has_data(state)); + }}; + SCOPED_TIMER(local_state.exec_time_counter()); if (local_state._need_read_for_const_expr) { if (has_more_const(state)) { @@ -168,7 +176,7 @@ Status UnionSourceOperatorX::get_block(RuntimeState* state, vectorized::Block* b } local_state._need_read_for_const_expr = has_more_const(state); } else if (_child_size != 0) { - std::unique_ptr output_block = vectorized::Block::create_unique(); + std::unique_ptr output_block; int child_idx = 0; RETURN_IF_ERROR(local_state._shared_state->data_queue.get_block_from_queue(&output_block, &child_idx)); @@ -180,11 +188,6 @@ Status UnionSourceOperatorX::get_block(RuntimeState* state, vectorized::Block* b local_state._shared_state->data_queue.push_free_block(std::move(output_block), child_idx); } local_state.reached_limit(block, eos); - //have executing const expr, queue have no data anymore, and child could be closed - *eos = (_child_size == 0 && !local_state._need_read_for_const_expr) || - (_child_size > 0 && local_state._shared_state->data_queue.is_all_finish() && - !_has_data(state)); - return Status::OK(); } diff --git a/be/src/pipeline/exec/union_source_operator.h b/be/src/pipeline/exec/union_source_operator.h index 40d02324cbd1e9..a70d55ab606c67 100644 --- a/be/src/pipeline/exec/union_source_operator.h +++ b/be/src/pipeline/exec/union_source_operator.h @@ -137,14 +137,14 @@ class UnionSourceOperatorX final : public OperatorX { private: bool _has_data(RuntimeState* state) const { - auto& local_state = state->get_local_state(operator_id())->cast(); + auto& local_state = get_local_state(state); if (_child_size == 0) { return local_state._need_read_for_const_expr; } return local_state._shared_state->data_queue.remaining_has_data(); } bool has_more_const(RuntimeState* state) const { - auto& local_state = state->get_local_state(operator_id())->cast(); + auto& local_state = get_local_state(state); return state->per_fragment_instance_idx() == 0 && local_state._const_expr_list_idx < local_state._const_expr_lists.size(); } diff --git a/be/src/pipeline/pipeline_fragment_context.cpp b/be/src/pipeline/pipeline_fragment_context.cpp index 44533e3a5964e9..82b18e8bfa8ae3 100644 --- a/be/src/pipeline/pipeline_fragment_context.cpp +++ b/be/src/pipeline/pipeline_fragment_context.cpp @@ -902,7 +902,7 @@ void PipelineFragmentContext::_close_fragment_instance() { _runtime_state->runtime_profile()->total_time_counter()->update( _fragment_watcher.elapsed_time()); static_cast(send_report(true)); - if (_is_report_success) { + if (_runtime_state->enable_profile()) { std::stringstream ss; // Compute the _local_time_percent before pretty_print the runtime_profile // Before add this operation, the print out like that: diff --git a/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.cpp b/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.cpp index 77a63209cc6326..7362af1c0a03e1 100644 --- a/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.cpp +++ b/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.cpp @@ -1262,7 +1262,7 @@ void PipelineXFragmentContext::_close_fragment_instance() { Defer defer_op {[&]() { _is_fragment_instance_closed = true; }}; _runtime_profile->total_time_counter()->update(_fragment_watcher.elapsed_time()); static_cast(send_report(true)); - if (_is_report_success) { + if (_runtime_state->enable_profile()) { std::stringstream ss; // Compute the _local_time_percent before pretty_print the runtime_profile // Before add this operation, the print out like that: diff --git a/be/src/runtime/exec_env.h b/be/src/runtime/exec_env.h index 171f4400ef2455..895f895701c2ae 100644 --- a/be/src/runtime/exec_env.h +++ b/be/src/runtime/exec_env.h @@ -33,6 +33,7 @@ #include "olap/memtable_memory_limiter.h" #include "olap/olap_define.h" #include "olap/options.h" +#include "olap/rowset/segment_v2/inverted_index_writer.h" #include "olap/tablet_fwd.h" #include "pipeline/pipeline_tracing.h" #include "runtime/frontend_info.h" // TODO(zhiqiang): find a way to remove this include header @@ -58,6 +59,7 @@ class FileCacheFactory; namespace segment_v2 { class InvertedIndexSearcherCache; class InvertedIndexQueryCache; +class TmpFileDirs; } // namespace segment_v2 class WorkloadSchedPolicyMgr; @@ -272,6 +274,8 @@ class ExecEnv { return _pipeline_tracer_ctx.get(); } + segment_v2::TmpFileDirs* get_tmp_file_dirs() { return _tmp_file_dirs.get(); } + private: ExecEnv(); @@ -385,6 +389,7 @@ class ExecEnv { RuntimeQueryStatiticsMgr* _runtime_query_statistics_mgr = nullptr; std::unique_ptr _pipeline_tracer_ctx; + std::unique_ptr _tmp_file_dirs; }; template <> diff --git a/be/src/runtime/exec_env_init.cpp b/be/src/runtime/exec_env_init.cpp index 3098788cff2669..0b441484f01ab2 100644 --- a/be/src/runtime/exec_env_init.cpp +++ b/be/src/runtime/exec_env_init.cpp @@ -151,6 +151,8 @@ Status ExecEnv::_init(const std::vector& store_paths, } init_doris_metrics(store_paths); _store_paths = store_paths; + _tmp_file_dirs = std::make_unique(_store_paths); + RETURN_IF_ERROR(_tmp_file_dirs->init()); _user_function_cache = new UserFunctionCache(); static_cast(_user_function_cache->init(doris::config::user_function_dir)); _external_scan_context_mgr = new ExternalScanContextMgr(this); diff --git a/be/src/runtime/memory/cache_policy.h b/be/src/runtime/memory/cache_policy.h index 6de3de600d819b..538b30099f49b9 100644 --- a/be/src/runtime/memory/cache_policy.h +++ b/be/src/runtime/memory/cache_policy.h @@ -44,6 +44,7 @@ class CachePolicy { TABLET_SCHEMA_CACHE = 14, CREATE_TABLET_RR_IDX_CACHE = 15, CLOUD_TABLET_CACHE = 16, + CLOUD_TXN_DELETE_BITMAP_CACHE = 17, }; static std::string type_string(CacheType type) { @@ -82,6 +83,8 @@ class CachePolicy { return "CreateTabletRRIdxCache"; case CacheType::CLOUD_TABLET_CACHE: return "CloudTabletCache"; + case CacheType::CLOUD_TXN_DELETE_BITMAP_CACHE: + return "CloudTxnDeleteBitmapCache"; default: LOG(FATAL) << "not match type of cache policy :" << static_cast(type); } diff --git a/be/src/runtime/plan_fragment_executor.cpp b/be/src/runtime/plan_fragment_executor.cpp index 0617a89bf010c1..c50b137ba6546f 100644 --- a/be/src/runtime/plan_fragment_executor.cpp +++ b/be/src/runtime/plan_fragment_executor.cpp @@ -624,7 +624,7 @@ void PlanFragmentExecutor::close() { } } - if (_is_report_success) { + if (_runtime_state->enable_profile()) { std::stringstream ss; // Compute the _local_time_percent before pretty_print the runtime_profile // Before add this operation, the print out like that: diff --git a/be/src/runtime/task_group/task_group_manager.cpp b/be/src/runtime/task_group/task_group_manager.cpp index 819e63c855d201..a336cccd3d21a6 100644 --- a/be/src/runtime/task_group/task_group_manager.cpp +++ b/be/src/runtime/task_group/task_group_manager.cpp @@ -72,33 +72,35 @@ TaskGroupPtr TaskGroupManager::get_task_group_by_id(uint64_t tg_id) { void TaskGroupManager::delete_task_group_by_ids(std::set used_wg_id) { int64_t begin_time = MonotonicMillis(); // 1 get delete group without running queries - std::set deleted_tg_ids; + std::vector deleted_task_groups; { std::lock_guard write_lock(_group_mutex); for (auto iter = _task_groups.begin(); iter != _task_groups.end(); iter++) { uint64_t tg_id = iter->first; - auto* task_group_ptr = iter->second.get(); + auto task_group_ptr = iter->second; if (used_wg_id.find(tg_id) == used_wg_id.end()) { task_group_ptr->shutdown(); // only when no query running in task group, its resource can be released in BE if (task_group_ptr->query_num() == 0) { LOG(INFO) << "There is no query in wg " << tg_id << ", delete it."; - deleted_tg_ids.insert(tg_id); + deleted_task_groups.push_back(task_group_ptr); } } } } // 2 stop active thread - for (uint64_t tg_id : deleted_tg_ids) { - _task_groups.at(tg_id)->try_stop_schedulers(); + for (auto& tg : deleted_task_groups) { + // There is not lock here, but the tg may be released by another + // thread, so that we should use shared ptr here, not use tg_id + tg->try_stop_schedulers(); } // 3 release resource in memory { std::lock_guard write_lock(_group_mutex); - for (uint64_t tg_id : deleted_tg_ids) { - _task_groups.erase(tg_id); + for (auto& tg : deleted_task_groups) { + _task_groups.erase(tg->id()); } } @@ -129,7 +131,7 @@ void TaskGroupManager::delete_task_group_by_ids(std::set used_wg_id) { } int64_t time_cost_ms = MonotonicMillis() - begin_time; LOG(INFO) << "finish clear unused task group, time cost: " << time_cost_ms - << "ms, deleted group size:" << deleted_tg_ids.size(); + << "ms, deleted group size:" << deleted_task_groups.size(); } void TaskGroupManager::stop() { diff --git a/be/src/service/http_service.cpp b/be/src/service/http_service.cpp index 84d930d267e27a..c802ff73d44416 100644 --- a/be/src/service/http_service.cpp +++ b/be/src/service/http_service.cpp @@ -23,6 +23,7 @@ #include #include +#include "cloud/cloud_compaction_action.h" #include "cloud/config.h" #include "common/config.h" #include "common/status.h" @@ -31,7 +32,6 @@ #include "http/action/check_rpc_channel_action.h" #include "http/action/check_tablet_segment_action.h" #include "http/action/checksum_action.h" -#include "http/action/cloud_compaction_action.h" #include "http/action/compaction_action.h" #include "http/action/config_action.h" #include "http/action/debug_point_action.h" @@ -337,9 +337,6 @@ void HttpService::register_local_handler(StorageEngine& engine) { } void HttpService::register_cloud_handler(CloudStorageEngine& engine) { - // TODO(plat1ko) - - // 2 compaction actions CloudCompactionAction* show_compaction_action = _pool.add(new CloudCompactionAction(CompactionActionType::SHOW_INFO, _env, engine, TPrivilegeHier::GLOBAL, TPrivilegeType::ADMIN)); @@ -353,7 +350,6 @@ void HttpService::register_cloud_handler(CloudStorageEngine& engine) { CloudCompactionAction* run_status_compaction_action = _pool.add( new CloudCompactionAction(CompactionActionType::RUN_COMPACTION_STATUS, _env, engine, TPrivilegeHier::GLOBAL, TPrivilegeType::ADMIN)); - _ev_http_server->register_handler(HttpMethod::GET, "/api/compaction/run_status", run_status_compaction_action); } diff --git a/be/src/util/doris_metrics.cpp b/be/src/util/doris_metrics.cpp index 0773ed68bd55f3..aceb4a7354cf62 100644 --- a/be/src/util/doris_metrics.cpp +++ b/be/src/util/doris_metrics.cpp @@ -87,10 +87,14 @@ DEFINE_COUNTER_METRIC_PROTOTYPE_5ARG(base_compaction_deltas_total, MetricUnit::R compaction_deltas_total, Labels({{"type", "base"}})); DEFINE_COUNTER_METRIC_PROTOTYPE_5ARG(cumulative_compaction_deltas_total, MetricUnit::ROWSETS, "", compaction_deltas_total, Labels({{"type", "cumulative"}})); +DEFINE_COUNTER_METRIC_PROTOTYPE_5ARG(full_compaction_deltas_total, MetricUnit::ROWSETS, "", + compaction_deltas_total, Labels({{"type", "base"}})); DEFINE_COUNTER_METRIC_PROTOTYPE_5ARG(base_compaction_bytes_total, MetricUnit::BYTES, "", compaction_bytes_total, Labels({{"type", "base"}})); DEFINE_COUNTER_METRIC_PROTOTYPE_5ARG(cumulative_compaction_bytes_total, MetricUnit::BYTES, "", compaction_bytes_total, Labels({{"type", "cumulative"}})); +DEFINE_COUNTER_METRIC_PROTOTYPE_5ARG(full_compaction_bytes_total, MetricUnit::BYTES, "", + compaction_bytes_total, Labels({{"type", "base"}})); DEFINE_COUNTER_METRIC_PROTOTYPE_5ARG(segment_read_total, MetricUnit::OPERATIONS, "(segment_v2) total number of segments read", segment_read, @@ -226,6 +230,8 @@ DorisMetrics::DorisMetrics() : _metric_registry(_s_registry_name) { INT_COUNTER_METRIC_REGISTER(_server_metric_entity, base_compaction_bytes_total); INT_COUNTER_METRIC_REGISTER(_server_metric_entity, cumulative_compaction_deltas_total); INT_COUNTER_METRIC_REGISTER(_server_metric_entity, cumulative_compaction_bytes_total); + INT_COUNTER_METRIC_REGISTER(_server_metric_entity, full_compaction_deltas_total); + INT_COUNTER_METRIC_REGISTER(_server_metric_entity, full_compaction_bytes_total); INT_COUNTER_METRIC_REGISTER(_server_metric_entity, segment_read_total); INT_COUNTER_METRIC_REGISTER(_server_metric_entity, segment_row_total); diff --git a/be/src/util/doris_metrics.h b/be/src/util/doris_metrics.h index 72ee81cd49b7db..e0aa2625cc83d4 100644 --- a/be/src/util/doris_metrics.h +++ b/be/src/util/doris_metrics.h @@ -86,6 +86,8 @@ class DorisMetrics { IntCounter* base_compaction_bytes_total = nullptr; IntCounter* cumulative_compaction_deltas_total = nullptr; IntCounter* cumulative_compaction_bytes_total = nullptr; + IntCounter* full_compaction_deltas_total = nullptr; + IntCounter* full_compaction_bytes_total = nullptr; IntCounter* publish_task_request_total = nullptr; IntCounter* publish_task_failed_total = nullptr; diff --git a/be/src/vec/columns/column_nullable.h b/be/src/vec/columns/column_nullable.h index de01907650e691..8dc4e54073a2da 100644 --- a/be/src/vec/columns/column_nullable.h +++ b/be/src/vec/columns/column_nullable.h @@ -86,7 +86,7 @@ class ColumnNullable final : public COWHelper { const char* get_family_name() const override { return "Nullable"; } std::string get_name() const override { return "Nullable(" + nested_column->get_name() + ")"; } MutableColumnPtr clone_resized(size_t size) const override; - size_t size() const override { return nested_column->size(); } + size_t size() const override { return assert_cast(*null_map).size(); } bool is_null_at(size_t n) const override { return assert_cast(*null_map).get_data()[n] != 0; } diff --git a/be/src/vec/exec/scan/new_file_scan_node.cpp b/be/src/vec/exec/scan/new_file_scan_node.cpp index 2ce80f4463a14f..a0ae03a864747d 100644 --- a/be/src/vec/exec/scan/new_file_scan_node.cpp +++ b/be/src/vec/exec/scan/new_file_scan_node.cpp @@ -71,20 +71,53 @@ void NewFileScanNode::set_scan_ranges(RuntimeState* state, _scan_ranges = scan_ranges; } else { // There is no need for the number of scanners to exceed the number of threads in thread pool. - _scan_ranges.clear(); - auto range_iter = scan_ranges.begin(); - for (int i = 0; i < max_scanners && range_iter != scan_ranges.end(); ++i, ++range_iter) { - _scan_ranges.push_back(*range_iter); + _scan_ranges.resize(max_scanners); + std::vector& scan_ranges_ = + const_cast&>(scan_ranges); + auto& first_ranges = scan_ranges_[0].scan_range.ext_scan_range.file_scan_range.ranges; + if (first_ranges[0].__isset.columns_from_path_keys && + !first_ranges[0].columns_from_path_keys.empty()) { + int num_keys = first_ranges[0].columns_from_path_keys.size(); + // In the insert statement, reading data in partition order can reduce the memory usage of BE + // and prevent the generation of smaller tables. + std::sort(scan_ranges_.begin(), scan_ranges_.end(), + [&num_keys](TScanRangeParams r1, TScanRangeParams r2) { + auto& path1 = r1.scan_range.ext_scan_range.file_scan_range.ranges[0] + .columns_from_path; + auto& path2 = r2.scan_range.ext_scan_range.file_scan_range.ranges[0] + .columns_from_path; + for (int i = 0; i < num_keys; ++i) { + if (path1[i] < path2[i]) { + return true; + } + } + return false; + }); } - for (int i = 0; range_iter != scan_ranges.end(); ++i, ++range_iter) { - if (i == max_scanners) { - i = 0; + int num_ranges = scan_ranges.size() / max_scanners; + int num_add_one = scan_ranges.size() - num_ranges * max_scanners; + int scan_index = 0; + int range_index = 0; + for (int i = 0; i < num_add_one; ++i) { + _scan_ranges[scan_index] = scan_ranges_[range_index++]; + auto& ranges = + _scan_ranges[scan_index++].scan_range.ext_scan_range.file_scan_range.ranges; + for (int j = 0; j < num_ranges; j++) { + auto& merged_ranges = scan_ranges_[range_index++] + .scan_range.ext_scan_range.file_scan_range.ranges; + ranges.insert(ranges.end(), merged_ranges.begin(), merged_ranges.end()); + } + } + for (int i = num_add_one; i < max_scanners; ++i) { + _scan_ranges[scan_index] = scan_ranges_[range_index++]; + auto& ranges = + _scan_ranges[scan_index++].scan_range.ext_scan_range.file_scan_range.ranges; + for (int j = 0; j < num_ranges - 1; j++) { + auto& merged_ranges = scan_ranges_[range_index++] + .scan_range.ext_scan_range.file_scan_range.ranges; + ranges.insert(ranges.end(), merged_ranges.begin(), merged_ranges.end()); } - auto& ranges = _scan_ranges[i].scan_range.ext_scan_range.file_scan_range.ranges; - auto& merged_ranges = range_iter->scan_range.ext_scan_range.file_scan_range.ranges; - ranges.insert(ranges.end(), merged_ranges.begin(), merged_ranges.end()); } - _scan_ranges.shrink_to_fit(); LOG(INFO) << "Merge " << scan_ranges.size() << " scan ranges to " << _scan_ranges.size(); } if (scan_ranges.size() > 0 && diff --git a/cloud/test/meta_server_test.cpp b/cloud/test/meta_server_test.cpp index 3cf6843be19963..af1277bebd606a 100644 --- a/cloud/test/meta_server_test.cpp +++ b/cloud/test/meta_server_test.cpp @@ -47,7 +47,7 @@ using namespace doris; int main(int argc, char** argv) { - const std::string conf_file = "cloud.conf"; + const std::string conf_file = "doris_cloud.conf"; if (!cloud::config::init(conf_file.c_str(), true)) { std::cerr << "failed to init config file, conf=" << conf_file << std::endl; return -1; diff --git a/docker/thirdparties/docker-compose/hive/scripts/create_preinstalled_table.hql b/docker/thirdparties/docker-compose/hive/scripts/create_preinstalled_table.hql index 00d2a699410b92..9baf13020f630e 100644 --- a/docker/thirdparties/docker-compose/hive/scripts/create_preinstalled_table.hql +++ b/docker/thirdparties/docker-compose/hive/scripts/create_preinstalled_table.hql @@ -1937,17 +1937,3 @@ with serdeproperties ,'seperatorChar'=','); insert into employee_gz values ('a', '1.1'), ('b', '2.2'); - -create database hive_schema_change; -use hive_schema_change; - -create table struct_test ( - id int, - sf struct>) stored as parquet; - -insert into struct_test values -(1, named_struct('f1', 1, 'f2', str_to_map('1:s2,2:s2'))), -(2, named_struct('f1', 2, 'f2', str_to_map('k1:s3,k2:s4'))), -(3, named_struct('f1', 3, 'f2', str_to_map('k1:s5,k2:s6'))); - -alter table struct_test change sf sf struct; diff --git a/docs/en/community/release-and-verify/release-verify.md b/docs/en/community/release-and-verify/release-verify.md index af344833976d21..92abf71d1741d1 100644 --- a/docs/en/community/release-and-verify/release-verify.md +++ b/docs/en/community/release-and-verify/release-verify.md @@ -101,11 +101,12 @@ Please see the compilation documentation of each component to verify the compila * Spark Doris Connector, see [compilation documentation](/docs/ecosystem/spark-doris-connector) ## 5. Vote +See the [ASF voting process]((https://www.apache.org/foundation/voting.html)) page for general information about voting. After the verification is completed, the following template can be used to send voting emails to the dev@doris: ``` -Here is my(your github id) vote +1 (binding)" ++1 (binding) or +1 (non-binding) I checked: @@ -120,8 +121,10 @@ I checked: Other comments... ``` -> 1. Indicate the Apache ID in the voting email to facilitate the Release Manager to count the votes. -> -> 2. If you are a Doris PMC member, please indicate (binding) in the voting email. Otherwise indicate (non-binding). -> -> 3. Release Manger himself/herself also needs to explicitly vote. +For a release vote to pass, a minimum of three positive binding votes and more positive binding votes than negative binding votes must be cast. Releases may not be vetoed. Votes cast by PMC members are binding, however, non-binding votes are greatly encouraged and a sign of a healthy project. + +Votes on whether a package is ready to release use majority approval -- at least three PMC members must vote affirmatively for release, and there must be more positive than negative votes. Releases may not be vetoed. For the sake of easy auditing, PMC member votes are typically marked "binding." However, the Release Manager needs to verify the validity of the votes. This can be done by checking whether the email addresses on the PMC roster match the email addresses of the voters. + +Generally the community will cancel the release vote if anyone identifies serious problems, but in most cases the ultimate decision lies with the individual serving as release manager. The specifics of the process may vary from project to project, but the 'minimum quorum of three +1 votes' rule is universal. + +**Note that there is no implicit +1 from the release manager, or from anyone in any ASF vote. Only explicit votes are valid.** The release manager is encouraged to vote on releases, like any reviewer would do. diff --git a/docs/zh-CN/community/release-and-verify/release-verify.md b/docs/zh-CN/community/release-and-verify/release-verify.md index 64823a667f395e..b108da9d17430f 100644 --- a/docs/zh-CN/community/release-and-verify/release-verify.md +++ b/docs/zh-CN/community/release-and-verify/release-verify.md @@ -105,10 +105,12 @@ INFO Totally checked 5611 files, valid: 3926, invalid: 0, ignored: 1685, fixed: ## 5. 投票 +有关投票的具体信息,请参阅 [ASF 投票流程](https://www.apache.org/foundation/voting.html)。 + 验证完成后,可以采用以下模板会发 dev@doris 邮件组中的投票邮件: ``` -Here is my(your github id) vote +1 (binding)" ++1 (binding) or +1 (non-binding) I checked: @@ -122,9 +124,11 @@ I checked: Other comments... ``` +PMC 成员拥有具有约束力的投票,但一般来说,社区鼓励所有成员投票,即使他们的投票只是建议性的。 + +版本发布投票采用多数批准——即至少三名 PMC 成员必须对发布投赞成票,并且赞成票必须多于反对票。PMC 成员的投票是有约束力的,而其他投票则不是。但为了审查方便,PMC 成员投票通常显示指定 “binding”。 +但 Release Manager 需要检查投票的有效性。这可以通过 PMC 的花名册来验证电子邮件地址是否一致。 + +一般来说,如果有人发现严重问题,社区将取消发布投票,但在大多数情况下,最终决定权在于 Release Manager。该流程的具体情况可能因项目而异,但“三+1 票的最低法定人数”规则是通用的。 -> 1. 在投票邮件中注明 Apache ID,以方便 Release Manager 统计票数。 -> -> 2. 如果你是 Doris PMC 成员,则在投票邮件中注明(binding)。否则注明(non-binding)。 -> -> 3. Release Manger 本身也需要显式的进行投票。 +请注意,**Release Manager 或任何 ASF 投票中的任何人都不会隐含 +1。只有明确投票才有效。** 但我们我们鼓励 Release Manager 对版本进行投票。 diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateTableStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateTableStmt.java index d201e06580441b..00ca450ff2e1c5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateTableStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateTableStmt.java @@ -280,11 +280,6 @@ public List getIndexes() { @Override public void analyze(Analyzer analyzer) throws UserException { - if (Config.isCloudMode() && properties != null - && properties.containsKey(PropertyAnalyzer.ENABLE_UNIQUE_KEY_MERGE_ON_WRITE)) { - // FIXME: MOW is not supported in cloud mode yet. - properties.put(PropertyAnalyzer.ENABLE_UNIQUE_KEY_MERGE_ON_WRITE, "false"); - } if (Strings.isNullOrEmpty(engineName) || engineName.equalsIgnoreCase(DEFAULT_ENGINE_NAME)) { this.properties = maybeRewriteByAutoBucket(distributionDesc, properties); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java index 825e55ee09e45b..0c73a19c7d551a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java @@ -57,6 +57,7 @@ import java.util.Optional; import java.util.Set; import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.ReentrantLock; import java.util.stream.Collectors; /** @@ -82,7 +83,10 @@ public abstract class Table extends MetaObject implements Writable, TableIf { @SerializedName(value = "createTime") protected long createTime; protected QueryableReentrantReadWriteLock rwLock; - + // Used for queuing commit transaction tasks to avoid fdb transaction conflicts, + // especially to reduce conflicts when obtaining delete bitmap update locks for + // MoW table + protected ReentrantLock commitLock; /* * fullSchema and nameToColumn should contains all columns, both visible and shadow. * eg. for OlapTable, when doing schema change, there will be some shadow columns which are not visible @@ -131,6 +135,7 @@ public Table(TableType type) { if (Config.check_table_lock_leaky) { this.readLockThreads = Maps.newConcurrentMap(); } + this.commitLock = new ReentrantLock(true); } public Table(long id, String tableName, TableType type, List fullSchema) { @@ -155,6 +160,7 @@ public Table(long id, String tableName, TableType type, List fullSchema) if (Config.check_table_lock_leaky) { this.readLockThreads = Maps.newConcurrentMap(); } + this.commitLock = new ReentrantLock(true); } public void markDropped() { @@ -297,6 +303,28 @@ public boolean tryWriteLockIfExist(long timeout, TimeUnit unit) { return false; } + public void commitLock() { + this.commitLock.lock(); + } + + public boolean tryCommitLock(long timeout, TimeUnit unit) { + try { + boolean res = this.commitLock.tryLock(timeout, unit); + if (!res && unit.toSeconds(timeout) >= 1) { + LOG.warn("Failed to try table {}'s cloud commit lock. timeout {} {}. Current owner: {}", + name, timeout, unit.name(), rwLock.getOwner()); + } + return res; + } catch (InterruptedException e) { + LOG.warn("failed to try cloud commit lock at table[" + name + "]", e); + return false; + } + } + + public void commitUnlock() { + this.commitLock.unlock(); + } + public boolean isTypeRead() { return isTypeRead; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/CloudGlobalTransactionMgr.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/CloudGlobalTransactionMgr.java index 56c3aed2833479..6be3875fef6c65 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/CloudGlobalTransactionMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/CloudGlobalTransactionMgr.java @@ -20,7 +20,11 @@ import org.apache.doris.catalog.Database; import org.apache.doris.catalog.DatabaseIf; import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.OlapTable; +import org.apache.doris.catalog.Partition; import org.apache.doris.catalog.Table; +import org.apache.doris.catalog.TabletInvertedIndex; +import org.apache.doris.catalog.TabletMeta; import org.apache.doris.cloud.proto.Cloud.AbortTxnRequest; import org.apache.doris.cloud.proto.Cloud.AbortTxnResponse; import org.apache.doris.cloud.proto.Cloud.BeginTxnRequest; @@ -33,6 +37,8 @@ import org.apache.doris.cloud.proto.Cloud.CommitTxnResponse; import org.apache.doris.cloud.proto.Cloud.GetCurrentMaxTxnRequest; import org.apache.doris.cloud.proto.Cloud.GetCurrentMaxTxnResponse; +import org.apache.doris.cloud.proto.Cloud.GetDeleteBitmapUpdateLockRequest; +import org.apache.doris.cloud.proto.Cloud.GetDeleteBitmapUpdateLockResponse; import org.apache.doris.cloud.proto.Cloud.GetTxnRequest; import org.apache.doris.cloud.proto.Cloud.GetTxnResponse; import org.apache.doris.cloud.proto.Cloud.LoadJobSourceTypePB; @@ -46,12 +52,15 @@ import org.apache.doris.common.Config; import org.apache.doris.common.DuplicatedRequestException; import org.apache.doris.common.FeNameFormat; +import org.apache.doris.common.InternalErrorCode; import org.apache.doris.common.LabelAlreadyUsedException; +import org.apache.doris.common.MarkedCountDownLatch; import org.apache.doris.common.MetaNotFoundException; import org.apache.doris.common.QuotaExceedException; import org.apache.doris.common.UserException; import org.apache.doris.common.util.DebugUtil; import org.apache.doris.common.util.InternalDatabaseUtil; +import org.apache.doris.common.util.MetaLockUtils; import org.apache.doris.load.loadv2.LoadJobFinalOperation; import org.apache.doris.load.routineload.RLTaskTxnCommitAttachment; import org.apache.doris.metric.MetricRepo; @@ -60,7 +69,14 @@ import org.apache.doris.persist.EditLog; import org.apache.doris.qe.ConnectContext; import org.apache.doris.rpc.RpcException; +import org.apache.doris.task.AgentBatchTask; +import org.apache.doris.task.AgentTaskExecutor; +import org.apache.doris.task.AgentTaskQueue; +import org.apache.doris.task.CalcDeleteBitmapTask; +import org.apache.doris.thrift.TCalcDeleteBitmapPartitionInfo; import org.apache.doris.thrift.TStatus; +import org.apache.doris.thrift.TStatusCode; +import org.apache.doris.thrift.TTaskType; import org.apache.doris.thrift.TUniqueId; import org.apache.doris.thrift.TWaitingTxnStatusRequest; import org.apache.doris.thrift.TWaitingTxnStatusResult; @@ -77,8 +93,14 @@ import org.apache.doris.transaction.TxnStateCallbackFactory; import org.apache.doris.transaction.TxnStateChangeCallback; +import com.google.common.base.Joiner; import com.google.common.base.Preconditions; import com.google.common.base.Strings; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.time.StopWatch; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -88,11 +110,17 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; +import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; public class CloudGlobalTransactionMgr implements GlobalTransactionMgrIface { private static final Logger LOG = LogManager.getLogger(CloudGlobalTransactionMgr.class); private static final String NOT_SUPPORTED_MSG = "Not supported in cloud mode"; + private static final int DELETE_BITMAP_LOCK_EXPIRATION_SECONDS = 10; + private static final int CALCULATE_DELETE_BITMAP_TASK_TIMEOUT_SECONDS = 15; private TxnStateCallbackFactory callbackFactory; @@ -295,6 +323,10 @@ private void commitTransaction(long dbId, List tableList, long transactio "disable_load_job is set to true, all load jobs are not allowed"); } + List mowTableList = getMowTableList(tableList); + if (tabletCommitInfos != null && !tabletCommitInfos.isEmpty() && !mowTableList.isEmpty()) { + calcDeleteBitmapForMow(dbId, mowTableList, transactionId, tabletCommitInfos); + } CommitTxnRequest.Builder builder = CommitTxnRequest.newBuilder(); builder.setDbId(dbId) @@ -302,6 +334,13 @@ private void commitTransaction(long dbId, List
tableList, long transactio .setIs2Pc(is2PC) .setCloudUniqueId(Config.cloud_unique_id); + // if tablet commit info is empty, no need to pass mowTableList to meta service. + if (tabletCommitInfos != null && !tabletCommitInfos.isEmpty()) { + for (OlapTable olapTable : mowTableList) { + builder.addMowTableIds(olapTable.getId()); + } + } + if (txnCommitAttachment != null) { if (txnCommitAttachment instanceof LoadJobFinalOperation) { LoadJobFinalOperation loadJobFinalOperation = (LoadJobFinalOperation) txnCommitAttachment; @@ -350,6 +389,11 @@ private void commitTransaction(long dbId, List
tableList, long transactio && commitTxnResponse.getStatus().getCode() != MetaServiceCode.TXN_ALREADY_VISIBLE) { LOG.warn("commitTxn failed, transactionId:{}, retryTime:{}, commitTxnResponse:{}", transactionId, retryTime, commitTxnResponse); + if (commitTxnResponse.getStatus().getCode() == MetaServiceCode.LOCK_EXPIRED) { + // DELETE_BITMAP_LOCK_ERR will be retried on be + throw new UserException(InternalErrorCode.DELETE_BITMAP_LOCK_ERR, + "delete bitmap update lock expired, transactionId:" + transactionId); + } StringBuilder internalMsgBuilder = new StringBuilder("commitTxn failed, transactionId:"); internalMsgBuilder.append(transactionId); @@ -372,6 +416,228 @@ private void commitTransaction(long dbId, List
tableList, long transactio } } + private List getMowTableList(List
tableList) { + List mowTableList = new ArrayList<>(); + for (Table table : tableList) { + if ((table instanceof OlapTable)) { + OlapTable olapTable = (OlapTable) table; + if (olapTable.getEnableUniqueKeyMergeOnWrite()) { + mowTableList.add(olapTable); + } + } + } + return mowTableList; + } + + private void calcDeleteBitmapForMow(long dbId, List tableList, long transactionId, + List tabletCommitInfos) + throws UserException { + Map>> backendToPartitionTablets = Maps.newHashMap(); + Map partitions = Maps.newHashMap(); + Map> tableToPartitions = Maps.newHashMap(); + getPartitionInfo(tableList, tabletCommitInfos, tableToPartitions, partitions, backendToPartitionTablets); + if (backendToPartitionTablets.isEmpty()) { + throw new UserException("The partition info is empty, table may be dropped, txnid=" + transactionId); + } + + getDeleteBitmapUpdateLock(tableToPartitions, transactionId); + Map partitionVersions = getPartitionVersions(partitions); + + Map> backendToPartitionInfos = getCalcDeleteBitmapInfo( + backendToPartitionTablets, partitionVersions); + sendCalcDeleteBitmaptask(dbId, transactionId, backendToPartitionInfos); + } + + private void getPartitionInfo(List tableList, + List tabletCommitInfos, + Map> tableToParttions, + Map partitions, + Map>> backendToPartitionTablets) { + Map tableMap = Maps.newHashMap(); + for (OlapTable olapTable : tableList) { + tableMap.put(olapTable.getId(), olapTable); + } + + List tabletIds = tabletCommitInfos.stream() + .map(TabletCommitInfo::getTabletId).collect(Collectors.toList()); + TabletInvertedIndex tabletInvertedIndex = Env.getCurrentEnv().getTabletInvertedIndex(); + List tabletMetaList = tabletInvertedIndex.getTabletMetaList(tabletIds); + for (int i = 0; i < tabletMetaList.size(); i++) { + TabletMeta tabletMeta = tabletMetaList.get(i); + long tableId = tabletMeta.getTableId(); + if (!tableMap.containsKey(tableId)) { + continue; + } + + long partitionId = tabletMeta.getPartitionId(); + long backendId = tabletCommitInfos.get(i).getBackendId(); + + if (!tableToParttions.containsKey(tableId)) { + tableToParttions.put(tableId, Sets.newHashSet()); + } + tableToParttions.get(tableId).add(partitionId); + + if (!backendToPartitionTablets.containsKey(backendId)) { + backendToPartitionTablets.put(backendId, Maps.newHashMap()); + } + Map> partitionToTablets = backendToPartitionTablets.get(backendId); + if (!partitionToTablets.containsKey(partitionId)) { + partitionToTablets.put(partitionId, Lists.newArrayList()); + } + partitionToTablets.get(partitionId).add(tabletIds.get(i)); + partitions.putIfAbsent(partitionId, tableMap.get(tableId).getPartition(partitionId)); + } + } + + private Map getPartitionVersions(Map partitionMap) { + Map partitionToVersions = Maps.newHashMap(); + partitionMap.forEach((key, value) -> { + long visibleVersion = value.getVisibleVersion(); + long newVersion = visibleVersion <= 0 ? 2 : visibleVersion + 1; + partitionToVersions.put(key, newVersion); + }); + return partitionToVersions; + } + + private Map> getCalcDeleteBitmapInfo( + Map>> backendToPartitionTablets, Map partitionVersions) { + Map> backendToPartitionInfos = Maps.newHashMap(); + for (Map.Entry>> entry : backendToPartitionTablets.entrySet()) { + List partitionInfos = Lists.newArrayList(); + for (Map.Entry> partitionToTables : entry.getValue().entrySet()) { + Long partitionId = partitionToTables.getKey(); + TCalcDeleteBitmapPartitionInfo partitionInfo = new TCalcDeleteBitmapPartitionInfo(partitionId, + partitionVersions.get(partitionId), + partitionToTables.getValue()); + partitionInfos.add(partitionInfo); + } + backendToPartitionInfos.put(entry.getKey(), partitionInfos); + } + return backendToPartitionInfos; + } + + private void getDeleteBitmapUpdateLock(Map> tableToParttions, long transactionId) + throws UserException { + StopWatch stopWatch = new StopWatch(); + stopWatch.start(); + for (Map.Entry> entry : tableToParttions.entrySet()) { + GetDeleteBitmapUpdateLockRequest.Builder builder = GetDeleteBitmapUpdateLockRequest.newBuilder(); + builder.setTableId(entry.getKey()) + .setLockId(transactionId) + .setInitiator(-1) + .setExpiration(DELETE_BITMAP_LOCK_EXPIRATION_SECONDS); + final GetDeleteBitmapUpdateLockRequest request = builder.build(); + GetDeleteBitmapUpdateLockResponse response = null; + + int retryTime = 0; + while (retryTime++ < Config.meta_service_rpc_retry_times) { + try { + response = MetaServiceProxy.getInstance().getDeleteBitmapUpdateLock(request); + if (LOG.isDebugEnabled()) { + LOG.debug("get delete bitmap lock, transactionId={}, Request: {}, Response: {}", + transactionId, request, response); + } + if (response.getStatus().getCode() != MetaServiceCode.LOCK_CONFLICT + && response.getStatus().getCode() != MetaServiceCode.KV_TXN_CONFLICT) { + break; + } + } catch (Exception e) { + LOG.warn("ignore get delete bitmap lock exception, transactionId={}, retryTime={}", + transactionId, retryTime, e); + } + // sleep random millis [20, 200] ms, avoid txn conflict + int randomMillis = 20 + (int) (Math.random() * (200 - 20)); + if (LOG.isDebugEnabled()) { + LOG.debug("randomMillis:{}", randomMillis); + } + try { + Thread.sleep(randomMillis); + } catch (InterruptedException e) { + LOG.info("InterruptedException: ", e); + } + } + Preconditions.checkNotNull(response); + Preconditions.checkNotNull(response.getStatus()); + if (response.getStatus().getCode() != MetaServiceCode.OK) { + LOG.warn("get delete bitmap lock failed, transactionId={}, for {} times, response:{}", + transactionId, retryTime, response); + if (response.getStatus().getCode() == MetaServiceCode.LOCK_CONFLICT + || response.getStatus().getCode() == MetaServiceCode.KV_TXN_CONFLICT) { + // DELETE_BITMAP_LOCK_ERR will be retried on be + throw new UserException(InternalErrorCode.DELETE_BITMAP_LOCK_ERR, + "Failed to get delete bitmap lock due to confilct"); + } + throw new UserException("Failed to get delete bitmap lock, code: " + response.getStatus().getCode()); + } + } + stopWatch.stop(); + LOG.info("get delete bitmap lock successfully. txns: {}. time cost: {} ms.", + transactionId, stopWatch.getTime()); + } + + private void sendCalcDeleteBitmaptask(long dbId, long transactionId, + Map> backendToPartitionInfos) + throws UserException { + if (backendToPartitionInfos.isEmpty()) { + return; + } + StopWatch stopWatch = new StopWatch(); + stopWatch.start(); + int totalTaskNum = backendToPartitionInfos.size(); + MarkedCountDownLatch countDownLatch = new MarkedCountDownLatch( + totalTaskNum); + AgentBatchTask batchTask = new AgentBatchTask(); + for (Map.Entry> entry : backendToPartitionInfos.entrySet()) { + CalcDeleteBitmapTask task = new CalcDeleteBitmapTask(entry.getKey(), + transactionId, + dbId, + entry.getValue(), + countDownLatch); + countDownLatch.addMark(entry.getKey(), transactionId); + // add to AgentTaskQueue for handling finish report. + // not check return value, because the add will success + AgentTaskQueue.addTask(task); + batchTask.addTask(task); + } + AgentTaskExecutor.submit(batchTask); + + boolean ok; + try { + ok = countDownLatch.await(CALCULATE_DELETE_BITMAP_TASK_TIMEOUT_SECONDS, TimeUnit.SECONDS); + } catch (InterruptedException e) { + LOG.warn("InterruptedException: ", e); + ok = false; + } + + if (!ok || !countDownLatch.getStatus().ok()) { + String errMsg = "Failed to calculate delete bitmap."; + // clear tasks + AgentTaskQueue.removeBatchTask(batchTask, TTaskType.CALCULATE_DELETE_BITMAP); + + if (!countDownLatch.getStatus().ok()) { + errMsg += countDownLatch.getStatus().getErrorMsg(); + if (countDownLatch.getStatus().getErrorCode() != TStatusCode.DELETE_BITMAP_LOCK_ERROR) { + throw new UserException(errMsg); + } + } else { + errMsg += " Timeout."; + List> unfinishedMarks = countDownLatch.getLeftMarks(); + // only show at most 3 results + List> subList = unfinishedMarks.subList(0, + Math.min(unfinishedMarks.size(), 3)); + if (!subList.isEmpty()) { + errMsg += " Unfinished mark: " + Joiner.on(", ").join(subList); + } + } + LOG.warn(errMsg); + // DELETE_BITMAP_LOCK_ERR will be retried on be + throw new UserException(InternalErrorCode.DELETE_BITMAP_LOCK_ERR, errMsg); + } + stopWatch.stop(); + LOG.info("calc delete bitmap task successfully. txns: {}. time cost: {} ms.", + transactionId, stopWatch.getTime()); + } + @Override public boolean commitAndPublishTransaction(DatabaseIf db, List
tableList, long transactionId, List tabletCommitInfos, long timeoutMillis) @@ -383,7 +649,17 @@ public boolean commitAndPublishTransaction(DatabaseIf db, List
tableList, public boolean commitAndPublishTransaction(DatabaseIf db, List
tableList, long transactionId, List tabletCommitInfos, long timeoutMillis, TxnCommitAttachment txnCommitAttachment) throws UserException { - commitTransaction(db.getId(), tableList, transactionId, tabletCommitInfos, txnCommitAttachment); + if (!MetaLockUtils.tryCommitLockTables(tableList, timeoutMillis, TimeUnit.MILLISECONDS)) { + // DELETE_BITMAP_LOCK_ERR will be retried on be + throw new UserException(InternalErrorCode.DELETE_BITMAP_LOCK_ERR, + "get table cloud commit lock timeout, tableList=(" + + StringUtils.join(tableList, ",") + ")"); + } + try { + commitTransaction(db.getId(), tableList, transactionId, tabletCommitInfos, txnCommitAttachment); + } finally { + MetaLockUtils.commitUnlockTables(tableList); + } return true; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/InternalErrorCode.java b/fe/fe-core/src/main/java/org/apache/doris/common/InternalErrorCode.java index 2bbd5c58efa02b..944f8e0486057e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/InternalErrorCode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/InternalErrorCode.java @@ -34,7 +34,10 @@ public enum InternalErrorCode { MANUAL_STOP_ERR(101), TOO_MANY_FAILURE_ROWS_ERR(102), CREATE_TASKS_ERR(103), - TASKS_ABORT_ERR(104); + TASKS_ABORT_ERR(104), + + // for MoW table + DELETE_BITMAP_LOCK_ERR(301); private long errCode; diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/util/MetaLockUtils.java b/fe/fe-core/src/main/java/org/apache/doris/common/util/MetaLockUtils.java index ece62cbc10d0e6..084c6f25f79f67 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/util/MetaLockUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/util/MetaLockUtils.java @@ -18,6 +18,7 @@ package org.apache.doris.common.util; import org.apache.doris.catalog.DatabaseIf; +import org.apache.doris.catalog.Table; import org.apache.doris.catalog.TableIf; import org.apache.doris.common.MetaNotFoundException; @@ -112,4 +113,27 @@ public static void writeUnlockTables(List tableList) { } } + public static void commitLockTables(List
tableList) { + for (Table table : tableList) { + table.commitLock(); + } + } + + public static void commitUnlockTables(List
tableList) { + for (int i = tableList.size() - 1; i >= 0; i--) { + tableList.get(i).commitUnlock(); + } + } + + public static boolean tryCommitLockTables(List
tableList, long timeout, TimeUnit unit) { + for (int i = 0; i < tableList.size(); i++) { + if (!tableList.get(i).tryCommitLock(timeout, unit)) { + for (int j = i - 1; j >= 0; j--) { + tableList.get(j).commitUnlock(); + } + return false; + } + } + return true; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java b/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java index 5625e9626de991..9e07574e57a4a7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java @@ -1439,7 +1439,7 @@ private static Map rewriteReplicaAllocationPropertiesByDatabase( // the user doesn't specify the property in `CreateTableStmt`/`CreateTableInfo` public static Map enableUniqueKeyMergeOnWriteIfNotExists(Map properties) { if (Config.isCloudMode()) { - // FIXME: MOW is not supported in cloud mode yet. + // the default value of enable_unique_key_merge_on_write is false for cloud mode yet. return properties; } if (properties != null && properties.get(PropertyAnalyzer.ENABLE_UNIQUE_KEY_MERGE_ON_WRITE) == null) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalTable.java index 6732abf2a58f43..7f82d0d38768bf 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalTable.java @@ -199,7 +199,7 @@ public long getRowCount() { try { makeSureInitialized(); } catch (Exception e) { - LOG.warn("Failed to initialize table {}.{}.{}", catalog.name, dbName, name, e); + LOG.warn("Failed to initialize table {}.{}.{}", catalog.getName(), dbName, name, e); return 0; } // All external table should get external row count from cache. diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java index bc2ffc40efc29d..95abc9e06c478b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java @@ -3116,6 +3116,7 @@ public void truncateTable(TruncateTableStmt truncateTableStmt) throws DdlExcepti rowsToTruncate += partition.getBaseIndex().getRowCount(); } } else { + rowsToTruncate = olapTable.getRowCount(); for (Partition partition : olapTable.getPartitions()) { // If need absolutely correct, should check running txn here. // But if the txn is in prepare state, cann't known which partitions had load data. @@ -3279,12 +3280,14 @@ public void truncateTable(TruncateTableStmt truncateTableStmt) throws DdlExcepti erasePartitionDropBackendReplicas(oldPartitions); + HashMap updateRecords = new HashMap<>(); + updateRecords.put(olapTable.getId(), rowsToTruncate); if (truncateEntireTable) { // Drop the whole table stats after truncate the entire table Env.getCurrentEnv().getAnalysisManager().dropStats(olapTable); } else { // Update the updated rows in table stats after truncate some partitions. - Env.getCurrentEnv().getAnalysisManager().updateUpdatedRows(olapTable.getId(), rowsToTruncate); + Env.getCurrentEnv().getAnalysisManager().updateUpdatedRows(updateRecords); } LOG.info("finished to truncate table {}, partitions: {}", tblRef.getName().toSql(), tblRef.getPartitionNames()); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java b/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java index 2c3e37ed022886..8f0d2b5458082b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java +++ b/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java @@ -127,7 +127,9 @@ import org.apache.doris.resource.workloadgroup.WorkloadGroup; import org.apache.doris.resource.workloadschedpolicy.WorkloadSchedPolicy; import org.apache.doris.statistics.AnalysisInfo; +import org.apache.doris.statistics.NewPartitionLoadedEvent; import org.apache.doris.statistics.TableStatsMeta; +import org.apache.doris.statistics.UpdateRowsEvent; import org.apache.doris.system.Backend; import org.apache.doris.system.Frontend; import org.apache.doris.transaction.TransactionState; @@ -935,6 +937,16 @@ public void readFields(DataInput in) throws IOException { isRead = true; break; } + case OperationType.OP_LOG_UPDATE_ROWS: { + data = UpdateRowsEvent.read(in); + isRead = true; + break; + } + case OperationType.OP_LOG_NEW_PARTITION_LOADED: { + data = NewPartitionLoadedEvent.read(in); + isRead = true; + break; + } default: { IOException e = new IOException(); LOG.error("UNKNOWN Operation Type {}", opCode, e); diff --git a/fe/fe-core/src/main/java/org/apache/doris/master/MasterImpl.java b/fe/fe-core/src/main/java/org/apache/doris/master/MasterImpl.java index 485463d8daf157..12f1600cd3c224 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/master/MasterImpl.java +++ b/fe/fe-core/src/main/java/org/apache/doris/master/MasterImpl.java @@ -37,6 +37,7 @@ import org.apache.doris.task.AgentTaskQueue; import org.apache.doris.task.AlterInvertedIndexTask; import org.apache.doris.task.AlterReplicaTask; +import org.apache.doris.task.CalcDeleteBitmapTask; import org.apache.doris.task.CheckConsistencyTask; import org.apache.doris.task.ClearAlterTask; import org.apache.doris.task.CloneTask; @@ -137,7 +138,8 @@ public TMasterResult finishTask(TFinishTaskRequest request) { && taskType != TTaskType.DOWNLOAD && taskType != TTaskType.MOVE && taskType != TTaskType.CLONE && taskType != TTaskType.PUBLISH_VERSION && taskType != TTaskType.CREATE && taskType != TTaskType.UPDATE_TABLET_META_INFO - && taskType != TTaskType.STORAGE_MEDIUM_MIGRATE) { + && taskType != TTaskType.STORAGE_MEDIUM_MIGRATE + && taskType != TTaskType.CALCULATE_DELETE_BITMAP) { return result; } } @@ -203,6 +205,9 @@ public TMasterResult finishTask(TFinishTaskRequest request) { case PUSH_COOLDOWN_CONF: finishPushCooldownConfTask(task); break; + case CALCULATE_DELETE_BITMAP: + finishCalcDeleteBitmap(task, request); + break; default: break; } @@ -628,4 +633,28 @@ private void finishPushCooldownConfTask(AgentTask task) { cooldownTask.setFinished(true); AgentTaskQueue.removeTask(task.getBackendId(), TTaskType.PUSH_COOLDOWN_CONF, task.getSignature()); } + + private void finishCalcDeleteBitmap(AgentTask task, TFinishTaskRequest request) { + // if we get here, this task will be removed from AgentTaskQueue for certain. + // because in this function, the only problem that cause failure is meta missing. + // and if meta is missing, we no longer need to resend this task + try { + CalcDeleteBitmapTask calcDeleteBitmapTask = (CalcDeleteBitmapTask) task; + if (request.getTaskStatus().getStatusCode() != TStatusCode.OK) { + calcDeleteBitmapTask.countDownToZero(request.getTaskStatus().getStatusCode(), + "backend: " + task.getBackendId() + ", error_tablet_size: " + + request.getErrorTabletIdsSize() + ", err_msg: " + + request.getTaskStatus().getErrorMsgs().toString()); + } else { + calcDeleteBitmapTask.countDownLatch(task.getBackendId(), calcDeleteBitmapTask.getTransactionId()); + if (LOG.isDebugEnabled()) { + LOG.debug("finish calc delete bitmap. transaction id: {}, be: {}, report version: {}", + calcDeleteBitmapTask.getTransactionId(), calcDeleteBitmapTask.getBackendId(), + request.getReportVersion()); + } + } + } finally { + AgentTaskQueue.removeTask(task.getBackendId(), TTaskType.CALCULATE_DELETE_BITMAP, task.getSignature()); + } + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CheckAfterRewrite.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CheckAfterRewrite.java index b1313a16a19f6e..754d3efa583fa5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CheckAfterRewrite.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CheckAfterRewrite.java @@ -108,16 +108,18 @@ private void checkAllSlotReferenceFromChildren(Plan plan) { notFromChildren = removeValidSlotsNotFromChildren(notFromChildren, childrenOutput); if (!notFromChildren.isEmpty()) { if (plan.arity() != 0 && plan.child(0) instanceof LogicalAggregate) { - throw new AnalysisException(String.format("%s not in agg's output", notFromChildren + throw new AnalysisException(String.format("%s not in aggregate's output", notFromChildren .stream().map(NamedExpression::getName).collect(Collectors.joining(", ")))); } else { throw new AnalysisException(String.format( - "Input slot(s) not in child's output: %s in plan: %s," - + " child output is: %s\n" + "plan tree:\n" + plan.treeString(), + "Input slot(s) not in child's output: %s in plan: %s\nchild output is: %s\nplan tree:\n%s", StringUtils.join(notFromChildren.stream().map(ExpressionTrait::toString) .collect(Collectors.toSet()), ", "), - plan, plan.children().stream().flatMap(child -> child.getOutput().stream()) - .collect(Collectors.toSet()))); + plan, + plan.children().stream() + .flatMap(child -> child.getOutput().stream()) + .collect(Collectors.toSet()), + plan.treeString())); } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CollectJoinConstraint.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CollectJoinConstraint.java index 459118435c4ccf..6948bc61b6824b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CollectJoinConstraint.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CollectJoinConstraint.java @@ -162,6 +162,12 @@ private void collectJoinConstraintList(LeadingHint leading, Long leftHand, Long } } } + if (minLeftHand == 0L) { + minLeftHand = leftHand; + } + if (minRightHand == 0L) { + minRightHand = rightHand; + } JoinConstraint newJoinConstraint = new JoinConstraint(minLeftHand, minRightHand, leftHand, rightHand, join.getJoinType(), isStrict); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewRule.java index 4840416667cff5..a11b910104524d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewRule.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewRule.java @@ -473,7 +473,8 @@ protected SplitPredicate predicatesCompensate( queryStructInfo, viewStructInfo, viewToQuerySlotMapping, - comparisonResult); + comparisonResult, + cascadesContext); // residual compensate final Set residualCompensatePredicates = Predicates.compensateResidualPredicate( queryStructInfo, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtils.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtils.java index baf72cc278e94e..ce12e059b77bf8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtils.java @@ -75,6 +75,9 @@ public static Optional getRelatedTableInfo(String column, Plan break; } } + if (columnExpr == null) { + return Optional.empty(); + } if (!(columnExpr instanceof SlotReference)) { return Optional.empty(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/Predicates.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/Predicates.java index 472e49d3b43a92..139230be5d4b97 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/Predicates.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/Predicates.java @@ -17,8 +17,12 @@ package org.apache.doris.nereids.rules.exploration.mv; +import org.apache.doris.nereids.CascadesContext; import org.apache.doris.nereids.rules.exploration.mv.mapping.EquivalenceClassSetMapping; import org.apache.doris.nereids.rules.exploration.mv.mapping.SlotMapping; +import org.apache.doris.nereids.rules.expression.ExpressionNormalization; +import org.apache.doris.nereids.rules.expression.ExpressionOptimization; +import org.apache.doris.nereids.rules.expression.ExpressionRewriteContext; import org.apache.doris.nereids.trees.expressions.EqualTo; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.SlotReference; @@ -143,8 +147,8 @@ public static Set compensateEquivalence(StructInfo queryStructInfo, public static Set compensateRangePredicate(StructInfo queryStructInfo, StructInfo viewStructInfo, SlotMapping viewToQuerySlotMapping, - ComparisonResult comparisonResult) { - // TODO Range predicates compensate, simplify implementation currently. + ComparisonResult comparisonResult, + CascadesContext cascadesContext) { SplitPredicate querySplitPredicate = queryStructInfo.getSplitPredicate(); SplitPredicate viewSplitPredicate = viewStructInfo.getSplitPredicate(); @@ -153,20 +157,32 @@ public static Set compensateRangePredicate(StructInfo queryStructInf Expression viewRangePredicateQueryBased = ExpressionUtils.replace(viewRangePredicate, viewToQuerySlotMapping.toSlotReferenceMap()); - Set queryRangeSet = - Sets.newHashSet(ExpressionUtils.extractConjunction(queryRangePredicate)); - Set viewRangeQueryBasedSet = - Sets.newHashSet(ExpressionUtils.extractConjunction(viewRangePredicateQueryBased)); - // remove unnecessary literal BooleanLiteral.TRUE - queryRangeSet.remove(BooleanLiteral.TRUE); - viewRangeQueryBasedSet.remove(BooleanLiteral.TRUE); - // query residual predicate can not contain all view residual predicate when view have residual predicate, - // bail out - if (!queryRangeSet.containsAll(viewRangeQueryBasedSet)) { + Set queryRangeSet = ExpressionUtils.extractConjunctionToSet(queryRangePredicate); + Set viewRangeQueryBasedSet = ExpressionUtils.extractConjunctionToSet(viewRangePredicateQueryBased); + Set differentExpressions = new HashSet<>(); + Sets.difference(queryRangeSet, viewRangeQueryBasedSet).copyInto(differentExpressions); + Sets.difference(viewRangeQueryBasedSet, queryRangeSet).copyInto(differentExpressions); + // the range predicate in query and view is same, don't need to compensate + if (differentExpressions.isEmpty()) { + return differentExpressions; + } + // try to normalize the different expressions + Set normalizedExpressions = + normalizeExpression(ExpressionUtils.and(differentExpressions), cascadesContext); + if (!queryRangeSet.containsAll(normalizedExpressions)) { + // normalized expressions is not in query, can not compensate return null; } - queryRangeSet.removeAll(viewRangeQueryBasedSet); - return queryRangeSet; + return normalizedExpressions; + } + + private static Set normalizeExpression(Expression expression, CascadesContext cascadesContext) { + ExpressionNormalization expressionNormalization = new ExpressionNormalization(); + ExpressionOptimization expressionOptimization = new ExpressionOptimization(); + ExpressionRewriteContext context = new ExpressionRewriteContext(cascadesContext); + expression = expressionNormalization.rewrite(expression, context); + expression = expressionOptimization.rewrite(expression, context); + return ExpressionUtils.extractConjunctionToSet(expression); } /** diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/PredicatesSplitter.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/PredicatesSplitter.java index da8f582f64b3b4..f1c0ae8f96a0ce 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/PredicatesSplitter.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/PredicatesSplitter.java @@ -22,6 +22,7 @@ import org.apache.doris.nereids.trees.expressions.ComparisonPredicate; import org.apache.doris.nereids.trees.expressions.EqualPredicate; import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.InPredicate; import org.apache.doris.nereids.trees.expressions.SlotReference; import org.apache.doris.nereids.trees.expressions.literal.Literal; import org.apache.doris.nereids.trees.expressions.visitor.DefaultExpressionVisitor; @@ -81,6 +82,17 @@ public Void visitComparisonPredicate(ComparisonPredicate comparisonPredicate, Vo return null; } + @Override + public Void visitInPredicate(InPredicate inPredicate, Void context) { + if (containOnlyColumnRef(inPredicate.getCompareExpr(), true) + && (ExpressionUtils.isAllLiteral(inPredicate.getOptions()))) { + rangePredicates.add(inPredicate); + } else { + residualPredicates.add(inPredicate); + } + return null; + } + @Override public Void visit(Expression expr, Void context) { residualPredicates.add(expr); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifyRange.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifyRange.java index c0cb834b2f4bfd..13666405bb94f6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifyRange.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifyRange.java @@ -98,8 +98,8 @@ public ValueDesc visit(Expression expr, Void context) { private ValueDesc buildRange(ComparisonPredicate predicate) { Expression rewrite = ExpressionRuleExecutor.normalize(predicate); Expression right = rewrite.child(1); - // only handle `NumericType` - if (right.isLiteral() && right.getDataType().isNumericType()) { + // only handle `NumericType` and `DateLikeType` + if (right.isLiteral() && (right.getDataType().isNumericType() || right.getDataType().isDateLikeType())) { return ValueDesc.range((ComparisonPredicate) rewrite); } return new UnknownValue(predicate); @@ -132,9 +132,10 @@ public ValueDesc visitEqualTo(EqualTo equalTo, Void context) { @Override public ValueDesc visitInPredicate(InPredicate inPredicate, Void context) { - // only handle `NumericType` + // only handle `NumericType` and `DateLikeType` if (ExpressionUtils.isAllLiteral(inPredicate.getOptions()) - && ExpressionUtils.matchNumericType(inPredicate.getOptions())) { + && (ExpressionUtils.matchNumericType(inPredicate.getOptions()) + || ExpressionUtils.matchDateLikeType(inPredicate.getOptions()))) { return ValueDesc.discrete(inPredicate); } return new UnknownValue(inPredicate); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/FilterEstimation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/FilterEstimation.java index f7f8667e706a1b..ad569bffd47a66 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/FilterEstimation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/FilterEstimation.java @@ -33,6 +33,7 @@ import org.apache.doris.nereids.trees.expressions.LessThanEqual; import org.apache.doris.nereids.trees.expressions.Like; import org.apache.doris.nereids.trees.expressions.Not; +import org.apache.doris.nereids.trees.expressions.NullSafeEqual; import org.apache.doris.nereids.trees.expressions.Or; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.SlotReference; @@ -239,7 +240,8 @@ private Statistics calculateWhenBothColumn(ComparisonPredicate cp, EstimationCon Expression left = cp.left(); Expression right = cp.right(); if (cp instanceof EqualPredicate) { - return estimateColumnEqualToColumn(left, statsForLeft, right, statsForRight, context); + return estimateColumnEqualToColumn(left, statsForLeft, right, statsForRight, + cp instanceof NullSafeEqual, context); } if (cp instanceof GreaterThan || cp instanceof GreaterThanEqual) { return estimateColumnLessThanColumn(right, statsForRight, left, statsForLeft, context); @@ -488,7 +490,7 @@ private Statistics estimateBinaryComparisonFilter(Expression leftExpr, ColumnSta } private Statistics estimateColumnEqualToColumn(Expression leftExpr, ColumnStatistic leftStats, - Expression rightExpr, ColumnStatistic rightStats, EstimationContext context) { + Expression rightExpr, ColumnStatistic rightStats, boolean keepNull, EstimationContext context) { StatisticRange leftRange = StatisticRange.from(leftStats, leftExpr.getDataType()); StatisticRange rightRange = StatisticRange.from(rightStats, rightExpr.getDataType()); StatisticRange leftIntersectRight = leftRange.intersect(rightRange); @@ -497,11 +499,16 @@ private Statistics estimateColumnEqualToColumn(Expression leftExpr, ColumnStatis intersectBuilder.setNdv(intersect.getDistinctValues()); intersectBuilder.setMinValue(intersect.getLow()); intersectBuilder.setMaxValue(intersect.getHigh()); - intersectBuilder.setNumNulls(0); + double numNull = 0; + if (keepNull) { + numNull = Math.min(leftStats.numNulls, rightStats.numNulls); + } + intersectBuilder.setNumNulls(numNull); double sel = 1 / StatsMathUtil.nonZeroDivisor(Math.max(leftStats.ndv, rightStats.ndv)); - Statistics updatedStatistics = context.statistics.withSel(sel); + Statistics updatedStatistics = context.statistics.withSel(sel, numNull); updatedStatistics.addColumnStats(leftExpr, intersectBuilder.build()); updatedStatistics.addColumnStats(rightExpr, intersectBuilder.build()); + context.addKeyIfSlot(leftExpr); context.addKeyIfSlot(rightExpr); return updatedStatistics; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateTableInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateTableInfo.java index 3620f2e0413220..400db4fc65f039 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateTableInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateTableInfo.java @@ -225,12 +225,6 @@ public void validate(ConnectContext ctx) { properties = Maps.newHashMap(); } - if (Config.isCloudMode() && properties != null - && properties.containsKey(PropertyAnalyzer.ENABLE_UNIQUE_KEY_MERGE_ON_WRITE)) { - // FIXME: MOW is not supported in cloud mode yet. - properties.put(PropertyAnalyzer.ENABLE_UNIQUE_KEY_MERGE_ON_WRITE, "false"); - } - if (Strings.isNullOrEmpty(engineName) || engineName.equalsIgnoreCase("olap")) { if (distribution == null) { throw new AnalysisException("Create olap table should contain distribution desc"); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java index 8aee620b9f98e8..d59661d21c94d0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java @@ -460,6 +460,10 @@ public static boolean matchNumericType(List children) { return children.stream().allMatch(c -> c.getDataType().isNumericType()); } + public static boolean matchDateLikeType(List children) { + return children.stream().allMatch(c -> c.getDataType().isDateLikeType()); + } + public static boolean hasNullLiteral(List children) { return children.stream().anyMatch(c -> c instanceof NullLiteral); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java b/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java index 88675d5cca3b00..d21012dd658426 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java @@ -88,7 +88,9 @@ import org.apache.doris.statistics.AnalysisJobInfo; import org.apache.doris.statistics.AnalysisManager; import org.apache.doris.statistics.AnalysisTaskInfo; +import org.apache.doris.statistics.NewPartitionLoadedEvent; import org.apache.doris.statistics.TableStatsMeta; +import org.apache.doris.statistics.UpdateRowsEvent; import org.apache.doris.system.Backend; import org.apache.doris.system.Frontend; import org.apache.doris.transaction.TransactionState; @@ -1181,8 +1183,14 @@ public static void loadJournal(Env env, Long logId, JournalEntity journal) { env.getExternalMetaIdMgr().replayMetaIdMappingsLog((MetaIdMappingsLog) journal.getData()); break; } - case OperationType.OP_LOG_UPDATE_ROWS: - case OperationType.OP_LOG_NEW_PARTITION_LOADED: + case OperationType.OP_LOG_UPDATE_ROWS: { + env.getAnalysisManager().replayUpdateRowsRecord((UpdateRowsEvent) journal.getData()); + break; + } + case OperationType.OP_LOG_NEW_PARTITION_LOADED: { + env.getAnalysisManager().replayNewPartitionLoadedEvent((NewPartitionLoadedEvent) journal.getData()); + break; + } case OperationType.OP_LOG_ALTER_COLUMN_STATS: { // TODO: implement this while statistics finished related work. break; @@ -2022,6 +2030,14 @@ public void logCreateTableStats(TableStatsMeta tableStats) { logEdit(OperationType.OP_UPDATE_TABLE_STATS, tableStats); } + public void logUpdateRowsRecord(UpdateRowsEvent record) { + logEdit(OperationType.OP_LOG_UPDATE_ROWS, record); + } + + public void logNewPartitionLoadedEvent(NewPartitionLoadedEvent event) { + logEdit(OperationType.OP_LOG_NEW_PARTITION_LOADED, event); + } + public void logDeleteTableStats(TableStatsDeletionLog log) { logEdit(OperationType.OP_DELETE_TABLE_STATS, log); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/plsql/Console.java b/fe/fe-core/src/main/java/org/apache/doris/plsql/Console.java index 024ea356759483..91cc0988652bdb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/plsql/Console.java +++ b/fe/fe-core/src/main/java/org/apache/doris/plsql/Console.java @@ -20,6 +20,8 @@ package org.apache.doris.plsql; +import org.apache.doris.common.ErrorCode; + public interface Console { void print(String msg); @@ -27,7 +29,7 @@ public interface Console { void printError(String msg); - void flushConsole(); + void printError(ErrorCode errorCode, String msg); Console STANDARD = new Console() { @Override @@ -46,6 +48,8 @@ public void printError(String msg) { } @Override - public void flushConsole() {} + public void printError(ErrorCode errorCode, String msg) { + System.err.println(errorCode.toString() + ", " + msg); + } }; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/plsql/Exec.java b/fe/fe-core/src/main/java/org/apache/doris/plsql/Exec.java index 34e84ab4097ee3..3caf8181ff0af0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/plsql/Exec.java +++ b/fe/fe-core/src/main/java/org/apache/doris/plsql/Exec.java @@ -20,6 +20,7 @@ package org.apache.doris.plsql; +import org.apache.doris.common.ErrorCode; import org.apache.doris.nereids.PLLexer; import org.apache.doris.nereids.PLParser; import org.apache.doris.nereids.PLParser.Allocate_cursor_stmtContext; @@ -150,6 +151,8 @@ import org.antlr.v4.runtime.tree.TerminalNode; import org.apache.commons.io.FileUtils; import org.apache.commons.lang3.exception.ExceptionUtils; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.ByteArrayInputStream; import java.io.Closeable; @@ -176,6 +179,7 @@ * PL/SQL script executor */ public class Exec extends org.apache.doris.nereids.PLParserBaseVisitor implements Closeable { + private static final Logger LOG = LogManager.getLogger(Exec.class); public static final String VERSION = "PL/SQL 0.1"; public static final String ERRORCODE = "ERRORCODE"; @@ -839,6 +843,8 @@ public Integer run(String[] args) throws Exception { init(); try { parseAndEval(arguments); + } catch (Exception e) { + exec.signal(e); } finally { close(); } @@ -901,8 +907,6 @@ private CharStream sourceStream(Arguments arguments) throws IOException { */ public void init() { enterGlobalScope(); - // specify the default log4j2 properties file. - System.setProperty("log4j.configurationFile", "hive-log4j2.properties"); if (conf == null) { conf = new Conf(); } @@ -944,7 +948,7 @@ private ParseTree parse(CharStream input) throws IOException { PLParser parser = newParser(tokens); ParseTree tree = parser.program(); if (trace) { - console.printError("Parser tree: " + tree.toStringTree(parser)); + console.printLine("Parser tree: " + tree.toStringTree(parser)); } return tree; } @@ -978,7 +982,7 @@ private PLParser newParser(CommonTokenStream tokens) { boolean parseArguments(String[] args) { boolean parsed = arguments.parse(args); if (parsed && arguments.hasVersionOption()) { - console.printError(VERSION); + console.printLine(VERSION); return false; } if (!parsed || arguments.hasHelpOption() @@ -1092,13 +1096,24 @@ public void printExceptions() { if (sig.type == Signal.Type.VALIDATION) { error(((PlValidationException) sig.exception).getCtx(), sig.exception.getMessage()); } else if (sig.type == Signal.Type.SQLEXCEPTION) { - console.printError("Unhandled exception in PL/SQL. " + ExceptionUtils.getStackTrace(sig.exception)); + LOG.warn(ExceptionUtils.getStackTrace(sig.exception)); + console.printError(ErrorCode.ERR_SP_BAD_SQLSTATE, + "Unhandled exception in PL/SQL. " + sig.exception.toString()); } else if (sig.type == Signal.Type.UNSUPPORTED_OPERATION) { - console.printError(sig.value == null ? "Unsupported operation" : sig.value); + console.printError(ErrorCode.ERR_SP_BAD_SQLSTATE, + sig.value == null ? "Unsupported operation" : sig.value); + } else if (sig.type == Signal.Type.TOO_MANY_ROWS) { + console.printError(ErrorCode.ERR_SP_BAD_SQLSTATE, + sig.value == null ? "Too many rows exception" : sig.value); + } else if (sig.type == Signal.Type.NOTFOUND) { + console.printError(ErrorCode.ERR_SP_FETCH_NO_DATA, + sig.value == null ? "Not found data exception" : sig.value); } else if (sig.exception != null) { - console.printError("HPL/SQL error: " + ExceptionUtils.getStackTrace(sig.exception)); + LOG.warn(ExceptionUtils.getStackTrace(sig.exception)); + console.printError(ErrorCode.ERR_SP_BAD_SQLSTATE, + "PL/SQL error: " + sig.exception.toString()); } else if (sig.value != null) { - console.printError(sig.value); + console.printError(ErrorCode.ERR_SP_BAD_SQLSTATE, sig.value); } else { trace(null, "Signal: " + sig.type); } @@ -1156,14 +1171,9 @@ public Integer visitStmt(StmtContext ctx) { @Override public Integer visitDoris_statement(Doris_statementContext ctx) { Integer rc = exec.stmt.statement(ctx); - if (rc != 0) { - printExceptions(); - throw new RuntimeException(exec.signalPeek().getValue()); - } // Sometimes the query results are not returned to the mysql client, - // such as declare result; select … into result; + // such as `declare result; select … into result;`, not need finalize. resultListener.onFinalize(); - console.flushConsole(); // if running from plsql.sh return rc; } @@ -1793,6 +1803,8 @@ public Integer visitCall_stmt(Call_stmtContext ctx) { } else if (ctx.multipartIdentifier() != null) { functionCall(ctx, ctx.multipartIdentifier(), null); } + } catch (Exception e) { + exec.signal(e); } finally { exec.inCallStmt = false; } @@ -2379,9 +2391,9 @@ public void info(ParserRuleContext ctx, String message) { return; } if (ctx != null) { - console.printError("Ln:" + ctx.getStart().getLine() + " " + message); + console.printLine("Ln:" + ctx.getStart().getLine() + " " + message); } else { - console.printError(message); + console.printLine(message); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/plsql/Stmt.java b/fe/fe-core/src/main/java/org/apache/doris/plsql/Stmt.java index 1567f020701c16..d86a648b99ad83 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/plsql/Stmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/plsql/Stmt.java @@ -134,11 +134,11 @@ public Integer statement(ParserRuleContext ctx) { exec.setSqlSuccess(); if (query.next()) { exec.setSqlCode(SqlCodes.TOO_MANY_ROWS); - exec.signal(Signal.Type.TOO_MANY_ROWS); + exec.signal(Signal.Type.TOO_MANY_ROWS, "too many rows into variables"); } } else { exec.setSqlCode(SqlCodes.NO_DATA_FOUND); - exec.signal(Signal.Type.NOTFOUND); + exec.signal(Signal.Type.NOTFOUND, "no rows into variables"); } } else if (ctx instanceof Doris_statementContext) { // only from visitStatement // Print all results for standalone Statement. @@ -406,7 +406,7 @@ public Integer fetch(Fetch_stmtContext ctx) { return 1; } else if (exec.getOffline()) { exec.setSqlCode(SqlCodes.NO_DATA_FOUND); - exec.signal(Signal.Type.NOTFOUND); + exec.signal(Signal.Type.NOTFOUND, "fetch not found data"); return 0; } // Assign values from the row to local variables @@ -590,7 +590,7 @@ public Integer assignFromSelect(Assignment_stmt_select_itemContext ctx) { exec.setSqlSuccess(); } else { exec.setSqlCode(SqlCodes.NO_DATA_FOUND); - exec.signal(Signal.Type.NOTFOUND); + exec.signal(Signal.Type.NOTFOUND, "assign from select not found data"); } } catch (QueryException | AnalysisException e) { exec.signal(query); diff --git a/fe/fe-core/src/main/java/org/apache/doris/plsql/executor/PlSqlOperation.java b/fe/fe-core/src/main/java/org/apache/doris/plsql/executor/PlSqlOperation.java index b98507c1a41e46..8c5baf23764f23 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/plsql/executor/PlSqlOperation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/plsql/executor/PlSqlOperation.java @@ -44,30 +44,34 @@ public Exec getExec() { } public void execute(ConnectContext ctx, String statement) { - ctx.setRunProcedure(true); - ctx.setProcedureExec(exec); - result.reset(); try { - Arguments args = new Arguments(); - args.parse(new String[] { "-e", statement }); - exec.parseAndEval(args); - // Exception is not thrown after catch. - // For example, select a not exist table will return empty results, exception - // will put into signals. - exec.printExceptions(); - String error = result.getError(); - String msg = result.getMsg(); - if (!error.isEmpty()) { - ctx.getState().setError("plsql exec error, " + error); - } else if (!msg.isEmpty()) { - ctx.getState().setOk(0, 0, msg); + ctx.setRunProcedure(true); + ctx.setProcedureExec(exec); + result.reset(); + try { + Arguments args = new Arguments(); + args.parse(new String[] {"-e", statement}); + exec.parseAndEval(args); + } catch (Exception e) { + exec.signal(e); + } finally { + // Exception is not thrown after catch. + // For example, select a not exist table will return empty results, exception + // will put into signals. + exec.printExceptions(); + String error = result.getError(); + String msg = result.getMsg(); + if (!error.isEmpty()) { + ctx.getState().setError(result.getLastErrorCode(), error); + } else if (!msg.isEmpty()) { + ctx.getState().setOk(0, 0, msg); + } + ctx.getMysqlChannel().reset(); + ctx.setRunProcedure(false); + ctx.setProcedureExec(null); } - ctx.getMysqlChannel().reset(); - ctx.setRunProcedure(false); - ctx.setProcedureExec(null); } catch (Exception e) { - exec.printExceptions(); - ctx.getState().setError(ErrorCode.ERR_UNKNOWN_ERROR, result.getError() + " " + e.getMessage()); + ctx.getState().setError(ErrorCode.ERR_UNKNOWN_ERROR, e.getMessage()); LOG.warn(e); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/plsql/executor/PlsqlResult.java b/fe/fe-core/src/main/java/org/apache/doris/plsql/executor/PlsqlResult.java index c632b930a8adc8..76e600e7fcc0c7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/plsql/executor/PlsqlResult.java +++ b/fe/fe-core/src/main/java/org/apache/doris/plsql/executor/PlsqlResult.java @@ -17,6 +17,7 @@ package org.apache.doris.plsql.executor; +import org.apache.doris.common.ErrorCode; import org.apache.doris.mysql.MysqlEofPacket; import org.apache.doris.mysql.MysqlSerializer; import org.apache.doris.mysql.MysqlServerStatusFlag; @@ -27,11 +28,13 @@ import org.apache.doris.qe.ConnectProcessor; import org.apache.doris.qe.QueryState; +import com.google.common.collect.Lists; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import java.io.IOException; import java.nio.ByteBuffer; +import java.util.List; // If running from mysql client, first send schema column, // and then send the ByteBuffer through the mysql channel. @@ -44,12 +47,14 @@ public class PlsqlResult implements ResultListener, Console { private Metadata metadata = null; private StringBuilder msg; private StringBuilder error; + private List errorCodes; private boolean isSendFields; public PlsqlResult() { this.msg = new StringBuilder(); this.error = new StringBuilder(); this.isSendFields = false; + this.errorCodes = Lists.newArrayList(); } public void reset() { @@ -58,6 +63,7 @@ public void reset() { isSendFields = false; error.delete(0, error.length()); msg.delete(0, msg.length()); + errorCodes.clear(); } public void setProcessor(ConnectProcessor processor) { @@ -72,6 +78,14 @@ public String getError() { return error.toString(); } + public List getErrorCodes() { + return errorCodes; + } + + public ErrorCode getLastErrorCode() { + return errorCodes.isEmpty() ? ErrorCode.ERR_UNKNOWN_ERROR : errorCodes.get(errorCodes.size() - 1); + } + @Override public void onMysqlRow(ByteBuffer rows) { ConnectContext ctx = processor != null ? processor.getConnectContext() : ConnectContext.get(); @@ -106,6 +120,7 @@ public void onEof() { @Override public void onFinalize() { + // If metadata not null, it means that mysql channel sent query results. if (metadata == null) { return; } @@ -166,24 +181,14 @@ public void printLine(String msg) { @Override public void printError(String msg) { - this.error.append(msg); + errorCodes.add(ErrorCode.ERR_UNKNOWN_ERROR); + this.error.append("\n" + errorCodes.size() + ". " + getLastErrorCode().toString() + ", " + msg); } @Override - public void flushConsole() { - ConnectContext ctx = processor != null ? processor.getConnectContext() : ConnectContext.get(); - boolean needSend = false; - if (error.length() > 0) { - ctx.getState().setError("hplsql exec error, " + error.toString()); - needSend = true; - } else if (msg.length() > 0) { - ctx.getState().setOk(0, 0, msg.toString()); - needSend = true; - } - if (needSend) { - finalizeCommand(); - reset(); - } + public void printError(ErrorCode errorCode, String msg) { + errorCodes.add(errorCode != null ? errorCode : ErrorCode.ERR_UNKNOWN_ERROR); + this.error.append("\n" + errorCodes.size() + ". " + getLastErrorCode().toString() + ", " + msg); } private void finalizeCommand() { diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java index 17e62d40f1f9f6..8c764767c9c442 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java @@ -1827,7 +1827,7 @@ private TNetworkAddress toArrowFlightHost(TNetworkAddress host) throws Exception throw new UserException(SystemInfoService.NO_BACKEND_LOAD_AVAILABLE_MSG); } if (backend.getArrowFlightSqlPort() < 0) { - return null; + throw new UserException("be arrow_flight_sql_port cannot be empty."); } return backend.getArrowFlightAddress(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/MysqlConnectProcessor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/MysqlConnectProcessor.java index 7e9ec551861db6..61c93adbf9680f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/MysqlConnectProcessor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/MysqlConnectProcessor.java @@ -153,7 +153,7 @@ private void handleExecute() { } } catch (Throwable e) { // Catch all throwable. - // If reach here, maybe palo bug. + // If reach here, maybe doris bug. LOG.warn("Process one query failed because unknown reason: ", e); ctx.getState().setError(ErrorCode.ERR_UNKNOWN_ERROR, e.getClass().getSimpleName() + ", msg: " + e.getMessage()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisManager.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisManager.java index eafcff3e7f231f..b265b88f7028fe 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisManager.java @@ -996,21 +996,31 @@ public TableStatsMeta findTableStatsStatus(long tblId) { } // Invoke this when load transaction finished. - public void updateUpdatedRows(long tblId, long rows) { - TableStatsMeta statsStatus = idToTblStats.get(tblId); - if (statsStatus != null) { - statsStatus.updatedRows.addAndGet(rows); - logCreateTableStats(statsStatus); + public void updateUpdatedRows(Map records) { + if (!Env.getCurrentEnv().isMaster() || Env.isCheckpointThread() || records == null || records.isEmpty()) { + return; + } + for (Entry record : records.entrySet()) { + TableStatsMeta statsStatus = idToTblStats.get(record.getKey()); + if (statsStatus != null) { + statsStatus.updatedRows.addAndGet(record.getValue()); + } } + logUpdateRowsRecord(new UpdateRowsEvent(records)); } // Set to true means new partition loaded data - public void setNewPartitionLoaded(long tblId) { - TableStatsMeta statsStatus = idToTblStats.get(tblId); - if (statsStatus != null && Env.getCurrentEnv().isMaster() && !Env.isCheckpointThread()) { - statsStatus.newPartitionLoaded.set(true); - logCreateTableStats(statsStatus); + public void setNewPartitionLoaded(List tableIds) { + if (!Env.getCurrentEnv().isMaster() || Env.isCheckpointThread() || tableIds == null || tableIds.isEmpty()) { + return; + } + for (long tableId : tableIds) { + TableStatsMeta statsStatus = idToTblStats.get(tableId); + if (statsStatus != null) { + statsStatus.newPartitionLoaded.set(true); + } } + logNewPartitionLoadedEvent(new NewPartitionLoadedEvent(tableIds)); } public void updateTableStatsStatus(TableStatsMeta tableStats) { @@ -1026,6 +1036,38 @@ public void logCreateTableStats(TableStatsMeta tableStats) { Env.getCurrentEnv().getEditLog().logCreateTableStats(tableStats); } + public void logUpdateRowsRecord(UpdateRowsEvent record) { + Env.getCurrentEnv().getEditLog().logUpdateRowsRecord(record); + } + + public void logNewPartitionLoadedEvent(NewPartitionLoadedEvent event) { + Env.getCurrentEnv().getEditLog().logNewPartitionLoadedEvent(event); + } + + public void replayUpdateRowsRecord(UpdateRowsEvent event) { + if (event == null || event.getRecords() == null) { + return; + } + for (Entry record : event.getRecords().entrySet()) { + TableStatsMeta statsStatus = idToTblStats.get(record.getKey()); + if (statsStatus != null) { + statsStatus.updatedRows.addAndGet(record.getValue()); + } + } + } + + public void replayNewPartitionLoadedEvent(NewPartitionLoadedEvent event) { + if (event == null || event.getTableIds() == null) { + return; + } + for (long tableId : event.getTableIds()) { + TableStatsMeta statsStatus = idToTblStats.get(tableId); + if (statsStatus != null) { + statsStatus.newPartitionLoaded.set(true); + } + } + } + public void registerSysJob(AnalysisInfo jobInfo, Map taskInfos) { recordAnalysisJob(jobInfo); analysisJobIdToTaskMap.put(jobInfo.jobId, taskInfos); diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/NewPartitionLoadedEvent.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/NewPartitionLoadedEvent.java index d09cb2df6c423a..891eafd2dda863 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/NewPartitionLoadedEvent.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/NewPartitionLoadedEvent.java @@ -27,20 +27,25 @@ import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; -import java.util.HashMap; -import java.util.Map; +import java.util.List; public class NewPartitionLoadedEvent implements Writable { - @SerializedName("partitionIdToTableId") - public final Map partitionIdToTableId = new HashMap<>(); + @SerializedName("tableIds") + private List tableIds; @VisibleForTesting - public NewPartitionLoadedEvent() {} + public NewPartitionLoadedEvent(List tableIds) { + this.tableIds = tableIds; + } // No need to be thread safe, only publish thread will call this. - public void addPartition(long tableId, long partitionId) { - partitionIdToTableId.put(tableId, partitionId); + public void addTableId(long tableId) { + tableIds.add(tableId); + } + + public List getTableIds() { + return tableIds; } @Override @@ -51,7 +56,6 @@ public void write(DataOutput out) throws IOException { public static NewPartitionLoadedEvent read(DataInput dataInput) throws IOException { String json = Text.readString(dataInput); - NewPartitionLoadedEvent newPartitionLoadedEvent = GsonUtils.GSON.fromJson(json, NewPartitionLoadedEvent.class); - return newPartitionLoadedEvent; + return GsonUtils.GSON.fromJson(json, NewPartitionLoadedEvent.class); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/Statistics.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/Statistics.java index 1eb94dcb777be0..b9fa7b5541fcc9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/Statistics.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/Statistics.java @@ -95,11 +95,15 @@ public boolean checkColumnStatsValid(ColumnStatistic columnStatistic) { } public Statistics withSel(double sel) { + return withSel(sel, 0); + } + + public Statistics withSel(double sel, double numNull) { sel = StatsMathUtil.minNonNaN(sel, 1); if (Double.isNaN(rowCount)) { return this; } - double newCount = rowCount * sel; + double newCount = rowCount * sel + numNull; return new Statistics(newCount, widthInJoinCluster, new HashMap<>(expressionToColumnStats)); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/UpdateRowsEvent.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/UpdateRowsEvent.java index 04f185c8b73fc0..8cce3d2939190f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/UpdateRowsEvent.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/UpdateRowsEvent.java @@ -21,30 +21,24 @@ import org.apache.doris.common.io.Writable; import org.apache.doris.persist.gson.GsonUtils; -import com.google.common.annotations.VisibleForTesting; import com.google.gson.annotations.SerializedName; import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; -import java.util.HashMap; import java.util.Map; public class UpdateRowsEvent implements Writable { - @SerializedName("tableIdToUpdateRows") - public final Map tableIdToUpdateRows = new HashMap<>(); + @SerializedName("records") + private Map records; - @VisibleForTesting - public UpdateRowsEvent() {} + public UpdateRowsEvent(Map records) { + this.records = records; + } - // No need to be thread safe, only publish thread will call this. - public void addUpdateRows(long tableId, long rows) { - if (tableIdToUpdateRows.containsKey(tableId)) { - tableIdToUpdateRows.put(tableId, tableIdToUpdateRows.get(tableId) + rows); - } else { - tableIdToUpdateRows.put(tableId, rows); - } + public Map getRecords() { + return records; } @Override @@ -53,9 +47,8 @@ public void write(DataOutput out) throws IOException { Text.writeString(out, json); } - public static UpdateRowsEvent read(DataInput dataInput) throws IOException { - String json = Text.readString(dataInput); - UpdateRowsEvent updateRowsEvent = GsonUtils.GSON.fromJson(json, UpdateRowsEvent.class); - return updateRowsEvent; + public static UpdateRowsEvent read(DataInput in) throws IOException { + String json = Text.readString(in); + return GsonUtils.GSON.fromJson(json, UpdateRowsEvent.class); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/task/AgentBatchTask.java b/fe/fe-core/src/main/java/org/apache/doris/task/AgentBatchTask.java index 1f2e662c757360..1de23992fed6dd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/task/AgentBatchTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/task/AgentBatchTask.java @@ -26,6 +26,7 @@ import org.apache.doris.thrift.TAgentTaskRequest; import org.apache.doris.thrift.TAlterInvertedIndexReq; import org.apache.doris.thrift.TAlterTabletReqV2; +import org.apache.doris.thrift.TCalcDeleteBitmapRequest; import org.apache.doris.thrift.TCheckConsistencyReq; import org.apache.doris.thrift.TClearAlterTaskRequest; import org.apache.doris.thrift.TClearTransactionTaskRequest; @@ -392,6 +393,15 @@ private TAgentTaskRequest toAgentTaskRequest(AgentTask task) { tAgentTaskRequest.setGcBinlogReq(request); return tAgentTaskRequest; } + case CALCULATE_DELETE_BITMAP: { + CalcDeleteBitmapTask calcDeleteBitmapTask = (CalcDeleteBitmapTask) task; + TCalcDeleteBitmapRequest request = calcDeleteBitmapTask.toThrift(); + if (LOG.isDebugEnabled()) { + LOG.debug(request.toString()); + } + tAgentTaskRequest.setCalcDeleteBitmapReq(request); + return tAgentTaskRequest; + } default: if (LOG.isDebugEnabled()) { LOG.debug("could not find task type for task [{}]", task); diff --git a/fe/fe-core/src/main/java/org/apache/doris/task/CalcDeleteBitmapTask.java b/fe/fe-core/src/main/java/org/apache/doris/task/CalcDeleteBitmapTask.java new file mode 100644 index 00000000000000..4188cf61849a91 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/task/CalcDeleteBitmapTask.java @@ -0,0 +1,119 @@ +// 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. + +package org.apache.doris.task; + +import org.apache.doris.common.MarkedCountDownLatch; +import org.apache.doris.common.Status; +import org.apache.doris.thrift.TCalcDeleteBitmapPartitionInfo; +import org.apache.doris.thrift.TCalcDeleteBitmapRequest; +import org.apache.doris.thrift.TStatusCode; +import org.apache.doris.thrift.TTaskType; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.ArrayList; +import java.util.List; + +public class CalcDeleteBitmapTask extends AgentTask { + private static final Logger LOG = LogManager.getLogger(CreateReplicaTask.class); + // used for synchronous process + private MarkedCountDownLatch latch; + private long transactionId; + private List partitionInfos; + private List errorTablets; + + public CalcDeleteBitmapTask(long backendId, long transactionId, long dbId, + List partitionInfos, + MarkedCountDownLatch latch) { + super(null, backendId, TTaskType.CALCULATE_DELETE_BITMAP, dbId, -1L, -1L, -1L, -1L, transactionId); + this.transactionId = transactionId; + this.partitionInfos = partitionInfos; + this.errorTablets = new ArrayList(); + this.isFinished = false; + this.latch = latch; + } + + public void countDownLatch(long backendId, long transactionId) { + if (this.latch != null) { + if (latch.markedCountDown(backendId, transactionId)) { + if (LOG.isDebugEnabled()) { + LOG.debug("CalcDeleteBitmapTask current latch count: {}, backend: {}, transactionId:{}", + latch.getCount(), backendId, transactionId); + } + } + } + } + + // call this always means one of tasks is failed. count down to zero to finish entire task + public void countDownToZero(String errMsg) { + if (this.latch != null) { + latch.countDownToZero(new Status(TStatusCode.CANCELLED, errMsg)); + if (LOG.isDebugEnabled()) { + LOG.debug("CalcDeleteBitmapTask download to zero. error msg: {}", errMsg); + } + } + } + + public void countDownToZero(TStatusCode code, String errMsg) { + if (this.latch != null) { + latch.countDownToZero(new Status(code, errMsg)); + if (LOG.isDebugEnabled()) { + LOG.debug("CalcDeleteBitmapTask download to zero. error msg: {}", errMsg); + } + } + } + + public void setLatch(MarkedCountDownLatch latch) { + this.latch = latch; + } + + public TCalcDeleteBitmapRequest toThrift() { + TCalcDeleteBitmapRequest calcDeleteBitmapRequest = new TCalcDeleteBitmapRequest(transactionId, + partitionInfos); + return calcDeleteBitmapRequest; + } + + public long getTransactionId() { + return transactionId; + } + + public List getCalcDeleteBimapPartitionInfos() { + return partitionInfos; + } + + public synchronized List getErrorTablets() { + return errorTablets; + } + + public synchronized void addErrorTablets(List errorTablets) { + this.errorTablets.clear(); + if (errorTablets == null) { + return; + } + this.errorTablets.addAll(errorTablets); + } + + public void setIsFinished(boolean isFinished) { + this.isFinished = isFinished; + } + + public boolean isFinished() { + return isFinished; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/transaction/DatabaseTransactionMgr.java b/fe/fe-core/src/main/java/org/apache/doris/transaction/DatabaseTransactionMgr.java index 0e5608791e71ee..af94917f97d201 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/transaction/DatabaseTransactionMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/transaction/DatabaseTransactionMgr.java @@ -1912,6 +1912,7 @@ private void updateCatalogAfterCommitted(TransactionState transactionState, Data private boolean updateCatalogAfterVisible(TransactionState transactionState, Database db) { Set errorReplicaIds = transactionState.getErrorReplicas(); AnalysisManager analysisManager = Env.getCurrentEnv().getAnalysisManager(); + List newPartitionLoadedTableIds = new ArrayList<>(); for (TableCommitInfo tableCommitInfo : transactionState.getIdToTableCommitInfos().values()) { long tableId = tableCommitInfo.getTableId(); OlapTable table = (OlapTable) db.getTableNullable(tableId); @@ -1973,7 +1974,7 @@ private boolean updateCatalogAfterVisible(TransactionState transactionState, Dat long versionTime = partitionCommitInfo.getVersionTime(); if (partition.getVisibleVersion() == Partition.PARTITION_INIT_VERSION && version > Partition.PARTITION_INIT_VERSION) { - analysisManager.setNewPartitionLoaded(tableId); + newPartitionLoadedTableIds.add(tableId); } partition.updateVisibleVersionAndTime(version, versionTime); if (LOG.isDebugEnabled()) { @@ -2000,7 +2001,8 @@ private boolean updateCatalogAfterVisible(TransactionState transactionState, Dat if (LOG.isDebugEnabled()) { LOG.debug("table id to loaded rows:{}", tableIdToNumDeltaRows); } - tableIdToNumDeltaRows.forEach(analysisManager::updateUpdatedRows); + analysisManager.setNewPartitionLoaded(newPartitionLoadedTableIds); + analysisManager.updateUpdatedRows(tableIdToNumDeltaRows); return true; } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/SimplifyRangeTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/SimplifyRangeTest.java index 1b56e0d06fe1c6..16476e371464fd 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/SimplifyRangeTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/SimplifyRangeTest.java @@ -158,6 +158,140 @@ public void testSimplify() { } + @Test + public void testSimplifyDate() { + executor = new ExpressionRuleExecutor(ImmutableList.of(SimplifyRange.INSTANCE)); + // assertRewrite("TA", "TA"); + assertRewrite( + "(TA >= date '2024-01-01' and TA <= date '2024-01-03') or (TA > date '2024-01-05' and TA < date '2024-01-07')", + "(TA >= date '2024-01-01' and TA <= date '2024-01-03') or (TA > date '2024-01-05' and TA < date '2024-01-07')"); + assertRewrite( + "(TA > date '2024-01-03' and TA < date '2024-01-01') or (TA > date '2024-01-07'and TA < date '2024-01-05')", + "FALSE"); + assertRewrite("TA > date '2024-01-03' and TA < date '2024-01-01'", "FALSE"); + assertRewrite("TA >= date '2024-01-01' and TA < date '2024-01-01'", + "TA >= date '2024-01-01' and TA < date '2024-01-01'"); + assertRewrite("TA = date '2024-01-01' and TA > date '2024-01-10'", "FALSE"); + assertRewrite("TA > date '2024-01-05' or TA < date '2024-01-01'", + "TA > date '2024-01-05' or TA < date '2024-01-01'"); + assertRewrite("TA > date '2024-01-05' or TA > date '2024-01-01' or TA > date '2024-01-10'", + "TA > date '2024-01-01'"); + assertRewrite("TA > date '2024-01-05' or TA > date '2024-01-01' or TA < date '2024-01-10'", "TA IS NOT NULL"); + assertRewriteNotNull("TA > date '2024-01-05' or TA > date '2024-01-01' or TA < date '2024-01-10'", "TRUE"); + assertRewrite("TA > date '2024-01-05' and TA > date '2024-01-01' and TA > date '2024-01-10'", + "TA > date '2024-01-10'"); + assertRewrite("TA > date '2024-01-05' and TA > date '2024-01-01' and TA < date '2024-01-10'", + "TA > date '2024-01-05' and TA < date '2024-01-10'"); + assertRewrite("TA > date '2024-01-05' or TA < date '2024-01-05'", + "TA > date '2024-01-05' or TA < date '2024-01-05'"); + assertRewrite("TA > date '2024-01-01' or TA < date '2024-01-10'", "TA IS NOT NULL"); + assertRewriteNotNull("TA > date '2024-01-01' or TA < date '2024-01-10'", "TRUE"); + assertRewrite("TA > date '2024-01-05' and TA < date '2024-01-10'", + "TA > date '2024-01-05' and TA < date '2024-01-10'"); + assertRewrite("TA > date '2024-01-05' and TA > date '2024-01-10'", "TA > date '2024-01-10'"); + assertRewrite("(TA > date '2024-01-01' and TA > date '2024-01-10') or TA > date '2024-01-20'", + "TA > date '2024-01-10'"); + assertRewrite("(TA > date '2024-01-01' or TA > date '2024-01-10') and TA > date '2024-01-20'", + "TA > date '2024-01-20'"); + assertRewrite("TA > date '2024-01-05' or TA > date '2024-01-05'", "TA > date '2024-01-05'"); + assertRewrite( + "(TA > date '2024-01-10' or TA > date '2024-01-20') and (TB > date '2024-01-10' and TB < date '2024-01-20')", + "TA > date '2024-01-10' and (TB > date '2024-01-10' and TB < date '2024-01-20') "); + assertRewrite("TA in (date '2024-01-01',date '2024-01-02',date '2024-01-03') and TA > date '2024-01-10'", + "FALSE"); + assertRewrite("TA in (date '2024-01-01',date '2024-01-02',date '2024-01-03') and TA >= date '2024-01-01'", + "TA in (date '2024-01-01',date '2024-01-02',date '2024-01-03')"); + assertRewrite("TA in (date '2024-01-01',date '2024-01-02',date '2024-01-03') and TA > date '2024-01-01'", + "((TA = date '2024-01-02') OR (TA = date '2024-01-03'))"); + assertRewrite("TA in (date '2024-01-01',date '2024-01-02',date '2024-01-03') or TA >= date '2024-01-01'", + "TA >= date '2024-01-01'"); + assertRewrite("TA in (date '2024-01-01')", "TA in (date '2024-01-01')"); + assertRewrite("TA in (date '2024-01-01',date '2024-01-02',date '2024-01-03') and TA < date '2024-01-10'", + "TA in (date '2024-01-01',date '2024-01-02',date '2024-01-03')"); + assertRewrite("TA in (date '2024-01-01',date '2024-01-02',date '2024-01-03') and TA < date '2024-01-01'", + "FALSE"); + assertRewrite("TA in (date '2024-01-01',date '2024-01-02',date '2024-01-03') or TA < date '2024-01-01'", + "TA in (date '2024-01-01',date '2024-01-02',date '2024-01-03') or TA < date '2024-01-01'"); + assertRewrite("TA in (date '2024-01-01',date '2024-01-02') or TA in (date '2024-01-02', date '2024-01-03')", + "TA in (date '2024-01-01',date '2024-01-02',date '2024-01-03')"); + assertRewrite("TA in (date '2024-01-01',date '2024-01-02') and TA in (date '2024-01-03', date '2024-01-04')", + "FALSE"); + assertRewrite("TA = date '2024-01-03' and TA = date '2024-01-01'", "FALSE"); + assertRewrite("TA in (date '2024-01-01') and TA in (date '2024-01-03')", "FALSE"); + assertRewrite("TA in (date '2024-01-03') and TA in (date '2024-01-03')", "TA = date '2024-01-03'"); + assertRewrite("(TA > date '2024-01-03' and TA < date '2024-01-01') and TB < date '2024-01-05'", "FALSE"); + assertRewrite("(TA > date '2024-01-03' and TA < date '2024-01-01') or TB < date '2024-01-05'", + "TB < date '2024-01-05'"); + } + + @Test + public void testSimplifyDateTime() { + executor = new ExpressionRuleExecutor(ImmutableList.of(SimplifyRange.INSTANCE)); + // assertRewrite("TA", "TA"); + assertRewrite( + "(TA >= timestamp '2024-01-01 00:00:00' and TA <= timestamp '2024-01-03 00:00:00') or (TA > timestamp '2024-01-05 00:00:00' and TA < timestamp '2024-01-07 00:00:00')", + "(TA >= timestamp '2024-01-01 00:00:00' and TA <= timestamp '2024-01-03 00:00:00') or (TA > timestamp '2024-01-05 00:00:00' and TA < timestamp '2024-01-07 00:00:00')"); + assertRewrite( + "(TA > timestamp '2024-01-03 00:00:10' and TA < timestamp '2024-01-01 00:00:10') or (TA > timestamp '2024-01-07 00:00:10'and TA < timestamp '2024-01-05 00:00:10')", + "FALSE"); + assertRewrite("TA > timestamp '2024-01-03 00:00:10' and TA < timestamp '2024-01-01 01:00:00'", "FALSE"); + assertRewrite("TA >= timestamp '2024-01-01 00:00:10' and TA < timestamp '2024-01-01 00:00:10'", + "TA >= timestamp '2024-01-01 00:00:10' and TA < timestamp '2024-01-01 00:00:10'"); + assertRewrite("TA = timestamp '2024-01-01 10:00:10' and TA > timestamp '2024-01-10 00:00:10'", "FALSE"); + assertRewrite("TA > timestamp '2024-01-05 00:00:10' or TA < timestamp '2024-01-01 00:00:10'", + "TA > timestamp '2024-01-05 00:00:10' or TA < timestamp '2024-01-01 00:00:10'"); + assertRewrite("TA > timestamp '2024-01-05 00:00:10' or TA > timestamp '2024-01-01 00:00:10' or TA > timestamp '2024-01-10 00:00:10'", + "TA > timestamp '2024-01-01 00:00:10'"); + assertRewrite("TA > timestamp '2024-01-05 00:00:10' or TA > timestamp '2024-01-01 00:00:10' or TA < timestamp '2024-01-10 00:00:10'", "TA IS NOT NULL"); + assertRewriteNotNull("TA > timestamp '2024-01-05 00:00:10' or TA > timestamp '2024-01-01 00:00:10' or TA < timestamp '2024-01-10 00:00:10'", "TRUE"); + assertRewrite("TA > timestamp '2024-01-05 00:00:10' and TA > timestamp '2024-01-01 00:00:10' and TA > timestamp '2024-01-10 00:00:15'", + "TA > timestamp '2024-01-10 00:00:15'"); + assertRewrite("TA > timestamp '2024-01-05 00:00:10' and TA > timestamp '2024-01-01 00:00:10' and TA < timestamp '2024-01-10 00:00:10'", + "TA > timestamp '2024-01-05 00:00:10' and TA < timestamp '2024-01-10 00:00:10'"); + assertRewrite("TA > timestamp '2024-01-05 00:00:10' or TA < timestamp '2024-01-05 00:00:10'", + "TA > timestamp '2024-01-05 00:00:10' or TA < timestamp '2024-01-05 00:00:10'"); + assertRewrite("TA > timestamp '2024-01-01 00:02:10' or TA < timestamp '2024-01-10 00:02:10'", "TA IS NOT NULL"); + assertRewriteNotNull("TA > timestamp '2024-01-01 00:00:00' or TA < timestamp '2024-01-10 00:00:00'", "TRUE"); + assertRewrite("TA > timestamp '2024-01-05 01:00:00' and TA < timestamp '2024-01-10 01:00:00'", + "TA > timestamp '2024-01-05 01:00:00' and TA < timestamp '2024-01-10 01:00:00'"); + assertRewrite("TA > timestamp '2024-01-05 01:00:00' and TA > timestamp '2024-01-10 01:00:00'", "TA > timestamp '2024-01-10 01:00:00'"); + assertRewrite("(TA > timestamp '2024-01-01 01:00:00' and TA > timestamp '2024-01-10 01:00:00') or TA > timestamp '2024-01-20 01:00:00'", + "TA > timestamp '2024-01-10 01:00:00'"); + assertRewrite("(TA > timestamp '2024-01-01 01:00:00' or TA > timestamp '2024-01-10 01:00:00') and TA > timestamp '2024-01-20 01:00:00'", + "TA > timestamp '2024-01-20 01:00:00'"); + assertRewrite("TA > timestamp '2024-01-05 01:00:00' or TA > timestamp '2024-01-05 01:00:00'", "TA > timestamp '2024-01-05 01:00:00'"); + assertRewrite( + "(TA > timestamp '2024-01-10 01:00:00' or TA > timestamp '2024-01-20 01:00:00') and (TB > timestamp '2024-01-10 01:00:00' and TB < timestamp '2024-01-20 01:00:00')", + "TA > timestamp '2024-01-10 01:00:00' and (TB > timestamp '2024-01-10 01:00:00' and TB < timestamp '2024-01-20 01:00:00') "); + assertRewrite("TA in (timestamp '2024-01-01 01:00:00',timestamp '2024-01-02 02:00:00',timestamp '2024-01-03 03:00:00') and TA > timestamp '2024-01-10 01:00:00'", + "FALSE"); + assertRewrite("TA in (timestamp '2024-01-01 01:00:00',timestamp '2024-01-02 01:50:00',timestamp '2024-01-03 02:00:00') and TA >= timestamp '2024-01-01'", + "TA in (timestamp '2024-01-01 01:00:00',timestamp '2024-01-02 01:50:00',timestamp '2024-01-03 02:00:00')"); + assertRewrite("TA in (timestamp '2024-01-01 02:00:00',timestamp '2024-01-02 02:00:00',timestamp '2024-01-03 02:00:00') and TA > timestamp '2024-01-01 02:10:00'", + "((TA = timestamp '2024-01-02 02:00:00') OR (TA = timestamp '2024-01-03 02:00:00'))"); + assertRewrite("TA in (timestamp '2024-01-01 02:00:00',timestamp '2024-01-02 02:00:00',timestamp '2024-01-03 02:00:00') or TA >= timestamp '2024-01-01 01:00:00'", + "TA >= timestamp '2024-01-01 01:00:00'"); + assertRewrite("TA in (timestamp '2024-01-01 02:00:00')", "TA in (timestamp '2024-01-01 02:00:00')"); + assertRewrite("TA in (timestamp '2024-01-01 02:00:00',timestamp '2024-01-02 02:00:00',timestamp '2024-01-03 02:00:00') and TA < timestamp '2024-01-10 02:00:00'", + "TA in (timestamp '2024-01-01 02:00:00',timestamp '2024-01-02 02:00:00',timestamp '2024-01-03 02:00:00')"); + assertRewrite("TA in (timestamp '2024-01-01 02:00:00',timestamp '2024-01-02 02:00:00',timestamp '2024-01-03 02:00:00') and TA < timestamp '2024-01-01 02:00:00'", + "FALSE"); + assertRewrite("TA in (timestamp '2024-01-01 02:00:00',timestamp '2024-01-02 02:00:00',timestamp '2024-01-03 02:00:00') and TA < timestamp '2024-01-01 02:00:01'", + "TA = timestamp '2024-01-01 02:00:00'"); + assertRewrite("TA in (timestamp '2024-01-01 02:00:00',timestamp '2024-01-02 02:00:00',timestamp '2024-01-03 02:00:00') or TA < timestamp '2024-01-01 01:00:00'", + "TA in (timestamp '2024-01-01 02:00:00',timestamp '2024-01-02 02:00:00',timestamp '2024-01-03 02:00:00') or TA < timestamp '2024-01-01 01:00:00'"); + assertRewrite("TA in (timestamp '2024-01-01 00:00:00',timestamp '2024-01-02 00:00:00') or TA in (timestamp '2024-01-02 00:00:00', timestamp '2024-01-03 00:00:00')", + "TA in (timestamp '2024-01-01 00:00:00',timestamp '2024-01-02 00:00:00',timestamp '2024-01-03 00:00:00')"); + assertRewrite("TA in (timestamp '2024-01-01 00:50:00',timestamp '2024-01-02 00:50:00') and TA in (timestamp '2024-01-03 00:50:00', timestamp '2024-01-04 00:50:00')", + "FALSE"); + assertRewrite("TA = timestamp '2024-01-03 00:50:00' and TA = timestamp '2024-01-01 00:50:00'", "FALSE"); + assertRewrite("TA in (timestamp '2024-01-01 00:50:00') and TA in (timestamp '2024-01-03 00:50:00')", "FALSE"); + assertRewrite("TA in (timestamp '2024-01-03 00:50:00') and TA in (timestamp '2024-01-03 00:50:00')", "TA = timestamp '2024-01-03 00:50:00'"); + assertRewrite("(TA > timestamp '2024-01-03 00:50:00' and TA < timestamp '2024-01-01 00:50:00') and TB < timestamp '2024-01-05 00:50:00'", "FALSE"); + assertRewrite("(TA > timestamp '2024-01-03 00:50:00' and TA < timestamp '2024-01-01 00:50:00') or TB < timestamp '2024-01-05 00:50:00'", + "TB < timestamp '2024-01-05 00:50:00'"); + } + private void assertRewrite(String expression, String expected) { Map mem = Maps.newHashMap(); Expression needRewriteExpression = replaceUnboundSlot(PARSER.parseExpression(expression), mem); diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/stats/FilterEstimationTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/stats/FilterEstimationTest.java index 0206a093257af0..7ef1d8736a4b82 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/stats/FilterEstimationTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/stats/FilterEstimationTest.java @@ -29,6 +29,7 @@ import org.apache.doris.nereids.trees.expressions.LessThan; import org.apache.doris.nereids.trees.expressions.LessThanEqual; import org.apache.doris.nereids.trees.expressions.Not; +import org.apache.doris.nereids.trees.expressions.NullSafeEqual; import org.apache.doris.nereids.trees.expressions.Or; import org.apache.doris.nereids.trees.expressions.SlotReference; import org.apache.doris.nereids.trees.expressions.literal.DateLiteral; @@ -40,6 +41,7 @@ import org.apache.doris.statistics.ColumnStatistic; import org.apache.doris.statistics.ColumnStatisticBuilder; import org.apache.doris.statistics.Statistics; +import org.apache.doris.statistics.StatisticsBuilder; import com.google.common.collect.Lists; import org.apache.commons.math3.util.Precision; @@ -1103,4 +1105,38 @@ public void testNumNullsOrIsNull() { Assertions.assertEquals(result.getRowCount(), 10.0, 0.01); } + @Test + public void testNullSafeEqual() { + ColumnStatisticBuilder columnStatisticBuilder = new ColumnStatisticBuilder() + .setNdv(2) + .setAvgSizeByte(4) + .setNumNulls(8) + .setMaxValue(2) + .setMinValue(1) + .setCount(10); + ColumnStatistic aStats = columnStatisticBuilder.build(); + SlotReference a = new SlotReference("a", IntegerType.INSTANCE); + + columnStatisticBuilder.setNdv(2) + .setAvgSizeByte(4) + .setNumNulls(7) + .setMaxValue(2) + .setMinValue(1) + .setCount(10); + ColumnStatistic bStats = columnStatisticBuilder.build(); + SlotReference b = new SlotReference("b", IntegerType.INSTANCE); + + StatisticsBuilder statsBuilder = new StatisticsBuilder(); + statsBuilder.setRowCount(100); + statsBuilder.putColumnStatistics(a, aStats); + statsBuilder.putColumnStatistics(b, bStats); + + NullSafeEqual nse = new NullSafeEqual(a, b); + FilterEstimation estimator = new FilterEstimation(); + Statistics resultNse = estimator.estimate(nse, statsBuilder.build()); + + EqualTo eq = new EqualTo(a, b); + Statistics resultEq = estimator.estimate(eq, statsBuilder.build()); + Assertions.assertEquals(7, resultNse.getRowCount() - resultEq.getRowCount()); + } } diff --git a/fe/pom.xml b/fe/pom.xml index c99550544f9877..a759a310ddca1b 100644 --- a/fe/pom.xml +++ b/fe/pom.xml @@ -321,7 +321,7 @@ under the License. 2.1 4.5.13 4.4.15 - 1.12.625 + 1.12.669 3.0.9 0.2.14 3.3.6 @@ -836,6 +836,27 @@ under the License. netty-transport-udt ${netty-all.version} + + io.netty + netty-transport-sctp + ${netty-all.version} + + + io.netty + netty-transport-sctp + ${netty-all.version} + + + io.netty + netty-transport-classes-kqueue + ${netty-all.version} + + + io.netty + netty-transport-native-epoll + linux-x86_64 + ${netty-all.version} + io.netty netty-transport-rxtx @@ -915,6 +936,22 @@ under the License. com.amazonaws * + + org.apache.solr + * + + + org.elasticsearch.client + * + + + org.elasticsearch.plugin + * + + + org.elasticsearch + * + diff --git a/gensrc/proto/internal_service.proto b/gensrc/proto/internal_service.proto index cf45d0395229e7..463d04f218eee2 100644 --- a/gensrc/proto/internal_service.proto +++ b/gensrc/proto/internal_service.proto @@ -533,12 +533,14 @@ message PMergeFilterRequest { optional PInFilter in_filter = 7; optional bool is_pipeline = 8; optional bool opt_remote_rf = 9; + optional PColumnType column_type = 10; }; message PMergeFilterResponse { required PStatus status = 1; }; +// delete PPublishFilterRequest after upgrade doris 2.1 message PPublishFilterRequest { required int32 filter_id = 1; required PUniqueId query_id = 2; @@ -549,6 +551,7 @@ message PPublishFilterRequest { optional PInFilter in_filter = 7; optional bool is_pipeline = 8; optional int64 merge_time = 9; + optional PColumnType column_type = 10; }; message PPublishFilterRequestV2 { diff --git a/gensrc/thrift/DataSinks.thrift b/gensrc/thrift/DataSinks.thrift index 8b15585aa5d6a1..8b62e5c1bc6656 100644 --- a/gensrc/thrift/DataSinks.thrift +++ b/gensrc/thrift/DataSinks.thrift @@ -282,15 +282,40 @@ struct TWriteLocation { 2: optional string target_path } +struct TWritePartition { + 1: optional list values + 2: optional TWriteLocation location + 3: optional PlanNodes.TFileFormatType file_format +} + +struct TSortedColumn { + 1: optional string sort_column_name + 2: optional i32 order // asc(1) or desc(0) +} + +struct TWriteBucket { + 1: optional list bucketed_by + 2: optional i32 bucket_version + 3: optional i32 bucket_count + 4: optional list sorted_by +} + +enum TWriteCompressionType { + SNAPPY = 3, + LZ4 = 4, + ZLIB = 6, + ZSTD = 7, +} + struct THiveTableSink { 1: optional string db_name 2: optional string table_name 3: optional list data_column_names 4: optional list partition_column_names - 5: optional list partitions - 6: optional list buckets + 5: optional list partitions + 6: optional list buckets 7: optional PlanNodes.TFileFormatType file_format - 8: optional Types.TCompressionType compression_type + 8: optional TWriteCompressionType compression_type 9: optional TWriteLocation location } diff --git a/gensrc/thrift/Partitions.thrift b/gensrc/thrift/Partitions.thrift index 0c69f01e259a3f..1602f405a9a113 100644 --- a/gensrc/thrift/Partitions.thrift +++ b/gensrc/thrift/Partitions.thrift @@ -20,8 +20,6 @@ namespace java org.apache.doris.thrift include "Exprs.thrift" include "Types.thrift" -include "PlanNodes.thrift" -include "DataSinks.thrift" enum TPartitionType { UNPARTITIONED, @@ -93,22 +91,3 @@ struct TDataPartition { 2: optional list partition_exprs 3: optional list partition_infos } - -struct TWritePartition { - 1: optional list values - 2: optional DataSinks.TWriteLocation location - 3: optional PlanNodes.TFileFormatType file_format -} - -struct TSortedColumn { - 1: optional string sort_column_name - 2: optional i32 order // asc(1) or desc(0) -} - -struct TWriteBucket { - 1: optional list bucketed_by - 2: optional i32 bucket_version - 3: optional i32 bucket_count - 4: optional list sorted_by -} - diff --git a/gensrc/thrift/Types.thrift b/gensrc/thrift/Types.thrift index e5ede58b3078b2..b6ec18d8197ee5 100644 --- a/gensrc/thrift/Types.thrift +++ b/gensrc/thrift/Types.thrift @@ -728,18 +728,6 @@ struct TUserIdentity { 3: optional bool is_domain } -enum TCompressionType { - UNKNOWN_COMPRESSION = 0, - DEFAULT_COMPRESSION = 1, - NO_COMPRESSION = 2, - SNAPPY = 3, - LZ4 = 4, - LZ4F = 5, - ZLIB = 6, - ZSTD = 7, - LZ4HC = 8 -} - const i32 TSNAPSHOT_REQ_VERSION1 = 3; // corresponding to alpha rowset const i32 TSNAPSHOT_REQ_VERSION2 = 4; // corresponding to beta rowset // the snapshot request should always set prefer snapshot version to TPREFER_SNAPSHOT_REQ_VERSION diff --git a/regression-test/data/nereids_p0/hint/fix_leading.out b/regression-test/data/nereids_p0/hint/fix_leading.out index 1410c8e0e6e62d..e4c2d1808c1298 100644 --- a/regression-test/data/nereids_p0/hint/fix_leading.out +++ b/regression-test/data/nereids_p0/hint/fix_leading.out @@ -4,10 +4,11 @@ PhysicalResultSink --PhysicalDistribute[DistributionSpecGather] ----PhysicalProject ------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t3.c3) and (t1.c1 = t4.c4)) otherCondition=() ---------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +--------NestedLoopJoin[CROSS_JOIN] ----------PhysicalOlapScan[t1] -----------PhysicalDistribute[DistributionSpecHash] -------------PhysicalOlapScan[t2] +----------PhysicalDistribute[DistributionSpecReplicated] +------------filter((t2.c2 = t2.c2)) +--------------PhysicalOlapScan[t2] --------PhysicalDistribute[DistributionSpecHash] ----------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() ------------PhysicalOlapScan[t3] @@ -19,3 +20,198 @@ Used: leading({ t1 t2 } { t3 t4 } ) UnUsed: SyntaxError: +-- !select2_1_1 -- +1876000 + +-- !select2_1_2 -- +1876000 + +-- !select2_1_3 -- +1876000 + +-- !select2_1_4 -- +1876000 + +-- !select2_1_5 -- +1876000 + +-- !select2_1_6 -- +1876000 + +-- !select2_1_7 -- +1876000 + +-- !select2_1_8 -- +1876000 + +-- !select2_1_9 -- +1876000 + +-- !select2_1_10 -- +1876000 + +-- !select2_1_11 -- +1876000 + +-- !select2_1_12 -- +1876000 + +-- !select2_1_13 -- +1876000 + +-- !select2_2_1 -- +1751202 + +-- !select2_2_2 -- +1751202 + +-- !select2_2_3 -- +1751202 + +-- !select2_2_4 -- +1751202 + +-- !select2_2_5 -- +1751202 + +-- !select2_2_6 -- +1751202 + +-- !select2_2_7 -- +1751202 + +-- !select2_2_8 -- +1751202 + +-- !select2_2_9 -- +1751202 + +-- !select2_2_10 -- +1751202 + +-- !select2_2_11 -- +1751202 + +-- !select2_2_12 -- +1751202 + +-- !select2_2_13 -- +1751202 + +-- !select2_3_1 -- +816000 + +-- !select2_3_2 -- +816000 + +-- !select2_3_3 -- +816000 + +-- !select2_3_4 -- +816000 + +-- !select2_3_5 -- +816000 + +-- !select2_3_6 -- +816000 + +-- !select2_3_7 -- +816000 + +-- !select2_3_8 -- +816000 + +-- !select2_3_9 -- +816000 + +-- !select2_3_10 -- +816000 + +-- !select2_3_11 -- +816000 + +-- !select2_3_12 -- +816000 + +-- !select2_3_13 -- +816000 + +-- !select2_4_1 -- +125000 + +-- !select2_4_2 -- +125000 + +-- !select2_4_3 -- +125000 + +-- !select2_4_4 -- +125000 + +-- !select2_4_5 -- +125000 + +-- !select2_4_6 -- +125000 + +-- !select2_4_7 -- +125000 + +-- !select2_4_8 -- +125000 + +-- !select2_4_9 -- +125000 + +-- !select2_4_10 -- +125000 + +-- !select2_4_11 -- +125000 + +-- !select2_4_12 -- +125000 + +-- !select2_4_13 -- +125000 + +-- !select2_5_1 -- +816000 + +-- !select2_5_2 -- +816000 + +-- !select2_5_3 -- +816000 + +-- !select2_5_4 -- +816000 + +-- !select2_5_5 -- +816000 + +-- !select2_5_6 -- +816000 + +-- !select2_5_7 -- +816000 + +-- !select2_5_8 -- +816000 + +-- !select2_5_9 -- +816000 + +-- !select2_5_10 -- +816000 + +-- !select2_5_11 -- +816000 + +-- !select2_5_12 -- +816000 + +-- !select2_5_13 -- +816000 + diff --git a/regression-test/pipeline/cloud_p0/conf/regression-conf-custom.groovy b/regression-test/pipeline/cloud_p0/conf/regression-conf-custom.groovy index 1518def58c11c3..ba7214cc43ccbb 100644 --- a/regression-test/pipeline/cloud_p0/conf/regression-conf-custom.groovy +++ b/regression-test/pipeline/cloud_p0/conf/regression-conf-custom.groovy @@ -16,7 +16,7 @@ // under the License. testGroups = "p0" -testDirectories = "ddl_p0,database_p0,load,load_p0,query_p0,table_p0,account_p0,autobucket,bitmap_functions,bloom_filter_p0,cast_decimal_to_boolean,cast_double_to_decimal,compression_p0,connector_p0,correctness,correctness_p0,csv_header_p0,data_model_p0,database_p0,datatype_p0,delete_p0,demo_p0,empty_relation,export_p0,external_table_p0,fault_injection_p0,flink_connector_p0,insert_overwrite_p0,insert_p0,internal_schema_p0,javaudf_p0,job_p0,json_p0,jsonb_p0,meta_action_p0,metrics_p0,mtmv_p0,mysql_fulltext,mysql_ssl_p0,mysql_tupleconvert_p0,mysqldump_p0,nereids_arith_p0,nereids_function_p0,nereids_p0,nereids_rules_p0,nereids_syntax_p0,nereids_tpcds_p0,nereids_tpch_p0,partition_p0,performance_p0,point_query_p0,postgres,query_profile,row_store,show_p0,source_p0,sql_block_rule_p0,ssb_unique_load_zstd_p0,ssb_unique_sql_zstd_p0,statistics,table_p0,tpch_unique_sql_zstd_p0,trino_p0,types,types_p0,update,version_p0,view_p0,with_clause_p0,workload_manager_p0,schema_change_p0,variant_p0,variant_github_events_p0_new,variant_github_events_p0" +testDirectories = "ddl_p0,database_p0,load,load_p0,query_p0,table_p0,account_p0,autobucket,bitmap_functions,bloom_filter_p0,cast_decimal_to_boolean,cast_double_to_decimal,compression_p0,connector_p0,correctness,correctness_p0,csv_header_p0,data_model_p0,database_p0,datatype_p0,delete_p0,demo_p0,empty_relation,export_p0,external_table_p0,fault_injection_p0,flink_connector_p0,insert_overwrite_p0,insert_p0,internal_schema_p0,javaudf_p0,job_p0,json_p0,jsonb_p0,meta_action_p0,metrics_p0,mtmv_p0,mysql_fulltext,mysql_ssl_p0,mysql_tupleconvert_p0,mysqldump_p0,nereids_arith_p0,nereids_function_p0,nereids_p0,nereids_rules_p0,nereids_syntax_p0,nereids_tpcds_p0,nereids_tpch_p0,partition_p0,performance_p0,postgres,query_profile,row_store,show_p0,source_p0,sql_block_rule_p0,ssb_unique_load_zstd_p0,ssb_unique_sql_zstd_p0,statistics,table_p0,tpch_unique_sql_zstd_p0,trino_p0,types,types_p0,update,version_p0,view_p0,with_clause_p0,workload_manager_p0,schema_change_p0,variant_p0,variant_github_events_p0_new,variant_github_events_p0,unique_with_mow_p0" //exclude groups and exclude suites is more prior than include groups and include suites. -excludeSuites = "test_index_failure_injection,test_dump_image,test_profile,test_spark_load,test_refresh_mtmv,test_bitmap_filter,test_information_schema_external" -excludeDirectories = "workload_manager_p1,nereids_rules_p0/subquery" +excludeSuites = "test_index_failure_injection,test_dump_image,test_profile,test_spark_load,test_refresh_mtmv,test_bitmap_filter,test_information_schema_external,test_primary_key_partial_update_parallel" +excludeDirectories = "workload_manager_p1,nereids_rules_p0/subquery,unique_with_mow_p0/cluster_key,unique_with_mow_p0/ssb_unique_sql_zstd_cluster" diff --git a/regression-test/pipeline/cloud_p0/conf/session_variables.sql b/regression-test/pipeline/cloud_p0/conf/session_variables.sql index 96a849bc1ad68c..962464502ec409 100644 --- a/regression-test/pipeline/cloud_p0/conf/session_variables.sql +++ b/regression-test/pipeline/cloud_p0/conf/session_variables.sql @@ -3,3 +3,4 @@ set global insert_visible_timeout_ms=60000; set global enable_auto_analyze=false; set global enable_audit_plugin=true; set global enable_memtable_on_sink_node=false; +set global enable_pipeline_x_engine=false; diff --git a/regression-test/pipeline/cloud_p1/conf/session_variables.sql b/regression-test/pipeline/cloud_p1/conf/session_variables.sql index 8885476ee06a43..b38279c9b42df8 100644 --- a/regression-test/pipeline/cloud_p1/conf/session_variables.sql +++ b/regression-test/pipeline/cloud_p1/conf/session_variables.sql @@ -3,4 +3,4 @@ set global insert_visible_timeout_ms=60000; set global enable_auto_analyze=false; set global enable_audit_plugin=true; set global enable_memtable_on_sink_node=false; -set global enable_two_phase_read_opt = false; +set global enable_pipeline_x_engine=false; \ No newline at end of file diff --git a/regression-test/pipeline/external/conf/fe.conf b/regression-test/pipeline/external/conf/fe.conf index 38312c5647e1c9..6f0cfb79253cc0 100644 --- a/regression-test/pipeline/external/conf/fe.conf +++ b/regression-test/pipeline/external/conf/fe.conf @@ -30,6 +30,9 @@ JAVA_OPTS="-Xmx4096m -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=$DORIS_HOM # For jdk 9+, this JAVA_OPTS will be used as default JVM options JAVA_OPTS_FOR_JDK_9="-Xmx4096m -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=$DORIS_HOME/log/fe.jmap -XX:SurvivorRatio=8 -XX:MaxTenuringThreshold=7 -XX:+CMSClassUnloadingEnabled -XX:-CMSParallelRemarkEnabled -XX:CMSInitiatingOccupancyFraction=80 -XX:SoftRefLRUPolicyMSPerMB=0 -Xlog:gc*:$DORIS_HOME/log/fe.gc.log.$DATE:time" +# For jdk 17+, this JAVA_OPTS will be used as default JVM options +JAVA_OPTS_FOR_JDK_17="-Djavax.security.auth.useSubjectCredsOnly=false -Xmx8192m -Xms8192m -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=$DORIS_HOME/log/ -Xlog:gc*:$DORIS_HOME/log/fe.gc.log.$CUR_DATE:time" + ## ## the lowercase properties are read by main program. ## diff --git a/regression-test/pipeline/p0/conf/fe.conf b/regression-test/pipeline/p0/conf/fe.conf index 0b9c904ffa6540..d808ea83b85660 100644 --- a/regression-test/pipeline/p0/conf/fe.conf +++ b/regression-test/pipeline/p0/conf/fe.conf @@ -34,7 +34,7 @@ JAVA_OPTS="-Djavax.security.auth.useSubjectCredsOnly=false -Xss4m -Xmx4096m -XX: JAVA_OPTS_FOR_JDK_9="-Djavax.security.auth.useSubjectCredsOnly=false -Xss4m -Xmx4096m -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=$DORIS_HOME/log/fe.jmap -XX:+UseG1GC -XX:MaxGCPauseMillis=200 -XX:+PrintGCDateStamps -XX:+PrintGCDetails -Xloggc:$DORIS_HOME/log/fe.gc.log.$CUR_DATE -Dlog4j2.formatMsgNoLookups=true" # For jdk 17+, this JAVA_OPTS will be used as default JVM options -JAVA_OPTS_FOR_JDK_17="-Djavax.security.auth.useSubjectCredsOnly=false -XX:+UseZGC -Xmx8192m -Xms8192m -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=$DORIS_HOME/log/ -Xlog:gc*:$DORIS_HOME/log/fe.gc.log.$CUR_DATE:time" +JAVA_OPTS_FOR_JDK_17="-Djavax.security.auth.useSubjectCredsOnly=false -Xmx8192m -Xms8192m -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=$DORIS_HOME/log/ -Xlog:gc*:$DORIS_HOME/log/fe.gc.log.$CUR_DATE:time" ## ## the lowercase properties are read by main program. diff --git a/regression-test/pipeline/p1/conf/fe.conf b/regression-test/pipeline/p1/conf/fe.conf index 01ee2e6da2e255..eb8ab90af0898d 100644 --- a/regression-test/pipeline/p1/conf/fe.conf +++ b/regression-test/pipeline/p1/conf/fe.conf @@ -27,9 +27,14 @@ LOG_DIR = ${DORIS_HOME}/log DATE = `date +%Y%m%d-%H%M%S` # JAVA_OPTS="-Xmx4096m -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=$DORIS_HOME/log/fe.jmap -XX:+UseMembar -XX:SurvivorRatio=8 -XX:MaxTenuringThreshold=7 -XX:+PrintGCDateStamps -XX:+PrintGCDetails -XX:+UseConcMarkSweepGC -XX:+UseParNewGC -XX:+CMSClassUnloadingEnabled -XX:-CMSParallelRemarkEnabled -XX:CMSInitiatingOccupancyFraction=80 -XX:SoftRefLRUPolicyMSPerMB=0 -Xloggc:$DORIS_HOME/log/fe.gc.log.$DATE" JAVA_OPTS="-Djavax.security.auth.useSubjectCredsOnly=false -Xss4m -Xmx4096m -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=$DORIS_HOME/log/fe.jmap -XX:+UseG1GC -XX:MaxGCPauseMillis=200 -XX:+PrintGCDateStamps -XX:+PrintGCDetails -Xloggc:$DORIS_HOME/log/fe.gc.log.$CUR_DATE -Dlog4j2.formatMsgNoLookups=true" + # For jdk 9+, this JAVA_OPTS will be used as default JVM options # JAVA_OPTS_FOR_JDK_9="-Xmx4096m -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=$DORIS_HOME/log/fe.jmap -XX:SurvivorRatio=8 -XX:MaxTenuringThreshold=7 -XX:+CMSClassUnloadingEnabled -XX:-CMSParallelRemarkEnabled -XX:CMSInitiatingOccupancyFraction=80 -XX:SoftRefLRUPolicyMSPerMB=0 -Xlog:gc*:$DORIS_HOME/log/fe.gc.log.$DATE:time" JAVA_OPTS_FOR_JDK_9="-Djavax.security.auth.useSubjectCredsOnly=false -Xss4m -Xmx4096m -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=$DORIS_HOME/log/fe.jmap -XX:+UseG1GC -XX:MaxGCPauseMillis=200 -XX:+PrintGCDateStamps -XX:+PrintGCDetails -Xloggc:$DORIS_HOME/log/fe.gc.log.$CUR_DATE -Dlog4j2.formatMsgNoLookups=true" + +# For jdk 17+, this JAVA_OPTS will be used as default JVM options +JAVA_OPTS_FOR_JDK_17="-Djavax.security.auth.useSubjectCredsOnly=false -Xmx8192m -Xms8192m -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=$DORIS_HOME/log/ -Xlog:gc*:$DORIS_HOME/log/fe.gc.log.$CUR_DATE:time" + ## ## the lowercase properties are read by main program. ## diff --git a/regression-test/plugins/plugin_get_table_properties.groovy b/regression-test/plugins/plugin_get_table_properties.groovy index 84d16d2d45d453..17aef81365f2a3 100644 --- a/regression-test/plugins/plugin_get_table_properties.groovy +++ b/regression-test/plugins/plugin_get_table_properties.groovy @@ -23,8 +23,12 @@ Suite.metaClass.get_table_replica_num = {String tb_name /* param */ -> def createTbl = result[0][1].toString() def regexPattern = /"replication_allocation" = "tag.location.default: (\d+)"/ def matcher = (createTbl =~ regexPattern) - assertTrue(matcher.find()) - return matcher.group(1).toInteger() + //assertTrue(matcher.find()) + if (matcher.find()) { + return matcher.group(1).toInteger() + } else { + return 1 + } } -logger.info("Added 'get_table_replica_num' function to Suite") \ No newline at end of file +logger.info("Added 'get_table_replica_num' function to Suite") diff --git a/regression-test/suites/external_table_p0/hive/test_hive_parquet.groovy b/regression-test/suites/external_table_p0/hive/test_hive_parquet.groovy index dff86e77285391..c381a3c236b105 100644 --- a/regression-test/suites/external_table_p0/hive/test_hive_parquet.groovy +++ b/regression-test/suites/external_table_p0/hive/test_hive_parquet.groovy @@ -169,15 +169,6 @@ suite("test_hive_parquet", "p0,external,hive,external_docker,external_docker_hiv """ } - def q23 = { - sql """use hive_schema_change""" - // not support the schema change of complex type - test { - sql "select * from struct_test" - exception "Wrong data type for column 'f2'" - } - } - String enabled = context.config.otherConfigs.get("enableHiveTest") if (enabled != null && enabled.equalsIgnoreCase("true")) { try { @@ -216,7 +207,6 @@ suite("test_hive_parquet", "p0,external,hive,external_docker,external_docker_hiv q20() q21() q22() - q23() sql """explain physical plan select l_partkey from partition_table where (nation != 'cn' or city !='beijing') and (l_quantity > 28 or l_extendedprice > 30000) diff --git a/regression-test/suites/fault_injection_p0/test_build_index_fault.groovy b/regression-test/suites/fault_injection_p0/test_build_index_fault.groovy index f5b95137c32dc5..796a7b03443cd7 100644 --- a/regression-test/suites/fault_injection_p0/test_build_index_fault.groovy +++ b/regression-test/suites/fault_injection_p0/test_build_index_fault.groovy @@ -16,7 +16,7 @@ // under the License. -suite("test_build_index_fault", "inverted_index, nonConcurrent"){ +suite("test_build_index_fault", "inverted_index, nonConcurrent,p2"){ // prepare test table def timeout = 60000 def delta_time = 1000 diff --git a/regression-test/suites/fault_injection_p0/test_segcompaction_fault_injection.groovy b/regression-test/suites/fault_injection_p0/test_segcompaction_fault_injection.groovy index a850111dee85fa..ceac616a8f1430 100644 --- a/regression-test/suites/fault_injection_p0/test_segcompaction_fault_injection.groovy +++ b/regression-test/suites/fault_injection_p0/test_segcompaction_fault_injection.groovy @@ -18,7 +18,7 @@ import org.codehaus.groovy.runtime.IOGroovyMethods import org.apache.doris.regression.util.Http -suite("test_segcompaction_correctness", "nonConcurrent") { +suite("test_segcompaction_correctness", "nonConcurrent,p2") { def tableName = "segcompaction_correctness_test" def create_table_sql = """ CREATE TABLE IF NOT EXISTS ${tableName} ( diff --git a/regression-test/suites/fault_injection_p0/test_too_many_segments_fault_injection.groovy b/regression-test/suites/fault_injection_p0/test_too_many_segments_fault_injection.groovy index e1ea554b976fda..9c55d2938ba0be 100644 --- a/regression-test/suites/fault_injection_p0/test_too_many_segments_fault_injection.groovy +++ b/regression-test/suites/fault_injection_p0/test_too_many_segments_fault_injection.groovy @@ -37,7 +37,7 @@ def create_table_sql = """ """ def columns = "col_0, col_1, col_2, col_3, col_4, col_5, col_6, col_7, col_8, col_9, col_10, col_11, col_12, col_13, col_14, col_15, col_16, col_17, col_18, col_19, col_20, col_21, col_22, col_23, col_24, col_25, col_26, col_27, col_28, col_29, col_30, col_31, col_32, col_33, col_34, col_35, col_36, col_37, col_38, col_39, col_40, col_41, col_42, col_43, col_44, col_45, col_46, col_47, col_48, col_49" -suite("test_too_many_segments", "nonConcurrent") { // the epic -238 case +suite("test_too_many_segments", "nonConcurrent,p2") { // the epic -238 case def runLoadWithTooManySegments = { String ak = getS3AK() String sk = getS3SK() diff --git a/regression-test/suites/nereids_p0/aggregate/aggregate.groovy b/regression-test/suites/nereids_p0/aggregate/aggregate.groovy index e9cb3b31b8df6f..2446fcf09239bc 100644 --- a/regression-test/suites/nereids_p0/aggregate/aggregate.groovy +++ b/regression-test/suites/nereids_p0/aggregate/aggregate.groovy @@ -334,7 +334,7 @@ suite("aggregate") { SELECT k1, k2 FROM tempbaseall GROUP BY k1; """ - exception "java.sql.SQLException: errCode = 2, detailMessage = k2 not in agg's output" + exception "java.sql.SQLException: errCode = 2, detailMessage = k2 not in aggregate's output" } test { diff --git a/regression-test/suites/nereids_p0/hint/fix_leading.groovy b/regression-test/suites/nereids_p0/hint/fix_leading.groovy index 6804a60e487610..52f93e3d3e2a36 100644 --- a/regression-test/suites/nereids_p0/hint/fix_leading.groovy +++ b/regression-test/suites/nereids_p0/hint/fix_leading.groovy @@ -39,6 +39,118 @@ suite("fix_leading") { sql """create table t3 (c3 int, c33 int) distributed by hash(c3) buckets 3 properties('replication_num' = '1');""" sql """create table t4 (c4 int, c44 int) distributed by hash(c4) buckets 3 properties('replication_num' = '1');""" + streamLoad { + table "t1" + db "fix_leading" + set 'column_separator', '|' + set 'format', 'csv' + file 't1.csv' + time 10000 + } + + streamLoad { + table "t2" + db "fix_leading" + set 'column_separator', '|' + set 'format', 'csv' + file 't2.csv' + time 10000 + } + + streamLoad { + table "t3" + db "fix_leading" + set 'column_separator', '|' + set 'format', 'csv' + file 't3.csv' + time 10000 + } + + streamLoad { + table "t4" + db "fix_leading" + set 'column_separator', '|' + set 'format', 'csv' + file 't4.csv' + time 10000 + } + // bug fix 1: {t1 t2}{t3 t4} miss levels - qt_select1 """explain shape plan select /*+ leading({t1 t2}{t3 t4}) */ * from t1 join t2 on c1 = c2 join t3 on c1 = c3 join t4 on c1 = c4;""" + qt_select1 """explain shape plan select /*+ leading({t1 t2}{t3 t4}) */ * from t1 join t2 on c2 = c2 join t3 on c1 = c3 join t4 on c1 = c4;""" + + // bug fix 2: fix left outer join without edge with other tables + // left join + left join + qt_select2_1_1 """select count(*) from t1 left join t2 on c2 = c2 left join t3 on c2 = c3;""" + qt_select2_1_2 """select /*+ leading(t1 t2 t3) */ count(*) from t1 left join t2 on c2 = c2 left join t3 on c2 = c3;""" + qt_select2_1_3 """select /*+ leading(t1 {t2 t3}) */ count(*) from t1 left join t2 on c2 = c2 left join t3 on c2 = c3;""" + qt_select2_1_4 """select /*+ leading(t1 t3 t2) */ count(*) from t1 left join t2 on c2 = c2 left join t3 on c2 = c3;""" + qt_select2_1_5 """select /*+ leading(t1 {t2 t2}) */ count(*) from t1 left join t2 on c2 = c2 left join t3 on c2 = c3;""" + qt_select2_1_6 """select /*+ leading(t2 t1 t3) */ count(*) from t1 left join t2 on c2 = c2 left join t3 on c2 = c3;""" + qt_select2_1_7 """select /*+ leading(t2 {t1 t3}) */ count(*) from t1 left join t2 on c2 = c2 left join t3 on c2 = c3;""" + qt_select2_1_8 """select /*+ leading(t2 t3 t1) */ count(*) from t1 left join t2 on c2 = c2 left join t3 on c2 = c3;""" + qt_select2_1_9 """select /*+ leading(t2 {t3 t1}) */ count(*) from t1 left join t2 on c2 = c2 left join t3 on c2 = c3;""" + qt_select2_1_10 """select /*+ leading(t3 t1 t2) */ count(*) from t1 left join t2 on c2 = c2 left join t3 on c2 = c3;""" + qt_select2_1_11 """select /*+ leading(t3 {t1 t2}) */ count(*) from t1 left join t2 on c2 = c2 left join t3 on c2 = c3;""" + qt_select2_1_12 """select /*+ leading(t3 t2 t1) */ count(*) from t1 left join t2 on c2 = c2 left join t3 on c2 = c3;""" + qt_select2_1_13 """select /*+ leading(t3 {t2 t1}) */ count(*) from t1 left join t2 on c2 = c2 left join t3 on c2 = c3;""" + + // left join + right join + qt_select2_2_1 """select count(*) from t1 left join t2 on c2 = c2 right join t3 on c2 = c3;""" + qt_select2_2_2 """select /*+ leading(t1 t2 t3) */ count(*) from t1 left join t2 on c2 = c2 right join t3 on c2 = c3;""" + qt_select2_2_3 """select /*+ leading(t1 {t2 t3}) */ count(*) from t1 left join t2 on c2 = c2 right join t3 on c2 = c3;""" + qt_select2_2_4 """select /*+ leading(t1 t3 t2) */ count(*) from t1 left join t2 on c2 = c2 right join t3 on c2 = c3;""" + qt_select2_2_5 """select /*+ leading(t1 {t2 t2}) */ count(*) from t1 left join t2 on c2 = c2 right join t3 on c2 = c3;""" + qt_select2_2_6 """select /*+ leading(t2 t1 t3) */ count(*) from t1 left join t2 on c2 = c2 right join t3 on c2 = c3;""" + qt_select2_2_7 """select /*+ leading(t2 {t1 t3}) */ count(*) from t1 left join t2 on c2 = c2 right join t3 on c2 = c3;""" + qt_select2_2_8 """select /*+ leading(t2 t3 t1) */ count(*) from t1 left join t2 on c2 = c2 right join t3 on c2 = c3;""" + qt_select2_2_9 """select /*+ leading(t2 {t3 t1}) */ count(*) from t1 left join t2 on c2 = c2 right join t3 on c2 = c3;""" + qt_select2_2_10 """select /*+ leading(t3 t1 t2) */ count(*) from t1 left join t2 on c2 = c2 right join t3 on c2 = c3;""" + qt_select2_2_11 """select /*+ leading(t3 {t1 t2}) */ count(*) from t1 left join t2 on c2 = c2 right join t3 on c2 = c3;""" + qt_select2_2_12 """select /*+ leading(t3 t2 t1) */ count(*) from t1 left join t2 on c2 = c2 right join t3 on c2 = c3;""" + qt_select2_2_13 """select /*+ leading(t3 {t2 t1}) */ count(*) from t1 left join t2 on c2 = c2 right join t3 on c2 = c3;""" + + // left join + semi join + qt_select2_3_1 """select count(*) from t1 left join t2 on c2 = c2 left semi join t3 on c2 = c3;""" + qt_select2_3_2 """select /*+ leading(t1 t2 t3) */ count(*) from t1 left join t2 on c2 = c2 left semi join t3 on c2 = c3;""" + qt_select2_3_3 """select /*+ leading(t1 {t2 t3}) */ count(*) from t1 left join t2 on c2 = c2 left semi join t3 on c2 = c3;""" + qt_select2_3_4 """select /*+ leading(t1 t3 t2) */ count(*) from t1 left join t2 on c2 = c2 left semi join t3 on c2 = c3;""" + qt_select2_3_5 """select /*+ leading(t1 {t2 t2}) */ count(*) from t1 left join t2 on c2 = c2 left semi join t3 on c2 = c3;""" + qt_select2_3_6 """select /*+ leading(t2 t1 t3) */ count(*) from t1 left join t2 on c2 = c2 left semi join t3 on c2 = c3;""" + qt_select2_3_7 """select /*+ leading(t2 {t1 t3}) */ count(*) from t1 left join t2 on c2 = c2 left semi join t3 on c2 = c3;""" + qt_select2_3_8 """select /*+ leading(t2 t3 t1) */ count(*) from t1 left join t2 on c2 = c2 left semi join t3 on c2 = c3;""" + qt_select2_3_9 """select /*+ leading(t2 {t3 t1}) */ count(*) from t1 left join t2 on c2 = c2 left semi join t3 on c2 = c3;""" + qt_select2_3_10 """select /*+ leading(t3 t1 t2) */ count(*) from t1 left join t2 on c2 = c2 left semi join t3 on c2 = c3;""" + qt_select2_3_11 """select /*+ leading(t3 {t1 t2}) */ count(*) from t1 left join t2 on c2 = c2 left semi join t3 on c2 = c3;""" + qt_select2_3_12 """select /*+ leading(t3 t2 t1) */ count(*) from t1 left join t2 on c2 = c2 left semi join t3 on c2 = c3;""" + qt_select2_3_13 """select /*+ leading(t3 {t2 t1}) */ count(*) from t1 left join t2 on c2 = c2 left semi join t3 on c2 = c3;""" + + // left join + anti join + qt_select2_4_1 """select count(*) from t1 left join t2 on c2 = c2 left anti join t3 on c2 = c3;""" + qt_select2_4_2 """select /*+ leading(t1 t2 t3) */ count(*) from t1 left join t2 on c2 = c2 left anti join t3 on c2 = c3;""" + qt_select2_4_3 """select /*+ leading(t1 {t2 t3}) */ count(*) from t1 left join t2 on c2 = c2 left anti join t3 on c2 = c3;""" + qt_select2_4_4 """select /*+ leading(t1 t3 t2) */ count(*) from t1 left join t2 on c2 = c2 left anti join t3 on c2 = c3;""" + qt_select2_4_5 """select /*+ leading(t1 {t2 t2}) */ count(*) from t1 left join t2 on c2 = c2 left anti join t3 on c2 = c3;""" + qt_select2_4_6 """select /*+ leading(t2 t1 t3) */ count(*) from t1 left join t2 on c2 = c2 left anti join t3 on c2 = c3;""" + qt_select2_4_7 """select /*+ leading(t2 {t1 t3}) */ count(*) from t1 left join t2 on c2 = c2 left anti join t3 on c2 = c3;""" + qt_select2_4_8 """select /*+ leading(t2 t3 t1) */ count(*) from t1 left join t2 on c2 = c2 left anti join t3 on c2 = c3;""" + qt_select2_4_9 """select /*+ leading(t2 {t3 t1}) */ count(*) from t1 left join t2 on c2 = c2 left anti join t3 on c2 = c3;""" + qt_select2_4_10 """select /*+ leading(t3 t1 t2) */ count(*) from t1 left join t2 on c2 = c2 left anti join t3 on c2 = c3;""" + qt_select2_4_11 """select /*+ leading(t3 {t1 t2}) */ count(*) from t1 left join t2 on c2 = c2 left anti join t3 on c2 = c3;""" + qt_select2_4_12 """select /*+ leading(t3 t2 t1) */ count(*) from t1 left join t2 on c2 = c2 left anti join t3 on c2 = c3;""" + qt_select2_4_13 """select /*+ leading(t3 {t2 t1}) */ count(*) from t1 left join t2 on c2 = c2 left anti join t3 on c2 = c3;""" + + // right join + semi join + qt_select2_5_1 """select count(*) from t1 right join t2 on c2 = c2 left semi join t3 on c2 = c3;""" + qt_select2_5_2 """select /*+ leading(t1 t2 t3) */ count(*) from t1 right join t2 on c2 = c2 left semi join t3 on c2 = c3;""" + qt_select2_5_3 """select /*+ leading(t1 {t2 t3}) */ count(*) from t1 right join t2 on c2 = c2 left semi join t3 on c2 = c3;""" + qt_select2_5_4 """select /*+ leading(t1 t3 t2) */ count(*) from t1 right join t2 on c2 = c2 left semi join t3 on c2 = c3;""" + qt_select2_5_5 """select /*+ leading(t1 {t2 t2}) */ count(*) from t1 right join t2 on c2 = c2 left semi join t3 on c2 = c3;""" + qt_select2_5_6 """select /*+ leading(t2 t1 t3) */ count(*) from t1 right join t2 on c2 = c2 left semi join t3 on c2 = c3;""" + qt_select2_5_7 """select /*+ leading(t2 {t1 t3}) */ count(*) from t1 right join t2 on c2 = c2 left semi join t3 on c2 = c3;""" + qt_select2_5_8 """select /*+ leading(t2 t3 t1) */ count(*) from t1 right join t2 on c2 = c2 left semi join t3 on c2 = c3;""" + qt_select2_5_9 """select /*+ leading(t2 {t3 t1}) */ count(*) from t1 right join t2 on c2 = c2 left semi join t3 on c2 = c3;""" + qt_select2_5_10 """select /*+ leading(t3 t1 t2) */ count(*) from t1 right join t2 on c2 = c2 left semi join t3 on c2 = c3;""" + qt_select2_5_11 """select /*+ leading(t3 {t1 t2}) */ count(*) from t1 right join t2 on c2 = c2 left semi join t3 on c2 = c3;""" + qt_select2_5_12 """select /*+ leading(t3 t2 t1) */ count(*) from t1 right join t2 on c2 = c2 left semi join t3 on c2 = c3;""" + qt_select2_5_13 """select /*+ leading(t3 {t2 t1}) */ count(*) from t1 right join t2 on c2 = c2 left semi join t3 on c2 = c3;""" } diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension_equal/filter_equal_or_notequal.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension_equal/filter_equal_or_notequal.groovy index 1422b5db5857cb..1800a91b84b59d 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension_equal/filter_equal_or_notequal.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension_equal/filter_equal_or_notequal.groovy @@ -282,20 +282,19 @@ suite("filter_equal_or_notequal_case") { notContains "${mv_name}(${mv_name})" } - // Todo: It is not currently supported and is expected to be // mv is range and sql equal and filter in mv range -// query_sql = """ -// select l_shipdate, o_orderdate, l_partkey, l_suppkey -// from lineitem_1 -// left join orders_1 -// on lineitem_1.l_orderkey = orders_1.o_orderkey -// where l_shipdate = '2023-10-19' -// """ -// explain { -// sql("${query_sql}") -// contains "${mv_name}(${mv_name})" -// } -// compare_res(query_sql + " order by 1,2,3,4") + query_sql = """ + select l_shipdate, o_orderdate, l_partkey, l_suppkey + from lineitem_1 + left join orders_1 + on lineitem_1.l_orderkey = orders_1.o_orderkey + where l_shipdate = '2023-10-19' + """ + explain { + sql("${query_sql}") + contains "${mv_name}(${mv_name})" + } + compare_res(query_sql + " order by 1,2,3,4") // mv is range and sql is range and sql range is bigger than mv query_sql = """ @@ -350,20 +349,19 @@ suite("filter_equal_or_notequal_case") { notContains "${mv_name}(${mv_name})" } - // Todo: It is not currently supported and is expected to be // mv is range and sql is range and sql range is bigger than mv -// query_sql = """ -// select l_shipdate, o_orderdate, l_partkey, l_suppkey -// from lineitem_1 -// left join orders_1 -// on lineitem_1.l_orderkey = orders_1.o_orderkey -// where l_shipdate > '2023-10-19' -// """ -// explain { -// sql("${query_sql}") -// contains "${mv_name}(${mv_name})" -// } -// compare_res(query_sql + " order by 1,2,3,4") + query_sql = """ + select l_shipdate, o_orderdate, l_partkey, l_suppkey + from lineitem_1 + left join orders_1 + on lineitem_1.l_orderkey = orders_1.o_orderkey + where l_shipdate > '2023-10-18' + """ + explain { + sql("${query_sql}") + contains "${mv_name}(${mv_name})" + } + compare_res(query_sql + " order by 1,2,3,4") // mv is range and sql is range and sql range is not in mv range query_sql = """ @@ -418,20 +416,34 @@ suite("filter_equal_or_notequal_case") { notContains "${mv_name}(${mv_name})" } - // Todo: It is not currently supported and is expected to be // sql range is in mv range -// query_sql = """ -// select l_shipdate, o_orderdate, l_partkey, l_suppkey -// from lineitem_1 -// left join orders_1 -// on lineitem_1.l_orderkey = orders_1.o_orderkey -// where l_shipdate in ('2023-10-18') -// """ -// explain { -// sql("${query_sql}") -// contains "${mv_name}(${mv_name})" -// } -// compare_res(query_sql + " order by 1,2,3,4") + // single value + query_sql = """ + select l_shipdate, o_orderdate, l_partkey, l_suppkey + from lineitem_1 + left join orders_1 + on lineitem_1.l_orderkey = orders_1.o_orderkey + where l_shipdate in ('2023-10-18') + """ + explain { + sql("${query_sql}") + contains "${mv_name}(${mv_name})" + } + compare_res(query_sql + " order by 1,2,3,4") + + // multi value + query_sql = """ + select l_shipdate, o_orderdate, l_partkey, l_suppkey + from lineitem_1 + left join orders_1 + on lineitem_1.l_orderkey = orders_1.o_orderkey + where l_shipdate in ('2023-10-18', '2023-11-18') + """ + explain { + sql("${query_sql}") + contains "${mv_name}(${mv_name})" + } + compare_res(query_sql + " order by 1,2,3,4") // sql range like mv range query_sql = """ @@ -498,6 +510,20 @@ suite("filter_equal_or_notequal_case") { } compare_res(query_sql + " order by 1,2,3,4") + + query_sql = """ + select l_shipdate, o_orderdate, l_partkey, l_suppkey + from lineitem_1 + left join orders_1 + on lineitem_1.l_orderkey = orders_1.o_orderkey + where l_shipdate > '2023-10-17' and l_shipdate < '2023-10-19' + """ + explain { + sql("${query_sql}") + contains "${mv_name}(${mv_name})" + } + compare_res(query_sql + " order by 1,2,3,4") + // mtmv_sql = """ select l_shipdate, o_orderdate, l_partkey, l_suppkey, o_orderkey diff --git a/regression-test/suites/schema_change_p0/test_unique_model_schema_key_change.groovy b/regression-test/suites/schema_change_p0/test_unique_model_schema_key_change.groovy index 52f211a98e7ccc..4f59aead784432 100644 --- a/regression-test/suites/schema_change_p0/test_unique_model_schema_key_change.groovy +++ b/regression-test/suites/schema_change_p0/test_unique_model_schema_key_change.groovy @@ -254,4 +254,141 @@ suite("test_unique_model_schema_key_change","p0") { }, insertSql, true,"${tbName}")*/ + + /** + * Test the unique model by modify a value type + */ + + + initTable = " CREATE TABLE IF NOT EXISTS ${tbName}\n" + + " (\n" + + " `user_id` LARGEINT NOT NULL COMMENT \"用户id\",\n" + + " `username` VARCHAR(50) NOT NULL COMMENT \"用户昵称\",\n" + + " `is_teacher` BOOLEAN COMMENT \"是否是老师\",\n" + + " `city` VARCHAR(20) COMMENT \"用户所在城市\",\n" + + " `age` SMALLINT COMMENT \"用户年龄\",\n" + + " `sex` TINYINT COMMENT \"用户性别\",\n" + + " `phone` LARGEINT COMMENT \"用户电话\",\n" + + " `address` VARCHAR(500) COMMENT \"用户地址\",\n" + + " `register_time` DATETIME COMMENT \"用户注册时间\"\n" + + " )\n" + + " UNIQUE KEY(`user_id`, `username`, `is_teacher`)\n" + + " DISTRIBUTED BY HASH(`user_id`) BUCKETS 1\n" + + " PROPERTIES (\n" + + " \"replication_allocation\" = \"tag.location.default: 1\",\n" + + " \"enable_unique_key_merge_on_write\" = \"true\"\n" + + " );" + + initTableData = "insert into ${tbName} values(123456789, 'Alice', 0, 'Beijing', 25, 0, 13812345678, 'No. 123 Street, Beijing', '2022-01-01 10:00:00')," + + " (234567890, 'Bob', 0, 'Shanghai', 30, 1, 13998765432, 'No. 456 Street, Shanghai', '2022-02-02 12:00:00')," + + " (345678901, 'Carol', 1, 'Guangzhou', 28, 0, 13724681357, 'No. 789 Street, Guangzhou', '2022-03-03 14:00:00')," + + " (456789012, 'Dave', 0, 'Shenzhen', 35, 1, 13680864279, 'No. 987 Street, Shenzhen', '2022-04-04 16:00:00')," + + " (567890123, 'Eve', 0, 'Chengdu', 27, 0, 13572468091, 'No. 654 Street, Chengdu', '2022-05-05 18:00:00')," + + " (678901234, 'Frank', 1, 'Hangzhou', 32, 1, 13467985213, 'No. 321 Street, Hangzhou', '2022-06-06 20:00:00')," + + " (789012345, 'Grace', 0, 'Xian', 29, 0, 13333333333, 'No. 222 Street, Xian', '2022-07-07 22:00:00');" + + //Test the unique model by modify a key type from BOOLEAN to TINYINT + //TODO java.sql.SQLException: errCode = 2, detailMessage = Can not change BOOLEAN to TINYINT +/* sql initTable + sql initTableData + sql """ alter table ${tbName} MODIFY column is_teacher TINYINT KEY DEFAULT "0" """ + insertSql = "insert into ${tbName} values(123456689, 'Alice', 1, 'Yaan', 25, 0, 13812345678, 'No. 123 Street, Beijing', '2022-01-01 10:00:00'); " + waitForSchemaChangeDone({ + sql getTableStatusSql + time 60 + }, insertSql, true,"${tbName}")*/ + + //Test the unique model by modify a key type from BOOLEAN to SMALLINT + //TODO java.sql.SQLException: errCode = 2, detailMessage = Can not change BOOLEAN to SMALLINT +/* sql initTable + sql initTableData + sql """ alter table ${tbName} MODIFY column is_teacher SMALLINT KEY DEFAULT "0" """ + insertSql = "insert into ${tbName} values(123456689, 'Alice', 1, 'Yaan', 25, 0, 13812345678, 'No. 123 Street, Beijing', '2022-01-01 10:00:00'); " + waitForSchemaChangeDone({ + sql getTableStatusSql + time 60 + }, insertSql, true,"${tbName}")*/ + + //Test the unique model by modify a key type from BOOLEAN to INT + //TODO java.sql.SQLException: errCode = 2, detailMessage = Can not change BOOLEAN to INT +/* sql initTable + sql initTableData + sql """ alter table ${tbName} MODIFY column is_teacher INT KEY DEFAULT "0" """ + insertSql = "insert into ${tbName} values(123456689, 'Alice', 1, 'Yaan', 25, 0, 13812345678, 'No. 123 Street, Beijing', '2022-01-01 10:00:00'); " + waitForSchemaChangeDone({ + sql getTableStatusSql + time 60 + }, insertSql, true,"${tbName}")*/ + + + //Test the unique model by modify a key type from BOOLEAN to BIGINT + //TODO java.sql.SQLException: errCode = 2, detailMessage = Can not change BOOLEAN to BIGINT +/* sql initTable + sql initTableData + sql """ alter table ${tbName} MODIFY column is_teacher BIGINT KEY DEFAULT "0" """ + insertSql = "insert into ${tbName} values(123456689, 'Alice', 1, 'Yaan', 25, 0, 13812345678, 'No. 123 Street, Beijing', '2022-01-01 10:00:00'); " + waitForSchemaChangeDone({ + sql getTableStatusSql + time 60 + }, insertSql, true,"${tbName}")*/ + + + //Test the unique model by modify a key type from BOOLEAN to FLOAT + //TODO Float or double can not used as a key, use decimal instead +/* sql initTable + sql initTableData + sql """ alter table ${tbName} MODIFY column is_teacher FLOAT KEY DEFAULT "0" """ + insertSql = "insert into ${tbName} values(123456689, 'Alice', 1.0, 'Yaan', 25, 0, 13812345678, 'No. 123 Street, Beijing', '2022-01-01 10:00:00'); " + waitForSchemaChangeDone({ + sql getTableStatusSql + time 60 + }, insertSql, true,"${tbName}")*/ + + + //Test the unique model by modify a key type from BOOLEAN to DECIMAL + //TODO java.sql.SQLException: errCode = 2, detailMessage = Can not change BOOLEAN to DECIMAL32 +/* sql initTable + sql initTableData + sql """ alter table ${tbName} MODIFY column is_teacher DECIMAL KEY DEFAULT "0" """ + insertSql = "insert into ${tbName} values(123456689, 'Alice', 1.0, 'Yaan', 25, 0, 13812345678, 'No. 123 Street, Beijing', '2022-01-01 10:00:00'); " + waitForSchemaChangeDone({ + sql getTableStatusSql + time 60 + }, insertSql, true,"${tbName}")*/ + + //Test the unique model by modify a key type from BOOLEAN to CHAR + //TODO java.sql.SQLException: errCode = 2, detailMessage = Can not change BOOLEAN to CHAR +/* sql initTable + sql initTableData + sql """ alter table ${tbName} MODIFY column is_teacher CHAR KEY DEFAULT "0" """ + insertSql = "insert into ${tbName} values(123456689, 'Alice', '1', 'Yaan', 25, 0, 13812345678, 'No. 123 Street, Beijing', '2022-01-01 10:00:00'); " + waitForSchemaChangeDone({ + sql getTableStatusSql + time 60 + }, insertSql, true,"${tbName}")*/ + + //Test the unique model by modify a key type from BOOLEAN to STRING + //TODO java.sql.SQLException: errCode = 2, detailMessage = String Type should not be used in key column[is_teacher]. +/* sql initTable + sql initTableData + sql """ alter table ${tbName} MODIFY column is_teacher STRING KEY DEFAULT "0" """ + insertSql = "insert into ${tbName} values(123456689, 'Alice', '1', 'Yaan', 25, 0, 13812345678, 'No. 123 Street, Beijing', '2022-01-01 10:00:00'); " + waitForSchemaChangeDone({ + sql getTableStatusSql + time 60 + }, insertSql, true,"${tbName}")*/ + + //Test the unique model by modify a key type from BOOLEAN to VARCHAR + //TODO java.sql.SQLException: errCode = 2, detailMessage = Can not change BOOLEAN to VARCHAR +/* sql initTable + sql initTableData + sql """ alter table ${tbName} MODIFY column is_teacher VARCHAR(32) KEY DEFAULT "0" """ + insertSql = "insert into ${tbName} values(123456689, 'Alice', '1', 'Yaan', 25, 0, 13812345678, 'No. 123 Street, Beijing', '2022-01-01 10:00:00'); " + waitForSchemaChangeDone({ + sql getTableStatusSql + time 60 + }, insertSql, true,"${tbName}")*/ + + } + diff --git a/regression-test/suites/statistics/test_update_rows_and_partition_first_load.groovy b/regression-test/suites/statistics/test_update_rows_and_partition_first_load.groovy new file mode 100644 index 00000000000000..5bfa58abd621f3 --- /dev/null +++ b/regression-test/suites/statistics/test_update_rows_and_partition_first_load.groovy @@ -0,0 +1,137 @@ +// 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. + +suite("test_update_rows_and_partition_first_load", "p2") { + + String ak = getS3AK() + String sk = getS3SK() + String enabled = context.config.otherConfigs.get("enableBrokerLoad") + if (enabled != null && enabled.equalsIgnoreCase("true")) { + sql """DROP DATABASE IF EXISTS test_update_rows_and_partition_first_load""" + sql """CREATE DATABASE test_update_rows_and_partition_first_load""" + sql """use test_update_rows_and_partition_first_load""" + sql """ + CREATE TABLE update_rows_test1 ( + id int NULL, + name String NULL + )ENGINE=OLAP + DUPLICATE KEY(`id`) + DISTRIBUTED BY HASH(`id`) BUCKETS 3 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + sql """ + CREATE TABLE update_rows_test2 ( + id int NULL, + name String NULL + )ENGINE=OLAP + DUPLICATE KEY(`id`) + DISTRIBUTED BY HASH(`id`) BUCKETS 3 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + sql """ + CREATE TABLE `partition_test1` ( + `id` INT NOT NULL, + `name` VARCHAR(25) NOT NULL + ) ENGINE=OLAP + DUPLICATE KEY(`id`) + COMMENT 'OLAP' + PARTITION BY RANGE(`id`) + (PARTITION p1 VALUES [("0"), ("100")), + PARTITION p2 VALUES [("100"), ("200")), + PARTITION p3 VALUES [("200"), ("300"))) + DISTRIBUTED BY HASH(`id`) BUCKETS 1 + PROPERTIES ( + "replication_num" = "1"); + """ + + sql """ + CREATE TABLE `partition_test2` ( + `id` INT NOT NULL, + `name` VARCHAR(25) NOT NULL + ) ENGINE=OLAP + DUPLICATE KEY(`id`) + COMMENT 'OLAP' + PARTITION BY RANGE(`id`) + (PARTITION p1 VALUES [("0"), ("100")), + PARTITION p2 VALUES [("100"), ("200")), + PARTITION p3 VALUES [("200"), ("300"))) + DISTRIBUTED BY HASH(`id`) BUCKETS 1 + PROPERTIES ( + "replication_num" = "1"); + """ + + sql """analyze table update_rows_test1 with sync""" + sql """analyze table update_rows_test2 with sync""" + sql """analyze table partition_test1 with sync""" + sql """analyze table partition_test2 with sync""" + + def label = "part_" + UUID.randomUUID().toString().replace("-", "0") + sql """ + LOAD LABEL ${label} ( + DATA INFILE("s3://doris-build-1308700295/regression/load/data/update_rows_1.csv") + INTO TABLE update_rows_test1 + COLUMNS TERMINATED BY ",", + DATA INFILE("s3://doris-build-1308700295/regression/load/data/update_rows_2.csv") + INTO TABLE update_rows_test2 + COLUMNS TERMINATED BY ",", + DATA INFILE("s3://doris-build-1308700295/regression/load/data/update_rows_1.csv") + INTO TABLE partition_test1 + COLUMNS TERMINATED BY ",", + DATA INFILE("s3://doris-build-1308700295/regression/load/data/update_rows_2.csv") + INTO TABLE partition_test2 + COLUMNS TERMINATED BY "," + ) + WITH S3 ( + "AWS_ACCESS_KEY" = "$ak", + "AWS_SECRET_KEY" = "$sk", + "AWS_ENDPOINT" = "cos.ap-beijing.myqcloud.com", + "AWS_REGION" = "ap-beijing" + ); + """ + + boolean finished = false; + for(int i = 0; i < 120; i++) { + def result = sql """show load where label = "$label" """ + if (result[0][2] == "FINISHED") { + finished = true; + break; + } + logger.info("Load not finished, wait one second.") + Thread.sleep(1000) + } + if (finished) { + def result = sql """show table stats update_rows_test1""" + assertEquals("5", result[0][0]) + result = sql """show table stats update_rows_test2""" + assertEquals("6", result[0][0]) + result = sql """show table stats partition_test1""" + assertEquals("5", result[0][0]) + assertEquals("true", result[0][6]) + result = sql """show table stats partition_test2""" + assertEquals("true", result[0][6]) + assertEquals("6", result[0][0]) + } + sql """DROP DATABASE IF EXISTS test_update_rows_and_partition_first_load""" + } +} + diff --git a/regression-test/suites/workload_manager_p0/test_curd_wlg.groovy b/regression-test/suites/workload_manager_p0/test_curd_wlg.groovy index c9ba3671403b97..4b4fa51486ee01 100644 --- a/regression-test/suites/workload_manager_p0/test_curd_wlg.groovy +++ b/regression-test/suites/workload_manager_p0/test_curd_wlg.groovy @@ -314,6 +314,7 @@ suite("test_crud_wlg") { sql "alter workload group test_group properties ( 'max_queue_size'='1' );" sql "alter workload group test_group properties ( 'queue_timeout'='500' );" + Thread.sleep(10000) test { sql "select /*+SET_VAR(parallel_fragment_exec_instance_num=1)*/ * from ${table_name};" @@ -321,7 +322,8 @@ suite("test_crud_wlg") { } sql "alter workload group test_group properties ( 'max_concurrency'='10' );" - sql "select 1;" + Thread.sleep(10000) + sql "select /*+SET_VAR(parallel_fragment_exec_instance_num=1)*/ * from ${table_name};" sql "set workload_group=normal;" sql "drop workload group test_group;" }