diff --git a/README.md b/README.md index dda7bf265e237f..8a6aed6370a95f 100644 --- a/README.md +++ b/README.md @@ -135,6 +135,8 @@ This project is used by the following companies. Learn more about their use case - [Demandbase](https://starrocks.medium.com/demandbase-ditches-denormalization-by-switching-off-clickhouse-44195d795a83) - [Shopee](https://celerdata.com/blog/how-shopee-3xed-their-query-performance-with-starrocks) - [Naver](https://starrocks.medium.com/how-join-changed-how-we-approach-data-infra-at-naver-3a5bb1dac49f) +- [HerdWatch](https://medium.com/p/a7916a7e87bf) - [TRM Labs](https://www.trmlabs.com/post/from-bigquery-to-lakehouse-how-we-built-a-petabyte-scale-data-analytics-platform-part-1) +- [Verisoul](https://celerdata.com/blog/verisoul-enables-real-time-analytics-by-transitioning-off-bigquery) - [Trip.com](https://starrocks.medium.com/trip-com-starrocks-efficiently-supports-high-concurrent-queries-dramatically-reduces-labor-and-1e1921dd6bf8) - [Didi](https://www.starrocks.io/blog/reduced-80-cost-didis-journey-from-multiple-olap-engines-to-starrocks) diff --git a/be/src/bench/get_dict_codes_bench.cpp b/be/src/bench/get_dict_codes_bench.cpp index aa23ccceb96fa8..65d949463d6438 100644 --- a/be/src/bench/get_dict_codes_bench.cpp +++ b/be/src/bench/get_dict_codes_bench.cpp @@ -15,7 +15,6 @@ #include #include -#include #include #include @@ -75,7 +74,7 @@ static void BM_GetDictCodesWithMap(benchmark::State& state) { if (column->size() == 0) { continue; } - const std::vector& null_data = down_cast(column.get())->immutable_null_column_data(); + const auto& null_data = down_cast(column.get())->immutable_null_column_data(); bool has_null = column->has_null(); bool all_null = false; @@ -90,7 +89,7 @@ static void BM_GetDictCodesWithMap(benchmark::State& state) { auto* dict_nullable_column = down_cast(column.get()); auto* dict_value_binary_column = down_cast(dict_nullable_column->data_column().get()); - std::vector dict_values_filtered = dict_value_binary_column->get_data(); + auto dict_values_filtered = dict_value_binary_column->get_data(); if (!has_null) { dict_codes.reserve(dict_values_filtered.size()); for (size_t i = 0; i < dict_values_filtered.size(); i++) { diff --git a/be/src/bench/parquet_dict_decode_bench.cpp b/be/src/bench/parquet_dict_decode_bench.cpp index 57ae7fcb147b56..d770f188c4c2e3 100644 --- a/be/src/bench/parquet_dict_decode_bench.cpp +++ b/be/src/bench/parquet_dict_decode_bench.cpp @@ -66,7 +66,7 @@ static void BM_DictDecoder(benchmark::State& state) { std::random_device rd; std::mt19937 rng(rd()); std::uniform_int_distribution dist(0, 99); - std::vector dict_codes; + Buffer dict_codes; int count = 0; for (int i = 0; i < kTestChunkSize; i++) { diff --git a/be/src/block_cache/datacache_utils.cpp b/be/src/block_cache/datacache_utils.cpp index a8e4d6876f7c40..7e60536a165b21 100644 --- a/be/src/block_cache/datacache_utils.cpp +++ b/be/src/block_cache/datacache_utils.cpp @@ -124,22 +124,6 @@ Status DataCacheUtils::parse_conf_datacache_disk_paths(const std::string& config return Status::OK(); } -Status DataCacheUtils::parse_conf_datacache_disk_spaces(const std::string& config_disk_path, - const std::string& config_disk_size, bool ignore_broken_disk, - std::vector* disk_spaces) { - std::vector paths; - RETURN_IF_ERROR(parse_conf_datacache_disk_paths(config_disk_path, &paths, ignore_broken_disk)); - for (auto& p : paths) { - int64_t disk_size = parse_conf_datacache_disk_size(p, config_disk_size, -1); - if (disk_size < 0) { - LOG(ERROR) << "invalid disk size for datacache: " << disk_size; - return Status::InvalidArgument("invalid disk size for datacache"); - } - disk_spaces->push_back({.path = p, .size = static_cast(disk_size)}); - } - return Status::OK(); -} - void DataCacheUtils::clean_residual_datacache(const std::string& disk_path) { if (!FileSystem::Default()->path_exists(disk_path).ok()) { // ignore none existed disk path diff --git a/be/src/block_cache/datacache_utils.h b/be/src/block_cache/datacache_utils.h index 14f28d4da3c999..5bf10d8781336a 100644 --- a/be/src/block_cache/datacache_utils.h +++ b/be/src/block_cache/datacache_utils.h @@ -32,10 +32,6 @@ class DataCacheUtils { static Status parse_conf_datacache_disk_paths(const std::string& config_path, std::vector* paths, bool ignore_broken_disk); - static Status parse_conf_datacache_disk_spaces(const std::string& config_disk_path, - const std::string& config_disk_size, bool ignore_broken_disk, - std::vector* disk_spaces); - static void clean_residual_datacache(const std::string& disk_path); static Status change_disk_path(const std::string& old_disk_path, const std::string& new_disk_path); diff --git a/be/src/common/config.h b/be/src/common/config.h index f452c8bba74fca..e90c7dd0152d40 100644 --- a/be/src/common/config.h +++ b/be/src/common/config.h @@ -201,9 +201,6 @@ CONF_Int32(be_service_threads, "64"); // key=value pair of default query options for StarRocks, separated by ',' CONF_String(default_query_options, ""); -// If non-zero, StarRocks will output memory usage every log_mem_usage_interval'th fragment completion. -// CONF_Int32(log_mem_usage_interval, "0"); - // Controls the number of threads to run work per core. It's common to pick 2x // or 3x the number of cores. This keeps the cores busy without causing excessive // thrashing. @@ -1183,7 +1180,7 @@ CONF_Double(datacache_scheduler_threads_per_cpu, "0.125"); // If false, the raw data will be written to disk directly and read from disk without promotion. // For object data, such as parquet footer object, which can only be cached in memory are not affected // by this configuration. -CONF_Bool(datacache_tiered_cache_enable, "true"); +CONF_Bool(datacache_tiered_cache_enable, "false"); // Whether to persist cached data CONF_Bool(datacache_persistence_enable, "true"); // DataCache engines, alternatives: starcache. @@ -1566,4 +1563,6 @@ CONF_mInt32(json_parse_many_batch_size, "1000000"); CONF_mBool(enable_dynamic_batch_size_for_json_parse_many, "true"); CONF_mInt32(put_combined_txn_log_thread_pool_num_max, "64"); CONF_mBool(enable_put_combinded_txn_log_parallel, "false"); +// used to control whether the metrics/ interface collects table metrics +CONF_mBool(enable_collect_table_metrics, "true"); } // namespace starrocks::config diff --git a/be/src/exec/connector_scan_node.cpp b/be/src/exec/connector_scan_node.cpp index 9b8720aca38410..82b6b0a914e80e 100644 --- a/be/src/exec/connector_scan_node.cpp +++ b/be/src/exec/connector_scan_node.cpp @@ -74,6 +74,17 @@ Status ConnectorScanNode::init(const TPlanNode& tnode, RuntimeState* state) { } _estimate_scan_row_bytes(); + if (tnode.__isset.lake_scan_node) { + if (tnode.lake_scan_node.__isset.enable_topn_filter_back_pressure && + tnode.lake_scan_node.enable_topn_filter_back_pressure) { + _enable_topn_filter_back_pressure = true; + _back_pressure_max_rounds = tnode.lake_scan_node.back_pressure_max_rounds; + _back_pressure_num_rows = tnode.lake_scan_node.back_pressure_num_rows; + _back_pressure_throttle_time = tnode.lake_scan_node.back_pressure_throttle_time; + _back_pressure_throttle_time_upper_bound = tnode.lake_scan_node.back_pressure_throttle_time_upper_bound; + } + } + return Status::OK(); } diff --git a/be/src/exec/olap_scan_node.cpp b/be/src/exec/olap_scan_node.cpp index 75e4ed115c2742..8c5747e8a11751 100644 --- a/be/src/exec/olap_scan_node.cpp +++ b/be/src/exec/olap_scan_node.cpp @@ -109,6 +109,15 @@ Status OlapScanNode::init(const TPlanNode& tnode, RuntimeState* state) { } } + if (tnode.olap_scan_node.__isset.enable_topn_filter_back_pressure && + tnode.olap_scan_node.enable_topn_filter_back_pressure) { + _enable_topn_filter_back_pressure = true; + _back_pressure_max_rounds = tnode.olap_scan_node.back_pressure_max_rounds; + _back_pressure_num_rows = tnode.olap_scan_node.back_pressure_num_rows; + _back_pressure_throttle_time = tnode.olap_scan_node.back_pressure_throttle_time; + _back_pressure_throttle_time_upper_bound = tnode.olap_scan_node.back_pressure_throttle_time_upper_bound; + } + _estimate_scan_and_output_row_bytes(); return Status::OK(); diff --git a/be/src/exec/pipeline/aggregate/aggregate_streaming_sink_operator.h b/be/src/exec/pipeline/aggregate/aggregate_streaming_sink_operator.h index 2eba42c087b43c..d4164ac4c53ab0 100644 --- a/be/src/exec/pipeline/aggregate/aggregate_streaming_sink_operator.h +++ b/be/src/exec/pipeline/aggregate/aggregate_streaming_sink_operator.h @@ -77,6 +77,7 @@ class AggregateStreamingSinkOperator : public Operator { AggrAutoState _auto_state{}; AggrAutoContext _auto_context; LimitedMemAggState _limited_mem_state; + DECLARE_ONCE_DETECTOR(_set_finishing_once); }; class AggregateStreamingSinkOperatorFactory final : public OperatorFactory { diff --git a/be/src/exec/pipeline/operator.h b/be/src/exec/pipeline/operator.h index c2e6cea2769f07..54176a2c362bab 100644 --- a/be/src/exec/pipeline/operator.h +++ b/be/src/exec/pipeline/operator.h @@ -182,7 +182,7 @@ class Operator { Status eval_conjuncts(const std::vector& conjuncts, Chunk* chunk, FilterPtr* filter = nullptr); // equal to ExecNode::eval_join_runtime_filters, is used to apply bloom-filters to Operators. - void eval_runtime_bloom_filters(Chunk* chunk); + virtual void eval_runtime_bloom_filters(Chunk* chunk); // Pseudo plan_node_id for final sink, such as result_sink, table_sink static const int32_t s_pseudo_plan_node_id_for_final_sink; @@ -276,6 +276,8 @@ class Operator { void set_observer(PipelineObserver* observer) { _observer = observer; } PipelineObserver* observer() const { return _observer; } + void _init_rf_counters(bool init_bloom); + protected: OperatorFactory* _factory; const int32_t _id; @@ -334,7 +336,6 @@ class Operator { PipelineObserver* _observer = nullptr; private: - void _init_rf_counters(bool init_bloom); void _init_conjuct_counters(); std::shared_ptr _mem_tracker; diff --git a/be/src/exec/pipeline/scan/scan_operator.cpp b/be/src/exec/pipeline/scan/scan_operator.cpp index 5c62844853296d..109292a10a93ee 100644 --- a/be/src/exec/pipeline/scan/scan_operator.cpp +++ b/be/src/exec/pipeline/scan/scan_operator.cpp @@ -90,6 +90,19 @@ Status ScanOperator::prepare(RuntimeState* state) { _prepare_chunk_source_timer = ADD_TIMER(_unique_metrics, "PrepareChunkSourceTime"); _submit_io_task_timer = ADD_TIMER(_unique_metrics, "SubmitTaskTime"); + if (_scan_node->is_enable_topn_filter_back_pressure()) { + if (auto* runtime_filters = runtime_bloom_filters(); runtime_filters != nullptr) { + auto has_topn_filters = + std::any_of(runtime_filters->descriptors().begin(), runtime_filters->descriptors().end(), + [](const auto& e) { return e.second->is_topn_filter(); }); + if (has_topn_filters) { + _topn_filter_back_pressure = std::make_unique( + 0.1, _scan_node->get_back_pressure_throttle_time_upper_bound(), + _scan_node->get_back_pressure_max_rounds(), _scan_node->get_back_pressure_throttle_time(), + _scan_node->get_back_pressure_num_rows()); + } + } + } RETURN_IF_ERROR(do_prepare(state)); return Status::OK(); } @@ -139,6 +152,10 @@ bool ScanOperator::has_output() const { return true; } + if (!_morsel_queue->empty() && _topn_filter_back_pressure && _topn_filter_back_pressure->should_throttle()) { + return false; + } + // Try to buffer enough chunks for exec thread, to reduce scheduling overhead. // It's like the Linux Block-Scheduler's Unplug algorithm, so we just name it unplug. // The default threshould of unpluging is BufferCapacity/DOP/4, and its range is [1, 16] @@ -275,6 +292,7 @@ StatusOr ScanOperator::pull_chunk(RuntimeState* state) { begin_pull_chunk(res); // for query cache mechanism, we should emit EOS chunk when we receive the last chunk. auto [owner_id, is_eos] = _should_emit_eos(res); + evaluate_topn_runtime_filters(res.get()); eval_runtime_bloom_filters(res.get()); res->owner_info().set_owner_id(owner_id, is_eos); } diff --git a/be/src/exec/pipeline/scan/scan_operator.h b/be/src/exec/pipeline/scan/scan_operator.h index bc2e8907110ae4..515352e8fac52b 100644 --- a/be/src/exec/pipeline/scan/scan_operator.h +++ b/be/src/exec/pipeline/scan/scan_operator.h @@ -14,9 +14,11 @@ #pragma once +#include "exec/exec_node.h" #include "exec/pipeline/pipeline_fwd.h" #include "exec/pipeline/scan/balanced_chunk_buffer.h" #include "exec/pipeline/source_operator.h" +#include "exec/pipeline/topn_runtime_filter_back_pressure.h" #include "exec/query_cache/cache_operator.h" #include "exec/query_cache/lane_arbiter.h" #include "exec/workgroup/work_group_fwd.h" @@ -171,6 +173,57 @@ class ScanOperator : public SourceOperator { return _scan_status; } + void evaluate_topn_runtime_filters(Chunk* chunk) { + if (chunk == nullptr || chunk->is_empty() || !_topn_filter_back_pressure) { + return; + } + if (auto* topn_runtime_filters = runtime_bloom_filters()) { + auto input_num_rows = chunk->num_rows(); + _init_topn_runtime_filter_counters(); + topn_runtime_filters->evaluate(chunk, _topn_filter_eval_context); + _topn_filter_back_pressure->inc_num_rows(chunk->num_rows()); + if (_topn_filter_eval_context.selectivity.empty()) { + _topn_filter_back_pressure->update_selectivity(1.0); + } else { + double selectivity = _topn_filter_eval_context.selectivity.begin()->first; + if (input_num_rows > 1024) { + _topn_filter_back_pressure->update_selectivity(selectivity); + } + } + } + } + + void _init_topn_runtime_filter_counters() { + if (_topn_filter_eval_context.join_runtime_filter_timer == nullptr) { + _topn_filter_eval_context.mode = RuntimeBloomFilterEvalContext::Mode::M_ONLY_TOPN; + _topn_filter_eval_context.join_runtime_filter_timer = ADD_TIMER(_common_metrics, "TopnRuntimeFilterTime"); + _topn_filter_eval_context.join_runtime_filter_hash_timer = + ADD_TIMER(_common_metrics, "TopnRuntimeFilterHashTime"); + _topn_filter_eval_context.join_runtime_filter_input_counter = + ADD_COUNTER(_common_metrics, "TopnRuntimeFilterInputRows", TUnit::UNIT); + _topn_filter_eval_context.join_runtime_filter_output_counter = + ADD_COUNTER(_common_metrics, "TopnRuntimeFilterOutputRows", TUnit::UNIT); + _topn_filter_eval_context.join_runtime_filter_eval_counter = + ADD_COUNTER(_common_metrics, "TopnRuntimeFilterEvaluate", TUnit::UNIT); + _topn_filter_eval_context.driver_sequence = _runtime_filter_probe_sequence; + } + } + + void eval_runtime_bloom_filters(Chunk* chunk) override { + if (chunk == nullptr || chunk->is_empty()) { + return; + } + + if (auto* bloom_filters = runtime_bloom_filters()) { + _init_rf_counters(true); + if (_topn_filter_back_pressure) { + _bloom_filter_eval_context.mode = RuntimeBloomFilterEvalContext::Mode::M_WITHOUT_TOPN; + } + bloom_filters->evaluate(chunk, _bloom_filter_eval_context); + } + ExecNode::eval_filter_null_values(chunk, filter_null_value_columns()); + } + protected: ScanNode* _scan_node = nullptr; const int32_t _dop; @@ -234,6 +287,9 @@ class ScanOperator : public SourceOperator { RuntimeProfile::Counter* _prepare_chunk_source_timer = nullptr; RuntimeProfile::Counter* _submit_io_task_timer = nullptr; + RuntimeBloomFilterEvalContext _topn_filter_eval_context; + std::unique_ptr _topn_filter_back_pressure = nullptr; + DECLARE_RACE_DETECTOR(race_pull_chunk) }; @@ -261,7 +317,6 @@ class ScanOperatorFactory : public SourceOperatorFactory { protected: ScanNode* const _scan_node; - std::shared_ptr _scan_task_group; }; diff --git a/be/src/exec/pipeline/topn_runtime_filter_back_pressure.h b/be/src/exec/pipeline/topn_runtime_filter_back_pressure.h new file mode 100644 index 00000000000000..b51391865f66f3 --- /dev/null +++ b/be/src/exec/pipeline/topn_runtime_filter_back_pressure.h @@ -0,0 +1,108 @@ +// 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 +#include +#include + +namespace starrocks::pipeline { +using std::chrono::milliseconds; +using std::chrono::steady_clock; +class TopnRfBackPressure { + enum Phase { PH_UNTHROTTLE, PH_THROTTLE, PH_PASS_THROUGH }; + + template + class ScaleGenerator { + public: + ScaleGenerator(T initial_value, T delta, double factor, std::function next_cb) + : initial_value(initial_value), delta(delta), factor(factor), next_cb(next_cb), value(initial_value) {} + + T limit() { return value; } + void next() { + value += delta; + value *= factor; + } + bool has_next() { return next_cb(value); } + + private: + const T initial_value; + const T delta; + const double factor; + const std::function next_cb; + T value; + }; + +public: + void update_selectivity(double selectivity) { _current_selectivity = selectivity; } + void inc_num_rows(size_t num_rows) { _current_num_rows += num_rows; } + + bool should_throttle() { + if (_phase == PH_PASS_THROUGH) { + return false; + } else if (!_round_limiter.has_next() || !_throttle_time_limiter.has_next() || !_num_rows_limiter.has_next() || + _current_selectivity <= _selectivity_lower_bound || + _current_total_throttle_time >= _throttle_time_upper_bound) { + _phase = PH_PASS_THROUGH; + return false; + } + + if (_phase == PH_UNTHROTTLE) { + if (_current_num_rows <= _num_rows_limiter.limit()) { + return false; + } + _phase = PH_THROTTLE; + _current_throttle_deadline = duration_cast(steady_clock::now().time_since_epoch()).count() + + _throttle_time_limiter.limit(); + return true; + } else { + auto now = duration_cast(steady_clock::now().time_since_epoch()).count(); + if (now < _current_throttle_deadline) { + return true; + } + _phase = PH_UNTHROTTLE; + _current_num_rows = 0; + _current_total_throttle_time += _throttle_time_limiter.limit(); + _round_limiter.next(); + _throttle_time_limiter.next(); + _num_rows_limiter.next(); + return false; + } + } + + TopnRfBackPressure(double selectivity_lower_bound, int64_t throttle_time_upper_bound, int max_rounds, + int64_t throttle_time, size_t num_rows) + : _selectivity_lower_bound(selectivity_lower_bound), + _throttle_time_upper_bound(throttle_time_upper_bound), + _round_limiter(0, 1, 1.0, [max_rounds](int r) { return r < max_rounds; }), + _throttle_time_limiter(throttle_time, 0, 1.0, [](int64_t) { return true; }), + _num_rows_limiter(num_rows, 0, 2.0, [](size_t n) { return n < std::numeric_limits::max() / 2; }) { + } + +private: + const double _selectivity_lower_bound; + const int64_t _throttle_time_upper_bound; + Phase _phase{PH_UNTHROTTLE}; + ScaleGenerator _round_limiter; + ScaleGenerator _throttle_time_limiter; + ScaleGenerator _num_rows_limiter; + int64_t _current_throttle_deadline{-1}; + int64_t _current_total_throttle_time{0}; + size_t _current_num_rows{0}; + double _current_selectivity{1.0}; +}; + +} // namespace starrocks::pipeline diff --git a/be/src/exec/query_cache/cache_manager.cpp b/be/src/exec/query_cache/cache_manager.cpp index 5a3d0362bde9f9..2535dad1264f7d 100644 --- a/be/src/exec/query_cache/cache_manager.cpp +++ b/be/src/exec/query_cache/cache_manager.cpp @@ -25,7 +25,8 @@ static void delete_cache_entry(const CacheKey& key, void* value) { void CacheManager::populate(const std::string& key, const CacheValue& value) { auto* cache_value = new CacheValue(value); - auto* handle = _cache.insert(key, cache_value, cache_value->size(), &delete_cache_entry, CachePriority::NORMAL); + size_t value_size = cache_value->size(); + auto* handle = _cache.insert(key, cache_value, value_size, value_size, &delete_cache_entry, CachePriority::NORMAL); _cache.release(handle); } diff --git a/be/src/exec/scan_node.h b/be/src/exec/scan_node.h index a3853449908f62..29b4ea2ec4b7e2 100644 --- a/be/src/exec/scan_node.h +++ b/be/src/exec/scan_node.h @@ -131,6 +131,21 @@ class ScanNode : public ExecNode { const std::vector& column_access_paths() const { return _column_access_paths; } + bool is_enable_topn_filter_back_pressure() const { return this->_enable_topn_filter_back_pressure; } + void set_enable_topn_filter_back_pressure(bool value) { this->_enable_topn_filter_back_pressure = value; } + int get_back_pressure_max_rounds() const { return this->_back_pressure_max_rounds; } + void set_back_pressure_max_rounds(int value) { this->_back_pressure_max_rounds = value; } + size_t get_back_pressure_num_rows() const { return this->_back_pressure_num_rows; } + void set_back_pressure_num_rows(size_t value) { this->_back_pressure_num_rows = value; } + int64_t get_back_pressure_throttle_time() const { return this->_back_pressure_throttle_time; } + void set_back_pressure_throttle_time(int64_t value) { this->_back_pressure_throttle_time = value; } + int64_t get_back_pressure_throttle_time_upper_bound() const { + return this->_back_pressure_throttle_time_upper_bound; + } + void set_back_pressure_throttle_time_upper_bound(int64_t value) { + this->_back_pressure_throttle_time_upper_bound = value; + } + protected: RuntimeProfile::Counter* _bytes_read_counter = nullptr; // # bytes read from the scanner // # rows/tuples read from the scanner (including those discarded by eval_conjucts()) @@ -150,6 +165,12 @@ class ScanNode : public ExecNode { int32_t _io_tasks_per_scan_operator = config::io_tasks_per_scan_operator; std::vector _column_access_paths; + + bool _enable_topn_filter_back_pressure = false; + int _back_pressure_max_rounds = 5; + size_t _back_pressure_num_rows = 10240; + int64_t _back_pressure_throttle_time = 500; + int64_t _back_pressure_throttle_time_upper_bound = 5000; }; } // namespace starrocks diff --git a/be/src/exec/spill/file_block_manager.cpp b/be/src/exec/spill/file_block_manager.cpp index c2ead83bd5d127..1f72d456285da4 100644 --- a/be/src/exec/spill/file_block_manager.cpp +++ b/be/src/exec/spill/file_block_manager.cpp @@ -234,7 +234,10 @@ StatusOr FileBlockManager::get_or_create_container(const << plan_node_name; uint64_t id = _next_container_id++; std::string container_dir = dir->dir() + "/" + print_id(_query_id); - RETURN_IF_ERROR(dir->fs()->create_dir_if_missing(container_dir)); + if (_last_created_container_dir != container_dir) { + RETURN_IF_ERROR(dir->fs()->create_dir_if_missing(container_dir)); + _last_created_container_dir = container_dir; + } ASSIGN_OR_RETURN(auto block_container, FileBlockContainer::create(dir, _query_id, fragment_instance_id, plan_node_id, plan_node_name, id, block_size)); RETURN_IF_ERROR(block_container->open()); diff --git a/be/src/exec/spill/file_block_manager.h b/be/src/exec/spill/file_block_manager.h index e25d216250f385..be4d8058b08e9f 100644 --- a/be/src/exec/spill/file_block_manager.h +++ b/be/src/exec/spill/file_block_manager.h @@ -48,6 +48,7 @@ class FileBlockManager : public BlockManager { TUniqueId _query_id; std::atomic _next_container_id = 0; + std::string _last_created_container_dir; DirManager* _dir_mgr = nullptr; }; diff --git a/be/src/exec/tablet_sink.cpp b/be/src/exec/tablet_sink.cpp index 44556c432a4961..e4ce1581e013e7 100644 --- a/be/src/exec/tablet_sink.cpp +++ b/be/src/exec/tablet_sink.cpp @@ -65,7 +65,6 @@ #include "util/compression/compression_utils.h" #include "util/defer_op.h" #include "util/stack_util.h" -#include "util/starrocks_metrics.h" #include "util/thread.h" #include "util/thrift_rpc_helper.h" #include "util/uid_util.h" diff --git a/be/src/exprs/dictmapping_expr.h b/be/src/exprs/dictmapping_expr.h index 4f0d651c8f2a9c..8441a7b9d69b88 100644 --- a/be/src/exprs/dictmapping_expr.h +++ b/be/src/exprs/dictmapping_expr.h @@ -44,6 +44,8 @@ class DictMappingExpr final : public Expr { StatusOr evaluate_checked(ExprContext* context, Chunk* ptr) override; + bool is_dictmapping_expr() const override { return true; } + template Status rewrite(Rewrite&& rewriter) { std::call_once(*_rewrite_once_flag, [&]() { diff --git a/be/src/exprs/expr.h b/be/src/exprs/expr.h index b9ac42915947d1..c7f221643667cc 100644 --- a/be/src/exprs/expr.h +++ b/be/src/exprs/expr.h @@ -122,6 +122,7 @@ class Expr { bool is_cast_expr() const { return _node_type == TExprNodeType::CAST_EXPR; } virtual bool is_lambda_function() const { return false; } virtual bool is_literal() const { return false; } + virtual bool is_dictmapping_expr() const { return false; } // In most time, this field is passed from FE // Sometimes we want to construct expr on BE implicitly and we have knowledge about `monotonicity` diff --git a/be/src/exprs/jit/jit_engine.cpp b/be/src/exprs/jit/jit_engine.cpp index 706832a2db96fe..8cb2ca04672bb2 100644 --- a/be/src/exprs/jit/jit_engine.cpp +++ b/be/src/exprs/jit/jit_engine.cpp @@ -98,12 +98,13 @@ Status JitObjectCache::register_func(JITScalarFunction func) { // put into LRU cache auto* cache = new JitCacheEntry(_obj_code, _func); GlobalEnv::GetInstance()->jit_cache_mem_tracker()->consume(cache_func_size); - auto* handle = _lru_cache->insert(_cache_key, (void*)cache, cache_func_size, [](const CacheKey& key, void* value) { - auto* entry = ((JitCacheEntry*)value); - // maybe release earlier as the std::shared_ptr is hold by caller - GlobalEnv::GetInstance()->jit_cache_mem_tracker()->release(entry->obj_buff->getBufferSize()); - delete entry; - }); + auto* handle = _lru_cache->insert( + _cache_key, (void*)cache, cache_func_size, cache_func_size, [](const CacheKey& key, void* value) { + auto* entry = ((JitCacheEntry*)value); + // maybe release earlier as the std::shared_ptr is hold by caller + GlobalEnv::GetInstance()->jit_cache_mem_tracker()->release(entry->obj_buff->getBufferSize()); + delete entry; + }); if (handle == nullptr) { delete cache; LOG(WARNING) << "JIT register func failed, func = " << _cache_key << ", ir size = " << cache_func_size; diff --git a/be/src/exprs/lambda_function.cpp b/be/src/exprs/lambda_function.cpp index 7aa4cca599a862..a5b8428de772c3 100644 --- a/be/src/exprs/lambda_function.cpp +++ b/be/src/exprs/lambda_function.cpp @@ -58,6 +58,10 @@ Status LambdaFunction::extract_outer_common_exprs(RuntimeState* state, ExprConte } }); + if (expr->is_dictmapping_expr()) { + return Status::OK(); + } + // for the lambda function, we only consider extracting the outer common expression from the lambda expr, // not its arguments int child_num = expr->is_lambda_function() ? 1 : expr->get_num_children(); diff --git a/be/src/exprs/runtime_filter_bank.cpp b/be/src/exprs/runtime_filter_bank.cpp index 66cb8149b15c21..7c390d2f78f922 100644 --- a/be/src/exprs/runtime_filter_bank.cpp +++ b/be/src/exprs/runtime_filter_bank.cpp @@ -196,22 +196,6 @@ StatusOr RuntimeFilterHelper::rewrite_runtime_filter_in_cross_join return expr; } -struct FilterZoneMapWithMinMaxOp { - template - bool operator()(const RuntimeFilter* expr, const Column* min_column, const Column* max_column) { - using CppType = RunTimeCppType; - auto* filter = (RuntimeBloomFilter*)(expr); - const CppType* min_value = ColumnHelper::unpack_cpp_data_one_value(min_column); - const CppType* max_value = ColumnHelper::unpack_cpp_data_one_value(max_column); - return filter->filter_zonemap_with_min_max(min_value, max_value); - } -}; - -bool RuntimeFilterHelper::filter_zonemap_with_min_max(LogicalType type, const RuntimeFilter* filter, - const Column* min_column, const Column* max_column) { - return type_dispatch_filter(type, false, FilterZoneMapWithMinMaxOp(), filter, min_column, max_column); -} - Status RuntimeFilterBuildDescriptor::init(ObjectPool* pool, const TRuntimeFilterDescription& desc, RuntimeState* state) { _filter_id = desc.filter_id; @@ -391,9 +375,14 @@ void RuntimeFilterProbeCollector::close(RuntimeState* state) { // do_evaluate is reentrant, can be called concurrently by multiple operators that shared the same // RuntimeFilterProbeCollector. void RuntimeFilterProbeCollector::do_evaluate(Chunk* chunk, RuntimeBloomFilterEvalContext& eval_context) { - if ((eval_context.input_chunk_nums++ & 31) == 0) { + if (eval_context.mode == RuntimeBloomFilterEvalContext::Mode::M_ONLY_TOPN) { update_selectivity(chunk, eval_context); return; + } else { + if ((eval_context.input_chunk_nums++ & 31) == 0) { + update_selectivity(chunk, eval_context); + return; + } } auto& seletivity_map = eval_context.selectivity; @@ -409,7 +398,8 @@ void RuntimeFilterProbeCollector::do_evaluate(Chunk* chunk, RuntimeBloomFilterEv for (auto& kv : seletivity_map) { RuntimeFilterProbeDescriptor* rf_desc = kv.second; const RuntimeFilter* filter = rf_desc->runtime_filter(eval_context.driver_sequence); - if (filter == nullptr || filter->always_true()) { + bool skip_topn = eval_context.mode == RuntimeBloomFilterEvalContext::Mode::M_WITHOUT_TOPN; + if ((skip_topn && rf_desc->is_topn_filter()) || filter == nullptr || filter->always_true()) { continue; } auto* ctx = rf_desc->probe_expr_ctx(); @@ -586,9 +576,18 @@ void RuntimeFilterProbeCollector::update_selectivity(Chunk* chunk, RuntimeBloomF for (auto& kv : _descriptors) { RuntimeFilterProbeDescriptor* rf_desc = kv.second; const RuntimeFilter* filter = rf_desc->runtime_filter(eval_context.driver_sequence); - if (filter == nullptr || filter->always_true()) { + bool should_use = + eval_context.mode == RuntimeBloomFilterEvalContext::Mode::M_ONLY_TOPN && rf_desc->is_topn_filter(); + if (filter == nullptr || (!should_use && filter->always_true())) { continue; } + if (eval_context.mode == RuntimeBloomFilterEvalContext::Mode::M_WITHOUT_TOPN && rf_desc->is_topn_filter()) { + continue; + } else if (eval_context.mode == RuntimeBloomFilterEvalContext::Mode::M_ONLY_TOPN && + !rf_desc->is_topn_filter()) { + continue; + } + auto& selection = eval_context.running_context.use_merged_selection ? eval_context.running_context.merged_selection : eval_context.running_context.selection; @@ -630,6 +629,8 @@ void RuntimeFilterProbeCollector::update_selectivity(Chunk* chunk, RuntimeBloomF dest[j] = src[j] & dest[j]; } } + } else if (rf_desc->is_topn_filter() && eval_context.mode == RuntimeBloomFilterEvalContext::Mode::M_ONLY_TOPN) { + seletivity_map.emplace(selectivity, rf_desc); } } if (!seletivity_map.empty()) { diff --git a/be/src/exprs/runtime_filter_bank.h b/be/src/exprs/runtime_filter_bank.h index 84ce68e8b748d9..ab6f5dc8e43e68 100644 --- a/be/src/exprs/runtime_filter_bank.h +++ b/be/src/exprs/runtime_filter_bank.h @@ -69,9 +69,6 @@ class RuntimeFilterHelper { static StatusOr rewrite_runtime_filter_in_cross_join_node(ObjectPool* pool, ExprContext* conjunct, Chunk* chunk); - static bool filter_zonemap_with_min_max(LogicalType type, const RuntimeFilter* filter, const Column* min_column, - const Column* max_column); - // create min/max predicate from filter. static void create_min_max_value_predicate(ObjectPool* pool, SlotId slot_id, LogicalType slot_type, const RuntimeFilter* filter, Expr** min_max_predicate); @@ -216,6 +213,7 @@ class RuntimeFilterProbeDescriptor : public WithLayoutMixin { int64_t _ready_timestamp = 0; int8_t _join_mode; bool _is_topn_filter = false; + bool _skip_wait = false; // Indicates that the runtime filter was built from the colocate group execution build side. bool _is_group_colocate_rf = false; @@ -231,7 +229,12 @@ class RuntimeFilterProbeDescriptor : public WithLayoutMixin { // into RuntimeBloomFilterEvalContext and make do_evaluate function can be called concurrently. struct RuntimeBloomFilterEvalContext { RuntimeBloomFilterEvalContext() = default; - + enum Mode { + M_ALL, + M_WITHOUT_TOPN, + M_ONLY_TOPN, + }; + Mode mode = Mode::M_ALL; std::map selectivity; size_t input_chunk_nums = 0; int run_filter_nums = 0; diff --git a/be/src/formats/parquet/scalar_column_reader.cpp b/be/src/formats/parquet/scalar_column_reader.cpp index 2a3aa11159f58c..3b2cdb7054ef8d 100644 --- a/be/src/formats/parquet/scalar_column_reader.cpp +++ b/be/src/formats/parquet/scalar_column_reader.cpp @@ -16,7 +16,6 @@ #include "formats/parquet/column_reader.h" #include "formats/parquet/stored_column_reader_with_index.h" -#include "formats/parquet/utils.h" #include "formats/parquet/zone_map_filter_evaluator.h" #include "gutil/casts.h" #include "io/shared_buffered_input_stream.h" @@ -348,29 +347,16 @@ StatusOr RawColumnReader::_page_index_zone_map_filter(const std::vector& range, const Filter* filter, ColumnPtr& dst) { DCHECK(get_column_parquet_field()->is_nullable ? dst->is_nullable() : true); _need_lazy_decode = - _dict_filter_ctx != nullptr || (_can_lazy_decode && filter != nullptr && + _dict_filter_ctx != nullptr || (_can_lazy_dict_decode && filter != nullptr && SIMD::count_nonzero(*filter) * 1.0 / filter->size() < FILTER_RATIO); ColumnContentType content_type = !_need_lazy_decode ? ColumnContentType::VALUE : ColumnContentType::DICT_CODE; + auto need_lazy_covert = _can_lazy_convert && _converter->need_convert; if (_need_lazy_decode) { - if (_dict_code == nullptr) { - _dict_code = ColumnHelper::create_column( - TypeDescriptor::from_logical_type(ColumnDictFilterContext::kDictCodePrimitiveType), true); - } - _ori_column = dst; - dst = _dict_code; - dst->reserve(range.span_size()); - } - if (!_converter->need_convert) { - SCOPED_RAW_TIMER(&_opts.stats->column_read_ns); - return _reader->read_range(range, filter, content_type, dst.get()); + return _read_range_impl(range, filter, content_type, dst); + } else if (need_lazy_covert) { + return _read_range_impl(range, filter, content_type, dst); } else { - auto column = _converter->create_src_column(); - { - SCOPED_RAW_TIMER(&_opts.stats->column_read_ns); - RETURN_IF_ERROR(_reader->read_range(range, filter, content_type, column.get())); - } - SCOPED_RAW_TIMER(&_opts.stats->column_convert_ns); - return _converter->convert(column, dst.get()); + return _read_range_impl(range, filter, content_type, dst); } } @@ -399,32 +385,99 @@ bool ScalarColumnReader::try_to_use_dict_filter(ExprContext* ctx, bool is_decode } Status ScalarColumnReader::fill_dst_column(ColumnPtr& dst, ColumnPtr& src) { - if (!_need_lazy_decode) { - dst->swap_column(*src); + auto need_lazy_covert = _can_lazy_convert && _converter->need_convert; + if (_need_lazy_decode) { + return _fill_dst_column_impl(dst, src); + } else if (need_lazy_covert) { + return _fill_dst_column_impl(dst, src); } else { - if (_dict_filter_ctx == nullptr || _dict_filter_ctx->is_decode_needed) { - ColumnPtr& dict_values = dst; - dict_values->reserve(src->size()); - - // decode dict code to dict values. - // note that in dict code, there could be null value. - const ColumnPtr& dict_codes = src; - auto* codes_nullable_column = ColumnHelper::as_raw_column(dict_codes); - auto* codes_column = - ColumnHelper::as_raw_column>(codes_nullable_column->data_column()); - RETURN_IF_ERROR( - _reader->get_dict_values(codes_column->get_data(), *codes_nullable_column, dict_values.get())); - DCHECK_EQ(dict_codes->size(), dict_values->size()); - if (dict_values->is_nullable()) { - auto* nullable_values = down_cast(dict_values.get()); - nullable_values->swap_null_column(*codes_nullable_column); - } + return _fill_dst_column_impl(dst, src); + } +} + +template +Status ScalarColumnReader::_read_range_impl(const Range& range, const Filter* filter, + ColumnContentType content_type, ColumnPtr& dst) { + if constexpr (LAZY_DICT_DECODE) { + if (_tmp_code_column == nullptr) { + _tmp_code_column = ColumnHelper::create_column( + TypeDescriptor::from_logical_type(ColumnDictFilterContext::kDictCodePrimitiveType), true); + } + _ori_column = dst; + dst = _tmp_code_column; + dst->reserve(range.span_size()); + SCOPED_RAW_TIMER(&_opts.stats->column_read_ns); + return _reader->read_range(range, filter, content_type, dst.get()); + } else { + if (!_converter->need_convert) { + SCOPED_RAW_TIMER(&_opts.stats->column_read_ns); + return _reader->read_range(range, filter, content_type, dst.get()); } else { - dst->append_default(src->size()); + if (_tmp_intermediate_column == nullptr) { + _tmp_intermediate_column = _converter->create_src_column(); + } + _tmp_intermediate_column->reserve(range.span_size()); + { + SCOPED_RAW_TIMER(&_opts.stats->column_read_ns); + RETURN_IF_ERROR(_reader->read_range(range, filter, content_type, _tmp_intermediate_column.get())); + } + if constexpr (!LAZY_CONVERT) { + { + SCOPED_RAW_TIMER(&_opts.stats->column_convert_ns); + RETURN_IF_ERROR(_converter->convert(_tmp_intermediate_column, dst.get())); + } + _tmp_intermediate_column->reset_column(); + return Status::OK(); + } else { + _ori_column = dst; + dst = _tmp_intermediate_column; + return Status::OK(); + } } + } +} - src->reset_column(); +Status ScalarColumnReader::_dict_decode(ColumnPtr& dst, ColumnPtr& src) { + Column* dict_values = ColumnHelper::get_data_column(dst.get()); + dict_values->reserve(src->size()); + + // decode dict code to dict values. + // note that in dict code, there could be null value. + const ColumnPtr& dict_codes = src; + auto* codes_nullable_column = ColumnHelper::as_raw_column(dict_codes); + auto* codes_column = ColumnHelper::as_raw_column>(codes_nullable_column->data_column()); + RETURN_IF_ERROR(_reader->get_dict_values(codes_column->get_data(), *codes_nullable_column, dict_values)); + DCHECK_EQ(dict_codes->size(), dict_values->size()); + if (dst->is_nullable()) { + auto* nullable_values = down_cast(dst.get()); + nullable_values->swap_null_column(*codes_nullable_column); + } + src->reset_column(); + return Status::OK(); +} + +template +Status ScalarColumnReader::_fill_dst_column_impl(ColumnPtr& dst, ColumnPtr& src) { + if constexpr (LAZY_DICT_DECODE) { + if (_dict_filter_ctx != nullptr && !_dict_filter_ctx->is_decode_needed) { + dst->append_default(src->size()); + src->reset_column(); + } else { + static_assert(!LAZY_CONVERT, "LAZY_DICT_DECODE && LAZY_CONVERT == true is not supported by now"); + RETURN_IF_ERROR(_dict_decode(dst, src)); + } src = _ori_column; + } else { + if constexpr (LAZY_CONVERT) { + { + SCOPED_RAW_TIMER(&_opts.stats->column_convert_ns); + RETURN_IF_ERROR(_converter->convert(src, dst.get())); + } + src->reset_column(); + src = _ori_column; + } else { + dst->swap_column(*src); + } } return Status::OK(); } diff --git a/be/src/formats/parquet/scalar_column_reader.h b/be/src/formats/parquet/scalar_column_reader.h index 5a30971cd856f9..a7b8994218094b 100644 --- a/be/src/formats/parquet/scalar_column_reader.h +++ b/be/src/formats/parquet/scalar_column_reader.h @@ -17,6 +17,7 @@ #include "formats/parquet/column_converter.h" #include "formats/parquet/column_reader.h" #include "formats/parquet/stored_column_reader.h" +#include "formats/parquet/utils.h" #include "util/thrift_util.h" namespace starrocks::parquet { @@ -145,7 +146,8 @@ class ScalarColumnReader final : public RawColumnReader { } void set_can_lazy_decode(bool can_lazy_decode) override { - _can_lazy_decode = can_lazy_decode && _col_type->is_string_type() && column_all_pages_dict_encoded(); + _can_lazy_convert = can_lazy_decode; + _can_lazy_dict_decode = can_lazy_decode && _col_type->is_string_type() && column_all_pages_dict_encoded(); } Status filter_dict_column(const ColumnPtr& column, Filter* filter, const std::vector& sub_field_path, @@ -170,18 +172,29 @@ class ScalarColumnReader final : public RawColumnReader { } private: + template + Status _read_range_impl(const Range& range, const Filter* filter, ColumnContentType content_type, + ColumnPtr& dst); + + template + Status _fill_dst_column_impl(ColumnPtr& dst, ColumnPtr& src); + + Status _dict_decode(ColumnPtr& dst, ColumnPtr& src); + std::unique_ptr _converter; std::unique_ptr _dict_filter_ctx; const TypeDescriptor* _col_type = nullptr; - // _can_lazy_decode means string type and all page dict code - bool _can_lazy_decode = false; + // _can_lazy_dict_decode means string type and all page dict code + bool _can_lazy_dict_decode = false; + bool _can_lazy_convert = false; // we use lazy decode adaptively because of RLE && decoder may be better than filter && decoder static constexpr double FILTER_RATIO = 0.2; bool _need_lazy_decode = false; // dict code - ColumnPtr _dict_code = nullptr; + ColumnPtr _tmp_code_column = nullptr; + ColumnPtr _tmp_intermediate_column = nullptr; ColumnPtr _ori_column = nullptr; }; diff --git a/be/src/formats/parquet/statistics_helper.cpp b/be/src/formats/parquet/statistics_helper.cpp index 9831c86293aa4a..f8610e97b794d9 100644 --- a/be/src/formats/parquet/statistics_helper.cpp +++ b/be/src/formats/parquet/statistics_helper.cpp @@ -132,63 +132,6 @@ Status StatisticsHelper::decode_value_into_column(const ColumnPtr& column, const return Status::OK(); } -bool StatisticsHelper::can_be_used_for_statistics_filter(ExprContext* ctx, - StatisticsHelper::StatSupportedFilter& filter_type) { - const Expr* root_expr = ctx->root(); - if (root_expr->node_type() == TExprNodeType::IN_PRED && root_expr->op() == TExprOpcode::FILTER_IN) { - const Expr* c = root_expr->get_child(0); - if (c->node_type() != TExprNodeType::type::SLOT_REF) { - return false; - } - - const TypeDescriptor& td = c->type(); - // TODO: support more data type - if (!td.is_integer_type() && !td.is_string_type() && !td.is_date_type()) { - return false; - } - - LogicalType ltype = c->type().type; - - switch (ltype) { -#define M(NAME) \ - case LogicalType::NAME: { \ - if (dynamic_cast*>(root_expr) != nullptr) { \ - filter_type = StatisticsHelper::StatSupportedFilter::FILTER_IN; \ - return true; \ - } else { \ - return false; \ - } \ - } - APPLY_FOR_ALL_SCALAR_TYPE(M); -#undef M - default: - return false; - } - } else if (root_expr->node_type() == TExprNodeType::FUNCTION_CALL) { - std::string null_function_name; - if (root_expr->is_null_scalar_function(null_function_name)) { - const Expr* c = root_expr->get_child(0); - if (c->node_type() != TExprNodeType::type::SLOT_REF) { - return false; - } else { - if (null_function_name == "null") { - filter_type = StatisticsHelper::StatSupportedFilter::IS_NULL; - } else { - filter_type = StatisticsHelper::StatSupportedFilter::IS_NOT_NULL; - } - return true; - } - } else { - return false; - } - } else if (root_expr->node_type() == TExprNodeType::RUNTIME_FILTER_MIN_MAX_EXPR) { - filter_type = StatisticsHelper::StatSupportedFilter::RF_MIN_MAX; - return true; - } else { - return false; - } -} - void translate_to_string_value(const ColumnPtr& col, size_t i, std::string& value) { if (col->is_date()) { value = col->get(i).get_date().to_string(); @@ -219,167 +162,6 @@ void translate_to_string_value(const ColumnPtr& col, size_t i, std::string& valu }); } -Status StatisticsHelper::min_max_filter_on_min_max_stat(const std::vector& min_values, - const std::vector& max_values, - const std::vector& null_pages, - const std::vector& null_counts, ExprContext* ctx, - const ParquetField* field, const std::string& timezone, - Filter& selected) { - const Expr* root_expr = ctx->root(); - LogicalType ltype = root_expr->type().type; - switch (ltype) { -#define M(NAME) \ - case LogicalType::NAME: { \ - return min_max_filter_on_min_max_stat_t(min_values, max_values, null_pages, null_counts, \ - ctx, field, timezone, selected); \ - } - APPLY_FOR_ALL_SCALAR_TYPE(M); -#undef M - default: - return Status::OK(); - } -} - -template -Status StatisticsHelper::min_max_filter_on_min_max_stat_t(const std::vector& min_values, - const std::vector& max_values, - const std::vector& null_pages, - const std::vector& null_counts, ExprContext* ctx, - const ParquetField* field, const std::string& timezone, - Filter& selected) { - const Expr* root_expr = ctx->root(); - const auto* min_max_filter = dynamic_cast*>(root_expr); - bool rf_has_null = min_max_filter->has_null(); - - ColumnPtr min_column = ColumnHelper::create_column(root_expr->type(), true); - ColumnPtr max_column = ColumnHelper::create_column(root_expr->type(), true); - - auto rf_min_value = min_max_filter->get_min_value(); - auto rf_max_value = min_max_filter->get_max_value(); - - RETURN_IF_ERROR(StatisticsHelper::decode_value_into_column(min_column, min_values, null_pages, root_expr->type(), - field, timezone)); - RETURN_IF_ERROR(StatisticsHelper::decode_value_into_column(max_column, max_values, null_pages, root_expr->type(), - field, timezone)); - - for (size_t i = 0; i < min_values.size(); i++) { - if (!selected[i]) { - continue; - } - if (rf_has_null && null_counts[i] > 0) { - selected[i] = 1; - continue; - } - if (null_pages[i]) { - selected[i] = 0; - continue; - } - - auto zonemap_min_v = ColumnHelper::get_data_column_by_type(min_column.get())->get_data()[i]; - auto zonemap_max_v = ColumnHelper::get_data_column_by_type(max_column.get())->get_data()[i]; - if (zonemap_min_v > rf_max_value || zonemap_max_v < rf_min_value) { - selected[i] = 0; - continue; - } - } - - return Status::OK(); -} - -Status StatisticsHelper::in_filter_on_min_max_stat(const std::vector& min_values, - const std::vector& max_values, - const std::vector& null_pages, - const std::vector& null_counts, ExprContext* ctx, - const ParquetField* field, const std::string& timezone, - Filter& selected) { - const Expr* root_expr = ctx->root(); - DCHECK(root_expr->node_type() == TExprNodeType::IN_PRED && root_expr->op() == TExprOpcode::FILTER_IN); - const Expr* c = root_expr->get_child(0); - LogicalType ltype = c->type().type; - ColumnPtr values; - bool is_runtime_filter = false; - bool has_null = false; - switch (ltype) { -#define M(NAME) \ - case LogicalType::NAME: { \ - const auto* in_filter = dynamic_cast*>(root_expr); \ - if (in_filter != nullptr) { \ - values = in_filter->get_all_values(); \ - has_null = in_filter->null_in_set(); \ - is_runtime_filter = in_filter->is_join_runtime_filter(); \ - break; \ - } else { \ - return Status::OK(); \ - } \ - } - APPLY_FOR_ALL_SCALAR_TYPE(M); -#undef M - default: - return Status::OK(); - } - - // TODO: there is no need to use nullable column, - // but there are many places in our reader just treat column as nullable, and use down_cast - ColumnPtr min_col = ColumnHelper::create_column(c->type(), true); - min_col->reserve(min_values.size()); - RETURN_IF_ERROR(decode_value_into_column(min_col, min_values, null_pages, c->type(), field, timezone)); - min_col = down_cast(min_col.get())->data_column(); - ColumnPtr max_col = ColumnHelper::create_column(c->type(), true); - max_col->reserve(max_values.size()); - RETURN_IF_ERROR(decode_value_into_column(max_col, max_values, null_pages, c->type(), field, timezone)); - max_col = down_cast(max_col.get())->data_column(); - - // logic and example: - // there are two pairs of min/max value like [1, 4] (which means min_value is 1 and max value is 4), [4, 6] - // the in_const_predicate get values as [2, 3, 7] (which means we have a predicate like `in [2, 3, 7]`) - // so the step to decide if there is value in in-filter locate in min->max: - // 1. treat values in in-filter as a col, [2,3,7] - // 2. for each pair min/max value create predicate col >= min and col =< max - // [2,3,7] >= 1 & [2, 3, 7] <= 4; - // 3. evaluate the predicate, if there is nonzero in the result, - // we know there is at least one value locate in min->max - // [2,3,7] >= 1 -> [1, 1, 1] [2, 3, 7] <= 4 -> [1, 1, 0] - // so the result is [1, 1, 0], which means [2, 3] locate in 1->4 - // for [4, 6], [2, 3, 7] >= 4 -> [0, 0, 1], [2, 3, 7] =< 6 -> [1, 1, 0] - // [0, 0, 1] & [1, 1, 0] -> [0, 0, 0], so there is no value locate in 4->6 - - for (size_t i = 0; i < min_values.size(); i++) { - // just skip the area that filtered - if (!selected[i]) { - continue; - } - if (is_runtime_filter && has_null && null_counts[i] > 0) { - selected[i] = 1; - continue; - } - if (null_pages[i]) { - selected[i] = 0; - continue; - } - - ObjectPool pool; - std::string min_value; - std::string max_value; - - translate_to_string_value(min_col, i, min_value); - translate_to_string_value(max_col, i, max_value); - - Filter filter(values->size(), 1); - - ColumnPredicate* pred_ge = pool.add(new_column_ge_predicate(get_type_info(ltype), 0, min_value)); - RETURN_IF_ERROR(pred_ge->evaluate_and(values.get(), filter.data())); - if (!SIMD::contain_nonzero(filter)) { - selected[i] = 0; - continue; - } - ColumnPredicate* pred_le = pool.add(new_column_le_predicate(get_type_info(ltype), 0, max_value)); - RETURN_IF_ERROR(pred_le->evaluate_and(values.get(), filter.data())); - selected[i] = SIMD::contain_nonzero(filter) ? selected[i] : 0; - } - - return Status::OK(); -} - Status StatisticsHelper::get_min_max_value(const FileMetaData* file_metadata, const TypeDescriptor& type, const tparquet::ColumnMetaData* column_meta, const ParquetField* field, std::vector& min_values, std::vector& max_values) { @@ -409,23 +191,6 @@ Status StatisticsHelper::get_min_max_value(const FileMetaData* file_metadata, co return Status::OK(); } -Status StatisticsHelper::get_has_nulls(const tparquet::ColumnMetaData* column_meta, std::vector& has_nulls) { - if (!column_meta->statistics.__isset.null_count) { - return Status::Aborted("No null_count in column statistics"); - } - has_nulls.emplace_back(column_meta->statistics.null_count > 0); - return Status::OK(); -} - -Status StatisticsHelper::get_null_counts(const tparquet::ColumnMetaData* column_meta, - std::vector& null_counts) { - if (!column_meta->statistics.__isset.null_count) { - return Status::Aborted("No null_count in column statistics"); - } - null_counts.emplace_back(column_meta->statistics.null_count); - return Status::OK(); -} - bool StatisticsHelper::has_correct_min_max_stats(const FileMetaData* file_metadata, const tparquet::ColumnMetaData& column_meta, const SortOrder& sort_order) { diff --git a/be/src/formats/parquet/statistics_helper.h b/be/src/formats/parquet/statistics_helper.h index ce66a59857601a..723078da168f47 100644 --- a/be/src/formats/parquet/statistics_helper.h +++ b/be/src/formats/parquet/statistics_helper.h @@ -31,37 +31,11 @@ class StatisticsHelper { const std::vector& null_pages, const TypeDescriptor& type, const ParquetField* field, const std::string& timezone); - static bool can_be_used_for_statistics_filter(ExprContext* ctx, StatSupportedFilter& filter_type); - - static Status in_filter_on_min_max_stat(const std::vector& min_values, - const std::vector& max_values, - const std::vector& null_pages, - const std::vector& null_counts, ExprContext* ctx, - const ParquetField* field, const std::string& timezone, Filter& selected); - - static Status min_max_filter_on_min_max_stat(const std::vector& min_values, - const std::vector& max_values, - const std::vector& null_pages, - const std::vector& null_counts, ExprContext* ctx, - const ParquetField* field, const std::string& timezone, - Filter& selected); - - template - static Status min_max_filter_on_min_max_stat_t(const std::vector& min_values, - const std::vector& max_values, - const std::vector& null_pages, - const std::vector& null_counts, ExprContext* ctx, - const ParquetField* field, const std::string& timezone, - Filter& selected); - // get min/max value from row group stats static Status get_min_max_value(const FileMetaData* file_meta_data, const TypeDescriptor& type, const tparquet::ColumnMetaData* column_meta, const ParquetField* field, std::vector& min_values, std::vector& max_values); - static Status get_has_nulls(const tparquet::ColumnMetaData* column_meta, std::vector& has_nulls); - static Status get_null_counts(const tparquet::ColumnMetaData* column_meta, std::vector& null_counts); - static bool has_correct_min_max_stats(const FileMetaData* file_metadata, const tparquet::ColumnMetaData& column_meta, const SortOrder& sort_order); }; diff --git a/be/src/fs/fd_cache.cpp b/be/src/fs/fd_cache.cpp index f57b6caceb51fe..67c8abde7724c1 100644 --- a/be/src/fs/fd_cache.cpp +++ b/be/src/fs/fd_cache.cpp @@ -33,7 +33,7 @@ FdCache::~FdCache() { FdCache::Handle* FdCache::insert(std::string_view path, int fd) { void* value = reinterpret_cast(static_cast(fd)); // NOLINT - Cache::Handle* h = _cache->insert(CacheKey(path.data(), path.size()), value, 1, fd_deleter); + Cache::Handle* h = _cache->insert(CacheKey(path.data(), path.size()), value, 1, 1, fd_deleter); return reinterpret_cast(h); } diff --git a/be/src/http/action/metrics_action.cpp b/be/src/http/action/metrics_action.cpp index 7a00a3d6eabd44..e97561af5d382e 100644 --- a/be/src/http/action/metrics_action.cpp +++ b/be/src/http/action/metrics_action.cpp @@ -117,6 +117,7 @@ const std::string SimpleCoreMetricsVisitor::MAX_DISK_IO_UTIL_PERCENT = "max_disk const std::string SimpleCoreMetricsVisitor::MAX_NETWORK_SEND_BYTES_RATE = "max_network_send_bytes_rate"; const std::string SimpleCoreMetricsVisitor::MAX_NETWORK_RECEIVE_BYTES_RATE = "max_network_receive_bytes_rate"; +const std::string TableMetricsPrefix = "table_"; void PrometheusMetricsVisitor::visit(const std::string& prefix, const std::string& name, MetricCollector* collector) { if (collector->empty() || name.empty()) { return; @@ -127,6 +128,9 @@ void PrometheusMetricsVisitor::visit(const std::string& prefix, const std::strin } else { metric_name = prefix + "_" + name; } + if (!config::enable_collect_table_metrics && name.starts_with(TableMetricsPrefix)) { + return; + } // Output metric type _ss << "# TYPE " << metric_name << " " << collector->type() << "\n"; switch (collector->type()) { @@ -290,6 +294,9 @@ void JsonMetricsVisitor::visit(const std::string& prefix, const std::string& nam if (collector->empty() || name.empty()) { return; } + if (!config::enable_collect_table_metrics && name.starts_with(TableMetricsPrefix)) { + return; + } rapidjson::Document::AllocatorType& allocator = doc.GetAllocator(); switch (collector->type()) { diff --git a/be/src/service/staros_worker.cpp b/be/src/service/staros_worker.cpp index 7986f247b0ae1d..8ce68aa2e45c8c 100644 --- a/be/src/service/staros_worker.cpp +++ b/be/src/service/staros_worker.cpp @@ -81,10 +81,21 @@ StarOSWorker::StarOSWorker() : _mtx(), _shards(), _fs_cache(new_lru_cache(1024)) StarOSWorker::~StarOSWorker() = default; +static const uint64_t kUnknownTableId = UINT64_MAX; uint64_t StarOSWorker::get_table_id(const ShardInfo& shard) { const auto& properties = shard.properties; - CHECK(properties.contains("tableId")); - return std::stoull(properties.at("tableId")); + auto iter = properties.find("tableId"); + if (iter == properties.end()) { + DCHECK(false) << "tableId doesn't exist in shard properties"; + return kUnknownTableId; + } + const auto& tableId = properties.at("tableId"); + try { + return std::stoull(tableId); + } catch (const std::exception& e) { + DCHECK(false) << "failed to parse tableId: " << tableId << ", " << e.what(); + return kUnknownTableId; + } } absl::Status StarOSWorker::add_shard(const ShardInfo& shard) { @@ -334,7 +345,7 @@ absl::StatusOr> StarOSWorker::new_shar // Put the FileSysatem into LRU cache auto value = new CacheValue(fs); - handle = _fs_cache->insert(key, value, 1, cache_value_deleter); + handle = _fs_cache->insert(key, value, 1, 1, cache_value_deleter); if (handle == nullptr) { delete value; } else { diff --git a/be/src/storage/delta_writer.cpp b/be/src/storage/delta_writer.cpp index be5e55ed5c314e..ff5360f773283a 100644 --- a/be/src/storage/delta_writer.cpp +++ b/be/src/storage/delta_writer.cpp @@ -572,7 +572,8 @@ Status DeltaWriter::flush_memtable_async(bool eos) { if ((_mem_table != nullptr && _mem_table->get_result_chunk() != nullptr) || eos) { auto replicate_token = _replicate_token.get(); return _flush_token->submit( - std::move(_mem_table), eos, [replicate_token, this](std::unique_ptr seg, bool eos) { + std::move(_mem_table), eos, + [replicate_token, this](SegmentPBPtr seg, bool eos, int64_t flush_data_size) { if (seg) { _tablet->add_in_writing_data_size(_opt.txn_id, seg->data_size()); } @@ -597,7 +598,7 @@ Status DeltaWriter::flush_memtable_async(bool eos) { } else { if (_mem_table != nullptr && _mem_table->get_result_chunk() != nullptr) { return _flush_token->submit( - std::move(_mem_table), eos, [this](std::unique_ptr seg, bool eos) { + std::move(_mem_table), eos, [this](SegmentPBPtr seg, bool eos, int64_t flush_data_size) { if (seg) { _tablet->add_in_writing_data_size(_opt.txn_id, seg->data_size()); } @@ -616,7 +617,7 @@ Status DeltaWriter::flush_memtable_async(bool eos) { } } else if (_replica_state == Peer) { if (_mem_table != nullptr && _mem_table->get_result_chunk() != nullptr) { - return _flush_token->submit(std::move(_mem_table), eos, [this](std::unique_ptr seg, bool eos) { + return _flush_token->submit(std::move(_mem_table), eos, [this](SegmentPBPtr seg, bool eos, int64_t f) { if (seg) { _tablet->add_in_writing_data_size(_opt.txn_id, seg->data_size()); } diff --git a/be/src/storage/lake/delta_writer.cpp b/be/src/storage/lake/delta_writer.cpp index 4efbc32094dfe5..c282051e94c002 100644 --- a/be/src/storage/lake/delta_writer.cpp +++ b/be/src/storage/lake/delta_writer.cpp @@ -55,13 +55,15 @@ class TabletWriterSink : public MemTableSink { DISALLOW_COPY_AND_MOVE(TabletWriterSink); - Status flush_chunk(const Chunk& chunk, starrocks::SegmentPB* segment = nullptr, bool eos = false) override { + Status flush_chunk(const Chunk& chunk, starrocks::SegmentPB* segment = nullptr, bool eos = false, + int64_t* flush_data_size = nullptr) override { RETURN_IF_ERROR(_writer->write(chunk, segment)); return _writer->flush(segment); } Status flush_chunk_with_deletes(const Chunk& upserts, const Column& deletes, - starrocks::SegmentPB* segment = nullptr, bool eos = false) override { + starrocks::SegmentPB* segment = nullptr, bool eos = false, + int64_t* flush_data_size = nullptr) override { RETURN_IF_ERROR(_writer->flush_del_file(deletes)); RETURN_IF_ERROR(_writer->write(upserts, segment)); return _writer->flush(segment); @@ -275,7 +277,7 @@ Status DeltaWriterImpl::build_schema_and_writer() { if (config::enable_load_spill && !(_tablet_schema->keys_type() == KeysType::PRIMARY_KEYS && (!_merge_condition.empty() || is_partial_update() || _tablet_schema->has_separate_sort_key()))) { - if (_load_spill_block_mgr == nullptr) { + if (_load_spill_block_mgr == nullptr || !_load_spill_block_mgr->is_initialized()) { _load_spill_block_mgr = std::make_unique(UniqueId(_load_id).to_thrift(), _tablet_id, _txn_id, _tablet_manager->tablet_root_location(_tablet_id)); @@ -318,21 +320,25 @@ inline Status DeltaWriterImpl::flush_async() { if (_miss_auto_increment_column && _mem_table->get_result_chunk() != nullptr) { RETURN_IF_ERROR(fill_auto_increment_id(*_mem_table->get_result_chunk())); } - st = _flush_token->submit(std::move(_mem_table), _eos, [this](std::unique_ptr seg, bool eos) { - if (_immutable_tablet_size > 0 && !_is_immutable.load(std::memory_order_relaxed)) { - if (seg) { - _tablet_manager->add_in_writing_data_size(_tablet_id, seg->data_size()); - } - if (_tablet_manager->in_writing_data_size(_tablet_id) > _immutable_tablet_size) { - _is_immutable.store(true, std::memory_order_relaxed); - } - VLOG(2) << "flush memtable, tablet=" << _tablet_id << ", txn=" << _txn_id - << " _immutable_tablet_size=" << _immutable_tablet_size - << ", segment_size=" << (seg ? seg->data_size() : 0) - << ", in_writing_data_size=" << _tablet_manager->in_writing_data_size(_tablet_id) - << ", is_immutable=" << _is_immutable.load(std::memory_order_relaxed); - } - }); + st = _flush_token->submit( + std::move(_mem_table), _eos, [this](SegmentPBPtr seg, bool eos, int64_t flush_data_size) { + if (_immutable_tablet_size > 0 && !_is_immutable.load(std::memory_order_relaxed)) { + if (seg) { + _tablet_manager->add_in_writing_data_size(_tablet_id, seg->data_size()); + } else if (flush_data_size > 0) { + // When enable load spill, seg is nullptr, so we need to use flush_data_size + _tablet_manager->add_in_writing_data_size(_tablet_id, flush_data_size); + } + if (_tablet_manager->in_writing_data_size(_tablet_id) > _immutable_tablet_size) { + _is_immutable.store(true, std::memory_order_relaxed); + } + VLOG(2) << "flush memtable, tablet=" << _tablet_id << ", txn=" << _txn_id + << " _immutable_tablet_size=" << _immutable_tablet_size + << ", flush data size=" << (seg ? seg->data_size() : flush_data_size) + << ", in_writing_data_size=" << _tablet_manager->in_writing_data_size(_tablet_id) + << ", is_immutable=" << _is_immutable.load(std::memory_order_relaxed); + } + }); _mem_table.reset(nullptr); _last_write_ts = 0; } diff --git a/be/src/storage/lake/load_spill_block_manager.cpp b/be/src/storage/lake/load_spill_block_manager.cpp index 0ebebc943df0aa..c0dfe5aea9143f 100644 --- a/be/src/storage/lake/load_spill_block_manager.cpp +++ b/be/src/storage/lake/load_spill_block_manager.cpp @@ -100,6 +100,7 @@ Status LoadSpillBlockManager::init() { std::move(remote_block_manager)); // init block container _block_container = std::make_unique(); + _initialized = true; return Status::OK(); } diff --git a/be/src/storage/lake/load_spill_block_manager.h b/be/src/storage/lake/load_spill_block_manager.h index 292027b4462d6c..49aa7323aedce0 100644 --- a/be/src/storage/lake/load_spill_block_manager.h +++ b/be/src/storage/lake/load_spill_block_manager.h @@ -70,6 +70,8 @@ class LoadSpillBlockManager { // Initializes the LoadSpillBlockManager. Status init(); + bool is_initialized() const { return _initialized; } + // acquire Block from BlockManager StatusOr acquire_block(size_t block_size); // return Block to BlockManager @@ -88,6 +90,7 @@ class LoadSpillBlockManager { std::unique_ptr _remote_dir_manager; // Manager for remote directories. std::unique_ptr _block_manager; // Manager for blocks. std::unique_ptr _block_container; // Container for blocks. + bool _initialized = false; // Whether the manager is initialized. }; } // namespace lake diff --git a/be/src/storage/lake/metacache.cpp b/be/src/storage/lake/metacache.cpp index bd2b829166e170..a89fe398411d72 100644 --- a/be/src/storage/lake/metacache.cpp +++ b/be/src/storage/lake/metacache.cpp @@ -89,7 +89,7 @@ Metacache::Metacache(int64_t cache_capacity) : _cache(new_lru_cache(cache_capaci 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::Handle* handle = _cache->insert(CacheKey(key), ptr, size, size, cache_value_deleter); _cache->release(handle); } diff --git a/be/src/storage/lake/spill_mem_table_sink.cpp b/be/src/storage/lake/spill_mem_table_sink.cpp index fbb859036bcffa..b4d9ae72681184 100644 --- a/be/src/storage/lake/spill_mem_table_sink.cpp +++ b/be/src/storage/lake/spill_mem_table_sink.cpp @@ -36,6 +36,7 @@ Status LoadSpillOutputDataStream::append(RuntimeState* state, const std::vector< // preallocate block RETURN_IF_ERROR(_preallocate(total_size)); // append data + _append_bytes += total_size; return _block->append(data); } @@ -123,7 +124,8 @@ Status SpillMemTableSink::_do_spill(const Chunk& chunk, const spill::SpillOutput return Status::OK(); } -Status SpillMemTableSink::flush_chunk(const Chunk& chunk, starrocks::SegmentPB* segment, bool eos) { +Status SpillMemTableSink::flush_chunk(const Chunk& chunk, starrocks::SegmentPB* segment, bool eos, + int64_t* flush_data_size) { if (eos && _block_manager->block_container()->empty()) { // If there is only one flush, flush it to segment directly RETURN_IF_ERROR(_writer->write(chunk, segment)); @@ -137,11 +139,15 @@ Status SpillMemTableSink::flush_chunk(const Chunk& chunk, starrocks::SegmentPB* RETURN_IF_ERROR(_do_spill(chunk, output)); // 3. flush RETURN_IF_ERROR(output->flush()); + // record append bytes to `flush_data_size` + if (flush_data_size != nullptr) { + *flush_data_size = output->append_bytes(); + } return Status::OK(); } Status SpillMemTableSink::flush_chunk_with_deletes(const Chunk& upserts, const Column& deletes, - starrocks::SegmentPB* segment, bool eos) { + starrocks::SegmentPB* segment, bool eos, int64_t* flush_data_size) { if (eos && _block_manager->block_container()->empty()) { // If there is only one flush, flush it to segment directly RETURN_IF_ERROR(_writer->flush_del_file(deletes)); @@ -149,7 +155,7 @@ Status SpillMemTableSink::flush_chunk_with_deletes(const Chunk& upserts, const C return _writer->flush(segment); } // 1. flush upsert - RETURN_IF_ERROR(flush_chunk(upserts, segment, eos)); + RETURN_IF_ERROR(flush_chunk(upserts, segment, eos, flush_data_size)); // 2. flush deletes RETURN_IF_ERROR(_writer->flush_del_file(deletes)); return Status::OK(); diff --git a/be/src/storage/lake/spill_mem_table_sink.h b/be/src/storage/lake/spill_mem_table_sink.h index 45dbde158b8c47..64feafd6f0d19d 100644 --- a/be/src/storage/lake/spill_mem_table_sink.h +++ b/be/src/storage/lake/spill_mem_table_sink.h @@ -40,6 +40,8 @@ class LoadSpillOutputDataStream : public spill::SpillOutputDataStream { bool is_remote() const override; + int64_t append_bytes() const { return _append_bytes; } + private: Status _preallocate(size_t block_size); @@ -49,6 +51,7 @@ class LoadSpillOutputDataStream : public spill::SpillOutputDataStream { private: LoadSpillBlockManager* _block_manager = nullptr; spill::BlockPtr _block; + int64_t _append_bytes = 0; }; class SpillMemTableSink : public MemTableSink { @@ -56,10 +59,12 @@ class SpillMemTableSink : public MemTableSink { SpillMemTableSink(LoadSpillBlockManager* block_manager, TabletWriter* writer, RuntimeProfile* profile); ~SpillMemTableSink() override = default; - Status flush_chunk(const Chunk& chunk, starrocks::SegmentPB* segment = nullptr, bool eos = false) override; + Status flush_chunk(const Chunk& chunk, starrocks::SegmentPB* segment = nullptr, bool eos = false, + int64_t* flush_data_size = nullptr) override; Status flush_chunk_with_deletes(const Chunk& upserts, const Column& deletes, - starrocks::SegmentPB* segment = nullptr, bool eos = false) override; + starrocks::SegmentPB* segment = nullptr, bool eos = false, + int64_t* flush_data_size = nullptr) override; Status merge_blocks_to_segments(); diff --git a/be/src/storage/memtable.cpp b/be/src/storage/memtable.cpp index b78a794031ad09..de657d61a48007 100644 --- a/be/src/storage/memtable.cpp +++ b/be/src/storage/memtable.cpp @@ -326,7 +326,7 @@ Status MemTable::finalize() { return Status::OK(); } -Status MemTable::flush(SegmentPB* seg_info, bool eos) { +Status MemTable::flush(SegmentPB* seg_info, bool eos, int64_t* flush_data_size) { if (UNLIKELY(_result_chunk == nullptr)) { return Status::OK(); } @@ -339,9 +339,9 @@ Status MemTable::flush(SegmentPB* seg_info, bool eos) { { SCOPED_RAW_TIMER(&duration_ns); if (_deletes) { - RETURN_IF_ERROR(_sink->flush_chunk_with_deletes(*_result_chunk, *_deletes, seg_info, eos)); + RETURN_IF_ERROR(_sink->flush_chunk_with_deletes(*_result_chunk, *_deletes, seg_info, eos, flush_data_size)); } else { - RETURN_IF_ERROR(_sink->flush_chunk(*_result_chunk, seg_info, eos)); + RETURN_IF_ERROR(_sink->flush_chunk(*_result_chunk, seg_info, eos, flush_data_size)); } } auto io_stat = scope.current_scoped_tls_io(); diff --git a/be/src/storage/memtable.h b/be/src/storage/memtable.h index 2e6f37c1ff6a05..10f504b0364b31 100644 --- a/be/src/storage/memtable.h +++ b/be/src/storage/memtable.h @@ -97,7 +97,7 @@ class MemTable { // return true suggests caller should flush this memory table StatusOr insert(const Chunk& chunk, const uint32_t* indexes, uint32_t from, uint32_t size); - Status flush(SegmentPB* seg_info = nullptr, bool eos = false); + Status flush(SegmentPB* seg_info = nullptr, bool eos = false, int64_t* flush_data_size = nullptr); Status finalize(); diff --git a/be/src/storage/memtable_flush_executor.cpp b/be/src/storage/memtable_flush_executor.cpp index 040b29c45d98f0..67ac6d84cb35c8 100644 --- a/be/src/storage/memtable_flush_executor.cpp +++ b/be/src/storage/memtable_flush_executor.cpp @@ -44,7 +44,7 @@ namespace starrocks { class MemtableFlushTask final : public Runnable { public: MemtableFlushTask(FlushToken* flush_token, std::unique_ptr memtable, bool eos, - std::function, bool)> cb) + std::function, bool, int64_t)> cb) : _flush_token(flush_token), _memtable(std::move(memtable)), _eos(eos), @@ -57,12 +57,13 @@ class MemtableFlushTask final : public Runnable { _flush_token->_stats.queueing_memtable_num--; _flush_token->_stats.pending_time_ns += MonotonicNanos() - _create_time_ns; std::unique_ptr segment = nullptr; + int64_t flush_data_size = 0; if (_memtable) { SCOPED_THREAD_LOCAL_MEM_SETTER(_memtable->mem_tracker(), false); segment = std::make_unique(); _flush_token->_stats.cur_flush_count++; - _flush_token->_flush_memtable(_memtable.get(), segment.get(), _eos); + _flush_token->_flush_memtable(_memtable.get(), segment.get(), _eos, &flush_data_size); _flush_token->_stats.cur_flush_count--; _memtable.reset(); @@ -78,7 +79,7 @@ class MemtableFlushTask final : public Runnable { } if (_cb) { - _cb(std::move(segment), _eos); + _cb(std::move(segment), _eos, flush_data_size); } } @@ -86,7 +87,7 @@ class MemtableFlushTask final : public Runnable { FlushToken* _flush_token; std::unique_ptr _memtable; bool _eos; - std::function, bool)> _cb; + std::function, bool, int64_t)> _cb; int64_t _create_time_ns; }; @@ -97,7 +98,7 @@ std::ostream& operator<<(std::ostream& os, const FlushStatistic& stat) { } Status FlushToken::submit(std::unique_ptr memtable, bool eos, - std::function, bool)> cb) { + std::function, bool, int64_t)> cb) { RETURN_IF_ERROR(status()); if (memtable == nullptr && !eos) { return Status::InternalError(fmt::format("memtable=null eos=false")); @@ -130,13 +131,13 @@ Status FlushToken::wait() { return _status; } -void FlushToken::_flush_memtable(MemTable* memtable, SegmentPB* segment, bool eos) { +void FlushToken::_flush_memtable(MemTable* memtable, SegmentPB* segment, bool eos, int64_t* flush_data_size) { // If previous flush has failed, return directly if (!status().ok()) { return; } - set_status(memtable->flush(segment, eos)); + set_status(memtable->flush(segment, eos, flush_data_size)); _stats.flush_count++; _stats.memtable_stats += memtable->get_stat(); } diff --git a/be/src/storage/memtable_flush_executor.h b/be/src/storage/memtable_flush_executor.h index f633f1bec67803..3c8caf6c2a4cf6 100644 --- a/be/src/storage/memtable_flush_executor.h +++ b/be/src/storage/memtable_flush_executor.h @@ -62,6 +62,7 @@ struct FlushStatistic { }; std::ostream& operator<<(std::ostream& os, const FlushStatistic& stat); +using SegmentPBPtr = std::unique_ptr; // A thin wrapper of ThreadPoolToken to submit task. // For a tablet, there may be multiple memtables, which will be flushed to disk @@ -76,7 +77,7 @@ class FlushToken { : _flush_token(std::move(flush_pool_token)), _status() {} Status submit(std::unique_ptr mem_table, bool eos = false, - std::function, bool)> cb = nullptr); + std::function cb = nullptr); // error has happpens, so we cancel this token // And remove all tasks in the queue. @@ -109,7 +110,7 @@ class FlushToken { private: friend class MemtableFlushTask; - void _flush_memtable(MemTable* memtable, SegmentPB* segment, bool eos); + void _flush_memtable(MemTable* memtable, SegmentPB* segment, bool eos, int64_t* flush_data_size); std::unique_ptr _flush_token; diff --git a/be/src/storage/memtable_rowset_writer_sink.h b/be/src/storage/memtable_rowset_writer_sink.h index 740e63885cfdb7..d30a671bfeb5f8 100644 --- a/be/src/storage/memtable_rowset_writer_sink.h +++ b/be/src/storage/memtable_rowset_writer_sink.h @@ -28,12 +28,13 @@ class MemTableRowsetWriterSink : public MemTableSink { DISALLOW_COPY(MemTableRowsetWriterSink); - Status flush_chunk(const Chunk& chunk, SegmentPB* seg_info = nullptr, bool eos = false) override { + Status flush_chunk(const Chunk& chunk, SegmentPB* seg_info = nullptr, bool eos = false, + int64_t* flush_data_size = nullptr) override { return _rowset_writer->flush_chunk(chunk, seg_info); } Status flush_chunk_with_deletes(const Chunk& upserts, const Column& deletes, SegmentPB* seg_info = nullptr, - bool eos = false) override { + bool eos = false, int64_t* flush_data_size = nullptr) override { return _rowset_writer->flush_chunk_with_deletes(upserts, deletes, seg_info); } diff --git a/be/src/storage/memtable_sink.h b/be/src/storage/memtable_sink.h index df23a1041d8e36..542b704660d59c 100644 --- a/be/src/storage/memtable_sink.h +++ b/be/src/storage/memtable_sink.h @@ -29,9 +29,10 @@ class MemTableSink { public: virtual ~MemTableSink() = default; - virtual Status flush_chunk(const Chunk& chunk, starrocks::SegmentPB* seg_info = nullptr, bool eos = false) = 0; + virtual Status flush_chunk(const Chunk& chunk, starrocks::SegmentPB* seg_info = nullptr, bool eos = false, + int64_t* flush_data_size = nullptr) = 0; virtual Status flush_chunk_with_deletes(const Chunk& upserts, const Column& deletes, SegmentPB* seg_info = nullptr, - bool eos = false) = 0; + bool eos = false, int64_t* flush_data_size = nullptr) = 0; }; } // namespace starrocks diff --git a/be/src/storage/page_cache.cpp b/be/src/storage/page_cache.cpp index ed74e3318140a2..d7a6ed9b765793 100644 --- a/be/src/storage/page_cache.cpp +++ b/be/src/storage/page_cache.cpp @@ -87,14 +87,12 @@ void StoragePageCache::prune() { } StoragePageCache::StoragePageCache(MemTracker* mem_tracker, size_t capacity) - : _mem_tracker(mem_tracker), _cache(new_lru_cache(capacity, ChargeMode::MEMSIZE)) {} + : _mem_tracker(mem_tracker), _cache(new_lru_cache(capacity)) {} StoragePageCache::~StoragePageCache() = default; void StoragePageCache::set_capacity(size_t capacity) { -#ifndef BE_TEST SCOPED_THREAD_LOCAL_MEM_TRACKER_SETTER(_mem_tracker); -#endif _cache->set_capacity(capacity); } @@ -111,9 +109,7 @@ uint64_t StoragePageCache::get_hit_count() { } bool StoragePageCache::adjust_capacity(int64_t delta, size_t min_capacity) { -#ifndef BE_TEST SCOPED_THREAD_LOCAL_MEM_TRACKER_SETTER(_mem_tracker); -#endif return _cache->adjust_capacity(delta, min_capacity); } @@ -127,13 +123,13 @@ bool StoragePageCache::lookup(const CacheKey& key, PageCacheHandle* handle) { } void StoragePageCache::insert(const CacheKey& key, const Slice& data, PageCacheHandle* handle, bool in_memory) { - // mem size should equals to data size when running UT - int64_t mem_size = data.size; #ifndef BE_TEST - mem_size = malloc_usable_size(data.data); + int64_t mem_size = malloc_usable_size(data.data); tls_thread_status.mem_release(mem_size); SCOPED_THREAD_LOCAL_MEM_TRACKER_SETTER(_mem_tracker); tls_thread_status.mem_consume(mem_size); +#else + int64_t mem_size = data.size; #endif auto deleter = [](const starrocks::CacheKey& key, void* value) { delete[](uint8_t*) value; }; @@ -144,7 +140,7 @@ void StoragePageCache::insert(const CacheKey& key, const Slice& data, PageCacheH } // Use mem size managed by memory allocator as this record charge size. At the same time, we should record this record size // for data fetching when lookup. - auto* lru_handle = _cache->insert(key.encode(), data.data, mem_size, deleter, priority, data.size); + auto* lru_handle = _cache->insert(key.encode(), data.data, data.size, mem_size, deleter, priority); *handle = PageCacheHandle(_cache.get(), lru_handle); } diff --git a/be/src/storage/page_cache.h b/be/src/storage/page_cache.h index cbc34f9388f3d2..8239440d8ce3e5 100644 --- a/be/src/storage/page_cache.h +++ b/be/src/storage/page_cache.h @@ -134,11 +134,7 @@ class PageCacheHandle { PageCacheHandle(Cache* cache, Cache::Handle* handle) : _cache(cache), _handle(handle) {} ~PageCacheHandle() { if (_handle != nullptr) { -#ifndef BE_TEST - MemTracker* prev_tracker = - tls_thread_status.set_mem_tracker(GlobalEnv::GetInstance()->page_cache_mem_tracker()); - DeferOp op([&] { tls_thread_status.set_mem_tracker(prev_tracker); }); -#endif + SCOPED_THREAD_LOCAL_MEM_TRACKER_SETTER(GlobalEnv::GetInstance()->page_cache_mem_tracker()); _cache->release(_handle); } } diff --git a/be/src/storage/rowset/metadata_cache.cpp b/be/src/storage/rowset/metadata_cache.cpp index dad990b223e4dc..589e05837261cb 100644 --- a/be/src/storage/rowset/metadata_cache.cpp +++ b/be/src/storage/rowset/metadata_cache.cpp @@ -53,7 +53,7 @@ void MetadataCache::set_capacity(size_t capacity) { } void MetadataCache::_insert(const std::string& key, std::weak_ptr* ptr, size_t size) { - Cache::Handle* handle = _cache->insert(CacheKey(key), ptr, size, _cache_value_deleter); + Cache::Handle* handle = _cache->insert(CacheKey(key), ptr, size, size, _cache_value_deleter); _cache->release(handle); } diff --git a/be/src/storage/sstable/table.cpp b/be/src/storage/sstable/table.cpp index 9e120d0fdb4a80..8926346a53c1a3 100644 --- a/be/src/storage/sstable/table.cpp +++ b/be/src/storage/sstable/table.cpp @@ -194,7 +194,8 @@ Iterator* Table::BlockReader(void* arg, const ReadOptions& options, const Slice& if (s.ok()) { block = new Block(contents); if (contents.cachable && options.fill_cache) { - cache_handle = block_cache->insert(key, block, block->size(), &DeleteCachedBlock); + size_t block_size = block->size(); + cache_handle = block_cache->insert(key, block, block_size, block_size, &DeleteCachedBlock); } } if (options.stat != nullptr) { diff --git a/be/src/util/lru_cache.cpp b/be/src/util/lru_cache.cpp index f277faefde38c5..17bcc61c5965a8 100644 --- a/be/src/util/lru_cache.cpp +++ b/be/src/util/lru_cache.cpp @@ -294,9 +294,8 @@ void LRUCache::_evict_one_entry(LRUHandle* e) { _usage -= e->charge; } -Cache::Handle* LRUCache::insert(const CacheKey& key, uint32_t hash, void* value, size_t charge, - void (*deleter)(const CacheKey& key, void* value), CachePriority priority, - size_t value_size) { +Cache::Handle* LRUCache::insert(const CacheKey& key, uint32_t hash, void* value, size_t value_size, size_t charge, + void (*deleter)(const CacheKey& key, void* value), CachePriority priority) { auto* e = reinterpret_cast(malloc(sizeof(LRUHandle) - 1 + key.size())); e->value = value; e->deleter = deleter; @@ -397,8 +396,7 @@ uint32_t ShardedLRUCache::_shard(uint32_t hash) { return hash >> (32 - kNumShardBits); } -ShardedLRUCache::ShardedLRUCache(size_t capacity, ChargeMode charge_mode) - : _last_id(0), _capacity(capacity), _charge_mode(charge_mode) { +ShardedLRUCache::ShardedLRUCache(size_t capacity) : _last_id(0), _capacity(capacity) { const size_t per_shard = (_capacity + (kNumShards - 1)) / kNumShards; for (auto& _shard : _shards) { _shard.set_capacity(per_shard); @@ -429,11 +427,10 @@ bool ShardedLRUCache::adjust_capacity(int64_t delta, size_t min_capacity) { return true; } -Cache::Handle* ShardedLRUCache::insert(const CacheKey& key, void* value, size_t charge, - void (*deleter)(const CacheKey& key, void* value), CachePriority priority, - size_t value_size) { +Cache::Handle* ShardedLRUCache::insert(const CacheKey& key, void* value, size_t value_size, size_t charge, + void (*deleter)(const CacheKey& key, void* value), CachePriority priority) { const uint32_t hash = _hash_slice(key); - return _shards[_shard(hash)].insert(key, hash, value, charge, deleter, priority, value_size); + return _shards[_shard(hash)].insert(key, hash, value, value_size, charge, deleter, priority); } Cache::Handle* ShardedLRUCache::lookup(const CacheKey& key) { @@ -457,8 +454,7 @@ void* ShardedLRUCache::value(Handle* handle) { Slice ShardedLRUCache::value_slice(Handle* handle) { auto lru_handle = reinterpret_cast(handle); - size_t record_size = _charge_mode == ChargeMode::VALUESIZE ? lru_handle->charge : lru_handle->value_size; - return {(char*)lru_handle->value, record_size}; + return {(char*)lru_handle->value, lru_handle->value_size}; } uint64_t ShardedLRUCache::new_id() { @@ -531,8 +527,8 @@ void ShardedLRUCache::get_cache_status(rapidjson::Document* document) { } } -Cache* new_lru_cache(size_t capacity, ChargeMode charge_mode) { - return new ShardedLRUCache(capacity, charge_mode); +Cache* new_lru_cache(size_t capacity) { + return new ShardedLRUCache(capacity); } } // namespace starrocks diff --git a/be/src/util/lru_cache.h b/be/src/util/lru_cache.h index 966f74ffe2fad8..f4a5571fbcdcd0 100644 --- a/be/src/util/lru_cache.h +++ b/be/src/util/lru_cache.h @@ -20,16 +20,9 @@ namespace starrocks { class Cache; class CacheKey; -enum class ChargeMode { - // use value size as charge - VALUESIZE = 0, - // use allocator tracking size as charge - MEMSIZE = 1 -}; - // Create a new cache with a fixed size capacity. This implementation // of Cache uses a least-recently-used eviction policy. -extern Cache* new_lru_cache(size_t capacity, ChargeMode charge_mode = ChargeMode::VALUESIZE); +extern Cache* new_lru_cache(size_t capacity); class CacheKey { public: @@ -139,9 +132,9 @@ class Cache { // // When the inserted entry is no longer needed, the key and // value will be passed to "deleter". - virtual Handle* insert(const CacheKey& key, void* value, size_t charge, + virtual Handle* insert(const CacheKey& key, void* value, size_t value_size, size_t charge, void (*deleter)(const CacheKey& key, void* value), - CachePriority priority = CachePriority::NORMAL, size_t value_size = 0) = 0; + CachePriority priority = CachePriority::NORMAL) = 0; // If the cache has no mapping for "key", returns NULL. // @@ -275,9 +268,9 @@ class LRUCache { void set_capacity(size_t capacity); // Like Cache methods, but with an extra "hash" parameter. - Cache::Handle* insert(const CacheKey& key, uint32_t hash, void* value, size_t charge, + Cache::Handle* insert(const CacheKey& key, uint32_t hash, void* value, size_t value_size, size_t charge, void (*deleter)(const CacheKey& key, void* value), - CachePriority priority = CachePriority::NORMAL, size_t value_size = 0); + CachePriority priority = CachePriority::NORMAL); Cache::Handle* lookup(const CacheKey& key, uint32_t hash); void release(Cache::Handle* handle); void erase(const CacheKey& key, uint32_t hash); @@ -318,10 +311,11 @@ static const int kNumShards = 1 << kNumShardBits; class ShardedLRUCache : public Cache { public: - explicit ShardedLRUCache(size_t capacity, ChargeMode charge_mode = ChargeMode::VALUESIZE); + explicit ShardedLRUCache(size_t capacity); ~ShardedLRUCache() override = default; - Handle* insert(const CacheKey& key, void* value, size_t charge, void (*deleter)(const CacheKey& key, void* value), - CachePriority priority = CachePriority::NORMAL, size_t value_size = 0) override; + Handle* insert(const CacheKey& key, void* value, size_t value_size, size_t charge, + void (*deleter)(const CacheKey& key, void* value), + CachePriority priority = CachePriority::NORMAL) override; Handle* lookup(const CacheKey& key) override; void release(Handle* handle) override; void erase(const CacheKey& key) override; @@ -347,7 +341,6 @@ class ShardedLRUCache : public Cache { std::mutex _mutex; uint64_t _last_id; size_t _capacity; - ChargeMode _charge_mode; }; } // namespace starrocks diff --git a/be/src/util/table_metrics.h b/be/src/util/table_metrics.h index 989effa827df29..fddec4c9ece2c0 100644 --- a/be/src/util/table_metrics.h +++ b/be/src/util/table_metrics.h @@ -69,8 +69,11 @@ class TableMetricsManager { TableMetricsPtr get_table_metrics(uint64_t table_id) { std::shared_lock l(_mu); - DCHECK(_metrics_map.contains(table_id)); - return _metrics_map.at(table_id); + auto iter = _metrics_map.find(table_id); + if (iter != _metrics_map.end()) { + return iter->second; + } + return _blackhole_metrics; } void cleanup(); @@ -79,6 +82,11 @@ class TableMetricsManager { MetricRegistry* _metrics; std::shared_mutex _mu; phmap::flat_hash_map _metrics_map; + // In some cases, we may not be able to obtain the metrics for the corresponding table id, + // For example, when drop tablet and data load concurrently, + // the Tablets may have been deleted before the load begins, and the table metrics may be cleared. + // In such a scenario, we return blackhole metrics to ensure that subsequent processes can work well. + TableMetricsPtr _blackhole_metrics = std::make_shared(); // used for cleanup int64_t _last_cleanup_ts = 0; diff --git a/be/test/CMakeLists.txt b/be/test/CMakeLists.txt index b1d35aec8de8be..e4a912a3871426 100644 --- a/be/test/CMakeLists.txt +++ b/be/test/CMakeLists.txt @@ -203,7 +203,6 @@ set(EXEC_FILES ./formats/parquet/parquet_cli_reader_test.cpp ./formats/parquet/parquet_ut_base.cpp ./formats/parquet/page_index_test.cpp - ./formats/parquet/statistics_helper_test.cpp ./formats/disk_range_test.cpp ./geo/geo_types_test.cpp ./geo/wkt_parse_test.cpp diff --git a/be/test/exprs/jit_func_cache_test.cpp b/be/test/exprs/jit_func_cache_test.cpp index d59d2e3ba8e6b0..261d2293feeac3 100644 --- a/be/test/exprs/jit_func_cache_test.cpp +++ b/be/test/exprs/jit_func_cache_test.cpp @@ -39,8 +39,8 @@ class JITFunctionCacheTest : public ::testing::Test { } void mock_insert(string expr_name) { - auto* handle = - engine->get_func_cache()->insert(expr_name, nullptr, 1000, [](const CacheKey& key, void* value) {}); + auto* handle = engine->get_func_cache()->insert(expr_name, nullptr, 1000, 1000, + [](const CacheKey& key, void* value) {}); if (handle != nullptr) { engine->get_func_cache()->release(handle); } diff --git a/be/test/formats/parquet/statistics_helper_test.cpp b/be/test/formats/parquet/statistics_helper_test.cpp deleted file mode 100644 index efa88e19e90e4f..00000000000000 --- a/be/test/formats/parquet/statistics_helper_test.cpp +++ /dev/null @@ -1,185 +0,0 @@ -// 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 "formats/parquet/statistics_helper.h" - -#include - -#include "formats/parquet/parquet_ut_base.h" -#include "formats/parquet/schema.h" -#include "testutil/assert.h" - -namespace starrocks::parquet { - -class StatisticsHelperTest : public testing::Test { -public: - void SetUp() override { _runtime_state = _pool.add(new RuntimeState(TQueryGlobals())); } - void TearDown() override {} - - template - std::string int_to_string(T value) { - char chararray[sizeof(T)]; - memcpy(chararray, &value, sizeof(T)); - return std::string(chararray, sizeof(T)); - } - -protected: - RuntimeState* _runtime_state = nullptr; - ObjectPool _pool; -}; - -TEST_F(StatisticsHelperTest, TestInFilterInt) { - std::string timezone = "Asia/Shanghai"; - ParquetField field; - field.physical_type = tparquet::Type::type::INT32; - std::vector min_values; - min_values.emplace_back(int_to_string(1)); - min_values.emplace_back(int_to_string(4)); - std::vector max_values; - max_values.emplace_back(int_to_string(4)); - max_values.emplace_back(int_to_string(6)); - - std::set in_oprands{2, 3, 7}; - std::vector t_conjuncts; - ParquetUTBase::create_in_predicate_int_conjunct_ctxs(TExprOpcode::FILTER_IN, 0, in_oprands, &t_conjuncts); - EXPECT_EQ(t_conjuncts.size(), 1); - std::vector ctxs; - ParquetUTBase::create_conjunct_ctxs(&_pool, _runtime_state, &t_conjuncts, &ctxs); - EXPECT_EQ(ctxs.size(), 1); - - std::vector null_counts{0, 0}; - std::vector null_pages{false, false}; - Filter selected(min_values.size(), true); - auto st = StatisticsHelper::in_filter_on_min_max_stat(min_values, max_values, null_pages, null_counts, ctxs[0], - &field, timezone, selected); - ASSERT_OK(st); - ASSERT_TRUE(selected[0]); - ASSERT_FALSE(selected[1]); -} - -TEST_F(StatisticsHelperTest, TestInFilterString) { - std::string timezone = "Asia/Shanghai"; - ParquetField field; - field.physical_type = tparquet::Type::type::BYTE_ARRAY; - std::vector min_values{"abc"}; - std::vector max_values{"def"}; - - { - std::set in_oprands{"a", "ab"}; - std::vector t_conjuncts; - ParquetUTBase::create_in_predicate_string_conjunct_ctxs(TExprOpcode::FILTER_IN, 0, in_oprands, &t_conjuncts); - EXPECT_EQ(t_conjuncts.size(), 1); - std::vector ctxs; - ParquetUTBase::create_conjunct_ctxs(&_pool, _runtime_state, &t_conjuncts, &ctxs); - EXPECT_EQ(ctxs.size(), 1); - - std::vector null_counts{0}; - std::vector null_pages{false}; - Filter selected(min_values.size(), true); - auto st = StatisticsHelper::in_filter_on_min_max_stat(min_values, max_values, null_pages, null_counts, ctxs[0], - &field, timezone, selected); - ASSERT_OK(st); - ASSERT_FALSE(selected[0]); - } - - { - std::set in_oprands{"ac", "de"}; - std::vector t_conjuncts; - ParquetUTBase::create_in_predicate_string_conjunct_ctxs(TExprOpcode::FILTER_IN, 0, in_oprands, &t_conjuncts); - EXPECT_EQ(t_conjuncts.size(), 1); - std::vector ctxs; - ParquetUTBase::create_conjunct_ctxs(&_pool, _runtime_state, &t_conjuncts, &ctxs); - EXPECT_EQ(ctxs.size(), 1); - - std::vector null_counts{0}; - std::vector null_pages{false}; - Filter selected(min_values.size(), true); - auto st = StatisticsHelper::in_filter_on_min_max_stat(min_values, max_values, null_pages, null_counts, ctxs[0], - &field, timezone, selected); - ASSERT_OK(st); - ASSERT_TRUE(selected[0]); - } -} - -TEST_F(StatisticsHelperTest, TestInFilterDate) { - std::string timezone = "Asia/Shanghai"; - ParquetField field; - field.physical_type = tparquet::Type::type::INT32; - std::vector min_values; - // 2020-01-01, 2021-01-01 - min_values.emplace_back(int_to_string(18262)); - min_values.emplace_back(int_to_string(18628)); - std::vector max_values; - // 2020-12-31, 2021-12-31 - max_values.emplace_back(int_to_string(18627)); - max_values.emplace_back(int_to_string(18992)); - - std::set in_oprands{"2020-01-01", "2022-01-01"}; - std::vector t_conjuncts; - ParquetUTBase::create_in_predicate_date_conjunct_ctxs(TExprOpcode::FILTER_IN, 0, TPrimitiveType::DATE, in_oprands, - &t_conjuncts); - EXPECT_EQ(t_conjuncts.size(), 1); - std::vector ctxs; - ParquetUTBase::create_conjunct_ctxs(&_pool, _runtime_state, &t_conjuncts, &ctxs); - EXPECT_EQ(ctxs.size(), 1); - - std::vector null_counts{0, 0}; - std::vector null_pages{false, false}; - Filter selected(min_values.size(), true); - auto st = StatisticsHelper::in_filter_on_min_max_stat(min_values, max_values, null_pages, null_counts, ctxs[0], - &field, timezone, selected); - ASSERT_OK(st); - ASSERT_TRUE(selected[0]); - ASSERT_FALSE(selected[1]); -} - -TEST_F(StatisticsHelperTest, TestInFilterDatetime) { - std::string timezone = "Asia/Shanghai"; - ParquetField field; - field.physical_type = tparquet::Type::type::INT64; - field.schema_element.type = tparquet::Type::INT64; - field.schema_element.__isset.logicalType = true; - field.schema_element.logicalType.__isset.TIMESTAMP = true; - field.schema_element.logicalType.TIMESTAMP.isAdjustedToUTC = true; - field.schema_element.logicalType.TIMESTAMP.unit.__isset.MILLIS = true; - - std::vector min_values; - // 2020-01-01 00:00:00, 2021-01-01 00:00:00 - min_values.emplace_back(int_to_string(1577808000000)); - min_values.emplace_back(int_to_string(1609430400000)); - std::vector max_values; - // 2020-12-31 23:59:59, 2021-12-31 23:59:59 - max_values.emplace_back(int_to_string(1609430399000)); - max_values.emplace_back(int_to_string(1640966399000)); - - std::set in_oprands{"2020-01-01 00:00:00", "2022-01-01 00:00:00"}; - std::vector t_conjuncts; - ParquetUTBase::create_in_predicate_date_conjunct_ctxs(TExprOpcode::FILTER_IN, 0, TPrimitiveType::DATETIME, - in_oprands, &t_conjuncts); - EXPECT_EQ(t_conjuncts.size(), 1); - std::vector ctxs; - ParquetUTBase::create_conjunct_ctxs(&_pool, _runtime_state, &t_conjuncts, &ctxs); - EXPECT_EQ(ctxs.size(), 1); - - std::vector null_counts{0, 0}; - std::vector null_pages{false, false}; - Filter selected(min_values.size(), true); - auto st = StatisticsHelper::in_filter_on_min_max_stat(min_values, max_values, null_pages, null_counts, ctxs[0], - &field, timezone, selected); - ASSERT_OK(st); - ASSERT_TRUE(selected[0]); - ASSERT_FALSE(selected[1]); -} - -} // namespace starrocks::parquet diff --git a/be/test/storage/lake/async_delta_writer_test.cpp b/be/test/storage/lake/async_delta_writer_test.cpp index adb569307ce264..deb3a97d2c32dc 100644 --- a/be/test/storage/lake/async_delta_writer_test.cpp +++ b/be/test/storage/lake/async_delta_writer_test.cpp @@ -574,9 +574,7 @@ TEST_F(LakeAsyncDeltaWriterTest, test_block_merger) { CountDownLatch latch(10); // flush multi times and generate spill blocks int64_t old_val = config::write_buffer_size; - bool old_val2 = config::enable_load_spill; config::write_buffer_size = 1; - config::enable_load_spill = true; ASSIGN_OR_ABORT(auto delta_writer, AsyncDeltaWriterBuilder() .set_tablet_manager(_tablet_mgr.get()) .set_tablet_id(tablet_id) @@ -585,6 +583,7 @@ TEST_F(LakeAsyncDeltaWriterTest, test_block_merger) { .set_mem_tracker(_mem_tracker.get()) .set_schema_id(_tablet_schema->id()) .set_profile(&_dummy_runtime_profile) + .set_immutable_tablet_size(10000000) .build()); ASSERT_OK(delta_writer->open()); for (int i = 0; i < 10; i++) { @@ -596,7 +595,6 @@ TEST_F(LakeAsyncDeltaWriterTest, test_block_merger) { } latch.wait(); config::write_buffer_size = old_val; - config::enable_load_spill = old_val2; // finish CountDownLatch latch2(1); delta_writer->finish([&](StatusOr res) { @@ -604,6 +602,7 @@ TEST_F(LakeAsyncDeltaWriterTest, test_block_merger) { latch2.count_down(); }); latch2.wait(); + ASSERT_TRUE(_tablet_mgr->in_writing_data_size(tablet_id) > 0); } } // namespace starrocks::lake diff --git a/be/test/storage/memtable_flush_executor_test.cpp b/be/test/storage/memtable_flush_executor_test.cpp index 53d806e1a9d607..ece5a05e0e357b 100644 --- a/be/test/storage/memtable_flush_executor_test.cpp +++ b/be/test/storage/memtable_flush_executor_test.cpp @@ -327,7 +327,7 @@ TEST_F(MemTableFlushExecutorTest, testMemtableFlushWithSeg) { bool ret_eos = true; ASSERT_TRUE(flush_token ->submit(std::move(mem_table), false, - [&](std::unique_ptr seg, bool eos) { + [&](std::unique_ptr seg, bool eos, int64_t flush_data_size) { ret_num_rows = seg->num_rows(); ret_eos = eos; }) @@ -359,7 +359,7 @@ TEST_F(MemTableFlushExecutorTest, testMemtableFlushWithNullSeg) { std::unique_ptr ret_seg = make_unique(); ASSERT_TRUE(flush_token ->submit(std::move(mem_table), true, - [&](std::unique_ptr seg, bool eos) { + [&](std::unique_ptr seg, bool eos, int64_t flush_data_size) { ret_seg = std::move(seg); ret_eos = eos; }) @@ -394,7 +394,7 @@ TEST_F(MemTableFlushExecutorTest, testMemtableFlushStatusNotOk) { flush_token->set_status(Status::NotSupported("Not Suppoted")); ASSERT_FALSE(flush_token->status().ok()); - flush_token->_flush_memtable(nullptr, nullptr, false); + flush_token->_flush_memtable(nullptr, nullptr, false, nullptr); ASSERT_TRUE(MemTableFlushExecutor::calc_max_threads_for_lake_table(data_dirs) > 0); } diff --git a/be/test/util/lru_cache_test.cpp b/be/test/util/lru_cache_test.cpp index 784dbba36f00c7..6e60808eabc222 100644 --- a/be/test/util/lru_cache_test.cpp +++ b/be/test/util/lru_cache_test.cpp @@ -105,12 +105,13 @@ class CacheTest : public testing::Test { void Insert(int key, int value, int charge) { std::string result; - _cache->release(_cache->insert(EncodeKey(&result, key), EncodeValue(value), charge, &CacheTest::Deleter)); + _cache->release( + _cache->insert(EncodeKey(&result, key), EncodeValue(value), charge, charge, &CacheTest::Deleter)); } void InsertDurable(int key, int value, int charge) { std::string result; - _cache->release(_cache->insert(EncodeKey(&result, key), EncodeValue(value), charge, &CacheTest::Deleter, + _cache->release(_cache->insert(EncodeKey(&result, key), EncodeValue(value), charge, charge, &CacheTest::Deleter, CachePriority::DURABLE)); } @@ -236,7 +237,7 @@ static void deleter(const CacheKey& key, void* v) { static void insert_LRUCache(LRUCache& cache, const CacheKey& key, int value, CachePriority priority) { uint32_t hash = key.hash(key.data(), key.size(), 0); - cache.release(cache.insert(key, hash, EncodeValue(value), value, &deleter, priority)); + cache.release(cache.insert(key, hash, EncodeValue(value), value, value, &deleter, priority)); } TEST_F(CacheTest, Usage) { @@ -317,7 +318,7 @@ TEST_F(CacheTest, SetCapacity) { // Insert kCacheSize entries, but not releasing. for (int i = 0; i < 32; i++) { std::string result; - handles[i] = _cache->insert(EncodeKey(&result, i), EncodeValue(1000 + kCacheSize), 1, &CacheTest::Deleter); + handles[i] = _cache->insert(EncodeKey(&result, i), EncodeValue(1000 + kCacheSize), 1, 1, &CacheTest::Deleter); } ASSERT_EQ(kCacheSize, _cache->get_capacity()); ASSERT_EQ(32, _cache->get_memory_usage()); @@ -331,7 +332,7 @@ TEST_F(CacheTest, SetCapacity) { // then release 32, usage should be 32. for (int i = 32; i < 64; i++) { std::string result; - handles[i] = _cache->insert(EncodeKey(&result, i), EncodeValue(1000 + kCacheSize), 1, &CacheTest::Deleter); + handles[i] = _cache->insert(EncodeKey(&result, i), EncodeValue(1000 + kCacheSize), 1, 1, &CacheTest::Deleter); } ASSERT_EQ(kCacheSize * 2, _cache->get_capacity()); ASSERT_EQ(64, _cache->get_memory_usage()); diff --git a/docs/en/administration/management/BE_configuration.md b/docs/en/administration/management/BE_configuration.md index db03eda7424d78..d711d9aec0da1c 100644 --- a/docs/en/administration/management/BE_configuration.md +++ b/docs/en/administration/management/BE_configuration.md @@ -3929,11 +3929,11 @@ When this value is set to less than `0`, the system uses the product of its abso ##### datacache_enable -- Default: false +- Default: true - Type: Boolean - Unit: - - Is mutable: No -- Description: Whether to enable Data Cache. `true` indicates Data Cache is enabled, and `false` indicates Data Cache is disabled. +- Description: Whether to enable Data Cache. `true` indicates Data Cache is enabled, and `false` indicates Data Cache is disabled. The default value is changed to `true` from v3.3. - Introduced in: - ##### datacache_mem_size @@ -4028,7 +4028,7 @@ When this value is set to less than `0`, the system uses the product of its abso ##### datacache_tiered_cache_enable -- Default: true +- Default: false - Type: Boolean - Unit: - - Is mutable: No diff --git a/docs/en/release_notes/release-3.2.md b/docs/en/release_notes/release-3.2.md index b17bc0f1827231..4aa62f5916da82 100644 --- a/docs/en/release_notes/release-3.2.md +++ b/docs/en/release_notes/release-3.2.md @@ -4,6 +4,30 @@ displayed_sidebar: docs # StarRocks version 3.2 +## 3.2.15 + +Release date: February 14, 2025 + +### New Features + +- Window functions support `max_by` and `min_by`. [#54961](https://github.com/StarRocks/starrocks/pull/54961) + +### Improvements + +- Added StarClient timeout parameters. [#54496](https://github.com/StarRocks/starrocks/pull/54496) + - star_client_read_timeout_seconds + - star_client_list_timeout_seconds + - star_client_write_timeout_seconds +- Tables with List partitioning strategies support partition pruning for DELETE statements. [#55400](https://github.com/StarRocks/starrocks/pull/55400) + +### Bug Fixes + +Fixed the following issues: + +- Stream Load fails when a node with an Alive status of false was scheduled. [#55371](https://github.com/StarRocks/starrocks/pull/55371) +- An error is returned during partial updates on Primary Key tables with Stream Load. [#53403](https://github.com/StarRocks/starrocks/pull/55430) +- bRPC error persists after BE node restart. [#40229](https://github.com/StarRocks/starrocks/pull/40229) + ## 3.2.14 Release date: January 8, 2025 diff --git a/docs/en/sql-reference/sql-functions/table-functions/files.md b/docs/en/sql-reference/sql-functions/table-functions/files.md index 0bbda3326a7b38..5ffd033751d183 100644 --- a/docs/en/sql-reference/sql-functions/table-functions/files.md +++ b/docs/en/sql-reference/sql-functions/table-functions/files.md @@ -522,7 +522,7 @@ unload_data_param ::= | **Key** | **Required** | **Description** | | ---------------- | ------------ | ------------------------------------------------------------ | -| compression | Yes | The compression method to use when unloading data. Valid values:
  • `uncompressed`: No compression algorithm is used.
  • `gzip`: Use the gzip compression algorithm.
  • `snappy`: Use the SNAPPY compression algorithm.
  • `zstd`: Use the Zstd compression algorithm.
  • `lz4`: Use the LZ4 compression algorithm.
| +| compression | Yes | The compression method to use when unloading data. Valid values:
  • `uncompressed`: No compression algorithm is used.
  • `gzip`: Use the gzip compression algorithm.
  • `snappy`: Use the SNAPPY compression algorithm.
  • `zstd`: Use the Zstd compression algorithm.
  • `lz4`: Use the LZ4 compression algorithm.
**NOTE**
Unloading into CSV files does not support data compression. You must set this item as `uncompressed`. | | partition_by | No | The list of columns that are used to partition data files into different storage paths. Multiple columns are separated by commas (,). FILES() extracts the key/value information of the specified columns and stores the data files under the storage paths featured with the extracted key/value pair. For further instructions, see Example 7. | | single | No | Whether to unload the data into a single file. Valid values:
  • `true`: The data is stored in a single data file.
  • `false` (Default): The data is stored in multiple files if the amount of data unloaded exceeds 512 MB.
| | target_max_file_size | No | The best-effort maximum size of each file in the batch to be unloaded. Unit: Bytes. Default value: 1073741824 (1 GB). When the size of data to be unloaded exceeds this value, the data will be divided into multiple files, and the size of each file will not significantly exceed this value. Introduced in v3.2.7. | diff --git a/docs/en/sql-reference/sql-statements/loading_unloading/routine_load/STOP_ROUTINE_LOAD.md b/docs/en/sql-reference/sql-statements/loading_unloading/routine_load/STOP_ROUTINE_LOAD.md index 1899701eea4877..e8b487c5b5c614 100644 --- a/docs/en/sql-reference/sql-statements/loading_unloading/routine_load/STOP_ROUTINE_LOAD.md +++ b/docs/en/sql-reference/sql-statements/loading_unloading/routine_load/STOP_ROUTINE_LOAD.md @@ -12,7 +12,7 @@ Stops a Routine Load job. -::: warning +:::warning - A stopped Routine Load job cannot be resumed. Therefore, please proceed with caution when executing this statement. - If you only need to pause the Routine Load job, you can execute [PAUSE ROUTINE LOAD](PAUSE_ROUTINE_LOAD.md). diff --git a/docs/en/unloading/unload_using_insert_into_files.md b/docs/en/unloading/unload_using_insert_into_files.md index 206d1e254ec67b..f36d47c1496153 100644 --- a/docs/en/unloading/unload_using_insert_into_files.md +++ b/docs/en/unloading/unload_using_insert_into_files.md @@ -55,7 +55,7 @@ For more about the remote storage systems and credential methods supported by FI INSERT INTO FILES supports unloading data into a single file or multiple files. You can further partition these data files by specifying separate storage paths for them. -When unloading data using INSERT INTO FILES, you must manually set the compression algorithm using the property `compression`. For more information on the data compression algorithm supported by StarRocks, see [Data compression](../table_design/data_compression.md). +When unloading data using INSERT INTO FILES, you must manually set the compression algorithm using the property `compression`. For more information on the data compression algorithm supported by FILES, see [unload_data_param](../sql-reference/sql-functions/table-functions/files.md#unload_data_param). ### Unload data into multiple files diff --git a/docs/zh/administration/management/BE_configuration.md b/docs/zh/administration/management/BE_configuration.md index 50d6e494f4a7b7..d0c5b0458ef5ad 100644 --- a/docs/zh/administration/management/BE_configuration.md +++ b/docs/zh/administration/management/BE_configuration.md @@ -3888,11 +3888,11 @@ curl http://:/varz ##### datacache_enable -- 默认值:false +- 默认值:true - 类型:Boolean - 单位:- - 是否动态:否 -- 描述:是否启用 Data Cache。`true` 表示启用,`false` 表示不启用。 +- 描述:是否启用 Data Cache。`true` 表示启用,`false` 表示不启用。自 v3.3 起,默认值变为 `true`。 - 引入版本:- ##### datacache_mem_size @@ -3987,7 +3987,7 @@ curl http://:/varz ##### datacache_tiered_cache_enable -- 默认值:true +- 默认值:false - 类型:Boolean - 单位:- - 是否动态:否 diff --git a/docs/zh/release_notes/release-3.2.md b/docs/zh/release_notes/release-3.2.md index e54df6de1c1228..7c4fd4ab27019b 100644 --- a/docs/zh/release_notes/release-3.2.md +++ b/docs/zh/release_notes/release-3.2.md @@ -4,6 +4,30 @@ displayed_sidebar: docs # StarRocks version 3.2 +## 3.2.15 + +发布日期:2025 年 2 月 14 日 + +### 新增特性 + +- 窗口函数支持 `max_by` 和 `min_by`。[#54961](https://github.com/StarRocks/starrocks/pull/54961) + +### 功能优化 + +- 新增 StarClient 超时参数。[#54496](https://github.com/StarRocks/starrocks/pull/54496) + - star_client_read_timeout_seconds + - star_client_list_timeout_seconds + - star_client_write_timeout_seconds +- List 分区表在执行 DELETE 语句时支持分区裁剪。[#55400](https://github.com/StarRocks/starrocks/pull/55400) + +### 问题修复 + +修复了如下问题: + +- Stream Load 调度到 Alive 状态为 false 的节点时,导入失败。[#55371](https://github.com/StarRocks/starrocks/pull/55371) +- Stream Load 部分列写入主键表报错。[#53403](https://github.com/StarRocks/starrocks/pull/55430) +- BE 节点重启后 bRPC 持续报错。[#40229](https://github.com/StarRocks/starrocks/pull/40229) + ## 3.2.14 发布日期:2025 年 1 月 8 日 diff --git a/docs/zh/sql-reference/sql-functions/table-functions/files.md b/docs/zh/sql-reference/sql-functions/table-functions/files.md index 64c4711cc78997..b7098e85828ef9 100644 --- a/docs/zh/sql-reference/sql-functions/table-functions/files.md +++ b/docs/zh/sql-reference/sql-functions/table-functions/files.md @@ -522,7 +522,7 @@ unload_data_param ::= | **参数** | **必填** | **说明** | | ---------------- | ------------ | ------------------------------------------------------------ | -| compression | 是 | 导出数据时要使用的压缩方法。有效值:
  • `uncompressed`:不使用任何压缩算法。
  • `gzip`:使用 gzip 压缩算法。
  • `snappy`:使用 SNAPPY 压缩算法。
  • `zstd`:使用 Zstd 压缩算法。
  • `lz4`:使用 LZ4 压缩算法。
| +| compression | 是 | 导出数据时要使用的压缩方法。有效值:
  • `uncompressed`:不使用任何压缩算法。
  • `gzip`:使用 gzip 压缩算法。
  • `snappy`:使用 SNAPPY 压缩算法。
  • `zstd`:使用 Zstd 压缩算法。
  • `lz4`:使用 LZ4 压缩算法。
**说明**
导出至 CSV 文件不支持数据压缩,需指定为 `uncompressed`。 | | partition_by | 否 | 用于将数据文件分区到不同存储路径的列,可以指定多个列。FILES() 提取指定列的 Key/Value 信息,并将数据文件存储在以对应 Key/Value 区分的子路径下。详细使用方法请见以下示例七。 | | single | 否 | 是否将数据导出到单个文件中。有效值:
  • `true`:数据存储在单个数据文件中。
  • `false`(默认):如果数据量超过 512 MB,,则数据会存储在多个文件中。
| | target_max_file_size | 否 | 分批导出时,单个文件的大致上限。单位:Byte。默认值:1073741824(1 GB)。当要导出的数据大小超过该值时,数据将被分成多个文件,每个文件的大小不会大幅超过该值。自 v3.2.7 起引入。| diff --git a/docs/zh/sql-reference/sql-statements/loading_unloading/INSERT.md b/docs/zh/sql-reference/sql-statements/loading_unloading/INSERT.md index 1567e688335721..40d128fe4179cb 100644 --- a/docs/zh/sql-reference/sql-statements/loading_unloading/INSERT.md +++ b/docs/zh/sql-reference/sql-statements/loading_unloading/INSERT.md @@ -162,7 +162,7 @@ INSERT OVERWRITE test WITH LABEL `label1` (c1, c2) SELECT * from test3; ```Plain INSERT INTO insert_wiki_edit - SELECT * FROM TABLE( + SELECT * FROM FILES( "path" = "s3://inserttest/parquet/insert_wiki_edit_append.parquet", "format" = "parquet", "aws.s3.access_key" = "xxxxxxxxxx", diff --git a/docs/zh/sql-reference/sql-statements/loading_unloading/routine_load/STOP_ROUTINE_LOAD.md b/docs/zh/sql-reference/sql-statements/loading_unloading/routine_load/STOP_ROUTINE_LOAD.md index 50d506d30a4411..84ba0e0b6d16c0 100644 --- a/docs/zh/sql-reference/sql-statements/loading_unloading/routine_load/STOP_ROUTINE_LOAD.md +++ b/docs/zh/sql-reference/sql-statements/loading_unloading/routine_load/STOP_ROUTINE_LOAD.md @@ -12,7 +12,7 @@ import RoutineLoadPrivNote from '../../../../_assets/commonMarkdown/RoutineLoadP -::: warning +:::warning 导入作业停止且无法恢复。因此请谨慎执行该语句。 diff --git a/docs/zh/unloading/unload_using_insert_into_files.md b/docs/zh/unloading/unload_using_insert_into_files.md index c3e521b0fb426c..0726ff40df716d 100644 --- a/docs/zh/unloading/unload_using_insert_into_files.md +++ b/docs/zh/unloading/unload_using_insert_into_files.md @@ -55,7 +55,7 @@ VALUES INSERT INTO FILES 支持将数据导出到单个文件或多个文件。您可以通过为这些文件指定不同的存储路径来进一步分区。 -在使用 INSERT INTO FILES 导出数据时,您必须通过设置 `compression` 属性手动设置压缩算法。有关 StarRocks 支持的数据压缩算法,请参阅[数据压缩](../table_design/data_compression.md)。 +在使用 INSERT INTO FILES 导出数据时,您必须通过设置 `compression` 属性手动设置压缩算法。有关数据导出支持的压缩算法,请参阅 [unload_data_param](../sql-reference/sql-functions/table-functions/files.md#unload_data_param)。 ### 导出数据到多个文件 diff --git a/fe/fe-core/src/main/java/com/starrocks/alter/LakeTableAsyncFastSchemaChangeJob.java b/fe/fe-core/src/main/java/com/starrocks/alter/LakeTableAsyncFastSchemaChangeJob.java index 33c1d71c9883a6..214a94b99cef51 100644 --- a/fe/fe-core/src/main/java/com/starrocks/alter/LakeTableAsyncFastSchemaChangeJob.java +++ b/fe/fe-core/src/main/java/com/starrocks/alter/LakeTableAsyncFastSchemaChangeJob.java @@ -88,8 +88,10 @@ protected TabletMetadataUpdateAgentTask createTask(PhysicalPartition partition, Set tablets) { String tag = String.format("%d_%d", partition.getId(), index.getId()); TabletMetadataUpdateAgentTask task = null; + boolean needUpdateSchema = false; for (IndexSchemaInfo info : schemaInfos) { if (info.indexId == index.getId()) { + needUpdateSchema = true; // `Set.add()` returns true means this set did not already contain the specified element boolean createSchemaFile = partitionsWithSchemaFile.add(tag); task = TabletMetadataUpdateAgentTaskFactory.createTabletSchemaUpdateTask(nodeId, @@ -97,6 +99,14 @@ protected TabletMetadataUpdateAgentTask createTask(PhysicalPartition partition, break; } } + + // if the index is not in schemaInfos, it means the schema of index are not needed to be modified, + // but we still need to update the tablet meta to improve the meta version + if (!needUpdateSchema) { + task = TabletMetadataUpdateAgentTaskFactory.createTabletSchemaUpdateTask(nodeId, + new ArrayList<>(tablets), null, false); + } + return task; } diff --git a/fe/fe-core/src/main/java/com/starrocks/alter/LakeTableSchemaChangeJob.java b/fe/fe-core/src/main/java/com/starrocks/alter/LakeTableSchemaChangeJob.java index acfd9e80a79583..aa4d6e47de38ec 100644 --- a/fe/fe-core/src/main/java/com/starrocks/alter/LakeTableSchemaChangeJob.java +++ b/fe/fe-core/src/main/java/com/starrocks/alter/LakeTableSchemaChangeJob.java @@ -233,10 +233,19 @@ void addShadowIndexToCatalog(@NotNull OlapTable table, long visibleTxnId) { } for (long shadowIdxId : indexIdMap.keySet()) { + List sortKeyColumnIndexes = null; + List sortKeyColumnUniqueIds = null; + long orgIndexId = indexIdMap.get(shadowIdxId); + if (orgIndexId == table.getBaseIndexId()) { + sortKeyColumnIndexes = sortKeyIdxes; + sortKeyColumnUniqueIds = sortKeyUniqueIds; + } + table.setIndexMeta(shadowIdxId, indexIdToName.get(shadowIdxId), indexSchemaMap.get(shadowIdxId), 0, 0, indexShortKeyMap.get(shadowIdxId), TStorageType.COLUMN, - table.getKeysTypeByIndexId(indexIdMap.get(shadowIdxId)), null, sortKeyIdxes, sortKeyUniqueIds); + table.getKeysTypeByIndexId(indexIdMap.get(shadowIdxId)), null, sortKeyColumnIndexes, + sortKeyColumnUniqueIds); MaterializedIndexMeta orgIndexMeta = table.getIndexMetaByIndexId(orgIndexId); Preconditions.checkNotNull(orgIndexMeta); MaterializedIndexMeta indexMeta = table.getIndexMetaByIndexId(shadowIdxId); diff --git a/fe/fe-core/src/main/java/com/starrocks/alter/SchemaChangeJobV2.java b/fe/fe-core/src/main/java/com/starrocks/alter/SchemaChangeJobV2.java index 5545a0ef9ed5a2..ffe96aab623d3a 100644 --- a/fe/fe-core/src/main/java/com/starrocks/alter/SchemaChangeJobV2.java +++ b/fe/fe-core/src/main/java/com/starrocks/alter/SchemaChangeJobV2.java @@ -471,14 +471,22 @@ private void addShadowIndexToCatalog(OlapTable tbl) { } for (long shadowIdxId : indexIdMap.keySet()) { + List sortKeyColumnIndexes = null; + List sortKeyColumnUniqueIds = null; + long orgIndexId = indexIdMap.get(shadowIdxId); + if (orgIndexId == tbl.getBaseIndexId()) { + sortKeyColumnIndexes = sortKeyIdxes; + sortKeyColumnUniqueIds = sortKeyUniqueIds; + } + tbl.setIndexMeta(shadowIdxId, indexIdToName.get(shadowIdxId), indexSchemaMap.get(shadowIdxId), indexSchemaVersionAndHashMap.get(shadowIdxId).schemaVersion, indexSchemaVersionAndHashMap.get(shadowIdxId).schemaHash, indexShortKeyMap.get(shadowIdxId), TStorageType.COLUMN, - tbl.getKeysTypeByIndexId(orgIndexId), null, sortKeyIdxes, - sortKeyUniqueIds); + tbl.getKeysTypeByIndexId(orgIndexId), null, sortKeyColumnIndexes, + sortKeyColumnUniqueIds); MaterializedIndexMeta orgIndexMeta = tbl.getIndexMetaByIndexId(orgIndexId); Preconditions.checkNotNull(orgIndexMeta); MaterializedIndexMeta indexMeta = tbl.getIndexMetaByIndexId(shadowIdxId); diff --git a/fe/fe-core/src/main/java/com/starrocks/catalog/ResourceGroupMgr.java b/fe/fe-core/src/main/java/com/starrocks/catalog/ResourceGroupMgr.java index 00ae62f7e52a9d..31db4859d41d27 100644 --- a/fe/fe-core/src/main/java/com/starrocks/catalog/ResourceGroupMgr.java +++ b/fe/fe-core/src/main/java/com/starrocks/catalog/ResourceGroupMgr.java @@ -41,6 +41,7 @@ import com.starrocks.sql.ast.CreateResourceGroupStmt; import com.starrocks.sql.ast.DropResourceGroupStmt; import com.starrocks.sql.ast.ShowResourceGroupStmt; +import com.starrocks.sql.optimizer.cost.feature.CostPredictor; import com.starrocks.system.BackendResourceStat; import com.starrocks.thrift.TWorkGroup; import com.starrocks.thrift.TWorkGroupOp; @@ -599,7 +600,9 @@ public TWorkGroup chooseResourceGroup(ConnectContext ctx, ResourceGroupClassifie String user = getUnqualifiedUser(ctx); String remoteIp = ctx.getRemoteIP(); final double planCpuCost = ctx.getAuditEventBuilder().build().planCpuCosts; - final double planMemCost = ctx.getAuditEventBuilder().build().planMemCosts; + final double planMemCost = CostPredictor.getServiceBasedCostPredictor().isAvailable() ? + ctx.getAuditEventBuilder().build().predictMemBytes : + ctx.getAuditEventBuilder().build().planMemCosts; // check short query first if (shortQueryResourceGroup != null) { 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 b719810ed08538..96e5a5169e9892 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 @@ -2524,6 +2524,11 @@ public class Config extends ConfigBase { //============================== Plan Feature Extraction BEGIN ========================================// @ConfField(mutable = true, comment = "Collect features of query plan into a log file") public static boolean enable_plan_feature_collection = false; + @ConfField(mutable = true) + public static boolean enable_query_cost_prediction = false; + @ConfField(mutable = true) + public static String query_cost_prediction_service_address = "http://localhost:5000"; + @ConfField public static String feature_log_dir = StarRocksFE.STARROCKS_HOME_DIR + "/log"; @ConfField @@ -3470,7 +3475,7 @@ public class Config extends ConfigBase { public static int query_deploy_threadpool_size = max(50, getRuntime().availableProcessors() * 10); @ConfField(mutable = true) - public static long automated_cluster_snapshot_interval_seconds = 1800; + public static long automated_cluster_snapshot_interval_seconds = 600; @ConfField(mutable = false) public static int max_historical_automated_cluster_snapshot_jobs = 100; diff --git a/fe/fe-core/src/main/java/com/starrocks/common/ConfigBase.java b/fe/fe-core/src/main/java/com/starrocks/common/ConfigBase.java index c7a40661268ee0..673bae7f9709b6 100644 --- a/fe/fe-core/src/main/java/com/starrocks/common/ConfigBase.java +++ b/fe/fe-core/src/main/java/com/starrocks/common/ConfigBase.java @@ -36,6 +36,8 @@ import com.google.common.base.Strings; import com.google.common.collect.Lists; +import com.starrocks.common.util.DateUtils; +import com.starrocks.common.util.Util; import com.starrocks.qe.ConnectContext; import com.starrocks.rpc.ThriftConnectionPool; import com.starrocks.rpc.ThriftRPCRequestExecutor; @@ -47,13 +49,22 @@ import com.starrocks.thrift.TSetConfigResponse; import com.starrocks.thrift.TStatus; import com.starrocks.thrift.TStatusCode; +import org.apache.commons.lang3.StringUtils; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import java.io.BufferedReader; +import java.io.BufferedWriter; import java.io.FileReader; +import java.io.FileWriter; +import java.io.IOException; import java.lang.annotation.Retention; import java.lang.annotation.RetentionPolicy; import java.lang.reflect.Field; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.time.LocalDateTime; import java.util.Arrays; import java.util.HashMap; import java.util.HashSet; @@ -87,20 +98,31 @@ public class ConfigBase { } protected Properties props; + private static boolean isPersisted = false; + private static String configPath; protected static Field[] configFields; protected static Map allMutableConfigs = new HashMap<>(); public void init(String propFile) throws Exception { + configPath = propFile; configFields = this.getClass().getFields(); initAllMutableConfigs(); props = new Properties(); try (FileReader reader = new FileReader(propFile)) { props.load(reader); } + if (Files.isWritable(Path.of(propFile)) && !Util.isRunningInContainer()) { + isPersisted = true; + } + replacedByEnv(); setFields(); } + public static boolean isIsPersisted() { + return isPersisted; + } + public static void initAllMutableConfigs() { for (Field field : configFields) { ConfField confField = field.getAnnotation(ConfField.class); @@ -298,7 +320,22 @@ public static void setConfigField(Field f, String confVal) throws Exception { } } - public static synchronized void setMutableConfig(String key, String value) throws InvalidConfException { + public static synchronized void setMutableConfig(String key, String value, + boolean isPersisted, String userIdentity) throws InvalidConfException { + if (isPersisted) { + if (!ConfigBase.isIsPersisted()) { + String errMsg = "set persisted config failed, because current running mode is not persisted"; + LOG.warn(errMsg); + throw new InvalidConfException(errMsg); + } + + try { + appendPersistedProperties(key, value, userIdentity); + } catch (IOException e) { + throw new InvalidConfException("Failed to set config '" + key + "'. err: " + e.getMessage()); + } + } + Field field = allMutableConfigs.get(key); if (field == null) { throw new InvalidConfException(ErrorCode.ERROR_CONFIG_NOT_EXIST, key); @@ -313,6 +350,54 @@ public static synchronized void setMutableConfig(String key, String value) throw LOG.info("set config {} to {}", key, value); } + private static void appendPersistedProperties(String key, String value, String userIdentity) throws IOException { + Properties props = new Properties(); + Path path = Paths.get(configPath); + List lines = Files.readAllLines(path); + try (BufferedReader reader = new BufferedReader(new FileReader(configPath))) { + props.load(reader); + } + + String oldValue = props.getProperty(key); + String comment; + if (StringUtils.isEmpty(oldValue)) { + comment = String.format("# The user %s added %s=%s at %s", userIdentity, key, value, + LocalDateTime.now().format(DateUtils.DATE_TIME_FORMATTER_UNIX)); + } else { + comment = String.format("# The user %s changed %s to %s at %s", userIdentity, oldValue, value, + LocalDateTime.now().format(DateUtils.DATE_TIME_FORMATTER_UNIX)); + } + + boolean keyExists = false; + // Keep the original configuration file format + try (BufferedWriter writer = new BufferedWriter(new FileWriter(configPath))) { + for (String s : lines) { + String line = s.trim(); + + // Compatible with key=value & key = value + if (line.matches("^" + key + "\\s*=\\s*.*$")) { + keyExists = true; + writer.write(comment); + writer.newLine(); + writer.write(key + " = " + value); + writer.newLine(); + continue; + } + + writer.write(s); + writer.newLine(); + } + + if (!keyExists) { + writer.newLine(); + writer.write(comment); + writer.newLine(); + writer.write(key + " = " + value); + writer.newLine(); + } + } + } + private static boolean isAliasesMatch(PatternMatcher matcher, String[] aliases) { if (matcher == null) { return true; @@ -386,9 +471,9 @@ public static synchronized List> getConfigInfo(PatternMatcher match return configs; } - public static synchronized void setConfig(AdminSetConfigStmt stmt) throws InvalidConfException { - setFrontendConfig(stmt.getConfig().getMap()); - + public static synchronized void setConfig(AdminSetConfigStmt stmt) throws DdlException { + String user = ConnectContext.get().getCurrentUserIdentity().getUser(); + setFrontendConfig(stmt.getConfig().getMap(), stmt.isPersistent(), user); List allFrontends = GlobalStateMgr.getCurrentState().getNodeMgr().getFrontends(null); int timeout = ConnectContext.get().getExecTimeout() * 1000 + Config.thrift_rpc_timeout_ms; StringBuilder errMsg = new StringBuilder(); @@ -396,37 +481,45 @@ public static synchronized void setConfig(AdminSetConfigStmt stmt) throws Invali if (fe.getHost().equals(GlobalStateMgr.getCurrentState().getNodeMgr().getSelfNode().first)) { continue; } + errMsg.append(callFrontNodeSetConfig(stmt, fe, timeout, errMsg)); + } + if (!errMsg.isEmpty()) { + ErrorReport.reportDdlException(ErrorCode.ERROR_SET_CONFIG_FAILED, errMsg.toString()); + } + } - TSetConfigRequest request = new TSetConfigRequest(); - request.setKeys(Lists.newArrayList(stmt.getConfig().getKey())); - request.setValues(Lists.newArrayList(stmt.getConfig().getValue())); - try { - TSetConfigResponse response = ThriftRPCRequestExecutor.call( - ThriftConnectionPool.frontendPool, - new TNetworkAddress(fe.getHost(), fe.getRpcPort()), - timeout, - client -> client.setConfig(request)); - TStatus status = response.getStatus(); - if (status.getStatus_code() != TStatusCode.OK) { - errMsg.append("set config for fe[").append(fe.getHost()).append("] failed: "); - if (status.getError_msgs() != null && !status.getError_msgs().isEmpty()) { - errMsg.append(String.join(",", status.getError_msgs())); - } - errMsg.append(";"); + private static synchronized StringBuilder callFrontNodeSetConfig(AdminSetConfigStmt stmt, Frontend fe, int timeout, + StringBuilder errMsg) { + TSetConfigRequest request = new TSetConfigRequest(); + request.setKeys(Lists.newArrayList(stmt.getConfig().getKey())); + request.setValues(Lists.newArrayList(stmt.getConfig().getValue())); + request.setIs_persistent(stmt.isPersistent()); + request.setUser_identity(ConnectContext.get().getCurrentUserIdentity().getUser()); + try { + TSetConfigResponse response = ThriftRPCRequestExecutor.call( + ThriftConnectionPool.frontendPool, + new TNetworkAddress(fe.getHost(), fe.getRpcPort()), + timeout, + client -> client.setConfig(request)); + TStatus status = response.getStatus(); + if (status.getStatus_code() != TStatusCode.OK) { + errMsg.append("set config for fe[").append(fe.getHost()).append("] failed: "); + if (status.getError_msgs() != null && status.getError_msgs().size() > 0) { + errMsg.append(String.join(",", status.getError_msgs())); } - } catch (Exception e) { - LOG.warn("set remote fe: {} config failed", fe.getHost(), e); - errMsg.append("set config for fe[").append(fe.getHost()).append("] failed: ").append(e.getMessage()); + errMsg.append(";"); } + } catch (Exception e) { + LOG.warn("set remote fe: {} config failed", fe.getHost(), e); + errMsg.append("set config for fe[").append(fe.getHost()).append("] failed: ").append(e.getMessage()); } - if (errMsg.length() > 0) { - throw new InvalidConfException(ErrorCode.ERROR_SET_CONFIG_FAILED, errMsg.toString()); - } + return errMsg; } - public static synchronized void setFrontendConfig(Map configs) throws InvalidConfException { + public static synchronized void setFrontendConfig(Map configs, boolean isPersisted, String userIdentity) + throws InvalidConfException { for (Map.Entry entry : configs.entrySet()) { - ConfigBase.setMutableConfig(entry.getKey(), entry.getValue()); + ConfigBase.setMutableConfig(entry.getKey(), entry.getValue(), isPersisted, userIdentity); } } } diff --git a/fe/fe-core/src/main/java/com/starrocks/common/util/Util.java b/fe/fe-core/src/main/java/com/starrocks/common/util/Util.java index cc319892017b5a..8fe97491914c4d 100644 --- a/fe/fe-core/src/main/java/com/starrocks/common/util/Util.java +++ b/fe/fe-core/src/main/java/com/starrocks/common/util/Util.java @@ -55,6 +55,7 @@ import java.io.BufferedReader; import java.io.ByteArrayOutputStream; +import java.io.File; import java.io.IOException; import java.io.InputStream; import java.io.InputStreamReader; @@ -502,4 +503,8 @@ public static ConnectContext getOrCreateInnerContext() { ctx.setGlobalStateMgr(GlobalStateMgr.getCurrentState()); return ctx; } + + public static boolean isRunningInContainer() { + return new File("/.dockerenv").exists(); + } } diff --git a/fe/fe-core/src/main/java/com/starrocks/connector/jdbc/OracleSchemaResolver.java b/fe/fe-core/src/main/java/com/starrocks/connector/jdbc/OracleSchemaResolver.java index a2126154ab413f..d46bbd8cef0f05 100644 --- a/fe/fe-core/src/main/java/com/starrocks/connector/jdbc/OracleSchemaResolver.java +++ b/fe/fe-core/src/main/java/com/starrocks/connector/jdbc/OracleSchemaResolver.java @@ -14,6 +14,7 @@ package com.starrocks.connector.jdbc; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import com.starrocks.catalog.Column; import com.starrocks.catalog.JDBCTable; @@ -23,8 +24,11 @@ import com.starrocks.catalog.Type; import com.starrocks.common.DdlException; import com.starrocks.common.SchemaConstants; +import com.starrocks.common.util.TimeUtils; +import com.starrocks.connector.exception.StarRocksConnectorException; import java.sql.Connection; +import java.sql.PreparedStatement; import java.sql.ResultSet; import java.sql.SQLException; import java.sql.Types; @@ -153,4 +157,94 @@ public Type convertColumnType(int dataType, String typeName, int columnSize, int return ScalarType.createUnifiedDecimalType(precision, max(digits, 0)); } } + + @Override + public List listPartitionNames(Connection connection, String databaseName, String tableName) { + final String partitionNamesQuery = "SELECT PARTITION_NAME AS NAME FROM ALL_TAB_PARTITIONS " + + "WHERE TABLE_OWNER = ? AND TABLE_NAME = ? AND PARTITION_NAME IS NOT NULL " + + "ORDER BY PARTITION_POSITION"; + try (PreparedStatement ps = connection.prepareStatement(partitionNamesQuery)) { + ps.setString(1, databaseName.toUpperCase()); + ps.setString(2, tableName.toUpperCase()); + final ResultSet rs = ps.executeQuery(); + final ImmutableList.Builder list = ImmutableList.builder(); + while (rs.next()) { + final String partitionName = rs.getString("NAME"); + list.add(partitionName); + } + return list.build(); + } catch (SQLException e) { + throw new RuntimeException("Failed to fetch partition names: " + e.getMessage(), e); + } + } + + /** + * desc ALL_PART_KEY_COLUMNS; + * Name Null? Type + * ----------------------------------------- -------- ---------------------------- + * OWNER VARCHAR2(128) + * NAME VARCHAR2(128) + * OBJECT_TYPE CHAR(5) + * COLUMN_NAME VARCHAR2(4000) + * COLUMN_POSITION NUMBER + * COLLATED_COLUMN_ID NUMBER + */ + @Override + public List listPartitionColumns(Connection connection, String databaseName, String tableName) { + final String partitionColumnsQuery = "SELECT DISTINCT COLUMN_NAME FROM ALL_PART_KEY_COLUMNS " + + "WHERE OWNER = ? AND NAME = ? ORDER BY COLUMN_POSITION"; + try (PreparedStatement ps = connection.prepareStatement(partitionColumnsQuery)) { + ps.setString(1, databaseName.toUpperCase()); + ps.setString(2, tableName.toUpperCase()); + final ResultSet rs = ps.executeQuery(); + final ImmutableList.Builder list = ImmutableList.builder(); + while (rs.next()) { + String partitionColumn = rs.getString("COLUMN_NAME"); + list.add(partitionColumn); + } + return list.build(); + } catch (SQLException e) { + throw new RuntimeException("Failed to fetch partition columns: " + e.getMessage(), e); + } + } + + public List getPartitions(Connection connection, Table table) { + final JDBCTable jdbcTable = (JDBCTable) table; + final String query = getPartitionQuery(table); + try (PreparedStatement ps = connection.prepareStatement(query)) { + ps.setString(1, jdbcTable.getCatalogDBName()); + ps.setString(2, jdbcTable.getCatalogTableName()); + final ResultSet rs = ps.executeQuery(); + final ImmutableList.Builder list = ImmutableList.builder(); + if (null != rs) { + while (rs.next()) { + final String[] partitionNames = rs.getString("NAME"). + replace("'", "").split(","); + final long createTime = rs.getTimestamp("MODIFIED_TIME").getTime(); + for (String partitionName : partitionNames) { + list.add(new Partition(partitionName, createTime)); + } + } + final ImmutableList partitions = list.build(); + return partitions.isEmpty() + ? Lists.newArrayList(new Partition(table.getName(), TimeUtils.getEpochSeconds())) + : partitions; + } else { + return Lists.newArrayList(new Partition(table.getName(), TimeUtils.getEpochSeconds())); + } + } catch (SQLException | NullPointerException e) { + throw new StarRocksConnectorException(e.getMessage(), e); + } + } + + private static String getPartitionQuery(Table table) { + final String partitionsQuery = "SELECT PARTITION_NAME AS NAME, " + + "LAST_ANALYZED AS MODIFIED_TIME " + + "FROM ALL_TAB_PARTITIONS WHERE TABLE_OWNER = ? AND TABLE_NAME = ? " + + "AND PARTITION_NAME IS NOT NULL ORDER BY PARTITION_POSITION"; + final String nonPartitionQuery = "SELECT TABLE_NAME AS NAME, " + + "LAST_ANALYZED AS MODIFIED_TIME " + + "FROM ALL_TABLES WHERE OWNER = ? AND TABLE_NAME = ? "; + return table.isUnPartitioned() ? nonPartitionQuery : partitionsQuery; + } } \ No newline at end of file diff --git a/fe/fe-core/src/main/java/com/starrocks/connector/paimon/PaimonFilter.java b/fe/fe-core/src/main/java/com/starrocks/connector/paimon/PaimonFilter.java deleted file mode 100644 index c665caaeea6eb7..00000000000000 --- a/fe/fe-core/src/main/java/com/starrocks/connector/paimon/PaimonFilter.java +++ /dev/null @@ -1,63 +0,0 @@ -// 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.connector.paimon; - -import com.starrocks.sql.optimizer.operator.scalar.ScalarOperator; - -import java.util.List; -import java.util.Objects; - -public class PaimonFilter { - private final String databaseName; - private final String tableName; - private final ScalarOperator predicate; - private final List fieldNames; - - public PaimonFilter(String databaseName, String tableName, ScalarOperator predicate, List fieldNames) { - this.databaseName = databaseName; - this.tableName = tableName; - this.predicate = predicate; - this.fieldNames = fieldNames; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - - if (o == null || getClass() != o.getClass()) { - return false; - } - - PaimonFilter that = (PaimonFilter) o; - boolean predicateEqual = false; - if (predicate != null && that.predicate != null) { - predicateEqual = predicate.equals(that.predicate); - } else if (predicate == null && that.predicate == null) { - predicateEqual = true; - } - - return Objects.equals(databaseName, that.databaseName) && - Objects.equals(tableName, that.tableName) && - predicateEqual && - fieldNames.equals(that.fieldNames); - } - - @Override - public int hashCode() { - return Objects.hash(databaseName, tableName, fieldNames); - } -} diff --git a/fe/fe-core/src/main/java/com/starrocks/connector/paimon/PaimonMetadata.java b/fe/fe-core/src/main/java/com/starrocks/connector/paimon/PaimonMetadata.java index ee38b78268c8df..8b26c5299360d4 100644 --- a/fe/fe-core/src/main/java/com/starrocks/connector/paimon/PaimonMetadata.java +++ b/fe/fe-core/src/main/java/com/starrocks/connector/paimon/PaimonMetadata.java @@ -29,6 +29,7 @@ import com.starrocks.connector.GetRemoteFilesParams; import com.starrocks.connector.HdfsEnvironment; import com.starrocks.connector.PartitionInfo; +import com.starrocks.connector.PredicateSearchKey; import com.starrocks.connector.RemoteFileDesc; import com.starrocks.connector.RemoteFileInfo; import com.starrocks.connector.TableVersionRange; @@ -78,7 +79,7 @@ public class PaimonMetadata implements ConnectorMetadata { private final String catalogName; private final Map tables = new ConcurrentHashMap<>(); private final Map databases = new ConcurrentHashMap<>(); - private final Map paimonSplits = new ConcurrentHashMap<>(); + private final Map paimonSplits = new ConcurrentHashMap<>(); private final Map partitionInfos = new ConcurrentHashMap<>(); private final ConnectorProperties properties; @@ -249,9 +250,12 @@ public boolean tableExists(String dbName, String tableName) { public List getRemoteFiles(Table table, GetRemoteFilesParams params) { RemoteFileInfo remoteFileInfo = new RemoteFileInfo(); PaimonTable paimonTable = (PaimonTable) table; - PaimonFilter filter = - new PaimonFilter(paimonTable.getCatalogDBName(), paimonTable.getCatalogTableName(), params.getPredicate(), - params.getFieldNames()); + long latestSnapshotId = -1L; + if (paimonTable.getNativeTable().latestSnapshotId().isPresent()) { + latestSnapshotId = paimonTable.getNativeTable().latestSnapshotId().getAsLong(); + } + PredicateSearchKey filter = PredicateSearchKey.of(paimonTable.getCatalogDBName(), + paimonTable.getCatalogTableName(), latestSnapshotId, params.getPredicate()); if (!paimonSplits.containsKey(filter)) { ReadBuilder readBuilder = paimonTable.getNativeTable().newReadBuilder(); int[] projected = diff --git a/fe/fe-core/src/main/java/com/starrocks/connector/partitiontraits/JDBCPartitionTraits.java b/fe/fe-core/src/main/java/com/starrocks/connector/partitiontraits/JDBCPartitionTraits.java index 8cfaca4d6a1b23..301db0cfce1148 100644 --- a/fe/fe-core/src/main/java/com/starrocks/connector/partitiontraits/JDBCPartitionTraits.java +++ b/fe/fe-core/src/main/java/com/starrocks/connector/partitiontraits/JDBCPartitionTraits.java @@ -15,9 +15,11 @@ import com.google.common.collect.Range; import com.starrocks.analysis.Expr; +import com.starrocks.catalog.BaseTableInfo; import com.starrocks.catalog.Column; import com.starrocks.catalog.JDBCPartitionKey; import com.starrocks.catalog.JDBCTable; +import com.starrocks.catalog.MaterializedView; import com.starrocks.catalog.PartitionKey; import com.starrocks.common.AnalysisException; import com.starrocks.connector.PartitionInfo; @@ -27,6 +29,7 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.Set; public class JDBCPartitionTraits extends DefaultTraits { @Override @@ -66,5 +69,18 @@ public Optional maxPartitionRefreshTs() { .map(com.starrocks.connector.PartitionInfo::getModifiedTime) .max(Long::compareTo); } + + @Override + public Set getUpdatedPartitionNames(List baseTables, + MaterializedView.AsyncRefreshContext context) { + + try { + return super.getUpdatedPartitionNames(baseTables, context); + } catch (Exception e) { + // some external table traits do not support getPartitionNameWithPartitionInfo, will throw exception, + // just return null + return null; + } + } } diff --git a/fe/fe-core/src/main/java/com/starrocks/http/rest/QueryDetailAction.java b/fe/fe-core/src/main/java/com/starrocks/http/rest/QueryDetailAction.java index 55cbabe7da7fe2..99b56b833ce4c4 100644 --- a/fe/fe-core/src/main/java/com/starrocks/http/rest/QueryDetailAction.java +++ b/fe/fe-core/src/main/java/com/starrocks/http/rest/QueryDetailAction.java @@ -57,7 +57,7 @@ public static void registerAction(ActionController controller) throws IllegalArg } @Override - public void execute(BaseRequest request, BaseResponse response) { + public void executeWithoutPassword(BaseRequest request, BaseResponse response) { String eventTimeStr = request.getSingleParameter("event_time"); if (eventTimeStr == null) { response.getContent().append("not valid parameter"); diff --git a/fe/fe-core/src/main/java/com/starrocks/lake/snapshot/ClusterSnapshot.java b/fe/fe-core/src/main/java/com/starrocks/lake/snapshot/ClusterSnapshot.java index 5cff66cda96148..8241cef9345fa5 100644 --- a/fe/fe-core/src/main/java/com/starrocks/lake/snapshot/ClusterSnapshot.java +++ b/fe/fe-core/src/main/java/com/starrocks/lake/snapshot/ClusterSnapshot.java @@ -21,7 +21,9 @@ import com.starrocks.thrift.TClusterSnapshotsItem; public class ClusterSnapshot { - public enum ClusterSnapshotType { AUTOMATED, MANUAL, INCREMENTAL } + public enum ClusterSnapshotType { + AUTOMATED, MANUAL, INCREMENTAL + } @SerializedName(value = "id") private long id; @@ -40,10 +42,11 @@ public enum ClusterSnapshotType { AUTOMATED, MANUAL, INCREMENTAL } @SerializedName(value = "starMgrJournal") private long starMgrJournalId; - public ClusterSnapshot() {} + public ClusterSnapshot() { + } public ClusterSnapshot(long id, String snapshotName, String storageVolumeName, long createdTimeMs, - long finishedTimeMs, long feJournalId, long starMgrJournalId) { + long finishedTimeMs, long feJournalId, long starMgrJournalId) { this.id = id; this.snapshotName = snapshotName; this.type = ClusterSnapshotType.AUTOMATED; diff --git a/fe/fe-core/src/main/java/com/starrocks/lake/snapshot/ClusterSnapshotCheckpointScheduler.java b/fe/fe-core/src/main/java/com/starrocks/lake/snapshot/ClusterSnapshotCheckpointScheduler.java index 88c1abcd0aa595..c2d78ec1a9b537 100644 --- a/fe/fe-core/src/main/java/com/starrocks/lake/snapshot/ClusterSnapshotCheckpointScheduler.java +++ b/fe/fe-core/src/main/java/com/starrocks/lake/snapshot/ClusterSnapshotCheckpointScheduler.java @@ -35,7 +35,8 @@ public class ClusterSnapshotCheckpointScheduler extends FrontendDaemon { private boolean firstRun; - public ClusterSnapshotCheckpointScheduler(CheckpointController feController, CheckpointController starMgrController) { + public ClusterSnapshotCheckpointScheduler(CheckpointController feController, + CheckpointController starMgrController) { super("cluster_snapshot_checkpoint_scheduler", Config.automated_cluster_snapshot_interval_seconds * 1000L); this.feController = feController; this.starMgrController = starMgrController; @@ -66,7 +67,7 @@ protected void runAfterCatalogReady() { protected void runCheckpointScheduler() { String errMsg = ""; ClusterSnapshotJob job = GlobalStateMgr.getCurrentState().getClusterSnapshotMgr() - .createAutomatedSnapshotJob(); /* INITIALIZING state */ + .createAutomatedSnapshotJob(); /* INITIALIZING state */ do { // step 1: capture consistent journal id for checkpoint @@ -76,8 +77,9 @@ protected void runCheckpointScheduler() { break; } job.setJournalIds(consistentIds.first, consistentIds.second); - LOG.info("Successful capture consistent journal id, FE checkpoint journal Id: {}, StarMgr checkpoint journal Id: {}", - consistentIds.first, consistentIds.second); + LOG.info( + "Successful capture consistent journal id, FE checkpoint journal Id: {}, StarMgr checkpoint journal Id: {}", + consistentIds.first, consistentIds.second); // step 2: make two controllers accept the requested id to do checkpoint control job.setState(ClusterSnapshotJobState.SNAPSHOTING); @@ -87,7 +89,7 @@ protected void runCheckpointScheduler() { long feCheckpointJournalId = consistentIds.first; if (feImageJournalId < feCheckpointJournalId) { Pair createFEImageRet = feController.runCheckpointControllerWithIds(feImageJournalId, - feCheckpointJournalId); + feCheckpointJournalId); if (!createFEImageRet.first) { errMsg = "checkpoint failed for FE image: " + createFEImageRet.second; break; @@ -101,8 +103,8 @@ protected void runCheckpointScheduler() { long starMgrImageJournalId = starMgrController.getImageJournalId(); long starMgrCheckpointJournalId = consistentIds.second; if (starMgrImageJournalId < starMgrCheckpointJournalId) { - Pair createStarMgrImageRet = - starMgrController.runCheckpointControllerWithIds(starMgrImageJournalId, starMgrCheckpointJournalId); + Pair createStarMgrImageRet = starMgrController + .runCheckpointControllerWithIds(starMgrImageJournalId, starMgrCheckpointJournalId); if (!createStarMgrImageRet.first) { errMsg = "checkpoint failed for starMgr image: " + createStarMgrImageRet.second; break; @@ -122,12 +124,14 @@ protected void runCheckpointScheduler() { errMsg = "upload image failed, err msg: " + e.getMessage(); break; } - LOG.info("Finish upload image for Cluster Snapshot, FE checkpoint journal Id: {}, StarMgr checkpoint journal Id: {}", - job.getFeJournalId(), job.getStarMgrJournalId()); + LOG.info( + "Finish upload image for Cluster Snapshot, FE checkpoint journal Id: {}, StarMgr checkpoint journal Id: {}", + job.getFeJournalId(), job.getStarMgrJournalId()); } while (false); if (!GlobalStateMgr.getCurrentState().getClusterSnapshotMgr().isAutomatedSnapshotOn()) { - errMsg = "Job: " + job.getSnapshotName() + " has been cancelled because automated cluster snapshot has been turn off"; + errMsg = "Job: " + job.getSnapshotName() + + " has been cancelled because automated cluster snapshot has been turn off"; } if (!errMsg.isEmpty()) { @@ -138,32 +142,40 @@ protected void runCheckpointScheduler() { } else { job.setState(ClusterSnapshotJobState.FINISHED); job.logJob(); - LOG.info("Finish Cluster Snapshot checkpoint, FE checkpoint journal Id: {}, StarMgr checkpoint journal Id: {}", - job.getFeJournalId(), job.getStarMgrJournalId()); + LOG.info( + "Finish Cluster Snapshot checkpoint, FE checkpoint journal Id: {}, StarMgr checkpoint journal Id: {}", + job.getFeJournalId(), job.getStarMgrJournalId()); } } /* - * Definition of consistent: Suppose there are two images generated by FE and StarMgr, call FEImageNew + * Definition of consistent: Suppose there are two images generated by FE and + * StarMgr, call FEImageNew * and StarMgrImageNew and satisfy: * FEImageNew = FEImageOld + editlog(i) + ... + editlog(j) * StarMgrImageNew = StarMgrImageOld + editlog(k) + ... + editlog(m) * * Define Tj = generated time of editlog(j), Tmax = max(Tj, Tm) - * Consistency means all editlogs generated before Tmax (no matter the editlog is belong to FE or starMgr) + * Consistency means all editlogs generated before Tmax (no matter the editlog + * is belong to FE or starMgr) * should be included in the image generated by checkpoint. - * In other words, there must be no holes before the `maximum` editlog contained in the two images + * In other words, there must be no holes before the `maximum` editlog contained + * in the two images * generated by checkpoint. * - * How to get the consistent id: because editlog is generated and flush in a synchronous way, so we can simply + * How to get the consistent id: because editlog is generated and flush in a + * synchronous way, so we can simply * get the `snapshot` of maxJouranlId for both FE side and StarMgr side. * We get the `snapshot` in a lock-free way. As shown in the code below: - * (1) if feCheckpointIdT1 == feCheckpointIdT3 means in [T1, T3], no editlog added for FE side - * (2) if starMgrCheckpointIdT2 == starMgrCheckpointIdT4 means in [T2, T4], no editlog added for StarMgr side + * (1) if feCheckpointIdT1 == feCheckpointIdT3 means in [T1, T3], no editlog + * added for FE side + * (2) if starMgrCheckpointIdT2 == starMgrCheckpointIdT4 means in [T2, T4], no + * editlog added for StarMgr side * - * Because T1 < T2 < T3 < T4, from (1),(2) -> [T2, T3] no editlog added for FE side and StarMgr side + * Because T1 < T2 < T3 < T4, from (1),(2) -> [T2, T3] no editlog added for FE + * side and StarMgr side * So we get the snapshots are feCheckpointIdT3 and starMgrCheckpointIdT2 - */ + */ private Pair captureConsistentCheckpointIdBetweenFEAndStarMgr() { if (feController == null || starMgrController == null) { return null; @@ -175,11 +187,11 @@ private Pair captureConsistentCheckpointIdBetweenFEAndStarMgr() { long starMgrCheckpointIdT2 = starMgrController.getJournal().getMaxJournalId(); long feCheckpointIdT3 = feController.getJournal().getMaxJournalId(); long starMgrCheckpointIdT4 = starMgrController.getJournal().getMaxJournalId(); - + if (feCheckpointIdT1 == feCheckpointIdT3 && starMgrCheckpointIdT2 == starMgrCheckpointIdT4) { return Pair.create(feCheckpointIdT3, starMgrCheckpointIdT2); } - + try { Thread.sleep(100); } catch (Exception ignore) { diff --git a/fe/fe-core/src/main/java/com/starrocks/lake/snapshot/ClusterSnapshotJob.java b/fe/fe-core/src/main/java/com/starrocks/lake/snapshot/ClusterSnapshotJob.java index a3ce79cefe7f30..10b01cef86a4f0 100644 --- a/fe/fe-core/src/main/java/com/starrocks/lake/snapshot/ClusterSnapshotJob.java +++ b/fe/fe-core/src/main/java/com/starrocks/lake/snapshot/ClusterSnapshotJob.java @@ -30,16 +30,21 @@ public class ClusterSnapshotJob implements Writable { public static final Logger LOG = LogManager.getLogger(ClusterSnapshotJob.class); + /* * INITIALIZING: INIT state for the snapshot. - * SNAPSHOTING: Doing checkpoint/image generation by replaying log for image both for FE and StarMgr and - * then upload the image into remote storage + * SNAPSHOTING: Doing checkpoint/image generation by replaying log for image + * both for FE and StarMgr and + * then upload the image into remote storage * UPLOADING: Uploading image file into remote storage * FINISHED: Finish backup snapshot * EXPIRED: Not the latest finished backup snapshot - * DELETED: Not the lastest finished backup snapshot and the cluster snapshot has been deleted from remote + * DELETED: Not the lastest finished backup snapshot and the cluster snapshot + * has been deleted from remote */ - public enum ClusterSnapshotJobState { INITIALIZING, SNAPSHOTING, UPLOADING, FINISHED, EXPIRED, DELETED, ERROR } + public enum ClusterSnapshotJobState { + INITIALIZING, SNAPSHOTING, UPLOADING, FINISHED, EXPIRED, DELETED, ERROR + } @SerializedName(value = "snapshot") private ClusterSnapshot snapshot; @@ -58,7 +63,8 @@ public void setState(ClusterSnapshotJobState state) { this.state = state; if (state == ClusterSnapshotJobState.FINISHED) { snapshot.setFinishedTimeMs(System.currentTimeMillis()); - GlobalStateMgr.getCurrentState().getClusterSnapshotMgr().clearFinishedAutomatedClusterSnapshot(getSnapshotName()); + GlobalStateMgr.getCurrentState().getClusterSnapshotMgr() + .clearFinishedAutomatedClusterSnapshot(getSnapshotName()); } } @@ -108,8 +114,8 @@ public ClusterSnapshotJobState getState() { public boolean isUnFinishedState() { return state == ClusterSnapshotJobState.INITIALIZING || - state == ClusterSnapshotJobState.SNAPSHOTING || - state == ClusterSnapshotJobState.UPLOADING; + state == ClusterSnapshotJobState.SNAPSHOTING || + state == ClusterSnapshotJobState.UPLOADING; } public boolean isError() { diff --git a/fe/fe-core/src/main/java/com/starrocks/lake/snapshot/ClusterSnapshotMgr.java b/fe/fe-core/src/main/java/com/starrocks/lake/snapshot/ClusterSnapshotMgr.java index 6f3b3299bff0d6..f4a8ee934c6db4 100644 --- a/fe/fe-core/src/main/java/com/starrocks/lake/snapshot/ClusterSnapshotMgr.java +++ b/fe/fe-core/src/main/java/com/starrocks/lake/snapshot/ClusterSnapshotMgr.java @@ -20,7 +20,6 @@ import com.starrocks.common.Config; import com.starrocks.common.StarRocksException; import com.starrocks.lake.snapshot.ClusterSnapshotJob.ClusterSnapshotJobState; -import com.starrocks.leader.CheckpointController; import com.starrocks.persist.ClusterSnapshotLog; import com.starrocks.persist.ImageWriter; import com.starrocks.persist.gson.GsonPostProcessable; @@ -33,6 +32,7 @@ import com.starrocks.server.RunMode; import com.starrocks.sql.ast.AdminSetAutomatedSnapshotOffStmt; import com.starrocks.sql.ast.AdminSetAutomatedSnapshotOnStmt; +import com.starrocks.staros.StarMgrServer; import com.starrocks.storagevolume.StorageVolume; import com.starrocks.thrift.TClusterSnapshotJobsResponse; import com.starrocks.thrift.TClusterSnapshotsResponse; @@ -57,7 +57,8 @@ public class ClusterSnapshotMgr implements GsonPostProcessable { private ClusterSnapshotCheckpointScheduler clusterSnapshotCheckpointScheduler; - public ClusterSnapshotMgr() {} + public ClusterSnapshotMgr() { + } // Turn on automated snapshot, use stmt for extension in future public void setAutomatedSnapshotOn(AdminSetAutomatedSnapshotOnStmt stmt) { @@ -83,13 +84,13 @@ public boolean isAutomatedSnapshotOn() { // Turn off automated snapshot, use stmt for extension in future public void setAutomatedSnapshotOff(AdminSetAutomatedSnapshotOffStmt stmt) { - ClusterSnapshotLog log = new ClusterSnapshotLog(); - log.setAutomatedSnapshotOff(); - GlobalStateMgr.getCurrentState().getEditLog().logClusterSnapshotLog(log); - clearFinishedAutomatedClusterSnapshot(null); setAutomatedSnapshotOff(); + + ClusterSnapshotLog log = new ClusterSnapshotLog(); + log.setAutomatedSnapshotOff(); + GlobalStateMgr.getCurrentState().getEditLog().logClusterSnapshotLog(log); } protected void setAutomatedSnapshotOff() { @@ -132,14 +133,14 @@ public ClusterSnapshotJob createAutomatedSnapshotJob() { ClusterSnapshotJob job = new ClusterSnapshotJob(id, snapshotName, storageVolumeName, createTimeMs); job.logJob(); - addJob(job); - + addSnapshotJob(job); + LOG.info("Create automated cluster snapshot job successfully, job id: {}, snapshot name: {}", id, snapshotName); return job; } - public StorageVolume getAutomatedSnapshotSv() { + public StorageVolume getAutomatedSnapshotStorageVolume() { if (storageVolumeName == null) { return null; } @@ -166,7 +167,7 @@ public ClusterSnapshot getAutomatedSnapshot() { return job.getSnapshot(); } - public synchronized void addJob(ClusterSnapshotJob job) { + public void addSnapshotJob(ClusterSnapshotJob job) { automatedSnapshotJobs.put(job.getId(), job); int maxSize = Math.max(Config.max_historical_automated_cluster_snapshot_jobs, 2); @@ -175,7 +176,7 @@ public synchronized void addJob(ClusterSnapshotJob job) { } } - public synchronized long getValidDeletionTimeMsByAutomatedSnapshot() { + public long getSafeDeletionTimeMs() { if (!isAutomatedSnapshotOn()) { return Long.MAX_VALUE; } @@ -197,26 +198,26 @@ public synchronized long getValidDeletionTimeMsByAutomatedSnapshot() { return previousAutomatedSnapshotCreatedTimsMs; } - public synchronized boolean isTableSafeToDeleteTablet(long tableId) { + public boolean isTableSafeToDeleteTablet(long tableId) { if (!isAutomatedSnapshotOn()) { return true; } - boolean valid = true; + boolean safe = true; Map alterJobs = GlobalStateMgr.getCurrentState().getRollupHandler().getAlterJobsV2(); alterJobs.putAll(GlobalStateMgr.getCurrentState().getSchemaChangeHandler().getAlterJobsV2()); for (Map.Entry entry : alterJobs.entrySet()) { AlterJobV2 alterJob = entry.getValue(); if (alterJob.getTableId() == tableId) { - valid = (alterJob.getFinishedTimeMs() < getValidDeletionTimeMsByAutomatedSnapshot()); + safe = (alterJob.getFinishedTimeMs() < getSafeDeletionTimeMs()); break; } } - return valid; + return safe; } public boolean isDeletionSafeToExecute(long deletionCreatedTimeMs) { - return deletionCreatedTimeMs < getValidDeletionTimeMsByAutomatedSnapshot(); + return deletionCreatedTimeMs < getSafeDeletionTimeMs(); } public NavigableMap getAutomatedSnapshotJobs() { @@ -261,7 +262,7 @@ public void removeAutomatedFinalizeJobs(int removeCount) { --removeCount; } - if (removeCount == 0) { + if (removeCount <= 0) { break; } } @@ -271,22 +272,24 @@ public void removeAutomatedFinalizeJobs(int removeCount) { } } - public void startCheckpointScheduler(CheckpointController feController, CheckpointController starMgrController) { + public void start() { if (RunMode.isSharedDataMode() && clusterSnapshotCheckpointScheduler == null) { - clusterSnapshotCheckpointScheduler = new ClusterSnapshotCheckpointScheduler(feController, starMgrController); + clusterSnapshotCheckpointScheduler = new ClusterSnapshotCheckpointScheduler( + GlobalStateMgr.getCurrentState().getCheckpointController(), + StarMgrServer.getCurrentState().getCheckpointController()); clusterSnapshotCheckpointScheduler.start(); } } - public TClusterSnapshotJobsResponse getAllJobsInfo() { + public TClusterSnapshotJobsResponse getAllSnapshotJobsInfo() { TClusterSnapshotJobsResponse response = new TClusterSnapshotJobsResponse(); - for (Map.Entry entry : automatedSnapshotJobs.entrySet()) { - response.addToItems(entry.getValue().getInfo()); + for (ClusterSnapshotJob job : automatedSnapshotJobs.values()) { + response.addToItems(job.getInfo()); } return response; } - public TClusterSnapshotsResponse getAllInfo() { + public TClusterSnapshotsResponse getAllSnapshotsInfo() { TClusterSnapshotsResponse response = new TClusterSnapshotsResponse(); ClusterSnapshot automatedSnapshot = getAutomatedSnapshot(); if (isAutomatedSnapshotOn() && automatedSnapshot != null) { @@ -313,7 +316,7 @@ public void replayLog(ClusterSnapshotLog log) { switch (state) { case INITIALIZING: { - addJob(job); + addSnapshotJob(job); break; } case SNAPSHOTING: diff --git a/fe/fe-core/src/main/java/com/starrocks/lake/snapshot/ClusterSnapshotUtils.java b/fe/fe-core/src/main/java/com/starrocks/lake/snapshot/ClusterSnapshotUtils.java index fcb149a9fd9d74..fcf585b524bc89 100644 --- a/fe/fe-core/src/main/java/com/starrocks/lake/snapshot/ClusterSnapshotUtils.java +++ b/fe/fe-core/src/main/java/com/starrocks/lake/snapshot/ClusterSnapshotUtils.java @@ -23,7 +23,7 @@ public class ClusterSnapshotUtils { public static String getSnapshotImagePath(StorageVolume sv, String snapshotName) { return String.join("/", sv.getLocations().get(0), - GlobalStateMgr.getCurrentState().getStarOSAgent().getRawServiceId(), "meta/image", snapshotName); + GlobalStateMgr.getCurrentState().getStarOSAgent().getRawServiceId(), "meta/image", snapshotName); } public static void uploadAutomatedSnapshotToRemote(String snapshotName) throws StarRocksException { @@ -31,7 +31,7 @@ public static void uploadAutomatedSnapshotToRemote(String snapshotName) throws S return; } - StorageVolume sv = GlobalStateMgr.getCurrentState().getClusterSnapshotMgr().getAutomatedSnapshotSv(); + StorageVolume sv = GlobalStateMgr.getCurrentState().getClusterSnapshotMgr().getAutomatedSnapshotStorageVolume(); String snapshotImagePath = getSnapshotImagePath(sv, snapshotName); String localImagePath = GlobalStateMgr.getServingState().getImageDir(); @@ -43,7 +43,7 @@ public static void clearAutomatedSnapshotFromRemote(String snapshotName) throws return; } - StorageVolume sv = GlobalStateMgr.getCurrentState().getClusterSnapshotMgr().getAutomatedSnapshotSv(); + StorageVolume sv = GlobalStateMgr.getCurrentState().getClusterSnapshotMgr().getAutomatedSnapshotStorageVolume(); BrokerDesc brokerDesc = new BrokerDesc(sv.getProperties()); String snapshotImagePath = getSnapshotImagePath(sv, snapshotName); diff --git a/fe/fe-core/src/main/java/com/starrocks/lake/vacuum/AutovacuumDaemon.java b/fe/fe-core/src/main/java/com/starrocks/lake/vacuum/AutovacuumDaemon.java index 0a8063b3bae3a1..02a30f236a665d 100644 --- a/fe/fe-core/src/main/java/com/starrocks/lake/vacuum/AutovacuumDaemon.java +++ b/fe/fe-core/src/main/java/com/starrocks/lake/vacuum/AutovacuumDaemon.java @@ -167,8 +167,8 @@ private void vacuumPartitionImpl(Database db, OlapTable table, PhysicalPartition vacuumRequest.graceTimestamp = startTime / MILLISECONDS_PER_SECOND - Config.lake_autovacuum_grace_period_minutes * 60; vacuumRequest.graceTimestamp = Math.min(vacuumRequest.graceTimestamp, - Math.max(GlobalStateMgr.getCurrentState().getClusterSnapshotMgr() - .getValidDeletionTimeMsByAutomatedSnapshot() / MILLISECONDS_PER_SECOND, 1)); + Math.max(GlobalStateMgr.getCurrentState().getClusterSnapshotMgr() + .getSafeDeletionTimeMs() / MILLISECONDS_PER_SECOND, 1)); vacuumRequest.minActiveTxnId = minActiveTxnId; vacuumRequest.partitionId = partition.getId(); vacuumRequest.deleteTxnLog = needDeleteTxnLog; diff --git a/fe/fe-core/src/main/java/com/starrocks/load/loadv2/BrokerLoadJob.java b/fe/fe-core/src/main/java/com/starrocks/load/loadv2/BrokerLoadJob.java index f1f764813ab630..a63f7853a81296 100644 --- a/fe/fe-core/src/main/java/com/starrocks/load/loadv2/BrokerLoadJob.java +++ b/fe/fe-core/src/main/java/com/starrocks/load/loadv2/BrokerLoadJob.java @@ -64,7 +64,6 @@ import com.starrocks.metric.TableMetricsRegistry; import com.starrocks.persist.AlterLoadJobOperationLog; import com.starrocks.qe.ConnectContext; -import com.starrocks.qe.OriginStatement; import com.starrocks.qe.QeProcessorImpl; import com.starrocks.qe.scheduler.Coordinator; import com.starrocks.server.GlobalStateMgr; @@ -104,6 +103,7 @@ public class BrokerLoadJob extends BulkLoadJob { private ConnectContext context; private List newLoadingTasks = Lists.newArrayList(); private long writeDurationMs = 0; + private LoadStmt stmt; // only for log replay public BrokerLoadJob() { @@ -116,13 +116,14 @@ public void setConnectContext(ConnectContext context) { this.context = context; } - public BrokerLoadJob(long dbId, String label, BrokerDesc brokerDesc, OriginStatement originStmt, ConnectContext context) + public BrokerLoadJob(long dbId, String label, BrokerDesc brokerDesc, LoadStmt stmt, ConnectContext context) throws MetaNotFoundException { - super(dbId, label, originStmt); + super(dbId, label, stmt != null ? stmt.getOrigStmt() : null); this.timeoutSecond = Config.broker_load_default_timeout_second; this.brokerDesc = brokerDesc; this.jobType = EtlJobType.BROKER; this.context = context; + this.stmt = stmt; if (context != null) { this.warehouseId = context.getCurrentWarehouseId(); } @@ -304,6 +305,7 @@ private void createLoadingTask(Database db, BrokerPendingTaskAttachment attachme .setLoadJobType(TLoadJobType.BROKER) .setPriority(priority) .setOriginStmt(originStmt) + .setLoadStmt(stmt) .setPartialUpdateMode(mode) .setFileStatusList(attachment.getFileStatusByTable(aggKey)) .setFileNum(attachment.getFileNumByTable(aggKey)) diff --git a/fe/fe-core/src/main/java/com/starrocks/load/loadv2/BulkLoadJob.java b/fe/fe-core/src/main/java/com/starrocks/load/loadv2/BulkLoadJob.java index 5f1f779296e828..457fb2505d6022 100644 --- a/fe/fe-core/src/main/java/com/starrocks/load/loadv2/BulkLoadJob.java +++ b/fe/fe-core/src/main/java/com/starrocks/load/loadv2/BulkLoadJob.java @@ -135,7 +135,7 @@ public static BulkLoadJob fromLoadStmt(LoadStmt stmt, ConnectContext context) th switch (stmt.getEtlJobType()) { case BROKER: bulkLoadJob = new BrokerLoadJob(db.getId(), stmt.getLabel().getLabelName(), - stmt.getBrokerDesc(), stmt.getOrigStmt(), context); + stmt.getBrokerDesc(), stmt, context); break; case SPARK: bulkLoadJob = new SparkLoadJob(db.getId(), stmt.getLabel().getLabelName(), 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 cbf0b466f10201..e7ef53c238f6b9 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 @@ -58,6 +58,9 @@ import com.starrocks.qe.scheduler.Coordinator; import com.starrocks.server.GlobalStateMgr; import com.starrocks.sql.LoadPlanner; +import com.starrocks.sql.analyzer.AstToSQLBuilder; +import com.starrocks.sql.ast.LoadStmt; +import com.starrocks.sql.common.AuditEncryptionChecker; import com.starrocks.thrift.TBrokerFileStatus; import com.starrocks.thrift.TLoadJobType; import com.starrocks.thrift.TPartialUpdateMode; @@ -100,6 +103,7 @@ public class LoadLoadingTask extends LoadTask { private LoadPlanner loadPlanner; private final OriginStatement originStmt; + private final LoadStmt loadStmt; private final List> fileStatusList; private final int fileNum; @@ -125,6 +129,7 @@ private LoadLoadingTask(Builder builder) { this.context = builder.context; this.loadJobType = builder.loadJobType; this.originStmt = builder.originStmt; + this.loadStmt = builder.loadStmt; this.partialUpdateMode = builder.partialUpdateMode; this.failMsg = new FailMsg(FailMsg.CancelType.LOAD_RUN_FAIL); this.loadId = builder.loadId; @@ -188,7 +193,12 @@ public RuntimeProfile buildTopLevelProfile(boolean isFinished) { String.format("%s-%s", Version.STARROCKS_VERSION, Version.STARROCKS_COMMIT_HASH)); summaryProfile.addInfoString(ProfileManager.USER, context.getQualifiedUser()); summaryProfile.addInfoString(ProfileManager.DEFAULT_DB, context.getDatabase()); - summaryProfile.addInfoString(ProfileManager.SQL_STATEMENT, originStmt.originStmt); + if (AuditEncryptionChecker.needEncrypt(loadStmt)) { + summaryProfile.addInfoString(ProfileManager.SQL_STATEMENT, + AstToSQLBuilder.toSQLOrDefault(loadStmt, originStmt.originStmt)); + } else { + summaryProfile.addInfoString(ProfileManager.SQL_STATEMENT, originStmt.originStmt); + } summaryProfile.addInfoString("Timeout", DebugUtil.getPrettyStringMs(timeoutS * 1000)); summaryProfile.addInfoString("Strict Mode", String.valueOf(strictMode)); summaryProfile.addInfoString("Partial Update", String.valueOf(partialUpdate)); @@ -327,6 +337,7 @@ public static class Builder { private TPartialUpdateMode partialUpdateMode; private ConnectContext context; private OriginStatement originStmt; + private LoadStmt loadStmt; private List> fileStatusList; private int fileNum = 0; private LoadTaskCallback callback; @@ -440,6 +451,11 @@ public Builder setOriginStmt(OriginStatement originStmt) { return this; } + public Builder setLoadStmt(LoadStmt loadStmt) { + this.loadStmt = loadStmt; + return this; + } + public Builder setFileStatusList(List> fileStatusList) { this.fileStatusList = fileStatusList; return this; diff --git a/fe/fe-core/src/main/java/com/starrocks/planner/OlapScanNode.java b/fe/fe-core/src/main/java/com/starrocks/planner/OlapScanNode.java index 455f266ac645d1..fb2636b1ca1ddf 100644 --- a/fe/fe-core/src/main/java/com/starrocks/planner/OlapScanNode.java +++ b/fe/fe-core/src/main/java/com/starrocks/planner/OlapScanNode.java @@ -196,6 +196,12 @@ public class OlapScanNode extends ScanNode { // Set just once per query. private boolean alreadyFoundSomeLivingCn = false; + boolean enableTopnFilterBackPressure = false; + long backPressureThrottleTimeUpperBound = -1; + int backPressureMaxRounds = -1; + long backPressureThrottleTime = -1; + long backPressureNumRows = -1; + // Constructs node to scan given data files of table 'tbl'. // Constructs node to scan given data files of table 'tbl'. public OlapScanNode(PlanNodeId id, TupleDescriptor desc, String planNodeName) { @@ -1000,13 +1006,19 @@ protected void toThrift(TPlanNode msg) { } assignOrderByHints(keyColumnNames); - if (olapTable.isCloudNativeTableOrMaterializedView()) { msg.node_type = TPlanNodeType.LAKE_SCAN_NODE; msg.lake_scan_node = new TLakeScanNode(desc.getId().asInt(), keyColumnNames, keyColumnTypes, isPreAggregation); msg.lake_scan_node.setSort_key_column_names(keyColumnNames); msg.lake_scan_node.setRollup_name(olapTable.getIndexNameById(selectedIndexId)); + if (enableTopnFilterBackPressure) { + msg.lake_scan_node.setEnable_topn_filter_back_pressure(true); + msg.lake_scan_node.setBack_pressure_max_rounds(backPressureMaxRounds); + msg.lake_scan_node.setBack_pressure_num_rows(backPressureNumRows); + msg.lake_scan_node.setBack_pressure_throttle_time(backPressureThrottleTime); + msg.lake_scan_node.setBack_pressure_throttle_time_upper_bound(backPressureThrottleTimeUpperBound); + } if (!conjuncts.isEmpty()) { msg.lake_scan_node.setSql_predicates(getExplainString(conjuncts)); } @@ -1045,6 +1057,13 @@ protected void toThrift(TPlanNode msg) { msg.olap_scan_node.setSchema_id(schemaId); msg.olap_scan_node.setSort_key_column_names(keyColumnNames); msg.olap_scan_node.setRollup_name(olapTable.getIndexNameById(selectedIndexId)); + if (enableTopnFilterBackPressure) { + msg.olap_scan_node.setEnable_topn_filter_back_pressure(true); + msg.olap_scan_node.setBack_pressure_max_rounds(backPressureMaxRounds); + msg.olap_scan_node.setBack_pressure_num_rows(backPressureNumRows); + msg.olap_scan_node.setBack_pressure_throttle_time(backPressureThrottleTime); + msg.olap_scan_node.setBack_pressure_throttle_time_upper_bound(backPressureThrottleTimeUpperBound); + } if (!conjuncts.isEmpty()) { msg.olap_scan_node.setSql_predicates(getExplainString(conjuncts)); } @@ -1516,4 +1535,27 @@ public void clearScanNodeForThriftBuild() { public boolean isRunningAsConnectorOperator() { return false; } + + @Override + public boolean pushDownRuntimeFilters(RuntimeFilterPushDownContext context, Expr probeExpr, + List partitionByExprs) { + boolean accept = super.pushDownRuntimeFilters(context, probeExpr, partitionByExprs); + if (accept && context.getDescription().runtimeFilterType() + .equals(RuntimeFilterDescription.RuntimeFilterType.TOPN_FILTER)) { + boolean toManyData = this.getCardinality() != -1 && this.cardinality > 50000000; + int backPressureMode = Optional.ofNullable(ConnectContext.get()) + .map(ctx -> ctx.getSessionVariable().getTopnFilterBackPressureMode()) + .orElse(0); + if ((backPressureMode == 1 && toManyData) || backPressureMode == 2) { + this.enableTopnFilterBackPressure = true; + this.backPressureMaxRounds = ConnectContext.get().getSessionVariable().getBackPressureMaxRounds(); + this.backPressureThrottleTimeUpperBound = + ConnectContext.get().getSessionVariable().getBackPressureThrottleTimeUpperBound(); + this.backPressureNumRows = 10 * context.getDescription().getTopN(); + this.backPressureThrottleTime = this.backPressureThrottleTimeUpperBound / + Math.max(this.backPressureMaxRounds, 1); + } + } + return accept; + } } diff --git a/fe/fe-core/src/main/java/com/starrocks/planner/RuntimeFilterDescription.java b/fe/fe-core/src/main/java/com/starrocks/planner/RuntimeFilterDescription.java index 282f355ad5de4b..fac3ddb7c13165 100644 --- a/fe/fe-core/src/main/java/com/starrocks/planner/RuntimeFilterDescription.java +++ b/fe/fe-core/src/main/java/com/starrocks/planner/RuntimeFilterDescription.java @@ -75,6 +75,8 @@ public enum RuntimeFilterType { private boolean onlyLocal; + private long topn; + // ExecGroupInfo. used for check build colocate runtime filter private boolean isBuildFromColocateGroup = false; private int execGroupId = -1; @@ -149,6 +151,14 @@ public void setSortInfo(SortInfo sortInfo) { this.sortInfo = sortInfo; } + public void setTopN(long value) { + this.topn = value; + } + + public long getTopN() { + return this.topn; + } + public boolean canProbeUse(PlanNode node, RuntimeFilterPushDownContext rfPushCtx) { if (!canAcceptFilter(node, rfPushCtx)) { return false; diff --git a/fe/fe-core/src/main/java/com/starrocks/planner/SortNode.java b/fe/fe-core/src/main/java/com/starrocks/planner/SortNode.java index 721f4197580768..ed553f45ded485 100644 --- a/fe/fe-core/src/main/java/com/starrocks/planner/SortNode.java +++ b/fe/fe-core/src/main/java/com/starrocks/planner/SortNode.java @@ -181,6 +181,7 @@ public void buildRuntimeFilters(IdGenerator generator, Descript rf.setSortInfo(getSortInfo()); rf.setBuildExpr(orderBy); rf.setRuntimeFilterType(RuntimeFilterDescription.RuntimeFilterType.TOPN_FILTER); + rf.setTopN(offset < 0 ? limit : offset + limit); RuntimeFilterPushDownContext rfPushDownCtx = new RuntimeFilterPushDownContext(rf, descTbl, execGroupSets); for (PlanNode child : children) { if (child.pushDownRuntimeFilters(rfPushDownCtx, orderBy, Lists.newArrayList())) { diff --git a/fe/fe-core/src/main/java/com/starrocks/plugin/AuditEvent.java b/fe/fe-core/src/main/java/com/starrocks/plugin/AuditEvent.java index 9708b370236446..27f2562694190d 100644 --- a/fe/fe-core/src/main/java/com/starrocks/plugin/AuditEvent.java +++ b/fe/fe-core/src/main/java/com/starrocks/plugin/AuditEvent.java @@ -142,6 +142,8 @@ public enum EventType { @AuditField(value = "Features", ignore_zero = true) public String features; + @AuditField(value = "PredictMemBytes", ignore_zero = true) + public long predictMemBytes = 0; @AuditField(value = "IsForwardToLeader") public boolean isForwardToLeader = false; @@ -305,6 +307,11 @@ public AuditEventBuilder setNumSlots(int numSlots) { return this; } + public AuditEventBuilder setPredictMemBytes(long memBytes) { + auditEvent.predictMemBytes = memBytes; + return this; + } + public AuditEventBuilder setBigQueryLogCPUSecondThreshold(long bigQueryLogCPUSecondThreshold) { auditEvent.bigQueryLogCPUSecondThreshold = bigQueryLogCPUSecondThreshold; return this; 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 c498538bd804bb..6e60b073c0f422 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 @@ -167,6 +167,7 @@ public class DefaultCoordinator extends Coordinator { private boolean isShortCircuit = false; private boolean isBinaryRow = false; + private long estimatedMemCost; private ExecutionSchedule scheduler; public static class Factory implements Coordinator.Factory { @@ -403,6 +404,15 @@ public void setTimeoutSecond(int timeoutSecond) { jobSpec.setQueryTimeout(timeoutSecond); } + @Override + public void setPredictedCost(long memBytes) { + this.estimatedMemCost = memBytes; + } + + public long getPredictedCost() { + return estimatedMemCost; + } + @Override public void clearExportStatus() { lock.lock(); @@ -578,7 +588,10 @@ public Status scheduleNextTurn(TUniqueId fragmentInstanceId) { @Override public String getSchedulerExplain() { - return executionDAG.getFragmentsInPreorder().stream() + String predict = Config.enable_query_cost_prediction ? + "predicted memory cost: " + getPredictedCost() + "\n" : ""; + return predict + + executionDAG.getFragmentsInPreorder().stream() .map(ExecutionFragment::getExplainString) .collect(Collectors.joining("\n")); } 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 1dc09971c11237..9b7cbf664ed28e 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 @@ -855,6 +855,11 @@ public static MaterializedViewRewriteMode parse(String str) { public static final String ENABLE_SCAN_PREDICATE_EXPR_REUSE = "enable_scan_predicate_expr_reuse"; + // 0 for disable, 1 for too many data; 2 for force + public static final String TOPN_FILTER_BACK_PRESSURE_MODE = "topn_filter_back_pressure_mode"; + public static final String BACK_PRESSURE_MAX_ROUNDS = "back_pressure_back_rounds"; + public static final String BACK_PRESSURE_THROTTLE_TIME_UPPER_BOUND = "back_pressure_throttle_time_upper_bound"; + public static final List DEPRECATED_VARIABLES = ImmutableList.builder() .add(CODEGEN_LEVEL) .add(MAX_EXECUTION_TIME) @@ -1679,6 +1684,13 @@ public static MaterializedViewRewriteMode parse(String str) { @VarAttr(name = ENABLE_SCAN_PREDICATE_EXPR_REUSE, flag = VariableMgr.INVISIBLE) private boolean enableScanPredicateExprReuse = true; + @VarAttr(name = TOPN_FILTER_BACK_PRESSURE_MODE) + private int topnFilterBackPressureMode = 0; + @VarAttr(name = BACK_PRESSURE_MAX_ROUNDS) + private int backPressureMaxRounds = 3; + @VarAttr(name = BACK_PRESSURE_THROTTLE_TIME_UPPER_BOUND) + private long backPressureThrottleTimeUpperBound = 300; + public int getCboPruneJsonSubfieldDepth() { return cboPruneJsonSubfieldDepth; } @@ -4575,6 +4587,30 @@ public boolean isEnableRewriteUnnestBitmapToArray() { return enableRewriteUnnestBitmapToArray; } + public int getTopnFilterBackPressureMode() { + return topnFilterBackPressureMode; + } + + public void setTopnFilterBackPressureMode(int value) { + this.topnFilterBackPressureMode = value; + } + + public int getBackPressureMaxRounds() { + return this.backPressureMaxRounds; + } + + public void setBackPressureMaxRounds(int value) { + this.backPressureMaxRounds = value; + } + + public long getBackPressureThrottleTimeUpperBound() { + return this.backPressureThrottleTimeUpperBound; + } + + public void setBackPressureThrottleTimeUpperBound(long value) { + this.backPressureThrottleTimeUpperBound = value; + } + // Serialize to thrift object // used for rest api public TQueryOptions toThrift() { 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 db3b67a7aabce5..bd48c470471efd 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 @@ -199,6 +199,7 @@ import com.starrocks.sql.common.MetaUtils; import com.starrocks.sql.common.StarRocksPlannerException; import com.starrocks.sql.optimizer.OptExpression; +import com.starrocks.sql.optimizer.cost.feature.CostPredictor; import com.starrocks.sql.optimizer.dump.QueryDumpInfo; import com.starrocks.sql.optimizer.operator.physical.PhysicalValuesOperator; import com.starrocks.sql.optimizer.operator.scalar.ScalarOperator; @@ -1227,6 +1228,10 @@ private Coordinator.Factory getCoordinatorFactory() { return new DefaultCoordinator.Factory(); } + private CostPredictor getCostPredictor() { + return CostPredictor.getServiceBasedCostPredictor(); + } + // Process a select statement. private void handleQueryStmt(ExecPlan execPlan) throws Exception { // Every time set no send flag and clean all data in buffer @@ -1291,6 +1296,14 @@ private void handleQueryStmt(ExecPlan execPlan) throws Exception { coord = getCoordinatorFactory().createQueryScheduler(context, fragments, scanNodes, descTable); } + // Predict the cost of this query + if (Config.enable_query_cost_prediction) { + CostPredictor predictor = getCostPredictor(); + long memBytes = predictor.predictMemoryBytes(execPlan); + coord.setPredictedCost(memBytes); + context.getAuditEventBuilder().setPredictMemBytes(memBytes); + } + QeProcessorImpl.INSTANCE.registerQuery(context.getExecutionId(), new QeProcessorImpl.QueryInfo(context, originStmt.originStmt, coord)); 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 63be518a9094ba..d00a09346026ad 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 @@ -253,6 +253,9 @@ public static List getCommitInfos(Coordinator coord) { public abstract void setTimeoutSecond(int timeoutSecond); + public void setPredictedCost(long memBytes) { + } + public abstract boolean isProfileAlreadyReported(); public abstract String getWarehouseName(); diff --git a/fe/fe-core/src/main/java/com/starrocks/qe/scheduler/slot/SlotEstimatorFactory.java b/fe/fe-core/src/main/java/com/starrocks/qe/scheduler/slot/SlotEstimatorFactory.java index 996957e96e7c8b..9d0d9c6c9ea867 100644 --- a/fe/fe-core/src/main/java/com/starrocks/qe/scheduler/slot/SlotEstimatorFactory.java +++ b/fe/fe-core/src/main/java/com/starrocks/qe/scheduler/slot/SlotEstimatorFactory.java @@ -22,6 +22,7 @@ import com.starrocks.planner.PlanNode; import com.starrocks.qe.ConnectContext; import com.starrocks.qe.DefaultCoordinator; +import com.starrocks.sql.optimizer.cost.feature.CostPredictor; import com.starrocks.thrift.TScanRangeLocation; import com.starrocks.thrift.TScanRangeLocations; @@ -30,7 +31,7 @@ import java.util.Map; import java.util.stream.Collectors; -import static com.starrocks.sql.optimizer.Utils.computeMaxLEPower2; +import static com.starrocks.sql.optimizer.Utils.computeMinGEPower2; public class SlotEstimatorFactory { public static SlotEstimator create(QueryQueueOptions opts) { @@ -50,10 +51,15 @@ public int estimateSlots(QueryQueueOptions opts, ConnectContext context, Default public static class MemoryBasedSlotsEstimator implements SlotEstimator { @Override public int estimateSlots(QueryQueueOptions opts, ConnectContext context, DefaultCoordinator coord) { - final long planMemCosts = (long) context.getAuditEventBuilder().build().planMemCosts; - long numSlotsPerWorker = planMemCosts / opts.v2().getNumWorkers() / opts.v2().getMemBytesPerSlot(); + long memCost; + if (CostPredictor.getServiceBasedCostPredictor().isAvailable() && coord.getPredictedCost() > 0) { + memCost = coord.getPredictedCost(); + } else { + memCost = (long) context.getAuditEventBuilder().build().planMemCosts; + } + long numSlotsPerWorker = memCost / opts.v2().getMemBytesPerSlot(); numSlotsPerWorker = Math.max(numSlotsPerWorker, 0); - numSlotsPerWorker = computeMaxLEPower2((int) numSlotsPerWorker); + numSlotsPerWorker = computeMinGEPower2((int) numSlotsPerWorker); long numSlots = numSlotsPerWorker * opts.v2().getNumWorkers(); numSlots = Math.max(numSlots, 1); diff --git a/fe/fe-core/src/main/java/com/starrocks/server/GlobalStateMgr.java b/fe/fe-core/src/main/java/com/starrocks/server/GlobalStateMgr.java index 3f6cfdcdf212c2..562b470fa0345b 100644 --- a/fe/fe-core/src/main/java/com/starrocks/server/GlobalStateMgr.java +++ b/fe/fe-core/src/main/java/com/starrocks/server/GlobalStateMgr.java @@ -1441,8 +1441,7 @@ private void startLeaderOnlyDaemonThreads() { temporaryTableCleaner.start(); if (RunMode.isSharedDataMode()) { - clusterSnapshotMgr.startCheckpointScheduler(checkpointController, - StarMgrServer.getCurrentState().getCheckpointController()); + clusterSnapshotMgr.start(); } reportHandler.start(); tabletCollector.start(); diff --git a/fe/fe-core/src/main/java/com/starrocks/server/NodeMgr.java b/fe/fe-core/src/main/java/com/starrocks/server/NodeMgr.java index d30ef85a71f7ac..9f8810b08de1ca 100644 --- a/fe/fe-core/src/main/java/com/starrocks/server/NodeMgr.java +++ b/fe/fe-core/src/main/java/com/starrocks/server/NodeMgr.java @@ -41,6 +41,7 @@ import com.google.gson.stream.JsonReader; import com.starrocks.catalog.BrokerMgr; import com.starrocks.common.Config; +import com.starrocks.common.ConfigBase; import com.starrocks.common.DdlException; import com.starrocks.common.Pair; import com.starrocks.common.util.NetUtils; @@ -1144,6 +1145,12 @@ public List getQueryStatisticsInfoFromOtherFEs() { return statisticsItems; } + public void setFrontendConfig(Map configs, boolean isPersisted, String userIdentity) throws DdlException { + for (Map.Entry entry : configs.entrySet()) { + ConfigBase.setMutableConfig(entry.getKey(), entry.getValue(), isPersisted, userIdentity); + } + } + public Frontend getMySelf() { return frontends.get(nodeName); } diff --git a/fe/fe-core/src/main/java/com/starrocks/service/FrontendServiceImpl.java b/fe/fe-core/src/main/java/com/starrocks/service/FrontendServiceImpl.java index c0790472fe2a27..254ba418f578b5 100644 --- a/fe/fe-core/src/main/java/com/starrocks/service/FrontendServiceImpl.java +++ b/fe/fe-core/src/main/java/com/starrocks/service/FrontendServiceImpl.java @@ -1947,8 +1947,8 @@ public TSetConfigResponse setConfig(TSetConfigRequest request) throws TException } } } - - ConfigBase.setFrontendConfig(configs); + ConfigBase.setFrontendConfig(configs, request.isIs_persistent(), + request.getUser_identity()); return new TSetConfigResponse(new TStatus(TStatusCode.OK)); } catch (DdlException e) { TStatus status = new TStatus(TStatusCode.INTERNAL_ERROR); @@ -3179,11 +3179,11 @@ static List getPartitionMetaImpl(Collection tabletMe @Override public TClusterSnapshotsResponse getClusterSnapshotsInfo(TClusterSnapshotsRequest params) { - return GlobalStateMgr.getCurrentState().getClusterSnapshotMgr().getAllInfo(); + return GlobalStateMgr.getCurrentState().getClusterSnapshotMgr().getAllSnapshotsInfo(); } @Override public TClusterSnapshotJobsResponse getClusterSnapshotJobsInfo(TClusterSnapshotJobsRequest params) { - return GlobalStateMgr.getCurrentState().getClusterSnapshotMgr().getAllJobsInfo(); + return GlobalStateMgr.getCurrentState().getClusterSnapshotMgr().getAllSnapshotJobsInfo(); } } diff --git a/fe/fe-core/src/main/java/com/starrocks/sql/ast/AdminSetConfigStmt.java b/fe/fe-core/src/main/java/com/starrocks/sql/ast/AdminSetConfigStmt.java index 3921478ff2dc64..a2dafe203fefa9 100644 --- a/fe/fe-core/src/main/java/com/starrocks/sql/ast/AdminSetConfigStmt.java +++ b/fe/fe-core/src/main/java/com/starrocks/sql/ast/AdminSetConfigStmt.java @@ -28,11 +28,13 @@ public enum ConfigType { private final ConfigType type; private Property config; + private final boolean persistent; - public AdminSetConfigStmt(ConfigType type, Property config, NodePosition pos) { + public AdminSetConfigStmt(ConfigType type, Property config, boolean persistent, NodePosition pos) { super(pos); this.type = type; this.config = config; + this.persistent = persistent; } public ConfigType getType() { @@ -43,9 +45,13 @@ public Property getConfig() { return config; } + public boolean isPersistent() { + return persistent; + } + @Override public RedirectStatus getRedirectStatus() { - return RedirectStatus.NO_FORWARD; + return RedirectStatus.FORWARD_NO_SYNC; } @Override diff --git a/fe/fe-core/src/main/java/com/starrocks/sql/optimizer/cost/feature/CostPredictor.java b/fe/fe-core/src/main/java/com/starrocks/sql/optimizer/cost/feature/CostPredictor.java new file mode 100644 index 00000000000000..c4ab352d016453 --- /dev/null +++ b/fe/fe-core/src/main/java/com/starrocks/sql/optimizer/cost/feature/CostPredictor.java @@ -0,0 +1,149 @@ +// 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.sql.optimizer.cost.feature; + +import com.google.common.annotations.VisibleForTesting; +import com.starrocks.common.Config; +import com.starrocks.sql.plan.ExecPlan; +import org.apache.http.HttpEntity; +import org.apache.http.HttpStatus; +import org.apache.http.client.methods.CloseableHttpResponse; +import org.apache.http.client.methods.HttpGet; +import org.apache.http.client.methods.HttpPost; +import org.apache.http.entity.StringEntity; +import org.apache.http.impl.client.CloseableHttpClient; +import org.apache.http.impl.client.HttpClients; +import org.apache.http.util.EntityUtils; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.io.Closeable; +import java.io.IOException; +import java.time.Duration; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +public abstract class CostPredictor { + + public abstract long predictMemoryBytes(ExecPlan plan); + + public static ServiceBasedCostPredictor getServiceBasedCostPredictor() { + return ServiceBasedCostPredictor.getInstance(); + } + + /** + * Use a remote HTTP service to predict the query cost + */ + public static class ServiceBasedCostPredictor extends CostPredictor implements Closeable { + + public static final String PREDICT_URL = "/predict_csv"; + public static final String HEALTH_URL = "/health_check"; + private static final ServiceBasedCostPredictor INSTANCE = new ServiceBasedCostPredictor(); + private static final Logger LOG = LogManager.getLogger(ServiceBasedCostPredictor.class); + + private static final ScheduledExecutorService DAEMON; + private static final Duration HEALTH_CHECK_INTERVAL = Duration.ofSeconds(30); + private volatile int lastHealthCheckStatusCode = HttpStatus.SC_OK; + + static { + DAEMON = Executors.newSingleThreadScheduledExecutor(); + DAEMON.scheduleAtFixedRate( + () -> ServiceBasedCostPredictor.getInstance().healthCheck(), + HEALTH_CHECK_INTERVAL.getSeconds(), + HEALTH_CHECK_INTERVAL.getSeconds(), + TimeUnit.SECONDS); + } + + private final CloseableHttpClient httpClient = HttpClients.createDefault(); + + /** + * Return the singleton instance of predictor, which is thread-safe to be shared among threads + */ + public static ServiceBasedCostPredictor getInstance() { + return INSTANCE; + } + + private ServiceBasedCostPredictor() { + } + + @Override + public long predictMemoryBytes(ExecPlan plan) { + PlanFeatures planFeatures = FeatureExtractor.extractFeatures(plan.getPhysicalPlan()); + String header = PlanFeatures.featuresHeader(); + String featureString = planFeatures.toFeatureCsv(); + + try { + // Use Apache HttpClient to send the HTTP request + HttpPost httpPost = new HttpPost(Config.query_cost_prediction_service_address + PREDICT_URL); + + // Encode the request in CSV format + String csvData = header + "\n" + featureString; + StringEntity entity = new StringEntity(csvData); + entity.setContentType("text/csv"); + httpPost.setEntity(entity); + + CloseableHttpResponse response = httpClient.execute(httpPost); + int status = response.getStatusLine().getStatusCode(); + if (status == HttpStatus.SC_OK) { + HttpEntity responseEntity = response.getEntity(); + String responseBody = EntityUtils.toString(responseEntity); + return (long) Double.parseDouble(responseBody); + } else { + // Handle the error + throw new IOException("Failed to predict memory bytes: HTTP error code " + status); + } + } catch (IOException e) { + // Log the error or handle it appropriately + throw new RuntimeException("Failed to predict memory bytes", e); + } + } + + @Override + public void close() throws IOException { + httpClient.close(); + } + + public boolean isAvailable() { + return Config.enable_query_cost_prediction && lastHealthCheckStatusCode == HttpStatus.SC_OK; + } + + @VisibleForTesting + protected void doHealthCheck() { + healthCheck(); + } + + /** + * Do health check of the service + */ + private void healthCheck() { + if (!Config.enable_query_cost_prediction) { + return; + } + String address = Config.query_cost_prediction_service_address + HEALTH_URL; + HttpGet httpGet = new HttpGet(address); + try { + CloseableHttpResponse response = httpClient.execute(httpGet); + lastHealthCheckStatusCode = response.getStatusLine().getStatusCode(); + if (lastHealthCheckStatusCode != HttpStatus.SC_OK) { + LOG.warn("service is not healthy, address={} status_code={}", address, lastHealthCheckStatusCode); + } + } catch (Throwable e) { + lastHealthCheckStatusCode = HttpStatus.SC_INTERNAL_SERVER_ERROR; + LOG.warn("service is not healthy, address={}", address, e); + } + } + } +} diff --git a/fe/fe-core/src/main/java/com/starrocks/sql/optimizer/cost/feature/PlanFeatures.java b/fe/fe-core/src/main/java/com/starrocks/sql/optimizer/cost/feature/PlanFeatures.java index c61957465ebb39..4b601eca631b2b 100644 --- a/fe/fe-core/src/main/java/com/starrocks/sql/optimizer/cost/feature/PlanFeatures.java +++ b/fe/fe-core/src/main/java/com/starrocks/sql/optimizer/cost/feature/PlanFeatures.java @@ -39,6 +39,26 @@ public class PlanFeatures { ); // query plan + private static final int TOP_N_TABLES = 3; + private static final int NUM_ENV_FEATURES = 3; + private static final int NUM_VAR_FEATURES = 1; + private static final int NUM_OPERATOR_FEATURES; + + static { + int num = 0; + for (int start = OperatorType.PHYSICAL.ordinal() + 1; + start < OperatorType.SCALAR.ordinal(); + start++) { + OperatorType opType = OperatorType.values()[start]; + if (skipOperator(opType)) { + continue; + } + num += OperatorFeatures.vectorLength(opType); + num += AggregatedFeature.numExtraFeatures(opType); + } + NUM_OPERATOR_FEATURES = num; + } + private List operatorFeatureVectors; private final Set tables = Sets.newHashSet(); @@ -50,6 +70,24 @@ public class PlanFeatures { // variables private long dop; + public static String featuresHeader() { + final String DELIMITER = ","; + List fields = Lists.newArrayList(); + for (int i = 0; i < TOP_N_TABLES; i++) { + fields.add("tables_" + i); + } + for (int i = 0; i < NUM_ENV_FEATURES; i++) { + fields.add("env_" + i); + } + for (int i = 0; i < NUM_VAR_FEATURES; i++) { + fields.add("var_" + i); + } + for (int i = 0; i < NUM_OPERATOR_FEATURES; i++) { + fields.add("operators_" + i); + } + + return Joiner.on(DELIMITER).join(fields); + } /** * The string representation like: tables=[1,2,3]|operators=[4,5,6]|.... @@ -69,6 +107,34 @@ public String toFeatureString() { return sb.toString(); } + /** + * The CSV representation like tables_0,tables_1,tables_2,operators_0,operators_1.... + * + * @return CSV format string + */ + public String toFeatureCsv() { + final String DELIMITER = ","; + StringBuilder sb = new StringBuilder(); + + var topTables = extractTopTables(); + for (var tableId : topTables) { + sb.append(String.valueOf(tableId)).append(DELIMITER); + } + // env + sb.append(numBeNodes).append(DELIMITER); + sb.append(avgCpuCoreOfBE).append(DELIMITER); + sb.append((long) Math.log1p(memCapacityOfBE)).append(DELIMITER); + // var + sb.append(dop).append(DELIMITER); + // operators + for (long operatorFeature : operatorFeatureVectors) { + sb.append(operatorFeature).append(DELIMITER); + } + + String result = sb.toString(); + return result.substring(0, result.length() - 1); + } + public void setNumBeNodes(int numBeNodes) { this.numBeNodes = numBeNodes; } @@ -109,7 +175,7 @@ public void addOperatorFeatures(Map operatorFea this.operatorFeatureVectors = operatorVector; } - public boolean skipOperator(OperatorType operatorType) { + public static boolean skipOperator(OperatorType operatorType) { if (EXCLUDE_OPERATORS.contains(operatorType)) { return true; } @@ -140,7 +206,6 @@ private static void aggregate(OperatorFeatures tree, Map extractTopTables() { - final int TOP_N_TABLES = 3; List result = Lists.newArrayList(); for (int i = 0; i < TOP_N_TABLES; i++) { @@ -196,6 +261,10 @@ public List finish() { return result; } + public static int numExtraFeatures(OperatorType type) { + return 2; + } + public static List empty(OperatorType type) { List result = Lists.newArrayList(); result.add((long) type.ordinal()); diff --git a/fe/fe-core/src/main/java/com/starrocks/sql/parser/AstBuilder.java b/fe/fe-core/src/main/java/com/starrocks/sql/parser/AstBuilder.java index 5d6ef9878a1dd9..395f1c7f120b83 100644 --- a/fe/fe-core/src/main/java/com/starrocks/sql/parser/AstBuilder.java +++ b/fe/fe-core/src/main/java/com/starrocks/sql/parser/AstBuilder.java @@ -2491,7 +2491,8 @@ public ParseNode visitShowStreamLoadStatement(StarRocksParser.ShowStreamLoadStat @Override public ParseNode visitAdminSetConfigStatement(StarRocksParser.AdminSetConfigStatementContext context) { Property config = (Property) visitProperty(context.property()); - return new AdminSetConfigStmt(AdminSetConfigStmt.ConfigType.FRONTEND, config, createPos(context)); + boolean persistent = context.PERSISTENT() != null; + return new AdminSetConfigStmt(AdminSetConfigStmt.ConfigType.FRONTEND, config, persistent, createPos(context)); } @Override diff --git a/fe/fe-core/src/main/java/com/starrocks/sql/parser/StarRocks.g4 b/fe/fe-core/src/main/java/com/starrocks/sql/parser/StarRocks.g4 index ae4c7fb30159c2..b82164755c1180 100644 --- a/fe/fe-core/src/main/java/com/starrocks/sql/parser/StarRocks.g4 +++ b/fe/fe-core/src/main/java/com/starrocks/sql/parser/StarRocks.g4 @@ -698,7 +698,7 @@ cancelRefreshMaterializedViewStatement // ------------------------------------------- Admin Statement --------------------------------------------------------- adminSetConfigStatement - : ADMIN SET FRONTEND CONFIG '(' property ')' + : ADMIN SET FRONTEND CONFIG '(' property ')' (WITH PERSISTENT)? ; adminSetReplicaStatusStatement : ADMIN SET REPLICA STATUS properties diff --git a/fe/fe-core/src/main/java/com/starrocks/task/TabletMetadataUpdateAgentTaskFactory.java b/fe/fe-core/src/main/java/com/starrocks/task/TabletMetadataUpdateAgentTaskFactory.java index 1fcd5f8593766a..c30f2e1f9a6727 100644 --- a/fe/fe-core/src/main/java/com/starrocks/task/TabletMetadataUpdateAgentTaskFactory.java +++ b/fe/fe-core/src/main/java/com/starrocks/task/TabletMetadataUpdateAgentTaskFactory.java @@ -314,7 +314,9 @@ private UpdateTabletSchemaTask(long backendId, List tablets, TTabletSchema boolean createSchemaFile) { super(backendId, tablets.hashCode()); this.tablets = new ArrayList<>(tablets); - this.tabletSchema = Objects.requireNonNull(tabletSchema, "tabletSchema is null"); + // tabletSchema may be null when the table has multi materialized index + // and the schema of some materialized indexes are not needed to be updated + this.tabletSchema = tabletSchema; this.createSchemaFile = createSchemaFile; } @@ -330,7 +332,11 @@ public List getTTabletMetaInfoList() { for (Long tabletId : tablets) { TTabletMetaInfo metaInfo = new TTabletMetaInfo(); metaInfo.setTablet_id(tabletId); - metaInfo.setTablet_schema(tabletSchema); + + if (tabletSchema != null) { + metaInfo.setTablet_schema(tabletSchema); + } + metaInfos.add(metaInfo); metaInfo.setCreate_schema_file(create); create = false; diff --git a/fe/fe-core/src/test/java/com/starrocks/alter/LakeSyncMaterializedViewTest.java b/fe/fe-core/src/test/java/com/starrocks/alter/LakeSyncMaterializedViewTest.java index 2226d3f37176c3..e26d2ac596614c 100644 --- a/fe/fe-core/src/test/java/com/starrocks/alter/LakeSyncMaterializedViewTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/alter/LakeSyncMaterializedViewTest.java @@ -658,4 +658,26 @@ public void testCreateMVWithAggState() throws Exception { } starRocksAssert.dropTable("t1"); } + + @Test + public void testDropColumnWithMVByFastSchema() throws Exception { + starRocksAssert.useDatabase("test"); + starRocksAssert.withTable("CREATE TABLE t1 (\n" + + " k1 int,\n" + + " k2 int,\n" + + " k3 int,\n" + + " k4 int)\n" + + " DUPLICATE KEY(k1)\n" + + " DISTRIBUTED BY HASH(k1) BUCKETS 3;"); + { + starRocksAssert.withMaterializedView("CREATE MATERIALIZED VIEW mv1 " + + "AS SELECT k1,sum(k2) AS sum_k2 FROM t1 WHERE k3>2 GROUP BY k1;"); + + starRocksAssert.alterTable("ALTER TABLE t1 DROP COLUMN k4;"); + starRocksAssert.checkSchemaChangeJob(); + + starRocksAssert.dropTable("t1"); + starRocksAssert.dropMaterializedView("mv1"); + } + } } diff --git a/fe/fe-core/src/test/java/com/starrocks/analysis/AdminSetConfigStmtTest.java b/fe/fe-core/src/test/java/com/starrocks/analysis/AdminSetConfigStmtTest.java index dfed0c6f2098b8..cf8521aadcddfa 100644 --- a/fe/fe-core/src/test/java/com/starrocks/analysis/AdminSetConfigStmtTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/analysis/AdminSetConfigStmtTest.java @@ -36,7 +36,6 @@ import com.starrocks.common.ConfigBase; import com.starrocks.common.DdlException; -import com.starrocks.common.InvalidConfException; import com.starrocks.common.Pair; import com.starrocks.ha.FrontendNodeType; import com.starrocks.qe.ConnectContext; @@ -135,7 +134,7 @@ public Pair getSelfNode() { String stmt = "admin set frontend config(\"alter_table_timeout_second\" = \"60\");"; AdminSetConfigStmt adminSetConfigStmt = (AdminSetConfigStmt) UtFrameUtils.parseStmtWithNewParser(stmt, connectContext); - expectedEx.expect(InvalidConfException.class); + expectedEx.expect(DdlException.class); ConfigBase.setConfig(adminSetConfigStmt); } } diff --git a/fe/fe-core/src/test/java/com/starrocks/catalog/CreateTableTest.java b/fe/fe-core/src/test/java/com/starrocks/catalog/CreateTableTest.java index 411a134025557c..2a32e31a635bcb 100644 --- a/fe/fe-core/src/test/java/com/starrocks/catalog/CreateTableTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/catalog/CreateTableTest.java @@ -215,7 +215,7 @@ public void testNormal() throws DdlException { + "partition by range(k2)\n" + "(partition p1 values less than(\"10\"))\n" + "distributed by hash(k2) buckets 1\n" + "properties('replication_num' = '1');")); - ConfigBase.setMutableConfig("enable_strict_storage_medium_check", "false"); + ConfigBase.setMutableConfig("enable_strict_storage_medium_check", "false", false, ""); ExceptionChecker .expectThrowsNoException(() -> createTable("create table test.tb7(key1 int, key2 varchar(10)) \n" + @@ -403,7 +403,7 @@ public void testAbnormal() throws DdlException { + "duplicate key(k1, k2, k3)\n" + "distributed by hash(k1) buckets 1\n" + "properties('replication_num' = '1');")); - ConfigBase.setMutableConfig("enable_strict_storage_medium_check", "true"); + ConfigBase.setMutableConfig("enable_strict_storage_medium_check", "true", false, ""); ExceptionChecker .expectThrowsWithMsg(DdlException.class, "Failed to find enough hosts with storage " + diff --git a/fe/fe-core/src/test/java/com/starrocks/common/ConfigTest.java b/fe/fe-core/src/test/java/com/starrocks/common/ConfigTest.java index 46a8fa3035f59e..39ee67954b4f9c 100644 --- a/fe/fe-core/src/test/java/com/starrocks/common/ConfigTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/common/ConfigTest.java @@ -77,13 +77,65 @@ public void testMultiAlias() throws Exception { @Test public void testConfigSetCompatibleWithOldName() throws Exception { - Config.setMutableConfig("schedule_slot_num_per_path", "4"); + Config.setMutableConfig("schedule_slot_num_per_path", "4", false, ""); PatternMatcher matcher = PatternMatcher.createMysqlPattern("schedule_slot_num_per_path", false); List> configs = Config.getConfigInfo(matcher); Assert.assertEquals("4", configs.get(0).get(2)); Assert.assertEquals(4, Config.tablet_sched_slot_num_per_path); } + @Test + public void testMutableConfig() throws Exception { + PatternMatcher matcher = PatternMatcher.createMysqlPattern("adaptive_choose_instances_threshold", false); + List> configs = Config.getConfigInfo(matcher); + Assert.assertEquals("99", configs.get(0).get(2)); + + PatternMatcher matcher2 = PatternMatcher.createMysqlPattern("agent_task_resend_wait_time_ms", false); + List> configs2 = Config.getConfigInfo(matcher2); + Assert.assertEquals("998", configs2.get(0).get(2)); + + Config.setMutableConfig("adaptive_choose_instances_threshold", "98", true, "root"); + configs = Config.getConfigInfo(matcher); + Assert.assertEquals("98", configs.get(0).get(2)); + Assert.assertEquals(98, Config.adaptive_choose_instances_threshold); + + Config.setMutableConfig("agent_task_resend_wait_time_ms", "999", true, "root"); + configs2 = Config.getConfigInfo(matcher2); + Assert.assertEquals("999", configs2.get(0).get(2)); + Assert.assertEquals(999, Config.agent_task_resend_wait_time_ms); + // Write config twice + Config.setMutableConfig("agent_task_resend_wait_time_ms", "1000", true, "root"); + configs2 = Config.getConfigInfo(matcher2); + Assert.assertEquals("1000", configs2.get(0).get(2)); + Assert.assertEquals(1000, Config.agent_task_resend_wait_time_ms); + + // Reload from file + URL resource = getClass().getClassLoader().getResource("conf/config_test.properties"); + config.init(Paths.get(resource.toURI()).toFile().getAbsolutePath()); + configs = Config.getConfigInfo(matcher); + configs2 = Config.getConfigInfo(matcher2); + Assert.assertEquals("98", configs.get(0).get(2)); + Assert.assertEquals("1000", configs2.get(0).get(2)); + Assert.assertEquals(98, Config.adaptive_choose_instances_threshold); + Assert.assertEquals(1000, Config.agent_task_resend_wait_time_ms); + } + + @Test + public void testDisableStoreConfig() throws Exception { + Config.setMutableConfig("adaptive_choose_instances_threshold", "98", false, ""); + PatternMatcher matcher = PatternMatcher.createMysqlPattern("adaptive_choose_instances_threshold", false); + List> configs = Config.getConfigInfo(matcher); + Assert.assertEquals("98", configs.get(0).get(2)); + Assert.assertEquals(98, Config.adaptive_choose_instances_threshold); + + // Reload from file + URL resource = getClass().getClassLoader().getResource("conf/config_test.properties"); + config.init(Paths.get(resource.toURI()).toFile().getAbsolutePath()); + configs = Config.getConfigInfo(matcher); + Assert.assertEquals("99", configs.get(0).get(2)); + Assert.assertEquals(99, Config.adaptive_choose_instances_threshold); + } + private static class ConfigForArray extends ConfigBase { @ConfField(mutable = true) diff --git a/fe/fe-core/src/test/java/com/starrocks/connector/jdbc/OracleSchemaResolverTest.java b/fe/fe-core/src/test/java/com/starrocks/connector/jdbc/OracleSchemaResolverTest.java index c2147e79467a41..ac1c9959d283bc 100644 --- a/fe/fe-core/src/test/java/com/starrocks/connector/jdbc/OracleSchemaResolverTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/connector/jdbc/OracleSchemaResolverTest.java @@ -16,10 +16,13 @@ import com.google.common.collect.Lists; import com.mockrunner.mock.jdbc.MockResultSet; +import com.starrocks.catalog.Column; import com.starrocks.catalog.Database; import com.starrocks.catalog.JDBCResource; import com.starrocks.catalog.JDBCTable; import com.starrocks.catalog.Table; +import com.starrocks.catalog.Type; +import com.starrocks.connector.ConnectorMetadatRequestContext; import com.zaxxer.hikari.HikariDataSource; import mockit.Expectations; import mockit.Mocked; @@ -28,6 +31,7 @@ import org.junit.Test; import java.sql.Connection; +import java.sql.PreparedStatement; import java.sql.SQLException; import java.sql.Types; import java.util.Arrays; @@ -42,10 +46,14 @@ public class OracleSchemaResolverTest { @Mocked Connection connection; + @Mocked + PreparedStatement preparedStatement; + private Map properties; private MockResultSet dbResult; private MockResultSet tableResult; private MockResultSet columnResult; + private MockResultSet partitionsResult; @Before public void setUp() throws SQLException { @@ -71,6 +79,23 @@ public void setUp() throws SQLException { properties.put(JDBCResource.PASSWORD, "123456"); properties.put(JDBCResource.CHECK_SUM, "xxxx"); properties.put(JDBCResource.DRIVER_URL, "xxxx"); + + partitionsResult = new MockResultSet("partitions"); + partitionsResult.addColumn("NAME", Arrays.asList("'20230810'")); + partitionsResult.addColumn("COLUMN_NAME", Arrays.asList("`d`")); + partitionsResult.addColumn("MODIFIED_TIME", Arrays.asList("2023-08-01 00:00:00")); + + new Expectations() { + { + dataSource.getConnection(); + result = connection; + minTimes = 0; + + preparedStatement.executeQuery(); + result = partitionsResult; + minTimes = 0; + } + }; } @Test @@ -187,4 +212,42 @@ public void testGetTable() throws SQLException { Assert.fail(); } } + + @Test + public void testGetPartitions() { + try { + JDBCMetadata jdbcMetadata = new JDBCMetadata(properties, "catalog", dataSource); + JDBCTable jdbcTable = new JDBCTable(100000, "tbl1", Arrays.asList(new Column("d", Type.VARCHAR)), + Arrays.asList(new Column("d", Type.VARCHAR)), "test", "catalog", properties); + Integer size = jdbcMetadata.getPartitions(jdbcTable, Arrays.asList("20230810")).size(); + Assert.assertTrue(size > 0); + } catch (Exception e) { + Assert.fail(e.getMessage()); + } + } + + @Test + public void testListPartitionNames() { + try { + JDBCMetadata jdbcMetadata = new JDBCMetadata(properties, "catalog", dataSource); + List partitionNames = jdbcMetadata.listPartitionNames("test", "tbl1", + ConnectorMetadatRequestContext.DEFAULT); + Assert.assertFalse(partitionNames.isEmpty()); + } catch (Exception e) { + Assert.fail(e.getMessage()); + } + } + + @Test + public void testListPartitionColumns() { + try { + JDBCMetadata jdbcMetadata = new JDBCMetadata(properties, "catalog", dataSource); + List partitionCols = jdbcMetadata.listPartitionColumns("test", "tbl1", + Arrays.asList(new Column("`d`", Type.VARCHAR))); + Integer size = partitionCols.size(); + Assert.assertTrue(size > 0); + } catch (Exception e) { + Assert.fail(e.getMessage()); + } + } } diff --git a/fe/fe-core/src/test/java/com/starrocks/lake/ClusterSnapshotTest.java b/fe/fe-core/src/test/java/com/starrocks/lake/ClusterSnapshotTest.java index a74c186b51bc4e..9f4f32d4f095a6 100644 --- a/fe/fe-core/src/test/java/com/starrocks/lake/ClusterSnapshotTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/lake/ClusterSnapshotTest.java @@ -139,12 +139,12 @@ public RunMode getCurrentRunMode() { @Mock public void copyFromLocal(String srcPath, String destPath, Map properties) { return; - } //IOException + } // IOException @Mock public void deletePath(String path, Map loadProperties) { return; - } //IOException + } // IOException }; setAutomatedSnapshotOff(false); @@ -153,7 +153,7 @@ public void deletePath(String path, Map loadProperties) { private void setAutomatedSnapshotOn(boolean testReplay) { if (!testReplay) { GlobalStateMgr.getCurrentState().getClusterSnapshotMgr().setAutomatedSnapshotOn( - new AdminSetAutomatedSnapshotOnStmt(storageVolumeName)); + new AdminSetAutomatedSnapshotOnStmt(storageVolumeName)); } else { GlobalStateMgr.getCurrentState().getClusterSnapshotMgr().setAutomatedSnapshotOn(storageVolumeName); } @@ -162,7 +162,7 @@ private void setAutomatedSnapshotOn(boolean testReplay) { private void setAutomatedSnapshotOff(boolean testReplay) { if (!testReplay) { GlobalStateMgr.getCurrentState().getClusterSnapshotMgr().setAutomatedSnapshotOff( - new AdminSetAutomatedSnapshotOffStmt()); + new AdminSetAutomatedSnapshotOffStmt()); } else { GlobalStateMgr.getCurrentState().getClusterSnapshotMgr().setAutomatedSnapshotOff(); } @@ -176,10 +176,10 @@ private void initStorageVolume() throws AlreadyExistsException, DdlException, Me storageParams.put(AWS_S3_ENDPOINT, "endpoint"); storageParams.put(AWS_S3_USE_AWS_SDK_DEFAULT_BEHAVIOR, "true"); String svKey = GlobalStateMgr.getCurrentState().getStorageVolumeMgr() - .createStorageVolume(storageVolumeName, "S3", locations, storageParams, Optional.empty(), ""); + .createStorageVolume(storageVolumeName, "S3", locations, storageParams, Optional.empty(), ""); Assert.assertEquals(true, GlobalStateMgr.getCurrentState().getStorageVolumeMgr().exists(storageVolumeName)); Assert.assertEquals(storageVolumeName, - GlobalStateMgr.getCurrentState().getStorageVolumeMgr().getStorageVolumeName(svKey)); + GlobalStateMgr.getCurrentState().getStorageVolumeMgr().getStorageVolumeName(svKey)); initSv = true; } } @@ -211,13 +211,15 @@ public void testOperationOfAutomatedSnapshot() throws DdlException { ClusterSnapshot snapshot = GlobalStateMgr.getCurrentState().getClusterSnapshotMgr().getAutomatedSnapshot(); Assert.assertTrue(job.getInfo() != null); Assert.assertTrue(snapshot.getInfo() != null); - Assert.assertTrue(GlobalStateMgr.getCurrentState().getClusterSnapshotMgr().getAllInfo().getItemsSize() == 1); - Assert.assertTrue(GlobalStateMgr.getCurrentState().getClusterSnapshotMgr().getAllJobsInfo().getItemsSize() == 1); - - ExceptionChecker.expectThrowsNoException(() -> - ClusterSnapshotUtils.uploadAutomatedSnapshotToRemote(job.getSnapshotName())); - ExceptionChecker.expectThrowsNoException(() -> - ClusterSnapshotUtils.clearAutomatedSnapshotFromRemote(job.getSnapshotName())); + Assert.assertTrue( + GlobalStateMgr.getCurrentState().getClusterSnapshotMgr().getAllSnapshotsInfo().getItemsSize() == 1); + Assert.assertTrue( + GlobalStateMgr.getCurrentState().getClusterSnapshotMgr().getAllSnapshotJobsInfo().getItemsSize() == 1); + + ExceptionChecker.expectThrowsNoException( + () -> ClusterSnapshotUtils.uploadAutomatedSnapshotToRemote(job.getSnapshotName())); + ExceptionChecker.expectThrowsNoException( + () -> ClusterSnapshotUtils.clearAutomatedSnapshotFromRemote(job.getSnapshotName())); setAutomatedSnapshotOff(false); } @@ -241,28 +243,28 @@ public void testReplayClusterSnapshotLog() { job.setState(ClusterSnapshotJobState.INITIALIZING); logSnapshotJob.setSnapshotJob(job); GlobalStateMgr.getCurrentState().getClusterSnapshotMgr().replayLog(logSnapshotJob); - Assert.assertTrue(GlobalStateMgr.getCurrentState().getClusterSnapshotMgr().getAllJobsInfo() - .getItems().get(0).state == "INITIALIZING"); + Assert.assertTrue(GlobalStateMgr.getCurrentState().getClusterSnapshotMgr().getAllSnapshotJobsInfo() + .getItems().get(0).state == "INITIALIZING"); job.setState(ClusterSnapshotJobState.SNAPSHOTING); logSnapshotJob.setSnapshotJob(job); GlobalStateMgr.getCurrentState().getClusterSnapshotMgr().replayLog(logSnapshotJob); - Assert.assertTrue(GlobalStateMgr.getCurrentState().getClusterSnapshotMgr().getAllJobsInfo() - .getItems().get(0).state == "SNAPSHOTING"); + Assert.assertTrue(GlobalStateMgr.getCurrentState().getClusterSnapshotMgr().getAllSnapshotJobsInfo() + .getItems().get(0).state == "SNAPSHOTING"); job.setState(ClusterSnapshotJobState.UPLOADING); logSnapshotJob.setSnapshotJob(job); GlobalStateMgr.getCurrentState().getClusterSnapshotMgr().replayLog(logSnapshotJob); - Assert.assertTrue(GlobalStateMgr.getCurrentState().getClusterSnapshotMgr().getAllJobsInfo() - .getItems().get(0).state == "UPLOADING"); + Assert.assertTrue(GlobalStateMgr.getCurrentState().getClusterSnapshotMgr().getAllSnapshotJobsInfo() + .getItems().get(0).state == "UPLOADING"); job.setState(ClusterSnapshotJobState.FINISHED); logSnapshotJob.setSnapshotJob(job); GlobalStateMgr.getCurrentState().getClusterSnapshotMgr().replayLog(logSnapshotJob); - Assert.assertTrue(GlobalStateMgr.getCurrentState().getClusterSnapshotMgr().getAllJobsInfo() - .getItems().get(0).state == "FINISHED"); + Assert.assertTrue(GlobalStateMgr.getCurrentState().getClusterSnapshotMgr().getAllSnapshotJobsInfo() + .getItems().get(0).state == "FINISHED"); job.setState(ClusterSnapshotJobState.ERROR); logSnapshotJob.setSnapshotJob(job); GlobalStateMgr.getCurrentState().getClusterSnapshotMgr().replayLog(logSnapshotJob); - Assert.assertTrue(GlobalStateMgr.getCurrentState().getClusterSnapshotMgr().getAllJobsInfo() - .getItems().get(0).state == "ERROR"); + Assert.assertTrue(GlobalStateMgr.getCurrentState().getClusterSnapshotMgr().getAllSnapshotJobsInfo() + .getItems().get(0).state == "ERROR"); } @Test @@ -290,10 +292,11 @@ public long getMaxJournalId() { Config.automated_cluster_snapshot_interval_seconds = 1; CheckpointController feController = new CheckpointController("fe", new BDBJEJournal(null, ""), ""); CheckpointController starMgrController = new CheckpointController("starMgr", new BDBJEJournal(null, ""), ""); - ClusterSnapshotCheckpointScheduler scheduler = new ClusterSnapshotCheckpointScheduler(feController, starMgrController); + ClusterSnapshotCheckpointScheduler scheduler = new ClusterSnapshotCheckpointScheduler(feController, + starMgrController); scheduler.start(); - while (GlobalStateMgr.getCurrentState().getClusterSnapshotMgr().getAutomatedSnapshot() == null) { + while (GlobalStateMgr.getCurrentState().getClusterSnapshotMgr().getAutomatedSnapshot() == null) { try { Thread.sleep(100); } catch (Exception e) { @@ -302,7 +305,7 @@ public long getMaxJournalId() { scheduler.setStop(); - while (scheduler.isRunning()) { + while (scheduler.isRunning()) { try { Thread.sleep(100); } catch (Exception e) { @@ -321,16 +324,16 @@ public boolean isSharedDataMode() { }; ClusterSnapshotMgr localClusterSnapshotMgr = new ClusterSnapshotMgr(); - Assert.assertTrue(localClusterSnapshotMgr.getValidDeletionTimeMsByAutomatedSnapshot() == Long.MAX_VALUE); + Assert.assertTrue(localClusterSnapshotMgr.getSafeDeletionTimeMs() == Long.MAX_VALUE); localClusterSnapshotMgr.setAutomatedSnapshotOn(storageVolumeName); - Assert.assertEquals(localClusterSnapshotMgr.getValidDeletionTimeMsByAutomatedSnapshot(), 0L); + Assert.assertEquals(localClusterSnapshotMgr.getSafeDeletionTimeMs(), 0L); ClusterSnapshotJob job1 = localClusterSnapshotMgr.createAutomatedSnapshotJob(); job1.setState(ClusterSnapshotJobState.FINISHED); - Assert.assertEquals(localClusterSnapshotMgr.getValidDeletionTimeMsByAutomatedSnapshot(), 0L); + Assert.assertEquals(localClusterSnapshotMgr.getSafeDeletionTimeMs(), 0L); ClusterSnapshotJob job2 = localClusterSnapshotMgr.createAutomatedSnapshotJob(); job2.setState(ClusterSnapshotJobState.FINISHED); - Assert.assertEquals(localClusterSnapshotMgr.getValidDeletionTimeMsByAutomatedSnapshot(), job1.getCreatedTimeMs()); + Assert.assertEquals(localClusterSnapshotMgr.getSafeDeletionTimeMs(), job1.getCreatedTimeMs()); localClusterSnapshotMgr.setAutomatedSnapshotOff(); localClusterSnapshotMgr = new ClusterSnapshotMgr(); @@ -342,7 +345,6 @@ public boolean isSharedDataMode() { alterjob2.setJobState(AlterJobV2.JobState.FINISHED); alterjob2.setFinishedTimeMs(1000); - MaterializedViewHandler rollupHandler = new MaterializedViewHandler(); SchemaChangeHandler schemaChangeHandler = new SchemaChangeHandler(); schemaChangeHandler.addAlterJobV2(alterjob1); diff --git a/fe/fe-core/src/test/java/com/starrocks/load/loadv2/LoadLoadingTaskTest.java b/fe/fe-core/src/test/java/com/starrocks/load/loadv2/LoadLoadingTaskTest.java index ba60e4dc72b317..48c7e77177b45f 100644 --- a/fe/fe-core/src/test/java/com/starrocks/load/loadv2/LoadLoadingTaskTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/load/loadv2/LoadLoadingTaskTest.java @@ -14,6 +14,7 @@ package com.starrocks.load.loadv2; +import com.starrocks.analysis.BrokerDesc; import com.starrocks.catalog.CatalogIdGenerator; import com.starrocks.catalog.Database; import com.starrocks.catalog.KeysType; @@ -26,6 +27,7 @@ import com.starrocks.qe.SessionVariable; import com.starrocks.qe.VariableMgr; import com.starrocks.server.GlobalStateMgr; +import com.starrocks.sql.ast.LoadStmt; import com.starrocks.thrift.TUniqueId; import com.starrocks.transaction.GlobalTransactionMgr; import com.starrocks.transaction.TransactionState; @@ -96,7 +98,8 @@ public void testBuildTopLevelProfile(@Mocked GlobalStateMgr globalStateMgr, // Call the method under test Database database = new Database(10000L, "test"); OlapTable olapTable = new OlapTable(10001L, "tbl", null, KeysType.AGG_KEYS, null, null); - LoadLoadingTask loadLoadingTask = new LoadLoadingTask.Builder().setDb(database) + LoadStmt stmt = new LoadStmt(null, null, new BrokerDesc(null), null, null); + LoadLoadingTask loadLoadingTask = new LoadLoadingTask.Builder().setDb(database).setLoadStmt(stmt) .setTable(olapTable).setContext(connectContext).setOriginStmt(new OriginStatement("")).build(); RuntimeProfile profile = loadLoadingTask.buildRunningTopLevelProfile(); // Perform assertions to verify the behavior diff --git a/fe/fe-core/src/test/java/com/starrocks/planner/ViewBaseMvRewriteOnTpcdsTest.java b/fe/fe-core/src/test/java/com/starrocks/planner/ViewBaseMvRewriteOnTpcdsTest.java index 5bdc8a5a23b685..09acb847c5c41f 100644 --- a/fe/fe-core/src/test/java/com/starrocks/planner/ViewBaseMvRewriteOnTpcdsTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/planner/ViewBaseMvRewriteOnTpcdsTest.java @@ -39,6 +39,7 @@ public static void beforeClass() throws Exception { MaterializedViewTestBase.beforeClass(); connectContext.getSessionVariable().setEnableViewBasedMvRewrite(true); + connectContext.getSessionVariable().setMaterializedViewRewriteMode("force"); starRocksAssert.useDatabase("test"); connectContext.executeSql("drop table if exists customer"); diff --git a/fe/fe-core/src/test/java/com/starrocks/qe/AdaptiveChooseNodesTest.java b/fe/fe-core/src/test/java/com/starrocks/qe/AdaptiveChooseNodesTest.java index aa82da86861912..945cf428b6e593 100644 --- a/fe/fe-core/src/test/java/com/starrocks/qe/AdaptiveChooseNodesTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/qe/AdaptiveChooseNodesTest.java @@ -49,7 +49,7 @@ public static void beforeAll() throws Exception { @Test @Order(1) void testIncreaseNodes() throws Exception { - ConfigBase.setMutableConfig("adaptive_choose_instances_threshold", "3"); + ConfigBase.setMutableConfig("adaptive_choose_instances_threshold", "3", false, ""); connectContext.getSessionVariable().setChooseExecuteInstancesMode("auto"); connectContext.getSessionVariable().setPipelineDop(2); connectContext.setExecutionId(new TUniqueId(0x33, 0x0)); @@ -88,7 +88,7 @@ void testIncreaseNodes() throws Exception { @Test @Order(2) void testDecreaseNodesInPipeline() throws Exception { - ConfigBase.setMutableConfig("adaptive_choose_instances_threshold", "3"); + ConfigBase.setMutableConfig("adaptive_choose_instances_threshold", "3", false, ""); connectContext.getSessionVariable().setChooseExecuteInstancesMode("auto"); connectContext.getSessionVariable().setPipelineDop(2); connectContext.setExecutionId(new TUniqueId(0x33, 0x0)); @@ -161,7 +161,7 @@ private static Stream getCases() { public static void afterAll() throws Exception { connectContext.getSessionVariable().setEnablePipelineEngine(true); connectContext.getSessionVariable().setChooseExecuteInstancesMode("locality"); - ConfigBase.setMutableConfig("adaptive_choose_instances_threshold", "32"); + ConfigBase.setMutableConfig("adaptive_choose_instances_threshold", "32", false, ""); } diff --git a/fe/fe-core/src/test/java/com/starrocks/qe/scheduler/slot/SlotEstimatorTest.java b/fe/fe-core/src/test/java/com/starrocks/qe/scheduler/slot/SlotEstimatorTest.java index 05af382c3b740c..e48d48107d4dd0 100644 --- a/fe/fe-core/src/test/java/com/starrocks/qe/scheduler/slot/SlotEstimatorTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/qe/scheduler/slot/SlotEstimatorTest.java @@ -40,13 +40,13 @@ public void testMemoryBasedSlotsEstimator() throws Exception { DefaultCoordinator coordinator = getScheduler("SELECT * FROM lineitem"); connectContext.getAuditEventBuilder().setPlanMemCosts(-1L); - assertThat(estimator.estimateSlots(opts, connectContext, coordinator)).isEqualTo(1); + assertThat(estimator.estimateSlots(opts, connectContext, coordinator)).isEqualTo(3); connectContext.getAuditEventBuilder().setPlanMemCosts(memLimitBytesPerWorker * numWorkers); assertThat(estimator.estimateSlots(opts, connectContext, coordinator)).isEqualTo(opts.v2().getTotalSlots()); connectContext.getAuditEventBuilder().setPlanMemCosts(memLimitBytesPerWorker * numWorkers - 10); - assertThat(estimator.estimateSlots(opts, connectContext, coordinator)).isEqualTo(opts.v2().getTotalSlots() / 2); + assertThat(estimator.estimateSlots(opts, connectContext, coordinator)).isEqualTo(opts.v2().getTotalSlots()); connectContext.getAuditEventBuilder().setPlanMemCosts(memLimitBytesPerWorker * numWorkers + 10); assertThat(estimator.estimateSlots(opts, connectContext, coordinator)).isEqualTo(opts.v2().getTotalSlots()); diff --git a/fe/fe-core/src/test/java/com/starrocks/service/FrontendServiceImplTest.java b/fe/fe-core/src/test/java/com/starrocks/service/FrontendServiceImplTest.java index 2cf19f4ee2c569..e1c7253afa577e 100644 --- a/fe/fe-core/src/test/java/com/starrocks/service/FrontendServiceImplTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/service/FrontendServiceImplTest.java @@ -28,6 +28,7 @@ import com.starrocks.common.AnalysisException; import com.starrocks.common.Config; import com.starrocks.common.FeConstants; +import com.starrocks.common.PatternMatcher; import com.starrocks.common.StarRocksException; import com.starrocks.common.util.concurrent.lock.LockTimeoutException; import com.starrocks.ha.FrontendNodeType; @@ -1155,7 +1156,7 @@ public void testStreamLoadPutColumnMapException() { } @Test - public void testSetFrontendConfig() throws TException { + public void testSetFrontendConfig() throws Exception { FrontendServiceImpl impl = new FrontendServiceImpl(exeEnv); TSetConfigRequest request = new TSetConfigRequest(); request.keys = Lists.newArrayList("mysql_server_version"); @@ -1163,6 +1164,17 @@ public void testSetFrontendConfig() throws TException { TSetConfigResponse result = impl.setConfig(request); Assert.assertEquals("5.1.1", GlobalVariable.version); + + request.keys = Lists.newArrayList("adaptive_choose_instances_threshold"); + request.values = Lists.newArrayList("98"); + request.setUser_identity("root"); + request.setIs_persistent(true); + impl.setConfig(request); + + PatternMatcher matcher = PatternMatcher.createMysqlPattern("adaptive_choose_instances_threshold", false); + List> configs = Config.getConfigInfo(matcher); + Assert.assertEquals("98", configs.get(0).get(2)); + Assert.assertEquals(98, Config.adaptive_choose_instances_threshold); } @Test diff --git a/fe/fe-core/src/test/java/com/starrocks/sql/analyzer/PrivilegeCheckerTest.java b/fe/fe-core/src/test/java/com/starrocks/sql/analyzer/PrivilegeCheckerTest.java index 631f5a874d27ed..48a03d2d84a3e4 100644 --- a/fe/fe-core/src/test/java/com/starrocks/sql/analyzer/PrivilegeCheckerTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/sql/analyzer/PrivilegeCheckerTest.java @@ -965,7 +965,7 @@ public void testTableSelectDeleteInsertUpdateColumn() throws Exception { " k4 int) ENGINE=OLAP PRIMARY KEY(`k1`) distributed by hash(k1) " + "buckets 3 properties('replication_num' = '1');"; starRocksAssert.withTable(createTblStmtStr4); - Config.setMutableConfig("authorization_enable_column_level_privilege", "true"); + Config.setMutableConfig("authorization_enable_column_level_privilege", "true", false, ""); // select verifyGrantRevoke( "select * from db1.tbl1", @@ -1082,7 +1082,7 @@ public void testTableSelectDeleteInsertUpdateColumn() throws Exception { "grant delete on db3.tprimary to test", "revoke delete on db3.tprimary from test", "Access denied; you need (at least one of) the DELETE privilege(s) on TABLE tprimary for this operation", "Access denied; you need (at least one of) the SELECT privilege(s) on COLUMN tbl1.k1,k2 for this operation"); - Config.setMutableConfig("authorization_enable_column_level_privilege", "false"); + Config.setMutableConfig("authorization_enable_column_level_privilege", "false", false, ""); starRocksAssert.dropTable("db3.tprimary"); } diff --git a/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/OptimizerTaskTest.java b/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/OptimizerTaskTest.java index 8df67e9e1fcff0..f23bb081f129a7 100644 --- a/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/OptimizerTaskTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/OptimizerTaskTest.java @@ -745,11 +745,13 @@ public void testTopDownRewrite(@Mocked OlapTable olapTable1) { } }; + ctx.getSessionVariable().setOptimizerExecuteTimeout(10000); Optimizer optimizer = OptimizerFactory.create(OptimizerFactory.mockContext(ctx, columnRefFactory)); OptExpression physicalTree = optimizer.optimize(expression, new ColumnRefSet()); Operator root = physicalTree.getOp(); assertEquals(root.getOpType(), OperatorType.PHYSICAL_LIMIT); + ctx.getSessionVariable().setOptimizerExecuteTimeout(3000); } @Test diff --git a/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/cost/PlanFeaturesTest.java b/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/cost/PlanFeaturesTest.java index d426927fa79e61..966266bf84ce54 100644 --- a/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/cost/PlanFeaturesTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/cost/PlanFeaturesTest.java @@ -22,9 +22,12 @@ import com.starrocks.sql.plan.PlanTestBase; import org.apache.commons.lang3.StringUtils; import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.CsvSource; +import java.util.List; + class PlanFeaturesTest extends PlanTestBase { @ParameterizedTest @@ -34,7 +37,7 @@ class PlanFeaturesTest extends PlanTestBase { "| 39,1,0,8,0,2,0,3;40,1,0,8,2,2,4,0,0,1,1;44,1,0,9,0,2,0,0,0,1,1", "select max(v1) from t0 where v1 < 100 limit 100" + "|tables=[0,0,10003] " + - "| [39,1,0,8,0,2,0,3;40,1,0,8,2,2,4,0,0,1,1;44,1,0,8,0,2,0,0,0,1,1", + "| 39,1,0,8,0,2,0,3;40,1,0,8,2,2,4,0,0,1,1;44,1,0,8,0,2,0,0,0,1,1", "select v1, count(*) from t0 group by v1 " + "| tables=[0,0,10003] " + "| 40,1,0,16,2,2,0,0,1,1,1;44,1,0,8,0,2,0,0,0,0,0", @@ -50,11 +53,33 @@ public void testBasic(String query, String expectedTables, String expected) thro ExecPlan execPlan = getExecPlan(query); OptExpression physicalPlan = execPlan.getPhysicalPlan(); PlanFeatures planFeatures = FeatureExtractor.extractFeatures(physicalPlan); + + // feature string String string = planFeatures.toFeatureString(); Assertions.assertTrue(string.startsWith(expectedTables), string); Splitter.on(";").splitToList(expected).forEach(slice -> { Assertions.assertTrue(string.contains(slice), "slice is " + slice + ", feature is " + string); }); + + // feature csv + String csv = planFeatures.toFeatureCsv(); + Splitter.on(";").splitToList(expected).forEach(slice -> { + Assertions.assertTrue(csv.contains(slice), "slice is " + slice + ", feature is " + string); + }); + } + + @Test + public void testHeader() { + String header = PlanFeatures.featuresHeader(); + List strings = Splitter.on(",").splitToList(header); + long numTables = strings.stream().filter(x -> x.startsWith("tables")).count(); + long numEnvs = strings.stream().filter(x -> x.startsWith("env")).count(); + long numVars = strings.stream().filter(x -> x.startsWith("var")).count(); + long numOperators = strings.stream().filter(x -> x.startsWith("operators")).count(); + Assertions.assertEquals(3, numTables); + Assertions.assertEquals(3, numEnvs); + Assertions.assertEquals(1, numVars); + Assertions.assertEquals(173, numOperators); } } \ No newline at end of file diff --git a/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/cost/feature/CostPredictorTest.java b/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/cost/feature/CostPredictorTest.java new file mode 100644 index 00000000000000..355945f5d2fd70 --- /dev/null +++ b/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/cost/feature/CostPredictorTest.java @@ -0,0 +1,71 @@ +// 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.sql.optimizer.cost.feature; + +import com.starrocks.common.Config; +import com.starrocks.common.Pair; +import com.starrocks.sql.plan.ExecPlan; +import com.starrocks.sql.plan.PlanTestBase; +import com.starrocks.utframe.UtFrameUtils; +import com.sun.net.httpserver.HttpServer; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.io.OutputStream; +import java.net.InetSocketAddress; + +public class CostPredictorTest extends PlanTestBase { + + @Test + public void testServiceBasedPredictor() throws Exception { + String sql = "select count(*) from t0 where v1 < 100 limit 100 "; + Pair planAndFragment = UtFrameUtils.getPlanAndFragment(connectContext, sql); + var instance = CostPredictor.ServiceBasedCostPredictor.getInstance(); + + Assertions.assertThrows(RuntimeException.class, () -> instance.predictMemoryBytes(planAndFragment.second)); + + // Create a simple HTTP server to mock the prediction service + HttpServer server = HttpServer.create(new InetSocketAddress(0), 0); + int port = server.getAddress().getPort(); + final long predictedResult = 10000L; + server.createContext(CostPredictor.ServiceBasedCostPredictor.PREDICT_URL, exchange -> { + String response = String.valueOf(predictedResult); + exchange.sendResponseHeaders(200, response.length()); + try (OutputStream os = exchange.getResponseBody()) { + os.write(response.getBytes()); + } + }); + server.createContext(CostPredictor.ServiceBasedCostPredictor.HEALTH_URL, exchange -> { + exchange.sendResponseHeaders(200, 0); + }); + server.start(); + Config.query_cost_prediction_service_address = "http://localhost:" + port; + + // Running the test case predictMemoryBytes again with the mocked service + // This time, the test should not throw an exception + Assertions.assertEquals(predictedResult, instance.predictMemoryBytes(planAndFragment.second)); + + // test health check + Config.enable_query_cost_prediction = true; + instance.doHealthCheck(); + Assertions.assertTrue(instance.isAvailable()); + + server.removeContext(CostPredictor.ServiceBasedCostPredictor.HEALTH_URL); + instance.doHealthCheck(); + Assertions.assertFalse(instance.isAvailable()); + + server.stop(0); + } +} \ No newline at end of file diff --git a/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/rule/transformation/materialization/MvTimeSeriesRewriteWithOlapTest.java b/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/rule/transformation/materialization/MvTimeSeriesRewriteWithOlapTest.java index 07265ee6ea82fc..242d60d382b5e4 100644 --- a/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/rule/transformation/materialization/MvTimeSeriesRewriteWithOlapTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/rule/transformation/materialization/MvTimeSeriesRewriteWithOlapTest.java @@ -36,6 +36,8 @@ public class MvTimeSeriesRewriteWithOlapTest extends MVTestBase { @BeforeClass public static void beforeClass() throws Exception { MVTestBase.beforeClass(); + + connectContext.getSessionVariable().setMaterializedViewRewriteMode("force"); starRocksAssert.withTable("CREATE TABLE t0(\n" + " k1 datetime,\n" + " v1 INT,\n" + @@ -129,7 +131,7 @@ public void testAggregateTimeSeriesRollupWithoutGroupBy() throws Exception { PlanTestBase.assertContains(plan, " TABLE: t0\n" + " PREAGGREGATION: ON\n" + " PREDICATES: date_trunc('day', 22: k1) < '2024-01-01 01:00:00', 22: k1 >= '2024-01-01 01:00:00'\n" + - " partitions=4/5"); + " partitions=1/5"); starRocksAssert.dropMaterializedView("test_mv1"); } @@ -155,7 +157,7 @@ public void testAggregateTimeSeriesRollupWithGroupBy() throws Exception { PlanTestBase.assertContains(plan, " TABLE: t0\n" + " PREAGGREGATION: ON\n" + " PREDICATES: date_trunc('day', 24: k1) < '2024-01-01 01:00:00', 24: k1 >= '2024-01-01 01:00:00'\n" + - " partitions=4/5"); + " partitions=1/5"); } { // date column should be the same with date_trunc('day', ct) @@ -184,7 +186,7 @@ public void testAggregateTimeSeriesRollupWithGroupBy() throws Exception { " PREDICATES: (date_trunc('day', 24: k1) > '2024-01-31 01:00:00') OR " + "(date_trunc('day', 24: k1) < '2024-01-01 01:00:00'), 24: k1 <= '2024-02-01 01:00:00', " + "24: k1 >= '2024-01-01 01:00:00'\n" + - " partitions=3/5"); + " partitions=2/5"); } starRocksAssert.dropMaterializedView("test_mv1"); } @@ -215,11 +217,11 @@ public void testAggregateTimeSeriesRollupWithNestedMV() throws Exception { PlanTestBase.assertContains(plan, " TABLE: test_mv1\n" + " PREAGGREGATION: ON\n" + " PREDICATES: date_trunc('month', 45: dt) < '2024-01-01 01:00:00', 45: dt >= '2024-01-01 01:00:00'\n" + - " partitions=62/63"); + " partitions=31/63"); PlanTestBase.assertContains(plan, " TABLE: t0\n" + " PREAGGREGATION: ON\n" + " PREDICATES: date_trunc('day', 25: k1) < '2024-01-01 01:00:00', 25: k1 >= '2024-01-01 01:00:00'\n" + - " partitions=4/5"); + " partitions=1/5"); starRocksAssert.dropMaterializedView("test_mv1"); starRocksAssert.dropMaterializedView("test_mv2"); } @@ -286,7 +288,7 @@ public void testAggregateTimeSeriesWithCountDistinct() throws Exception { PlanTestBase.assertContains(plan, " TABLE: t0\n" + " PREAGGREGATION: ON\n" + " PREDICATES: date_trunc('day', 14: k1) < '2024-01-01 01:00:00', 14: k1 >= '2024-01-01 01:00:00'\n" + - " partitions=4/5"); + " partitions=1/5"); PlanTestBase.assertContains(plan, " 16:AGGREGATE (update serialize)\n" + " | output: array_unique_agg(18: count)\n" + " | group by: \n" + diff --git a/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/rule/transformation/materialization/MvTransparentRewriteWithOlapTableTest.java b/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/rule/transformation/materialization/MvTransparentRewriteWithOlapTableTest.java index e8e5aee701fe88..dda9cebad2b10a 100644 --- a/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/rule/transformation/materialization/MvTransparentRewriteWithOlapTableTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/rule/transformation/materialization/MvTransparentRewriteWithOlapTableTest.java @@ -45,6 +45,7 @@ public class MvTransparentRewriteWithOlapTableTest extends MVTestBase { public static void beforeClass() throws Exception { MVTestBase.beforeClass(); + connectContext.getSessionVariable().setMaterializedViewRewriteMode("force"); m1 = new MTable("m1", "k1", ImmutableList.of( "k1 INT", diff --git a/fe/fe-core/src/test/java/com/starrocks/sql/plan/ReplayFromDumpForSharedDataTest.java b/fe/fe-core/src/test/java/com/starrocks/sql/plan/ReplayFromDumpForSharedDataTest.java index 3599535e189715..9a466a8d92971b 100644 --- a/fe/fe-core/src/test/java/com/starrocks/sql/plan/ReplayFromDumpForSharedDataTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/sql/plan/ReplayFromDumpForSharedDataTest.java @@ -15,7 +15,6 @@ package com.starrocks.sql.plan; import com.starrocks.common.Config; -import com.starrocks.common.FeConstants; import com.starrocks.common.Pair; import com.starrocks.common.io.Writable; import com.starrocks.persist.EditLog; @@ -34,20 +33,16 @@ public class ReplayFromDumpForSharedDataTest extends ReplayFromDumpTestBase { @BeforeClass public static void beforeClass() throws Exception { UtFrameUtils.createMinStarRocksCluster(RunMode.SHARED_DATA); + // create connect context + connectContext = UtFrameUtils.createDefaultCtx(); + starRocksAssert = new StarRocksAssert(connectContext); Config.show_execution_groups = false; // Should disable Dynamic Partition in replay dump test Config.dynamic_partition_enable = false; Config.tablet_sched_disable_colocate_overall_balance = true; - // create connect context - connectContext = UtFrameUtils.createDefaultCtx(); - connectContext.getSessionVariable().setOptimizerExecuteTimeout(30000); - connectContext.getSessionVariable().setJoinImplementationMode("auto"); - connectContext.getSessionVariable().setCboPushDownAggregateMode(-1); - starRocksAssert = new StarRocksAssert(connectContext); - FeConstants.runningUnitTest = true; - FeConstants.showScanNodeLocalShuffleColumnsInExplain = false; - FeConstants.enablePruneEmptyOutputScan = false; - FeConstants.showJoinLocalShuffleInExplain = false; + UtFrameUtils.setDefaultConfigForAsyncMVTest(connectContext); + // set default config for timeliness mvs + UtFrameUtils.mockTimelinessForAsyncMVTest(connectContext); new MockUp() { @Mock diff --git a/fe/fe-core/src/test/java/com/starrocks/utframe/StarRocksAssert.java b/fe/fe-core/src/test/java/com/starrocks/utframe/StarRocksAssert.java index 5e4442dd65fed3..8764a1f9728f4d 100644 --- a/fe/fe-core/src/test/java/com/starrocks/utframe/StarRocksAssert.java +++ b/fe/fe-core/src/test/java/com/starrocks/utframe/StarRocksAssert.java @@ -1170,6 +1170,26 @@ private void checkAlterJob() throws InterruptedException { } } + public void checkSchemaChangeJob() throws Exception { + Map alterJobs = GlobalStateMgr.getCurrentState(). + getSchemaChangeHandler().getAlterJobsV2(); + for (AlterJobV2 alterJobV2 : alterJobs.values()) { + if (alterJobV2.getJobState().isFinalState()) { + continue; + } + Database database = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb(alterJobV2.getDbId()); + Table table = + GlobalStateMgr.getCurrentState().getLocalMetastore().getTable(database.getId(), alterJobV2.getTableId()); + Preconditions.checkState(table instanceof OlapTable); + OlapTable olapTable = (OlapTable) table; + int retry = 0; + while (olapTable.getState() != OlapTable.OlapTableState.NORMAL && retry++ < 6000) { + Thread.sleep(10); + } + Assert.assertEquals(AlterJobV2.JobState.FINISHED, alterJobV2.getJobState()); + } + } + public QueryAssert query(String sql) { return new QueryAssert(ctx, sql); } diff --git a/fe/fe-core/src/test/resources/conf/config_test.properties b/fe/fe-core/src/test/resources/conf/config_test.properties index af49af8b913055..5dea22635d4852 100644 --- a/fe/fe-core/src/test/resources/conf/config_test.properties +++ b/fe/fe-core/src/test/resources/conf/config_test.properties @@ -1 +1,3 @@ -tablet_sched_slot_num_per_path = 3 \ No newline at end of file +tablet_sched_slot_num_per_path = 3 +adaptive_choose_instances_threshold = 99 +agent_task_resend_wait_time_ms=998 \ No newline at end of file diff --git a/gensrc/thrift/FrontendService.thrift b/gensrc/thrift/FrontendService.thrift index 14bf9e5f5df6a0..148a57cd0ad4ca 100644 --- a/gensrc/thrift/FrontendService.thrift +++ b/gensrc/thrift/FrontendService.thrift @@ -1388,6 +1388,8 @@ struct TAbortRemoteTxnResponse { struct TSetConfigRequest { 1: optional list keys 2: optional list values + 3: optional bool is_persistent + 4: optional string user_identity } struct TSetConfigResponse { diff --git a/gensrc/thrift/PlanNodes.thrift b/gensrc/thrift/PlanNodes.thrift index bf3a645f85b2db..06f76fa1eba00c 100644 --- a/gensrc/thrift/PlanNodes.thrift +++ b/gensrc/thrift/PlanNodes.thrift @@ -619,6 +619,13 @@ struct TOlapScanNode { 40: optional TVectorSearchOptions vector_search_options 41: optional TTableSampleOptions sample_options; + + //back pressure + 50: optional bool enable_topn_filter_back_pressure + 51: optional i32 back_pressure_max_rounds + 52: optional i64 back_pressure_throttle_time + 53: optional i64 back_pressure_throttle_time_upper_bound + 54: optional i64 back_pressure_num_rows } struct TJDBCScanNode { @@ -652,6 +659,13 @@ struct TLakeScanNode { 32: optional bool output_chunk_by_bucket 33: optional bool output_asc_hint 34: optional bool partition_order_hint + + //back pressure + 38: optional bool enable_topn_filter_back_pressure + 39: optional i32 back_pressure_max_rounds + 40: optional i64 back_pressure_throttle_time + 41: optional i64 back_pressure_throttle_time_upper_bound + 42: optional i64 back_pressure_num_rows } struct TEqJoinCondition { diff --git a/test/sql/test_global_dict/R/dict_lake_period_version b/test/sql/test_global_dict/R/dict_lake_period_version index 0a37cd24a937a0..4a29e16ce81c77 100644 --- a/test/sql/test_global_dict/R/dict_lake_period_version +++ b/test/sql/test_global_dict/R/dict_lake_period_version @@ -5,6 +5,9 @@ create external catalog hive_sql_test_${uuid0} PROPERTIES ("type"="hive", "hive. use hive_sql_test_${uuid0}.hive_oss_db; -- result: -- !result +drop table if exists dict_version_test force; +-- result: +-- !result create table dict_version_test ( c1 int, c2 string diff --git a/test/sql/test_global_dict/R/global_dict_on_lake b/test/sql/test_global_dict/R/global_dict_on_lake index 075757818ab414..5c5fba879045c1 100644 --- a/test/sql/test_global_dict/R/global_dict_on_lake +++ b/test/sql/test_global_dict/R/global_dict_on_lake @@ -5,6 +5,9 @@ create external catalog hive_sql_test_${uuid0} PROPERTIES ("type"="hive", "hive. use hive_sql_test_${uuid0}.hive_oss_db; -- result: -- !result +drop table if exists low_card_test force; +-- result: +-- !result create table low_card_test ( c1 int, c2 string diff --git a/test/sql/test_global_dict/T/dict_lake_period_version b/test/sql/test_global_dict/T/dict_lake_period_version index 55007cd7bb1a42..c3700db1340763 100644 --- a/test/sql/test_global_dict/T/dict_lake_period_version +++ b/test/sql/test_global_dict/T/dict_lake_period_version @@ -3,7 +3,7 @@ create external catalog hive_sql_test_${uuid0} PROPERTIES ("type"="hive", "hive.metastore.uris"="${hive_metastore_uris}"); use hive_sql_test_${uuid0}.hive_oss_db; - +drop table if exists dict_version_test force; create table dict_version_test ( c1 int, c2 string diff --git a/test/sql/test_global_dict/T/global_dict_on_lake b/test/sql/test_global_dict/T/global_dict_on_lake index 41ee6730aa37d2..ed1490701bdc1f 100644 --- a/test/sql/test_global_dict/T/global_dict_on_lake +++ b/test/sql/test_global_dict/T/global_dict_on_lake @@ -3,7 +3,7 @@ create external catalog hive_sql_test_${uuid0} PROPERTIES ("type"="hive", "hive.metastore.uris"="${hive_metastore_uris}"); use hive_sql_test_${uuid0}.hive_oss_db; - +drop table if exists low_card_test force; create table low_card_test ( c1 int, c2 string diff --git a/test/sql/test_hive/R/test_hive_catalog b/test/sql/test_hive/R/test_hive_catalog index 406310ab52b988..cad1fe01bb0ae4 100644 --- a/test/sql/test_hive/R/test_hive_catalog +++ b/test/sql/test_hive/R/test_hive_catalog @@ -2,7 +2,15 @@ set enable_dynamic_prune_scan_range=false; -- result: -- !result -create external catalog hive_sql_test_${uuid0} PROPERTIES ("type"="hive", "hive.metastore.uris"="${hive_metastore_uris}"); +create external catalog hive_sql_test_${uuid0} PROPERTIES ("type"="hive", + "hive.metastore.uris"="${hive_metastore_uris}", + "aws.s3.access_key"="${oss_ak}", + "aws.s3.secret_key"="${oss_sk}", + "aws.s3.endpoint"="${oss_endpoint}" +); +-- result: +-- !result +drop table if exists hive_sql_test_${uuid0}.hive_oss_db.not_exist_${uuid0}; -- result: -- !result select * from hive_sql_test_${uuid0}.hive_oss_db.hive_oss_par_parquet_snappy where col_date = (select max(col_date) from hive_sql_test_${uuid0}.hive_oss_db.hive_oss_par_parquet_snappy); @@ -50,4 +58,4 @@ drop catalog unified_catalog_${uuid0}; -- !result drop catalog hive_sql_test_${uuid0}; -- result: --- !result +-- !result \ No newline at end of file diff --git a/test/sql/test_hive/T/test_hive_catalog b/test/sql/test_hive/T/test_hive_catalog index 9d9cc692814a12..3cbd5a7e735b51 100644 --- a/test/sql/test_hive/T/test_hive_catalog +++ b/test/sql/test_hive/T/test_hive_catalog @@ -1,7 +1,15 @@ -- name: testHiveCatalog set enable_dynamic_prune_scan_range=false; -create external catalog hive_sql_test_${uuid0} PROPERTIES ("type"="hive", "hive.metastore.uris"="${hive_metastore_uris}"); +create external catalog hive_sql_test_${uuid0} PROPERTIES ("type"="hive", + "hive.metastore.uris"="${hive_metastore_uris}", + "aws.s3.access_key"="${oss_ak}", + "aws.s3.secret_key"="${oss_sk}", + "aws.s3.endpoint"="${oss_endpoint}" +); + +-- test drop if exists +drop table if exists hive_sql_test_${uuid0}.hive_oss_db.not_exist_${uuid0}; -- only partition column Predicate with runtime filter select * from hive_sql_test_${uuid0}.hive_oss_db.hive_oss_par_parquet_snappy where col_date = (select max(col_date) from hive_sql_test_${uuid0}.hive_oss_db.hive_oss_par_parquet_snappy); diff --git a/test/sql/test_low_cardinality/R/test_low_cardinality2 b/test/sql/test_low_cardinality/R/test_low_cardinality2 index 2a33193dd7f2cd..d59a0350e8805a 100644 --- a/test/sql/test_low_cardinality/R/test_low_cardinality2 +++ b/test/sql/test_low_cardinality/R/test_low_cardinality2 @@ -307,11 +307,11 @@ insert into s2 select * from s2; -- !result [UC] analyze full table s1; -- result: -test_db_3569e679a63f46fcb95f5e935f2b4716.s1 analyze status OK +test_db_b2ce4f72f18e49288a5dbce50a0cb798.s1 analyze status OK -- !result [UC] analyze full table s2; -- result: -test_db_3569e679a63f46fcb95f5e935f2b4716.s2 analyze status OK +test_db_b2ce4f72f18e49288a5dbce50a0cb798.s2 analyze status OK -- !result function: wait_global_dict_ready('v3', 's1') -- result: @@ -460,8 +460,8 @@ where x2 = "anhui" GROUP BY x1 HAVING MAX(y2) = "JX"; -- result: -nauhciS anhui SH1 JX ixgnauG anhui QH1 JX +nauhciS anhui SH1 JX -- !result CREATE TABLE `supplier` ( `s_suppkey` int(11) NOT NULL COMMENT "", @@ -488,7 +488,7 @@ FROM -- !result [UC] analyze full table supplier; -- result: -test_db_3569e679a63f46fcb95f5e935f2b4716.supplier analyze status OK +test_db_b2ce4f72f18e49288a5dbce50a0cb798.supplier analyze status OK -- !result function: wait_global_dict_ready('s_region', 'supplier') -- result: @@ -524,3 +524,10 @@ with a as ( select s_region, S_NAME from supplier where S_SUPPKEY < 10 limit 10 0 0 -- !result +select s_region, array_map(x -> concat(x, date_format(cast(concat('202001', s_region) as datetime), "%Y%m")), ['a', 'b', 'c']) +from supplier where s_suppkey > 4093; +-- result: +15 ["a202001","b202001","c202001"] +14 ["a202001","b202001","c202001"] +0 [null,null,null] +-- !result \ No newline at end of file diff --git a/test/sql/test_low_cardinality/T/test_low_cardinality2 b/test/sql/test_low_cardinality/T/test_low_cardinality2 index a75656da425262..56b594ad9c3401 100644 --- a/test/sql/test_low_cardinality/T/test_low_cardinality2 +++ b/test/sql/test_low_cardinality/T/test_low_cardinality2 @@ -422,3 +422,8 @@ as ( select S_NAME, mx_addr from agged_supplier_2 l group by S_NAME,mx_addr ), agged_supplier_5 as ( select l.S_NAME, l.mx_addr from agged_supplier_4 l join supplier r where r.s_suppkey=10 ) select l.S_NAME,l.mx_addr from agged_supplier_5 l right join [shuffle] supplier z on l.S_NAME = z.S_NAME and l.mx_addr = z.s_region order by 2 nulls last,1 limit 10; with a as ( select s_region, S_NAME from supplier where S_SUPPKEY < 10 limit 10 ), b as ( select max(upper(s_region)) as mx from a group by S_NAME ), c as ( select lower(mx) = lower('lw') from b )select *from c; + + +select s_region, array_map(x -> concat(x, date_format(cast(concat('202001', s_region) as datetime), "%Y%m")), ['a', 'b', 'c']) +from supplier where s_suppkey > 4093; + diff --git a/test/sql/test_topn_filter_throttle_scan/R/test_topn_filter_throttle_scan b/test/sql/test_topn_filter_throttle_scan/R/test_topn_filter_throttle_scan new file mode 100644 index 00000000000000..d374389af3983a --- /dev/null +++ b/test/sql/test_topn_filter_throttle_scan/R/test_topn_filter_throttle_scan @@ -0,0 +1,75 @@ +-- name: test_topn_filter_throttle_scan +DROP TABLE if exists t0; +-- result: +-- !result +CREATE TABLE if not exists t0 +( +c0 INT NOT NULL, +c1 INT NOT NULL +) ENGINE=OLAP +DUPLICATE KEY(`c0` ) +COMMENT "OLAP" +DISTRIBUTED BY HASH(`c0` ) BUCKETS 1 +PROPERTIES( +"replication_num" = "1", +"in_memory" = "false", +"storage_format" = "default" +); +-- result: +-- !result +DROP TABLE if exists t1; +-- result: +-- !result +CREATE TABLE if not exists t1 +( +c0 INT NOT NULL, +c1 INT NOT NULL +) ENGINE=OLAP +DUPLICATE KEY(`c0` ) +COMMENT "OLAP" +DISTRIBUTED BY HASH(`c0` ) BUCKETS 1 +PROPERTIES( +"replication_num" = "1", +"in_memory" = "false", +"storage_format" = "default" +); +-- result: +-- !result +insert into t0 select c0, 0 from Table(generate_series(1,10000)) ta(c0), Table(generate_series(1,1)); +-- result: +-- !result +insert into t0 select t0.* from t0, Table(generate_series(1,10)); +-- result: +-- !result +insert into t1 select c0, c1 from Table(generate_series(1,10000)) ta(c0), Table(generate_series(1,10)) tb(c1); +-- result: +-- !result +set topn_filter_back_pressure_mode='2'; +-- result: +-- !result +set enable_profile='true'; +-- result: +-- !result +set enable_async_profile=false; +-- result: +-- !result +[UC]select t0.c1, repeat('x', t0.c1+t1.c1) as a, repeat('y',t1.c0 + t1.c0) as b from t0 join[broadcast] t1 on t0.c0 = t1.c0 order by 1 limit 10; +-- result: +0 xxxxxxx yy +0 xxxxxxxxx yy +0 xxxxxx yy +0 xxx yy +0 xxxxx yy +0 xx yy +0 x yy +0 xxxx yy +0 xxxxxxxx yy +0 xxxxxxxxxx yy +-- !result +create table profile_table properties("replication_num"="1") as select line from table(unnest(split(get_query_profile(last_query_id()),"\n"))) t(line); +-- result: +-- !result +select assert_true(count(1)>0) from profile_table where line like '%TopnRuntimeFilter%'; +-- result: +1 +-- !result \ No newline at end of file diff --git a/test/sql/test_topn_filter_throttle_scan/T/test_topn_filter_throttle_scan b/test/sql/test_topn_filter_throttle_scan/T/test_topn_filter_throttle_scan new file mode 100644 index 00000000000000..67c0aceb6538d3 --- /dev/null +++ b/test/sql/test_topn_filter_throttle_scan/T/test_topn_filter_throttle_scan @@ -0,0 +1,40 @@ +-- name: test_topn_filter_throttle_scan + DROP TABLE if exists t0; + + CREATE TABLE if not exists t0 + ( + c0 INT NOT NULL, + c1 INT NOT NULL + ) ENGINE=OLAP + DUPLICATE KEY(`c0` ) + COMMENT "OLAP" + DISTRIBUTED BY HASH(`c0` ) BUCKETS 1 + PROPERTIES( + "replication_num" = "1", + "in_memory" = "false", + "storage_format" = "default" + ); + DROP TABLE if exists t1; + + CREATE TABLE if not exists t1 + ( + c0 INT NOT NULL, + c1 INT NOT NULL + ) ENGINE=OLAP + DUPLICATE KEY(`c0` ) + COMMENT "OLAP" + DISTRIBUTED BY HASH(`c0` ) BUCKETS 1 + PROPERTIES( + "replication_num" = "1", + "in_memory" = "false", + "storage_format" = "default" + ); +insert into t0 select c0, 0 from Table(generate_series(1,10000)) ta(c0), Table(generate_series(1,1)); +insert into t0 select t0.* from t0, Table(generate_series(1,10)); +insert into t1 select c0, c1 from Table(generate_series(1,10000)) ta(c0), Table(generate_series(1,10)) tb(c1); +set topn_filter_back_pressure_mode='2'; +set enable_profile='true'; +set enable_async_profile=false; +[UC]select t0.c1, repeat('x', t0.c1+t1.c1) as a, repeat('y',t1.c0 + t1.c0) as b from t0 join[broadcast] t1 on t0.c0 = t1.c0 order by 1 limit 10; +create table profile_table properties("replication_num"="1") as select line from table(unnest(split(get_query_profile(last_query_id()),"\n"))) t(line); +select assert_true(count(1)>0) from profile_table where line like '%TopnRuntimeFilter%';