diff --git a/be/src/exec/hdfs_scanner_text.cpp b/be/src/exec/hdfs_scanner_text.cpp index aa7b1a93345d4..69dda3cd86e46 100644 --- a/be/src/exec/hdfs_scanner_text.cpp +++ b/be/src/exec/hdfs_scanner_text.cpp @@ -308,6 +308,7 @@ Status HdfsTextScanner::parse_csv(int chunk_size, ChunkPtr* chunk) { csv::Converter::Options options; // Use to custom Hive array format + options.is_hive = true; options.array_format_type = csv::ArrayFormatType::kHive; options.array_hive_collection_delimiter = _collection_delimiter; options.array_hive_mapkey_delimiter = _mapkey_delimiter; diff --git a/be/src/exec/pipeline/fragment_executor.cpp b/be/src/exec/pipeline/fragment_executor.cpp index c07ce9cd5afb6..4a885772af880 100644 --- a/be/src/exec/pipeline/fragment_executor.cpp +++ b/be/src/exec/pipeline/fragment_executor.cpp @@ -135,6 +135,9 @@ Status FragmentExecutor::_prepare_query_ctx(ExecEnv* exec_env, const UnifiedExec if (query_options.__isset.enable_profile && query_options.enable_profile) { _query_ctx->set_enable_profile(); } + if (query_options.__isset.big_query_profile_second_threshold) { + _query_ctx->set_big_query_profile_threshold(query_options.big_query_profile_second_threshold); + } if (query_options.__isset.pipeline_profile_level) { _query_ctx->set_profile_level(query_options.pipeline_profile_level); } diff --git a/be/src/exec/pipeline/query_context.h b/be/src/exec/pipeline/query_context.h index 2a5fea2864d8a..36a470d29d42b 100644 --- a/be/src/exec/pipeline/query_context.h +++ b/be/src/exec/pipeline/query_context.h @@ -89,7 +89,18 @@ class QueryContext : public std::enable_shared_from_this { duration_cast(steady_clock::now().time_since_epoch() + _query_expire_seconds).count(); } void set_enable_profile() { _enable_profile = true; } - bool enable_profile() { return _enable_profile; } + bool enable_profile() { + if (_enable_profile) { + return true; + } + if (_big_query_profile_threshold_ns <= 0) { + return false; + } + return MonotonicNanos() - _query_begin_time > _big_query_profile_threshold_ns; + } + void set_big_query_profile_threshold(int64_t big_query_profile_threshold_s) { + _big_query_profile_threshold_ns = 1'000'000'000L * big_query_profile_threshold_s; + } void set_runtime_profile_report_interval(int64_t runtime_profile_report_interval_s) { _runtime_profile_report_interval_ns = 1'000'000'000L * runtime_profile_report_interval_s; } @@ -202,6 +213,7 @@ class QueryContext : public std::enable_shared_from_this { std::once_flag _init_mem_tracker_once; std::shared_ptr _profile; bool _enable_profile = false; + int64_t _big_query_profile_threshold_ns = 0; int64_t _runtime_profile_report_interval_ns = std::numeric_limits::max(); TPipelineProfileLevel::type _profile_level; std::shared_ptr _mem_tracker; diff --git a/be/src/formats/csv/converter.h b/be/src/formats/csv/converter.h index 99bc95f869914..4a5af93dc65f0 100644 --- a/be/src/formats/csv/converter.h +++ b/be/src/formats/csv/converter.h @@ -59,6 +59,7 @@ class Converter { // TODO: user configurable. bool bool_alpha = true; + bool is_hive = false; // Here used to control array parse format. // Considering Hive array format is different from traditional array format, // so here we provide some variables to customize array format, and you can diff --git a/be/src/formats/csv/string_converter.cpp b/be/src/formats/csv/string_converter.cpp index 2d87a1b345f07..0c7db4c3cff53 100644 --- a/be/src/formats/csv/string_converter.cpp +++ b/be/src/formats/csv/string_converter.cpp @@ -60,12 +60,24 @@ bool StringConverter::read_string(Column* column, const Slice& s, const Options& max_size = options.type_desc->len; } - if (config::enable_check_string_lengths && - ((s.size > TypeDescriptor::MAX_VARCHAR_LENGTH) || (max_size > 0 && s.size > max_size))) { + bool length_check_status = true; + // Hive table, not limit string length <= 1mb anymore + if (options.is_hive) { + if (UNLIKELY(max_size > 0 && s.size > max_size)) { + length_check_status = false; + } + } else { + if ((config::enable_check_string_lengths && + ((s.size > TypeDescriptor::MAX_VARCHAR_LENGTH) || (max_size > 0 && s.size > max_size)))) { + length_check_status = false; + } + } + if (!length_check_status) { VLOG(3) << strings::Substitute("Column [$0]'s length exceed max varchar length. str_size($1), max_size($2)", column->get_name(), s.size, max_size); return false; } + down_cast(column)->append(s); return true; } @@ -107,8 +119,20 @@ bool StringConverter::read_quoted_string(Column* column, const Slice& tmp_s, con max_size = options.type_desc->len; } size_t ext_size = new_size - old_size; - if (config::enable_check_string_lengths && - ((ext_size > TypeDescriptor::MAX_VARCHAR_LENGTH) || (max_size > 0 && ext_size > max_size))) { + + bool length_check_status = true; + // Hive table, not limit string length <= 1mb anymore + if (options.is_hive) { + if (max_size > 0 && ext_size > max_size) { + length_check_status = false; + } + } else { + if (config::enable_check_string_lengths && + ((ext_size > TypeDescriptor::MAX_VARCHAR_LENGTH) || (max_size > 0 && ext_size > max_size))) { + length_check_status = false; + } + } + if (!length_check_status) { bytes.resize(old_size); VLOG(3) << strings::Substitute( "Column [$0]'s length exceed max varchar length. old_size($1), new_size($2), ext_size($3), " @@ -116,6 +140,7 @@ bool StringConverter::read_quoted_string(Column* column, const Slice& tmp_s, con column->get_name(), old_size, new_size, ext_size, max_size); return false; } + offsets.push_back(bytes.size()); return true; } diff --git a/be/src/formats/csv/varbinary_converter.cpp b/be/src/formats/csv/varbinary_converter.cpp index bf8f3e23c0ce5..11178ceaa9e83 100644 --- a/be/src/formats/csv/varbinary_converter.cpp +++ b/be/src/formats/csv/varbinary_converter.cpp @@ -77,8 +77,19 @@ bool VarBinaryConverter::read_string(Column* column, const Slice& s, const Optio len = s.size; } + bool length_check_status = true; // check if length exceed max varbinary length - if (UNLIKELY((len > TypeDescriptor::MAX_VARCHAR_LENGTH) || (max_size != -1 && len > max_size))) { + if (options.is_hive) { + if (UNLIKELY(max_size != -1 && len > max_size)) { + length_check_status = false; + } + } else { + if (config::enable_check_string_lengths && + ((len > TypeDescriptor::MAX_VARCHAR_LENGTH) || (max_size != -1 && len > max_size))) { + length_check_status = false; + } + } + if (!length_check_status) { LOG(WARNING) << "Column [" << column->get_name() << "]'s length exceed max varbinary length."; return false; } diff --git a/be/src/service/service_be/lake_service.cpp b/be/src/service/service_be/lake_service.cpp index 0299d946b35a9..6c5d7ad3b8866 100644 --- a/be/src/service/service_be/lake_service.cpp +++ b/be/src/service/service_be/lake_service.cpp @@ -147,30 +147,21 @@ void LakeServiceImpl::publish_version(::google::protobuf::RpcController* control return; } - auto enable_trace = config::lake_enable_publish_version_trace_log; auto start_ts = butil::gettimeofday_us(); auto thread_pool = publish_version_thread_pool(_env); auto latch = BThreadCountDownLatch(request->tablet_ids_size()); bthread::Mutex response_mtx; - Trace* trace = nullptr; - scoped_refptr trace_gurad; - - if (enable_trace) { - trace_gurad = scoped_refptr(new Trace()); - trace = trace_gurad.get(); - TRACE_TO(trace, "got request. txn_id=$0 new_version=$1 #tablets=$2", request->txn_ids(0), - request->new_version(), request->tablet_ids_size()); - } + scoped_refptr trace_gurad = scoped_refptr(new Trace()); + Trace* trace = trace_gurad.get(); + TRACE_TO(trace, "got request. txn_id=$0 new_version=$1 #tablets=$2", request->txn_ids(0), request->new_version(), + request->tablet_ids_size()); for (auto tablet_id : request->tablet_ids()) { auto task = [&, tablet_id]() { DeferOp defer([&] { latch.count_down(); }); - Trace* sub_trace = nullptr; - if (enable_trace) { - scoped_refptr child_trace(new Trace); - sub_trace = child_trace.get(); - trace->AddChildTrace("PublishTablet", sub_trace); - } + scoped_refptr child_trace(new Trace); + Trace* sub_trace = child_trace.get(); + trace->AddChildTrace("PublishTablet", sub_trace); ADOPT_TRACE(sub_trace); TRACE("start publish tablet $0 at thread $1", tablet_id, Thread::current_thread()->tid()); @@ -182,6 +173,7 @@ void LakeServiceImpl::publish_version(::google::protobuf::RpcController* control auto commit_time = request->commit_time(); g_publish_tablet_version_queuing_latency << (run_ts - start_ts); + TRACE_COUNTER_INCREMENT("tablet_id", tablet_id); auto res = lake::publish_version(_tablet_mgr, tablet_id, base_version, new_version, txns, commit_time); if (res.ok()) { auto metadata = std::move(res).value(); @@ -213,11 +205,11 @@ void LakeServiceImpl::publish_version(::google::protobuf::RpcController* control latch.wait(); auto cost = butil::gettimeofday_us() - start_ts; auto is_slow = cost >= config::lake_publish_version_slow_log_ms * 1000; - if (enable_trace && is_slow) { + if (config::lake_enable_publish_version_trace_log && is_slow) { LOG(INFO) << "Published txn " << request->txn_ids(0) << ". cost=" << cost << "us\n" << trace->DumpToString(); } else if (is_slow) { LOG(INFO) << "Published txn " << request->txn_ids(0) << ". #tablets=" << request->tablet_ids_size() - << " cost=" << cost << "us"; + << " cost=" << cost << "us, trace: " << trace->MetricsAsJSON(); } TEST_SYNC_POINT("LakeServiceImpl::publish_version:return"); } diff --git a/be/src/storage/CMakeLists.txt b/be/src/storage/CMakeLists.txt index 227340c6f2a63..35cb4729fc752 100644 --- a/be/src/storage/CMakeLists.txt +++ b/be/src/storage/CMakeLists.txt @@ -195,6 +195,7 @@ add_library(Storage STATIC lake/meta_file.cpp lake/lake_primary_index.cpp lake/vertical_compaction_task.cpp + lake/metacache.cpp binlog_util.cpp binlog_file_writer.cpp binlog_file_reader.cpp diff --git a/be/src/storage/compaction_manager.cpp b/be/src/storage/compaction_manager.cpp index 17d5de2564311..9a18015807c92 100644 --- a/be/src/storage/compaction_manager.cpp +++ b/be/src/storage/compaction_manager.cpp @@ -27,7 +27,7 @@ namespace starrocks { CompactionManager::CompactionManager() : _next_task_id(0) {} -CompactionManager::~CompactionManager() { +void CompactionManager::stop() { _stop.store(true, std::memory_order_release); if (_scheduler_thread.joinable()) { _scheduler_thread.join(); diff --git a/be/src/storage/compaction_manager.h b/be/src/storage/compaction_manager.h index 228246661dddb..959c42cb05b0e 100644 --- a/be/src/storage/compaction_manager.h +++ b/be/src/storage/compaction_manager.h @@ -38,7 +38,9 @@ class CompactionManager { public: CompactionManager(); - ~CompactionManager(); + ~CompactionManager() = default; + + void stop(); void init_max_task_num(int32_t num); diff --git a/be/src/storage/delta_writer.cpp b/be/src/storage/delta_writer.cpp index 3287672236097..7a4fd159466fb 100644 --- a/be/src/storage/delta_writer.cpp +++ b/be/src/storage/delta_writer.cpp @@ -371,7 +371,7 @@ Status DeltaWriter::write(const Chunk& chunk, const uint32_t* indexes, uint32_t _opt.tablet_id, _replica_state_name(_replica_state))); } - if (!_mem_table->check_supported_column_partial_update(chunk)) { + if (_tablet->keys_type() == KeysType::PRIMARY_KEYS && !_mem_table->check_supported_column_partial_update(chunk)) { return Status::InternalError( fmt::format("can't partial update for column with row. tablet_id: {}", _opt.tablet_id)); } diff --git a/be/src/storage/lake/lake_primary_index.cpp b/be/src/storage/lake/lake_primary_index.cpp index e61d87c5eb59a..f0e441f3ba46f 100644 --- a/be/src/storage/lake/lake_primary_index.cpp +++ b/be/src/storage/lake/lake_primary_index.cpp @@ -29,6 +29,7 @@ static bvar::LatencyRecorder g_load_pk_index_latency("lake_load_pk_index"); Status LakePrimaryIndex::lake_load(Tablet* tablet, const TabletMetadata& metadata, int64_t base_version, const MetaFileBuilder* builder) { + TRACE_COUNTER_SCOPE_LATENCY_US("primary_index_load_latency_us"); std::lock_guard lg(_lock); if (_loaded) { return _status; diff --git a/be/src/storage/lake/meta_file.cpp b/be/src/storage/lake/meta_file.cpp index 97475e549056f..17721904a03ba 100644 --- a/be/src/storage/lake/meta_file.cpp +++ b/be/src/storage/lake/meta_file.cpp @@ -17,10 +17,11 @@ #include #include "fs/fs_util.h" -#include "gutil/strings/escaping.h" #include "storage/del_vector.h" #include "storage/lake/location_provider.h" +#include "storage/lake/metacache.h" #include "storage/lake/update_manager.h" +#include "storage/protobuf_file.h" #include "util/coding.h" #include "util/defer_op.h" #include "util/raw_container.h" @@ -223,7 +224,7 @@ void MetaFileBuilder::_fill_delvec_cache() { // find delvec ptr by segment id auto delvec_iter = _segmentid_to_delvec.find(cache_item.second); if (delvec_iter != _segmentid_to_delvec.end() && delvec_iter->second != nullptr) { - _tablet.tablet_mgr()->cache_delvec(cache_item.first, delvec_iter->second); + _tablet.tablet_mgr()->metacache()->cache_delvec(cache_item.first, delvec_iter->second); } } } @@ -236,100 +237,47 @@ void MetaFileBuilder::handle_failure() { } } -MetaFileReader::MetaFileReader(const std::string& filepath, bool fill_cache) { - RandomAccessFileOptions opts{.skip_fill_local_cache = !fill_cache}; - auto rf = fs::new_random_access_file(opts, filepath); - if (rf.ok()) { - _access_file = std::move(*rf); - } else { - _err_status = rf.status(); - } - _tablet_meta = std::make_unique(); - _load = false; -} - -Status MetaFileReader::load() { - if (_access_file == nullptr) return _err_status; - - ASSIGN_OR_RETURN(auto file_size, _access_file->get_size()); - if (file_size <= 4) { - return Status::Corruption( - fmt::format("meta file {} is corrupt, invalid file size {}", _access_file->filename(), file_size)); - } - std::string metadata_str; - raw::stl_string_resize_uninitialized(&metadata_str, file_size); - RETURN_IF_ERROR(_access_file->read_at_fully(0, metadata_str.data(), file_size)); - bool parsed = _tablet_meta->ParseFromArray(metadata_str.data(), static_cast(file_size)); - if (!parsed) { - return Status::Corruption(fmt::format("failed to parse tablet meta {}", _access_file->filename())); - } - _load = true; - TRACE("end load tablet metadata"); - return Status::OK(); -} - -Status MetaFileReader::load_by_cache(const std::string& filepath, TabletManager* tablet_mgr) { - // 1. lookup meta cache first - if (auto ptr = tablet_mgr->lookup_tablet_metadata(filepath); ptr != nullptr) { - _tablet_meta = ptr; - _load = true; - return Status::OK(); - } else { - // 2. load directly - return load(); - } -} - -Status MetaFileReader::get_del_vec(TabletManager* tablet_mgr, uint32_t segment_id, DelVector* delvec) { - if (_access_file == nullptr) return _err_status; - if (!_load) return Status::InternalError("meta file reader not loaded"); +Status get_del_vec(TabletManager* tablet_mgr, const TabletMetadata& metadata, uint32_t segment_id, DelVector* delvec) { // find delvec by segment id - auto iter = _tablet_meta->delvec_meta().delvecs().find(segment_id); - if (iter != _tablet_meta->delvec_meta().delvecs().end()) { - VLOG(2) << fmt::format("MetaFileReader get_del_vec {} segid {}", _tablet_meta->delvec_meta().ShortDebugString(), - segment_id); + auto iter = metadata.delvec_meta().delvecs().find(segment_id); + if (iter != metadata.delvec_meta().delvecs().end()) { + VLOG(2) << fmt::format("get_del_vec {} segid {}", metadata.delvec_meta().ShortDebugString(), segment_id); std::string buf; raw::stl_string_resize_uninitialized(&buf, iter->second.size()); // find in cache - std::string cache_key = delvec_cache_key(_tablet_meta->id(), iter->second); - DelVectorPtr delvec_cache_ptr = tablet_mgr->lookup_delvec(cache_key); - if (delvec_cache_ptr != nullptr) { - delvec->copy_from(*delvec_cache_ptr); + std::string cache_key = delvec_cache_key(metadata.id(), iter->second); + auto cached_delvec = tablet_mgr->metacache()->lookup_delvec(cache_key); + if (cached_delvec != nullptr) { + delvec->copy_from(*cached_delvec); return Status::OK(); } // lookup delvec file name and then read it - auto iter2 = _tablet_meta->delvec_meta().version_to_file().find(iter->second.version()); - if (iter2 == _tablet_meta->delvec_meta().version_to_file().end()) { - LOG(ERROR) << "Can't find delvec file name for tablet: " << _tablet_meta->id() + auto iter2 = metadata.delvec_meta().version_to_file().find(iter->second.version()); + if (iter2 == metadata.delvec_meta().version_to_file().end()) { + LOG(ERROR) << "Can't find delvec file name for tablet: " << metadata.id() << ", version: " << iter->second.version(); return Status::InternalError("Can't find delvec file name"); } const auto& delvec_name = iter2->second.name(); RandomAccessFileOptions opts{.skip_fill_local_cache = true}; - ASSIGN_OR_RETURN(auto rf, fs::new_random_access_file( - opts, tablet_mgr->delvec_location(_tablet_meta->id(), delvec_name))); + ASSIGN_OR_RETURN(auto rf, + fs::new_random_access_file(opts, tablet_mgr->delvec_location(metadata.id(), delvec_name))); RETURN_IF_ERROR(rf->read_at_fully(iter->second.offset(), buf.data(), iter->second.size())); // parse delvec RETURN_IF_ERROR(delvec->load(iter->second.version(), buf.data(), iter->second.size())); // put in cache - delvec_cache_ptr = std::make_shared(); + auto delvec_cache_ptr = std::make_shared(); delvec_cache_ptr->copy_from(*delvec); - tablet_mgr->cache_delvec(cache_key, delvec_cache_ptr); + tablet_mgr->metacache()->cache_delvec(cache_key, delvec_cache_ptr); TRACE("end load delvec"); return Status::OK(); } - VLOG(2) << fmt::format("MetaFileReader get_del_vec not found, segmentid {} tablet_meta {}", segment_id, - _tablet_meta->delvec_meta().ShortDebugString()); + VLOG(2) << fmt::format("get_del_vec not found, segmentid {} tablet_meta {}", segment_id, + metadata.delvec_meta().ShortDebugString()); return Status::OK(); } -StatusOr MetaFileReader::get_meta() { - if (_access_file == nullptr) return _err_status; - if (!_load) return Status::InternalError("meta file reader not loaded"); - return std::move(_tablet_meta); -} - bool is_primary_key(TabletMetadata* metadata) { return metadata->schema().keys_type() == KeysType::PRIMARY_KEYS; } diff --git a/be/src/storage/lake/meta_file.h b/be/src/storage/lake/meta_file.h index dcb47b185075c..7e9f2d5dc7c50 100644 --- a/be/src/storage/lake/meta_file.h +++ b/be/src/storage/lake/meta_file.h @@ -79,24 +79,7 @@ class MetaFileBuilder { std::shared_ptr> _trash_files; }; -class MetaFileReader { -public: - explicit MetaFileReader(const std::string& filepath, bool fill_cache); - ~MetaFileReader() {} - // load tablet meta from file - Status load(); - // try to load tablet meta from cache first, if not exist then load from file - Status load_by_cache(const std::string& filepath, TabletManager* tablet_mgr); - Status get_del_vec(TabletManager* tablet_mgr, uint32_t segment_id, DelVector* delvec); - StatusOr get_meta(); - -private: - std::unique_ptr _access_file; - std::shared_ptr _tablet_meta; - Status _err_status; - bool _load; -}; - +Status get_del_vec(TabletManager* tablet_mgr, const TabletMetadata& metadata, uint32_t segment_id, DelVector* delvec); bool is_primary_key(TabletMetadata* metadata); bool is_primary_key(const TabletMetadata& metadata); diff --git a/be/src/storage/lake/metacache.cpp b/be/src/storage/lake/metacache.cpp new file mode 100644 index 0000000000000..ef2aa26991db8 --- /dev/null +++ b/be/src/storage/lake/metacache.cpp @@ -0,0 +1,238 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// Licensed 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 +// +// https://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 "storage/lake/metacache.h" + +#include + +#include "gen_cpp/lake_types.pb.h" +#include "storage/del_vector.h" +#include "storage/lake/tablet_manager.h" +#include "storage/rowset/segment.h" +#include "util/lru_cache.h" + +namespace starrocks::lake { + +static bvar::Adder g_metadata_cache_hit; +static bvar::Window> g_metadata_cache_hit_minute("lake", "metadata_cache_hit_minute", + &g_metadata_cache_hit, 60); + +static bvar::Adder g_metadata_cache_miss; +static bvar::Window> g_metadata_cache_miss_minute("lake", "metadata_cache_miss_minute", + &g_metadata_cache_miss, 60); + +static bvar::Adder g_txnlog_cache_hit; +static bvar::Window> g_txnlog_cache_hit_minute("lake", "txn_log_cache_hit_minute", + &g_txnlog_cache_hit, 60); + +static bvar::Adder g_txnlog_cache_miss; +static bvar::Window> g_txnlog_cache_miss_minute("lake", "txn_log_cache_miss_minute", + &g_txnlog_cache_miss, 60); + +static bvar::Adder g_schema_cache_hit; +static bvar::Window> g_schema_cache_hit_minute("lake", "schema_cache_hit_minute", + &g_schema_cache_hit, 60); + +static bvar::Adder g_schema_cache_miss; +static bvar::Window> g_schema_cache_miss_minute("lake", "schema_cache_miss_minute", + &g_schema_cache_miss, 60); + +static bvar::Adder g_dv_cache_hit; +static bvar::Window> g_dv_cache_hit_minute("lake", "delvec_cache_hit_minute", &g_dv_cache_hit, + 60); + +static bvar::Adder g_dv_cache_miss; +static bvar::Window> g_dv_cache_miss_minute("lake", "delvec_cache_miss_minute", &g_dv_cache_miss, + 60); + +static bvar::Adder g_segment_cache_hit; +static bvar::Window> g_segment_cache_hit_minute("lake", "segment_cache_hit_minute", + &g_segment_cache_hit, 60); + +static bvar::Adder g_segment_cache_miss; +static bvar::Window> g_segment_cache_miss_minute("lake", "segment_cache_miss_minute", + &g_segment_cache_miss, 60); + +#ifndef BE_TEST +static Metacache* get_metacache() { + auto mgr = ExecEnv::GetInstance()->lake_tablet_manager(); + return (mgr != nullptr) ? mgr->metacache() : nullptr; +} + +static size_t get_metacache_capacity(void*) { + auto cache = get_metacache(); + return (cache != nullptr) ? cache->capacity() : 0; +} + +static size_t get_metacache_usage(void*) { + auto cache = get_metacache(); + return (cache != nullptr) ? cache->memory_usage() : 0; +} + +static bvar::PassiveStatus g_metacache_capacity("lake", "metacache_capacity", get_metacache_capacity, nullptr); +static bvar::PassiveStatus g_metacache_usage("lake", "metacache_usage", get_metacache_usage, nullptr); +#endif + +Metacache::Metacache(int64_t cache_capacity) : _cache(new_lru_cache(cache_capacity)) {} + +Metacache::~Metacache() = default; + +void Metacache::insert(std::string_view key, CacheValue* ptr, size_t size) { + Cache::Handle* handle = _cache->insert(CacheKey(key), ptr, size, cache_value_deleter); + _cache->release(handle); +} + +std::shared_ptr Metacache::lookup_tablet_metadata(std::string_view key) { + auto handle = _cache->lookup(CacheKey(key)); + if (handle == nullptr) { + g_metadata_cache_miss << 1; + return nullptr; + } + DeferOp defer([this, handle]() { _cache->release(handle); }); + + try { + auto value = static_cast(_cache->value(handle)); + auto metadata = std::get>(*value); + g_metadata_cache_hit << 1; + return metadata; + } catch (const std::bad_variant_access& e) { + return nullptr; + } +} + +std::shared_ptr Metacache::lookup_txn_log(std::string_view key) { + auto handle = _cache->lookup(CacheKey(key)); + if (handle == nullptr) { + g_txnlog_cache_miss << 1; + return nullptr; + } + DeferOp defer([this, handle]() { _cache->release(handle); }); + + try { + auto value = static_cast(_cache->value(handle)); + auto log = std::get>(*value); + g_txnlog_cache_hit << 1; + return log; + } catch (const std::bad_variant_access& e) { + return nullptr; + } +} + +std::shared_ptr Metacache::lookup_tablet_schema(std::string_view key) { + auto handle = _cache->lookup(CacheKey(key)); + if (handle == nullptr) { + g_schema_cache_miss << 1; + return nullptr; + } + DeferOp defer([this, handle]() { _cache->release(handle); }); + + try { + auto value = static_cast(_cache->value(handle)); + auto schema = std::get>(*value); + g_schema_cache_hit << 1; + return schema; + } catch (const std::bad_variant_access& e) { + return nullptr; + } +} + +std::shared_ptr Metacache::lookup_segment(std::string_view key) { + auto handle = _cache->lookup(CacheKey(key)); + if (handle == nullptr) { + g_segment_cache_miss << 1; + return nullptr; + } + DeferOp defer([this, handle]() { _cache->release(handle); }); + + try { + auto value = static_cast(_cache->value(handle)); + auto segment = std::get>(*value); + g_segment_cache_hit << 1; + return segment; + } catch (const std::bad_variant_access& e) { + return nullptr; + } +} + +std::shared_ptr Metacache::lookup_delvec(std::string_view key) { + auto handle = _cache->lookup(CacheKey(key)); + if (handle == nullptr) { + g_dv_cache_miss << 1; + return nullptr; + } + DeferOp defer([this, handle]() { _cache->release(handle); }); + + try { + auto value = static_cast(_cache->value(handle)); + auto delvec = std::get>(*value); + g_dv_cache_hit << 1; + return delvec; + } catch (const std::bad_variant_access& e) { + return nullptr; + } +} + +void Metacache::cache_segment(std::string_view key, std::shared_ptr segment) { + auto mem_cost = segment->mem_usage(); + auto value = std::make_unique(std::move(segment)); + insert(key, value.release(), mem_cost); +} + +void Metacache::cache_delvec(std::string_view key, std::shared_ptr delvec) { + auto mem_cost = delvec->memory_usage(); + auto value = std::make_unique(std::move(delvec)); + insert(key, value.release(), mem_cost); +} + +void Metacache::cache_tablet_metadata(std::string_view key, std::shared_ptr metadata) { + auto value_ptr = std::make_unique(metadata); + insert(key, value_ptr.release(), metadata->SpaceUsedLong()); +} + +void Metacache::cache_txn_log(std::string_view key, std::shared_ptr log) { + auto value_ptr = std::make_unique(log); + insert(key, value_ptr.release(), log->SpaceUsedLong()); +} + +void Metacache::cache_tablet_schema(std::string_view key, std::shared_ptr schema, size_t size) { + auto cache_value = std::make_unique(schema); + insert(key, cache_value.release(), size); +} + +void Metacache::erase(std::string_view key) { + _cache->erase(CacheKey(key)); +} + +void Metacache::update_capacity(size_t new_capacity) { + size_t old_capacity = _cache->get_capacity(); + int64_t delta = (int64_t)new_capacity - (int64_t)old_capacity; + if (delta != 0) { + (void)_cache->adjust_capacity(delta); + VLOG(5) << "Changed metadache capacity from " << old_capacity << " to " << _cache->get_capacity(); + } +} + +void Metacache::prune() { + _cache->prune(); +} + +size_t Metacache::memory_usage() const { + return _cache->get_memory_usage(); +} + +size_t Metacache::capacity() const { + return _cache->get_capacity(); +} + +} // namespace starrocks::lake diff --git a/be/src/storage/lake/metacache.h b/be/src/storage/lake/metacache.h new file mode 100644 index 0000000000000..fbe9a47cf5368 --- /dev/null +++ b/be/src/storage/lake/metacache.h @@ -0,0 +1,85 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// Licensed 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 +// +// https://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 "gutil/macros.h" + +namespace starrocks { +class Cache; +class CacheKey; +class DelVector; +class Segment; +class TabletSchema; +} // namespace starrocks + +namespace starrocks::lake { + +class TabletMetadataPB; +class TxnLogPB; + +using CacheValue = + std::variant, std::shared_ptr, + std::shared_ptr, std::shared_ptr, std::shared_ptr>; + +class Metacache { +public: + explicit Metacache(int64_t cache_capacity); + + ~Metacache(); + + DISALLOW_COPY_AND_MOVE(Metacache); + + std::shared_ptr lookup_tablet_metadata(std::string_view key); + + std::shared_ptr lookup_txn_log(std::string_view key); + + std::shared_ptr lookup_tablet_schema(std::string_view key); + + std::shared_ptr lookup_segment(std::string_view key); + + std::shared_ptr lookup_delvec(std::string_view key); + + void cache_tablet_metadata(std::string_view key, std::shared_ptr metadata); + + void cache_tablet_schema(std::string_view key, std::shared_ptr schema, size_t size); + + void cache_txn_log(std::string_view key, std::shared_ptr log); + + void cache_segment(std::string_view key, std::shared_ptr segment); + + void cache_delvec(std::string_view key, std::shared_ptr delvec); + + void erase(std::string_view key); + + void update_capacity(size_t new_capacity); + + void prune(); + + size_t memory_usage() const; + + size_t capacity() const; + +private: + static void cache_value_deleter(const CacheKey& /*key*/, void* value) { delete static_cast(value); } + + void insert(std::string_view key, CacheValue* ptr, size_t size); + + std::unique_ptr _cache; +}; + +} // namespace starrocks::lake diff --git a/be/src/storage/lake/rowset_update_state.cpp b/be/src/storage/lake/rowset_update_state.cpp index ec3480d15b890..9e0ee97d031c9 100644 --- a/be/src/storage/lake/rowset_update_state.cpp +++ b/be/src/storage/lake/rowset_update_state.cpp @@ -451,6 +451,7 @@ Status RowsetUpdateState::_prepare_partial_update_states(const TxnLogPB_OpWrite& Status RowsetUpdateState::rewrite_segment(const TxnLogPB_OpWrite& op_write, const TabletMetadata& metadata, Tablet* tablet, std::map* replace_segments, std::vector* orphan_files) { + TRACE_COUNTER_SCOPE_LATENCY_US("rewrite_segment_latency_us"); const RowsetMetadata& rowset_meta = op_write.rowset(); auto root_path = tablet->metadata_root_location(); ASSIGN_OR_RETURN(auto fs, FileSystem::CreateSharedFromString(root_path)); diff --git a/be/src/storage/lake/schema_change.cpp b/be/src/storage/lake/schema_change.cpp index 1d231d3d0cef2..07c44d3aa5079 100644 --- a/be/src/storage/lake/schema_change.cpp +++ b/be/src/storage/lake/schema_change.cpp @@ -349,9 +349,7 @@ Status SchemaChangeHandler::process_update_tablet_meta(const TUpdateTabletMetaIn Status SchemaChangeHandler::do_process_update_tablet_meta(const TTabletMetaInfo& tablet_meta_info, int64_t txn_id) { if (tablet_meta_info.meta_type != TTabletMetaType::ENABLE_PERSISTENT_INDEX) { - // Only support ENABLE_PERSISTENT_INDEX for now - LOG(WARNING) << "not supported update meta type: " << tablet_meta_info.meta_type; - return Status::InternalError(fmt::format("not supported update meta type: {}", tablet_meta_info.meta_type)); + return Status::InternalError(fmt::format("unsupported update meta type: {}", tablet_meta_info.meta_type)); } MonotonicStopWatch timer; diff --git a/be/src/storage/lake/tablet.cpp b/be/src/storage/lake/tablet.cpp index e4b74cc60ce19..cb14e7c754c68 100644 --- a/be/src/storage/lake/tablet.cpp +++ b/be/src/storage/lake/tablet.cpp @@ -20,6 +20,7 @@ #include "runtime/exec_env.h" #include "storage/lake/filenames.h" #include "storage/lake/general_tablet_writer.h" +#include "storage/lake/metacache.h" #include "storage/lake/metadata_iterator.h" #include "storage/lake/pk_tablet_writer.h" #include "storage/lake/rowset.h" @@ -33,8 +34,8 @@ Status Tablet::put_metadata(const TabletMetadata& metadata) { return _mgr->put_tablet_metadata(metadata); } -Status Tablet::put_metadata(TabletMetadataPtr metadata) { - return _mgr->put_tablet_metadata(std::move(metadata)); +Status Tablet::put_metadata(const TabletMetadataPtr& metadata) { + return _mgr->put_tablet_metadata(metadata); } StatusOr Tablet::get_metadata(int64_t version) { @@ -69,8 +70,8 @@ Status Tablet::put_txn_log(const TxnLog& log) { return _mgr->put_txn_log(log); } -Status Tablet::put_txn_log(TxnLogPtr log) { - return _mgr->put_txn_log(std::move(log)); +Status Tablet::put_txn_log(const TxnLogPtr& log) { + return _mgr->put_txn_log(log); } StatusOr Tablet::get_txn_log(int64_t txn_id) { @@ -132,13 +133,13 @@ StatusOr> Tablet::get_rowsets(const TabletMetadata& metad StatusOr Tablet::load_segment(std::string_view segment_name, int seg_id, size_t* footer_size_hint, bool fill_data_cache, bool fill_metadata_cache) { auto segment_path = segment_location(segment_name); - auto segment = _mgr->lookup_segment(segment_path); + auto segment = _mgr->metacache()->lookup_segment(segment_path); if (segment == nullptr) { ASSIGN_OR_RETURN(auto tablet_schema, get_schema()); ASSIGN_OR_RETURN(auto fs, FileSystem::CreateSharedFromString(segment_path)); segment = std::make_shared(std::move(fs), segment_path, seg_id, std::move(tablet_schema), _mgr); if (fill_metadata_cache) { - _mgr->cache_segment(segment_path, segment); + _mgr->metacache()->cache_segment(segment_path, segment); } } // segment->open will read the footer, and it is time-consuming. diff --git a/be/src/storage/lake/tablet.h b/be/src/storage/lake/tablet.h index 54c75b9ee581f..9afc4bd6646da 100644 --- a/be/src/storage/lake/tablet.h +++ b/be/src/storage/lake/tablet.h @@ -64,7 +64,7 @@ class Tablet { [[nodiscard]] Status put_metadata(const TabletMetadata& metadata); - [[nodiscard]] Status put_metadata(TabletMetadataPtr metadata); + [[nodiscard]] Status put_metadata(const TabletMetadataPtr& metadata); StatusOr get_metadata(int64_t version); @@ -76,7 +76,7 @@ class Tablet { [[nodiscard]] Status put_txn_log(const TxnLog& log); - [[nodiscard]] Status put_txn_log(TxnLogPtr log); + [[nodiscard]] Status put_txn_log(const TxnLogPtr& log); StatusOr get_txn_log(int64_t txn_id); diff --git a/be/src/storage/lake/tablet_manager.cpp b/be/src/storage/lake/tablet_manager.cpp index 64d48823de3ec..189380d6cb553 100644 --- a/be/src/storage/lake/tablet_manager.cpp +++ b/be/src/storage/lake/tablet_manager.cpp @@ -18,10 +18,9 @@ #include #include -#include +#include #include -#include "agent/agent_server.h" #include "agent/master_info.h" #include "common/compiler_util.h" #include "fmt/format.h" @@ -35,6 +34,7 @@ #include "storage/lake/join_path.h" #include "storage/lake/location_provider.h" #include "storage/lake/meta_file.h" +#include "storage/lake/metacache.h" #include "storage/lake/tablet.h" #include "storage/lake/tablet_metadata.h" #include "storage/lake/txn_log.h" @@ -46,7 +46,6 @@ #include "storage/protobuf_file.h" #include "storage/tablet_schema_map.h" #include "testutil/sync_point.h" -#include "util/lru_cache.h" #include "util/raw_container.h" #include "util/trace.h" @@ -56,76 +55,15 @@ #endif namespace starrocks::lake { - -static bvar::Adder g_metadata_cache_hit; -static bvar::Window> g_metadata_cache_hit_minute("lake", "metadata_cache_hit_minute", - &g_metadata_cache_hit, 60); - -static bvar::Adder g_metadata_cache_miss; -static bvar::Window> g_metadata_cache_miss_minute("lake", "metadata_cache_miss_minute", - &g_metadata_cache_miss, 60); - -static bvar::Adder g_txnlog_cache_hit; -static bvar::Window> g_txnlog_cache_hit_minute("lake", "txn_log_cache_hit_minute", - &g_txnlog_cache_hit, 60); - -static bvar::Adder g_txnlog_cache_miss; -static bvar::Window> g_txnlog_cache_miss_minute("lake", "txn_log_cache_miss_minute", - &g_txnlog_cache_miss, 60); - -static bvar::Adder g_schema_cache_hit; -static bvar::Window> g_schema_cache_hit_minute("lake", "schema_cache_hit_minute", - &g_schema_cache_hit, 60); - -static bvar::Adder g_schema_cache_miss; -static bvar::Window> g_schema_cache_miss_minute("lake", "schema_cache_miss_minute", - &g_schema_cache_miss, 60); - -static bvar::Adder g_dv_cache_hit; -static bvar::Window> g_dv_cache_hit_minute("lake", "delvec_cache_hit_minute", &g_dv_cache_hit, - 60); - -static bvar::Adder g_dv_cache_miss; -static bvar::Window> g_dv_cache_miss_minute("lake", "delvec_cache_miss_minute", &g_dv_cache_miss, - 60); - -static bvar::Adder g_segment_cache_hit; -static bvar::Window> g_segment_cache_hit_minute("lake", "segment_cache_hit_minute", - &g_segment_cache_hit, 60); - -static bvar::Adder g_segment_cache_miss; -static bvar::Window> g_segment_cache_miss_minute("lake", "segment_cache_miss_minute", - &g_segment_cache_miss, 60); - static bvar::LatencyRecorder g_get_tablet_metadata_latency("lake", "get_tablet_metadata"); static bvar::LatencyRecorder g_put_tablet_metadata_latency("lake", "put_tablet_metadata"); static bvar::LatencyRecorder g_get_txn_log_latency("lake", "get_txn_log"); static bvar::LatencyRecorder g_put_txn_log_latency("lake", "put_txn_log"); static bvar::LatencyRecorder g_del_txn_log_latency("lake", "del_txn_log"); -#ifndef BE_TEST -static Cache* get_metacache() { - auto mgr = ExecEnv::GetInstance()->lake_tablet_manager(); - return (mgr != nullptr) ? mgr->metacache() : nullptr; -} - -static size_t get_metacache_capacity(void*) { - auto cache = get_metacache(); - return (cache != nullptr) ? cache->get_capacity() : 0; -} - -static size_t get_metacache_usage(void*) { - auto cache = get_metacache(); - return (cache != nullptr) ? cache->get_memory_usage() : 0; -} - -static bvar::PassiveStatus g_metacache_capacity("lake", "metacache_capacity", get_metacache_capacity, nullptr); -static bvar::PassiveStatus g_metacache_usage("lake", "metacache_usage", get_metacache_usage, nullptr); -#endif - TabletManager::TabletManager(LocationProvider* location_provider, UpdateManager* update_mgr, int64_t cache_capacity) : _location_provider(location_provider), - _metacache(new_lru_cache(cache_capacity)), + _metacache(std::make_unique(cache_capacity)), _compaction_scheduler(std::make_unique(this)), _update_mgr(update_mgr) { _update_mgr->set_tablet_mgr(this); @@ -177,123 +115,15 @@ std::string TabletManager::tablet_latest_metadata_cache_key(int64_t tablet_id) { return fmt::format("TL{}", tablet_id); } -void TabletManager::fill_metacache(std::string_view key, CacheValue* ptr, size_t size) { - Cache::Handle* handle = _metacache->insert(CacheKey(key), ptr, size, cache_value_deleter); - if (handle == nullptr) { - delete ptr; - } else { - _metacache->release(handle); - } -} - -TabletMetadataPtr TabletManager::lookup_tablet_metadata(std::string_view key) { - auto handle = _metacache->lookup(CacheKey(key)); - if (handle == nullptr) { - g_metadata_cache_miss << 1; - return nullptr; - } - g_metadata_cache_hit << 1; - auto value = static_cast(_metacache->value(handle)); - auto metadata = std::get(*value); - _metacache->release(handle); - return metadata; -} - -TabletMetadataPtr TabletManager::lookup_tablet_latest_metadata(std::string_view key) { - auto handle = _metacache->lookup(CacheKey(key)); - if (handle == nullptr) { - g_metadata_cache_miss << 1; - return nullptr; - } - g_metadata_cache_hit << 1; - auto value = static_cast(_metacache->value(handle)); - auto metadata = std::get(*value); - _metacache->release(handle); - return metadata; -} - -void TabletManager::cache_tablet_latest_metadata(TabletMetadataPtr metadata) { - auto value_ptr = std::make_unique(metadata); - fill_metacache(tablet_latest_metadata_cache_key(metadata->id()), value_ptr.release(), metadata->SpaceUsedLong()); -} - -TabletSchemaPtr TabletManager::lookup_tablet_schema(std::string_view key) { - auto handle = _metacache->lookup(CacheKey(key)); - if (handle == nullptr) { - g_schema_cache_miss << 1; - return nullptr; - } - g_schema_cache_hit << 1; - auto value = static_cast(_metacache->value(handle)); - auto schema = std::get(*value); - _metacache->release(handle); - return schema; -} - -TxnLogPtr TabletManager::lookup_txn_log(std::string_view key) { - auto handle = _metacache->lookup(CacheKey(key)); - if (handle == nullptr) { - g_txnlog_cache_miss << 1; - return nullptr; - } - g_txnlog_cache_hit << 1; - auto value = static_cast(_metacache->value(handle)); - auto log = std::get(*value); - _metacache->release(handle); - return log; -} - -SegmentPtr TabletManager::lookup_segment(std::string_view key) { - auto handle = _metacache->lookup(CacheKey(key)); - if (handle == nullptr) { - g_segment_cache_miss << 1; - return nullptr; - } - g_segment_cache_hit << 1; - auto value = static_cast(_metacache->value(handle)); - auto segment = std::get(*value); - _metacache->release(handle); - return segment; -} - -void TabletManager::cache_segment(std::string_view key, SegmentPtr segment) { - auto mem_cost = segment->mem_usage(); - auto value = std::make_unique(std::move(segment)); - fill_metacache(key, value.release(), mem_cost); -} - // current lru cache does not support updating value size, so use refill to update. void TabletManager::update_segment_cache_size(std::string_view key) { // use write lock to protect parallel segment size update std::unique_lock wrlock(_meta_lock); - auto segment = lookup_segment(key); + auto segment = _metacache->lookup_segment(key); if (segment == nullptr) { return; } - cache_segment(key, std::move(segment)); -} - -DelVectorPtr TabletManager::lookup_delvec(std::string_view key) { - auto handle = _metacache->lookup(CacheKey(key)); - if (handle == nullptr) { - g_dv_cache_miss << 1; - return nullptr; - } - g_dv_cache_hit << 1; - auto value = static_cast(_metacache->value(handle)); - auto delvec = std::get(*value); - _metacache->release(handle); - return delvec; -} - -void TabletManager::cache_delvec(std::string_view key, DelVectorPtr delvec) { - auto mem_cost = delvec->memory_usage(); - auto value = std::make_unique(std::move(delvec)); - fill_metacache(key, value.release(), mem_cost); -} - -void TabletManager::erase_metacache(std::string_view key) { - _metacache->erase(CacheKey(key)); + _metacache->cache_segment(key, std::move(segment)); } void TabletManager::prune_metacache() { @@ -351,7 +181,7 @@ StatusOr TabletManager::get_tablet(int64_t tablet_id) { return tablet; } -Status TabletManager::put_tablet_metadata(TabletMetadataPtr metadata) { +Status TabletManager::put_tablet_metadata(const TabletMetadataPtr& metadata) { TEST_ERROR_POINT("TabletManager::put_tablet_metadata"); // write metadata file auto t0 = butil::gettimeofday_us(); @@ -360,10 +190,9 @@ Status TabletManager::put_tablet_metadata(TabletMetadataPtr metadata) { ProtobufFile file(filepath); RETURN_IF_ERROR(file.save(*metadata)); - // put into metacache - auto value_ptr = std::make_unique(metadata); - fill_metacache(filepath, value_ptr.release(), metadata->SpaceUsedLong()); - cache_tablet_latest_metadata(metadata); + _metacache->cache_tablet_metadata(filepath, metadata); + _metacache->cache_tablet_metadata(tablet_latest_metadata_cache_key(metadata->id()), metadata); + auto t1 = butil::gettimeofday_us(); g_put_tablet_metadata_latency << (t1 - t0); TRACE("end write tablet metadata"); @@ -386,7 +215,7 @@ StatusOr TabletManager::load_tablet_metadata(const string& me } TabletMetadataPtr TabletManager::get_latest_cached_tablet_metadata(int64_t tablet_id) { - return lookup_tablet_latest_metadata(tablet_latest_metadata_cache_key(tablet_id)); + return _metacache->lookup_tablet_metadata(tablet_latest_metadata_cache_key(tablet_id)); } StatusOr TabletManager::get_tablet_metadata(int64_t tablet_id, int64_t version) { @@ -394,14 +223,13 @@ StatusOr TabletManager::get_tablet_metadata(int64_t tablet_id } StatusOr TabletManager::get_tablet_metadata(const string& path, bool fill_cache) { - if (auto ptr = lookup_tablet_metadata(path); ptr != nullptr) { + if (auto ptr = _metacache->lookup_tablet_metadata(path); ptr != nullptr) { TRACE("got cached tablet metadata"); return ptr; } ASSIGN_OR_RETURN(auto ptr, load_tablet_metadata(path, fill_cache)); if (fill_cache) { - auto value_ptr = std::make_unique(ptr); - fill_metacache(path, value_ptr.release(), ptr->SpaceUsedLong()); + _metacache->cache_tablet_metadata(path, ptr); } TRACE("end read tablet metadata"); return ptr; @@ -409,7 +237,7 @@ StatusOr TabletManager::get_tablet_metadata(const string& pat Status TabletManager::delete_tablet_metadata(int64_t tablet_id, int64_t version) { auto location = tablet_metadata_location(tablet_id, version); - erase_metacache(location); + _metacache->erase(location); return fs::delete_file(location); } @@ -451,14 +279,13 @@ StatusOr TabletManager::get_txn_vlog(const std::string& path, bool fi } StatusOr TabletManager::get_txn_log(const std::string& path, bool fill_cache) { - if (auto ptr = lookup_txn_log(path); ptr != nullptr) { + if (auto ptr = _metacache->lookup_txn_log(path); ptr != nullptr) { TRACE("got cached txn log"); return ptr; } ASSIGN_OR_RETURN(auto ptr, load_txn_log(path, fill_cache)); if (fill_cache) { - auto value_ptr = std::make_unique(ptr); - fill_metacache(path, value_ptr.release(), ptr->SpaceUsedLong()); + _metacache->cache_txn_log(path, ptr); } TRACE("end load txn log"); return ptr; @@ -472,7 +299,7 @@ StatusOr TabletManager::get_txn_vlog(int64_t tablet_id, int64_t versi return get_txn_log(txn_vlog_location(tablet_id, version), false); } -Status TabletManager::put_txn_log(TxnLogPtr log) { +Status TabletManager::put_txn_log(const TxnLogPtr& log) { if (UNLIKELY(!log->has_tablet_id())) { return Status::InvalidArgument("txn log does not have tablet id"); } @@ -485,9 +312,8 @@ Status TabletManager::put_txn_log(TxnLogPtr log) { ProtobufFile file(txn_log_path); RETURN_IF_ERROR(file.save(*log)); - // put txnlog into cache - auto value_ptr = std::make_unique(log); - fill_metacache(txn_log_path, value_ptr.release(), log->SpaceUsedLong()); + _metacache->cache_txn_log(txn_log_path, log); + auto t1 = butil::gettimeofday_us(); g_put_txn_log_latency << (t1 - t0); return Status::OK(); @@ -543,7 +369,7 @@ bool TabletManager::is_tablet_in_worker(int64_t tablet_id) { StatusOr TabletManager::get_tablet_schema(int64_t tablet_id, int64_t* version_hint) { // 1. direct lookup in cache, if there is schema info for the tablet auto cache_key = tablet_schema_cache_key(tablet_id); - auto ptr = lookup_tablet_schema(cache_key); + auto ptr = _metacache->lookup_tablet_schema(cache_key); RETURN_IF(ptr != nullptr, ptr); // Cache miss, load tablet metadata from remote storage use the hint version @@ -599,16 +425,15 @@ StatusOr TabletManager::get_tablet_schema(int64_t tablet_id, in return Status::InternalError(fmt::format("tablet schema {} failed to emplace in TabletSchemaMap", tablet_id)); } - // Save the schema into the in-memory cache - auto cache_value = std::make_unique(schema); auto cache_size = inserted ? schema->mem_usage() : 0; - fill_metacache(cache_key, cache_value.release(), cache_size); + _metacache->cache_tablet_schema(cache_key, schema, cache_size); + return schema; } StatusOr TabletManager::get_tablet_schema_by_index_id(int64_t tablet_id, int64_t index_id) { auto global_cache_key = global_schema_cache_key(index_id); - auto schema = lookup_tablet_schema(global_cache_key); + auto schema = _metacache->lookup_tablet_schema(global_cache_key); TEST_SYNC_POINT_CALLBACK("get_tablet_schema_by_index_id.1", &schema); if (schema != nullptr) { return schema; @@ -621,8 +446,7 @@ StatusOr TabletManager::get_tablet_schema_by_index_id(int64_t t VLOG(3) << "Got tablet schema of id " << index_id << " for tablet " << tablet_id; schema = std::move(schema_or).value(); // Save the schema into the in-memory cache, use the schema id as the cache key - auto cache_value = std::make_unique(schema); - fill_metacache(global_cache_key, cache_value.release(), 0); + _metacache->cache_tablet_schema(global_cache_key, schema, 0 /*TODO*/); return std::move(schema); } else { return schema_or.status(); @@ -657,9 +481,8 @@ Status TabletManager::create_schema_file(int64_t tablet_id, const TabletSchemaPB return Status::InternalError("failed to emplace the schema hash map"); } auto cache_key = global_schema_cache_key(schema_pb.id()); - auto cache_value = std::make_unique(schema); auto cache_size = inserted ? schema->mem_usage() : 0; - fill_metacache(cache_key, cache_value.release(), cache_size); + _metacache->cache_tablet_schema(cache_key, schema, cache_size /*TODO*/); return Status::OK(); } @@ -675,12 +498,7 @@ StatusOr TabletManager::load_and_parse_schema_file(const std::s } void TabletManager::update_metacache_limit(size_t new_capacity) { - size_t old_capacity = _metacache->get_capacity(); - int64_t delta = (int64_t)new_capacity - (int64_t)old_capacity; - if (delta != 0) { - (void)_metacache->adjust_capacity(delta); - VLOG(5) << "Changed metadache capacity from " << old_capacity << " to " << _metacache->get_capacity(); - } + _metacache->update_capacity(new_capacity); } int64_t TabletManager::in_writing_data_size(int64_t tablet_id) { @@ -712,8 +530,7 @@ void TabletManager::remove_in_writing_data_size(int64_t tablet_id, int64_t txn_i void TabletManager::TEST_set_global_schema_cache(int64_t schema_id, TabletSchemaPtr schema) { auto cache_key = global_schema_cache_key(schema_id); - auto cache_value = std::make_unique(schema); - fill_metacache(cache_key, cache_value.release(), 0); + _metacache->cache_tablet_schema(cache_key, std::move(schema), 0); } } // namespace starrocks::lake diff --git a/be/src/storage/lake/tablet_manager.h b/be/src/storage/lake/tablet_manager.h index d45c783fdc037..bc903a3d4e021 100644 --- a/be/src/storage/lake/tablet_manager.h +++ b/be/src/storage/lake/tablet_manager.h @@ -27,8 +27,6 @@ #include "storage/lake/types_fwd.h" namespace starrocks { -class Cache; -class CacheKey; class Segment; class TabletSchemaPB; class TCreateTabletReq; @@ -43,11 +41,11 @@ using TabletMetadataIter = MetadataIterator; using TxnLogIter = MetadataIterator; class CompactionScheduler; +class Metacache; class TabletManager { friend class Tablet; friend class MetaFileBuilder; - friend class MetaFileReader; public: // Does NOT take the ownership of |location_provider| and |location_provider| must outlive @@ -68,7 +66,7 @@ class TabletManager { [[nodiscard]] Status put_tablet_metadata(const TabletMetadata& metadata); - [[nodiscard]] Status put_tablet_metadata(TabletMetadataPtr metadata); + [[nodiscard]] Status put_tablet_metadata(const TabletMetadataPtr& metadata); StatusOr get_tablet_metadata(int64_t tablet_id, int64_t version); @@ -82,7 +80,7 @@ class TabletManager { [[nodiscard]] Status put_txn_log(const TxnLog& log); - [[nodiscard]] Status put_txn_log(TxnLogPtr log); + [[nodiscard]] Status put_txn_log(const TxnLogPtr& log); StatusOr get_txn_log(int64_t tablet_id, int64_t txn_id); @@ -130,7 +128,7 @@ class TabletManager { void update_metacache_limit(size_t limit); // The return value will never be null. - Cache* metacache() { return _metacache.get(); } + Metacache* metacache() { return _metacache.get(); } StatusOr get_tablet_data_size(int64_t tablet_id, int64_t* version_hint); @@ -146,12 +144,9 @@ class TabletManager { void update_segment_cache_size(std::string_view key); private: - using CacheValue = std::variant; - static std::string global_schema_cache_key(int64_t index_id); static std::string tablet_schema_cache_key(int64_t tablet_id); static std::string tablet_latest_metadata_cache_key(int64_t tablet_id); - static void cache_value_deleter(const CacheKey& /*key*/, void* value) { delete static_cast(value); } Status create_schema_file(int64_t tablet_id, const TabletSchemaPB& schema_pb); StatusOr load_and_parse_schema_file(const std::string& path); @@ -161,23 +156,8 @@ class TabletManager { StatusOr load_tablet_metadata(const std::string& metadata_location, bool fill_cache); StatusOr load_txn_log(const std::string& txn_log_location, bool fill_cache); - /// Cache operations - void fill_metacache(std::string_view key, CacheValue* ptr, size_t size); - void erase_metacache(std::string_view key); - - TabletMetadataPtr lookup_tablet_metadata(std::string_view key); - TxnLogPtr lookup_txn_log(std::string_view key); - TabletSchemaPtr lookup_tablet_schema(std::string_view key); - SegmentPtr lookup_segment(std::string_view key); - void cache_segment(std::string_view key, SegmentPtr segment); - DelVectorPtr lookup_delvec(std::string_view key); - void cache_delvec(std::string_view key, DelVectorPtr delvec); - // only store tablet's latest metadata - TabletMetadataPtr lookup_tablet_latest_metadata(std::string_view key); - void cache_tablet_latest_metadata(TabletMetadataPtr metadata); - LocationProvider* _location_provider; - std::unique_ptr _metacache; + std::unique_ptr _metacache; std::unique_ptr _compaction_scheduler; UpdateManager* _update_mgr; diff --git a/be/src/storage/lake/tablet_metadata.h b/be/src/storage/lake/tablet_metadata.h index 734d30209da52..73cbc26e17626 100644 --- a/be/src/storage/lake/tablet_metadata.h +++ b/be/src/storage/lake/tablet_metadata.h @@ -21,6 +21,7 @@ namespace starrocks::lake { using TabletMetadata = TabletMetadataPB; -using TabletMetadataPtr = std::shared_ptr; +using TabletMetadataPtr = std::shared_ptr; +using MutableTabletMetadataPtr = std::shared_ptr; } // namespace starrocks::lake diff --git a/be/src/storage/lake/transactions.cpp b/be/src/storage/lake/transactions.cpp index 61c2b02d44e5d..3330245679204 100644 --- a/be/src/storage/lake/transactions.cpp +++ b/be/src/storage/lake/transactions.cpp @@ -15,6 +15,7 @@ #include "storage/lake/transactions.h" #include "fs/fs_util.h" +#include "storage/lake/metacache.h" #include "storage/lake/tablet.h" #include "storage/lake/tablet_manager.h" #include "storage/lake/txn_log.h" @@ -106,7 +107,7 @@ StatusOr publish_version(TabletManager* tablet_mgr, int64_t t files_to_delete.emplace_back(log_path); - tablet_mgr->metacache()->erase(CacheKey(log_path)); + tablet_mgr->metacache()->erase(log_path); } // Apply vtxn logs for schema change @@ -134,7 +135,7 @@ StatusOr publish_version(TabletManager* tablet_mgr, int64_t t files_to_delete.emplace_back(vlog_path); - tablet_mgr->metacache()->erase(CacheKey(vlog_path)); + tablet_mgr->metacache()->erase(vlog_path); } } @@ -171,7 +172,7 @@ Status publish_log_version(TabletManager* tablet_mgr, int64_t tablet_id, int64_t return st; } else { delete_files_async({txn_log_path}); - tablet_mgr->metacache()->erase(CacheKey(txn_log_path)); + tablet_mgr->metacache()->erase(txn_log_path); return Status::OK(); } } @@ -211,7 +212,7 @@ void abort_txn(TabletManager* tablet_mgr, int64_t tablet_id, std::spanmetacache()->erase(CacheKey(log_path)); + tablet_mgr->metacache()->erase(log_path); } delete_files_async(std::move(files_to_delete)); diff --git a/be/src/storage/lake/txn_log_applier.cpp b/be/src/storage/lake/txn_log_applier.cpp index 09b5a57c2f185..3c02318f69509 100644 --- a/be/src/storage/lake/txn_log_applier.cpp +++ b/be/src/storage/lake/txn_log_applier.cpp @@ -36,7 +36,7 @@ class PrimaryKeyTxnLogApplier : public TxnLogApplier { phmap::priv::Allocator, 4, std::mutex, true>; public: - PrimaryKeyTxnLogApplier(Tablet tablet, std::shared_ptr metadata, int64_t new_version) + PrimaryKeyTxnLogApplier(Tablet tablet, MutableTabletMetadataPtr metadata, int64_t new_version) : _tablet(tablet), _metadata(std::move(metadata)), _base_version(_metadata->version()), @@ -194,7 +194,7 @@ class PrimaryKeyTxnLogApplier : public TxnLogApplier { static inline ParallelSet _s_schema_change_set; Tablet _tablet; - std::shared_ptr _metadata; + MutableTabletMetadataPtr _metadata; int64_t _base_version{0}; int64_t _new_version{0}; int64_t _max_txn_id{0}; // Used as the file name prefix of the delvec file @@ -205,7 +205,7 @@ class PrimaryKeyTxnLogApplier : public TxnLogApplier { class NonPrimaryKeyTxnLogApplier : public TxnLogApplier { public: - NonPrimaryKeyTxnLogApplier(Tablet tablet, std::shared_ptr metadata, int64_t new_version) + NonPrimaryKeyTxnLogApplier(Tablet tablet, MutableTabletMetadataPtr metadata, int64_t new_version) : _tablet(tablet), _metadata(std::move(metadata)), _new_version(new_version) {} Status apply(const TxnLogPB& log) override { @@ -344,11 +344,12 @@ class NonPrimaryKeyTxnLogApplier : public TxnLogApplier { } Tablet _tablet; - std::shared_ptr _metadata; + MutableTabletMetadataPtr _metadata; int64_t _new_version; }; -std::unique_ptr new_txn_log_applier(Tablet tablet, TabletMetadataPtr metadata, int64_t new_version) { +std::unique_ptr new_txn_log_applier(Tablet tablet, MutableTabletMetadataPtr metadata, + int64_t new_version) { if (metadata->schema().keys_type() == PRIMARY_KEYS) { return std::make_unique(tablet, std::move(metadata), new_version); } diff --git a/be/src/storage/lake/txn_log_applier.h b/be/src/storage/lake/txn_log_applier.h index d5e582edae487..c3ce1a5df683e 100644 --- a/be/src/storage/lake/txn_log_applier.h +++ b/be/src/storage/lake/txn_log_applier.h @@ -39,6 +39,7 @@ class TxnLogApplier { virtual std::shared_ptr> trash_files() = 0; }; -std::unique_ptr new_txn_log_applier(Tablet tablet, TabletMetadataPtr metadata, int64_t new_version); +std::unique_ptr new_txn_log_applier(Tablet tablet, MutableTabletMetadataPtr metadata, + int64_t new_version); } // namespace starrocks::lake \ No newline at end of file diff --git a/be/src/storage/lake/update_compaction_state.cpp b/be/src/storage/lake/update_compaction_state.cpp index c47ab125d7ac1..f2b71175086bd 100644 --- a/be/src/storage/lake/update_compaction_state.cpp +++ b/be/src/storage/lake/update_compaction_state.cpp @@ -21,6 +21,7 @@ #include "storage/lake/update_manager.h" #include "storage/primary_key_encoder.h" #include "storage/tablet_manager.h" +#include "util/trace.h" namespace starrocks::lake { @@ -36,6 +37,7 @@ CompactionState::~CompactionState() { } Status CompactionState::load_segments(Rowset* rowset, const TabletSchemaCSPtr& tablet_schema, uint32_t segment_id) { + TRACE_COUNTER_SCOPE_LATENCY_US("load_segments_latency_us"); if (segment_id >= pk_cols.size() && pk_cols.size() != 0) { std::string msg = strings::Substitute("Error segment id: $0 vs $1", segment_id, pk_cols.size()); LOG(WARNING) << msg; diff --git a/be/src/storage/lake/update_compaction_state.h b/be/src/storage/lake/update_compaction_state.h index 1c5591651e627..19974fb7b745d 100644 --- a/be/src/storage/lake/update_compaction_state.h +++ b/be/src/storage/lake/update_compaction_state.h @@ -39,6 +39,8 @@ class CompactionState { Status load_segments(Rowset* rowset, const TabletSchemaCSPtr& tablet_schema, uint32_t segment_id); void release_segments(uint32_t segment_id); + std::size_t memory_usage() const { return _memory_usage; } + std::vector pk_cols; private: diff --git a/be/src/storage/lake/update_manager.cpp b/be/src/storage/lake/update_manager.cpp index a06b8bbd023f4..4ddb1c1bcec96 100644 --- a/be/src/storage/lake/update_manager.cpp +++ b/be/src/storage/lake/update_manager.cpp @@ -79,6 +79,7 @@ StatusOr UpdateManager::prepare_primary_index(const TabletMetadata& } Status UpdateManager::commit_primary_index(IndexEntry* index_entry, Tablet* tablet) { + TRACE_COUNTER_SCOPE_LATENCY_US("primary_index_commit_latency_us"); if (index_entry != nullptr) { auto& index = index_entry->value(); if (index.enable_persistent_index()) { @@ -119,6 +120,7 @@ Status UpdateManager::publish_primary_key_tablet(const TxnLogPB_OpWrite& op_writ DeferOp remove_state_entry([&] { _update_state_cache.remove(state_entry); }); auto& state = state_entry->value(); RETURN_IF_ERROR(state.load(op_write, metadata, base_version, tablet, builder, true)); + TRACE_COUNTER_INCREMENT("state_bytes", state.memory_usage()); _update_state_cache.update_object_size(state_entry, state.memory_usage()); // 2. rewrite segment file if it is partial update std::vector orphan_files; @@ -200,16 +202,17 @@ Status UpdateManager::publish_primary_key_tablet(const TxnLogPB_OpWrite& op_writ builder->apply_opwrite(op_write, replace_segments, orphan_files); TRACE_COUNTER_INCREMENT("rowsetid", rowset_id); - TRACE_COUNTER_INCREMENT("#upserts", upserts.size()); - TRACE_COUNTER_INCREMENT("#deletes", state.deletes().size()); - TRACE_COUNTER_INCREMENT("#new_del", new_del); - TRACE_COUNTER_INCREMENT("#total_del", total_del); + TRACE_COUNTER_INCREMENT("upserts", upserts.size()); + TRACE_COUNTER_INCREMENT("deletes", state.deletes().size()); + TRACE_COUNTER_INCREMENT("new_del", new_del); + TRACE_COUNTER_INCREMENT("total_del", total_del); _print_memory_stats(); return Status::OK(); } Status UpdateManager::_do_update(uint32_t rowset_id, int32_t upsert_idx, const std::vector& upserts, PrimaryIndex& index, int64_t tablet_id, DeletesMap* new_deletes) { + TRACE_COUNTER_SCOPE_LATENCY_US("do_update_latency_us"); return index.upsert(rowset_id + upsert_idx, 0, *upserts[upsert_idx], new_deletes); } @@ -220,6 +223,7 @@ Status UpdateManager::_do_update_with_condition(Tablet* tablet, const TabletMeta const std::vector& upserts, PrimaryIndex& index, int64_t tablet_id, DeletesMap* new_deletes) { CHECK(condition_column >= 0); + TRACE_COUNTER_SCOPE_LATENCY_US("do_update_latency_us"); const auto& tablet_column = tablet_schema->column(condition_column); std::vector read_column_ids; read_column_ids.push_back(condition_column); @@ -290,6 +294,7 @@ Status UpdateManager::_do_update_with_condition(Tablet* tablet, const TabletMeta Status UpdateManager::_handle_index_op(Tablet* tablet, int64_t base_version, const std::function& op) { + TRACE_COUNTER_SCOPE_LATENCY_US("handle_index_op_latency_us"); auto index_entry = _index_cache.get(tablet->id()); if (index_entry == nullptr) { return Status::Uninitialized(fmt::format("Primary index not load yet, tablet_id: {}", tablet->id())); @@ -342,6 +347,7 @@ Status UpdateManager::get_column_values(Tablet* tablet, const TabletMetadata& me std::map>& rowids_by_rssid, vector>* columns, AutoIncrementPartialUpdateState* auto_increment_state) { + TRACE_COUNTER_SCOPE_LATENCY_US("get_column_values_latency_us"); std::stringstream cost_str; MonotonicStopWatch watch; watch.start(); @@ -442,10 +448,9 @@ Status UpdateManager::get_del_vec(const TabletSegmentId& tsid, int64_t version, // get delvec in meta file Status UpdateManager::get_del_vec_in_meta(const TabletSegmentId& tsid, int64_t meta_ver, DelVector* delvec) { - std::string filepath = _location_provider->tablet_metadata_location(tsid.tablet_id, meta_ver); - MetaFileReader reader(filepath, false); - RETURN_IF_ERROR(reader.load_by_cache(filepath, _tablet_mgr)); - RETURN_IF_ERROR(reader.get_del_vec(_tablet_mgr, tsid.segment_id, delvec)); + std::string filepath = _tablet_mgr->tablet_metadata_location(tsid.tablet_id, meta_ver); + ASSIGN_OR_RETURN(auto metadata, _tablet_mgr->get_tablet_metadata(filepath, false)); + RETURN_IF_ERROR(lake::get_del_vec(_tablet_mgr, *metadata, tsid.segment_id, delvec)); return Status::OK(); } @@ -495,9 +500,6 @@ size_t UpdateManager::get_rowset_num_deletes(int64_t tablet_id, int64_t version, Status UpdateManager::publish_primary_compaction(const TxnLogPB_OpCompaction& op_compaction, const TabletMetadata& metadata, Tablet tablet, IndexEntry* index_entry, MetaFileBuilder* builder, int64_t base_version) { - std::stringstream cost_str; - MonotonicStopWatch watch; - watch.start(); auto& index = index_entry->value(); // 1. iterate output rowset, update primary index and generate delvec std::shared_ptr tablet_schema = std::make_shared(metadata.schema()); @@ -520,6 +522,7 @@ Status UpdateManager::publish_primary_compaction(const TxnLogPB_OpCompaction& op // 2. update primary index, and generate delete info. for (size_t i = 0; i < compaction_state->pk_cols.size(); i++) { RETURN_IF_ERROR(compaction_state->load_segments(output_rowset.get(), tablet_schema, i)); + TRACE_COUNTER_INCREMENT("state_bytes", compaction_state->memory_usage()); auto& pk_col = compaction_state->pk_cols[i]; total_rows += pk_col->size(); uint32_t rssid = rowset_id + i; @@ -536,21 +539,19 @@ Status UpdateManager::publish_primary_compaction(const TxnLogPB_OpCompaction& op delvecs.emplace_back(rssid, dv); compaction_state->release_segments(i); } - cost_str << " [generate delvecs] " << watch.elapsed_time(); - watch.reset(); // 3. update TabletMeta and write to meta file for (auto&& each : delvecs) { builder->append_delvec(each.second, each.first); } builder->apply_opcompaction(op_compaction); - cost_str << " [apply meta] " << watch.elapsed_time(); - VLOG(2) << strings::Substitute( - "lake publish_primary_compaction: tablet_id:$0 input_rowset_size:$1 max_rowset_id:$2" - " total_deletes:$3 total_rows:$4 base_ver:$5 new_ver:$6 cost:$7", - tablet.id(), op_compaction.input_rowsets_size(), max_rowset_id, total_deletes, total_rows, base_version, - metadata.version(), cost_str.str()); + TRACE_COUNTER_INCREMENT("rowsetid", rowset_id); + TRACE_COUNTER_INCREMENT("max_rowsetid", max_rowset_id); + TRACE_COUNTER_INCREMENT("output_rows", total_rows); + TRACE_COUNTER_INCREMENT("input_rowsets_size", op_compaction.input_rowsets_size()); + TRACE_COUNTER_INCREMENT("total_del", total_deletes); + _print_memory_stats(); return Status::OK(); diff --git a/be/src/storage/lake/vacuum.cpp b/be/src/storage/lake/vacuum.cpp index ba79700437792..a47c72ed0a4d8 100644 --- a/be/src/storage/lake/vacuum.cpp +++ b/be/src/storage/lake/vacuum.cpp @@ -23,15 +23,14 @@ #include "common/config.h" #include "common/status.h" #include "fs/fs.h" -#include "gutil/stl_util.h" #include "storage/lake/filenames.h" #include "storage/lake/join_path.h" #include "storage/lake/location_provider.h" #include "storage/lake/meta_file.h" +#include "storage/lake/metacache.h" #include "storage/lake/tablet_manager.h" #include "storage/lake/tablet_metadata.h" #include "storage/lake/update_manager.h" -#include "storage/olap_common.h" #include "testutil/sync_point.h" #include "util/defer_op.h" #include "util/raw_container.h" diff --git a/be/src/storage/memtable.cpp b/be/src/storage/memtable.cpp index 734c9f20c00dd..5a5da654e8d44 100644 --- a/be/src/storage/memtable.cpp +++ b/be/src/storage/memtable.cpp @@ -166,21 +166,24 @@ bool MemTable::insert(const Chunk& chunk, const uint32_t* indexes, uint32_t from } bool is_column_with_row = false; - std::unique_ptr schema_without_full_row_column; auto full_row_col = std::make_unique(); - if (_vectorized_schema->field_names().back() == "__row") { - DCHECK_EQ(chunk.num_columns(), _vectorized_schema->num_fields() - 1); - std::vector cids(_vectorized_schema->num_fields() - 1); - for (int i = 0; i < _vectorized_schema->num_fields() - 1; i++) { - cids[i] = i; + if (_keys_type == PRIMARY_KEYS) { + std::unique_ptr schema_without_full_row_column; + if (_vectorized_schema->field_names().back() == "__row") { + DCHECK_EQ(chunk.num_columns(), _vectorized_schema->num_fields() - 1); + std::vector cids(_vectorized_schema->num_fields() - 1); + for (int i = 0; i < _vectorized_schema->num_fields() - 1; i++) { + cids[i] = i; + } + schema_without_full_row_column = std::make_unique(const_cast(_vectorized_schema), cids); + is_column_with_row = true; + // add row column + auto row_encoder = RowStoreEncoderFactory::instance()->get_or_create_encoder(SIMPLE); + (void)row_encoder->encode_chunk_to_full_row_column(*schema_without_full_row_column, chunk, + full_row_col.get()); + } else { + DCHECK_EQ(chunk.num_columns(), _vectorized_schema->num_fields()); } - schema_without_full_row_column = std::make_unique(const_cast(_vectorized_schema), cids); - is_column_with_row = true; - // add row column - auto row_encoder = RowStoreEncoderFactory::instance()->get_or_create_encoder(SIMPLE); - (void)row_encoder->encode_chunk_to_full_row_column(*schema_without_full_row_column, chunk, full_row_col.get()); - } else { - DCHECK_EQ(chunk.num_columns(), _vectorized_schema->num_fields()); } size_t cur_row_count = _chunk->num_rows(); diff --git a/be/src/storage/rowset/column_reader.cpp b/be/src/storage/rowset/column_reader.cpp index f090dd49be1f0..cfcee53e289b3 100644 --- a/be/src/storage/rowset/column_reader.cpp +++ b/be/src/storage/rowset/column_reader.cpp @@ -570,7 +570,7 @@ StatusOr> ColumnReader::new_iterator(ColumnAcces } } -size_t ColumnReader::mem_usage() { +size_t ColumnReader::mem_usage() const { size_t size = sizeof(ColumnReader) + _meta_mem_usage.load(std::memory_order_relaxed); if (_sub_readers != nullptr) { diff --git a/be/src/storage/rowset/column_reader.h b/be/src/storage/rowset/column_reader.h index 9a3d4e30327fd..16426c71046f0 100644 --- a/be/src/storage/rowset/column_reader.h +++ b/be/src/storage/rowset/column_reader.h @@ -154,7 +154,7 @@ class ColumnReader { uint32_t num_rows() const { return _segment->num_rows(); } - size_t mem_usage(); + size_t mem_usage() const; private: const std::string& file_name() const { return _segment->file_name(); } diff --git a/be/src/storage/rowset/segment.cpp b/be/src/storage/rowset/segment.cpp index 88825ed8bb760..764d4119464cf 100644 --- a/be/src/storage/rowset/segment.cpp +++ b/be/src/storage/rowset/segment.cpp @@ -433,7 +433,7 @@ Status Segment::get_short_key_index(std::vector* sk_index_values) { return Status::OK(); } -size_t Segment::_column_index_mem_usage() { +size_t Segment::_column_index_mem_usage() const { size_t size = 0; for (auto& r : _column_readers) { auto& reader = r.second; diff --git a/be/src/storage/rowset/segment.h b/be/src/storage/rowset/segment.h index c4b77f8a53320..49764e8979024 100644 --- a/be/src/storage/rowset/segment.h +++ b/be/src/storage/rowset/segment.h @@ -165,7 +165,9 @@ class Segment : public std::enable_shared_from_this { const ShortKeyIndexDecoder* decoder() const { return _sk_index_decoder.get(); } - size_t mem_usage() { return _basic_info_mem_usage() + _short_key_index_mem_usage() + _column_index_mem_usage(); } + size_t mem_usage() const { + return _basic_info_mem_usage() + _short_key_index_mem_usage() + _column_index_mem_usage(); + } int64_t get_data_size() { auto res = _fs->get_file_size(_fname); @@ -217,9 +219,9 @@ class Segment : public std::enable_shared_from_this { void _reset(); - size_t _basic_info_mem_usage() { return sizeof(Segment) + _fname.size(); } + size_t _basic_info_mem_usage() const { return sizeof(Segment) + _fname.size(); } - size_t _short_key_index_mem_usage() { + size_t _short_key_index_mem_usage() const { size_t size = _sk_index_handle.mem_usage(); if (_sk_index_decoder != nullptr) { size += _sk_index_decoder->mem_usage(); @@ -227,7 +229,7 @@ class Segment : public std::enable_shared_from_this { return size; } - size_t _column_index_mem_usage(); + size_t _column_index_mem_usage() const; // open segment file and read the minimum amount of necessary information (footer) Status _open(size_t* footer_length_hint, const FooterPointerPB* partial_rowset_footer, bool skip_fill_local_cache); diff --git a/be/src/storage/rowset_column_update_state.cpp b/be/src/storage/rowset_column_update_state.cpp index dbaf6651a429e..f9d3dcff40067 100644 --- a/be/src/storage/rowset_column_update_state.cpp +++ b/be/src/storage/rowset_column_update_state.cpp @@ -335,12 +335,12 @@ static StatusOr read_from_source_segment(Rowset* rowset, const Schema& // this function build delta writer for delta column group's file.(end with `.col`) StatusOr> RowsetColumnUpdateState::_prepare_delta_column_group_writer( - Rowset* rowset, const std::shared_ptr& tschema, uint32_t rssid, int64_t ver) { + Rowset* rowset, const std::shared_ptr& tschema, uint32_t rssid, int64_t ver, int idx) { ASSIGN_OR_RETURN(auto fs, FileSystem::CreateSharedFromString(rowset->rowset_path())); ASSIGN_OR_RETURN(auto rowsetid_segid, _find_rowset_seg_id(rssid)); // always 0 file suffix here, because alter table will execute after this version has been applied only. const std::string path = Rowset::delta_column_group_path(rowset->rowset_path(), rowsetid_segid.unique_rowset_id, - rowsetid_segid.segment_id, ver, 0); + rowsetid_segid.segment_id, ver, idx); (void)fs->delete_file(path); // delete .cols if already exist WritableFileOptions opts{.sync_on_close = true}; ASSIGN_OR_RETURN(auto wfile, fs->new_writable_file(opts, path)); @@ -599,6 +599,15 @@ Status RowsetColumnUpdateState::_insert_new_rows(const TabletSchemaCSPtr& tablet return Status::OK(); } +template +static std::vector append_fixed_batch(const std::vector& base_array, size_t offset, size_t batch_size) { + std::vector new_array; + for (int i = offset; i < offset + batch_size && i < base_array.size(); i++) { + new_array.push_back(base_array[i]); + } + return new_array; +} + Status RowsetColumnUpdateState::finalize(Tablet* tablet, Rowset* rowset, uint32_t rowset_id, PersistentIndexMetaPB& index_meta, MemTracker* tracker, vector>& delvecs, PrimaryIndex& index) { @@ -637,11 +646,27 @@ Status RowsetColumnUpdateState::finalize(Tablet* tablet, Rowset* rowset, uint32_ unique_update_column_ids.push_back(uid); } } - auto partial_tschema = TabletSchema::create(tschema, update_column_ids); - Schema partial_schema = ChunkHelper::convert_schema(tschema, update_column_uids); - // rss_id -> delta column group writer - std::map> delta_column_group_writer; + DCHECK(update_column_ids.size() == unique_update_column_ids.size()); + const size_t BATCH_HANDLE_COLUMN_CNT = config::vertical_compaction_max_columns_per_group; + + auto reclaim_update_cache_fn = [&](bool final_step) { + if (final_step || update_column_ids.size() > BATCH_HANDLE_COLUMN_CNT) { + // When final step or need to switch to next batch columns, we should reclaim cache + std::for_each(_update_chunk_cache.begin(), _update_chunk_cache.end(), [&](auto& cache) { + if (cache.get() != nullptr) { + tracker->release(cache->memory_usage()); + cache.reset(nullptr); + } + }); + } + }; + + auto build_writer_fn = [&](uint32_t rssid, const std::shared_ptr& partial_tschema, int idx) { + // we can generate delta column group by new version + return _prepare_delta_column_group_writer(rowset, partial_tschema, rssid, + latest_applied_version.major_number() + 1, idx); + }; // 2. getter all rss_rowid_to_update_rowid, and prepare .col writer by the way // rss_id -> rowid -> std::map rss_rowid_to_update_rowid; @@ -650,14 +675,6 @@ Status RowsetColumnUpdateState::finalize(Tablet* tablet, Rowset* rowset, uint32_ auto rssid = (uint32_t)(each.first >> 32); auto rowid = (uint32_t)(each.first & ROWID_MASK); rss_rowid_to_update_rowid[rssid][rowid] = std::make_pair(upt_id, each.second); - // prepare delta column writers by the way - if (delta_column_group_writer.count(rssid) == 0) { - // we can generate delta column group by new version - ASSIGN_OR_RETURN(auto writer, - _prepare_delta_column_group_writer(rowset, partial_tschema, rssid, - latest_applied_version.major_number() + 1)); - delta_column_group_writer[rssid] = std::move(writer); - } } } cost_str << " [generate delta column group writer] " << watch.elapsed_time(); @@ -668,57 +685,83 @@ Status RowsetColumnUpdateState::finalize(Tablet* tablet, Rowset* rowset, uint32_ int64_t total_finalize_dcg_time = 0; int64_t total_merge_column_time = 0; int64_t update_rows = 0; + int64_t handle_cnt = 0; // 3. read from raw segment file and update file, and generate `.col` files one by one for (const auto& each : rss_rowid_to_update_rowid) { update_rows += each.second.size(); - int64_t t1 = MonotonicMillis(); - ASSIGN_OR_RETURN(auto rowsetid_segid, _find_rowset_seg_id(each.first)); - const std::string seg_path = Rowset::segment_file_path(rowset->rowset_path(), rowsetid_segid.unique_rowset_id, - rowsetid_segid.segment_id); - // 3.1 read from source segment - ASSIGN_OR_RETURN(auto source_chunk_ptr, - read_from_source_segment(rowset, partial_schema, tablet, &stats, - latest_applied_version.major_number(), rowsetid_segid, seg_path)); - const size_t source_chunk_size = source_chunk_ptr->memory_usage(); - tracker->consume(source_chunk_size); - DeferOp tracker_defer([&]() { tracker->release(source_chunk_size); }); - // 3.2 read from update segment - int64_t t2 = MonotonicMillis(); - std::vector rowids; - auto update_chunk_ptr = ChunkHelper::new_chunk(partial_schema, each.second.size()); - RETURN_IF_ERROR(_read_chunk_from_update(each.second, partial_schema, tracker, rowset, &stats, rowids, - update_chunk_ptr.get())); - const size_t update_chunk_size = update_chunk_ptr->memory_usage(); - tracker->consume(update_chunk_size); - DeferOp tracker_defer2([&]() { tracker->release(update_chunk_size); }); - int64_t t3 = MonotonicMillis(); - // 3.3 merge source chunk and update chunk - RETURN_IF_EXCEPTION(source_chunk_ptr->update_rows(*update_chunk_ptr, rowids.data())); - // 3.4 write column to delta column file - int64_t t4 = MonotonicMillis(); - uint64_t segment_file_size = 0; - uint64_t index_size = 0; - uint64_t footer_position = 0; - padding_char_columns(partial_schema, partial_tschema, source_chunk_ptr.get()); - RETURN_IF_ERROR(delta_column_group_writer[each.first]->append_chunk(*source_chunk_ptr)); - RETURN_IF_ERROR( - delta_column_group_writer[each.first]->finalize(&segment_file_size, &index_size, &footer_position)); - int64_t t5 = MonotonicMillis(); - total_seek_source_segment_time += t2 - t1; - total_read_column_from_update_time += t3 - t2; - total_merge_column_time += t4 - t3; - total_finalize_dcg_time += t5 - t4; - // 3.5 generate delta columngroup - _rssid_to_delta_column_group[each.first] = std::make_shared(); // must record unique column id in delta column group - std::vector> dcg_column_ids{unique_update_column_ids}; - std::vector dcg_column_files{file_name(delta_column_group_writer[each.first]->segment_path())}; + // dcg_column_ids and dcg_column_files are mapped one to the other. E.g. + // {{1,2}, {3,4}} -> {"aaa.cols", "bbb.cols"} + // It means column_1 and column_2 are stored in aaa.cols, and column_3 and column_4 are stored in bbb.cols + std::vector> dcg_column_ids; + std::vector dcg_column_files; + // It is used for generate different .cols filename + int idx = 0; + for (uint32_t col_index = 0; col_index < update_column_ids.size(); col_index += BATCH_HANDLE_COLUMN_CNT) { + int64_t t1 = MonotonicMillis(); + // 3.1 build column id range + std::vector selective_update_column_ids = + append_fixed_batch(update_column_ids, col_index, BATCH_HANDLE_COLUMN_CNT); + std::vector selective_update_column_uids = + append_fixed_batch(update_column_uids, col_index, BATCH_HANDLE_COLUMN_CNT); + std::vector selective_unique_update_column_ids = + append_fixed_batch(unique_update_column_ids, col_index, BATCH_HANDLE_COLUMN_CNT); + // 3.2 build partial schema + auto partial_tschema = TabletSchema::create(tschema, selective_update_column_ids); + Schema partial_schema = ChunkHelper::convert_schema(tschema, selective_update_column_uids); + // 3.3 read from source segment + ASSIGN_OR_RETURN(auto rowsetid_segid, _find_rowset_seg_id(each.first)); + const std::string seg_path = Rowset::segment_file_path( + rowset->rowset_path(), rowsetid_segid.unique_rowset_id, rowsetid_segid.segment_id); + ASSIGN_OR_RETURN(auto source_chunk_ptr, + read_from_source_segment(rowset, partial_schema, tablet, &stats, + latest_applied_version.major_number(), rowsetid_segid, seg_path)); + const size_t source_chunk_size = source_chunk_ptr->memory_usage(); + tracker->consume(source_chunk_size); + DeferOp tracker_defer([&]() { tracker->release(source_chunk_size); }); + // 3.2 read from update segment + int64_t t2 = MonotonicMillis(); + std::vector rowids; + auto update_chunk_ptr = ChunkHelper::new_chunk(partial_schema, each.second.size()); + RETURN_IF_ERROR(_read_chunk_from_update(each.second, partial_schema, tracker, rowset, &stats, rowids, + update_chunk_ptr.get())); + const size_t update_chunk_size = update_chunk_ptr->memory_usage(); + tracker->consume(update_chunk_size); + DeferOp tracker_defer2([&]() { tracker->release(update_chunk_size); }); + int64_t t3 = MonotonicMillis(); + // 3.4 merge source chunk and update chunk + RETURN_IF_EXCEPTION(source_chunk_ptr->update_rows(*update_chunk_ptr, rowids.data())); + // 3.5 write column to delta column file + int64_t t4 = MonotonicMillis(); + uint64_t segment_file_size = 0; + uint64_t index_size = 0; + uint64_t footer_position = 0; + padding_char_columns(partial_schema, partial_tschema, source_chunk_ptr.get()); + ASSIGN_OR_RETURN(auto delta_column_group_writer, build_writer_fn(each.first, partial_tschema, idx++)); + RETURN_IF_ERROR(delta_column_group_writer->append_chunk(*source_chunk_ptr)); + RETURN_IF_ERROR(delta_column_group_writer->finalize(&segment_file_size, &index_size, &footer_position)); + int64_t t5 = MonotonicMillis(); + total_seek_source_segment_time += t2 - t1; + total_read_column_from_update_time += t3 - t2; + total_merge_column_time += t4 - t3; + total_finalize_dcg_time += t5 - t4; + // 3.6 prepare column id list and dcg file list + dcg_column_ids.push_back(selective_unique_update_column_ids); + dcg_column_files.push_back(file_name(delta_column_group_writer->segment_path())); + // 3.7. reclaim update chunk cache + reclaim_update_cache_fn(false); + handle_cnt++; + } + // 4 generate delta columngroup + _rssid_to_delta_column_group[each.first] = std::make_shared(); _rssid_to_delta_column_group[each.first]->init(latest_applied_version.major_number() + 1, dcg_column_ids, dcg_column_files); } + // reclaim update cache at final step + reclaim_update_cache_fn(true); cost_str << " [generate delta column group] " << watch.elapsed_time(); watch.reset(); - // 4. generate segment file for insert data + // 5. generate segment file for insert data if (txn_meta.partial_update_mode() == PartialUpdateMode::COLUMN_UPSERT_MODE) { // ignore insert missing rows if partial_update_mode == COLUMN_UPDATE_MODE RETURN_IF_ERROR(_insert_new_rows(tschema, tablet, EditVersion(latest_applied_version.major_number() + 1, 0), @@ -726,21 +769,14 @@ Status RowsetColumnUpdateState::finalize(Tablet* tablet, Rowset* rowset, uint32_ cost_str << " [insert missing rows] " << watch.elapsed_time(); watch.reset(); } - // 5. release update chunk cache - std::for_each(_update_chunk_cache.begin(), _update_chunk_cache.end(), [&](auto& cache) { - if (cache.get() != nullptr) { - tracker->release(cache->memory_usage()); - cache.reset(nullptr); - } - }); cost_str << strings::Substitute( " seek_source_segment(ms):$0 read_column_from_update(ms):$1 avg_merge_column_time(ms):$2 " "avg_finalize_dcg_time(ms):$3 ", total_seek_source_segment_time, total_read_column_from_update_time, total_merge_column_time, total_finalize_dcg_time); - cost_str << strings::Substitute("rss_cnt:$0 update_cnt:$1 column_cnt:$2 update_rows:$3", + cost_str << strings::Substitute("rss_cnt:$0 update_cnt:$1 column_cnt:$2 update_rows:$3 handle_cnt:$4", rss_rowid_to_update_rowid.size(), _partial_update_states.size(), - update_column_ids.size(), update_rows); + update_column_ids.size(), update_rows, handle_cnt); LOG(INFO) << "RowsetColumnUpdateState tablet_id: " << tablet->tablet_id() << ", txn_id: " << rowset->txn_id() << ", finalize cost:" << cost_str.str(); diff --git a/be/src/storage/rowset_column_update_state.h b/be/src/storage/rowset_column_update_state.h index 8d1f9addee215..1d527cd65f280 100644 --- a/be/src/storage/rowset_column_update_state.h +++ b/be/src/storage/rowset_column_update_state.h @@ -169,7 +169,7 @@ class RowsetColumnUpdateState { EditVersion latest_applied_version, const PrimaryIndex& index); StatusOr> _prepare_delta_column_group_writer( - Rowset* rowset, const std::shared_ptr& tschema, uint32_t rssid, int64_t ver); + Rowset* rowset, const std::shared_ptr& tschema, uint32_t rssid, int64_t ver, int idx); // to build `_partial_update_states` Status _prepare_partial_update_states(Tablet* tablet, Rowset* rowset, uint32_t start_idx, uint32_t end_idx, diff --git a/be/src/storage/storage_engine.cpp b/be/src/storage/storage_engine.cpp index 53d6c96a367ce..e244b0dcfe577 100644 --- a/be/src/storage/storage_engine.cpp +++ b/be/src/storage/storage_engine.cpp @@ -647,6 +647,10 @@ void StorageEngine::stop() { if (_update_manager) { _update_manager->stop(); } + + if (_compaction_manager) { + _compaction_manager->stop(); + } } void StorageEngine::clear_transaction_task(const TTransactionId transaction_id) { diff --git a/be/src/tools/meta_tool.cpp b/be/src/tools/meta_tool.cpp index 2ce35f4a128ff..d5785f0ca032c 100644 --- a/be/src/tools/meta_tool.cpp +++ b/be/src/tools/meta_tool.cpp @@ -88,8 +88,8 @@ using starrocks::PageFooterPB; DEFINE_string(root_path, "", "storage root path"); DEFINE_string(operation, "get_meta", - "valid operation: get_meta, flag, load_meta, delete_meta, delete_rowset_meta, " - "show_meta, check_table_meta_consistency, print_lake_metadata, print_lake_txn_log"); + "valid operation: get_meta, flag, load_meta, delete_meta, delete_rowset_meta, show_meta, " + "check_table_meta_consistency, print_lake_metadata, print_lake_txn_log, print_lake_schema"); DEFINE_int64(tablet_id, 0, "tablet_id for tablet meta"); DEFINE_string(tablet_uid, "", "tablet_uid for tablet meta"); DEFINE_int64(table_id, 0, "table id for table meta"); @@ -134,8 +134,9 @@ std::string get_usage(const std::string& progname) { ss << "./meta_tool.sh --operation=calc_checksum [--column_index=xx] --file=/path/to/segment/file\n"; ss << "./meta_tool.sh --operation=check_table_meta_consistency --root_path=/path/to/storage/path " "--table_id=tableid\n"; - ss << "cat 0001000000001394_0000000000000004.meta | ./meta_tool --operation=print_lake_metadata\n"; - ss << "cat 0001000000001391_0000000000000001.log | ./meta_tool --operation=print_lake_txn_log\n"; + ss << "cat 0001000000001394_0000000000000004.meta | ./meta_tool.sh --operation=print_lake_metadata\n"; + ss << "cat 0001000000001391_0000000000000001.log | ./meta_tool.sh --operation=print_lake_txn_log\n"; + ss << "cat SCHEMA_000000000004204C | ./meta_tool.sh --operation=print_lake_schema\n"; return ss.str(); } @@ -1076,6 +1077,21 @@ int meta_tool_main(int argc, char** argv) { return -1; } std::cout << json << '\n'; + } else if (FLAGS_operation == "print_lake_schema") { + starrocks::TabletSchemaPB schema; + if (!schema.ParseFromIstream(&std::cin)) { + std::cerr << "Fail to parse schema\n"; + return -1; + } + json2pb::Pb2JsonOptions options; + options.pretty_json = true; + std::string json; + std::string error; + if (!json2pb::ProtoMessageToJson(schema, &json, options, &error)) { + std::cerr << "Fail to convert protobuf to json: " << error << '\n'; + return -1; + } + std::cout << json << '\n'; } else { // operations that need root path should be written here std::set valid_operations = {"get_meta", diff --git a/be/src/util/lru_cache.cpp b/be/src/util/lru_cache.cpp index 7dd836dd2647f..4694ac5e867ff 100644 --- a/be/src/util/lru_cache.cpp +++ b/be/src/util/lru_cache.cpp @@ -193,22 +193,22 @@ void LRUCache::set_capacity(size_t capacity) { } } -uint64_t LRUCache::get_lookup_count() { +uint64_t LRUCache::get_lookup_count() const { std::lock_guard l(_mutex); return _lookup_count; } -uint64_t LRUCache::get_hit_count() { +uint64_t LRUCache::get_hit_count() const { std::lock_guard l(_mutex); return _hit_count; } -size_t LRUCache::get_usage() { +size_t LRUCache::get_usage() const { std::lock_guard l(_mutex); return _usage; } -size_t LRUCache::get_capacity() { +size_t LRUCache::get_capacity() const { std::lock_guard l(_mutex); return _capacity; } @@ -461,14 +461,14 @@ uint64_t ShardedLRUCache::new_id() { return ++(_last_id); } -size_t ShardedLRUCache::_get_stat(size_t (LRUCache::*mem_fun)()) { +size_t ShardedLRUCache::_get_stat(size_t (LRUCache::*mem_fun)() const) const { size_t n = 0; for (auto& shard : _shards) { n += (shard.*mem_fun)(); } return n; } -size_t ShardedLRUCache::get_capacity() { +size_t ShardedLRUCache::get_capacity() const { return _get_stat(&LRUCache::get_capacity); } @@ -480,15 +480,15 @@ void ShardedLRUCache::prune() { VLOG(7) << "Successfully prune cache, clean " << num_prune << " entries."; } -size_t ShardedLRUCache::get_memory_usage() { +size_t ShardedLRUCache::get_memory_usage() const { return _get_stat(&LRUCache::get_usage); } -size_t ShardedLRUCache::get_lookup_count() { +size_t ShardedLRUCache::get_lookup_count() const { return _get_stat(&LRUCache::get_lookup_count); } -size_t ShardedLRUCache::get_hit_count() { +size_t ShardedLRUCache::get_hit_count() const { return _get_stat(&LRUCache::get_hit_count); } diff --git a/be/src/util/lru_cache.h b/be/src/util/lru_cache.h index 166339d3164eb..e0b38e604a33c 100644 --- a/be/src/util/lru_cache.h +++ b/be/src/util/lru_cache.h @@ -179,10 +179,10 @@ class Cache { virtual void get_cache_status(rapidjson::Document* document) = 0; virtual void set_capacity(size_t capacity) = 0; - virtual size_t get_capacity() = 0; - virtual size_t get_memory_usage() = 0; - virtual size_t get_lookup_count() = 0; - virtual size_t get_hit_count() = 0; + virtual size_t get_capacity() const = 0; + virtual size_t get_memory_usage() const = 0; + virtual size_t get_lookup_count() const = 0; + virtual size_t get_hit_count() const = 0; // Decrease or increase cache capacity. virtual bool adjust_capacity(int64_t delta, size_t min_capacity = 0) = 0; @@ -275,10 +275,10 @@ class LRUCache { void erase(const CacheKey& key, uint32_t hash); int prune(); - uint64_t get_lookup_count(); - uint64_t get_hit_count(); - size_t get_usage(); - size_t get_capacity(); + uint64_t get_lookup_count() const; + uint64_t get_hit_count() const; + size_t get_usage() const; + size_t get_capacity() const; private: void _lru_remove(LRUHandle* e); @@ -291,7 +291,7 @@ class LRUCache { size_t _capacity{0}; // _mutex protects the following state. - std::mutex _mutex; + mutable std::mutex _mutex; size_t _usage{0}; // Dummy head of LRU list. @@ -323,17 +323,17 @@ class ShardedLRUCache : public Cache { void prune() override; void get_cache_status(rapidjson::Document* document) override; void set_capacity(size_t capacity) override; - size_t get_memory_usage() override; - size_t get_capacity() override; - uint64_t get_lookup_count() override; - uint64_t get_hit_count() override; + size_t get_memory_usage() const override; + size_t get_capacity() const override; + uint64_t get_lookup_count() const override; + uint64_t get_hit_count() const override; bool adjust_capacity(int64_t delta, size_t min_capacity = 0) override; private: static uint32_t _hash_slice(const CacheKey& s); static uint32_t _shard(uint32_t hash); void _set_capacity(size_t capacity); - size_t _get_stat(size_t (LRUCache::*mem_fun)()); + size_t _get_stat(size_t (LRUCache::*mem_fun)() const) const; LRUCache _shards[kNumShards]; std::mutex _mutex; diff --git a/be/test/CMakeLists.txt b/be/test/CMakeLists.txt index 5cbf0682c857f..6c49f7322b068 100644 --- a/be/test/CMakeLists.txt +++ b/be/test/CMakeLists.txt @@ -217,7 +217,7 @@ set(EXEC_FILES ./storage/lake/primary_key_compaction_task_test.cpp ./storage/lake/primary_key_publish_test.cpp ./storage/lake/meta_file_test.cpp - ./storage/lake/metadata_cache_test.cpp + ./storage/lake/metacache_test.cpp ./storage/rowset_update_state_test.cpp ./storage/rowset_column_update_state_test.cpp ./storage/rowset_column_partial_update_test.cpp diff --git a/be/test/storage/lake/meta_file_test.cpp b/be/test/storage/lake/meta_file_test.cpp index c9bf9c830ba72..db2cc243c0e8d 100644 --- a/be/test/storage/lake/meta_file_test.cpp +++ b/be/test/storage/lake/meta_file_test.cpp @@ -88,12 +88,7 @@ TEST_F(MetaFileTest, test_meta_rw) { EXPECT_TRUE(st.ok()); // 3. read meta from meta file - MetaFileReader reader(s_tablet_manager->tablet_metadata_location(tablet_id, 10), false); - EXPECT_TRUE(reader.load().ok()); - auto meta_st = reader.get_meta(); - EXPECT_TRUE(meta_st.ok()); - EXPECT_EQ((*meta_st)->id(), tablet_id); - EXPECT_EQ((*meta_st)->version(), 10); + ASSIGN_OR_ABORT(auto metadata2, s_tablet_manager->get_tablet_metadata(tablet_id, 10)); } TEST_F(MetaFileTest, test_delvec_rw) { @@ -125,19 +120,14 @@ TEST_F(MetaFileTest, test_delvec_rw) { EXPECT_TRUE(st.ok()); // 3. read delvec - MetaFileReader reader(s_tablet_manager->tablet_metadata_location(tablet_id, version), false); - EXPECT_TRUE(reader.load().ok()); DelVector after_delvec; - EXPECT_TRUE(reader.get_del_vec(s_tablet_manager.get(), segment_id, &after_delvec).ok()); + ASSIGN_OR_ABORT(auto metadata2, s_tablet_manager->get_tablet_metadata(tablet_id, version)); + EXPECT_TRUE(get_del_vec(s_tablet_manager.get(), *metadata2, segment_id, &after_delvec).ok()); EXPECT_EQ(before_delvec, after_delvec.save()); // 4. read meta - MetaFileReader reader2(s_tablet_manager->tablet_metadata_location(tablet_id, version), false); - EXPECT_TRUE(reader2.load().ok()); - auto meta_st = reader2.get_meta(); - EXPECT_TRUE(meta_st.ok()); - auto iter = (*meta_st)->delvec_meta().delvecs().find(segment_id); - EXPECT_TRUE(iter != (*meta_st)->delvec_meta().delvecs().end()); + auto iter = metadata2->delvec_meta().delvecs().find(segment_id); + EXPECT_TRUE(iter != metadata2->delvec_meta().delvecs().end()); auto delvec_pagepb = iter->second; EXPECT_EQ(delvec_pagepb.version(), version); @@ -155,34 +145,29 @@ TEST_F(MetaFileTest, test_delvec_rw) { EXPECT_TRUE(st.ok()); // 6. read again - MetaFileReader reader3(s_tablet_manager->tablet_metadata_location(tablet_id, version2), false); - EXPECT_TRUE(reader3.load().ok()); - meta_st = reader3.get_meta(); - EXPECT_TRUE(meta_st.ok()); + ASSIGN_OR_ABORT(auto metadata3, s_tablet_manager->get_tablet_metadata(tablet_id, version2)); - iter = (*meta_st)->delvec_meta().delvecs().find(segment_id); - EXPECT_TRUE(iter != (*meta_st)->delvec_meta().delvecs().end()); + iter = metadata3->delvec_meta().delvecs().find(segment_id); + EXPECT_TRUE(iter != metadata3->delvec_meta().delvecs().end()); auto delvecpb = iter->second; EXPECT_EQ(delvecpb.version(), version2); // 7. test reclaim delvec version to file name record - MetaFileReader reader4(s_tablet_manager->tablet_metadata_location(tablet_id, version2), false); - EXPECT_TRUE(reader4.load().ok()); - meta_st = reader4.get_meta(); - EXPECT_TRUE(meta_st.ok()); + ASSIGN_OR_ABORT(auto metadata4, s_tablet_manager->get_tablet_metadata(tablet_id, version2)); // clear all delvec meta element so that all element in // version_to_file map will also be removed // in this case, delvecs meta map has only one element [key=(segment=1234, value=(version=12, offset=0, size=35)] // delvec_to_file has also one element [key=(version=12), value=(delvec_file=xxx)] // after clearing, delvecs meta map will have nothing, and element in delvec_to_file will also be useless - (*meta_st)->mutable_delvec_meta()->mutable_delvecs()->clear(); + auto new_meta = std::make_shared(*metadata4); + new_meta->mutable_delvec_meta()->mutable_delvecs()->clear(); // insert a new delvec record into delvecs meta map with new version 13 // we expect the old element in delvec_to_file map (version 12) will be removed auto new_version = version2 + 1; - MetaFileBuilder builder3(*tablet, *meta_st); - (*meta_st)->set_version(new_version); + MetaFileBuilder builder3(*tablet, new_meta); + new_meta->set_version(new_version); DelVector dv3; dv3.set_empty(); EXPECT_TRUE(dv3.empty()); @@ -194,9 +179,8 @@ TEST_F(MetaFileTest, test_delvec_rw) { EXPECT_TRUE(st.ok()); // validate delvec file record with version 12 been removed - MetaFileReader reader5(s_tablet_manager->tablet_metadata_location(tablet_id, new_version), false); - EXPECT_TRUE(reader5.load().ok()); - auto version_to_file_map = (*meta_st)->delvec_meta().version_to_file(); + ASSIGN_OR_ABORT(auto metadata5, s_tablet_manager->get_tablet_metadata(tablet_id, new_version)); + auto version_to_file_map = metadata5->delvec_meta().version_to_file(); EXPECT_EQ(version_to_file_map.size(), 1); auto iter2 = version_to_file_map.find(version2); @@ -206,48 +190,6 @@ TEST_F(MetaFileTest, test_delvec_rw) { EXPECT_TRUE(iter2 != version_to_file_map.end()); } -TEST_F(MetaFileTest, test_delvec_read_meta_cache) { - // 1. generate metadata - const int64_t tablet_id = 10003; - const uint32_t segment_id = 1234; - const int64_t version = 11; - auto tablet = std::make_shared(s_tablet_manager.get(), tablet_id); - auto metadata = std::make_shared(); - metadata->set_id(tablet_id); - metadata->set_version(version); - metadata->set_next_rowset_id(110); - metadata->mutable_schema()->set_keys_type(PRIMARY_KEYS); - - // 2. write pk meta & delvec - MetaFileBuilder builder(*tablet, metadata); - DelVector dv; - dv.set_empty(); - EXPECT_TRUE(dv.empty()); - - std::shared_ptr ndv; - std::vector dels = {1, 3, 5, 7, 90000}; - dv.add_dels_as_new_version(dels, version, &ndv); - EXPECT_FALSE(ndv->empty()); - std::string before_delvec = ndv->save(); - builder.append_delvec(ndv, segment_id); - Status st = builder.finalize(next_id()); - EXPECT_TRUE(st.ok()); - - // 3. read delvec - MetaFileReader reader(s_tablet_manager->tablet_metadata_location(tablet_id, version), false); - auto tablet_meta_ptr = - s_tablet_manager->lookup_tablet_metadata(s_tablet_manager->tablet_metadata_location(tablet_id, version)); - EXPECT_TRUE(tablet_meta_ptr != nullptr); - EXPECT_EQ(tablet_meta_ptr->id(), tablet_id); - // call load_by cache for test - EXPECT_TRUE( - reader.load_by_cache(s_tablet_manager->tablet_metadata_location(tablet_id, version), s_tablet_manager.get()) - .ok()); - DelVector after_delvec; - EXPECT_TRUE(reader.get_del_vec(s_tablet_manager.get(), segment_id, &after_delvec).ok()); - EXPECT_EQ(before_delvec, after_delvec.save()); -} - TEST_F(MetaFileTest, test_delvec_read_loop) { // 1. generate metadata const int64_t tablet_id = 10002; @@ -279,10 +221,9 @@ TEST_F(MetaFileTest, test_delvec_read_loop) { EXPECT_TRUE(st.ok()); // 3. read delvec - MetaFileReader reader(s_tablet_manager->tablet_metadata_location(tablet_id, version), false); - EXPECT_TRUE(reader.load().ok()); DelVector after_delvec; - EXPECT_TRUE(reader.get_del_vec(s_tablet_manager.get(), segment_id, &after_delvec).ok()); + ASSIGN_OR_ABORT(auto meta, s_tablet_manager->get_tablet_metadata(tablet_id, version)); + EXPECT_TRUE(get_del_vec(s_tablet_manager.get(), *meta, segment_id, &after_delvec).ok()); EXPECT_EQ(before_delvec, after_delvec.save()); }; for (uint32_t segment_id = 1000; segment_id < 1200; segment_id++) { diff --git a/be/test/storage/lake/metadata_cache_test.cpp b/be/test/storage/lake/metacache_test.cpp similarity index 64% rename from be/test/storage/lake/metadata_cache_test.cpp rename to be/test/storage/lake/metacache_test.cpp index c970d1a6c889d..60577fbc6de59 100644 --- a/be/test/storage/lake/metadata_cache_test.cpp +++ b/be/test/storage/lake/metacache_test.cpp @@ -12,6 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include "storage/lake/metacache.h" + #include #include "column/chunk.h" @@ -20,10 +22,7 @@ #include "column/schema.h" #include "column/vectorized_fwd.h" #include "common/logging.h" -#include "fs/fs_util.h" #include "storage/chunk_helper.h" -#include "storage/lake/join_path.h" -#include "storage/lake/location_provider.h" #include "storage/lake/tablet_manager.h" #include "storage/lake/tablet_reader.h" #include "storage/lake/tablet_writer.h" @@ -36,9 +35,9 @@ namespace starrocks::lake { using namespace starrocks; -class LakeMetadataCacheTest : public TestBase { +class LakeMetacacheTest : public TestBase { public: - LakeMetadataCacheTest() : TestBase(kTestDirectory) { + LakeMetacacheTest() : TestBase(kTestDirectory) { _tablet_metadata = std::make_unique(); _tablet_metadata->set_id(next_id()); _tablet_metadata->set_version(1); @@ -88,7 +87,74 @@ class LakeMetadataCacheTest : public TestBase { std::shared_ptr _schema; }; -TEST_F(LakeMetadataCacheTest, test_segment_cache) { +TEST_F(LakeMetacacheTest, test_tablet_metadata_cache) { + auto* metacache = _tablet_mgr->metacache(); + + auto m1 = std::make_shared(); + metacache->cache_tablet_metadata("metadata1", m1); + + auto m2 = metacache->lookup_tablet_metadata("metadata1"); + EXPECT_EQ(m1.get(), m2.get()); + + auto m3 = metacache->lookup_tablet_metadata("metadata2"); + ASSERT_TRUE(m3 == nullptr); + + auto log = metacache->lookup_txn_log("metadata1"); + ASSERT_TRUE(log == nullptr); +} + +TEST_F(LakeMetacacheTest, test_txn_log_cache) { + auto* metacache = _tablet_mgr->metacache(); + + auto log1 = std::make_shared(); + metacache->cache_txn_log("log1", log1); + + auto log2 = metacache->lookup_txn_log("log1"); + EXPECT_EQ(log1.get(), log2.get()); + + auto log3 = metacache->lookup_txn_log("log2"); + ASSERT_TRUE(log3 == nullptr); + + auto schema = metacache->lookup_tablet_schema("log1"); + ASSERT_TRUE(schema == nullptr); +} + +TEST_F(LakeMetacacheTest, test_tablet_schema_cache) { + auto* metacache = _tablet_mgr->metacache(); + + auto schema1 = std::make_shared(); + metacache->cache_tablet_schema("schema1", schema1, 0); + + auto schema2 = metacache->lookup_tablet_schema("schema1"); + EXPECT_EQ(schema1.get(), schema2.get()); + + auto schema3 = metacache->lookup_tablet_schema("schema2"); + ASSERT_TRUE(schema3 == nullptr); + + auto delvec = metacache->lookup_delvec("schema1"); + ASSERT_TRUE(delvec == nullptr); +} + +TEST_F(LakeMetacacheTest, test_deletion_vector_cache) { + auto* metacache = _tablet_mgr->metacache(); + + auto dv1 = std::make_shared(); + metacache->cache_delvec("dv1", dv1); + + auto dv2 = metacache->lookup_delvec("dv1"); + EXPECT_EQ(dv1.get(), dv2.get()); + + auto dv3 = metacache->lookup_delvec("dv2"); + ASSERT_TRUE(dv3 == nullptr); + + auto seg = metacache->lookup_segment("dv1"); + ASSERT_TRUE(seg == nullptr); + + auto meta = metacache->lookup_tablet_metadata("dv1"); + ASSERT_TRUE(meta == nullptr); +} + +TEST_F(LakeMetacacheTest, test_segment_cache) { auto* metacache = _tablet_mgr->metacache(); // write data and metadata @@ -150,7 +216,7 @@ TEST_F(LakeMetadataCacheTest, test_segment_cache) { CHECK_OK(_tablet_mgr->put_tablet_metadata(*_tablet_metadata)); // no segment - auto sz0 = metacache->get_memory_usage(); + auto sz0 = metacache->memory_usage(); ASSIGN_OR_ABORT(auto reader, tablet.new_reader(2, *_schema)); ASSERT_OK(reader->prepare()); @@ -158,7 +224,7 @@ TEST_F(LakeMetadataCacheTest, test_segment_cache) { ASSERT_OK(reader->open(params)); // load segment without indexes - auto sz1 = metacache->get_memory_usage(); + auto sz1 = metacache->memory_usage(); auto read_chunk_ptr = ChunkHelper::new_chunk(*_schema, 1024); for (int j = 0; j < 2; ++j) { @@ -171,10 +237,36 @@ TEST_F(LakeMetadataCacheTest, test_segment_cache) { reader->close(); // load segment with indexes, and remove index meta (index meta is larger than index) - auto sz2 = metacache->get_memory_usage(); + auto sz2 = metacache->memory_usage(); std::cout << "metadata cache memory usage: " << sz0 << "-" << sz1 << "-" << sz2; ASSERT_GT(sz1, sz0); ASSERT_LT(sz2, sz1); } +TEST_F(LakeMetacacheTest, test_update_capacity) { + auto* metacache = _tablet_mgr->metacache(); + + auto old_cap = metacache->capacity(); + metacache->update_capacity(old_cap + 1024 * 1024); + auto new_cap = metacache->capacity(); + ASSERT_EQ(old_cap + 1024 * 1024, new_cap); + metacache->update_capacity(new_cap - 1024); + auto new_cap2 = metacache->capacity(); + ASSERT_EQ(new_cap - 1024, new_cap2); +} + +TEST_F(LakeMetacacheTest, test_prune) { + auto* metacache = _tablet_mgr->metacache(); + + auto meta = std::make_shared(); + metacache->cache_tablet_metadata("meta1", meta); + auto meta2 = metacache->lookup_tablet_metadata("meta1"); + ASSERT_TRUE(meta2 != nullptr); + + metacache->prune(); + + auto meta3 = metacache->lookup_tablet_metadata("meta1"); + ASSERT_TRUE(meta3 == nullptr); +} + } // namespace starrocks::lake diff --git a/be/test/storage/lake/rowset_test.cpp b/be/test/storage/lake/rowset_test.cpp index 27b07db4a2fad..c9583b7a3d06c 100644 --- a/be/test/storage/lake/rowset_test.cpp +++ b/be/test/storage/lake/rowset_test.cpp @@ -19,9 +19,8 @@ #include "column/schema.h" #include "column/vectorized_fwd.h" #include "common/logging.h" -#include "fs/fs_util.h" #include "storage/chunk_helper.h" -#include "storage/lake/location_provider.h" +#include "storage/lake/metacache.h" #include "storage/lake/tablet_manager.h" #include "storage/lake/tablet_writer.h" #include "storage/tablet_schema.h" @@ -152,17 +151,16 @@ TEST_F(LakeRowsetTest, test_load_segments) { ASSIGN_OR_ABORT(auto segments1, rowset->segments(false)); ASSERT_EQ(2, segments1.size()); for (const auto& seg : segments1) { - auto handle = cache->lookup(CacheKey(seg->file_name())); - ASSERT_TRUE(handle == nullptr); + auto segment = cache->lookup_segment(seg->file_name()); + ASSERT_TRUE(segment == nullptr); } // fill data cache: false, fill metadata cache: true ASSIGN_OR_ABORT(auto segments2, rowset->segments(false, true)); ASSERT_EQ(2, segments2.size()); for (const auto& seg : segments2) { - auto handle = cache->lookup(CacheKey(seg->file_name())); - ASSERT_TRUE(handle != nullptr); - cache->release(handle); + auto segment = cache->lookup_segment(seg->file_name()); + ASSERT_TRUE(segment != nullptr); } } diff --git a/be/test/storage/rowset_column_partial_update_test.cpp b/be/test/storage/rowset_column_partial_update_test.cpp index 2504daedd802e..720f43d679fda 100644 --- a/be/test/storage/rowset_column_partial_update_test.cpp +++ b/be/test/storage/rowset_column_partial_update_test.cpp @@ -58,8 +58,7 @@ class RowsetColumnPartialUpdateTest : public ::testing::Test, testing::WithParam } } - RowsetSharedPtr create_rowset(const TabletSharedPtr& tablet, const vector& keys, - Column* one_delete = nullptr) { + RowsetSharedPtr create_rowset(const TabletSharedPtr& tablet, const vector& keys, bool add_v3 = false) { RowsetWriterContext writer_context; RowsetId rowset_id = StorageEngine::instance()->next_rowset_id(); writer_context.rowset_id = rowset_id; @@ -81,13 +80,14 @@ class RowsetColumnPartialUpdateTest : public ::testing::Test, testing::WithParam cols[0]->append_datum(Datum(key)); cols[1]->append_datum(Datum((int16_t)(key % 100 + 1))); cols[2]->append_datum(Datum((int32_t)(key % 1000 + 2))); + if (add_v3) { + cols[3]->append_datum(Datum((int32_t)(key % 1000 + 3))); + } } - if (one_delete == nullptr && !keys.empty()) { + if (!keys.empty()) { CHECK_OK(writer->flush_chunk(*chunk)); - } else if (one_delete == nullptr) { + } else { CHECK_OK(writer->flush()); - } else if (one_delete != nullptr) { - CHECK_OK(writer->flush_chunk_with_deletes(*chunk, *one_delete)); } return *writer->build(); } @@ -169,17 +169,18 @@ class RowsetColumnPartialUpdateTest : public ::testing::Test, testing::WithParam auto schema = ChunkHelper::convert_schema(partial_schema); auto chunk = ChunkHelper::new_chunk(schema, keys.size()); - EXPECT_TRUE(2 == chunk->num_columns()); auto& cols = chunk->columns(); for (long key : keys) { + int idx = 0; for (int colid : column_indexes) { if (colid == 0) { - cols[0]->append_datum(Datum(key)); + cols[idx]->append_datum(Datum(key)); } else if (colid == 1) { - cols[1]->append_datum(Datum(v1_func(key))); + cols[idx]->append_datum(Datum(v1_func(key))); } else { - cols[1]->append_datum(Datum(v2_func(key))); + cols[idx]->append_datum(Datum(v2_func(key))); } + idx++; } } for (int i = 0; i < segment_num; i++) { @@ -980,6 +981,51 @@ TEST_P(RowsetColumnPartialUpdateTest, partial_update_too_many_segment_and_limit_ ASSERT_TRUE(StorageEngine::instance()->update_manager()->TEST_primary_index_refcnt(tablet->tablet_id(), 1)); } +TEST_P(RowsetColumnPartialUpdateTest, partial_update_multi_column_batch) { + const int N = 10; + // generate M upt files in each partial rowset + const int M = 100; + auto tablet = create_tablet(rand(), rand(), true); + ASSERT_EQ(1, tablet->updates()->version_history_count()); + + std::vector keys(N); + for (int i = 0; i < N; i++) { + keys[i] = i; + } + auto v1_func = [](int64_t k1) { return (int16_t)(k1 % 100 + 3); }; + auto v2_func = [](int64_t k1) { return (int32_t)(k1 % 1000 + 4); }; + std::vector rowsets; + rowsets.reserve(20); + // write full rowset first + for (int i = 0; i < 10; i++) { + rowsets.emplace_back(create_rowset(tablet, keys, true)); + } + std::vector> partial_schemas; + // partial update v1 and v2 at once + for (int i = 0; i < 10; i++) { + std::vector column_indexes = {0, 1, 2}; + partial_schemas.push_back(TabletSchema::create(tablet->tablet_schema(), column_indexes)); + rowsets.emplace_back( + create_partial_rowset(tablet, keys, column_indexes, v1_func, v2_func, partial_schemas[i], M)); + ASSERT_EQ(rowsets.back()->num_update_files(), M); + } + + int32_t old_val = config::vertical_compaction_max_columns_per_group; + config::vertical_compaction_max_columns_per_group = 1; + int64_t version = 1; + commit_rowsets(tablet, rowsets, version); + // check data + ASSERT_TRUE(check_tablet(tablet, version, N, [](int64_t k1, int64_t v1, int32_t v2) { + return (int16_t)(k1 % 100 + 3) == v1 && (int32_t)(k1 % 1000 + 4) == v2; + })); + config::vertical_compaction_max_columns_per_group = old_val; + // check refcnt + for (const auto& rs_ptr : rowsets) { + ASSERT_FALSE(StorageEngine::instance()->update_manager()->TEST_update_state_exist(tablet.get(), rs_ptr.get())); + } + ASSERT_TRUE(StorageEngine::instance()->update_manager()->TEST_primary_index_refcnt(tablet->tablet_id(), 1)); +} + INSTANTIATE_TEST_SUITE_P(RowsetColumnPartialUpdateTest, RowsetColumnPartialUpdateTest, ::testing::Values(1, 1024, 104857600)); diff --git a/docs/TOC.md b/docs/TOC.md index 915d6aa55c601..1baf03ba79a80 100644 --- a/docs/TOC.md +++ b/docs/TOC.md @@ -750,6 +750,7 @@ + [Generated columns](./sql-reference/sql-statements/generated_columns.md) + [System variables](./reference/System_variable.md) + [User-defined variables](./reference/user_defined_variables.md) + + [Prepared statements](./sql-reference/sql-statements/prepared_statement.md) + [Error code](./reference/Error_code.md) + [System limits](./reference/System_limit.md) + [AWS IAM policies](./reference/aws_iam_policies.md) diff --git a/docs/administration/Configuration.md b/docs/administration/Configuration.md index fc3f5ae477b77..5d3d0392727c3 100644 --- a/docs/administration/Configuration.md +++ b/docs/administration/Configuration.md @@ -509,6 +509,15 @@ ADMIN SET FRONTEND CONFIG ("key" = "value"); - **Default**: 10000 - **Description**: The timeout duration for committing (publishing) a write transaction to a StarRocks external table. The default value `10000` indicates a 10-second timeout duration. +##### enable_sync_publish + +- **Unit**: - +- **Default**: TRUE +- **Description**: Whether to synchronously execute the apply task at the publish phase of a load transaction. This parameter is applicable only to Primary Key tables. Valid values: + - `TRUE` (default): The apply task is synchronously executed at the publish phase of a load transaction. It means that the load transaction is reported as successful only after the apply task is completed, and the loaded data can truly be queried. When a task loads a large volume of data at a time or loads data frequently, setting this parameter to `true` can improve query performance and stability, but may increase load latency. + - `FALSE`: The apply task is asynchronously executed at the publish phase of a load transaction. It means that the load transaction is reported as successful after the apply task is submitted, but the loaded data cannot be immediately queried. In this case, concurrent queries need to wait for the apply task to complete or time out before they can continue. When a task loads a large volume of data at a time or loads data frequently, setting this parameter to `false` may affect query performance and stability. +- **Introduced in**: v3.2.0 + #### Storage ##### default_replication_num @@ -1517,7 +1526,7 @@ BE dynamic parameters are as follows. #### txn_commit_rpc_timeout_ms - **Default:** 60,000 ms -- **Description:** The timeout for a transaction commit RPC. +- **Description:** The timeout of a load transaction. From v3.1 onwards, this parameter controls the timeout of a transaction commit RPC. #### max_consumer_num_per_group diff --git a/docs/data_source/catalog/iceberg_catalog.md b/docs/data_source/catalog/iceberg_catalog.md index 403db9e21323d..ae09ee95076b7 100644 --- a/docs/data_source/catalog/iceberg_catalog.md +++ b/docs/data_source/catalog/iceberg_catalog.md @@ -475,7 +475,7 @@ CREATE EXTERNAL CATALOG iceberg_catalog_hms PROPERTIES ( "type" = "iceberg", - "hive.metastore.type" = "hive", + "iceberg.catalog.type" = "hive", "hive.metastore.uris" = "thrift://xx.xx.xx:9083" ); ``` diff --git a/docs/deployment/environment_configurations.md b/docs/deployment/environment_configurations.md index 847faeda18c7e..3995a8cac4785 100644 --- a/docs/deployment/environment_configurations.md +++ b/docs/deployment/environment_configurations.md @@ -268,7 +268,7 @@ Problems can occur with StarRocks if the values of **max file descriptors** and You can set the maximum number of file descriptors by running the following command: ```Bash -ulimit -n 65535 +ulimit -n 655350 ``` ### Max user processes diff --git a/docs/developers/trace-tools/Trace.md b/docs/developers/trace-tools/Trace.md index be60893bf6d01..c7262127638c1 100644 --- a/docs/developers/trace-tools/Trace.md +++ b/docs/developers/trace-tools/Trace.md @@ -16,7 +16,7 @@ Steps to enable tracing in SR: 1. Install [Jaeger](https://www.jaegertracing.io/docs/1.31/getting-started) - The guide above uses docker. For simplicity, you can also just download binary package and run locally, `https://github.com/jaegertracing/jaeger/releases` + The guide above uses docker. For simplicity, you can also just download [binary package](https://github.com/jaegertracing/jaeger/releases) and run locally. ``` decster@decster-MS-7C94:~/soft/jaeger-1.31.0-linux-amd64$ ll @@ -33,7 +33,7 @@ Steps to enable tracing in SR: decster@decster-MS-7C94:~/soft/jaeger-1.31.0-linux-amd64$ ./jaeger-all-in-one ``` -2. Config FE\&FE to enable tracing +2. Config FE\&FE to enable tracing. Currently, opentelemetry java & cpp sdk use different protocols, java uses grpc proto, while cpp uses thrift\&UDP, so the endpoint ports are different. ``` @@ -50,7 +50,7 @@ Steps to enable tracing in SR: ``` 3. Open jaeger web UI, usually in `http://localhost:16686/search` -4. Do some data ingestion (streamload\/insert into) and search TXN traces on web UI +4. Do some data ingestion (streamload/insert into) and search TXN traces on web UI ![trace_pic2.png](../../assets/trace_pic2.png)(trace_pic2.png) ![trace_pic3.png](../../assets/trace_pic3.png)(trace_pic3.png) diff --git a/docs/release_notes/release-3.0.md b/docs/release_notes/release-3.0.md index 6717ef6273cc5..e4e10d4d1c759 100644 --- a/docs/release_notes/release-3.0.md +++ b/docs/release_notes/release-3.0.md @@ -35,6 +35,10 @@ Fixed the following issues: Release date: September 12, 2023 +### Behavior Change + +- When using the [group_concat](../sql-reference/sql-functions/string-functions/group_concat.md) function, you must use the SEPARATOR keyword to declare the separator. + ### New Features - The aggregate function [group_concat](../sql-reference/sql-functions/string-functions/group_concat.md) supports the DISTINCT keyword and the ORDER BY clause. [#28778](https://github.com/StarRocks/starrocks/pull/28778) diff --git a/docs/release_notes/release-3.1.md b/docs/release_notes/release-3.1.md index 414ac1da1544c..202345ed6204b 100644 --- a/docs/release_notes/release-3.1.md +++ b/docs/release_notes/release-3.1.md @@ -4,17 +4,22 @@ Release date: September 25, 2023 -## New Features +### Behavior Change + +- When using the [group_concat](../sql-reference/sql-functions/string-functions/group_concat.md) function, you must use the SEPARATOR keyword to declare the separator. + +### New Features +- Primary Key tables created in shared-data StarRocks clusters support index persistence onto local disks in the same way as they do in shared-nothing StarRocks clusters. - The aggregate function [group_concat](../sql-reference/sql-functions/string-functions/group_concat.md) supports the DISTINCT keyword and the ORDER BY clause. [#28778](https://github.com/StarRocks/starrocks/pull/28778) - [Stream Load](../sql-reference/sql-statements/data-manipulation/STREAM_LOAD.md), [Broker Load](../sql-reference/sql-statements/data-manipulation/BROKER_LOAD.md), [Kafka Connector](../loading/Kafka-connector-starrocks.md), [Flink Connector](../loading/Flink-connector-starrocks.md), and [Spark Connector](../loading/Spark-connector-starrocks.md) support partial updates in column mode on a Primary Key table. [#28288](https://github.com/StarRocks/starrocks/pull/28288) - Data in partitions can be automatically cooled down over time. (This feature is not supported for [list partitioning](../table_design/list_partitioning.md).) [#29335](https://github.com/StarRocks/starrocks/pull/29335) [#29393](https://github.com/StarRocks/starrocks/pull/29393) -## Improvements +### Improvements Executing SQL commands with invalid comments now returns results consistent with MySQL. [#30210](https://github.com/StarRocks/starrocks/pull/30210) -## Bug Fixes +### Bug Fixes Fixed the following issues: diff --git a/docs/sql-reference/sql-statements/prepared_statement.md b/docs/sql-reference/sql-statements/prepared_statement.md new file mode 100644 index 0000000000000..655f98c04206e --- /dev/null +++ b/docs/sql-reference/sql-statements/prepared_statement.md @@ -0,0 +1,191 @@ +# Prepared statements + +From v3.2 onwards, StarRocks provides prepared statements for executing SQL statements multiple times with the same structure but different variables. This feature significantly improves execution efficiency and prevents SQL injection. + +## Description + +The prepared statements basically work as follows: + +1. **Preparation**: Users prepare a SQL statement where variables are represented by placeholders `?`. The FE parses the SQL statement and generates an execution plan. +2. **Execution**: After declaring variables, users pass these variables to the statement and execute the statement. Users can execute the same statement multiple times with different variables. + +**Benefits** + +- **Saves overhead of parsing**: In real-world business scenarios, an application often executes a statement multiple times with the same structure but different variables. With prepared statements supported, StarRocks needs to parse the statement only once during the preparation phase. Subsequent executions of the same statement with different variables can directly use the pre-generated parsing result. As such, statement execution performance is significantly improved, especially for complex queries. +- **Prevents SQL injection attacks**: By separating the statement from the variables and passing user-input data as parameters rather than directly concatenating the variables into the statement, StarRocks can prevent malicious users from executing malicious SQL codes. + +**Usages** + +Prepared statements are effective only in the current session and cannot be used in other sessions. After the current session exits, the prepared statements created in that session are automatically dropped. + +## Syntax + +The execution of a prepared statement consists of the following phases: + +- PREPARE: Prepare the statement where variables are represented by placeholders `?`. +- SET: Declare variables within the statement. +- EXECUTE: Pass the declared variables to the statement and execute it. +- DROP PREPARE or DEALLOCATE PREPARE: Delete the prepared statement. + +### PREPARE + +**Syntax:** + +```SQL +PREPARE FROM +``` + +**Parameters:** + +- `stmt_name`: the name given to the prepared statement, which is subsequently used to execute or deallocate that prepared statement. The name must be unique within a single session. +- `preparable_stmt`: the SQL statement to be prepared, where the placeholder for variables is a question mark (`?`). + +**Example:** + +Prepare an `INSERT INTO VALUES()` statement with specific values represented by placeholders `?`. + +```SQL +PREPARE insert_stmt FROM 'INSERT INTO users (id, country, city, revenue) VALUES (?, ?, ?, ?)'; +``` + +### SET + +**Syntax:** + +```SQL +SET @var_name = expr [, ...]; +``` + +**Parameters:** + +- `var_name`: the name of a user-defined variable. +- `expr`: a user-defined variable. + +**Example:** Declare four variables. + +```SQL +SET @id1 = 1, @country1 = 'USA', @city1 = 'New York', @revenue1 = 1000000; +``` + +For more information, see [user-defined variables](../../reference/user_defined_variables.md). + +### EXECUTE + +**Syntax:** + +```SQL +EXECUTE [USING @var_name [, @var_name] ...] +``` + +**Parameters:** + +- `var_name`: the name of a variable declared in the `SET` statement. +- `stmt_name`: the name of the prepared statement. + +**Example:** + +Pass variables to an `INSERT` statement and execute that statement. + +```SQL +EXECUTE insert_stmt USING @id1, @country1, @city1, @revenue1; +``` + +### DROP PREPARE or DEALLOCATE PREPARE + +**Syntax:** + +```SQL +{DEALLOCATE | DROP} PREPARE +``` + +**Parameters:** + +- `stmt_name`: The name of the prepared statement. + +**Example:** + +Delete a prepared statement. + +```SQL +DROP PREPARE insert_stmt; +``` + +## Examples + +### Use prepared statements + +The following example demonstrates how to use prepared statements to insert, delete, update, and query data from a StarRocks table: + +Assuming the following database named `demo` and table named `users` are alrealy created: + +```SQL +CREATE DATABASE IF NOT EXISTS demo; +USE demo; +CREATE TABLE users ( + id BIGINT NOT NULL, + country STRING, + city STRING, + revenue BIGINT +) +PRIMARY KEY (id) +DISTRIBUTED BY HASH(id); +``` + +1. Prepare statements for execution. + + ```SQL + PREPARE insert_stmt FROM 'INSERT INTO users (id, country, city, revenue) VALUES (?, ?, ?, ?)'; + PREPARE select_all_stmt FROM 'SELECT * FROM users'; + PREPARE select_by_id_stmt FROM 'SELECT * FROM users WHERE id = ?'; + PREPARE update_stmt FROM 'UPDATE users SET revenue = ? WHERE id = ?'; + PREPARE delete_stmt FROM 'DELETE FROM users WHERE id = ?'; + ``` + +2. Declare variables in these statements. + + ```SQL + SET @id1 = 1, @id2 = 2; + SET @country1 = 'USA', @country2 = 'Canada'; + SET @city1 = 'New York', @city2 = 'Toronto'; + SET @revenue1 = 1000000, @revenue2 = 1500000, @revenue3 = (SELECT (revenue) * 1.1 FROM users); + ``` + +3. Use the declared variables to execute the statements. + + ```SQL + -- Insert two rows of data. + EXECUTE insert_stmt USING @id1, @country1, @city1, @revenue1; + EXECUTE insert_stmt USING @id2, @country2, @city2, @revenue2; + + -- Query all data from the table. + EXECUTE select_all_stmt; + + -- Query data with ID 1 or 2 separately. + EXECUTE select_by_id_stmt USING @id1; + EXECUTE select_by_id_stmt USING @id2; + + -- Partially update the row with ID 1. Only update the revenue column. + EXECUTE update_stmt USING @revenue3, @id1; + + -- Delete data with ID 1. + EXECUTE delete_stmt USING @id1; + + -- Delete prepared statements. + DROP PREPARE insert_stmt; + DROP PREPARE select_all_stmt; + DROP PREPARE select_by_id_stmt; + DROP PREPARE update_stmt; + DROP PREPARE delete_stmt; + ``` + +### Using Prepared Statements in Java application + +The following example demonstrates how a Java application can use a JDBC driver to insert, delete, update, and query data from a StarRocks table: + +1. When specifying StarRocks' connection URL in JDBC, you need to enable server-side prepared statements: + + ```Plaintext + jdbc:mysql://:/useServerPrepStmts=true + ``` + +2. The StarRocks GitHub project provides a [Java code example](https://github.com/StarRocks/starrocks/blob/main/fe/fe-core/src/test/java/com/starrocks/analysis/PreparedStmtTest.java) that explains how to insert, delete, update, and query data from a StarRocks table through a JDBC driver. diff --git a/fe/fe-core/src/main/java/com/starrocks/backup/RestoreJob.java b/fe/fe-core/src/main/java/com/starrocks/backup/RestoreJob.java index f5f611fd0e6dc..d9882c8cf42dd 100644 --- a/fe/fe-core/src/main/java/com/starrocks/backup/RestoreJob.java +++ b/fe/fe-core/src/main/java/com/starrocks/backup/RestoreJob.java @@ -1435,10 +1435,10 @@ private Status allTabletCommitted(boolean isReplay) { protected void updateTablets(MaterializedIndex idx, PhysicalPartition part) { for (Tablet tablet : idx.getTablets()) { for (Replica replica : ((LocalTablet) tablet).getImmutableReplicas()) { - if (!replica.checkVersionCatchUp(part.getVisibleVersion(), false)) { - replica.updateRowCount(part.getVisibleVersion(), - replica.getDataSize(), replica.getRowCount()); - } + // force update all info for all replica + replica.updateForRestore(part.getVisibleVersion(), + replica.getDataSize(), replica.getRowCount()); + replica.setLastReportVersion(part.getVisibleVersion()); } } } diff --git a/fe/fe-core/src/main/java/com/starrocks/catalog/IcebergTable.java b/fe/fe-core/src/main/java/com/starrocks/catalog/IcebergTable.java index f135d61b9d215..e2b9557dc3f05 100644 --- a/fe/fe-core/src/main/java/com/starrocks/catalog/IcebergTable.java +++ b/fe/fe-core/src/main/java/com/starrocks/catalog/IcebergTable.java @@ -44,7 +44,6 @@ import com.starrocks.thrift.TTableType; import org.apache.iceberg.BaseTable; import org.apache.iceberg.PartitionField; -import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Snapshot; import org.apache.iceberg.SortField; import org.apache.iceberg.types.Types; @@ -161,10 +160,8 @@ public List getPartitionColumns() { } public List getPartitionColumnsIncludeTransformed() { List allPartitionColumns = new ArrayList<>(); - PartitionSpec currentSpec = getNativeTable().spec(); - boolean existPartitionEvolution = currentSpec.fields().stream().anyMatch(field -> field.transform().isVoid()); for (PartitionField field : getNativeTable().spec().fields()) { - if (!field.transform().isIdentity() && existPartitionEvolution) { + if (!field.transform().isIdentity() && hasPartitionTransformedEvolution()) { continue; } String baseColumnName = nativeTable.schema().findColumnName(field.sourceId()); @@ -201,6 +198,11 @@ public List getSortKeyIndexes() { return indexes; } + // day(dt) -> identity dt + public boolean hasPartitionTransformedEvolution() { + return getNativeTable().spec().fields().stream().anyMatch(field -> field.transform().isVoid()); + } + public void resetSnapshot() { snapshot = Optional.empty(); } diff --git a/fe/fe-core/src/main/java/com/starrocks/catalog/OlapTable.java b/fe/fe-core/src/main/java/com/starrocks/catalog/OlapTable.java index 37a1a7531785f..42eeeb9e7ca14 100644 --- a/fe/fe-core/src/main/java/com/starrocks/catalog/OlapTable.java +++ b/fe/fe-core/src/main/java/com/starrocks/catalog/OlapTable.java @@ -645,6 +645,9 @@ public Status resetIdsForRestore(GlobalStateMgr globalStateMgr, Database db, int origIdxIdToName.put(entry.getValue(), entry.getKey()); } + // reset table id + setId(globalStateMgr.getNextId()); + // reset all 'indexIdToXXX' map for (Map.Entry entry : origIdxIdToName.entrySet()) { long newIdxId = globalStateMgr.getNextId(); diff --git a/fe/fe-core/src/main/java/com/starrocks/catalog/Replica.java b/fe/fe-core/src/main/java/com/starrocks/catalog/Replica.java index f1cc89eea547d..64450e756eb15 100644 --- a/fe/fe-core/src/main/java/com/starrocks/catalog/Replica.java +++ b/fe/fe-core/src/main/java/com/starrocks/catalog/Replica.java @@ -399,6 +399,15 @@ public synchronized void updateVersion(long version) { this.lastSuccessVersion, this.minReadableVersion, dataSize, rowCount); } + public synchronized void updateForRestore(long newVersion, long newDataSize, + long newRowCount) { + this.version = newVersion; + this.lastFailedVersion = -1; + this.lastSuccessVersion = newVersion; + this.dataSize = newDataSize; + this.rowCount = newRowCount; + } + /* last failed version: LFV * last success version: LSV * version: V diff --git a/fe/fe-core/src/main/java/com/starrocks/catalog/ScalarType.java b/fe/fe-core/src/main/java/com/starrocks/catalog/ScalarType.java index 51bb744742e67..11d405dd36b00 100644 --- a/fe/fe-core/src/main/java/com/starrocks/catalog/ScalarType.java +++ b/fe/fe-core/src/main/java/com/starrocks/catalog/ScalarType.java @@ -310,8 +310,9 @@ public static ScalarType createDefaultString() { // Use for Hive string now. public static ScalarType createDefaultExternalTableString() { - ScalarType stringType = ScalarType.createVarcharType(ScalarType.MAX_VARCHAR_LENGTH); - return stringType; + // 1GB for each line, it's enough + final int MAX_LENGTH = 1024 * 1024 * 1024; + return ScalarType.createVarcharType(MAX_LENGTH); } public static ScalarType createMaxVarcharType() { diff --git a/fe/fe-core/src/main/java/com/starrocks/common/Config.java b/fe/fe-core/src/main/java/com/starrocks/common/Config.java index adfdd6dcd1b1d..ca2e13fd82697 100644 --- a/fe/fe-core/src/main/java/com/starrocks/common/Config.java +++ b/fe/fe-core/src/main/java/com/starrocks/common/Config.java @@ -1658,6 +1658,12 @@ public class Config extends ConfigBase { @ConfField(mutable = true) public static long statistic_analyze_status_keep_second = 3 * 24 * 3600L; // 3d + /** + * Enable statistics collection profile + */ + @ConfField(mutable = true) + public static boolean enable_statistics_collect_profile = false; + /** * Check expire partition statistics data when StarRocks start up */ diff --git a/fe/fe-core/src/main/java/com/starrocks/common/profile/Tracers.java b/fe/fe-core/src/main/java/com/starrocks/common/profile/Tracers.java index 3b454bb80d500..3f7cc3bb8e17a 100644 --- a/fe/fe-core/src/main/java/com/starrocks/common/profile/Tracers.java +++ b/fe/fe-core/src/main/java/com/starrocks/common/profile/Tracers.java @@ -72,7 +72,8 @@ public static void register() { public static void init(ConnectContext context, Mode mode, String moduleStr) { Tracers tracers = THREAD_LOCAL.get(); - boolean enableProfile = context.getSessionVariable().isEnableProfile(); + boolean enableProfile = + context.getSessionVariable().isEnableProfile() || context.getSessionVariable().isEnableBigQueryProfile(); boolean checkMV = context.getSessionVariable().isEnableMaterializedViewRewriteOrError(); Module module = getTraceModule(moduleStr); diff --git a/fe/fe-core/src/main/java/com/starrocks/connector/PartitionUtil.java b/fe/fe-core/src/main/java/com/starrocks/connector/PartitionUtil.java index 49e6aac8cb602..98fc838308b69 100644 --- a/fe/fe-core/src/main/java/com/starrocks/connector/PartitionUtil.java +++ b/fe/fe-core/src/main/java/com/starrocks/connector/PartitionUtil.java @@ -53,6 +53,7 @@ import com.starrocks.common.FeConstants; import com.starrocks.common.UserException; import com.starrocks.common.util.DateUtils; +import com.starrocks.common.util.TimeUtils; import com.starrocks.connector.exception.StarRocksConnectorException; import com.starrocks.server.GlobalStateMgr; import com.starrocks.sql.analyzer.SemanticException; @@ -67,10 +68,13 @@ import org.apache.iceberg.PartitionField; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.StructLike; +import org.apache.iceberg.types.Types; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import java.time.Instant; import java.time.LocalDateTime; +import java.time.temporal.ChronoUnit; import java.util.ArrayList; import java.util.Collections; import java.util.LinkedHashMap; @@ -94,10 +98,10 @@ public static PartitionKey createPartitionKey(List values, List return createPartitionKey(values, columns, Table.TableType.HIVE); } - public static PartitionKey createPartitionKey(List values, List columns, + public static PartitionKey createPartitionKeyWithType(List values, List types, Table.TableType tableType) throws AnalysisException { - Preconditions.checkState(values.size() == columns.size(), - "columns size is %s, but values size is %s", columns.size(), values.size()); + Preconditions.checkState(values.size() == types.size(), + "types size is %s, but values size is %s", types.size(), values.size()); PartitionKey partitionKey = null; switch (tableType) { @@ -127,7 +131,7 @@ public static PartitionKey createPartitionKey(List values, List // change string value to LiteralExpr, for (int i = 0; i < values.size(); i++) { String rawValue = values.get(i); - Type type = columns.get(i).getType(); + Type type = types.get(i); LiteralExpr exprValue; // rawValue could be null for delta table if (rawValue == null) { @@ -144,6 +148,14 @@ public static PartitionKey createPartitionKey(List values, List return partitionKey; } + public static PartitionKey createPartitionKey(List values, List columns, + Table.TableType tableType) throws AnalysisException { + Preconditions.checkState(values.size() == columns.size(), + "columns size is %s, but values size is %s", columns.size(), values.size()); + + return createPartitionKeyWithType(values, columns.stream().map(Column::getType).collect(Collectors.toList()), tableType); + } + // If partitionName is `par_col=0/par_date=2020-01-01`, return ["0", "2020-01-01"] public static List toPartitionValues(String partitionName) { // mimics Warehouse.makeValsFromName @@ -658,6 +670,14 @@ public static List getIcebergPartitionValues(PartitionSpec spec, StructL Class clazz = spec.javaClasses()[i]; String value = partitionField.transform().toHumanString(getPartitionValue(partitionData, i, clazz)); + + // currently starrocks date literal only support local datetime + org.apache.iceberg.types.Type icebergType = spec.schema().findType(partitionField.sourceId()); + if (partitionField.transform().isIdentity() && icebergType.equals(Types.TimestampType.withZone())) { + value = ChronoUnit.MICROS.addTo(Instant.ofEpochSecond(0).atZone(TimeUtils.getTimeZone().toZoneId()), + getPartitionValue(partitionData, i, clazz)).toLocalDateTime().toString(); + } + partitionValues.add(value); } diff --git a/fe/fe-core/src/main/java/com/starrocks/connector/iceberg/IcebergMetadata.java b/fe/fe-core/src/main/java/com/starrocks/connector/iceberg/IcebergMetadata.java index 38e90e7c2772b..cc2c93cda11f1 100644 --- a/fe/fe-core/src/main/java/com/starrocks/connector/iceberg/IcebergMetadata.java +++ b/fe/fe-core/src/main/java/com/starrocks/connector/iceberg/IcebergMetadata.java @@ -96,8 +96,9 @@ import static com.google.common.collect.ImmutableList.toImmutableList; import static com.starrocks.common.profile.Tracers.Module.EXTERNAL; +import static com.starrocks.connector.ColumnTypeConverter.fromIcebergType; import static com.starrocks.connector.PartitionUtil.convertIcebergPartitionToPartitionName; -import static com.starrocks.connector.PartitionUtil.createPartitionKey; +import static com.starrocks.connector.PartitionUtil.createPartitionKeyWithType; import static com.starrocks.connector.iceberg.IcebergApiConverter.parsePartitionFields; import static com.starrocks.connector.iceberg.IcebergApiConverter.toIcebergApiSchema; import static com.starrocks.connector.iceberg.IcebergCatalogType.GLUE_CATALOG; @@ -299,7 +300,7 @@ public List getPrunedPartitions(Table table, ScalarOperator predic PartitionSpec spec = icebergTable.getNativeTable().spec(); List partitionColumns = icebergTable.getPartitionColumnsIncludeTransformed(); for (FileScanTask fileScanTask : icebergSplitTasks) { - StructLike partitionData = fileScanTask.file().partition(); + org.apache.iceberg.PartitionData partitionData = (org.apache.iceberg.PartitionData) fileScanTask.file().partition(); List values = PartitionUtil.getIcebergPartitionValues(spec, partitionData); if (values.size() != partitionColumns.size()) { @@ -314,7 +315,32 @@ public List getPrunedPartitions(Table table, ScalarOperator predic } try { - partitionKeys.add(createPartitionKey(values, partitionColumns, table.getType())); + List srTypes = new ArrayList<>(); + for (PartitionField partitionField : spec.fields()) { + if (partitionField.transform().isVoid()) { + continue; + } + + if (!partitionField.transform().isIdentity()) { + Type sourceType = spec.schema().findType(partitionField.sourceId()); + Type resultType = partitionField.transform().getResultType(sourceType); + if (resultType == Types.DateType.get()) { + resultType = Types.IntegerType.get(); + } + srTypes.add(fromIcebergType(resultType)); + continue; + } + + srTypes.add(icebergTable.getColumn(partitionField.name()).getType()); + } + + if (icebergTable.hasPartitionTransformedEvolution()) { + srTypes = partitionColumns.stream() + .map(Column::getType) + .collect(Collectors.toList()); + } + + partitionKeys.add(createPartitionKeyWithType(values, srTypes, table.getType())); } catch (Exception e) { LOG.error("create partition key failed.", e); throw new StarRocksConnectorException(e.getMessage()); diff --git a/fe/fe-core/src/main/java/com/starrocks/credential/CloudConfigurationConstants.java b/fe/fe-core/src/main/java/com/starrocks/credential/CloudConfigurationConstants.java index 027a1ac816a7b..0ce2245485927 100644 --- a/fe/fe-core/src/main/java/com/starrocks/credential/CloudConfigurationConstants.java +++ b/fe/fe-core/src/main/java/com/starrocks/credential/CloudConfigurationConstants.java @@ -98,6 +98,7 @@ public class CloudConfigurationConstants { public static final String HDFS_PASSWORD = "hadoop.password"; public static final String HDFS_KERBEROS_PRINCIPAL_DEPRECATED = "kerberos_principal"; public static final String HDFS_KERBEROS_PRINCIPAL = "hadoop.kerberos.principal"; + public static final String HDFS_KERBEROS_TICKET_CACHE_PATH = "hadoop.security.kerberos.ticket.cache.path"; @Deprecated public static final String HDFS_KERBEROS_KEYTAB_DEPRECATED = "kerberos_keytab"; public static final String HADOOP_KERBEROS_KEYTAB = "hadoop.kerberos.keytab"; diff --git a/fe/fe-core/src/main/java/com/starrocks/credential/CloudConfigurationFactory.java b/fe/fe-core/src/main/java/com/starrocks/credential/CloudConfigurationFactory.java index f89e296307234..a61ac84f20da3 100644 --- a/fe/fe-core/src/main/java/com/starrocks/credential/CloudConfigurationFactory.java +++ b/fe/fe-core/src/main/java/com/starrocks/credential/CloudConfigurationFactory.java @@ -21,6 +21,7 @@ import com.starrocks.credential.azure.AzureCloudConfigurationProvider; import com.starrocks.credential.gcp.GCPCloudConfigurationProvoder; import com.starrocks.credential.hdfs.HDFSCloudConfigurationProvider; +import com.starrocks.credential.hdfs.StrictHDFSCloudConfigurationProvider; import com.starrocks.credential.tencent.TencentCloudConfigurationProvider; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.iceberg.aws.AwsProperties; @@ -37,15 +38,28 @@ public class CloudConfigurationFactory { new AliyunCloudConfigurationProvider(), new TencentCloudConfigurationProvider(), new HDFSCloudConfigurationProvider(), - new CloudConfigurationProvider() { - @Override - public CloudConfiguration build(Map properties) { - return new CloudConfiguration(); - } - }); + (Map properties) -> new CloudConfiguration()); + + static ImmutableList strictCloudConfigurationFactoryChain = ImmutableList.of( + new AWSCloudConfigurationProvider(), + new AzureCloudConfigurationProvider(), + new GCPCloudConfigurationProvoder(), + new AliyunCloudConfigurationProvider(), + new TencentCloudConfigurationProvider(), + new HDFSCloudConfigurationProvider(), + new StrictHDFSCloudConfigurationProvider(), + (Map properties) -> new CloudConfiguration()); public static CloudConfiguration buildCloudConfigurationForStorage(Map properties) { - for (CloudConfigurationProvider factory : cloudConfigurationFactoryChain) { + return buildCloudConfigurationForStorage(properties, false); + } + + public static CloudConfiguration buildCloudConfigurationForStorage(Map properties, boolean strictMode) { + ImmutableList factories = cloudConfigurationFactoryChain; + if (strictMode) { + factories = strictCloudConfigurationFactoryChain; + } + for (CloudConfigurationProvider factory : factories) { CloudConfiguration cloudConfiguration = factory.build(properties); if (cloudConfiguration != null) { cloudConfiguration.loadCommonFields(properties); diff --git a/fe/fe-core/src/main/java/com/starrocks/credential/hdfs/HDFSCloudConfigurationProvider.java b/fe/fe-core/src/main/java/com/starrocks/credential/hdfs/HDFSCloudConfigurationProvider.java index 97063173a361b..a6962892e600f 100644 --- a/fe/fe-core/src/main/java/com/starrocks/credential/hdfs/HDFSCloudConfigurationProvider.java +++ b/fe/fe-core/src/main/java/com/starrocks/credential/hdfs/HDFSCloudConfigurationProvider.java @@ -14,7 +14,7 @@ package com.starrocks.credential.hdfs; -import autovalue.shaded.com.google.common.common.base.Preconditions; +import com.google.common.base.Preconditions; import com.starrocks.credential.CloudConfiguration; import com.starrocks.credential.CloudConfigurationProvider; @@ -35,7 +35,7 @@ public class HDFSCloudConfigurationProvider implements CloudConfigurationProvider { - private static String getOrDefault(Map prop, String... args) { + protected static String getOrDefault(Map prop, String... args) { for (String k : args) { String v = prop.get(k); if (v != null) { @@ -45,8 +45,7 @@ private static String getOrDefault(Map prop, String... args) { return ""; } - @Override - public CloudConfiguration build(Map properties) { + protected Map preprocessProperties(Map properties) { Preconditions.checkNotNull(properties); Map prop = new HashMap<>(properties); @@ -59,6 +58,12 @@ public CloudConfiguration build(Map properties) { for (String k : keys) { prop.remove(k); } + return prop; + } + + @Override + public CloudConfiguration build(Map properties) { + Map prop = preprocessProperties(properties); HDFSCloudCredential hdfsCloudCredential = new HDFSCloudCredential( getOrDefault(properties, HDFS_AUTHENTICATION), @@ -72,7 +77,6 @@ public CloudConfiguration build(Map properties) { if (!hdfsCloudCredential.validate()) { return null; } - HDFSCloudConfiguration conf = new HDFSCloudConfiguration(hdfsCloudCredential); - return conf; + return new HDFSCloudConfiguration(hdfsCloudCredential); } } diff --git a/fe/fe-core/src/main/java/com/starrocks/credential/hdfs/HDFSCloudCredential.java b/fe/fe-core/src/main/java/com/starrocks/credential/hdfs/HDFSCloudCredential.java index 72ad0eeee908d..216d3752b392e 100644 --- a/fe/fe-core/src/main/java/com/starrocks/credential/hdfs/HDFSCloudCredential.java +++ b/fe/fe-core/src/main/java/com/starrocks/credential/hdfs/HDFSCloudCredential.java @@ -23,9 +23,12 @@ import java.util.Map; +import static com.starrocks.credential.CloudConfigurationConstants.HDFS_AUTHENTICATION; + public class HDFSCloudCredential implements CloudCredential { - public static final String EMPTY = "empty"; - private String authentication; + public static final String SIMPLE_AUTH = "simple"; + public static final String KERBEROS_AUTH = "kerberos"; + protected String authentication; private String userName; private String password; private String krbPrincipal; @@ -65,19 +68,14 @@ public void applyToConfiguration(Configuration configuration) { @Override public boolean validate() { - if (authentication.equals(EMPTY)) { - return true; - } - - if (authentication.equals("simple")) { + if (SIMPLE_AUTH.equals(authentication)) { return true; } - - if (authentication.equals("kerberos")) { + if (KERBEROS_AUTH.equals(authentication)) { if (krbPrincipal.isEmpty()) { return false; } - return !(krbKeyTabData.isEmpty() && krbKeyTabFile.isEmpty()); + return !(krbKeyTabFile.isEmpty() && krbKeyTabData.isEmpty()); } return false; @@ -104,6 +102,13 @@ public FileStoreInfo toFileStoreInfo() { FileStoreInfo.Builder fileStore = FileStoreInfo.newBuilder(); fileStore.setFsType(FileStoreType.HDFS); HDFSFileStoreInfo.Builder hdfsFileStoreInfo = HDFSFileStoreInfo.newBuilder(); + if (!authentication.isEmpty()) { + hdfsFileStoreInfo.putConfiguration(HDFS_AUTHENTICATION, authentication); + if (authentication.equals(SIMPLE_AUTH) && !userName.isEmpty()) { + hdfsFileStoreInfo.setUsername(userName); + } + } + hdfsFileStoreInfo.putAllConfiguration(hadoopConfiguration); fileStore.setHdfsFsInfo(hdfsFileStoreInfo.build()); return fileStore.build(); } diff --git a/fe/fe-core/src/main/java/com/starrocks/credential/hdfs/StrictHDFSCloudConfigurationProvider.java b/fe/fe-core/src/main/java/com/starrocks/credential/hdfs/StrictHDFSCloudConfigurationProvider.java new file mode 100644 index 0000000000000..5b595b268946a --- /dev/null +++ b/fe/fe-core/src/main/java/com/starrocks/credential/hdfs/StrictHDFSCloudConfigurationProvider.java @@ -0,0 +1,47 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// Licensed 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 +// +// https://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 com.starrocks.credential.hdfs; + +import com.starrocks.credential.CloudConfiguration; +import com.starrocks.credential.CloudConfigurationConstants; + +import java.util.Map; + +public class StrictHDFSCloudConfigurationProvider extends HDFSCloudConfigurationProvider { + @Override + public CloudConfiguration build(Map properties) { + Map prop = preprocessProperties(properties); + + HDFSCloudCredential hdfsCloudCredential = new StrictHDFSCloudCredential( + getOrDefault(properties, CloudConfigurationConstants.HDFS_AUTHENTICATION), + getOrDefault(properties, CloudConfigurationConstants.HDFS_USERNAME, + CloudConfigurationConstants.HDFS_USERNAME_DEPRECATED), + getOrDefault(properties, CloudConfigurationConstants.HDFS_PASSWORD, + CloudConfigurationConstants.HDFS_PASSWORD_DEPRECATED), + getOrDefault(properties, CloudConfigurationConstants.HDFS_KERBEROS_PRINCIPAL, + CloudConfigurationConstants.HDFS_KERBEROS_PRINCIPAL_DEPRECATED), + getOrDefault(properties, CloudConfigurationConstants.HADOOP_KERBEROS_KEYTAB, + CloudConfigurationConstants.HDFS_KERBEROS_KEYTAB_DEPRECATED), + getOrDefault(properties, CloudConfigurationConstants.HADOOP_KERBEROS_KEYTAB_CONTENT, + CloudConfigurationConstants.HDFS_KERBEROS_KEYTAB_CONTENT_DEPRECATED), + prop + ); + if (!hdfsCloudCredential.validate()) { + return null; + } + + return new HDFSCloudConfiguration(hdfsCloudCredential); + } +} diff --git a/fe/fe-core/src/main/java/com/starrocks/credential/hdfs/StrictHDFSCloudCredential.java b/fe/fe-core/src/main/java/com/starrocks/credential/hdfs/StrictHDFSCloudCredential.java new file mode 100644 index 0000000000000..b89d31f83a01c --- /dev/null +++ b/fe/fe-core/src/main/java/com/starrocks/credential/hdfs/StrictHDFSCloudCredential.java @@ -0,0 +1,32 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// Licensed 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 +// +// https://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 com.starrocks.credential.hdfs; + +import java.util.Map; + +public class StrictHDFSCloudCredential extends HDFSCloudCredential { + public StrictHDFSCloudCredential(String authentication, String username, String password, String kerberosPrincipal, + String keytab, String keytabContent, Map hadoopConfiguration) { + super(authentication, username, password, kerberosPrincipal, keytab, keytabContent, hadoopConfiguration); + } + + @Override + public boolean validate() { + if (!authentication.isEmpty()) { + return authentication.equals(SIMPLE_AUTH) || authentication.equals(KERBEROS_AUTH); + } + return true; + } +} diff --git a/fe/fe-core/src/main/java/com/starrocks/load/loadv2/LoadLoadingTask.java b/fe/fe-core/src/main/java/com/starrocks/load/loadv2/LoadLoadingTask.java index 36bfb996f6c78..6cc8dcec17d57 100644 --- a/fe/fe-core/src/main/java/com/starrocks/load/loadv2/LoadLoadingTask.java +++ b/fe/fe-core/src/main/java/com/starrocks/load/loadv2/LoadLoadingTask.java @@ -217,7 +217,7 @@ private void executeOnce() throws Exception { curCoordinator.getQueryProfile().getCounterTotalTime() .setValue(TimeUtils.getEstimatedTime(beginTimeInNanoSecond)); curCoordinator.collectProfileSync(); - profile.addChild(curCoordinator.buildMergedQueryProfile()); + profile.addChild(curCoordinator.buildQueryProfile(context.needMergeProfile())); StringBuilder builder = new StringBuilder(); profile.prettyPrint(builder, ""); diff --git a/fe/fe-core/src/main/java/com/starrocks/load/streamload/StreamLoadTask.java b/fe/fe-core/src/main/java/com/starrocks/load/streamload/StreamLoadTask.java index 13d2e0c149bce..a10416879b881 100644 --- a/fe/fe-core/src/main/java/com/starrocks/load/streamload/StreamLoadTask.java +++ b/fe/fe-core/src/main/java/com/starrocks/load/streamload/StreamLoadTask.java @@ -1058,7 +1058,7 @@ public void collectProfile() { if (coord.getQueryProfile() != null) { if (!isSyncStreamLoad()) { coord.collectProfileSync(); - profile.addChild(coord.buildMergedQueryProfile()); + profile.addChild(coord.buildQueryProfile(session == null || session.needMergeProfile())); } else { profile.addChild(coord.getQueryProfile()); } diff --git a/fe/fe-core/src/main/java/com/starrocks/planner/IcebergScanNode.java b/fe/fe-core/src/main/java/com/starrocks/planner/IcebergScanNode.java index 1e8c5d12d4641..779c78b43d403 100644 --- a/fe/fe-core/src/main/java/com/starrocks/planner/IcebergScanNode.java +++ b/fe/fe-core/src/main/java/com/starrocks/planner/IcebergScanNode.java @@ -31,6 +31,7 @@ import com.starrocks.catalog.Type; import com.starrocks.common.AnalysisException; import com.starrocks.common.UserException; +import com.starrocks.common.util.TimeUtils; import com.starrocks.connector.CatalogConnector; import com.starrocks.connector.PartitionUtil; import com.starrocks.connector.RemoteFileDesc; @@ -65,9 +66,12 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Snapshot; import org.apache.iceberg.StructLike; +import org.apache.iceberg.types.Types; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import java.time.Instant; +import java.time.temporal.ChronoUnit; import java.util.ArrayList; import java.util.HashSet; import java.util.List; @@ -189,6 +193,12 @@ private PartitionKey getPartitionKey(StructLike partition, PartitionSpec spec, L String partitionValue; partitionValue = field.transform().toHumanString(type, PartitionUtil.getPartitionValue(partition, index, javaClass)); + + // currently starrocks date literal only support local datetime + if (type.equals(Types.TimestampType.withZone())) { + partitionValue = ChronoUnit.MICROS.addTo(Instant.ofEpochSecond(0).atZone(TimeUtils.getTimeZone().toZoneId()), + PartitionUtil.getPartitionValue(partition, index, javaClass)).toLocalDateTime().toString(); + } partitionValues.add(partitionValue); cols.add(srIcebergTable.getColumn(field.name())); diff --git a/fe/fe-core/src/main/java/com/starrocks/qe/ConnectContext.java b/fe/fe-core/src/main/java/com/starrocks/qe/ConnectContext.java index bfc32de3aaa50..7f3e9a238485c 100644 --- a/fe/fe-core/src/main/java/com/starrocks/qe/ConnectContext.java +++ b/fe/fe-core/src/main/java/com/starrocks/qe/ConnectContext.java @@ -64,6 +64,7 @@ import com.starrocks.sql.optimizer.dump.DumpInfo; import com.starrocks.sql.optimizer.dump.QueryDumpInfo; import com.starrocks.sql.parser.SqlParser; +import com.starrocks.thrift.TPipelineProfileLevel; import com.starrocks.thrift.TUniqueId; import com.starrocks.thrift.TWorkGroup; import org.apache.logging.log4j.LogManager; @@ -557,6 +558,25 @@ public void setLastQueryId(UUID queryId) { this.lastQueryId = queryId; } + public boolean isProfileEnabled() { + if (sessionVariable == null) { + return false; + } + if (sessionVariable.isEnableProfile()) { + return true; + } + if (!sessionVariable.isEnableBigQueryProfile()) { + return false; + } + return System.currentTimeMillis() - getStartTime() > + 1000L * sessionVariable.getBigQueryProfileSecondThreshold(); + } + + public boolean needMergeProfile() { + return isProfileEnabled() && + sessionVariable.getPipelineProfileLevel() < TPipelineProfileLevel.DETAIL.getValue(); + } + public byte[] getAuthDataSalt() { return authDataSalt; } diff --git a/fe/fe-core/src/main/java/com/starrocks/qe/DefaultCoordinator.java b/fe/fe-core/src/main/java/com/starrocks/qe/DefaultCoordinator.java index 629dba265e690..7b09bcdb9dacb 100644 --- a/fe/fe-core/src/main/java/com/starrocks/qe/DefaultCoordinator.java +++ b/fe/fe-core/src/main/java/com/starrocks/qe/DefaultCoordinator.java @@ -788,10 +788,9 @@ public void cancel(PPlanFragmentCancelReason reason, String message) { cancelInternal(reason); } finally { try { - // when enable_profile is true, it disable count down profileDoneSignal for collect all backend's profile + // Disable count down profileDoneSignal for collect all backend's profile // but if backend has crashed, we need count down profileDoneSignal since it will not report by itself - if (connectContext.getSessionVariable().isEnableProfile() && - message.equals(FeConstants.BACKEND_NODE_NOT_FOUND_ERROR)) { + if (message.equals(FeConstants.BACKEND_NODE_NOT_FOUND_ERROR)) { queryProfile.finishAllInstances(Status.OK); LOG.info("count down profileDoneSignal since backend has crashed, query id: {}", DebugUtil.printId(jobSpec.getQueryId())); @@ -813,7 +812,7 @@ private void cancelInternal(PPlanFragmentCancelReason cancelReason) { cancelRemoteFragmentsAsync(cancelReason); if (cancelReason != PPlanFragmentCancelReason.LIMIT_REACH) { // count down to zero to notify all objects waiting for this - if (!connectContext.getSessionVariable().isEnableProfile()) { + if (!connectContext.isProfileEnabled()) { queryProfile.finishAllInstances(Status.OK); } } @@ -1021,8 +1020,8 @@ public boolean join(int timeoutS) { } @Override - public RuntimeProfile buildMergedQueryProfile() { - return queryProfile.buildMergedQueryProfile(); + public RuntimeProfile buildQueryProfile(boolean needMerge) { + return queryProfile.buildQueryProfile(needMerge); } /** diff --git a/fe/fe-core/src/main/java/com/starrocks/qe/SessionVariable.java b/fe/fe-core/src/main/java/com/starrocks/qe/SessionVariable.java index 591cd6b023103..0d401426695c8 100644 --- a/fe/fe-core/src/main/java/com/starrocks/qe/SessionVariable.java +++ b/fe/fe-core/src/main/java/com/starrocks/qe/SessionVariable.java @@ -462,6 +462,7 @@ public static MaterializedViewRewriteMode parse(String str) { public static final String BIG_QUERY_LOG_CPU_SECOND_THRESHOLD = "big_query_log_cpu_second_threshold"; public static final String BIG_QUERY_LOG_SCAN_BYTES_THRESHOLD = "big_query_log_scan_bytes_threshold"; public static final String BIG_QUERY_LOG_SCAN_ROWS_THRESHOLD = "big_query_log_scan_rows_threshold"; + public static final String BIG_QUERY_PROFILE_SECOND_THRESHOLD = "big_query_profile_second_threshold"; public static final String SQL_DIALECT = "sql_dialect"; @@ -847,6 +848,9 @@ public static MaterializedViewRewriteMode parse(String str) { @VariableMgr.VarAttr(name = ENABLE_ASYNC_PROFILE, flag = VariableMgr.INVISIBLE) private boolean enableAsyncProfile = true; + @VariableMgr.VarAttr(name = BIG_QUERY_PROFILE_SECOND_THRESHOLD) + private int bigQueryProfileSecondThreshold = 0; + @VariableMgr.VarAttr(name = RESOURCE_GROUP_ID, alias = RESOURCE_GROUP_ID_V2, show = RESOURCE_GROUP_ID_V2, flag = VariableMgr.INVISIBLE) private int resourceGroupId = 0; @@ -1653,6 +1657,14 @@ public void setEnableLoadProfile(boolean enableLoadProfile) { this.enableLoadProfile = enableLoadProfile; } + public boolean isEnableBigQueryProfile() { + return bigQueryProfileSecondThreshold > 0; + } + + public int getBigQueryProfileSecondThreshold() { + return bigQueryProfileSecondThreshold; + } + public int getWaitTimeoutS() { return waitTimeout; } @@ -2778,6 +2790,7 @@ public TQueryOptions toThrift() { tResult.setQuery_timeout(Math.min(Integer.MAX_VALUE / 1000, queryTimeoutS)); tResult.setQuery_delivery_timeout(Math.min(Integer.MAX_VALUE / 1000, queryDeliveryTimeoutS)); tResult.setEnable_profile(enableProfile); + tResult.setBig_query_profile_second_threshold(bigQueryProfileSecondThreshold); tResult.setRuntime_profile_report_interval(runtimeProfileReportInterval); tResult.setBatch_size(chunkSize); tResult.setLoad_mem_limit(loadMemLimit); diff --git a/fe/fe-core/src/main/java/com/starrocks/qe/StmtExecutor.java b/fe/fe-core/src/main/java/com/starrocks/qe/StmtExecutor.java index a47f277101ef6..95bd9db1b2568 100644 --- a/fe/fe-core/src/main/java/com/starrocks/qe/StmtExecutor.java +++ b/fe/fe-core/src/main/java/com/starrocks/qe/StmtExecutor.java @@ -612,7 +612,7 @@ public void execute() throws Exception { } } finally { boolean isAsync = false; - if (!needRetry && context.getSessionVariable().isEnableProfile()) { + if (!needRetry && context.isProfileEnabled()) { isAsync = tryProcessProfileAsync(execPlan); if (parsedStmt.isExplain() && StatementBase.ExplainLevel.ANALYZE.equals(parsedStmt.getExplainLevel())) { @@ -790,6 +790,7 @@ private boolean tryProcessProfileAsync(ExecPlan plan) { long startTime = context.getStartTime(); TUniqueId executionId = context.getExecutionId(); QueryDetail queryDetail = context.getQueryDetail(); + boolean needMerge = context.needMergeProfile(); // DO NOT use context int the async task, because the context is shared among consecutive queries. // profile of query1 maybe executed when query2 is under execution. @@ -798,7 +799,7 @@ private boolean tryProcessProfileAsync(ExecPlan plan) { summaryProfile.addInfoString(ProfileManager.PROFILE_COLLECT_TIME, DebugUtil.getPrettyStringMs(System.currentTimeMillis() - profileCollectStartTime)); summaryProfile.addInfoString("IsProfileAsync", String.valueOf(isAsync)); - profile.addChild(coord.buildMergedQueryProfile()); + profile.addChild(coord.buildQueryProfile(needMerge)); // Update TotalTime to include the Profile Collect Time and the time to build the profile. long now = System.currentTimeMillis(); @@ -1692,7 +1693,7 @@ public void handleDMLStmtWithProfile(ExecPlan execPlan, DmlStmt stmt) throws Exc throw t; } finally { boolean isAsync = false; - if (context.getSessionVariable().isEnableProfile()) { + if (context.isProfileEnabled()) { isAsync = tryProcessProfileAsync(execPlan); if (parsedStmt.isExplain() && StatementBase.ExplainLevel.ANALYZE.equals(parsedStmt.getExplainLevel())) { diff --git a/fe/fe-core/src/main/java/com/starrocks/qe/scheduler/Coordinator.java b/fe/fe-core/src/main/java/com/starrocks/qe/scheduler/Coordinator.java index d76e69cee9b98..2d2d3255cde0d 100644 --- a/fe/fe-core/src/main/java/com/starrocks/qe/scheduler/Coordinator.java +++ b/fe/fe-core/src/main/java/com/starrocks/qe/scheduler/Coordinator.java @@ -161,7 +161,7 @@ public void cancel(String cancelledMessage) { public abstract void setExecPlan(ExecPlan execPlan); - public abstract RuntimeProfile buildMergedQueryProfile(); + public abstract RuntimeProfile buildQueryProfile(boolean needMerge); public abstract RuntimeProfile getQueryProfile(); diff --git a/fe/fe-core/src/main/java/com/starrocks/qe/scheduler/QueryRuntimeProfile.java b/fe/fe-core/src/main/java/com/starrocks/qe/scheduler/QueryRuntimeProfile.java index a2d2496556c38..7c32d7d8c7693 100644 --- a/fe/fe-core/src/main/java/com/starrocks/qe/scheduler/QueryRuntimeProfile.java +++ b/fe/fe-core/src/main/java/com/starrocks/qe/scheduler/QueryRuntimeProfile.java @@ -35,7 +35,6 @@ import com.starrocks.qe.scheduler.dag.JobSpec; import com.starrocks.sql.plan.ExecPlan; import com.starrocks.task.LoadEtlTask; -import com.starrocks.thrift.TPipelineProfileLevel; import com.starrocks.thrift.TReportExecStatusParams; import com.starrocks.thrift.TSinkCommitInfo; import com.starrocks.thrift.TTabletCommitInfo; @@ -267,7 +266,7 @@ public void updateProfile(FragmentInstanceExecState execState, TReportExecStatus long now = System.currentTimeMillis(); long lastTime = lastRuntimeProfileUpdateTime.get(); if (topProfileSupplier != null && execPlan != null && connectContext != null && - connectContext.getSessionVariable().isEnableProfile() && + connectContext.isProfileEnabled() && // If it's the last done report, avoiding duplicate trigger (!execState.isFinished() || profileDoneSignal.getLeftMarks().size() > 1) && // Interval * 0.95 * 1000 to allow a certain range of deviation @@ -275,7 +274,7 @@ public void updateProfile(FragmentInstanceExecState execState, TReportExecStatus lastRuntimeProfileUpdateTime.compareAndSet(lastTime, now)) { RuntimeProfile profile = topProfileSupplier.get(); ExecPlan plan = execPlan; - profile.addChild(buildMergedQueryProfile()); + profile.addChild(buildQueryProfile(connectContext.needMergeProfile())); ProfilingExecPlan profilingPlan = plan == null ? null : plan.getProfilingPlan(); ProfileManager.getInstance().pushProfile(profilingPlan, profile); } @@ -312,20 +311,8 @@ public void updateLoadInformation(FragmentInstanceExecState execState, TReportEx } } - public RuntimeProfile buildMergedQueryProfile() { - SessionVariable sessionVariable = connectContext.getSessionVariable(); - - if (!sessionVariable.isEnableProfile()) { - return queryProfile; - } - - if (!jobSpec.isEnablePipeline()) { - return queryProfile; - } - - int profileLevel = sessionVariable.getPipelineProfileLevel(); - if (profileLevel >= TPipelineProfileLevel.DETAIL.getValue()) { - // We don't guarantee the detail level profile can work well with visualization feature. + public RuntimeProfile buildQueryProfile(boolean needMerge) { + if (!needMerge || !jobSpec.isEnablePipeline()) { return queryProfile; } diff --git a/fe/fe-core/src/main/java/com/starrocks/qe/scheduler/dag/JobSpec.java b/fe/fe-core/src/main/java/com/starrocks/qe/scheduler/dag/JobSpec.java index faf9401ccfff2..431e9e425e723 100644 --- a/fe/fe-core/src/main/java/com/starrocks/qe/scheduler/dag/JobSpec.java +++ b/fe/fe-core/src/main/java/com/starrocks/qe/scheduler/dag/JobSpec.java @@ -102,7 +102,8 @@ public static JobSpec fromQuerySpec(ConnectContext context, .descTable(descTable) .enableStreamPipeline(false) .isBlockQuery(false) - .needReport(context.getSessionVariable().isEnableProfile()) + .needReport(context.getSessionVariable().isEnableProfile() || + context.getSessionVariable().isEnableBigQueryProfile()) .queryGlobals(queryGlobals) .queryOptions(queryOptions) .commonProperties(context) diff --git a/fe/fe-core/src/main/java/com/starrocks/server/LocalMetastore.java b/fe/fe-core/src/main/java/com/starrocks/server/LocalMetastore.java index 2d50d0f4a0c66..695a6e7967c9a 100644 --- a/fe/fe-core/src/main/java/com/starrocks/server/LocalMetastore.java +++ b/fe/fe-core/src/main/java/com/starrocks/server/LocalMetastore.java @@ -116,6 +116,7 @@ import com.starrocks.common.NotImplementedException; import com.starrocks.common.Pair; import com.starrocks.common.Status; +import com.starrocks.common.TimeoutException; import com.starrocks.common.UserException; import com.starrocks.common.util.DynamicPartitionUtil; import com.starrocks.common.util.PropertyAnalyzer; @@ -1700,7 +1701,7 @@ private PhysicalPartition createPhysicalPartition(Database db, OlapTable olapTab physicalParition.createRollupIndex(index); } } - + return physicalParition; } @@ -1914,63 +1915,62 @@ private void buildPartitionsSequentially(long dbId, OlapTable table, List partitions, int numReplicas, + private void buildPartitionsConcurrently(long dbId, OlapTable table, List partitions, + int numReplicas, int numBackends) throws DdlException { int timeout = Math.max(1, numReplicas / numBackends) * Config.tablet_create_timeout_second; int numIndexes = partitions.stream().mapToInt( partition -> partition.getMaterializedIndices(IndexExtState.VISIBLE).size()).sum(); int maxTimeout = numIndexes * Config.max_create_table_timeout_second; MarkedCountDownLatch countDownLatch = new MarkedCountDownLatch<>(numReplicas); - Thread t = new Thread(() -> { - Map> taskSignatures = new HashMap<>(); - try { - int numFinishedTasks; - int numSendedTasks = 0; - for (PhysicalPartition partition : partitions) { - if (!countDownLatch.getStatus().ok()) { - break; - } - List tasks = buildCreateReplicaTasks(dbId, table, partition); - for (CreateReplicaTask task : tasks) { - List signatures = - taskSignatures.computeIfAbsent(task.getBackendId(), k -> new ArrayList<>()); - signatures.add(task.getSignature()); + Map> taskSignatures = new HashMap<>(); + try { + int numFinishedTasks; + int numSendedTasks = 0; + long startTime = System.currentTimeMillis(); + long maxWaitTimeMs = Math.min(timeout, maxTimeout) * 1000L; + for (PhysicalPartition partition : partitions) { + if (!countDownLatch.getStatus().ok()) { + break; + } + List tasks = buildCreateReplicaTasks(dbId, table, partition); + for (CreateReplicaTask task : tasks) { + List signatures = + taskSignatures.computeIfAbsent(task.getBackendId(), k -> new ArrayList<>()); + signatures.add(task.getSignature()); + } + sendCreateReplicaTasks(tasks, countDownLatch); + numSendedTasks += tasks.size(); + numFinishedTasks = numReplicas - (int) countDownLatch.getCount(); + // Since there is no mechanism to cancel tasks, if we send a lot of tasks at once and some error or timeout + // occurs in the middle of the process, it will create a lot of useless replicas that will be deleted soon and + // waste machine resources. Sending a lot of tasks at once may also block other users' tasks for a long time. + // To avoid these situations, new tasks are sent only when the average number of tasks on each node is less + // than 200. + // (numSendedTasks - numFinishedTasks) is number of tasks that have been sent but not yet finished. + while (numSendedTasks - numFinishedTasks > 200 * numBackends) { + long currentTime = System.currentTimeMillis(); + // Add timeout check + if (currentTime > startTime + maxWaitTimeMs) { + throw new TimeoutException("Wait in buildPartitionsConcurrently exceeded timeout"); } - sendCreateReplicaTasks(tasks, countDownLatch); - numSendedTasks += tasks.size(); + ThreadUtil.sleepAtLeastIgnoreInterrupts(100); numFinishedTasks = numReplicas - (int) countDownLatch.getCount(); - // Since there is no mechanism to cancel tasks, if we send a lot of tasks at once and some error or timeout - // occurs in the middle of the process, it will create a lot of useless replicas that will be deleted soon and - // waste machine resources. Sending a lot of tasks at once may also block other users' tasks for a long time. - // To avoid these situations, new tasks are sent only when the average number of tasks on each node is less - // than 200. - // (numSendedTasks - numFinishedTasks) is number of tasks that have been sent but not yet finished. - while (numSendedTasks - numFinishedTasks > 200 * numBackends) { - ThreadUtil.sleepAtLeastIgnoreInterrupts(100); - numFinishedTasks = numReplicas - (int) countDownLatch.getCount(); - } - } - countDownLatch.await(); - if (countDownLatch.getStatus().ok()) { - taskSignatures.clear(); } - } catch (Exception e) { - LOG.warn(e); - countDownLatch.countDownToZero(new Status(TStatusCode.UNKNOWN, e.toString())); - } finally { + } + waitForFinished(countDownLatch, Math.min(timeout, maxTimeout)); + } catch (Exception e) { + LOG.warn(e); + countDownLatch.countDownToZero(new Status(TStatusCode.UNKNOWN, e.getMessage())); + throw new DdlException(e.getMessage()); + } finally { + if (!countDownLatch.getStatus().ok()) { for (Map.Entry> entry : taskSignatures.entrySet()) { for (Long signature : entry.getValue()) { AgentTaskQueue.removeTask(entry.getKey(), TTaskType.CREATE, signature); } } } - }, "partition-build"); - t.start(); - try { - waitForFinished(countDownLatch, Math.min(timeout, maxTimeout)); - } catch (Exception e) { - countDownLatch.countDownToZero(new Status(TStatusCode.UNKNOWN, e.getMessage())); - throw e; } } @@ -5382,4 +5382,3 @@ public void load(SRMetaBlockReader reader) throws IOException, SRMetaBlockExcept defaultCluster = new Cluster(SystemInfoService.DEFAULT_CLUSTER, NEXT_ID_INIT_VALUE); } } - diff --git a/fe/fe-core/src/main/java/com/starrocks/sql/optimizer/operator/logical/LogicalIcebergScanOperator.java b/fe/fe-core/src/main/java/com/starrocks/sql/optimizer/operator/logical/LogicalIcebergScanOperator.java index 0c5aff73a76bd..1f80a693723ef 100644 --- a/fe/fe-core/src/main/java/com/starrocks/sql/optimizer/operator/logical/LogicalIcebergScanOperator.java +++ b/fe/fe-core/src/main/java/com/starrocks/sql/optimizer/operator/logical/LogicalIcebergScanOperator.java @@ -63,7 +63,9 @@ public void setScanOperatorPredicates(ScanOperatorPredicates predicates) { @Override public boolean isEmptyOutputRows() { - return !table.isUnPartitioned() && predicates.getSelectedPartitionIds().isEmpty(); + return !table.isUnPartitioned() && + !(((IcebergTable) table).hasPartitionTransformedEvolution()) && + predicates.getSelectedPartitionIds().isEmpty(); } @Override diff --git a/fe/fe-core/src/main/java/com/starrocks/statistic/StatisticExecutor.java b/fe/fe-core/src/main/java/com/starrocks/statistic/StatisticExecutor.java index b692df7086027..2584ab2615457 100644 --- a/fe/fe-core/src/main/java/com/starrocks/statistic/StatisticExecutor.java +++ b/fe/fe-core/src/main/java/com/starrocks/statistic/StatisticExecutor.java @@ -22,6 +22,7 @@ import com.starrocks.catalog.OlapTable; import com.starrocks.catalog.Partition; import com.starrocks.catalog.Table; +import com.starrocks.common.Config; import com.starrocks.common.Pair; import com.starrocks.common.Status; import com.starrocks.common.util.DebugUtil; @@ -261,6 +262,7 @@ public AnalyzeStatus collectStatistics(ConnectContext statsConnectCtx, Table table = statsJob.getTable(); try { + statsConnectCtx.getSessionVariable().setEnableProfile(Config.enable_statistics_collect_profile); GlobalStateMgr.getCurrentAnalyzeMgr().registerConnection(analyzeStatus.getId(), statsConnectCtx); // Only update running status without edit log, make restart job status is failed analyzeStatus.setStatus(StatsConstants.ScheduleStatus.RUNNING); diff --git a/fe/fe-core/src/main/java/com/starrocks/storagevolume/StorageVolume.java b/fe/fe-core/src/main/java/com/starrocks/storagevolume/StorageVolume.java index 79a965b60fd9f..d5aeb1228acda 100644 --- a/fe/fe-core/src/main/java/com/starrocks/storagevolume/StorageVolume.java +++ b/fe/fe-core/src/main/java/com/starrocks/storagevolume/StorageVolume.java @@ -21,6 +21,7 @@ import com.staros.proto.AzBlobCredentialInfo; import com.staros.proto.AzBlobFileStoreInfo; import com.staros.proto.FileStoreInfo; +import com.staros.proto.HDFSFileStoreInfo; import com.staros.proto.S3FileStoreInfo; import com.starrocks.common.io.Text; import com.starrocks.common.io.Writable; @@ -29,7 +30,6 @@ import com.starrocks.credential.CloudConfigurationConstants; import com.starrocks.credential.CloudConfigurationFactory; import com.starrocks.credential.CloudType; -import com.starrocks.credential.hdfs.HDFSCloudCredential; import com.starrocks.persist.gson.GsonPostProcessable; import com.starrocks.persist.gson.GsonUtils; import com.starrocks.server.GlobalStateMgr; @@ -44,9 +44,6 @@ import java.util.List; import java.util.Map; -import static com.starrocks.credential.CloudConfigurationConstants.AZURE_BLOB_CONTAINER; -import static com.starrocks.credential.CloudConfigurationConstants.HDFS_AUTHENTICATION; - public class StorageVolume implements Writable, GsonPostProcessable { public enum StorageVolumeType { UNKNOWN, @@ -93,7 +90,7 @@ public StorageVolume(String id, String name, String svt, List locations, this.params = new HashMap<>(params); Map configurationParams = new HashMap<>(params); preprocessAuthenticationIfNeeded(configurationParams); - this.cloudConfiguration = CloudConfigurationFactory.buildCloudConfigurationForStorage(configurationParams); + this.cloudConfiguration = CloudConfigurationFactory.buildCloudConfigurationForStorage(configurationParams, true); if (!isValidCloudConfiguration()) { Gson gson = new Gson(); throw new SemanticException("Storage params is not valid " + gson.toJson(params)); @@ -107,7 +104,7 @@ public StorageVolume(StorageVolume sv) { this.locations = new ArrayList<>(sv.locations); this.comment = sv.comment; this.enabled = sv.enabled; - this.cloudConfiguration = CloudConfigurationFactory.buildCloudConfigurationForStorage(sv.params); + this.cloudConfiguration = CloudConfigurationFactory.buildCloudConfigurationForStorage(sv.params, true); this.params = new HashMap<>(sv.params); } @@ -249,7 +246,12 @@ public static Map getParamsFromFileStoreInfo(FileStoreInfo fsInf } return params; case HDFS: - // TODO + HDFSFileStoreInfo hdfsFileStoreInfo = fsInfo.getHdfsFsInfo(); + params.putAll(hdfsFileStoreInfo.getConfigurationMap()); + String userName = hdfsFileStoreInfo.getUsername(); + if (!Strings.isNullOrEmpty(userName)) { + params.put(CloudConfigurationConstants.HDFS_USERNAME_DEPRECATED, userName); + } return params; case AZBLOB: AzBlobFileStoreInfo azBlobFileStoreInfo = fsInfo.getAzblobFsInfo(); @@ -270,11 +272,9 @@ public static Map getParamsFromFileStoreInfo(FileStoreInfo fsInf } private void preprocessAuthenticationIfNeeded(Map params) { - if (svt == StorageVolumeType.HDFS) { - params.computeIfAbsent(HDFS_AUTHENTICATION, key -> HDFSCloudCredential.EMPTY); - } else if (svt == StorageVolumeType.AZBLOB) { + if (svt == StorageVolumeType.AZBLOB) { String container = locations.get(0).split("/")[0]; - params.put(AZURE_BLOB_CONTAINER, container); + params.put(CloudConfigurationConstants.AZURE_BLOB_CONTAINER, container); } } diff --git a/fe/fe-core/src/test/java/com/starrocks/catalog/TypeTest.java b/fe/fe-core/src/test/java/com/starrocks/catalog/TypeTest.java index 67268efda43e3..af369a7172929 100644 --- a/fe/fe-core/src/test/java/com/starrocks/catalog/TypeTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/catalog/TypeTest.java @@ -251,7 +251,7 @@ public void testMapSerialAndDeser() { String json = GsonUtils.GSON.toJson(mapType); Type deType = GsonUtils.GSON.fromJson(json, Type.class); Assert.assertTrue(deType.isMapType()); - Assert.assertEquals("MAP>", deType.toString()); + Assert.assertEquals("MAP>", deType.toString()); // Make sure select fields are false when initialized Assert.assertFalse(deType.selectedFields[0]); Assert.assertFalse(deType.selectedFields[1]); @@ -271,7 +271,7 @@ public void testStructSerialAndDeser() { String json = GsonUtils.GSON.toJson(root); Type deType = GsonUtils.GSON.fromJson(json, Type.class); Assert.assertTrue(deType.isStructType()); - Assert.assertEquals("struct>", + Assert.assertEquals("struct>", deType.toString()); // test initialed fieldMap by ctor in deserializer. Assert.assertEquals(1, ((StructType) deType).getFieldPos("c1")); diff --git a/fe/fe-core/src/test/java/com/starrocks/connector/iceberg/IcebergMetadataTest.java b/fe/fe-core/src/test/java/com/starrocks/connector/iceberg/IcebergMetadataTest.java index 9b5d5624d3d1a..3d28858d69350 100644 --- a/fe/fe-core/src/test/java/com/starrocks/connector/iceberg/IcebergMetadataTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/connector/iceberg/IcebergMetadataTest.java @@ -55,6 +55,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.BaseTable; import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; import org.apache.iceberg.FileScanTask; import org.apache.iceberg.Snapshot; import org.apache.iceberg.TableScan; @@ -76,6 +77,8 @@ import java.util.Map; import static com.starrocks.catalog.Table.TableType.ICEBERG; +import static com.starrocks.catalog.Type.DATE; +import static com.starrocks.catalog.Type.DATETIME; import static com.starrocks.catalog.Type.INT; import static com.starrocks.catalog.Type.STRING; import static com.starrocks.connector.iceberg.IcebergConnector.HIVE_METASTORE_URIS; @@ -654,7 +657,7 @@ public void testPartitionPrune() { Assert.assertEquals(1, partitionKeys.size()); Assert.assertTrue(partitionKeys.get(0) instanceof IcebergPartitionKey); IcebergPartitionKey partitionKey = (IcebergPartitionKey) partitionKeys.get(0); - Assert.assertEquals("types: [VARCHAR]; keys: [0]; ", partitionKey.toString()); + Assert.assertEquals("types: [INT]; keys: [0]; ", partitionKey.toString()); mockedNativeTableA.newFastAppend().appendFile(FILE_A_2).commit(); mockedNativeTableA.refresh(); @@ -680,7 +683,7 @@ public void testPartitionPruneWithDuplicated() { Assert.assertEquals(1, partitionKeys.size()); Assert.assertTrue(partitionKeys.get(0) instanceof IcebergPartitionKey); PartitionKey partitionKey = partitionKeys.get(0); - Assert.assertEquals("types: [VARCHAR]; keys: [0]; ", partitionKey.toString()); + Assert.assertEquals("types: [INT]; keys: [0]; ", partitionKey.toString()); } @Test @@ -724,4 +727,81 @@ public void testGetRepeatedTableStats() { new OptimizerContext(null, null), icebergTable, colRefToColumnMetaMap, null, null, -1); Assert.assertEquals(2.0, statistics.getOutputRowCount(), 0.001); } + + @Test + public void testTimeStampIdentityPartitionPrune() { + Map config = new HashMap<>(); + config.put(HIVE_METASTORE_URIS, "thrift://188.122.12.1:8732"); + config.put(ICEBERG_CATALOG_TYPE, "hive"); + IcebergHiveCatalog icebergHiveCatalog = new IcebergHiveCatalog("iceberg_catalog", new Configuration(), config); + List columns = Lists.newArrayList(new Column("k1", INT), new Column("ts", DATETIME)); + IcebergMetadata metadata = new IcebergMetadata(CATALOG_NAME, HDFS_ENVIRONMENT, icebergHiveCatalog); + IcebergTable icebergTable = new IcebergTable(1, "srTableName", "iceberg_catalog", "resource_name", "db_name", + "table_name", columns, mockedNativeTableE, Maps.newHashMap()); + + org.apache.iceberg.PartitionKey partitionKey = new org.apache.iceberg.PartitionKey(SPEC_D_1, SCHEMA_D); + partitionKey.set(0, 1698608756000000L); + DataFile tsDataFiles = + DataFiles.builder(SPEC_D_1) + .withPath("/path/to/data-b4.parquet") + .withFileSizeInBytes(20) + .withPartition(partitionKey) + .withRecordCount(2) + .build(); + mockedNativeTableE.newAppend().appendFile(tsDataFiles).commit(); + mockedNativeTableE.refresh(); + List partitionKeys = metadata.getPrunedPartitions(icebergTable, null, 1); + Assert.assertEquals("2023-10-30 03:45:56", partitionKeys.get(0).getKeys().get(0).getStringValue()); + } + + @Test + public void testTransformedPartitionPrune() { + Map config = new HashMap<>(); + config.put(HIVE_METASTORE_URIS, "thrift://188.122.12.1:8732"); + config.put(ICEBERG_CATALOG_TYPE, "hive"); + IcebergHiveCatalog icebergHiveCatalog = new IcebergHiveCatalog("iceberg_catalog", new Configuration(), config); + List columns = Lists.newArrayList(new Column("k1", INT), new Column("ts", DATETIME)); + IcebergMetadata metadata = new IcebergMetadata(CATALOG_NAME, HDFS_ENVIRONMENT, icebergHiveCatalog); + IcebergTable icebergTable = new IcebergTable(1, "srTableName", "iceberg_catalog", "resource_name", "db_name", + "table_name", columns, mockedNativeTableD, Maps.newHashMap()); + + org.apache.iceberg.PartitionKey partitionKey = new org.apache.iceberg.PartitionKey(SPEC_D, SCHEMA_D); + partitionKey.set(0, 438292); + DataFile tsDataFiles = + DataFiles.builder(SPEC_D) + .withPath("/path/to/data-d.parquet") + .withFileSizeInBytes(20) + .withPartition(partitionKey) + .withRecordCount(2) + .build(); + mockedNativeTableD.newAppend().appendFile(tsDataFiles).commit(); + mockedNativeTableD.refresh(); + List partitionKeys = metadata.getPrunedPartitions(icebergTable, null, -1); + Assert.assertEquals("438292", partitionKeys.get(0).getKeys().get(0).getStringValue()); + } + + @Test + public void testDateDayPartitionPrune() { + Map config = new HashMap<>(); + config.put(HIVE_METASTORE_URIS, "thrift://188.122.12.1:8732"); + config.put(ICEBERG_CATALOG_TYPE, "hive"); + IcebergHiveCatalog icebergHiveCatalog = new IcebergHiveCatalog("iceberg_catalog", new Configuration(), config); + List columns = Lists.newArrayList(new Column("k1", INT), new Column("dt", DATE)); + IcebergMetadata metadata = new IcebergMetadata(CATALOG_NAME, HDFS_ENVIRONMENT, icebergHiveCatalog); + IcebergTable icebergTable = new IcebergTable(1, "srTableName", "iceberg_catalog", "resource_name", "db_name", + "table_name", columns, mockedNativeTableF, Maps.newHashMap()); + + org.apache.iceberg.PartitionKey partitionKey = new org.apache.iceberg.PartitionKey(SPEC_F, SCHEMA_F); + partitionKey.set(0, 19660); + DataFile tsDataFiles = DataFiles.builder(SPEC_F) + .withPath("/path/to/data-f.parquet") + .withFileSizeInBytes(20) + .withPartition(partitionKey) + .withRecordCount(2) + .build(); + mockedNativeTableF.newAppend().appendFile(tsDataFiles).commit(); + mockedNativeTableF.refresh(); + List partitionKeys = metadata.getPrunedPartitions(icebergTable, null, -1); + Assert.assertEquals("19660", partitionKeys.get(0).getKeys().get(0).getStringValue()); + } } \ No newline at end of file diff --git a/fe/fe-core/src/test/java/com/starrocks/connector/iceberg/TableTestBase.java b/fe/fe-core/src/test/java/com/starrocks/connector/iceberg/TableTestBase.java index 8fd13ed7ea03c..7ae6d34510897 100644 --- a/fe/fe-core/src/test/java/com/starrocks/connector/iceberg/TableTestBase.java +++ b/fe/fe-core/src/test/java/com/starrocks/connector/iceberg/TableTestBase.java @@ -48,6 +48,13 @@ public class TableTestBase { public static final Schema SCHEMA_B = new Schema(required(1, "k1", Types.IntegerType.get()), required(2, "k2", Types.IntegerType.get())); + + public static final Schema SCHEMA_D = + new Schema(required(1, "k1", Types.IntegerType.get()), required(2, "ts", Types.TimestampType.withZone())); + + public static final Schema SCHEMA_F = + new Schema(required(1, "k1", Types.IntegerType.get()), required(2, "dt", Types.DateType.get())); + protected static final int BUCKETS_NUMBER = 16; // Partition spec used to create tables @@ -55,6 +62,15 @@ public class TableTestBase { PartitionSpec.builderFor(SCHEMA_A).bucket("data", BUCKETS_NUMBER).build(); protected static final PartitionSpec SPEC_B = PartitionSpec.builderFor(SCHEMA_B).identity("k2").build(); + protected static final PartitionSpec SPEC_D = + PartitionSpec.builderFor(SCHEMA_D).hour("ts").build(); + + protected static final PartitionSpec SPEC_D_1 = + PartitionSpec.builderFor(SCHEMA_D).identity("ts").build(); + + protected static final PartitionSpec SPEC_F = + PartitionSpec.builderFor(SCHEMA_F).day("dt").build(); + public static final DataFile FILE_A = DataFiles.builder(SPEC_A) @@ -144,6 +160,10 @@ public class TableTestBase { public TestTables.TestTable mockedNativeTableA = null; public TestTables.TestTable mockedNativeTableB = null; public TestTables.TestTable mockedNativeTableC = null; + public TestTables.TestTable mockedNativeTableD = null; + public TestTables.TestTable mockedNativeTableE = null; + public TestTables.TestTable mockedNativeTableF = null; + protected final int formatVersion = 1; @Before @@ -155,6 +175,9 @@ public void setupTable() throws Exception { this.mockedNativeTableA = create(SCHEMA_A, SPEC_A, "ta", 1); this.mockedNativeTableB = create(SCHEMA_B, SPEC_B, "tb", 1); this.mockedNativeTableC = create(SCHEMA_B, SPEC_B, "tc", 2); + this.mockedNativeTableD = create(SCHEMA_D, SPEC_D, "td", 1); + this.mockedNativeTableE = create(SCHEMA_D, SPEC_D_1, "te", 1); + this.mockedNativeTableF = create(SCHEMA_F, SPEC_F, "tf", 1); } @After diff --git a/fe/fe-core/src/test/java/com/starrocks/connector/trino/TrinoViewTest.java b/fe/fe-core/src/test/java/com/starrocks/connector/trino/TrinoViewTest.java index b90168c451635..5d0b4e7eaf80a 100644 --- a/fe/fe-core/src/test/java/com/starrocks/connector/trino/TrinoViewTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/connector/trino/TrinoViewTest.java @@ -85,7 +85,7 @@ public void testTrinoViewDecodeBasic() { Assert.assertEquals(hiveView.getFullSchema().get(2).getName(), "salary"); Assert.assertEquals(hiveView.getFullSchema().get(2).getType(), ScalarType.DOUBLE); Assert.assertEquals(hiveView.getFullSchema().get(3).getName(), "new_col"); - Assert.assertEquals(hiveView.getFullSchema().get(3).getType(), ScalarType.STRING); + Assert.assertEquals(hiveView.getFullSchema().get(3).getType(), ScalarType.createDefaultExternalTableString()); } @Test @@ -133,11 +133,12 @@ public void testTrinoViewDecodeAllType() { Assert.assertEquals(hiveView.getFullSchema().get(14).getName(), "array_col"); Assert.assertEquals(hiveView.getFullSchema().get(14).getType(), ScalarType.ARRAY_INT); Assert.assertEquals(hiveView.getFullSchema().get(15).getName(), "map_col"); - Assert.assertEquals(hiveView.getFullSchema().get(15).getType(), new MapType(ScalarType.STRING, ScalarType.INT)); + Assert.assertEquals(hiveView.getFullSchema().get(15).getType(), + new MapType(ScalarType.createDefaultExternalTableString(), ScalarType.INT)); Assert.assertEquals(hiveView.getFullSchema().get(16).getName(), "struct_col"); ArrayList structFields = new ArrayList<>(); structFields.add(new StructField("field1", ScalarType.INT)); - structFields.add(new StructField("field2", ScalarType.STRING)); + structFields.add(new StructField("field2", ScalarType.createDefaultExternalTableString())); Assert.assertEquals(hiveView.getFullSchema().get(16).getType(), new StructType(structFields)); } @@ -173,14 +174,15 @@ public void testSchemaForTrinoView() { Assert.assertEquals(columnList.get(6).getType(), ScalarType.createDecimalV3NarrowestType(10, 2)); Assert.assertEquals(columnList.get(7).getType(), ScalarType.createVarcharType(20)); Assert.assertEquals(columnList.get(8).getType(), ScalarType.createCharType(10)); - Assert.assertEquals(columnList.get(9).getType(), ScalarType.STRING); + Assert.assertEquals(columnList.get(9).getType(), ScalarType.createDefaultExternalTableString()); Assert.assertEquals(columnList.get(10).getType(), ScalarType.BOOLEAN); Assert.assertEquals(columnList.get(11).getType(), ScalarType.DATETIME); Assert.assertEquals(columnList.get(12).getType(), ScalarType.ARRAY_INT); - Assert.assertEquals(columnList.get(13).getType(), new MapType(ScalarType.STRING, ScalarType.INT)); + Assert.assertEquals(columnList.get(13).getType(), + new MapType(ScalarType.createDefaultExternalTableString(), ScalarType.INT)); ArrayList structFields = new ArrayList<>(); structFields.add(new StructField("field1", ScalarType.INT)); - structFields.add(new StructField("field2", ScalarType.STRING)); + structFields.add(new StructField("field2", ScalarType.createDefaultExternalTableString())); Assert.assertEquals(columnList.get(14).getType(), new StructType(structFields)); } } diff --git a/fe/fe-core/src/test/java/com/starrocks/credential/CloudConfigurationFactoryTest.java b/fe/fe-core/src/test/java/com/starrocks/credential/CloudConfigurationFactoryTest.java index 07552b96f7b60..f3bc8f2fa771c 100644 --- a/fe/fe-core/src/test/java/com/starrocks/credential/CloudConfigurationFactoryTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/credential/CloudConfigurationFactoryTest.java @@ -206,6 +206,13 @@ public void testHDFSCloudConfiguration() { Assert.assertEquals(cc.toConfString(), "HDFSCloudConfiguration{resources='', jars='', cred=HDFSCloudCredential{authentication='simple', username='XX'," + " password='XX', krbPrincipal='', krbKeyTabFile='', krbKeyTabData=''}}"); + + map.clear(); + cc = CloudConfigurationFactory.buildCloudConfigurationForStorage(map); + Assert.assertEquals(CloudType.DEFAULT, cc.getCloudType()); + + cc = CloudConfigurationFactory.buildCloudConfigurationForStorage(map, true); + Assert.assertEquals(CloudType.HDFS, cc.getCloudType()); } @Test diff --git a/fe/fe-core/src/test/java/com/starrocks/statistic/StatisticsExecutorTest.java b/fe/fe-core/src/test/java/com/starrocks/statistic/StatisticsExecutorTest.java index 4d1d249177fb1..5d93a4f2e0d20 100644 --- a/fe/fe-core/src/test/java/com/starrocks/statistic/StatisticsExecutorTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/statistic/StatisticsExecutorTest.java @@ -137,8 +137,8 @@ public List executeStatisticDQL(ConnectContext context, String s "and column_name = \"c1\" " + "GROUP BY table_uuid, column_name UNION ALL " + "SELECT cast(6 as INT), column_name, sum(row_count), cast(sum(data_size) as bigint), " + - "hll_union_agg(ndv), sum(null_count), cast(max(cast(max as varchar(1048576))) as string), " + - "cast(min(cast(min as varchar(1048576))) as string) " + + "hll_union_agg(ndv), sum(null_count), cast(max(cast(max as varchar(1073741824))) as string), " + + "cast(min(cast(min as varchar(1073741824))) as string) " + "FROM external_column_statistics " + "WHERE table_uuid = \"hive0.partitioned_db.t1.0\"" + " and column_name = \"c2\" " + diff --git a/fe/fe-core/src/test/java/com/starrocks/storagevolume/StorageVolumeTest.java b/fe/fe-core/src/test/java/com/starrocks/storagevolume/StorageVolumeTest.java index fb4c1f7a3d047..46759567dd322 100644 --- a/fe/fe-core/src/test/java/com/starrocks/storagevolume/StorageVolumeTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/storagevolume/StorageVolumeTest.java @@ -22,6 +22,7 @@ import com.staros.proto.AzBlobFileStoreInfo; import com.staros.proto.FileStoreInfo; import com.staros.proto.FileStoreType; +import com.staros.proto.HDFSFileStoreInfo; import com.staros.proto.S3FileStoreInfo; import com.starrocks.common.AnalysisException; import com.starrocks.common.DdlException; @@ -63,9 +64,11 @@ import static com.starrocks.credential.CloudConfigurationConstants.AZURE_BLOB_ENDPOINT; import static com.starrocks.credential.CloudConfigurationConstants.AZURE_BLOB_SAS_TOKEN; import static com.starrocks.credential.CloudConfigurationConstants.AZURE_BLOB_SHARED_KEY; -import static com.starrocks.credential.CloudConfigurationConstants.HADOOP_KERBEROS_KEYTAB; import static com.starrocks.credential.CloudConfigurationConstants.HDFS_AUTHENTICATION; -import static com.starrocks.credential.CloudConfigurationConstants.HDFS_KERBEROS_PRINCIPAL; +import static com.starrocks.credential.CloudConfigurationConstants.HDFS_KERBEROS_KEYTAB_CONTENT_DEPRECATED; +import static com.starrocks.credential.CloudConfigurationConstants.HDFS_KERBEROS_KEYTAB_DEPRECATED; +import static com.starrocks.credential.CloudConfigurationConstants.HDFS_KERBEROS_PRINCIPAL_DEPRECATED; +import static com.starrocks.credential.CloudConfigurationConstants.HDFS_KERBEROS_TICKET_CACHE_PATH; import static com.starrocks.credential.CloudConfigurationConstants.HDFS_PASSWORD; import static com.starrocks.credential.CloudConfigurationConstants.HDFS_USERNAME; @@ -195,7 +198,7 @@ public void testAWSInvalidCredential() { @Test public void testHDFSSimpleCredential() { Map storageParams = new HashMap<>(); - storageParams.put(HDFS_AUTHENTICATION, "simple"); + storageParams.put(HDFS_AUTHENTICATION, HDFSCloudCredential.SIMPLE_AUTH); storageParams.put(HDFS_USERNAME, "username"); storageParams.put(HDFS_PASSWORD, "password"); storageParams.put("dfs.nameservices", "ha_cluster"); @@ -210,11 +213,25 @@ public void testHDFSSimpleCredential() { CloudConfiguration cloudConfiguration = sv.getCloudConfiguration(); Assert.assertEquals(CloudType.HDFS, cloudConfiguration.getCloudType()); HDFSCloudConfiguration hdfsCloudConfiguration = (HDFSCloudConfiguration) cloudConfiguration; - Assert.assertEquals("simple", hdfsCloudConfiguration.getHdfsCloudCredential().getAuthentication()); + Assert.assertEquals(HDFSCloudCredential.SIMPLE_AUTH, hdfsCloudConfiguration.getHdfsCloudCredential().getAuthentication()); Assert.assertEquals(5, hdfsCloudConfiguration.getHdfsCloudCredential().getHadoopConfiguration().size()); + FileStoreInfo fileStore = cloudConfiguration.toFileStoreInfo(); + Assert.assertEquals(FileStoreType.HDFS, fileStore.getFsType()); + Assert.assertTrue(fileStore.hasHdfsFsInfo()); + HDFSFileStoreInfo hdfsFileStoreInfo = fileStore.getHdfsFsInfo(); + Assert.assertEquals("username", hdfsFileStoreInfo.getUsername()); + Assert.assertEquals("simple", hdfsFileStoreInfo.getConfigurationMap().get(HDFS_AUTHENTICATION)); + Assert.assertEquals("ha_cluster", hdfsFileStoreInfo.getConfigurationMap().get("dfs.nameservices")); + Assert.assertEquals("ha_n1,ha_n2", hdfsFileStoreInfo.getConfigurationMap().get("dfs.ha.namenodes.ha_cluster")); + Assert.assertEquals(":", + hdfsFileStoreInfo.getConfiguration().get("dfs.namenode.rpc-address.ha_cluster.ha_n1")); + Assert.assertEquals(":", + hdfsFileStoreInfo.getConfiguration().get("dfs.namenode.rpc-address.ha_cluster.ha_n2")); + Assert.assertEquals("org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", + hdfsFileStoreInfo.getConfiguration().get("dfs.client.failover.proxy.provider")); Map storageParams1 = new HashMap<>(); - storageParams1.put(HDFS_AUTHENTICATION, "simple"); + storageParams1.put(HDFS_AUTHENTICATION, HDFSCloudCredential.SIMPLE_AUTH); storageParams1.put(HDFS_USERNAME, "username"); storageParams1.put(HDFS_PASSWORD, "password"); sv = new StorageVolume("2", "test", "hdfs", Arrays.asList("hdfs://abc"), @@ -222,16 +239,22 @@ public void testHDFSSimpleCredential() { cloudConfiguration = sv.getCloudConfiguration(); Assert.assertEquals(CloudType.HDFS, cloudConfiguration.getCloudType()); hdfsCloudConfiguration = (HDFSCloudConfiguration) cloudConfiguration; - Assert.assertEquals("simple", hdfsCloudConfiguration.getHdfsCloudCredential().getAuthentication()); + Assert.assertEquals(HDFSCloudCredential.SIMPLE_AUTH, hdfsCloudConfiguration.getHdfsCloudCredential().getAuthentication()); Assert.assertEquals(0, hdfsCloudConfiguration.getHdfsCloudCredential().getHadoopConfiguration().size()); + fileStore = cloudConfiguration.toFileStoreInfo(); + Assert.assertEquals(FileStoreType.HDFS, fileStore.getFsType()); + Assert.assertTrue(fileStore.hasHdfsFsInfo()); + hdfsFileStoreInfo = fileStore.getHdfsFsInfo(); + Assert.assertEquals("username", hdfsFileStoreInfo.getUsername()); } @Test public void testHDFSKerberosCredential() throws AnalysisException { Map storageParams = new HashMap<>(); - storageParams.put(HDFS_AUTHENTICATION, "kerberos"); - storageParams.put(HDFS_KERBEROS_PRINCIPAL, "nn/abc@ABC.COM"); - storageParams.put(HADOOP_KERBEROS_KEYTAB, "/keytab/hive.keytab"); + storageParams.put(HDFS_AUTHENTICATION, HDFSCloudCredential.KERBEROS_AUTH); + storageParams.put(HDFS_KERBEROS_PRINCIPAL_DEPRECATED, "nn/abc@ABC.COM"); + storageParams.put(HDFS_KERBEROS_KEYTAB_DEPRECATED, "/keytab/hive.keytab"); + storageParams.put(HDFS_KERBEROS_KEYTAB_CONTENT_DEPRECATED, "YWFhYWFh"); storageParams.put("dfs.nameservices", "ha_cluster"); storageParams.put("dfs.ha.namenodes.ha_cluster", "ha_n1,ha_n2"); storageParams.put("dfs.namenode.rpc-address.ha_cluster.ha_n1", ":"); @@ -246,6 +269,39 @@ public void testHDFSKerberosCredential() throws AnalysisException { HDFSCloudConfiguration hdfsCloudConfiguration = (HDFSCloudConfiguration) cloudConfiguration; Assert.assertEquals("kerberos", hdfsCloudConfiguration.getHdfsCloudCredential().getAuthentication()); Assert.assertEquals(5, hdfsCloudConfiguration.getHdfsCloudCredential().getHadoopConfiguration().size()); + FileStoreInfo fileStore = cloudConfiguration.toFileStoreInfo(); + Assert.assertEquals(FileStoreType.HDFS, fileStore.getFsType()); + Assert.assertTrue(fileStore.hasHdfsFsInfo()); + HDFSFileStoreInfo hdfsFileStoreInfo = fileStore.getHdfsFsInfo(); + Assert.assertEquals(HDFSCloudCredential.KERBEROS_AUTH, hdfsFileStoreInfo.getConfigurationMap().get(HDFS_AUTHENTICATION)); + Assert.assertEquals(5, hdfsCloudConfiguration.getHdfsCloudCredential().getHadoopConfiguration().size()); + Assert.assertEquals("ha_cluster", hdfsFileStoreInfo.getConfigurationMap().get("dfs.nameservices")); + Assert.assertEquals("ha_n1,ha_n2", hdfsFileStoreInfo.getConfigurationMap().get("dfs.ha.namenodes.ha_cluster")); + Assert.assertEquals(":", + hdfsFileStoreInfo.getConfigurationMap().get("dfs.namenode.rpc-address.ha_cluster.ha_n1")); + Assert.assertEquals(":", + hdfsFileStoreInfo.getConfigurationMap().get("dfs.namenode.rpc-address.ha_cluster.ha_n2")); + Assert.assertEquals("org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", + hdfsFileStoreInfo.getConfigurationMap().get("dfs.client.failover.proxy.provider")); + + storageParams.clear(); + storageParams.put(HDFS_AUTHENTICATION, HDFSCloudCredential.KERBEROS_AUTH); + storageParams.put(HDFS_KERBEROS_TICKET_CACHE_PATH, "/path/to/ticket/cache/path"); + sv = new StorageVolume("1", "test", "hdfs", Arrays.asList("hdfs://abc"), + storageParams, true, ""); + cloudConfiguration = sv.getCloudConfiguration(); + Assert.assertEquals(CloudType.HDFS, cloudConfiguration.getCloudType()); + hdfsCloudConfiguration = (HDFSCloudConfiguration) cloudConfiguration; + Assert.assertEquals(HDFSCloudCredential.KERBEROS_AUTH, + hdfsCloudConfiguration.getHdfsCloudCredential().getAuthentication()); + Assert.assertEquals(1, hdfsCloudConfiguration.getHdfsCloudCredential().getHadoopConfiguration().size()); + fileStore = cloudConfiguration.toFileStoreInfo(); + Assert.assertEquals(FileStoreType.HDFS, fileStore.getFsType()); + Assert.assertTrue(fileStore.hasHdfsFsInfo()); + hdfsFileStoreInfo = fileStore.getHdfsFsInfo(); + Assert.assertEquals("kerberos", hdfsFileStoreInfo.getConfigurationMap().get(HDFS_AUTHENTICATION)); + Assert.assertEquals("/path/to/ticket/cache/path", + hdfsFileStoreInfo.getConfigurationMap().get(HDFS_KERBEROS_TICKET_CACHE_PATH)); } @Test @@ -256,12 +312,30 @@ public void testHDFSEmptyCredential() { CloudConfiguration cloudConfiguration = sv.getCloudConfiguration(); Assert.assertEquals(CloudType.HDFS, cloudConfiguration.getCloudType()); HDFSCloudConfiguration hdfsCloudConfiguration = (HDFSCloudConfiguration) cloudConfiguration; - Assert.assertEquals(hdfsCloudConfiguration.getHdfsCloudCredential().getAuthentication(), HDFSCloudCredential.EMPTY); FileStoreInfo fileStore = cloudConfiguration.toFileStoreInfo(); Assert.assertEquals(FileStoreType.HDFS, fileStore.getFsType()); Assert.assertTrue(fileStore.hasHdfsFsInfo()); } + @Test + public void testHDFSViewFS() { + Map storageParams = new HashMap<>(); + storageParams.put("fs.viewfs.mounttable.ClusterX.link./data", "hdfs://nn1-clusterx.example.com:8020/data"); + storageParams.put("fs.viewfs.mounttable.ClusterX.link./project", "hdfs://nn2-clusterx.example.com:8020/project"); + StorageVolume sv = new StorageVolume("1", "test", "hdfs", Arrays.asList("hdfs://abc"), + storageParams, true, ""); + CloudConfiguration cloudConfiguration = sv.getCloudConfiguration(); + Assert.assertEquals(CloudType.HDFS, cloudConfiguration.getCloudType()); + FileStoreInfo fileStore = cloudConfiguration.toFileStoreInfo(); + Assert.assertEquals(FileStoreType.HDFS, fileStore.getFsType()); + Assert.assertTrue(fileStore.hasHdfsFsInfo()); + HDFSFileStoreInfo hdfsFileStoreInfo = fileStore.getHdfsFsInfo(); + Assert.assertEquals("hdfs://nn1-clusterx.example.com:8020/data", + hdfsFileStoreInfo.getConfigurationMap().get("fs.viewfs.mounttable.ClusterX.link./data")); + Assert.assertEquals("hdfs://nn2-clusterx.example.com:8020/project", + hdfsFileStoreInfo.getConfigurationMap().get("fs.viewfs.mounttable.ClusterX.link./project")); + } + @Test public void testHDFSAddConfigResources() { String runningDir = MockedFrontend.getInstance().getRunningDir(); @@ -388,6 +462,26 @@ public void testFromFileStoreInfo() { fs = FileStoreInfo.newBuilder().setS3FsInfo(s3fs).setFsKey("0").setFsType(FileStoreType.S3).build(); sv = StorageVolume.fromFileStoreInfo(fs); Assert.assertEquals(CloudType.AWS, sv.getCloudConfiguration().getCloudType()); + + HDFSFileStoreInfo hdfs = HDFSFileStoreInfo.newBuilder().setUsername("username") + .putConfiguration(HDFS_AUTHENTICATION, "simple") + .putConfiguration("dfs.nameservices", "ha_cluster") + .putConfiguration("dfs.ha.namenodes.ha_cluster", "ha_n1,ha_n2") + .putConfiguration("dfs.namenode.rpc-address.ha_cluster.ha_n1", ":") + .putConfiguration("dfs.namenode.rpc-address.ha_cluster.ha_n2", ":") + .putConfiguration("dfs.client.failover.proxy.provider", + "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider").build(); + fs = FileStoreInfo.newBuilder().setHdfsFsInfo(hdfs).setFsKey("0").setFsType(FileStoreType.HDFS).build(); + sv = StorageVolume.fromFileStoreInfo(fs); + Assert.assertEquals(CloudType.HDFS, sv.getCloudConfiguration().getCloudType()); + + hdfs = HDFSFileStoreInfo.newBuilder().putConfiguration(HDFS_AUTHENTICATION, "kerberos") + .putConfiguration(HDFS_KERBEROS_PRINCIPAL_DEPRECATED, "nn/abc@ABC.COM") + .putConfiguration(HDFS_KERBEROS_KEYTAB_DEPRECATED, "/keytab/hive.keytab") + .putConfiguration(HDFS_KERBEROS_KEYTAB_CONTENT_DEPRECATED, "YWFhYWFh").build(); + fs = FileStoreInfo.newBuilder().setHdfsFsInfo(hdfs).setFsKey("0").setFsType(FileStoreType.HDFS).build(); + sv = StorageVolume.fromFileStoreInfo(fs); + Assert.assertEquals(CloudType.HDFS, sv.getCloudConfiguration().getCloudType()); } @Test diff --git a/gensrc/thrift/InternalService.thrift b/gensrc/thrift/InternalService.thrift index ca250d775b15d..01fb835440b96 100644 --- a/gensrc/thrift/InternalService.thrift +++ b/gensrc/thrift/InternalService.thrift @@ -232,6 +232,8 @@ struct TQueryOptions { 106: optional bool enable_agg_spill_preaggregation; 107: optional i64 global_runtime_filter_build_max_size; 108: optional i64 runtime_filter_rpc_http_min_size; + + 109: optional i64 big_query_profile_second_threshold; } diff --git a/java-extensions/hudi-reader/pom.xml b/java-extensions/hudi-reader/pom.xml index 9ec9ab7bed746..49aaed93d4ebf 100644 --- a/java-extensions/hudi-reader/pom.xml +++ b/java-extensions/hudi-reader/pom.xml @@ -21,7 +21,7 @@ 3.3.6 0.12.2 0.25 - 1.11.2 + 1.11.3 1.5.4-2 diff --git a/java-extensions/hudi-reader/src/main/java/com/starrocks/hudi/reader/HudiScannerUtils.java b/java-extensions/hudi-reader/src/main/java/com/starrocks/hudi/reader/HudiScannerUtils.java index 4572ca33217be..5233e1f15c126 100644 --- a/java-extensions/hudi-reader/src/main/java/com/starrocks/hudi/reader/HudiScannerUtils.java +++ b/java-extensions/hudi-reader/src/main/java/com/starrocks/hudi/reader/HudiScannerUtils.java @@ -22,6 +22,7 @@ import java.time.ZoneOffset; import java.time.format.DateTimeFormatter; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; @@ -88,4 +89,46 @@ public static boolean isMaybeInt64Timestamp(ColumnType.TypeValue type) { || type == ColumnType.TypeValue.DATETIME_MILLIS || type == ColumnType.TypeValue.DATETIME); } + + public static String mapColumnTypeToHiveType(ColumnType type) { + ColumnType.TypeValue typeValue = type.getTypeValue(); + StringBuilder sb = new StringBuilder(); + if (type.isStruct()) { + List childNames = type.getChildNames(); + List childTypes = type.getChildTypes(); + sb.append("struct<"); + for (int i = 0; i < childNames.size(); i++) { + sb.append(childNames.get(i)); + sb.append(":"); + sb.append(mapColumnTypeToHiveType(childTypes.get(i))); + sb.append(","); + } + if (childNames.size() > 0) { + sb.deleteCharAt(sb.length() - 1); + } + sb.append(">"); + return sb.toString(); + } else if (type.isMap() || type.isArray()) { + List childTypes = type.getChildTypes(); + sb.append(type.getTypeValueString()); + sb.append("<"); + for (int i = 0; i < childTypes.size(); i++) { + sb.append(mapColumnTypeToHiveType(childTypes.get(i))); + sb.append(","); + } + if (childTypes.size() > 0) { + sb.deleteCharAt(sb.length() - 1); + } + sb.append(">"); + return sb.toString(); + } else if (type.isDecimal()) { + return type.getRawTypeValue(); + } else { + String value = type.getTypeValueString(); + if (value == null) { + throw new IllegalArgumentException("Invalid type: " + type.toString()); + } + return HIVE_TYPE_MAPPING.getOrDefault(value, value); + } + } } \ No newline at end of file diff --git a/java-extensions/hudi-reader/src/main/java/com/starrocks/hudi/reader/HudiSliceScanner.java b/java-extensions/hudi-reader/src/main/java/com/starrocks/hudi/reader/HudiSliceScanner.java index 112a7cf34d632..863413fd897b8 100644 --- a/java-extensions/hudi-reader/src/main/java/com/starrocks/hudi/reader/HudiSliceScanner.java +++ b/java-extensions/hudi-reader/src/main/java/com/starrocks/hudi/reader/HudiSliceScanner.java @@ -51,7 +51,6 @@ import java.util.stream.Collectors; import static com.google.common.base.Preconditions.checkArgument; -import static com.starrocks.hudi.reader.HudiScannerUtils.HIVE_TYPE_MAPPING; import static java.util.stream.Collectors.toList; public class HudiSliceScanner extends ConnectorScanner { @@ -80,9 +79,6 @@ public class HudiSliceScanner extends ConnectorScanner { private final ClassLoader classLoader; private final String fsOptionsProps; - public static final int MAX_DECIMAL32_PRECISION = 9; - public static final int MAX_DECIMAL64_PRECISION = 18; - public HudiSliceScanner(int fetchSize, Map params) { this.fetchSize = fetchSize; this.hiveColumnNames = params.get("hive_column_names"); @@ -131,12 +127,7 @@ private void parseRequiredTypes() { for (int i = 0; i < requiredFields.length; i++) { requiredColumnIds[i] = hiveColumnNameToIndex.get(requiredFields[i]); String type = hiveColumnNameToType.get(requiredFields[i]); - - if (type.startsWith("decimal")) { - parseDecimal(type, i); - } else { - requiredTypes[i] = new ColumnType(requiredFields[i], type); - } + requiredTypes[i] = new ColumnType(requiredFields[i], type); } // prune fields @@ -151,28 +142,6 @@ private void parseRequiredTypes() { } } - // convert decimal(x,y) to decimal - private void parseDecimal(String type, int i) { - int precision = -1; - int scale = -1; - int s = type.indexOf('('); - int e = type.indexOf(')'); - if (s != -1 && e != -1) { - String[] ps = type.substring(s + 1, e).split(","); - precision = Integer.parseInt(ps[0].trim()); - scale = Integer.parseInt(ps[1].trim()); - if (precision <= MAX_DECIMAL32_PRECISION) { - type = "decimal32"; - } else if (precision <= MAX_DECIMAL64_PRECISION) { - type = "decimal64"; - } else { - type = "decimal128"; - } - } - requiredTypes[i] = new ColumnType(requiredFields[i], type); - requiredTypes[i].setScale(scale); - } - private Properties makeProperties() { Properties properties = new Properties(); properties.setProperty("hive.io.file.readcolumn.ids", @@ -192,12 +161,12 @@ private Properties makeProperties() { } properties.setProperty("columns", this.hiveColumnNames); // recover INT64 based timestamp mark to hive type, TimestampMicros/TimestampMillis => timestamp + List types = new ArrayList<>(); + String[] hiveColumnNames = this.hiveColumnNames.split(","); for (int i = 0; i < this.hiveColumnTypes.length; i++) { - String type = this.hiveColumnTypes[i]; - if (HIVE_TYPE_MAPPING.containsKey(type)) { - type = HIVE_TYPE_MAPPING.get(type); - } + ColumnType columnType = new ColumnType(hiveColumnNames[i], hiveColumnTypes[i]); + String type = HudiScannerUtils.mapColumnTypeToHiveType(columnType); types.add(type); } properties.setProperty("columns.types", types.stream().collect(Collectors.joining(","))); diff --git a/java-extensions/jni-connector/src/main/java/com/starrocks/jni/connector/ColumnType.java b/java-extensions/jni-connector/src/main/java/com/starrocks/jni/connector/ColumnType.java index 781f8e371c3a2..9ee9c2aadc162 100644 --- a/java-extensions/jni-connector/src/main/java/com/starrocks/jni/connector/ColumnType.java +++ b/java-extensions/jni-connector/src/main/java/com/starrocks/jni/connector/ColumnType.java @@ -58,10 +58,17 @@ public enum TypeValue { List childNames; List childTypes; List fieldIndex; + String rawTypeValue; + int scale = -1; private static final Map PRIMITIVE_TYPE_VALUE_MAPPING = new HashMap<>(); private static final Map PRIMITIVE_TYPE_VALUE_SIZE = new HashMap<>(); + private static final Map PRIMITIVE_TYPE_VALUE_STRING_MAPPING = new HashMap<>(); + + private static final int MAX_DECIMAL32_PRECISION = 9; + private static final int MAX_DECIMAL64_PRECISION = 18; + static { PRIMITIVE_TYPE_VALUE_MAPPING.put("byte", TypeValue.BYTE); PRIMITIVE_TYPE_VALUE_MAPPING.put("boolean", TypeValue.BOOLEAN); @@ -82,6 +89,13 @@ public enum TypeValue { PRIMITIVE_TYPE_VALUE_MAPPING.put("decimal128", TypeValue.DECIMAL128); PRIMITIVE_TYPE_VALUE_MAPPING.put("tinyint", TypeValue.TINYINT); + for (String k : PRIMITIVE_TYPE_VALUE_MAPPING.keySet()) { + PRIMITIVE_TYPE_VALUE_STRING_MAPPING.put(PRIMITIVE_TYPE_VALUE_MAPPING.get(k), k); + } + PRIMITIVE_TYPE_VALUE_STRING_MAPPING.put(TypeValue.STRUCT, "struct"); + PRIMITIVE_TYPE_VALUE_STRING_MAPPING.put(TypeValue.MAP, "map"); + PRIMITIVE_TYPE_VALUE_STRING_MAPPING.put(TypeValue.ARRAY, "array"); + PRIMITIVE_TYPE_VALUE_SIZE.put(TypeValue.BYTE, 1); PRIMITIVE_TYPE_VALUE_SIZE.put(TypeValue.BOOLEAN, 1); PRIMITIVE_TYPE_VALUE_SIZE.put(TypeValue.SHORT, 2); @@ -206,7 +220,11 @@ private void parse(StringScanner scanner) { } break; default: { - typeValue = PRIMITIVE_TYPE_VALUE_MAPPING.getOrDefault(t, null); + if (t.startsWith("decimal")) { + typeValue = parseDecimal(t); + } else { + typeValue = PRIMITIVE_TYPE_VALUE_MAPPING.getOrDefault(t, null); + } } } @@ -260,6 +278,11 @@ public boolean isMapValueSelected() { return childNames.indexOf(FIELD_1_NAME) != -1; } + public boolean isDecimal() { + return typeValue == TypeValue.DECIMALV2 || typeValue == TypeValue.DECIMAL32 || typeValue == TypeValue.DECIMAL64 || + typeValue == TypeValue.DECIMAL128; + } + public int computeColumnSize() { switch (typeValue) { case UNKNOWN: @@ -306,6 +329,10 @@ public int getPrimitiveTypeValueSize() { return PRIMITIVE_TYPE_VALUE_SIZE.getOrDefault(typeValue, -1); } + public String getTypeValueString() { + return PRIMITIVE_TYPE_VALUE_STRING_MAPPING.get(typeValue); + } + public List getChildNames() { return childNames; } @@ -392,4 +419,33 @@ public void setScale(int scale) { public int getScale() { return scale; } + + public String getRawTypeValue() { + return rawTypeValue; + } + + // convert decimal(x,y) to decimal + private TypeValue parseDecimal(String rawType) { + String type = rawType; + int precision = -1; + int scale = -1; + int s = type.indexOf('('); + int e = type.indexOf(')'); + if (s != -1 && e != -1) { + String[] ps = type.substring(s + 1, e).split(","); + precision = Integer.parseInt(ps[0].trim()); + scale = Integer.parseInt(ps[1].trim()); + if (precision <= MAX_DECIMAL32_PRECISION) { + type = "decimal32"; + } else if (precision <= MAX_DECIMAL64_PRECISION) { + type = "decimal64"; + } else { + type = "decimal128"; + } + } + TypeValue value = PRIMITIVE_TYPE_VALUE_MAPPING.get(type); + rawTypeValue = rawType; + setScale(scale); + return value; + } } diff --git a/test/common/data/stream_load/sr_partial_update_3.csv b/test/common/data/stream_load/sr_partial_update_3.csv new file mode 100644 index 0000000000000..5d73fc17e668b --- /dev/null +++ b/test/common/data/stream_load/sr_partial_update_3.csv @@ -0,0 +1,2 @@ +100,k2_100,v9_100000,v10_100000,v14_10000,123456,v15_123456,v5_100000,v4_100000 +200,k2_200,v9_100000,v10_100000,v14_10000,123456,v15_123455,v5_100000,v4_100000 \ No newline at end of file diff --git a/test/sql/test_partial_update_column_mode/R/test_partial_update_many_columns b/test/sql/test_partial_update_column_mode/R/test_partial_update_many_columns new file mode 100644 index 0000000000000..8541590256f2a --- /dev/null +++ b/test/sql/test_partial_update_column_mode/R/test_partial_update_many_columns @@ -0,0 +1,109 @@ +-- name: test_partial_update_many_columns +show backends; +create database test_partial_update_many_columns; +-- result: +-- !result +use test_partial_update_many_columns; +-- result: +-- !result +CREATE table tab1 ( + k1 INTEGER, + k2 VARCHAR(50), + v1 INTEGER, + v2 INTEGER, + v3 INTEGER, + v4 varchar(50), + v5 varchar(50), + v6 INTEGER, + v7 INTEGER, + v8 INTEGER, + v9 varchar(50), + v10 varchar(50), + v11 INTEGER, + v12 INTEGER, + v13 INTEGER, + v14 varchar(50), + v15 varchar(50) +) +ENGINE=OLAP +PRIMARY KEY(`k1`,`k2`) +DISTRIBUTED BY HASH(`k1`) BUCKETS 10 +PROPERTIES ( + "replication_num" = "1" +); +-- result: +-- !result +set partial_update_mode = "column"; +-- result: +-- !result +insert into tab1 values (100, "k2_100", 100, 100, 100, "v4_100", "v5_100", 100, 100, 100, "v9_100", "v10_100", 100, 100, 100, "v14_100", "v15_100"); +-- result: +-- !result +insert into tab1 values (200, "k2_200", 200, 200, 200, "v4_200", "v5_200", 200, 200, 200, "v9_200", "v10_200", 200, 200, 200, "v14_200", "v15_200"); +-- result: +-- !result +insert into tab1 values (300, "k2_300", 300, 300, 300, "v4_300", "v5_300", 300, 300, 300, "v9_200", "v10_200", 200, 200, 200, "v14_200", "v15_200"); +-- result: +-- !result +insert into tab1 values (400, "k2_400", 300, 300, 300, "v4_300", "v5_300", 300, 300, 300, "v9_200", "v10_200", 200, 200, 200, "v14_200", "v15_200"); +-- result: +-- !result +insert into tab1 values (500, "k2_500", 300, 300, 300, "v4_300", "v5_300", 300, 300, 300, "v9_200", "v10_200", 200, 200, 200, "v14_200", "v15_200"); +-- result: +-- !result +insert into tab1 values (600, "k2_600", 300, 300, 300, "v4_300", "v5_300", 300, 300, 300, "v9_200", "v10_200", 200, 200, 200, "v14_200", "v15_200"); +-- result: +-- !result +select * from tab1; +-- result: +100 k2_100 100 100 100 v4_100 v5_100 100 100 100 v9_100 v10_100 100 100 100 v14_100 v15_100 +200 k2_200 200 200 200 v4_200 v5_200 200 200 200 v9_200 v10_200 200 200 200 v14_200 v15_200 +600 k2_600 300 300 300 v4_300 v5_300 300 300 300 v9_200 v10_200 200 200 200 v14_200 v15_200 +400 k2_400 300 300 300 v4_300 v5_300 300 300 300 v9_200 v10_200 200 200 200 v14_200 v15_200 +300 k2_300 300 300 300 v4_300 v5_300 300 300 300 v9_200 v10_200 200 200 200 v14_200 v15_200 +500 k2_500 300 300 300 v4_300 v5_300 300 300 300 v9_200 v10_200 200 200 200 v14_200 v15_200 +-- !result +update tab1 set v2 = 1000, v4 = "v4_1000", v6 = 1000, v8 = 1000, v10 = "v10_1000", v12 = 1000; +-- result: +-- !result +select * from tab1; +-- result: +500 k2_500 300 1000 300 v4_1000 v5_300 1000 300 1000 v9_200 v10_1000 200 1000 200 v14_200 v15_200 +100 k2_100 100 1000 100 v4_1000 v5_100 1000 100 1000 v9_100 v10_1000 100 1000 100 v14_100 v15_100 +600 k2_600 300 1000 300 v4_1000 v5_300 1000 300 1000 v9_200 v10_1000 200 1000 200 v14_200 v15_200 +400 k2_400 300 1000 300 v4_1000 v5_300 1000 300 1000 v9_200 v10_1000 200 1000 200 v14_200 v15_200 +300 k2_300 300 1000 300 v4_1000 v5_300 1000 300 1000 v9_200 v10_1000 200 1000 200 v14_200 v15_200 +200 k2_200 200 1000 200 v4_1000 v5_200 1000 200 1000 v9_200 v10_1000 200 1000 200 v14_200 v15_200 +-- !result +update tab1 set v2 = 1000, v3 = 1000, v4 = "v4_1000", v5 = "v5_1000", v6 = 1000, v7 = 1000, v8 = 1000, v9 = "v9_1000", v10 = "v10_1000", v11 = 1000, v12 = 1000; +-- result: +-- !result +select * from tab1; +-- result: +300 k2_300 300 1000 1000 v4_1000 v5_1000 1000 1000 1000 v9_1000 v10_1000 1000 1000 200 v14_200 v15_200 +500 k2_500 300 1000 1000 v4_1000 v5_1000 1000 1000 1000 v9_1000 v10_1000 1000 1000 200 v14_200 v15_200 +100 k2_100 100 1000 1000 v4_1000 v5_1000 1000 1000 1000 v9_1000 v10_1000 1000 1000 100 v14_100 v15_100 +200 k2_200 200 1000 1000 v4_1000 v5_1000 1000 1000 1000 v9_1000 v10_1000 1000 1000 200 v14_200 v15_200 +600 k2_600 300 1000 1000 v4_1000 v5_1000 1000 1000 1000 v9_1000 v10_1000 1000 1000 200 v14_200 v15_200 +400 k2_400 300 1000 1000 v4_1000 v5_1000 1000 1000 1000 v9_1000 v10_1000 1000 1000 200 v14_200 v15_200 +-- !result +shell: curl --location-trusted -u root: -T ${root_path}/lib/../common/data/stream_load/sr_partial_update_3.csv -XPUT -H partial_update:true -H label:test_partial_update_many_columns -H partial_update_mode:column -H column_separator:, -H columns:k1,k2,v9,v10,v14,v11,v15,v5,v4 ${url}/api/test_partial_update_many_columns/tab1/_stream_load +-- result: +0 +{ + "Status": "Success", + "Message": "OK" +} +-- !result +sync; +-- result: +-- !result +select * from tab1; +-- result: +500 k2_500 300 1000 1000 v4_1000 v5_1000 1000 1000 1000 v9_1000 v10_1000 1000 1000 200 v14_200 v15_200 +400 k2_400 300 1000 1000 v4_1000 v5_1000 1000 1000 1000 v9_1000 v10_1000 1000 1000 200 v14_200 v15_200 +300 k2_300 300 1000 1000 v4_1000 v5_1000 1000 1000 1000 v9_1000 v10_1000 1000 1000 200 v14_200 v15_200 +200 k2_200 200 1000 1000 v4_100000 v5_100000 1000 1000 1000 v9_100000 v10_100000 123456 1000 200 v14_10000 v15_123455 +600 k2_600 300 1000 1000 v4_1000 v5_1000 1000 1000 1000 v9_1000 v10_1000 1000 1000 200 v14_200 v15_200 +100 k2_100 100 1000 1000 v4_100000 v5_100000 1000 1000 1000 v9_100000 v10_100000 123456 1000 100 v14_10000 v15_123456 +-- !result \ No newline at end of file diff --git a/test/sql/test_partial_update_column_mode/T/test_partial_update_many_columns b/test/sql/test_partial_update_column_mode/T/test_partial_update_many_columns new file mode 100644 index 0000000000000..504d1259929c6 --- /dev/null +++ b/test/sql/test_partial_update_column_mode/T/test_partial_update_many_columns @@ -0,0 +1,45 @@ +-- name: test_partial_update_many_columns +show backends; +create database test_partial_update_many_columns; +use test_partial_update_many_columns; +CREATE table tab1 ( + k1 INTEGER, + k2 VARCHAR(50), + v1 INTEGER, + v2 INTEGER, + v3 INTEGER, + v4 varchar(50), + v5 varchar(50), + v6 INTEGER, + v7 INTEGER, + v8 INTEGER, + v9 varchar(50), + v10 varchar(50), + v11 INTEGER, + v12 INTEGER, + v13 INTEGER, + v14 varchar(50), + v15 varchar(50) +) +ENGINE=OLAP +PRIMARY KEY(`k1`,`k2`) +DISTRIBUTED BY HASH(`k1`) BUCKETS 10 +PROPERTIES ( + "replication_num" = "1" +); +set partial_update_mode = "column"; +insert into tab1 values (100, "k2_100", 100, 100, 100, "v4_100", "v5_100", 100, 100, 100, "v9_100", "v10_100", 100, 100, 100, "v14_100", "v15_100"); +insert into tab1 values (200, "k2_200", 200, 200, 200, "v4_200", "v5_200", 200, 200, 200, "v9_200", "v10_200", 200, 200, 200, "v14_200", "v15_200"); +insert into tab1 values (300, "k2_300", 300, 300, 300, "v4_300", "v5_300", 300, 300, 300, "v9_200", "v10_200", 200, 200, 200, "v14_200", "v15_200"); +insert into tab1 values (400, "k2_400", 300, 300, 300, "v4_300", "v5_300", 300, 300, 300, "v9_200", "v10_200", 200, 200, 200, "v14_200", "v15_200"); +insert into tab1 values (500, "k2_500", 300, 300, 300, "v4_300", "v5_300", 300, 300, 300, "v9_200", "v10_200", 200, 200, 200, "v14_200", "v15_200"); +insert into tab1 values (600, "k2_600", 300, 300, 300, "v4_300", "v5_300", 300, 300, 300, "v9_200", "v10_200", 200, 200, 200, "v14_200", "v15_200"); +select * from tab1; +update tab1 set v2 = 1000, v4 = "v4_1000", v6 = 1000, v8 = 1000, v10 = "v10_1000", v12 = 1000; +select * from tab1; +update tab1 set v2 = 1000, v3 = 1000, v4 = "v4_1000", v5 = "v5_1000", v6 = 1000, v7 = 1000, v8 = 1000, v9 = "v9_1000", v10 = "v10_1000", v11 = 1000, v12 = 1000; +select * from tab1; + +shell: curl --location-trusted -u root: -T ${root_path}/lib/../common/data/stream_load/sr_partial_update_3.csv -XPUT -H partial_update:true -H label:test_partial_update_many_columns -H partial_update_mode:column -H column_separator:, -H columns:k1,k2,v9,v10,v14,v11,v15,v5,v4 ${url}/api/test_partial_update_many_columns/tab1/_stream_load +sync; +select * from tab1; \ No newline at end of file