diff --git a/be/src/agent/task_worker_pool.cpp b/be/src/agent/task_worker_pool.cpp index 9f9590ef84140c..a90bbec7d8941f 100644 --- a/be/src/agent/task_worker_pool.cpp +++ b/be/src/agent/task_worker_pool.cpp @@ -273,8 +273,8 @@ Status check_migrate_request(StorageEngine& engine, const TStorageMediumMigrateR // check local disk capacity int64_t tablet_size = tablet->tablet_local_size(); if ((*dest_store)->reach_capacity_limit(tablet_size)) { - return Status::InternalError("reach the capacity limit of path {}, tablet_size={}", - (*dest_store)->path(), tablet_size); + return Status::Error("reach the capacity limit of path {}, tablet_size={}", + (*dest_store)->path(), tablet_size); } return Status::OK(); } diff --git a/be/src/common/config.cpp b/be/src/common/config.cpp index 0576aaba917a53..c5b3c71ddb935d 100644 --- a/be/src/common/config.cpp +++ b/be/src/common/config.cpp @@ -42,8 +42,7 @@ #include "io/fs/file_writer.h" #include "io/fs/local_file_system.h" -namespace doris { -namespace config { +namespace doris::config { // Dir of custom config file DEFINE_String(custom_config_dir, "${DORIS_HOME}/conf"); @@ -109,10 +108,6 @@ DEFINE_mInt32(hash_table_double_grow_degree, "31"); DEFINE_mInt32(max_fill_rate, "2"); DEFINE_mInt32(double_resize_threshold, "23"); -// Expand the hash table before inserting data, the maximum expansion size. -// There are fewer duplicate keys, reducing the number of resize hash tables -// There are many duplicate keys, and the hash table filled bucket is far less than the hash table build bucket. -DEFINE_mInt64(hash_table_pre_expanse_max_rows, "65535"); // The maximum low water mark of the system `/proc/meminfo/MemAvailable`, Unit byte, default 1.6G, // actual low water mark=min(1.6G, MemTotal * 10%), avoid wasting too much memory on machines @@ -846,16 +841,6 @@ DEFINE_String(function_service_protocol, "h2:grpc"); // use which load balancer to select server to connect DEFINE_String(rpc_load_balancer, "rr"); -// The maximum buffer/queue size to collect span. After the size is reached, spans are dropped. -// An export will be triggered when the number of spans in the queue reaches half of the maximum. -DEFINE_Int32(max_span_queue_size, "2048"); - -// The maximum batch size of every export spans. It must be smaller or equal to max_queue_size. -DEFINE_Int32(max_span_export_batch_size, "512"); - -// The time interval between two consecutive export spans. -DEFINE_Int32(export_span_schedule_delay_millis, "500"); - // a soft limit of string type length, the hard limit is 2GB - 4, but if too long will cause very low performance, // so we set a soft limit, default is 1MB DEFINE_mInt32(string_type_length_soft_limit_bytes, "1048576"); @@ -868,10 +853,6 @@ DEFINE_mInt32(jsonb_type_length_soft_limit_bytes, "1048576"); DEFINE_Validator(jsonb_type_length_soft_limit_bytes, [](const int config) -> bool { return config > 0 && config <= 2147483643; }); -// used for olap scanner to save memory, when the size of unused_object_pool -// is greater than object_pool_buffer_size, release the object in the unused_object_pool. -DEFINE_Int32(object_pool_buffer_size, "100"); - // Threshold of reading a small file into memory DEFINE_mInt32(in_memory_file_size, "1048576"); // 1MB @@ -907,7 +888,7 @@ DEFINE_Int32(concurrency_per_dir, "2"); // "whole_file_cache": the whole file. DEFINE_mString(file_cache_type, "file_block_cache"); DEFINE_Validator(file_cache_type, [](std::string_view config) -> bool { - return config == "" || config == "file_block_cache"; + return config.empty() || config == "file_block_cache"; }); DEFINE_Int32(s3_transfer_executor_pool_size, "2"); @@ -971,8 +952,8 @@ DEFINE_Bool(enable_fuzzy_mode, "false"); DEFINE_Bool(enable_debug_points, "false"); DEFINE_Int32(pipeline_executor_size, "0"); -// 128 MB -DEFINE_mInt64(local_exchange_buffer_mem_limit, "134217728"); +DEFINE_Bool(enable_workload_group_for_scan, "false"); +DEFINE_mInt64(workload_group_scan_task_wait_timeout_ms, "10000"); // Temp config. True to use optimization for bitmap_index apply predicate except leaf node of the and node. // Will remove after fully test. @@ -1153,6 +1134,9 @@ DEFINE_Bool(enable_snapshot_action, "false"); DEFINE_mInt32(variant_max_merged_tablet_schema_size, "2048"); +// 128 MB +DEFINE_mInt64(local_exchange_buffer_mem_limit, "134217728"); + // clang-format off #ifdef BE_TEST // test s3 @@ -1205,7 +1189,7 @@ bool replaceenv(std::string& s) { std::size_t pos = 0; std::size_t start = 0; while ((start = s.find("${", pos)) != std::string::npos) { - std::size_t end = s.find("}", start + 2); + std::size_t end = s.find('}', start + 2); if (end == std::string::npos) { return false; } @@ -1243,9 +1227,9 @@ bool strtox(const std::string& valstr, std::vector& retval) { } bool strtox(const std::string& valstr, bool& retval) { - if (valstr.compare("true") == 0) { + if (valstr == "true") { retval = true; - } else if (valstr.compare("false") == 0) { + } else if (valstr == "false") { retval = false; } else { return false; @@ -1605,18 +1589,17 @@ std::vector> get_config_info() { std::vector _config; _config.push_back(it.first); - _config.push_back(field_it->second.type); + _config.emplace_back(field_it->second.type); if (0 == strcmp(field_it->second.type, "bool")) { - _config.push_back(it.second == "1" ? "true" : "false"); + _config.emplace_back(it.second == "1" ? "true" : "false"); } else { _config.push_back(it.second); } - _config.push_back(field_it->second.valmutable ? "true" : "false"); + _config.emplace_back(field_it->second.valmutable ? "true" : "false"); configs.push_back(_config); } return configs; } -} // namespace config -} // namespace doris +} // namespace doris::config diff --git a/be/src/common/config.h b/be/src/common/config.h index 74f3ba50fabdff..6193c1a5b4207b 100644 --- a/be/src/common/config.h +++ b/be/src/common/config.h @@ -152,11 +152,6 @@ DECLARE_mInt32(max_fill_rate); DECLARE_mInt32(double_resize_threshold); -// Expand the hash table before inserting data, the maximum expansion size. -// There are fewer duplicate keys, reducing the number of resize hash tables -// There are many duplicate keys, and the hash table filled bucket is far less than the hash table build bucket. -DECLARE_mInt64(hash_table_pre_expanse_max_rows); - // The maximum low water mark of the system `/proc/meminfo/MemAvailable`, Unit byte, default 1.6G, // actual low water mark=min(1.6G, MemTotal * 10%), avoid wasting too much memory on machines // with large memory larger than 16G. @@ -908,26 +903,12 @@ DECLARE_String(function_service_protocol); // use which load balancer to select server to connect DECLARE_String(rpc_load_balancer); -// The maximum buffer/queue size to collect span. After the size is reached, spans are dropped. -// An export will be triggered when the number of spans in the queue reaches half of the maximum. -DECLARE_Int32(max_span_queue_size); - -// The maximum batch size of every export spans. It must be smaller or equal to max_queue_size. -DECLARE_Int32(max_span_export_batch_size); - -// The time interval between two consecutive export spans. -DECLARE_Int32(export_span_schedule_delay_millis); - // a soft limit of string type length, the hard limit is 2GB - 4, but if too long will cause very low performance, // so we set a soft limit, default is 1MB DECLARE_mInt32(string_type_length_soft_limit_bytes); DECLARE_mInt32(jsonb_type_length_soft_limit_bytes); -// used for olap scanner to save memory, when the size of unused_object_pool -// is greater than object_pool_buffer_size, release the object in the unused_object_pool. -DECLARE_Int32(object_pool_buffer_size); - // Threshold fo reading a small file into memory DECLARE_mInt32(in_memory_file_size); diff --git a/be/src/common/status.h b/be/src/common/status.h index e854cda360df12..14aec46cc226b6 100644 --- a/be/src/common/status.h +++ b/be/src/common/status.h @@ -8,8 +8,8 @@ #include // for TStatus #include #include -#include +#include #include #include #include @@ -93,7 +93,8 @@ namespace ErrorCode { E(VERSION_NOT_EXIST, -214, false); \ E(TABLE_NOT_FOUND, -215, true); \ E(TRY_LOCK_FAILED, -216, false); \ - E(OUT_OF_BOUND, -218, true); \ + E(EXCEEDED_LIMIT, -217, false); \ + E(OUT_OF_BOUND, -218, false); \ E(INVALID_ROOT_PATH, -222, true); \ E(NO_AVAILABLE_ROOT_PATH, -223, true); \ E(CHECK_LINES_ERROR, -224, true); \ diff --git a/be/src/common/sync_point.cpp b/be/src/common/sync_point.cpp new file mode 100644 index 00000000000000..816c5a82a94bac --- /dev/null +++ b/be/src/common/sync_point.cpp @@ -0,0 +1,236 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +// Most code of this file is copied from rocksdb SyncPoint. +// https://github.com/facebook/rocksdb + +// clang-format off +#include "sync_point.h" + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace doris { + +struct SyncPoint::Data { // impl +public: + Data() : enabled_(false) { } + virtual ~Data() {} + void process(const std::string& point, std::vector&& cb_args); + void load_dependency(const std::vector& dependencies); + void load_dependency_and_markers( + const std::vector& dependencies, + const std::vector& markers); + bool predecessors_all_cleared(const std::string& point); + void set_call_back(const std::string& point, + const std::function&&)>& callback); + void clear_call_back(const std::string& point); + void clear_all_call_backs(); + void enable_processing(); + void disable_processing(); + void clear_trace(); +private: + bool disable_by_marker(const std::string& point, std::thread::id thread_id); +private: + // successor/predecessor map loaded from load_dependency + std::unordered_map> successors_; + std::unordered_map> predecessors_; + std::unordered_map&&)>> callbacks_; + std::unordered_map> markers_; + std::unordered_map marked_thread_id_; + std::mutex mutex_; + std::condition_variable cv_; + // sync points that have been passed through + std::unordered_set cleared_points_; + std::atomic enabled_; + int num_callbacks_running_ = 0; +}; + +SyncPoint* SyncPoint::get_instance() { + static SyncPoint sync_point; + return &sync_point; +} +SyncPoint::SyncPoint() : + impl_(new Data) { +} +SyncPoint:: ~SyncPoint() { + delete impl_; +} +void SyncPoint::load_dependency(const std::vector& dependencies) { + impl_->load_dependency(dependencies); +} +void SyncPoint::load_dependency_and_markers( + const std::vector& dependencies, + const std::vector& markers) { + impl_->load_dependency_and_markers(dependencies, markers); +} +void SyncPoint::set_call_back(const std::string& point, + const std::function&&)>& callback) { + impl_->set_call_back(point, callback); +} +void SyncPoint::clear_call_back(const std::string& point) { + impl_->clear_call_back(point); +} +void SyncPoint::clear_all_call_backs() { + impl_->clear_all_call_backs(); +} +void SyncPoint::enable_processing() { + impl_->enable_processing(); +} +void SyncPoint::disable_processing() { + impl_->disable_processing(); +} +void SyncPoint::clear_trace() { + impl_->clear_trace(); +} +void SyncPoint::process(const std::string& point, std::vector&& cb_arg) { + impl_->process(point, std::move(cb_arg)); +} + +// ============================================================================= +// SyncPoint implementation +// ============================================================================= + +void SyncPoint::Data::load_dependency( + const std::vector& dependencies) { + std::lock_guard lock(mutex_); + successors_.clear(); + predecessors_.clear(); + cleared_points_.clear(); + for (const auto& dependency : dependencies) { + successors_[dependency.predecessor].push_back(dependency.successor); + predecessors_[dependency.successor].push_back(dependency.predecessor); + } + cv_.notify_all(); +} + +/** + * Markers are also dependency descriptions + */ +void SyncPoint::Data::load_dependency_and_markers( + const std::vector& dependencies, + const std::vector& markers) { + std::lock_guard lock(mutex_); + successors_.clear(); + predecessors_.clear(); + cleared_points_.clear(); + markers_.clear(); + marked_thread_id_.clear(); + for (const auto& dependency : dependencies) { + successors_[dependency.predecessor].push_back(dependency.successor); + predecessors_[dependency.successor].push_back(dependency.predecessor); + } + for (const auto& marker : markers) { + successors_[marker.predecessor].push_back(marker.successor); + predecessors_[marker.successor].push_back(marker.predecessor); + markers_[marker.predecessor].push_back(marker.successor); + } + cv_.notify_all(); +} + +bool SyncPoint::Data::predecessors_all_cleared(const std::string& point) { + for (const auto& pred : predecessors_[point]) { + if (cleared_points_.count(pred) == 0) { + return false; + } + } + return true; +} + +void SyncPoint::Data::clear_call_back(const std::string& point) { + std::unique_lock lock(mutex_); + callbacks_.erase(point); +} + +void SyncPoint::Data::clear_all_call_backs() { + std::unique_lock lock(mutex_); + callbacks_.clear(); +} + +void SyncPoint::Data::process(const std::string& point, std::vector&& cb_arg) { + if (!enabled_) { + return; + } + std::unique_lock lock(mutex_); + auto thread_id = std::this_thread::get_id(); + auto marker_iter = markers_.find(point); + // if current sync point is a marker + // record it in marked_thread_id_ for all its successors + if (marker_iter != markers_.end()) { + for (auto& marked_point : marker_iter->second) { + marked_thread_id_.emplace(marked_point, thread_id); + } + } + // if current point is a marker's successor + if (disable_by_marker(point, thread_id)) { + return; + } + while (!predecessors_all_cleared(point)) { + cv_.wait(lock); + if (disable_by_marker(point, thread_id)) { + return; + } + } + auto callback_pair = callbacks_.find(point); + if (callback_pair != callbacks_.end()) { + num_callbacks_running_++; + auto callback = callback_pair->second; + mutex_.unlock(); + callback(std::move(cb_arg)); + mutex_.lock(); + num_callbacks_running_--; + } + cleared_points_.insert(point); + cv_.notify_all(); +} + +bool SyncPoint::Data::disable_by_marker(const std::string& point, + std::thread::id thread_id) { + auto marked_point_iter = marked_thread_id_.find(point); + return marked_point_iter != marked_thread_id_.end() // is a successor + && thread_id != marked_point_iter->second; +} + +void SyncPoint::Data::set_call_back(const std::string& point, + const std::function&&)>& callback) { + std::lock_guard lock(mutex_); + callbacks_[point] = callback; +} + +void SyncPoint::Data::clear_trace() { + std::lock_guard lock(mutex_); + cleared_points_.clear(); +} + +void SyncPoint::Data::enable_processing() { + enabled_ = true; +} + +void SyncPoint::Data::disable_processing() { + enabled_ = false; +} + +} // namespace doris +// clang-format on +// vim: et tw=80 ts=2 sw=2 cc=80: diff --git a/be/src/common/sync_point.h b/be/src/common/sync_point.h new file mode 100644 index 00000000000000..18b3a63c05e700 --- /dev/null +++ b/be/src/common/sync_point.h @@ -0,0 +1,240 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +// Most code of this file is copied from rocksdb SyncPoint. +// https://github.com/facebook/rocksdb + +#pragma once +// clang-format off +#include +#include +#include +#include +#include + +namespace doris { + +#define SYNC_POINT_HOOK_RETURN_VALUE(expr, point_name, ...) \ + [&]() mutable { \ + TEST_SYNC_POINT_RETURN_WITH_VALUE(point_name, decltype((expr)) {}, __VA_ARGS__); \ + return (expr); \ + }() + +// This class provides facility to reproduce race conditions deterministically +// in unit tests. +// Developer could specify sync points in the codebase via TEST_SYNC_POINT. +// Each sync point represents a position in the execution stream of a thread. +// In the unit test, 'Happens After' relationship among sync points could be +// setup via SyncPoint::load_dependency, to reproduce a desired interleave of +// threads execution. +// Refer to (DBTest,TransactionLogIteratorRace), for an example use case. +class SyncPoint { +public: + static SyncPoint* get_instance(); + SyncPoint(const SyncPoint&) = delete; + SyncPoint& operator=(const SyncPoint&) = delete; + ~SyncPoint(); + struct SyncPointPair { + std::string predecessor; + std::string successor; + }; + + // call once at the beginning of a test to setup the dependency between + // sync points + // + // Example: + // load_dependency({{"point1", "point2"}, + // {"point2", "point3"}, + // {"point3", "point4"}}); + // + // test case thread thread for object being tested + // | | + // | | + // | \-------------0-------------\ | + // | \-> x sync point1 set in code + // | /----------1----------------/ | + // point2 o <-/ /-> x sync point4 set in code + // | / | + // z / | + // z /---------2-----------/ | there may be nothing + // | / | between point1 point4 + // ponit3 o --/ | they are for sync + // | | between test case and object + // v v + // + // vertical arrow means the procedure of each thread, the running order will + // be: + // test case thread -> point1 -> point2 -> point3 -> point4 -> object being + // tested + // + // we may do a lot of things between point2 and point3, say, change the + // object's status, call another method, propagate data race and etc. + void load_dependency(const std::vector& dependencies); + + // call once at the beginning of a test to setup the dependency between + // sync points and setup markers indicating the successor is only enabled + // when it is processed on the same thread as the predecessor. + // When adding a marker, it implicitly adds a dependency for the marker pair. + void load_dependency_and_markers( + const std::vector& dependencies, + const std::vector& markers); + + // The argument to the callback is passed through from + // TEST_SYNC_POINT_CALLBACK(); nullptr if TEST_SYNC_POINT or + // TEST_IDX_SYNC_POINT was used. + void set_call_back(const std::string& point, + const std::function&&)>& callback); + + // Clear callback function by point + void clear_call_back(const std::string& point); + + // Clear all call back functions. + void clear_all_call_backs(); + + // Enable sync point processing (disabled on startup) + void enable_processing(); + + // Disable sync point processing + void disable_processing(); + + // Remove the execution trace of all sync points + void clear_trace(); + + // Triggered by TEST_SYNC_POINT, blocking execution until all predecessors + // are executed. + // And/or call registered callback function, with argument `cb_args` + void process(const std::string& point, std::vector&& cb_args = {}); + + // TODO: it might be useful to provide a function that blocks until all + // sync points are cleared. + // We want this to be public so we can subclass the implementation + struct Data; + +private: + // Singleton + SyncPoint(); + Data* impl_; // impletation which is hidden in cpp file +}; + +template +T try_any_cast(const std::any& a) { + try { + return std::any_cast(a); + } catch (const std::bad_any_cast& e) { + std::cerr << e.what() << " expected=" << typeid(T).name() << " actual=" << a.type().name() << std::endl; + throw e; + } +} + +template +auto try_any_cast_ret(std::vector& any) { + return try_any_cast*>(any.back()); +} + +} // namespace doris + +#define SYNC_POINT(x) doris::SyncPoint::get_instance()->process(x) +#define IDX_SYNC_POINT(x, index) \ + doris::SyncPoint::get_instance()->process(x + std::to_string(index)) +#define SYNC_POINT_CALLBACK(x, ...) doris::SyncPoint::get_instance()->process(x, {__VA_ARGS__}) +#define SYNC_POINT_RETURN_WITH_VALUE(x, default_ret_val, ...) \ +{ \ + std::pair ret {default_ret_val, false}; \ + std::vector args {__VA_ARGS__}; \ + args.push_back(&ret); \ + doris::SyncPoint::get_instance()->process(x, std::move(args)); \ + if (ret.second) return std::move(ret.first); \ +} +#define SYNC_POINT_RETURN_WITH_VOID(x, ...) \ +{ \ + bool pred = false; \ + std::vector args {__VA_ARGS__}; \ + args.push_back(&pred); \ + doris::SyncPoint::get_instance()->process(x, std::move(args)); \ + if (pred) return; \ +} +#define SYNC_POINT_SINGLETON() (void)doris::SyncPoint::get_instance() + +// TEST_SYNC_POINT is no op in release build. +// Turn on this feature by defining the macro +#ifndef BE_TEST +# define TEST_SYNC_POINT(x) +# define TEST_IDX_SYNC_POINT(x, index) +# define TEST_SYNC_POINT_CALLBACK(x, ...) +# define TEST_SYNC_POINT_RETURN_WITH_VALUE(x, default_ret_val, ...) +# define TEST_SYNC_POINT_RETURN_WITH_VOID(x, ...) +// seldom called +# define TEST_SYNC_POINT_SINGLETON() +#else +// Use TEST_SYNC_POINT to specify sync points inside code base. +// Sync points can have happens-after depedency on other sync points, +// configured at runtime via SyncPoint::load_dependency. This could be +// utilized to re-produce race conditions between threads. +# define TEST_SYNC_POINT(x) SYNC_POINT(x) +# define TEST_IDX_SYNC_POINT(x, index) IDX_SYNC_POINT(x, index) +# define TEST_SYNC_POINT_CALLBACK(x, ...) SYNC_POINT_CALLBACK(x, __VA_ARGS__) +# define TEST_SYNC_POINT_SINGLETON() SYNC_POINT_SINGLETON() + +/** + * Inject return points for testing. + * + * Currently we can only insert more points to get context from tested thread + * and process in testing thread, e.g. + * + * tested thread: + * ... + * TEST_SYNC_POINT_RETURN_WITH_VALUE("point_ret", int(0), ctx0); + * ... + * + * testing thread: + * sync_point->add("point_ret", [](auto&& args) { + * auto ctx0 = try_any_cast(args[0]); + * auto pair = try_any_cast*>(args.back()); + * pair->first = ...; + * pair->second = ctx0; }); + * + * See sync_piont_test.cpp for more details. + */ +#pragma GCC diagnostic ignored "-Waddress" +# define TEST_SYNC_POINT_RETURN_WITH_VALUE(x, default_ret_val, ...) SYNC_POINT_RETURN_WITH_VALUE(x, default_ret_val, __VA_ARGS__) +# define TEST_SYNC_POINT_RETURN_WITH_VOID(x, ...) SYNC_POINT_RETURN_WITH_VOID(x, __VA_ARGS__) + +#endif // BE_TEST + +// TODO: define injection point in production env. +// the `if` expr can be live configure of the application +#ifndef ENABLE_INJECTION_POINT +# define TEST_INJECTION_POINT(x) +# define TEST_IDX_INJECTION_POINT(x, index) +# define TEST_INJECTION_POINT_CALLBACK(x, ...) +# define TEST_INJECTION_POINT_RETURN_WITH_VALUE(x, default_ret_val, ...) +# define TEST_INJECTION_POINT_RETURN_WITH_VOID(x, ...) +# define TEST_INJECTION_POINT_SINGLETON() +#else +namespace doris::config { +extern bool enable_injection_point; +} +# define TEST_INJECTION_POINT(x) if (doris::config::enable_injection_point) { SYNC_POINT(x); } +# define TEST_IDX_INJECTION_POINT(x, index) if (doris::config::enable_injection_point) { IDX_SYNC_POINT(x, index); } +# define TEST_INJECTION_POINT_CALLBACK(x, ...) if (doris::config::enable_injection_point) { SYNC_POINT_CALLBACK(x, __VA_ARGS__); } +# define TEST_INJECTION_POINT_SINGLETON() if (doris::config::enable_injection_point) { SYNC_POINT_SINGLETON(); } +# define TEST_INJECTION_POINT_RETURN_WITH_VALUE(x, default_ret_val, ...) if (doris::config::enable_injection_point) { SYNC_POINT_RETURN_WITH_VALUE(x, default_ret_val, __VA_ARGS__); } +# define TEST_INJECTION_POINT_RETURN_WITH_VOID(x, ...) if (doris::config::enable_injection_point) { SYNC_POINT_RETURN_WITH_VOID(x, __VA_ARGS__); } +#endif // ENABLE_INJECTION_POINT + +// clang-format on +// vim: et tw=80 ts=2 sw=2 cc=80: diff --git a/be/src/exec/exec_node.h b/be/src/exec/exec_node.h index 123097cfd53d16..eeed37907f9ebe 100644 --- a/be/src/exec/exec_node.h +++ b/be/src/exec/exec_node.h @@ -239,10 +239,6 @@ class ExecNode { size_t children_count() const { return _children.size(); } - // when the fragment is normal finished, call this method to do some finish work - // such as send the last buffer to remote. - virtual Status try_close(RuntimeState* state) { return Status::OK(); } - protected: friend class DataSink; diff --git a/be/src/http/action/http_stream.cpp b/be/src/http/action/http_stream.cpp index ef4decb0381959..8c75f6be7402ac 100644 --- a/be/src/http/action/http_stream.cpp +++ b/be/src/http/action/http_stream.cpp @@ -194,7 +194,7 @@ int HttpStreamAction::on_header(HttpRequest* req) { << " Bytes) exceeds the WAL (Write-Ahead Log) limit (" << config::wal_max_disk_size * 0.8 << " Bytes). Please set this load to \"group commit\"=false."; - st = Status::InternalError("Http load size too large."); + st = Status::Error("Http load size too large."); } } } diff --git a/be/src/http/action/stream_load.cpp b/be/src/http/action/stream_load.cpp index 7d336a932b676d..04c29a53027823 100644 --- a/be/src/http/action/stream_load.cpp +++ b/be/src/http/action/stream_load.cpp @@ -219,7 +219,7 @@ int StreamLoadAction::on_header(HttpRequest* req) { << " Bytes) exceeds the WAL (Write-Ahead Log) limit (" << config::wal_max_disk_size * 0.8 << " Bytes). Please set this load to \"group commit\"=false."; - st = Status::InternalError("Stream load size too large."); + st = Status::Error("Stream load size too large."); } } } diff --git a/be/src/http/action/tablet_migration_action.cpp b/be/src/http/action/tablet_migration_action.cpp index 3ccdde7e7ff34c..df75403497a76d 100644 --- a/be/src/http/action/tablet_migration_action.cpp +++ b/be/src/http/action/tablet_migration_action.cpp @@ -23,6 +23,7 @@ #include #include "common/config.h" +#include "common/status.h" #include "http/http_channel.h" #include "http/http_headers.h" #include "http/http_request.h" @@ -209,7 +210,9 @@ Status TabletMigrationAction::_check_migrate_request(int64_t tablet_id, int32_t if ((*dest_store)->reach_capacity_limit(tablet_size)) { LOG(WARNING) << "reach the capacity limit of path: " << (*dest_store)->path() << ", tablet size: " << tablet_size; - return Status::InternalError("Insufficient disk capacity"); + return Status::Error( + "reach the capacity limit of path {}, tablet_size={}", (*dest_store)->path(), + tablet_size); } return Status::OK(); diff --git a/be/src/olap/block_column_predicate.h b/be/src/olap/block_column_predicate.h index c2583e042a6eae..0ddebec3c18105 100644 --- a/be/src/olap/block_column_predicate.h +++ b/be/src/olap/block_column_predicate.h @@ -77,6 +77,8 @@ class BlockColumnPredicate { return true; } + virtual bool support_zonemap() const { return true; } + virtual bool evaluate_and(const std::pair& statistic) const { LOG(FATAL) << "should not reach here"; return true; @@ -118,6 +120,7 @@ class SingleColumnBlockPredicate : public BlockColumnPredicate { uint16_t selected_size) const override; void evaluate_and(vectorized::MutableColumns& block, uint16_t* sel, uint16_t selected_size, bool* flags) const override; + bool support_zonemap() const override { return _predicate->support_zonemap(); } bool evaluate_and(const std::pair& statistic) const override; bool evaluate_and(const segment_v2::BloomFilter* bf) const override; bool evaluate_and(const StringRef* dict_words, const size_t dict_num) const override; @@ -148,6 +151,16 @@ class MutilColumnBlockPredicate : public BlockColumnPredicate { } } + bool support_zonemap() const override { + for (const auto* child_block_predicate : _block_column_predicate_vec) { + if (!child_block_predicate->support_zonemap()) { + return false; + } + } + + return true; + } + void add_column_predicate(const BlockColumnPredicate* column_predicate) { _block_column_predicate_vec.push_back(column_predicate); } diff --git a/be/src/olap/column_predicate.h b/be/src/olap/column_predicate.h index 42d9a3d58f84d8..5c3ad1350627ec 100644 --- a/be/src/olap/column_predicate.h +++ b/be/src/olap/column_predicate.h @@ -192,6 +192,8 @@ class ColumnPredicate { virtual void evaluate_or(const vectorized::IColumn& column, const uint16_t* sel, uint16_t size, bool* flags) const {} + virtual bool support_zonemap() const { return true; } + virtual bool evaluate_and(const std::pair& statistic) const { return true; } diff --git a/be/src/olap/match_predicate.h b/be/src/olap/match_predicate.h index 915bfd445e6445..776c9c4c258e6d 100644 --- a/be/src/olap/match_predicate.h +++ b/be/src/olap/match_predicate.h @@ -50,6 +50,8 @@ class MatchPredicate : public ColumnPredicate { const std::string& get_value() const { return _value; } + bool support_zonemap() const override { return false; } + //evaluate predicate on Bitmap virtual Status evaluate(BitmapIndexIterator* iterator, uint32_t num_rows, roaring::Roaring* roaring) const override { diff --git a/be/src/olap/olap_common.h b/be/src/olap/olap_common.h index 1921902a9d038b..42bad24dfed5b1 100644 --- a/be/src/olap/olap_common.h +++ b/be/src/olap/olap_common.h @@ -325,6 +325,7 @@ struct OlapReaderStatistics { int64_t rows_key_range_filtered = 0; int64_t rows_stats_filtered = 0; + int64_t rows_stats_rp_filtered = 0; int64_t rows_bf_filtered = 0; int64_t rows_dict_filtered = 0; // Including the number of rows filtered out according to the Delete information in the Tablet, @@ -337,6 +338,10 @@ struct OlapReaderStatistics { // the number of rows filtered by various column indexes. int64_t rows_conditions_filtered = 0; int64_t block_conditions_filtered_ns = 0; + int64_t block_conditions_filtered_bf_ns = 0; + int64_t block_conditions_filtered_zonemap_ns = 0; + int64_t block_conditions_filtered_zonemap_rp_ns = 0; + int64_t block_conditions_filtered_dict_ns = 0; int64_t index_load_ns = 0; diff --git a/be/src/olap/rowset/segment_v2/inverted_index_cache.cpp b/be/src/olap/rowset/segment_v2/inverted_index_cache.cpp index 806d7473ca9b89..0440f6865a4a53 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index_cache.cpp +++ b/be/src/olap/rowset/segment_v2/inverted_index_cache.cpp @@ -21,12 +21,14 @@ #include #include // IWYU pragma: no_include -#include // IWYU pragma: keep -#include #include + +#include // IWYU pragma: keep +#include // IWYU pragma: no_include #include // IWYU pragma: keep #include +#include #include "common/logging.h" #include "olap/olap_common.h" @@ -38,8 +40,7 @@ #include "util/defer_op.h" #include "util/runtime_profile.h" -namespace doris { -namespace segment_v2 { +namespace doris::segment_v2 { Status FulltextIndexSearcherBuilder::build(DorisCompoundReader* directory, OptionalIndexSearcherPtr& output_searcher) { @@ -109,8 +110,7 @@ InvertedIndexSearcherCache::InvertedIndexSearcherCache(size_t capacity, uint32_t if (config::enable_inverted_index_cache_check_timestamp) { auto get_last_visit_time = [](const void* value) -> int64_t { - InvertedIndexSearcherCache::CacheValue* cache_value = - (InvertedIndexSearcherCache::CacheValue*)value; + auto* cache_value = (InvertedIndexSearcherCache::CacheValue*)value; return cache_value->last_visit_time; }; _cache = std::unique_ptr( @@ -146,8 +146,7 @@ Status InvertedIndexSearcherCache::get_index_searcher( cache_handle->owned = !use_cache; IndexSearcherPtr index_searcher; std::unique_ptr index_builder = nullptr; - auto mem_tracker = - std::unique_ptr(new MemTracker("InvertedIndexSearcherCacheWithRead")); + auto mem_tracker = std::make_unique("InvertedIndexSearcherCacheWithRead"); #ifndef BE_TEST { bool exists = false; @@ -280,7 +279,7 @@ Status InvertedIndexSearcherCache::insert(const io::FileSystemSPtr& fs, cache_value->index_searcher = std::move(index_searcher); cache_value->size = mem_tracker->consumption(); cache_value->last_visit_time = UnixMillis(); - auto lru_handle = _insert(cache_key, cache_value.release()); + auto* lru_handle = _insert(cache_key, cache_value.release()); _cache->release(lru_handle); return Status::OK(); } @@ -300,7 +299,7 @@ int64_t InvertedIndexSearcherCache::mem_consumption() { bool InvertedIndexSearcherCache::_lookup(const InvertedIndexSearcherCache::CacheKey& key, InvertedIndexCacheHandle* handle) { - auto lru_handle = _cache->lookup(key.index_file_path); + auto* lru_handle = _cache->lookup(key.index_file_path); if (lru_handle == nullptr) { return false; } @@ -311,8 +310,7 @@ bool InvertedIndexSearcherCache::_lookup(const InvertedIndexSearcherCache::Cache Cache::Handle* InvertedIndexSearcherCache::_insert(const InvertedIndexSearcherCache::CacheKey& key, CacheValue* value) { auto deleter = [](const doris::CacheKey& key, void* value) { - InvertedIndexSearcherCache::CacheValue* cache_value = - (InvertedIndexSearcherCache::CacheValue*)value; + auto* cache_value = (InvertedIndexSearcherCache::CacheValue*)value; delete cache_value; }; @@ -325,7 +323,7 @@ bool InvertedIndexQueryCache::lookup(const CacheKey& key, InvertedIndexQueryCach if (key.encode().empty()) { return false; } - auto lru_handle = _cache->lookup(key.encode()); + auto* lru_handle = _cache->lookup(key.encode()); if (lru_handle == nullptr) { return false; } @@ -348,8 +346,8 @@ void InvertedIndexQueryCache::insert(const CacheKey& key, std::shared_ptrinsert(key.encode(), (void*)cache_value_ptr.release(), - bitmap->getSizeInBytes(), deleter, CachePriority::NORMAL); + auto* lru_handle = _cache->insert(key.encode(), (void*)cache_value_ptr.release(), + bitmap->getSizeInBytes(), deleter, CachePriority::NORMAL); *handle = InvertedIndexQueryCacheHandle(_cache.get(), lru_handle); } @@ -360,5 +358,4 @@ int64_t InvertedIndexQueryCache::mem_consumption() { return 0L; } -} // namespace segment_v2 -} // namespace doris +} // namespace doris::segment_v2 diff --git a/be/src/olap/rowset/segment_v2/inverted_index_writer.cpp b/be/src/olap/rowset/segment_v2/inverted_index_writer.cpp index aab2a5a73f0736..c2cc0bbbefa728 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index_writer.cpp +++ b/be/src/olap/rowset/segment_v2/inverted_index_writer.cpp @@ -258,7 +258,6 @@ class InvertedIndexColumnWriterImpl : public InvertedIndexColumnWriter { } for (int i = 0; i < count; ++i) { - new_fulltext_field(empty_value.c_str(), 0); RETURN_IF_ERROR(add_null_document()); } } @@ -305,13 +304,10 @@ class InvertedIndexColumnWriterImpl : public InvertedIndexColumnWriter { get_parser_ignore_above_value_from_properties(_index_meta->properties()); auto ignore_above = std::stoi(ignore_above_value); for (int i = 0; i < count; ++i) { - // only ignore_above UNTOKENIZED strings - if (_parser_type == InvertedIndexParserType::PARSER_NONE && - v->get_size() > ignore_above) { - VLOG_DEBUG << "fulltext index value length can be at most " - << ignore_above_value << ", but got " - << "value length:" << v->get_size() << ", ignore this value"; - new_fulltext_field(empty_value.c_str(), 0); + // only ignore_above UNTOKENIZED strings and empty strings not tokenized + if ((_parser_type == InvertedIndexParserType::PARSER_NONE && + v->get_size() > ignore_above) || + (_parser_type != InvertedIndexParserType::PARSER_NONE && v->empty())) { RETURN_IF_ERROR(add_null_document()); } else { new_fulltext_field(v->get_data(), v->get_size()); @@ -358,13 +354,10 @@ class InvertedIndexColumnWriterImpl : public InvertedIndexColumnWriter { } auto value = join(strings, " "); - // only ignore_above UNTOKENIZED strings - if (_parser_type == InvertedIndexParserType::PARSER_NONE && - value.length() > ignore_above) { - VLOG_DEBUG << "fulltext index value length can be at most " - << ignore_above_value << ", but got " - << "value length:" << value.length() << ", ignore this value"; - new_fulltext_field(empty_value.c_str(), 0); + // only ignore_above UNTOKENIZED strings and empty strings not tokenized + if ((_parser_type == InvertedIndexParserType::PARSER_NONE && + value.length() > ignore_above) || + (_parser_type != InvertedIndexParserType::PARSER_NONE && value.empty())) { RETURN_IF_ERROR(add_null_document()); } else { new_fulltext_field(value.c_str(), value.length()); diff --git a/be/src/olap/rowset/segment_v2/segment_iterator.cpp b/be/src/olap/rowset/segment_v2/segment_iterator.cpp index 20382269b65a62..483e0e502df8da 100644 --- a/be/src/olap/rowset/segment_v2/segment_iterator.cpp +++ b/be/src/olap/rowset/segment_v2/segment_iterator.cpp @@ -529,90 +529,113 @@ Status SegmentIterator::_get_row_ranges_from_conditions(RowRanges* condition_row cids.insert(entry.first); } - // first filter data by bloom filter index - // bloom filter index only use CondColumn - RowRanges bf_row_ranges = RowRanges::create_single(num_rows()); - for (auto& cid : cids) { - DCHECK(_opts.col_id_to_predicates.count(cid) > 0); - if (!_segment->can_apply_predicate_safely(cid, _opts.col_id_to_predicates.at(cid).get(), - *_schema, _opts.io_ctx.reader_type)) { - continue; - } - // get row ranges by bf index of this column, - RowRanges column_bf_row_ranges = RowRanges::create_single(num_rows()); - RETURN_IF_ERROR(_column_iterators[cid]->get_row_ranges_by_bloom_filter( - _opts.col_id_to_predicates.at(cid).get(), &column_bf_row_ranges)); - RowRanges::ranges_intersection(bf_row_ranges, column_bf_row_ranges, &bf_row_ranges); - } + size_t pre_size = 0; - size_t pre_size = condition_row_ranges->count(); - RowRanges::ranges_intersection(*condition_row_ranges, bf_row_ranges, condition_row_ranges); - _opts.stats->rows_bf_filtered += (pre_size - condition_row_ranges->count()); - - RowRanges zone_map_row_ranges = RowRanges::create_single(num_rows()); - // second filter data by zone map - for (auto& cid : cids) { - DCHECK(_opts.col_id_to_predicates.count(cid) > 0); - if (!_segment->can_apply_predicate_safely(cid, _opts.col_id_to_predicates.at(cid).get(), - *_schema, _opts.io_ctx.reader_type)) { - continue; + { + SCOPED_RAW_TIMER(&_opts.stats->block_conditions_filtered_bf_ns); + // first filter data by bloom filter index + // bloom filter index only use CondColumn + RowRanges bf_row_ranges = RowRanges::create_single(num_rows()); + for (auto& cid : cids) { + DCHECK(_opts.col_id_to_predicates.count(cid) > 0); + if (!_segment->can_apply_predicate_safely(cid, _opts.col_id_to_predicates.at(cid).get(), + *_schema, _opts.io_ctx.reader_type)) { + continue; + } + // get row ranges by bf index of this column, + RowRanges column_bf_row_ranges = RowRanges::create_single(num_rows()); + RETURN_IF_ERROR(_column_iterators[cid]->get_row_ranges_by_bloom_filter( + _opts.col_id_to_predicates.at(cid).get(), &column_bf_row_ranges)); + RowRanges::ranges_intersection(bf_row_ranges, column_bf_row_ranges, &bf_row_ranges); } - // get row ranges by zone map of this column, - RowRanges column_row_ranges = RowRanges::create_single(num_rows()); - RETURN_IF_ERROR(_column_iterators[cid]->get_row_ranges_by_zone_map( - _opts.col_id_to_predicates.at(cid).get(), - _opts.del_predicates_for_zone_map.count(cid) > 0 - ? &(_opts.del_predicates_for_zone_map.at(cid)) - : nullptr, - &column_row_ranges)); - // intersect different columns's row ranges to get final row ranges by zone map - RowRanges::ranges_intersection(zone_map_row_ranges, column_row_ranges, - &zone_map_row_ranges); - } - std::shared_ptr runtime_predicate = nullptr; - if (_opts.use_topn_opt) { - auto query_ctx = _opts.runtime_state->get_query_ctx(); - runtime_predicate = query_ctx->get_runtime_predicate().get_predictate(); - if (runtime_predicate && _segment->can_apply_predicate_safely( - runtime_predicate->column_id(), runtime_predicate.get(), - *_schema, _opts.io_ctx.reader_type)) { - AndBlockColumnPredicate and_predicate; - auto single_predicate = new SingleColumnBlockPredicate(runtime_predicate.get()); - and_predicate.add_column_predicate(single_predicate); - - RowRanges column_rp_row_ranges = RowRanges::create_single(num_rows()); - RETURN_IF_ERROR( - _column_iterators[runtime_predicate->column_id()]->get_row_ranges_by_zone_map( - &and_predicate, nullptr, &column_rp_row_ranges)); + pre_size = condition_row_ranges->count(); + RowRanges::ranges_intersection(*condition_row_ranges, bf_row_ranges, condition_row_ranges); + _opts.stats->rows_bf_filtered += (pre_size - condition_row_ranges->count()); + } + { + SCOPED_RAW_TIMER(&_opts.stats->block_conditions_filtered_zonemap_ns); + RowRanges zone_map_row_ranges = RowRanges::create_single(num_rows()); + // second filter data by zone map + for (auto& cid : cids) { + DCHECK(_opts.col_id_to_predicates.count(cid) > 0); + if (!_segment->can_apply_predicate_safely(cid, _opts.col_id_to_predicates.at(cid).get(), + *_schema, _opts.io_ctx.reader_type)) { + continue; + } + // do not check zonemap if predicate does not support zonemap + if (!_opts.col_id_to_predicates.at(cid)->support_zonemap()) { + VLOG_DEBUG << "skip zonemap for column " << cid; + continue; + } + // get row ranges by zone map of this column, + RowRanges column_row_ranges = RowRanges::create_single(num_rows()); + RETURN_IF_ERROR(_column_iterators[cid]->get_row_ranges_by_zone_map( + _opts.col_id_to_predicates.at(cid).get(), + _opts.del_predicates_for_zone_map.count(cid) > 0 + ? &(_opts.del_predicates_for_zone_map.at(cid)) + : nullptr, + &column_row_ranges)); // intersect different columns's row ranges to get final row ranges by zone map - RowRanges::ranges_intersection(zone_map_row_ranges, column_rp_row_ranges, + RowRanges::ranges_intersection(zone_map_row_ranges, column_row_ranges, &zone_map_row_ranges); } - } - pre_size = condition_row_ranges->count(); - RowRanges::ranges_intersection(*condition_row_ranges, zone_map_row_ranges, - condition_row_ranges); - _opts.stats->rows_stats_filtered += (pre_size - condition_row_ranges->count()); + pre_size = condition_row_ranges->count(); + RowRanges::ranges_intersection(*condition_row_ranges, zone_map_row_ranges, + condition_row_ranges); - /// Low cardinality optimization is currently not very stable, so to prevent data corruption, - /// we are temporarily disabling its use in data compaction. - if (_opts.io_ctx.reader_type == ReaderType::READER_QUERY) { - RowRanges dict_row_ranges = RowRanges::create_single(num_rows()); - for (auto cid : cids) { - RowRanges tmp_row_ranges = RowRanges::create_single(num_rows()); - DCHECK(_opts.col_id_to_predicates.count(cid) > 0); - RETURN_IF_ERROR(_column_iterators[cid]->get_row_ranges_by_dict( - _opts.col_id_to_predicates.at(cid).get(), &tmp_row_ranges)); - RowRanges::ranges_intersection(dict_row_ranges, tmp_row_ranges, &dict_row_ranges); + std::shared_ptr runtime_predicate = nullptr; + if (_opts.use_topn_opt) { + SCOPED_RAW_TIMER(&_opts.stats->block_conditions_filtered_zonemap_ns); + auto query_ctx = _opts.runtime_state->get_query_ctx(); + runtime_predicate = query_ctx->get_runtime_predicate().get_predictate(); + if (runtime_predicate && + _segment->can_apply_predicate_safely(runtime_predicate->column_id(), + runtime_predicate.get(), *_schema, + _opts.io_ctx.reader_type)) { + AndBlockColumnPredicate and_predicate; + auto single_predicate = new SingleColumnBlockPredicate(runtime_predicate.get()); + and_predicate.add_column_predicate(single_predicate); + + RowRanges column_rp_row_ranges = RowRanges::create_single(num_rows()); + RETURN_IF_ERROR(_column_iterators[runtime_predicate->column_id()] + ->get_row_ranges_by_zone_map(&and_predicate, nullptr, + &column_rp_row_ranges)); + + // intersect different columns's row ranges to get final row ranges by zone map + RowRanges::ranges_intersection(zone_map_row_ranges, column_rp_row_ranges, + &zone_map_row_ranges); + } } - pre_size = condition_row_ranges->count(); - RowRanges::ranges_intersection(*condition_row_ranges, dict_row_ranges, + size_t pre_size2 = condition_row_ranges->count(); + RowRanges::ranges_intersection(*condition_row_ranges, zone_map_row_ranges, condition_row_ranges); - _opts.stats->rows_dict_filtered += (pre_size - condition_row_ranges->count()); + _opts.stats->rows_stats_rp_filtered += (pre_size2 - condition_row_ranges->count()); + _opts.stats->rows_stats_filtered += (pre_size - condition_row_ranges->count()); + } + + { + SCOPED_RAW_TIMER(&_opts.stats->block_conditions_filtered_dict_ns); + /// Low cardinality optimization is currently not very stable, so to prevent data corruption, + /// we are temporarily disabling its use in data compaction. + if (_opts.io_ctx.reader_type == ReaderType::READER_QUERY) { + RowRanges dict_row_ranges = RowRanges::create_single(num_rows()); + for (auto cid : cids) { + RowRanges tmp_row_ranges = RowRanges::create_single(num_rows()); + DCHECK(_opts.col_id_to_predicates.count(cid) > 0); + RETURN_IF_ERROR(_column_iterators[cid]->get_row_ranges_by_dict( + _opts.col_id_to_predicates.at(cid).get(), &tmp_row_ranges)); + RowRanges::ranges_intersection(dict_row_ranges, tmp_row_ranges, &dict_row_ranges); + } + + pre_size = condition_row_ranges->count(); + RowRanges::ranges_intersection(*condition_row_ranges, dict_row_ranges, + condition_row_ranges); + _opts.stats->rows_dict_filtered += (pre_size - condition_row_ranges->count()); + } } return Status::OK(); diff --git a/be/src/olap/schema_change.cpp b/be/src/olap/schema_change.cpp index c9a1283ede3af3..fd4bf312c90fcd 100644 --- a/be/src/olap/schema_change.cpp +++ b/be/src/olap/schema_change.cpp @@ -1339,10 +1339,10 @@ Status SchemaChangeHandler::_parse_request(const SchemaChangeParams& sc_params, Status SchemaChangeHandler::_init_column_mapping(ColumnMapping* column_mapping, const TabletColumn& column_schema, const std::string& value) { - column_mapping->default_value = WrapperField::create(column_schema); - - if (column_mapping->default_value == nullptr) { - return Status::Error("column_mapping->default_value is nullptr"); + if (auto field = WrapperField::create(column_schema); field.has_value()) { + column_mapping->default_value = field.value(); + } else { + return field.error(); } if (column_schema.is_nullable() && value.length() == 0) { diff --git a/be/src/olap/single_replica_compaction.cpp b/be/src/olap/single_replica_compaction.cpp index d7f4e54f1085c9..a5e060147d6b36 100644 --- a/be/src/olap/single_replica_compaction.cpp +++ b/be/src/olap/single_replica_compaction.cpp @@ -437,7 +437,9 @@ Status SingleReplicaCompaction::_download_files(DataDir* data_dir, HttpClient::execute_with_retry(DOWNLOAD_FILE_MAX_RETRY, 1, get_file_size_cb)); // check disk capacity if (data_dir->reach_capacity_limit(file_size)) { - return Status::InternalError("Disk reach capacity limit"); + return Status::Error( + "reach the capacity limit of path {}, file_size={}", data_dir->path(), + file_size); } total_file_size += file_size; diff --git a/be/src/olap/storage_engine.cpp b/be/src/olap/storage_engine.cpp index 4fb248d24e09b8..4e0fb2eddf19f6 100644 --- a/be/src/olap/storage_engine.cpp +++ b/be/src/olap/storage_engine.cpp @@ -402,7 +402,9 @@ Status StorageEngine::_check_file_descriptor_number() { LOG(ERROR) << "File descriptor number is less than " << config::min_file_descriptor_number << ". Please use (ulimit -n) to set a value equal or greater than " << config::min_file_descriptor_number; - return Status::InternalError("file descriptors limit is too small"); + return Status::Error( + "file descriptors limit {} is small than {}", l.rlim_cur, + config::min_file_descriptor_number); } return Status::OK(); } diff --git a/be/src/olap/tablet.cpp b/be/src/olap/tablet.cpp index 47fd726b375ecd..a5f34633d79240 100644 --- a/be/src/olap/tablet.cpp +++ b/be/src/olap/tablet.cpp @@ -3016,7 +3016,10 @@ Status Tablet::calc_segment_delete_bitmap(RowsetSharedPtr rowset, auto st = lookup_row_key(key, true, specified_rowsets, &loc, dummy_version.first - 1, segment_caches, &rowset_find); bool expected_st = st.ok() || st.is() || st.is(); - DCHECK(expected_st) << "unexpected error status while lookup_row_key:" << st; + // It's a defensive DCHECK, we need to exclude some common errors to avoid core-dump + // while stress test + DCHECK(expected_st || st.is()) + << "unexpected error status while lookup_row_key:" << st; if (!expected_st) { return st; } diff --git a/be/src/olap/task/engine_batch_load_task.cpp b/be/src/olap/task/engine_batch_load_task.cpp index 756d8012b57b35..6f4ff53e77c562 100644 --- a/be/src/olap/task/engine_batch_load_task.cpp +++ b/be/src/olap/task/engine_batch_load_task.cpp @@ -110,7 +110,7 @@ Status EngineBatchLoadTask::_init() { // check disk capacity if (_push_req.push_type == TPushType::LOAD_V2) { - if (tablet->data_dir()->reach_capacity_limit(_push_req.__isset.http_file_size)) { + if (tablet->data_dir()->reach_capacity_limit(_push_req.http_file_size)) { return Status::IOError("Disk does not have enough capacity"); } } diff --git a/be/src/olap/task/engine_clone_task.cpp b/be/src/olap/task/engine_clone_task.cpp index 997a5228c845b6..581cd3515c204a 100644 --- a/be/src/olap/task/engine_clone_task.cpp +++ b/be/src/olap/task/engine_clone_task.cpp @@ -515,7 +515,9 @@ Status EngineCloneTask::_download_files(DataDir* data_dir, const std::string& re HttpClient::execute_with_retry(DOWNLOAD_FILE_MAX_RETRY, 1, get_file_size_cb)); // check disk capacity if (data_dir->reach_capacity_limit(file_size)) { - return Status::InternalError("Disk reach capacity limit"); + return Status::Error( + "reach the capacity limit of path {}, file_size={}", data_dir->path(), + file_size); } total_file_size += file_size; diff --git a/be/src/olap/task/engine_storage_migration_task.cpp b/be/src/olap/task/engine_storage_migration_task.cpp index e896b9f3a8f512..8fba16c67df85c 100644 --- a/be/src/olap/task/engine_storage_migration_task.cpp +++ b/be/src/olap/task/engine_storage_migration_task.cpp @@ -24,6 +24,7 @@ #include #include #include +#include #include #include #include @@ -195,6 +196,21 @@ Status EngineStorageMigrationTask::_migrate() { int32_t end_version = 0; std::vector consistent_rowsets; + // During migration, if the rowsets being migrated undergoes a compaction operation, + // that will result in incorrect delete bitmaps after migration for mow table. Therefore, + // compaction will be prohibited for the mow table when migration. Moreover, it is useless + // to perform a compaction operation on the migration data, as the migration still migrates + // the data of rowsets before the compaction operation. + std::unique_lock full_compaction_lock(_tablet->get_full_compaction_lock(), std::defer_lock); + std::unique_lock base_compaction_lock(_tablet->get_base_compaction_lock(), std::defer_lock); + std::unique_lock cumu_compaction_lock(_tablet->get_cumulative_compaction_lock(), + std::defer_lock); + if (_tablet->enable_unique_key_merge_on_write()) { + full_compaction_lock.lock(); + base_compaction_lock.lock(); + cumu_compaction_lock.lock(); + } + // try hold migration lock first Status res; uint64_t shard = 0; diff --git a/be/src/olap/wal_table.cpp b/be/src/olap/wal_table.cpp index bde0e8dd69d65f..7f98c410b0779a 100644 --- a/be/src/olap/wal_table.cpp +++ b/be/src/olap/wal_table.cpp @@ -204,7 +204,17 @@ Status WalTable::_replay_wal_internal(const std::string& wal) { if (!st.ok()) { LOG(WARNING) << "abort txn " << wal_id << " fail"; } - RETURN_IF_ERROR(_get_column_info(_db_id, _table_id)); + auto get_st = _get_column_info(_db_id, _table_id); + if (!get_st.ok()) { + if (get_st.is()) { + { + std::lock_guard lock(_replay_wal_lock); + _replay_wal_map.erase(wal); + } + RETURN_IF_ERROR(_delete_wal(wal_id)); + } + return get_st; + } #endif RETURN_IF_ERROR(_send_request(wal_id, wal, label)); return Status::OK(); @@ -354,8 +364,7 @@ Status WalTable::_send_request(int64_t wal_id, const std::string& wal, const std } } else { LOG(INFO) << "success to replay wal =" << wal; - RETURN_IF_ERROR(_exec_env->wal_mgr()->delete_wal(wal_id)); - RETURN_IF_ERROR(_exec_env->wal_mgr()->erase_wal_status_queue(_table_id, wal_id)); + RETURN_IF_ERROR(_delete_wal(wal_id)); std::lock_guard lock(_replay_wal_lock); if (_replay_wal_map.erase(wal)) { LOG(INFO) << "erase " << wal << " from _replay_wal_map"; @@ -414,26 +423,21 @@ Status WalTable::_get_column_info(int64_t db_id, int64_t tb_id) { [&request, &result](FrontendServiceConnection& client) { client->getColumnInfo(result, request); })); - std::string columns_str = result.column_info; - std::vector column_element; - doris::vectorized::WalReader::string_split(columns_str, ",", column_element); + status = Status::create(result.status); + if (!status.ok()) { + return status; + } + std::vector column_element = result.columns; int64_t column_index = 1; _column_id_name_map.clear(); _column_id_index_map.clear(); for (auto column : column_element) { - auto pos = column.find(":"); - try { - auto column_name = column.substr(0, pos); - int64_t column_id = std::strtoll(column.substr(pos + 1).c_str(), NULL, 10); - _column_id_name_map.emplace(column_id, column_name); - _column_id_index_map.emplace(column_id, column_index); - column_index++; - } catch (const std::invalid_argument& e) { - return Status::InvalidArgument("Invalid format, {}", e.what()); - } + auto column_name = column.columnName; + auto column_id = column.columnId; + _column_id_name_map.emplace(column_id, column_name); + _column_id_index_map.emplace(column_id, column_index); + column_index++; } - - status = Status::create(result.status); } return status; } @@ -447,4 +451,10 @@ Status WalTable::_read_wal_header(const std::string& wal_path, std::string& colu return Status::OK(); } +Status WalTable::_delete_wal(int64_t wal_id) { + RETURN_IF_ERROR(_exec_env->wal_mgr()->delete_wal(wal_id)); + RETURN_IF_ERROR(_exec_env->wal_mgr()->erase_wal_status_queue(_table_id, wal_id)); + return Status::OK(); +} + } // namespace doris \ No newline at end of file diff --git a/be/src/olap/wal_table.h b/be/src/olap/wal_table.h index 354f4f16b05cc4..e3d66d577a2f93 100644 --- a/be/src/olap/wal_table.h +++ b/be/src/olap/wal_table.h @@ -50,6 +50,7 @@ class WalTable { Status _read_wal_header(const std::string& wal, std::string& columns); bool _need_replay(const replay_wal_info& info); Status _replay_wal_internal(const std::string& wal); + Status _delete_wal(int64_t wal_id); private: ExecEnv* _exec_env; diff --git a/be/src/olap/wrapper_field.cpp b/be/src/olap/wrapper_field.cpp index cc4cad38efbce4..71386900e48bbe 100644 --- a/be/src/olap/wrapper_field.cpp +++ b/be/src/olap/wrapper_field.cpp @@ -18,21 +18,23 @@ #include "olap/wrapper_field.h" #include -#include #include +#include #include #include "common/config.h" +#include "common/status.h" #include "olap/olap_common.h" #include "olap/olap_define.h" #include "olap/row_cursor.h" +#include "util/expected.hpp" namespace doris { const size_t DEFAULT_STRING_LENGTH = 50; -WrapperField* WrapperField::create(const TabletColumn& column, uint32_t len) { +Result WrapperField::create(const TabletColumn& column, uint32_t len) { bool is_string_type = (column.type() == FieldType::OLAP_FIELD_TYPE_CHAR || column.type() == FieldType::OLAP_FIELD_TYPE_VARCHAR || column.type() == FieldType::OLAP_FIELD_TYPE_HLL || @@ -44,12 +46,13 @@ WrapperField* WrapperField::create(const TabletColumn& column, uint32_t len) { if (is_string_type && len > max_length) { LOG(WARNING) << "length of string parameter is too long[len=" << len << ", max_len=" << max_length << "]."; - return nullptr; + return unexpected {Status::Error( + "length of string parameter is too long[len={}, max_len={}].", len, max_length)}; } Field* rep = FieldFactory::create(column); if (rep == nullptr) { - return nullptr; + return unexpected {Status::Uninitialized("Unsupport field creation of {}", column.name())}; } size_t variable_len = 0; @@ -67,9 +70,7 @@ WrapperField* WrapperField::create(const TabletColumn& column, uint32_t len) { } else { variable_len = column.length(); } - - WrapperField* wrapper = new WrapperField(rep, variable_len, is_string_type); - return wrapper; + return new WrapperField(rep, variable_len, is_string_type); } WrapperField* WrapperField::create_by_type(const FieldType& type, int32_t var_length) { @@ -83,8 +84,7 @@ WrapperField* WrapperField::create_by_type(const FieldType& type, int32_t var_le type == FieldType::OLAP_FIELD_TYPE_OBJECT || type == FieldType::OLAP_FIELD_TYPE_STRING || type == FieldType::OLAP_FIELD_TYPE_QUANTILE_STATE); - auto wrapper = new WrapperField(rep, var_length, is_string_type); - return wrapper; + return new WrapperField(rep, var_length, is_string_type); } WrapperField::WrapperField(Field* rep, size_t variable_len, bool is_string_type) @@ -98,7 +98,7 @@ WrapperField::WrapperField(Field* rep, size_t variable_len, bool is_string_type) if (_is_string_type) { _var_length = variable_len > DEFAULT_STRING_LENGTH ? DEFAULT_STRING_LENGTH : variable_len; - Slice* slice = reinterpret_cast(buf); + auto* slice = reinterpret_cast(buf); slice->size = _var_length; _string_content.reset(new char[slice->size]); slice->data = _string_content.get(); diff --git a/be/src/olap/wrapper_field.h b/be/src/olap/wrapper_field.h index 978c9d73789204..53ce8e1d3e8dae 100644 --- a/be/src/olap/wrapper_field.h +++ b/be/src/olap/wrapper_field.h @@ -34,7 +34,7 @@ enum class FieldType; class WrapperField { public: - static WrapperField* create(const TabletColumn& column, uint32_t len = 0); + static Result create(const TabletColumn& column, uint32_t len = 0); static WrapperField* create_by_type(const FieldType& type) { return create_by_type(type, 0); } static WrapperField* create_by_type(const FieldType& type, int32_t var_length); diff --git a/be/src/pipeline/exec/exchange_sink_operator.cpp b/be/src/pipeline/exec/exchange_sink_operator.cpp index 43bec0bd92d2e5..9f9b36d1cb2614 100644 --- a/be/src/pipeline/exec/exchange_sink_operator.cpp +++ b/be/src/pipeline/exec/exchange_sink_operator.cpp @@ -199,17 +199,22 @@ Status ExchangeSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& inf _wait_channel_timer.resize(local_size); auto deps_for_channels = AndDependency::create_shared( _parent->operator_id(), _parent->node_id(), state->get_query_ctx()); - for (auto channel : channels) { + auto deps_for_channels_mem_limit = AndDependency::create_shared( + _parent->operator_id(), _parent->node_id(), state->get_query_ctx()); + for (auto* channel : channels) { if (channel->is_local()) { _local_channels_dependency[dep_id] = channel->get_local_channel_dependency(); DCHECK(_local_channels_dependency[dep_id] != nullptr); deps_for_channels->add_child(_local_channels_dependency[dep_id]); _wait_channel_timer[dep_id] = ADD_CHILD_TIMER( _profile, fmt::format("WaitForLocalExchangeBuffer{}", dep_id), timer_name); + auto local_recvr = channel->local_recvr(); + deps_for_channels_mem_limit->add_child(local_recvr->get_mem_limit_dependency()); dep_id++; } } _exchange_sink_dependency->add_child(deps_for_channels); + _exchange_sink_dependency->add_child(deps_for_channels_mem_limit); } if (p._part_type == TPartitionType::HASH_PARTITIONED) { _partition_count = channels.size(); diff --git a/be/src/pipeline/exec/exchange_sink_operator.h b/be/src/pipeline/exec/exchange_sink_operator.h index 5df03ea7773059..a34c4f4b435c04 100644 --- a/be/src/pipeline/exec/exchange_sink_operator.h +++ b/be/src/pipeline/exec/exchange_sink_operator.h @@ -96,9 +96,14 @@ class LocalExchangeChannelDependency final : public Dependency { LocalExchangeChannelDependency(int id, int node_id, QueryContext* query_ctx) : Dependency(id, node_id, "LocalExchangeChannelDependency", true, query_ctx) {} ~LocalExchangeChannelDependency() override = default; - // TODO(gabriel): blocked by memory }; +class LocalExchangeMemLimitDependency final : public Dependency { + ENABLE_FACTORY_CREATOR(LocalExchangeMemLimitDependency); + LocalExchangeMemLimitDependency(int id, int node_id, QueryContext* query_ctx) + : Dependency(id, node_id, "LocalExchangeMemLimitDependency", true, query_ctx) {} + ~LocalExchangeMemLimitDependency() override = default; +}; class ExchangeSinkLocalState final : public PipelineXSinkLocalState { ENABLE_FACTORY_CREATOR(ExchangeSinkLocalState); using Base = PipelineXSinkLocalState; diff --git a/be/src/pipeline/exec/exchange_source_operator.cpp b/be/src/pipeline/exec/exchange_source_operator.cpp index 255cb151410fda..847891104c7b24 100644 --- a/be/src/pipeline/exec/exchange_source_operator.cpp +++ b/be/src/pipeline/exec/exchange_source_operator.cpp @@ -74,6 +74,7 @@ Status ExchangeLocalState::init(RuntimeState* state, LocalStateInfo& info) { stream_recvr = state->exec_env()->vstream_mgr()->create_recvr( state, p.input_row_desc(), state->fragment_instance_id(), p.node_id(), p.num_senders(), profile(), p.is_merging(), p.sub_plan_query_statistics_recvr()); + stream_recvr->create_mem_limit_dependency(p.operator_id(), p.node_id(), state->get_query_ctx()); auto* source_dependency = _dependency; const auto& queues = stream_recvr->sender_queues(); deps.resize(queues.size()); diff --git a/be/src/pipeline/exec/hashjoin_build_sink.cpp b/be/src/pipeline/exec/hashjoin_build_sink.cpp index 8cd0376a957fa7..8c1ff852433e14 100644 --- a/be/src/pipeline/exec/hashjoin_build_sink.cpp +++ b/be/src/pipeline/exec/hashjoin_build_sink.cpp @@ -22,6 +22,7 @@ #include "exprs/bloom_filter_func.h" #include "pipeline/exec/hashjoin_probe_operator.h" #include "pipeline/exec/operator.h" +#include "pipeline/pipeline_x/dependency.h" #include "vec/exec/join/vhash_join_node.h" #include "vec/utils/template_helpers.hpp" @@ -45,8 +46,7 @@ Status HashJoinBuildSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo RETURN_IF_ERROR(JoinBuildSinkLocalState::init(state, info)); SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_open_timer); - _shared_hash_table_dependency = SharedHashTableDependency::create_shared( - _parent->operator_id(), _parent->node_id(), state->get_query_ctx()); + _shared_hash_table_dependency = dependency_sptr(); auto& p = _parent->cast(); _shared_state->join_op_variants = p._join_op_variants; @@ -119,7 +119,7 @@ Status HashJoinBuildSinkLocalState::open(RuntimeState* state) { auto& p = _parent->cast(); for (size_t i = 0; i < p._runtime_filter_descs.size(); i++) { - if (auto bf = _runtime_filters[i]->get_bloomfilter()) { + if (auto* bf = _runtime_filters[i]->get_bloomfilter()) { RETURN_IF_ERROR(bf->init_with_fixed_length()); } } diff --git a/be/src/pipeline/exec/hashjoin_build_sink.h b/be/src/pipeline/exec/hashjoin_build_sink.h index 24faa4115dd1eb..b5ae146c1828b2 100644 --- a/be/src/pipeline/exec/hashjoin_build_sink.h +++ b/be/src/pipeline/exec/hashjoin_build_sink.h @@ -48,24 +48,18 @@ class HashJoinBuildSinkOperatorX; class SharedHashTableDependency final : public Dependency { public: + using SharedState = HashJoinSharedState; ENABLE_FACTORY_CREATOR(SharedHashTableDependency); SharedHashTableDependency(int id, int node_id, QueryContext* query_ctx) - : Dependency(id, node_id, "SharedHashTableDependency", true, query_ctx) {} + : Dependency(id, node_id, "SharedHashTableBuildDependency", true, query_ctx) {} ~SharedHashTableDependency() override = default; }; -class HashJoinBuildSinkDependency final : public Dependency { -public: - using SharedState = HashJoinSharedState; - HashJoinBuildSinkDependency(int id, int node_id, QueryContext* query_ctx) - : Dependency(id, node_id, "HashJoinBuildSinkDependency", true, query_ctx) {} - ~HashJoinBuildSinkDependency() override = default; -}; - class HashJoinBuildSinkLocalState final - : public JoinBuildSinkLocalState { + : public JoinBuildSinkLocalState { public: ENABLE_FACTORY_CREATOR(HashJoinBuildSinkLocalState); + using Base = JoinBuildSinkLocalState; using Parent = HashJoinBuildSinkOperatorX; HashJoinBuildSinkLocalState(DataSinkOperatorXBase* parent, RuntimeState* state); ~HashJoinBuildSinkLocalState() override = default; @@ -86,7 +80,6 @@ class HashJoinBuildSinkLocalState final void add_hash_buckets_filled_info(const std::string& info) const { _profile->add_info_string("HashTableFilledBuckets", info); } - Dependency* dependency() override { return _shared_hash_table_dependency.get(); } protected: void _hash_table_init(RuntimeState* state); diff --git a/be/src/pipeline/exec/join_build_sink_operator.cpp b/be/src/pipeline/exec/join_build_sink_operator.cpp index e2cc361c22f5d4..798b5d86b00bc4 100644 --- a/be/src/pipeline/exec/join_build_sink_operator.cpp +++ b/be/src/pipeline/exec/join_build_sink_operator.cpp @@ -115,7 +115,7 @@ void JoinBuildSinkOperatorX::_init_join_op() { } template class JoinBuildSinkOperatorX; -template class JoinBuildSinkLocalState; +template class JoinBuildSinkLocalState; template class JoinBuildSinkOperatorX; template class JoinBuildSinkLocalState; diff --git a/be/src/pipeline/exec/olap_scan_operator.cpp b/be/src/pipeline/exec/olap_scan_operator.cpp index c751c167d0c242..3e40e173585a08 100644 --- a/be/src/pipeline/exec/olap_scan_operator.cpp +++ b/be/src/pipeline/exec/olap_scan_operator.cpp @@ -63,6 +63,14 @@ Status OlapScanLocalState::_init_profile() { _block_init_seek_timer = ADD_TIMER(_segment_profile, "BlockInitSeekTime"); _block_init_seek_counter = ADD_COUNTER(_segment_profile, "BlockInitSeekCount", TUnit::UNIT); _block_conditions_filtered_timer = ADD_TIMER(_segment_profile, "BlockConditionsFilteredTime"); + _block_conditions_filtered_bf_timer = + ADD_TIMER(_segment_profile, "BlockConditionsFilteredBloomFilterTime"); + _block_conditions_filtered_zonemap_timer = + ADD_TIMER(_segment_profile, "BlockConditionsFilteredZonemapTime"); + _block_conditions_filtered_zonemap_rp_timer = + ADD_TIMER(_segment_profile, "BlockConditionsFilteredZonemapRuntimePredicateTime"); + _block_conditions_filtered_dict_timer = + ADD_TIMER(_segment_profile, "BlockConditionsFilteredDictTime"); _rows_vec_cond_filtered_counter = ADD_COUNTER(_segment_profile, "RowsVectorPredFiltered", TUnit::UNIT); @@ -87,6 +95,8 @@ Status OlapScanLocalState::_init_profile() { _output_col_timer = ADD_TIMER(_segment_profile, "OutputColumnTime"); _stats_filtered_counter = ADD_COUNTER(_segment_profile, "RowsStatsFiltered", TUnit::UNIT); + _stats_rp_filtered_counter = + ADD_COUNTER(_segment_profile, "RowsZonemapRuntimePredicateFiltered", TUnit::UNIT); _bf_filtered_counter = ADD_COUNTER(_segment_profile, "RowsBloomFilterFiltered", TUnit::UNIT); _dict_filtered_counter = ADD_COUNTER(_segment_profile, "RowsDictFiltered", TUnit::UNIT); _del_filtered_counter = ADD_COUNTER(_scanner_profile, "RowsDelFiltered", TUnit::UNIT); diff --git a/be/src/pipeline/exec/olap_scan_operator.h b/be/src/pipeline/exec/olap_scan_operator.h index f1db77b2054277..ffd8fecd0750b2 100644 --- a/be/src/pipeline/exec/olap_scan_operator.h +++ b/be/src/pipeline/exec/olap_scan_operator.h @@ -126,6 +126,7 @@ class OlapScanLocalState final : public ScanLocalState { std::map _filter_info; RuntimeProfile::Counter* _stats_filtered_counter = nullptr; + RuntimeProfile::Counter* _stats_rp_filtered_counter = nullptr; RuntimeProfile::Counter* _bf_filtered_counter = nullptr; RuntimeProfile::Counter* _dict_filtered_counter = nullptr; RuntimeProfile::Counter* _del_filtered_counter = nullptr; @@ -141,6 +142,10 @@ class OlapScanLocalState final : public ScanLocalState { RuntimeProfile::Counter* _block_init_seek_timer = nullptr; RuntimeProfile::Counter* _block_init_seek_counter = nullptr; RuntimeProfile::Counter* _block_conditions_filtered_timer = nullptr; + RuntimeProfile::Counter* _block_conditions_filtered_bf_timer = nullptr; + RuntimeProfile::Counter* _block_conditions_filtered_zonemap_timer = nullptr; + RuntimeProfile::Counter* _block_conditions_filtered_zonemap_rp_timer = nullptr; + RuntimeProfile::Counter* _block_conditions_filtered_dict_timer = nullptr; RuntimeProfile::Counter* _first_read_timer = nullptr; RuntimeProfile::Counter* _second_read_timer = nullptr; RuntimeProfile::Counter* _first_read_seek_timer = nullptr; diff --git a/be/src/pipeline/exec/result_sink_operator.cpp b/be/src/pipeline/exec/result_sink_operator.cpp index 33256dc7f006e6..dcf0d996c62d15 100644 --- a/be/src/pipeline/exec/result_sink_operator.cpp +++ b/be/src/pipeline/exec/result_sink_operator.cpp @@ -51,7 +51,7 @@ bool ResultSinkOperator::can_write() { } Status ResultSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { - RETURN_IF_ERROR(PipelineXSinkLocalState<>::init(state, info)); + RETURN_IF_ERROR(Base::init(state, info)); SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_open_timer); static const std::string timer_name = "WaitForDependencyTime"; @@ -62,18 +62,17 @@ Status ResultSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) RETURN_IF_ERROR(state->exec_env()->result_mgr()->create_sender( state->fragment_instance_id(), vectorized::RESULT_SINK_BUFFER_SIZE, &_sender, true, state->execution_timeout())); - _result_sink_dependency = ResultSinkDependency::create_shared( - _parent->operator_id(), _parent->node_id(), state->get_query_ctx()); + auto result_sink_dependency = dependency_sptr(); _blocks_sent_counter = ADD_COUNTER_WITH_LEVEL(_profile, "BlocksProduced", TUnit::UNIT, 1); _rows_sent_counter = ADD_COUNTER_WITH_LEVEL(_profile, "RowsProduced", TUnit::UNIT, 1); - ((PipBufferControlBlock*)_sender.get())->set_dependency(_result_sink_dependency); + ((PipBufferControlBlock*)_sender.get())->set_dependency(result_sink_dependency); return Status::OK(); } Status ResultSinkLocalState::open(RuntimeState* state) { SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_open_timer); - RETURN_IF_ERROR(PipelineXSinkLocalState<>::open(state)); + RETURN_IF_ERROR(Base::open(state)); auto& p = _parent->cast(); _output_vexpr_ctxs.resize(p._output_vexpr_ctxs.size()); for (size_t i = 0; i < _output_vexpr_ctxs.size(); i++) { @@ -150,7 +149,7 @@ Status ResultSinkOperatorX::_second_phase_fetch_data(RuntimeState* state, vectorized::Block* final_block) { auto row_id_col = final_block->get_by_position(final_block->columns() - 1); CHECK(row_id_col.name == BeConsts::ROWID_COL); - auto tuple_desc = _row_desc.tuple_descriptors()[0]; + auto* tuple_desc = _row_desc.tuple_descriptors()[0]; FetchOption fetch_option; fetch_option.desc = tuple_desc; fetch_option.t_fetch_opt = _fetch_option; @@ -167,7 +166,7 @@ Status ResultSinkLocalState::close(RuntimeState* state, Status exec_status) { } SCOPED_TIMER(_close_timer); SCOPED_TIMER(exec_time_counter()); - COUNTER_SET(_wait_for_dependency_timer, _result_sink_dependency->watcher_elapse_time()); + COUNTER_SET(_wait_for_dependency_timer, _dependency->watcher_elapse_time()); Status final_status = exec_status; if (_writer) { // close the writer @@ -189,7 +188,7 @@ Status ResultSinkLocalState::close(RuntimeState* state, Status exec_status) { static_cast(state->exec_env()->result_mgr()->cancel_at_time( time(nullptr) + config::result_buffer_cancelled_interval_time, state->fragment_instance_id())); - RETURN_IF_ERROR(PipelineXSinkLocalState<>::close(state, exec_status)); + RETURN_IF_ERROR(Base::close(state, exec_status)); return final_status; } diff --git a/be/src/pipeline/exec/result_sink_operator.h b/be/src/pipeline/exec/result_sink_operator.h index e3d56f176e6804..bed6aed8969679 100644 --- a/be/src/pipeline/exec/result_sink_operator.h +++ b/be/src/pipeline/exec/result_sink_operator.h @@ -20,6 +20,7 @@ #include #include "operator.h" +#include "pipeline/pipeline_x/dependency.h" #include "pipeline/pipeline_x/operator.h" #include "vec/sink/vresult_sink.h" @@ -45,23 +46,24 @@ class ResultSinkOperator final : public DataSinkOperator { +class ResultSinkLocalState final : public PipelineXSinkLocalState { ENABLE_FACTORY_CREATOR(ResultSinkLocalState); + using Base = PipelineXSinkLocalState; public: ResultSinkLocalState(DataSinkOperatorXBase* parent, RuntimeState* state) - : PipelineXSinkLocalState<>(parent, state) {} + : Base(parent, state) {} Status init(RuntimeState* state, LocalSinkStateInfo& info) override; Status open(RuntimeState* state) override; Status close(RuntimeState* state, Status exec_status) override; - Dependency* dependency() override { return _result_sink_dependency.get(); } RuntimeProfile::Counter* blocks_sent_counter() { return _blocks_sent_counter; } RuntimeProfile::Counter* rows_sent_counter() { return _rows_sent_counter; } @@ -72,7 +74,6 @@ class ResultSinkLocalState final : public PipelineXSinkLocalState<> { std::shared_ptr _sender; std::shared_ptr _writer; - std::shared_ptr _result_sink_dependency; RuntimeProfile::Counter* _blocks_sent_counter = nullptr; RuntimeProfile::Counter* _rows_sent_counter = nullptr; }; diff --git a/be/src/pipeline/exec/scan_operator.cpp b/be/src/pipeline/exec/scan_operator.cpp index 1e0f68131e87bb..af9529798a3df9 100644 --- a/be/src/pipeline/exec/scan_operator.cpp +++ b/be/src/pipeline/exec/scan_operator.cpp @@ -64,6 +64,14 @@ bool ScanOperator::can_read() { } } +bool ScanOperator::is_pending_finish() const { + return _node->_scanner_ctx && !_node->_scanner_ctx->no_schedule(); +} + +Status ScanOperator::try_close(RuntimeState* state) { + return _node->try_close(state); +} + bool ScanOperator::runtime_filters_are_ready_or_timeout() { return _node->runtime_filters_are_ready_or_timeout(); } @@ -73,8 +81,9 @@ std::string ScanOperator::debug_string() const { fmt::format_to(debug_string_buffer, "{}, scanner_ctx is null: {} ", SourceOperator::debug_string(), _node->_scanner_ctx == nullptr); if (_node->_scanner_ctx) { - fmt::format_to(debug_string_buffer, ", num_running_scanners = {}", - _node->_scanner_ctx->get_num_running_scanners()); + fmt::format_to(debug_string_buffer, ", num_running_scanners = {}, num_scheduling_ctx = {} ", + _node->_scanner_ctx->get_num_running_scanners(), + _node->_scanner_ctx->get_num_scheduling_ctx()); } return fmt::to_string(debug_string_buffer); } @@ -92,6 +101,9 @@ std::string ScanOperator::debug_string() const { template ScanLocalState::ScanLocalState(RuntimeState* state, OperatorXBase* parent) : ScanLocalStateBase(state, parent) { + _finish_dependency = std::make_shared( + parent->operator_id(), parent->node_id(), parent->get_name() + "_FINISH_DEPENDENCY", + state->get_query_ctx()); _filter_dependency = std::make_shared( parent->operator_id(), parent->node_id(), parent->get_name() + "_FILTER_DEPENDENCY", state->get_query_ctx()); @@ -109,15 +121,13 @@ bool ScanLocalState::should_run_serial() const { template Status ScanLocalState::init(RuntimeState* state, LocalStateInfo& info) { - RETURN_IF_ERROR(PipelineXLocalState<>::init(state, info)); + RETURN_IF_ERROR(PipelineXLocalState::init(state, info)); SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_open_timer); auto& p = _parent->cast(); RETURN_IF_ERROR(RuntimeFilterConsumer::init(state, p.ignore_data_distribution())); - _scan_dependency = ScanDependency::create_shared(PipelineXLocalState<>::_parent->operator_id(), - PipelineXLocalState<>::_parent->node_id(), - state->get_query_ctx()); + _scan_dependency = dependency_sptr(); set_scan_ranges(state, info.scan_ranges); _common_expr_ctxs_push_down.resize(p._common_expr_ctxs_push_down.size()); @@ -165,6 +175,7 @@ Status ScanLocalState::open(RuntimeState* state) { auto status = _eos ? Status::OK() : _prepare_scanners(); if (_scanner_ctx) { + _finish_dependency->block(); DCHECK(!_eos && _num_scanners->value() > 0); RETURN_IF_ERROR(_scanner_ctx->init()); RETURN_IF_ERROR(state->exec_env()->scanner_scheduler()->submit(_scanner_ctx)); @@ -554,17 +565,19 @@ template std::string ScanLocalState::debug_string(int indentation_level) const { fmt::memory_buffer debug_string_buffer; fmt::format_to(debug_string_buffer, "{}, _eos = {}", - PipelineXLocalState<>::debug_string(indentation_level), _eos.load()); + PipelineXLocalState::debug_string(indentation_level), + _eos.load()); if (_scanner_ctx) { fmt::format_to(debug_string_buffer, ""); - fmt::format_to(debug_string_buffer, - ", Scanner Context: (_is_finished = {}, _should_stop = {}, " - "_num_running_scanners={}, " - " _num_unfinished_scanners = {}, status = {}, error = {})", - _scanner_ctx->is_finished(), _scanner_ctx->should_stop(), - _scanner_ctx->get_num_running_scanners(), - _scanner_ctx->get_num_unfinished_scanners(), - _scanner_ctx->status().to_string(), _scanner_ctx->status_error()); + fmt::format_to( + debug_string_buffer, + ", Scanner Context: (_is_finished = {}, _should_stop = {}, " + "_num_running_scanners={}, " + "_num_scheduling_ctx = {}, _num_unfinished_scanners = {}, status = {}, error = {})", + _scanner_ctx->is_finished(), _scanner_ctx->should_stop(), + _scanner_ctx->get_num_running_scanners(), _scanner_ctx->get_num_scheduling_ctx(), + _scanner_ctx->get_num_unfinished_scanners(), _scanner_ctx->status().to_string(), + _scanner_ctx->status_error()); } return fmt::to_string(debug_string_buffer); @@ -1212,27 +1225,24 @@ template Status ScanLocalState::_prepare_scanners() { std::list scanners; RETURN_IF_ERROR(_init_scanners(&scanners)); - // Init scanner wrapper - for (auto it = scanners.begin(); it != scanners.end(); ++it) { - _scanners.emplace_back(std::make_shared(*it)); - } if (scanners.empty()) { _eos = true; _scan_dependency->set_ready(); } else { COUNTER_SET(_num_scanners, static_cast(scanners.size())); - RETURN_IF_ERROR(_start_scanners(_scanners)); + RETURN_IF_ERROR(_start_scanners(scanners)); } return Status::OK(); } template Status ScanLocalState::_start_scanners( - const std::list>& scanners) { + const std::list& scanners) { auto& p = _parent->cast(); _scanner_ctx = PipScannerContext::create_shared(state(), this, p._output_tuple_desc, scanners, p.limit(), state()->scan_queue_mem_limit(), - p._col_distribute_ids, 1, _scan_dependency); + p._col_distribute_ids, 1, _scan_dependency, + _finish_dependency); return Status::OK(); } @@ -1308,6 +1318,9 @@ Status ScanLocalState::_init_profile() { _max_scanner_thread_num = ADD_COUNTER(_runtime_profile, "MaxScannerThreadNum", TUnit::UNIT); + _wait_for_finish_dependency_timer = + ADD_TIMER(_runtime_profile, "WaitForPendingFinishDependency"); + return Status::OK(); } @@ -1415,6 +1428,17 @@ Status ScanOperatorX::open(RuntimeState* state) { return Status::OK(); } +template +Status ScanOperatorX::try_close(RuntimeState* state) { + auto& local_state = get_local_state(state); + if (local_state._scanner_ctx) { + // mark this scanner ctx as should_stop to make sure scanners will not be scheduled anymore + // TODO: there is a lock in `set_should_stop` may cause some slight impact + local_state._scanner_ctx->set_should_stop(); + } + return Status::OK(); +} + template Status ScanLocalState::close(RuntimeState* state) { if (_closed) { @@ -1426,12 +1450,13 @@ Status ScanLocalState::close(RuntimeState* state) { SCOPED_TIMER(exec_time_counter()); if (_scanner_ctx) { - _scanner_ctx->stop_scanners(state); + _scanner_ctx->clear_and_join(reinterpret_cast(this), state); } COUNTER_SET(_wait_for_dependency_timer, _scan_dependency->watcher_elapse_time()); + COUNTER_SET(_wait_for_finish_dependency_timer, _finish_dependency->watcher_elapse_time()); COUNTER_SET(_wait_for_rf_timer, _filter_dependency->watcher_elapse_time()); - return PipelineXLocalState<>::close(state); + return PipelineXLocalState::close(state); } template @@ -1485,7 +1510,7 @@ Status ScanOperatorX::get_block(RuntimeState* state, vectorized: if (eos) { source_state = SourceState::FINISHED; // reach limit, stop the scanners. - local_state._scanner_ctx->stop_scanners(state); + local_state._scanner_ctx->set_should_stop(); } return Status::OK(); diff --git a/be/src/pipeline/exec/scan_operator.h b/be/src/pipeline/exec/scan_operator.h index bf083d82d5d89c..afbcaea0a63a7b 100644 --- a/be/src/pipeline/exec/scan_operator.h +++ b/be/src/pipeline/exec/scan_operator.h @@ -24,6 +24,7 @@ #include "common/status.h" #include "operator.h" +#include "pipeline/pipeline_x/dependency.h" #include "pipeline/pipeline_x/operator.h" #include "runtime/descriptors.h" #include "vec/exec/scan/vscan_node.h" @@ -31,9 +32,6 @@ namespace doris { class ExecNode; } // namespace doris -namespace doris::vectorized { -class ScannerDelegate; -} namespace doris::pipeline { class PipScannerContext; @@ -51,13 +49,18 @@ class ScanOperator : public SourceOperator { bool can_read() override; // for source + bool is_pending_finish() const override; + bool runtime_filters_are_ready_or_timeout() override; std::string debug_string() const override; + + Status try_close(RuntimeState* state) override; }; class ScanDependency final : public Dependency { public: + using SharedState = FakeSharedState; ENABLE_FACTORY_CREATOR(ScanDependency); ScanDependency(int id, int node_id, QueryContext* query_ctx) : Dependency(id, node_id, "ScanDependency", query_ctx) {} @@ -97,10 +100,11 @@ class ScanDependency final : public Dependency { std::mutex _always_done_lock; }; -class ScanLocalStateBase : public PipelineXLocalState<>, public vectorized::RuntimeFilterConsumer { +class ScanLocalStateBase : public PipelineXLocalState, + public vectorized::RuntimeFilterConsumer { public: ScanLocalStateBase(RuntimeState* state, OperatorXBase* parent) - : PipelineXLocalState<>(state, parent), + : PipelineXLocalState(state, parent), vectorized::RuntimeFilterConsumer(parent->node_id(), parent->runtime_filter_descs(), parent->row_descriptor(), _conjuncts) {} virtual ~ScanLocalStateBase() = default; @@ -170,6 +174,7 @@ class ScanLocalStateBase : public PipelineXLocalState<>, public vectorized::Runt RuntimeProfile::Counter* _wait_for_scanner_done_timer = nullptr; // time of prefilter input block from scanner RuntimeProfile::Counter* _wait_for_eos_timer = nullptr; + RuntimeProfile::Counter* _wait_for_finish_dependency_timer = nullptr; RuntimeProfile::Counter* _wait_for_rf_timer = nullptr; }; @@ -209,9 +214,8 @@ class ScanLocalState : public ScanLocalStateBase { int64_t get_push_down_count() override; - Dependency* dependency() override { return _scan_dependency.get(); } - RuntimeFilterDependency* filterdependency() override { return _filter_dependency.get(); }; + Dependency* finishdependency() override { return _finish_dependency.get(); } protected: template @@ -347,7 +351,7 @@ class ScanLocalState : public ScanLocalStateBase { Status _prepare_scanners(); // Submit the scanner to the thread pool and start execution - Status _start_scanners(const std::list>& scanners); + Status _start_scanners(const std::list& scanners); // For some conjunct there is chance to elimate cast operator // Eg. Variant's sub column could eliminate cast in storage layer if @@ -410,13 +414,14 @@ class ScanLocalState : public ScanLocalStateBase { std::shared_ptr _filter_dependency; - // ScanLocalState owns the ownership of scanner, scanner context only has its weakptr - std::list> _scanners; + std::shared_ptr _finish_dependency; }; template class ScanOperatorX : public OperatorX { public: + Status try_close(RuntimeState* state) override; + Status init(const TPlanNode& tnode, RuntimeState* state) override; Status prepare(RuntimeState* state) override { return OperatorXBase::prepare(state); } Status open(RuntimeState* state) override; diff --git a/be/src/pipeline/pipeline_x/operator.cpp b/be/src/pipeline/pipeline_x/operator.cpp index d40f7f6adb3123..39598dadbd8325 100644 --- a/be/src/pipeline/pipeline_x/operator.cpp +++ b/be/src/pipeline/pipeline_x/operator.cpp @@ -622,7 +622,7 @@ template class StatefulOperatorX; template class StatefulOperatorX; template class StatefulOperatorX; -template class PipelineXSinkLocalState; +template class PipelineXSinkLocalState; template class PipelineXSinkLocalState; template class PipelineXSinkLocalState; template class PipelineXSinkLocalState; @@ -635,6 +635,7 @@ template class PipelineXSinkLocalState; template class PipelineXSinkLocalState; template class PipelineXSinkLocalState; template class PipelineXSinkLocalState; +template class PipelineXSinkLocalState; template class PipelineXLocalState; template class PipelineXLocalState; @@ -648,6 +649,7 @@ template class PipelineXLocalState; template class PipelineXLocalState; template class PipelineXLocalState; template class PipelineXLocalState; +template class PipelineXLocalState; template class AsyncWriterSink; template class AsyncWriterSink; diff --git a/be/src/pipeline/pipeline_x/operator.h b/be/src/pipeline/pipeline_x/operator.h index da52706b56cc49..c4e0a7e94a0e3e 100644 --- a/be/src/pipeline/pipeline_x/operator.h +++ b/be/src/pipeline/pipeline_x/operator.h @@ -348,6 +348,10 @@ class PipelineXLocalState : public PipelineXLocalStateBase { Dependency* dependency() override { return _dependency; } + auto dependency_sptr() { + return std::dynamic_pointer_cast(_dependency->shared_from_this()); + } + protected: DependencyType* _dependency = nullptr; typename DependencyType::SharedState* _shared_state = nullptr; @@ -608,6 +612,10 @@ class PipelineXSinkLocalState : public PipelineXSinkLocalStateBase { Dependency* dependency() override { return _dependency; } + auto dependency_sptr() { + return std::dynamic_pointer_cast(_dependency->shared_from_this()); + } + protected: DependencyType* _dependency = nullptr; typename DependencyType::SharedState* _shared_state = nullptr; diff --git a/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.cpp b/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.cpp index 7efe476c6dec11..86440051a907d2 100644 --- a/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.cpp +++ b/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.cpp @@ -212,6 +212,8 @@ Status PipelineXFragmentContext::prepare(const doris::TPipelineFragmentParams& r _runtime_state->set_total_load_streams(request.total_load_streams); _runtime_state->set_num_local_sink(request.num_local_sink); + _use_global_rf = request.__isset.parallel_instances && (request.__isset.per_node_shared_scans && + !request.per_node_shared_scans.empty()); // 2. Build pipelines with operators in this fragment. auto root_pipeline = add_pipeline(); RETURN_IF_ERROR_OR_CATCH_EXCEPTION(_build_pipelines( @@ -985,7 +987,7 @@ Status PipelineXFragmentContext::_create_operator(ObjectPool* pool, const TPlanN DataSinkOperatorXPtr sink; sink.reset(new HashJoinBuildSinkOperatorX(pool, next_sink_operator_id(), tnode, descs, - request.__isset.parallel_instances)); + _use_global_rf)); sink->set_dests_id({op->operator_id()}); RETURN_IF_ERROR(build_side_pipe->set_sink(sink)); RETURN_IF_ERROR(build_side_pipe->sink_x()->init(tnode, _runtime_state.get())); diff --git a/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.h b/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.h index be0e26461debeb..6fe3488096431d 100644 --- a/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.h +++ b/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.h @@ -168,6 +168,8 @@ class PipelineXFragmentContext : public PipelineFragmentContext { // this is a [n * m] matrix. n is parallelism of pipeline engine and m is the number of pipelines. std::vector>> _tasks; + bool _use_global_rf = false; + // It is used to manage the lifecycle of RuntimeFilterMergeController std::vector> _merge_controller_handlers; diff --git a/be/src/runtime/snapshot_loader.cpp b/be/src/runtime/snapshot_loader.cpp index afd41907289c19..34d26764ead29e 100644 --- a/be/src/runtime/snapshot_loader.cpp +++ b/be/src/runtime/snapshot_loader.cpp @@ -312,7 +312,9 @@ Status SnapshotLoader::download(const std::map& src_to // check disk capacity if (data_dir->reach_capacity_limit(file_len)) { - return Status::InternalError("capacity limit reached"); + return Status::Error( + "reach the capacity limit of path {}, file_size={}", data_dir->path(), + file_len); } // remove file which will be downloaded now. // this file will be added to local_files if it be downloaded successfully. @@ -545,7 +547,9 @@ Status SnapshotLoader::remote_http_download( // check disk capacity if (data_dir->reach_capacity_limit(file_size)) { - return Status::InternalError("Disk reach capacity limit"); + return Status::Error( + "reach the capacity limit of path {}, file_size={}", data_dir->path(), + file_size); } total_file_size += file_size; diff --git a/be/src/vec/common/hash_table/hash_map.h b/be/src/vec/common/hash_table/hash_map.h index 6efbdbb3e94ed0..cb2809492aebaf 100644 --- a/be/src/vec/common/hash_table/hash_map.h +++ b/be/src/vec/common/hash_table/hash_map.h @@ -226,6 +226,9 @@ class JoinHashMapTable : public HashMapTable template void prepare_build(size_t num_elem, int batch_size, bool has_null_key) { _has_null_key = has_null_key; + + // the first row in build side is not really from build side table + _empty_build_side = num_elem <= 1; max_batch_size = batch_size; bucket_size = calc_bucket_size(num_elem + 1); first.resize(bucket_size + 1); @@ -262,6 +265,14 @@ class JoinHashMapTable : public HashMapTable uint32_t* __restrict probe_idxs, bool& probe_visited, uint32_t* __restrict build_idxs, doris::vectorized::ColumnFilterHelper* mark_column) { + if constexpr (JoinOpType == doris::TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN) { + if (_empty_build_side) { + return _process_null_aware_left_anti_join_for_empty_build_side< + JoinOpType, with_other_conjuncts, is_mark_join>( + probe_idx, probe_rows, probe_idxs, build_idxs, mark_column); + } + } + if constexpr (is_mark_join) { return _find_batch_mark( keys, build_idx_map, probe_idx, probe_rows, probe_idxs, build_idxs, @@ -367,6 +378,29 @@ class JoinHashMapTable : public HashMapTable return std::tuple {probe_idx, 0U, matched_cnt}; } + template + auto _process_null_aware_left_anti_join_for_empty_build_side( + int probe_idx, int probe_rows, uint32_t* __restrict probe_idxs, + uint32_t* __restrict build_idxs, doris::vectorized::ColumnFilterHelper* mark_column) { + static_assert(JoinOpType == doris::TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN); + auto matched_cnt = 0; + const auto batch_size = max_batch_size; + + while (probe_idx < probe_rows && matched_cnt < batch_size) { + probe_idxs[matched_cnt] = probe_idx++; + if constexpr (is_mark_join) { + build_idxs[matched_cnt] = 0; + } + ++matched_cnt; + } + + if constexpr (is_mark_join && !with_other_conjuncts) { + mark_column->resize_fill(matched_cnt, 1); + } + + return std::tuple {probe_idx, 0U, matched_cnt}; + } + auto _find_batch_right_semi_anti(const Key* __restrict keys, const uint32_t* __restrict build_idx_map, int probe_idx, int probe_rows) { @@ -532,6 +566,7 @@ class JoinHashMapTable : public HashMapTable Cell cell; doris::vectorized::Arena* pool; bool _has_null_key = false; + bool _empty_build_side = true; }; template , diff --git a/be/src/vec/exec/format/parquet/vparquet_column_chunk_reader.cpp b/be/src/vec/exec/format/parquet/vparquet_column_chunk_reader.cpp index 928b5ae70bd67d..6feb9bc1025b33 100644 --- a/be/src/vec/exec/format/parquet/vparquet_column_chunk_reader.cpp +++ b/be/src/vec/exec/format/parquet/vparquet_column_chunk_reader.cpp @@ -97,10 +97,12 @@ Status ColumnChunkReader::next_page() { return next_page(); } else if (_page_reader->get_page_header()->type == tparquet::PageType::DATA_PAGE_V2) { _remaining_num_values = _page_reader->get_page_header()->data_page_header_v2.num_values; + _chunk_parsed_values += _remaining_num_values; _state = HEADER_PARSED; return Status::OK(); } else { _remaining_num_values = _page_reader->get_page_header()->data_page_header.num_values; + _chunk_parsed_values += _remaining_num_values; _state = HEADER_PARSED; return Status::OK(); } diff --git a/be/src/vec/exec/format/parquet/vparquet_column_chunk_reader.h b/be/src/vec/exec/format/parquet/vparquet_column_chunk_reader.h index 21ee808e48f6a9..c8a49e098a53e9 100644 --- a/be/src/vec/exec/format/parquet/vparquet_column_chunk_reader.h +++ b/be/src/vec/exec/format/parquet/vparquet_column_chunk_reader.h @@ -91,9 +91,17 @@ class ColumnChunkReader { Status init(); // Whether the chunk reader has a more page to read. - bool has_next_page() { return _page_reader->has_next_page(); } + bool has_next_page() { return _chunk_parsed_values < _metadata.num_values; } + // Deprecated // Seek to the specific page, page_header_offset must be the start offset of the page header. + // _end_offset may exceed the actual data area, so we can only use the number of parsed values + // to determine whether there are remaining pages to read. That's to say we can't use the + // PageLocation in parquet metadata to seek to the specified page. We should call next_page() + // and skip_page() to skip pages one by one. + // todo: change this interface to seek_to_page(int64_t page_header_offset, size_t num_parsed_values) + // and set _chunk_parsed_values = num_parsed_values + // [[deprecated]] void seek_to_page(int64_t page_header_offset) { _remaining_num_values = 0; _page_reader->seek_to_page(page_header_offset); @@ -201,6 +209,7 @@ class ColumnChunkReader { LevelDecoder _rep_level_decoder; LevelDecoder _def_level_decoder; + size_t _chunk_parsed_values = 0; uint32_t _remaining_num_values = 0; Slice _page_data; std::unique_ptr _decompress_buf; diff --git a/be/src/vec/exec/format/parquet/vparquet_page_reader.h b/be/src/vec/exec/format/parquet/vparquet_page_reader.h index 730b9a3001b01d..bdd0a8d0f5ff24 100644 --- a/be/src/vec/exec/format/parquet/vparquet_page_reader.h +++ b/be/src/vec/exec/format/parquet/vparquet_page_reader.h @@ -45,6 +45,11 @@ class PageReader { uint64_t length); ~PageReader() = default; + // Deprecated + // Parquet file may not be standardized, + // _end_offset may exceed the actual data area. + // ColumnChunkReader::has_next_page() use the number of parsed values for judgment + // [[deprecated]] bool has_next_page() const { return _offset < _end_offset; } Status next_page_header(); diff --git a/be/src/vec/exec/scan/new_olap_scan_node.cpp b/be/src/vec/exec/scan/new_olap_scan_node.cpp index ec13214900394d..38fd97f2413688 100644 --- a/be/src/vec/exec/scan/new_olap_scan_node.cpp +++ b/be/src/vec/exec/scan/new_olap_scan_node.cpp @@ -129,6 +129,14 @@ Status NewOlapScanNode::_init_profile() { _block_init_seek_timer = ADD_TIMER(_segment_profile, "BlockInitSeekTime"); _block_init_seek_counter = ADD_COUNTER(_segment_profile, "BlockInitSeekCount", TUnit::UNIT); _block_conditions_filtered_timer = ADD_TIMER(_segment_profile, "BlockConditionsFilteredTime"); + _block_conditions_filtered_bf_timer = + ADD_TIMER(_segment_profile, "BlockConditionsFilteredBloomFilterTime"); + _block_conditions_filtered_zonemap_timer = + ADD_TIMER(_segment_profile, "BlockConditionsFilteredZonemapTime"); + _block_conditions_filtered_zonemap_rp_timer = + ADD_TIMER(_segment_profile, "BlockConditionsFilteredZonemapRuntimePredicateTime"); + _block_conditions_filtered_dict_timer = + ADD_TIMER(_segment_profile, "BlockConditionsFilteredDictTime"); _rows_vec_cond_filtered_counter = ADD_COUNTER(_segment_profile, "RowsVectorPredFiltered", TUnit::UNIT); @@ -152,7 +160,9 @@ Status NewOlapScanNode::_init_profile() { _output_col_timer = ADD_TIMER(_segment_profile, "OutputColumnTime"); - _stats_filtered_counter = ADD_COUNTER(_segment_profile, "RowsStatsFiltered", TUnit::UNIT); + _stats_filtered_counter = ADD_COUNTER(_segment_profile, "RowsZonemapFiltered", TUnit::UNIT); + _stats_rp_filtered_counter = + ADD_COUNTER(_segment_profile, "RowsZonemapRuntimePredicateFiltered", TUnit::UNIT); _bf_filtered_counter = ADD_COUNTER(_segment_profile, "RowsBloomFilterFiltered", TUnit::UNIT); _dict_filtered_counter = ADD_COUNTER(_segment_profile, "RowsDictFiltered", TUnit::UNIT); _del_filtered_counter = ADD_COUNTER(_scanner_profile, "RowsDelFiltered", TUnit::UNIT); diff --git a/be/src/vec/exec/scan/new_olap_scan_node.h b/be/src/vec/exec/scan/new_olap_scan_node.h index c62e7028cdcc20..309bac56991dd0 100644 --- a/be/src/vec/exec/scan/new_olap_scan_node.h +++ b/be/src/vec/exec/scan/new_olap_scan_node.h @@ -153,6 +153,7 @@ class NewOlapScanNode : public VScanNode { std::map _filter_info; RuntimeProfile::Counter* _stats_filtered_counter = nullptr; + RuntimeProfile::Counter* _stats_rp_filtered_counter = nullptr; RuntimeProfile::Counter* _bf_filtered_counter = nullptr; RuntimeProfile::Counter* _dict_filtered_counter = nullptr; RuntimeProfile::Counter* _del_filtered_counter = nullptr; @@ -168,6 +169,10 @@ class NewOlapScanNode : public VScanNode { RuntimeProfile::Counter* _block_init_seek_timer = nullptr; RuntimeProfile::Counter* _block_init_seek_counter = nullptr; RuntimeProfile::Counter* _block_conditions_filtered_timer = nullptr; + RuntimeProfile::Counter* _block_conditions_filtered_bf_timer = nullptr; + RuntimeProfile::Counter* _block_conditions_filtered_zonemap_timer = nullptr; + RuntimeProfile::Counter* _block_conditions_filtered_zonemap_rp_timer = nullptr; + RuntimeProfile::Counter* _block_conditions_filtered_dict_timer = nullptr; RuntimeProfile::Counter* _first_read_timer = nullptr; RuntimeProfile::Counter* _second_read_timer = nullptr; RuntimeProfile::Counter* _first_read_seek_timer = nullptr; diff --git a/be/src/vec/exec/scan/new_olap_scanner.cpp b/be/src/vec/exec/scan/new_olap_scanner.cpp index de4aa3e27fa558..c6e401f3f5484b 100644 --- a/be/src/vec/exec/scan/new_olap_scanner.cpp +++ b/be/src/vec/exec/scan/new_olap_scanner.cpp @@ -590,6 +590,14 @@ void NewOlapScanner::_update_counters_before_close() { COUNTER_UPDATE(Parent->_block_init_seek_timer, stats.block_init_seek_ns); \ COUNTER_UPDATE(Parent->_block_init_seek_counter, stats.block_init_seek_num); \ COUNTER_UPDATE(Parent->_block_conditions_filtered_timer, stats.block_conditions_filtered_ns); \ + COUNTER_UPDATE(Parent->_block_conditions_filtered_bf_timer, \ + stats.block_conditions_filtered_bf_ns); \ + COUNTER_UPDATE(Parent->_block_conditions_filtered_zonemap_timer, \ + stats.block_conditions_filtered_zonemap_ns); \ + COUNTER_UPDATE(Parent->_block_conditions_filtered_zonemap_rp_timer, \ + stats.block_conditions_filtered_zonemap_rp_ns); \ + COUNTER_UPDATE(Parent->_block_conditions_filtered_dict_timer, \ + stats.block_conditions_filtered_dict_ns); \ COUNTER_UPDATE(Parent->_first_read_timer, stats.first_read_ns); \ COUNTER_UPDATE(Parent->_second_read_timer, stats.second_read_ns); \ COUNTER_UPDATE(Parent->_first_read_seek_timer, stats.block_first_read_seek_ns); \ @@ -608,6 +616,7 @@ void NewOlapScanner::_update_counters_before_close() { Parent->add_filter_info(id, info); \ } \ COUNTER_UPDATE(Parent->_stats_filtered_counter, stats.rows_stats_filtered); \ + COUNTER_UPDATE(Parent->_stats_rp_filtered_counter, stats.rows_stats_rp_filtered); \ COUNTER_UPDATE(Parent->_dict_filtered_counter, stats.rows_dict_filtered); \ COUNTER_UPDATE(Parent->_bf_filtered_counter, stats.rows_bf_filtered); \ COUNTER_UPDATE(Parent->_del_filtered_counter, stats.rows_del_filtered); \ diff --git a/be/src/vec/exec/scan/pip_scanner_context.h b/be/src/vec/exec/scan/pip_scanner_context.h index 56ceb20bf1502e..309aed96a8cf7f 100644 --- a/be/src/vec/exec/scan/pip_scanner_context.h +++ b/be/src/vec/exec/scan/pip_scanner_context.h @@ -31,9 +31,9 @@ class PipScannerContext : public vectorized::ScannerContext { public: PipScannerContext(RuntimeState* state, vectorized::VScanNode* parent, const TupleDescriptor* output_tuple_desc, - const std::list>& scanners, - int64_t limit_, int64_t max_bytes_in_blocks_queue, - const std::vector& col_distribute_ids, const int num_parallel_instances) + const std::list& scanners, int64_t limit_, + int64_t max_bytes_in_blocks_queue, const std::vector& col_distribute_ids, + const int num_parallel_instances) : vectorized::ScannerContext(state, parent, output_tuple_desc, scanners, limit_, max_bytes_in_blocks_queue, num_parallel_instances), _col_distribute_ids(col_distribute_ids), @@ -41,13 +41,14 @@ class PipScannerContext : public vectorized::ScannerContext { PipScannerContext(RuntimeState* state, ScanLocalStateBase* local_state, const TupleDescriptor* output_tuple_desc, - const std::list>& scanners, - int64_t limit_, int64_t max_bytes_in_blocks_queue, - const std::vector& col_distribute_ids, const int num_parallel_instances, - std::shared_ptr dependency) + const std::list& scanners, int64_t limit_, + int64_t max_bytes_in_blocks_queue, const std::vector& col_distribute_ids, + const int num_parallel_instances, + std::shared_ptr dependency, + std::shared_ptr finish_dependency) : vectorized::ScannerContext(state, output_tuple_desc, scanners, limit_, max_bytes_in_blocks_queue, num_parallel_instances, - local_state, dependency), + local_state, dependency, finish_dependency), _need_colocate_distribute(false) {} Status get_block_from_queue(RuntimeState* state, vectorized::BlockUPtr* block, bool* eos, @@ -110,6 +111,9 @@ class PipScannerContext : public vectorized::ScannerContext { return Status::OK(); } + // We should make those method lock free. + bool done() override { return _is_finished || _should_stop; } + void append_blocks_to_queue(std::vector& blocks) override { const int queue_size = _blocks_queues.size(); const int block_size = blocks.size(); diff --git a/be/src/vec/exec/scan/scanner_context.cpp b/be/src/vec/exec/scan/scanner_context.cpp index bbe55bfc067dfd..8bfe48187bb504 100644 --- a/be/src/vec/exec/scan/scanner_context.cpp +++ b/be/src/vec/exec/scan/scanner_context.cpp @@ -50,11 +50,11 @@ static bvar::Status g_bytes_in_scanner_queue("doris_bytes_in_scanner_qu static bvar::Status g_num_running_scanners("doris_num_running_scanners", 0); ScannerContext::ScannerContext(RuntimeState* state, const TupleDescriptor* output_tuple_desc, - const std::list>& scanners, - int64_t limit_, int64_t max_bytes_in_blocks_queue, - const int num_parallel_instances, + const std::list& scanners, int64_t limit_, + int64_t max_bytes_in_blocks_queue, const int num_parallel_instances, pipeline::ScanLocalStateBase* local_state, - std::shared_ptr dependency) + std::shared_ptr dependency, + std::shared_ptr finish_dependency) : _state(state), _parent(nullptr), _local_state(local_state), @@ -65,10 +65,11 @@ ScannerContext::ScannerContext(RuntimeState* state, const TupleDescriptor* outpu _max_bytes_in_queue(std::max(max_bytes_in_blocks_queue, (int64_t)1024) * num_parallel_instances), _scanner_scheduler(state->exec_env()->scanner_scheduler()), - _scanners(scanners.begin(), scanners.end()), - _all_scanners(scanners.begin(), scanners.end()), + _scanners(scanners), + _scanners_ref(scanners.begin(), scanners.end()), _num_parallel_instances(num_parallel_instances), - _dependency(dependency) { + _dependency(dependency), + _finish_dependency(finish_dependency) { // Use the task exec context as a lock between scanner threads and fragment exection threads _task_exec_ctx = _state->get_task_execution_context(); _query_id = _state->get_query_ctx()->query_id(); @@ -95,9 +96,8 @@ ScannerContext::ScannerContext(RuntimeState* state, const TupleDescriptor* outpu ScannerContext::ScannerContext(doris::RuntimeState* state, doris::vectorized::VScanNode* parent, const doris::TupleDescriptor* output_tuple_desc, - const std::list>& scanners, - int64_t limit_, int64_t max_bytes_in_blocks_queue, - const int num_parallel_instances, + const std::list& scanners, int64_t limit_, + int64_t max_bytes_in_blocks_queue, const int num_parallel_instances, pipeline::ScanLocalStateBase* local_state) : _state(state), _parent(parent), @@ -109,8 +109,8 @@ ScannerContext::ScannerContext(doris::RuntimeState* state, doris::vectorized::VS _max_bytes_in_queue(std::max(max_bytes_in_blocks_queue, (int64_t)1024) * num_parallel_instances), _scanner_scheduler(state->exec_env()->scanner_scheduler()), - _scanners(scanners.begin(), scanners.end()), - _all_scanners(scanners.begin(), scanners.end()), + _scanners(scanners), + _scanners_ref(scanners.begin(), scanners.end()), _num_parallel_instances(num_parallel_instances) { // Use the task exec context as a lock between scanner threads and fragment exection threads _task_exec_ctx = _state->get_task_execution_context(); @@ -189,6 +189,10 @@ Status ScannerContext::init() { } #endif + // 4. This ctx will be submitted to the scanner scheduler right after init. + // So set _num_scheduling_ctx to 1 here. + _num_scheduling_ctx = 1; + _num_unfinished_scanners = _scanners.size(); if (_parent) { @@ -286,9 +290,11 @@ Status ScannerContext::get_block_from_queue(RuntimeState* state, vectorized::Blo bool is_scheduled = false; if (!done() && to_be_schedule && _num_running_scanners == 0) { is_scheduled = true; - auto submit_status = _scanner_scheduler->submit(shared_from_this()); - if (!submit_status.ok()) { - set_status_on_error(submit_status, false); + auto state = _scanner_scheduler->submit(shared_from_this()); + if (state.ok()) { + _num_scheduling_ctx++; + } else { + set_status_on_error(state, false); } } @@ -381,19 +387,44 @@ Status ScannerContext::validate_block_schema(Block* block) { return Status::OK(); } +void ScannerContext::set_should_stop() { + std::lock_guard l(_transfer_lock); + _should_stop = true; + _set_scanner_done(); + for (const VScannerWPtr& scanner : _scanners_ref) { + if (VScannerSPtr sc = scanner.lock()) { + sc->try_stop(); + } + } + _blocks_queue_added_cv.notify_one(); + set_ready_to_finish(); +} + void ScannerContext::inc_num_running_scanners(int32_t inc) { std::lock_guard l(_transfer_lock); _num_running_scanners += inc; g_num_running_scanners.set_value(_num_running_scanners); } -void ScannerContext::dec_num_running_scanners(int32_t scanner_dec) { +void ScannerContext::dec_num_scheduling_ctx() { std::lock_guard l(_transfer_lock); - _num_running_scanners -= scanner_dec; + _num_scheduling_ctx--; + set_ready_to_finish(); + if (_num_running_scanners == 0 && _num_scheduling_ctx == 0) { + _ctx_finish_cv.notify_one(); + } g_num_running_scanners.set_value(_num_running_scanners); + +} + +void ScannerContext::set_ready_to_finish() { + // `_should_stop == true` means this task has already ended and wait for pending finish now. + if (_finish_dependency && done() && _num_running_scanners == 0 && _num_scheduling_ctx == 0) { + _finish_dependency->set_ready(); + } } -void ScannerContext::set_status_on_error(const Status& status, bool need_lock) { +bool ScannerContext::set_status_on_error(const Status& status, bool need_lock) { std::unique_lock l(_transfer_lock, std::defer_lock); if (need_lock) { l.lock(); @@ -404,20 +435,14 @@ void ScannerContext::set_status_on_error(const Status& status, bool need_lock) { _blocks_queue_added_cv.notify_one(); _should_stop = true; _set_scanner_done(); + return true; } + return false; } -void ScannerContext::stop_scanners(RuntimeState* state) { - std::unique_lock l(_transfer_lock); - _should_stop = true; - _set_scanner_done(); - for (const std::weak_ptr& scanner : _all_scanners) { - if (std::shared_ptr sc = scanner.lock()) { - sc->_scanner->try_stop(); - } - } - _blocks_queue.clear(); - // TODO yiguolei, call mark close to scanners +template +Status ScannerContext::_close_and_clear_scanners(Parent* parent, RuntimeState* state) { + std::unique_lock l(_scanners_lock); if (state->enable_profile()) { std::stringstream scanner_statistics; std::stringstream scanner_rows_read; @@ -425,38 +450,76 @@ void ScannerContext::stop_scanners(RuntimeState* state) { scanner_statistics << "["; scanner_rows_read << "["; scanner_wait_worker_time << "["; - // Scanners can in 3 state - // state 1: in scanner context, not scheduled - // state 2: in scanner worker pool's queue, scheduled but not running - // state 3: scanner is running. - for (auto& scanner_ref : _all_scanners) { - auto scanner = scanner_ref.lock(); - if (scanner == nullptr) { - continue; - } + for (auto finished_scanner_time : _finished_scanner_runtime) { + scanner_statistics << PrettyPrinter::print(finished_scanner_time, TUnit::TIME_NS) + << ", "; + } + for (auto finished_scanner_rows : _finished_scanner_rows_read) { + scanner_rows_read << PrettyPrinter::print(finished_scanner_rows, TUnit::UNIT) << ", "; + } + for (auto finished_scanner_wait_time : _finished_scanner_wait_worker_time) { + scanner_wait_worker_time + << PrettyPrinter::print(finished_scanner_wait_time, TUnit::TIME_NS) << ", "; + } + // Only unfinished scanners here + for (auto& scanner : _scanners) { + // Scanners are in ObjPool in ScanNode, + // so no need to delete them here. // Add per scanner running time before close them - scanner_statistics << PrettyPrinter::print(scanner->_scanner->get_time_cost_ns(), - TUnit::TIME_NS) + scanner_statistics << PrettyPrinter::print(scanner->get_time_cost_ns(), TUnit::TIME_NS) << ", "; - scanner_rows_read << PrettyPrinter::print(scanner->_scanner->get_rows_read(), - TUnit::UNIT) + scanner_rows_read << PrettyPrinter::print(scanner->get_rows_read(), TUnit::UNIT) << ", "; scanner_wait_worker_time - << PrettyPrinter::print(scanner->_scanner->get_scanner_wait_worker_timer(), + << PrettyPrinter::print(scanner->get_scanner_wait_worker_timer(), TUnit::TIME_NS) << ", "; - // since there are all scanners, some scanners is running, so that could not call scanner - // close here. } scanner_statistics << "]"; scanner_rows_read << "]"; scanner_wait_worker_time << "]"; - _scanner_profile->add_info_string("PerScannerRunningTime", scanner_statistics.str()); - _scanner_profile->add_info_string("PerScannerRowsRead", scanner_rows_read.str()); - _scanner_profile->add_info_string("PerScannerWaitTime", scanner_wait_worker_time.str()); + parent->scanner_profile()->add_info_string("PerScannerRunningTime", + scanner_statistics.str()); + parent->scanner_profile()->add_info_string("PerScannerRowsRead", scanner_rows_read.str()); + parent->scanner_profile()->add_info_string("PerScannerWaitTime", + scanner_wait_worker_time.str()); } + // Only unfinished scanners here + for (auto& scanner : _scanners) { + static_cast(scanner->close(state)); + // Scanners are in ObjPool in ScanNode, + // so no need to delete them here. + } + _scanners.clear(); + return Status::OK(); +} - _blocks_queue_added_cv.notify_one(); +template +void ScannerContext::clear_and_join(Parent* parent, RuntimeState* state) { + std::unique_lock l(_transfer_lock); + do { + if (_num_running_scanners == 0 && _num_scheduling_ctx == 0) { + break; + } else { + DCHECK(!state->enable_pipeline_exec()) + << " _num_running_scanners: " << _num_running_scanners + << " _num_scheduling_ctx: " << _num_scheduling_ctx; + while (!(_num_running_scanners == 0 && _num_scheduling_ctx == 0)) { + _ctx_finish_cv.wait(l); + } + break; + } + } while (false); + // Must wait all running scanners stop running. + // So that we can make sure to close all scanners. + static_cast(_close_and_clear_scanners(parent, state)); + + _blocks_queue.clear(); +} + +bool ScannerContext::no_schedule() { + std::unique_lock l(_transfer_lock); + return _num_running_scanners == 0 && _num_scheduling_ctx == 0; } void ScannerContext::_set_scanner_done() { @@ -469,11 +532,12 @@ std::string ScannerContext::debug_string() { return fmt::format( "id: {}, sacnners: {}, blocks in queue: {}," " status: {}, _should_stop: {}, _is_finished: {}, free blocks: {}," - " limit: {}, _num_running_scanners: {}, _max_thread_num: {}," + " limit: {}, _num_running_scanners: {}, _num_scheduling_ctx: {}, _max_thread_num: {}," " _block_per_scanner: {}, _cur_bytes_in_queue: {}, MAX_BYTE_OF_QUEUE: {}", ctx_id, _scanners.size(), _blocks_queue.size(), status().ok(), _should_stop, - _is_finished, _free_blocks.size_approx(), limit, _num_running_scanners, _max_thread_num, - _block_per_scanner, _cur_bytes_in_queue, _max_bytes_in_queue); + _is_finished, _free_blocks.size_approx(), limit, _num_running_scanners, + _num_scheduling_ctx, _max_thread_num, _block_per_scanner, _cur_bytes_in_queue, + _max_bytes_in_queue); } void ScannerContext::reschedule_scanner_ctx() { @@ -481,67 +545,84 @@ void ScannerContext::reschedule_scanner_ctx() { if (done()) { return; } - auto submit_status = _scanner_scheduler->submit(shared_from_this()); + auto state = _scanner_scheduler->submit(shared_from_this()); //todo(wb) rethinking is it better to mark current scan_context failed when submit failed many times? - if (!submit_status.ok()) { - set_status_on_error(submit_status, false); + if (state.ok()) { + _num_scheduling_ctx++; + } else { + set_status_on_error(state, false); } } -void ScannerContext::push_back_scanner_and_reschedule(std::shared_ptr scanner) { - std::lock_guard l(_transfer_lock); - // Use a transfer lock to avoid the scanner be scheduled concurrently. For example, that after - // calling "_scanners.push_front(scanner)", there may be other ctx in scheduler - // to schedule that scanner right away, and in that schedule run, the scanner may be marked as closed - // before we call the following if() block. - if (scanner->_scanner->need_to_close()) { - --_num_unfinished_scanners; - if (_num_unfinished_scanners == 0) { - _dispose_coloate_blocks_not_in_queue(); - _is_finished = true; - _set_scanner_done(); - _blocks_queue_added_cv.notify_one(); - return; - } +void ScannerContext::push_back_scanner_and_reschedule(VScannerSPtr scanner) { + { + std::unique_lock l(_scanners_lock); + _scanners.push_front(scanner); } + std::lock_guard l(_transfer_lock); - _scanners.push_front(scanner); + // In pipeline engine, doris will close scanners when `no_schedule`. + // We have to decrease _num_running_scanners before schedule, otherwise + // schedule does not woring due to _num_running_scanners. + _num_running_scanners--; + set_ready_to_finish(); - if (should_be_scheduled()) { - auto submit_status = _scanner_scheduler->submit(shared_from_this()); - if (!submit_status.ok()) { - set_status_on_error(submit_status, false); + if (!done() && should_be_scheduled()) { + auto state = _scanner_scheduler->submit(shared_from_this()); + if (state.ok()) { + _num_scheduling_ctx++; + } else { + set_status_on_error(state, false); } } + + // Notice that after calling "_scanners.push_front(scanner)", there may be other ctx in scheduler + // to schedule that scanner right away, and in that schedule run, the scanner may be marked as closed + // before we call the following if() block. + // So we need "scanner->set_counted_down()" to avoid "_num_unfinished_scanners" being decreased twice by + // same scanner. + if (scanner->need_to_close() && scanner->set_counted_down() && + (--_num_unfinished_scanners) == 0) { + _dispose_coloate_blocks_not_in_queue(); + _is_finished = true; + _set_scanner_done(); + _blocks_queue_added_cv.notify_one(); + } + _ctx_finish_cv.notify_one(); } -// This method is called in scanner scheduler, and task context is hold -void ScannerContext::get_next_batch_of_scanners( - std::list>* current_run) { - std::lock_guard l(_transfer_lock); - // Update the sched counter for profile - Defer defer {[&]() { _scanner_sched_counter->update(current_run->size()); }}; +void ScannerContext::get_next_batch_of_scanners(std::list* current_run) { // 1. Calculate how many scanners should be scheduled at this run. - // If there are enough space in blocks queue, - // the scanner number depends on the _free_blocks numbers - int thread_slot_num = get_available_thread_slot_num(); + int thread_slot_num = 0; + { + // If there are enough space in blocks queue, + // the scanner number depends on the _free_blocks numbers + thread_slot_num = get_available_thread_slot_num(); + } // 2. get #thread_slot_num scanners from ctx->scanners // and put them into "this_run". - for (int i = 0; i < thread_slot_num && !_scanners.empty();) { - std::weak_ptr scanner_ref = _scanners.front(); - std::shared_ptr scanner = scanner_ref.lock(); - _scanners.pop_front(); - if (scanner == nullptr) { - continue; - } - if (scanner->_scanner->need_to_close()) { - static_cast(scanner->_scanner->close(_state)); - } else { - current_run->push_back(scanner_ref); - i++; + { + std::unique_lock l(_scanners_lock); + for (int i = 0; i < thread_slot_num && !_scanners.empty();) { + VScannerSPtr scanner = _scanners.front(); + _scanners.pop_front(); + if (scanner->need_to_close()) { + _finished_scanner_runtime.push_back(scanner->get_time_cost_ns()); + _finished_scanner_rows_read.push_back(scanner->get_rows_read()); + _finished_scanner_wait_worker_time.push_back( + scanner->get_scanner_wait_worker_timer()); + static_cast(scanner->close(_state)); + } else { + current_run->push_back(scanner); + i++; + } } } } +template void ScannerContext::clear_and_join(pipeline::ScanLocalStateBase* parent, + RuntimeState* state); +template void ScannerContext::clear_and_join(VScanNode* parent, RuntimeState* state); + } // namespace doris::vectorized diff --git a/be/src/vec/exec/scan/scanner_context.h b/be/src/vec/exec/scan/scanner_context.h index 0957ccc09b5a17..bce5fc6c92957d 100644 --- a/be/src/vec/exec/scan/scanner_context.h +++ b/be/src/vec/exec/scan/scanner_context.h @@ -53,7 +53,6 @@ class TaskGroup; namespace vectorized { class VScanner; -class ScannerDelegate; class VScanNode; class ScannerScheduler; class SimplifiedScanScheduler; @@ -71,7 +70,7 @@ class ScannerContext : public std::enable_shared_from_this { public: ScannerContext(RuntimeState* state, VScanNode* parent, const TupleDescriptor* output_tuple_desc, - const std::list>& scanners, int64_t limit_, + const std::list& scanners, int64_t limit_, int64_t max_bytes_in_blocks_queue, const int num_parallel_instances = 1, pipeline::ScanLocalStateBase* local_state = nullptr); @@ -93,9 +92,9 @@ class ScannerContext : public std::enable_shared_from_this { // When a scanner complete a scan, this method will be called // to return the scanner to the list for next scheduling. - void push_back_scanner_and_reschedule(std::shared_ptr scanner); + void push_back_scanner_and_reschedule(VScannerSPtr scanner); - void set_status_on_error(const Status& status, bool need_lock = true); + bool set_status_on_error(const Status& status, bool need_lock = true); Status status() { if (_process_status.is()) { @@ -104,21 +103,34 @@ class ScannerContext : public std::enable_shared_from_this { return _process_status; } + // Called by ScanNode. + // Used to notify the scheduler that this ScannerContext can stop working. + void set_should_stop(); + // Return true if this ScannerContext need no more process - bool done() const { return _is_finished || _should_stop; } + virtual bool done() { return _is_finished || _should_stop; } bool is_finished() { return _is_finished.load(); } bool should_stop() { return _should_stop.load(); } bool status_error() { return _status_error.load(); } void inc_num_running_scanners(int32_t scanner_inc); - void dec_num_running_scanners(int32_t scanner_dec); + void set_ready_to_finish(); int get_num_running_scanners() const { return _num_running_scanners; } int get_num_unfinished_scanners() const { return _num_unfinished_scanners; } - void get_next_batch_of_scanners(std::list>* current_run); + void dec_num_scheduling_ctx(); + + int get_num_scheduling_ctx() const { return _num_scheduling_ctx; } + + void get_next_batch_of_scanners(std::list* current_run); + + template + void clear_and_join(Parent* parent, RuntimeState* state); + + bool no_schedule(); virtual std::string debug_string(); @@ -126,6 +138,7 @@ class ScannerContext : public std::enable_shared_from_this { void incr_num_ctx_scheduling(int64_t num) { _scanner_ctx_sched_counter->update(num); } void incr_ctx_scheduling_time(int64_t num) { _scanner_ctx_sched_time->update(num); } + void incr_num_scanner_scheduling(int64_t num) { _scanner_sched_counter->update(num); } std::string parent_name(); @@ -137,7 +150,7 @@ class ScannerContext : public std::enable_shared_from_this { << _max_bytes_in_queue << " _serving_blocks_num " << _serving_blocks_num << " _free_blocks_capacity " << _free_blocks_capacity << " estimated_block_bytes " << _estimated_block_bytes; - return !done() && (_cur_bytes_in_queue < _max_bytes_in_queue / 2) && + return (_cur_bytes_in_queue < _max_bytes_in_queue / 2) && (_serving_blocks_num < allowed_blocks_num()); } @@ -160,8 +173,6 @@ class ScannerContext : public std::enable_shared_from_this { SimplifiedScanScheduler* get_simple_scan_scheduler() { return _simple_scan_scheduler; } - void stop_scanners(RuntimeState* state); - void reschedule_scanner_ctx(); // the unique id of this context @@ -174,12 +185,17 @@ class ScannerContext : public std::enable_shared_from_this { std::weak_ptr get_task_execution_context() { return _task_exec_ctx; } +private: + template + Status _close_and_clear_scanners(Parent* parent, RuntimeState* state); + protected: ScannerContext(RuntimeState* state_, const TupleDescriptor* output_tuple_desc, - const std::list>& scanners_, int64_t limit_, + const std::list& scanners_, int64_t limit_, int64_t max_bytes_in_blocks_queue_, const int num_parallel_instances, pipeline::ScanLocalStateBase* local_state, - std::shared_ptr dependency); + std::shared_ptr dependency, + std::shared_ptr finish_dependency); virtual void _dispose_coloate_blocks_not_in_queue() {} void _set_scanner_done(); @@ -263,11 +279,9 @@ class ScannerContext : public std::enable_shared_from_this { // and then if the scanner is not finished, will be pushed back to this list. // Not need to protect by lock, because only one scheduler thread will access to it. std::mutex _scanners_lock; - // Scanner's ownership belong to vscannode or scanoperator, scanner context does not own it. - // ScannerContext has to check if scanner is deconstructed before use it. - std::list> _scanners; + std::list _scanners; // weak pointer for _scanners, used in stop function - std::vector> _all_scanners; + std::vector _scanners_ref; std::vector _finished_scanner_runtime; std::vector _finished_scanner_rows_read; std::vector _finished_scanner_wait_worker_time; @@ -284,6 +298,7 @@ class ScannerContext : public std::enable_shared_from_this { RuntimeProfile::Counter* _scanner_wait_batch_timer = nullptr; std::shared_ptr _dependency = nullptr; + std::shared_ptr _finish_dependency = nullptr; }; } // namespace vectorized } // namespace doris diff --git a/be/src/vec/exec/scan/scanner_scheduler.cpp b/be/src/vec/exec/scan/scanner_scheduler.cpp index 17d0068e74f90e..334c25704b3dce 100644 --- a/be/src/vec/exec/scan/scanner_scheduler.cpp +++ b/be/src/vec/exec/scan/scanner_scheduler.cpp @@ -180,14 +180,20 @@ void ScannerScheduler::_schedule_scanners(std::shared_ptr ctx) { watch.reset(); watch.start(); ctx->incr_num_ctx_scheduling(1); + size_t size = 0; + Defer defer {[&]() { + ctx->incr_num_scanner_scheduling(size); + ctx->dec_num_scheduling_ctx(); + }}; if (ctx->done()) { return; } - std::list> this_run; + std::list this_run; ctx->get_next_batch_of_scanners(&this_run); - if (this_run.empty()) { + size = this_run.size(); + if (!size) { // There will be 2 cases when this_run is empty: // 1. The blocks queue reaches limit. // The consumer will continue scheduling the ctx. @@ -206,14 +212,9 @@ void ScannerScheduler::_schedule_scanners(std::shared_ptr ctx) { if (ctx->thread_token != nullptr) { // TODO llj tg how to treat this? while (iter != this_run.end()) { - std::shared_ptr scanner_delegate = (*iter).lock(); - if (scanner_delegate == nullptr) { - continue; - } - scanner_delegate->_scanner->start_wait_worker_timer(); - auto s = ctx->thread_token->submit_func([this, scanner_ref = *iter, ctx]() { - this->_scanner_scan(this, ctx, scanner_ref); - }); + (*iter)->start_wait_worker_timer(); + auto s = ctx->thread_token->submit_func( + [this, scanner = *iter, ctx] { this->_scanner_scan(this, ctx, scanner); }); if (s.ok()) { this_run.erase(iter++); } else { @@ -223,32 +224,28 @@ void ScannerScheduler::_schedule_scanners(std::shared_ptr ctx) { } } else { while (iter != this_run.end()) { - std::shared_ptr scanner_delegate = (*iter).lock(); - if (scanner_delegate == nullptr) { - continue; - } - scanner_delegate->_scanner->start_wait_worker_timer(); - TabletStorageType type = scanner_delegate->_scanner->get_storage_type(); + (*iter)->start_wait_worker_timer(); + TabletStorageType type = (*iter)->get_storage_type(); bool ret = false; if (type == TabletStorageType::STORAGE_TYPE_LOCAL) { if (auto* scan_sche = ctx->get_simple_scan_scheduler()) { - auto work_func = [this, scanner_ref = *iter, ctx]() { - this->_scanner_scan(this, ctx, scanner_ref); + auto work_func = [this, scanner = *iter, ctx] { + this->_scanner_scan(this, ctx, scanner); }; SimplifiedScanTask simple_scan_task = {work_func, ctx}; ret = scan_sche->get_scan_queue()->try_put(simple_scan_task); } else { PriorityThreadPool::Task task; - task.work_function = [this, scanner_ref = *iter, ctx]() { - this->_scanner_scan(this, ctx, scanner_ref); + task.work_function = [this, scanner = *iter, ctx] { + this->_scanner_scan(this, ctx, scanner); }; task.priority = nice; ret = _local_scan_thread_pool->offer(task); } } else { PriorityThreadPool::Task task; - task.work_function = [this, scanner_ref = *iter, ctx]() { - this->_scanner_scan(this, ctx, scanner_ref); + task.work_function = [this, scanner = *iter, ctx] { + this->_scanner_scan(this, ctx, scanner); }; task.priority = nice; ret = _remote_scan_thread_pool->offer(task); @@ -266,22 +263,13 @@ void ScannerScheduler::_schedule_scanners(std::shared_ptr ctx) { } void ScannerScheduler::_scanner_scan(ScannerScheduler* scheduler, - std::shared_ptr ctx, - std::weak_ptr scanner_ref) { - Defer defer {[&]() { ctx->dec_num_running_scanners(1); }}; + std::shared_ptr ctx, VScannerSPtr scanner) { auto task_lock = ctx->get_task_execution_context().lock(); if (task_lock == nullptr) { // LOG(WARNING) << "could not lock task execution context, query " << print_id(_query_id) // << " maybe finished"; return; } - // will release scanner if it is the last one, task lock is hold here, to ensure - // that scanner could call scannode's method during deconstructor - std::shared_ptr scanner_delegate = scanner_ref.lock(); - auto& scanner = scanner_delegate->_scanner; - if (scanner_delegate == nullptr) { - return; - } SCOPED_ATTACH_TASK(scanner->runtime_state()); // for cpu hard limit, thread name should not be reset if (ctx->_should_reset_thread_name) { @@ -408,7 +396,7 @@ void ScannerScheduler::_scanner_scan(ScannerScheduler* scheduler, if (eos || should_stop) { scanner->mark_to_need_to_close(); } - ctx->push_back_scanner_and_reschedule(scanner_delegate); + ctx->push_back_scanner_and_reschedule(scanner); } void ScannerScheduler::_register_metrics() { diff --git a/be/src/vec/exec/scan/scanner_scheduler.h b/be/src/vec/exec/scan/scanner_scheduler.h index 9fedd27dbd8bb3..eb4d1380e3947c 100644 --- a/be/src/vec/exec/scan/scanner_scheduler.h +++ b/be/src/vec/exec/scan/scanner_scheduler.h @@ -36,7 +36,7 @@ class BlockingQueue; } // namespace doris namespace doris::vectorized { -class ScannerDelegate; + class ScannerContext; // Responsible for the scheduling and execution of all Scanners of a BE node. @@ -79,7 +79,7 @@ class ScannerScheduler { void _schedule_scanners(std::shared_ptr ctx); // execution thread function void _scanner_scan(ScannerScheduler* scheduler, std::shared_ptr ctx, - std::weak_ptr scanner); + VScannerSPtr scanner); void _register_metrics(); diff --git a/be/src/vec/exec/scan/vscan_node.cpp b/be/src/vec/exec/scan/vscan_node.cpp index b780fc1a8a97f0..5176d7900b3c7e 100644 --- a/be/src/vec/exec/scan/vscan_node.cpp +++ b/be/src/vec/exec/scan/vscan_node.cpp @@ -273,7 +273,7 @@ Status VScanNode::get_next(RuntimeState* state, vectorized::Block* block, bool* reached_limit(block, eos); if (*eos) { // reach limit, stop the scanners. - _scanner_ctx->stop_scanners(state); + _scanner_ctx->set_should_stop(); } return Status::OK(); @@ -318,8 +318,8 @@ Status VScanNode::_init_profile() { return Status::OK(); } -void VScanNode::_start_scanners(const std::list>& scanners, - const int query_parallel_instance_num) { +Status VScanNode::_start_scanners(const std::list& scanners, + const int query_parallel_instance_num) { if (_is_pipeline_scan) { int max_queue_size = _shared_scan_opt ? std::max(query_parallel_instance_num, 1) : 1; _scanner_ctx = pipeline::PipScannerContext::create_shared( @@ -329,29 +329,41 @@ void VScanNode::_start_scanners(const std::list _scanner_ctx = ScannerContext::create_shared(_state, this, _output_tuple_desc, scanners, limit(), _state->scan_queue_mem_limit()); } + return Status::OK(); } Status VScanNode::close(RuntimeState* state) { if (is_closed()) { return Status::OK(); } - RETURN_IF_ERROR(ExecNode::close(state)); return Status::OK(); } void VScanNode::release_resource(RuntimeState* state) { if (_scanner_ctx) { - if (!state->enable_pipeline_exec() || _should_create_scanner) { + if (!state->enable_pipeline_exec()) { // stop and wait the scanner scheduler to be done // _scanner_ctx may not be created for some short circuit case. - _scanner_ctx->stop_scanners(state); + _scanner_ctx->set_should_stop(); + _scanner_ctx->clear_and_join(this, state); + } else if (_should_create_scanner) { + _scanner_ctx->clear_and_join(this, state); } } - _scanners.clear(); + ExecNode::release_resource(state); } +Status VScanNode::try_close(RuntimeState* state) { + if (_scanner_ctx) { + // mark this scanner ctx as should_stop to make sure scanners will not be scheduled anymore + // TODO: there is a lock in `set_should_stop` may cause some slight impact + _scanner_ctx->set_should_stop(); + } + return Status::OK(); +} + Status VScanNode::_normalize_conjuncts() { // The conjuncts is always on output tuple, so use _output_tuple_desc; std::vector slots = _output_tuple_desc->slots(); @@ -1317,15 +1329,11 @@ VScanNode::PushDownType VScanNode::_should_push_down_in_predicate(VInPredicate* Status VScanNode::_prepare_scanners(const int query_parallel_instance_num) { std::list scanners; RETURN_IF_ERROR(_init_scanners(&scanners)); - // Init scanner wrapper - for (auto it = scanners.begin(); it != scanners.end(); ++it) { - _scanners.emplace_back(std::make_shared(*it)); - } if (scanners.empty()) { _eos = true; } else { COUNTER_SET(_num_scanners, static_cast(scanners.size())); - _start_scanners(_scanners, query_parallel_instance_num); + RETURN_IF_ERROR(_start_scanners(scanners, query_parallel_instance_num)); } return Status::OK(); } diff --git a/be/src/vec/exec/scan/vscan_node.h b/be/src/vec/exec/scan/vscan_node.h index d4a054cacd5a82..5917d0ff46b5eb 100644 --- a/be/src/vec/exec/scan/vscan_node.h +++ b/be/src/vec/exec/scan/vscan_node.h @@ -87,16 +87,6 @@ struct FilterPredicates { std::vector>> in_filters; }; -// We want to close scanner automatically, so using a delegate class -// and call close method in the delegate class's dctor. -class ScannerDelegate { -public: - VScannerSPtr _scanner; - ScannerDelegate(VScannerSPtr& scanner_ptr) : _scanner(scanner_ptr) {} - ~ScannerDelegate() { static_cast(_scanner->close(_scanner->runtime_state())); } - ScannerDelegate(ScannerDelegate&&) = delete; -}; - class VScanNode : public ExecNode, public RuntimeFilterConsumer { public: VScanNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs) @@ -166,6 +156,8 @@ class VScanNode : public ExecNode, public RuntimeFilterConsumer { Status alloc_resource(RuntimeState* state) override; void release_resource(RuntimeState* state) override; + Status try_close(RuntimeState* state); + bool should_run_serial() const { return _should_run_serial || _state->enable_scan_node_run_serial(); } @@ -270,11 +262,8 @@ class VScanNode : public ExecNode, public RuntimeFilterConsumer { int _max_scan_key_num; int _max_pushdown_conditions_per_column; - // ScanNode owns the ownership of scanner, scanner context only has its weakptr - std::list> _scanners; - - // Each scan node will generates a ScannerContext to do schedule work - // ScannerContext will be added to scanner scheduler + // Each scan node will generates a ScannerContext to manage all Scanners. + // See comments of ScannerContext for more details std::shared_ptr _scanner_ctx = nullptr; // indicate this scan node has no more data to return @@ -448,8 +437,8 @@ class VScanNode : public ExecNode, public RuntimeFilterConsumer { const std::string& fn_name, int slot_ref_child = -1); // Submit the scanner to the thread pool and start execution - void _start_scanners(const std::list>& scanners, - const int query_parallel_instance_num); + Status _start_scanners(const std::list& scanners, + const int query_parallel_instance_num); }; } // namespace doris::vectorized diff --git a/be/src/vec/exec/scan/vscanner.h b/be/src/vec/exec/scan/vscanner.h index 6046d87ac91ef5..29daf9a68c557f 100644 --- a/be/src/vec/exec/scan/vscanner.h +++ b/be/src/vec/exec/scan/vscanner.h @@ -145,6 +145,16 @@ class VScanner { void set_status_on_failure(const Status& st) { _status = st; } + // return false if _is_counted_down is already true, + // otherwise, set _is_counted_down to true and return true. + bool set_counted_down() { + if (_is_counted_down) { + return false; + } + _is_counted_down = true; + return true; + } + protected: void _discard_conjuncts() { for (auto& conjunct : _conjuncts) { @@ -205,6 +215,8 @@ class VScanner { int64_t _scan_cpu_timer = 0; bool _is_load = false; + // set to true after decrease the "_num_unfinished_scanners" in scanner context + bool _is_counted_down = false; bool _is_init = true; @@ -215,5 +227,6 @@ class VScanner { }; using VScannerSPtr = std::shared_ptr; +using VScannerWPtr = std::weak_ptr; } // namespace doris::vectorized diff --git a/be/src/vec/exprs/vexpr.cpp b/be/src/vec/exprs/vexpr.cpp index 97611867e78aa4..3f06dbb7f548a3 100644 --- a/be/src/vec/exprs/vexpr.cpp +++ b/be/src/vec/exprs/vexpr.cpp @@ -182,9 +182,9 @@ VExpr::VExpr(const TExprNode& node) VExpr::VExpr(const VExpr& vexpr) = default; -VExpr::VExpr(const TypeDescriptor& type, bool is_slotref, bool is_nullable) +VExpr::VExpr(TypeDescriptor type, bool is_slotref, bool is_nullable) : _opcode(TExprOpcode::INVALID_OPCODE), - _type(type), + _type(std::move(type)), _fn_context_index(-1), _prepared(false) { if (is_slotref) { @@ -197,13 +197,13 @@ VExpr::VExpr(const TypeDescriptor& type, bool is_slotref, bool is_nullable) Status VExpr::prepare(RuntimeState* state, const RowDescriptor& row_desc, VExprContext* context) { ++context->_depth_num; if (context->_depth_num > config::max_depth_of_expr_tree) { - return Status::InternalError( + return Status::Error( "The depth of the expression tree is too big, make it less than {}", config::max_depth_of_expr_tree); } - for (int i = 0; i < _children.size(); ++i) { - RETURN_IF_ERROR(_children[i]->prepare(state, row_desc, context)); + for (auto& i : _children) { + RETURN_IF_ERROR(i->prepare(state, row_desc, context)); } --context->_depth_num; return Status::OK(); @@ -211,8 +211,8 @@ Status VExpr::prepare(RuntimeState* state, const RowDescriptor& row_desc, VExprC Status VExpr::open(RuntimeState* state, VExprContext* context, FunctionContext::FunctionStateScope scope) { - for (int i = 0; i < _children.size(); ++i) { - RETURN_IF_ERROR(_children[i]->open(state, context, scope)); + for (auto& i : _children) { + RETURN_IF_ERROR(i->open(state, context, scope)); } if (scope == FunctionContext::FRAGMENT_LOCAL) { RETURN_IF_ERROR(VExpr::get_const_col(context, nullptr)); diff --git a/be/src/vec/exprs/vexpr.h b/be/src/vec/exprs/vexpr.h index e9f011a2c35e32..708b57ab638b78 100644 --- a/be/src/vec/exprs/vexpr.h +++ b/be/src/vec/exprs/vexpr.h @@ -80,7 +80,7 @@ class VExpr { VExpr(const TExprNode& node); VExpr(const VExpr& vexpr); - VExpr(const TypeDescriptor& type, bool is_slotref, bool is_nullable); + VExpr(TypeDescriptor type, bool is_slotref, bool is_nullable); // only used for test VExpr() = default; virtual ~VExpr() = default; diff --git a/be/src/vec/runtime/vdata_stream_recvr.cpp b/be/src/vec/runtime/vdata_stream_recvr.cpp index 56f6c51e68490d..0dc47363a8eef5 100644 --- a/be/src/vec/runtime/vdata_stream_recvr.cpp +++ b/be/src/vec/runtime/vdata_stream_recvr.cpp @@ -354,8 +354,7 @@ VDataStreamRecvr::VDataStreamRecvr( _profile(profile), _peak_memory_usage_counter(nullptr), _sub_plan_query_statistics_recvr(sub_plan_query_statistics_recvr), - _enable_pipeline(state->enable_pipeline_exec()), - _mem_available(std::make_shared(true)) { + _enable_pipeline(state->enable_pipeline_exec()) { // DataStreamRecvr may be destructed after the instance execution thread ends. _mem_tracker = std::make_unique("VDataStreamRecvr:" + print_id(_fragment_instance_id)); @@ -506,12 +505,21 @@ void VDataStreamRecvr::update_blocks_memory_usage(int64_t size) { _blocks_memory_usage->add(size); auto val = _blocks_memory_usage_current_value.fetch_add(size); if (val + size > config::exchg_node_buffer_size_bytes) { - *_mem_available = false; + if (_exchange_sink_mem_limit_dependency) { + _exchange_sink_mem_limit_dependency->block(); + } } else { - *_mem_available = true; + if (_exchange_sink_mem_limit_dependency) { + _exchange_sink_mem_limit_dependency->set_ready(); + } } } +void VDataStreamRecvr::create_mem_limit_dependency(int id, int node_id, QueryContext* query_ctx) { + _exchange_sink_mem_limit_dependency = + pipeline::LocalExchangeMemLimitDependency::create_shared(id, node_id, query_ctx); +} + void VDataStreamRecvr::close() { if (_is_closed) { return; diff --git a/be/src/vec/runtime/vdata_stream_recvr.h b/be/src/vec/runtime/vdata_stream_recvr.h index e0b63459ad2d5e..122a9d763e1be9 100644 --- a/be/src/vec/runtime/vdata_stream_recvr.h +++ b/be/src/vec/runtime/vdata_stream_recvr.h @@ -42,6 +42,7 @@ #include "common/object_pool.h" #include "common/status.h" #include "runtime/descriptors.h" +#include "runtime/query_context.h" #include "runtime/query_statistics.h" #include "util/runtime_profile.h" #include "util/stopwatch.hpp" @@ -61,6 +62,7 @@ class RuntimeState; namespace pipeline { struct ExchangeDataDependency; class LocalExchangeChannelDependency; +class LocalExchangeMemLimitDependency; class ExchangeLocalState; } // namespace pipeline @@ -130,6 +132,10 @@ class VDataStreamRecvr { std::shared_ptr get_local_channel_dependency( int sender_id); + void create_mem_limit_dependency(int id, int node_id, QueryContext* query_ctx); + + auto get_mem_limit_dependency() { return _exchange_sink_mem_limit_dependency; } + private: void update_blocks_memory_usage(int64_t size); class PipSenderQueue; @@ -189,7 +195,8 @@ class VDataStreamRecvr { std::vector> _sender_to_local_channel_dependency; - std::shared_ptr _mem_available; + // use to limit sink write + std::shared_ptr _exchange_sink_mem_limit_dependency; }; class ThreadClosure : public google::protobuf::Closure { diff --git a/be/src/vec/sink/vdata_stream_sender.h b/be/src/vec/sink/vdata_stream_sender.h index f59dad266f87ab..0e727a41f03e73 100644 --- a/be/src/vec/sink/vdata_stream_sender.h +++ b/be/src/vec/sink/vdata_stream_sender.h @@ -321,6 +321,11 @@ class Channel { void set_receiver_eof(Status st) { _receiver_status = st; } + auto local_recvr() { + DCHECK(is_local()); + return _local_recvr; + } + protected: bool _recvr_is_valid() { if (_local_recvr && !_local_recvr->is_closed()) { diff --git a/docs/en/docs/admin-manual/http-actions/fe/debug-point-action.md b/docs/en/docs/admin-manual/http-actions/fe/debug-point-action.md index cac2afdcd25dac..84ad9bf324a3f1 100644 --- a/docs/en/docs/admin-manual/http-actions/fe/debug-point-action.md +++ b/docs/en/docs/admin-manual/http-actions/fe/debug-point-action.md @@ -26,9 +26,17 @@ under the License. # Debug Point -Debug point is used in code test. When enabling a debug point, it can run related code. +Debug point is a piece of code, inserted into FE or BE code, when program running into this code, -Both FE and BE support debug points. +it can change variables or behaviors of the program. + +It is mainly used for unit test or regression test when it is impossible to trigger some exceptions through normal means. + +Each debug point has a name, the name can be whatever you want, there are swithes to enable and disable debug points, + +and you can also pass data to debug points. + +Both FE and BE support debug point, and after inserting debug point code, recompilation of FE or BE is needed. ## Code Example @@ -36,8 +44,8 @@ FE example ```java private Status foo() { - // dbug_fe_foo_do_nothing is the debug point name. - // When it's active,DebugPointUtil.isEnable("dbug_fe_foo_do_nothing") will return true. + // dbug_fe_foo_do_nothing is the debug point name + // when it's active, DebugPointUtil.isEnable("dbug_fe_foo_do_nothing") returns true if (DebugPointUtil.isEnable("dbug_fe_foo_do_nothing")) { return Status.Nothing; } @@ -48,13 +56,13 @@ private Status foo() { } ``` -BE 桩子示例代码 +BE example ```c++ void Status foo() { - // dbug_be_foo_do_nothing is the debug point name. - // When it's active,DEBUG_EXECUTE_IF will execute the code block. - DEBUG_EXECUTE_IF("dbug_be_foo_do_nothing", { return Status.Nothing; }); + // dbug_be_foo_do_nothing is the debug point name + // when it's active, DBUG_EXECUTE_IF will execute the code block + DBUG_EXECUTE_IF("dbug_be_foo_do_nothing", { return Status.Nothing; }); do_foo_action(); @@ -62,32 +70,36 @@ void Status foo() { } ``` -## Global config -To activate debug points, need set `enable_debug_points` to true. +## Global Config + +To enable debug points globally, we need to set `enable_debug_points` to true, + +`enable_debug_points` is located in FE's fe.conf and BE's be.conf. -`enable_debug_points` was located in FE's fe.conf and BE's be.conf。 +## Activate A Specified Debug Point -## Enable Debug Point +After debug points are enabled globally, a http request with a debug point name should be send to FE or BE node,
+only after that, when the program running into the specified debug point, related code can be executed. ### API ``` - POST /api/debug_point/add/{debug_point_name}[?timeout=&execute=] +POST /api/debug_point/add/{debug_point_name}[?timeout=&execute=] ``` ### Query Parameters * `debug_point_name` - Debug point name. Require. + Debug point name. Required. * `timeout` - Timeout in seconds. When timeout, the debug point will be disable. Default is -1, not timeout. Optional. + Timeout in seconds. When timeout, the debug point will be deactivated. Default is -1, never timeout. Optional. * `execute` - Max active times。Default is -1, unlimit active times. Optional. + After activating, the max times the debug point can be executed. Default is -1, unlimited times. Optional. ### Request body @@ -96,24 +108,105 @@ None ### Response - ``` - { - msg: "OK", - code: 0 - } - ``` +``` +{ + msg: "OK", + code: 0 +} +``` ### Examples -Enable debug point `foo`, activate no more than five times. +After activating debug point `foo`, executed no more than five times. - ``` - curl -X POST "http://127.0.0.1:8030/api/debug_point/add/foo?execute=5" +``` +curl -X POST "http://127.0.0.1:8030/api/debug_point/add/foo?execute=5" + +``` + + +## Pass Custom Parameters +When activating debug point, besides "timeout" and "execute" mentioned above, passing custom parameters is also allowed.
+A parameter is a key-value pair in the form of "key=value" in url path, after debug point name glued by charactor '?'.
+See examples below. + +### API + +``` +POST /api/debug_point/add/{debug_point_name}[?k1=v1&k2=v2&k3=v3...] +``` +* `k1=v1`
+ k1 is parameter name
+ v1 is parameter value
+ multiple key-value pairs are concatenated by `&`
+ + + +### Request body + +None + +### Response + +``` +{ + msg: "OK", + code: 0 +} +``` + +### Examples +Assuming a FE node with configuration http_port=8030 in fe.conf,
+the following http request activates a debug point named `foo` in FE node and passe parameter `percent` and `duration`: +>NOTE: User name and password may be needed. +``` +curl -u root: -X POST "http://127.0.0.1:8030/api/debug_point/add/foo?percent=0.5&duration=3" +``` + +``` +NOTE: +1. Inside FE and BE code, names and values of parameters are taken as strings. +2. Parameter names and values are case sensitive in http request and FE/BE code. +3. FE and BE share same url paths of REST API, it's just their IPs and Ports are different. +``` + +### Use parameters in FE and BE code +Following request activates debug point `OlapTableSink.write_random_choose_sink` in FE and passes parameter `needCatchUp` and `sinkNum`: +``` +curl -u root: -X POST "http://127.0.0.1:8030/api/debug_point/add/OlapTableSink.write_random_choose_sink?needCatchUp=true&sinkNum=3" +``` + +The code in FE checks debug point `OlapTableSink.write_random_choose_sink` and gets parameter values: +```java +private void debugWriteRandomChooseSink(Tablet tablet, long version, Multimap bePathsMap) { + DebugPoint debugPoint = DebugPointUtil.getDebugPoint("OlapTableSink.write_random_choose_sink"); + if (debugPoint == null) { + return; + } + boolean needCatchup = debugPoint.param("needCatchUp", false); + int sinkNum = debugPoint.param("sinkNum", 0); + ... +} +``` + +Following request activates debug point `TxnManager.prepare_txn.random_failed` in BE and passes parameter `percent`: +``` +curl -X POST "http://127.0.0.1:8040/api/debug_point/add/TxnManager.prepare_txn.random_failed?percent=0.7 +``` + +The code in BE checks debug point `TxnManager.prepare_txn.random_failed` and gets parameter value: +```c++ +DBUG_EXECUTE_IF("TxnManager.prepare_txn.random_failed", + {if (rand() % 100 < (100 * dp->param("percent", 0.5))) { + LOG_WARNING("TxnManager.prepare_txn.random_failed random failed"); + return Status::InternalError("debug prepare txn random failed"); + }} +); +``` + - ``` - ## Disable Debug Point ### API @@ -137,10 +230,10 @@ None ### Response ``` - { - msg: "OK", - code: 0 - } +{ + msg: "OK", + code: 0 +} ``` ### Examples @@ -149,17 +242,17 @@ None Disable debug point `foo`。 - ``` - curl -X POST "http://127.0.0.1:8030/api/debug_point/remove/foo" +``` +curl -X POST "http://127.0.0.1:8030/api/debug_point/remove/foo" - ``` +``` ## Clear Debug Points ### API ``` - POST /api/debug_point/clear +POST /api/debug_point/clear ``` @@ -170,16 +263,78 @@ None ### Response - ``` - { - msg: "OK", - code: 0 - } - ``` +``` +{ + msg: "OK", + code: 0 +} +``` ### Examples - ``` - curl -X POST "http://127.0.0.1:8030/api/debug_point/clear" - ``` \ No newline at end of file +``` +curl -X POST "http://127.0.0.1:8030/api/debug_point/clear" +``` + +## Debug Points in Regression Test + +>In community's CI system, `enable_debug_points` configuration of FE and BE are true by default. + +The Regression test framework also provides methods to activate and deactivate a particular debug point,
+they are declared as below: +```groovy +// "name" is the debug point to activate, "params" is a list of key-value pairs passed to debug point +def enableDebugPointForAllFEs(String name, Map params = null); +def enableDebugPointForAllBEs(String name, Map params = null); +// "name" is the debug point to deactivate +def disableDebugPointForAllFEs(String name); +def disableDebugPointForAllFEs(String name); +``` +`enableDebugPointForAllFEs()` or `enableDebugPointForAllBEs()` needs to be called before the test actions you want to generate error,
+and `disableDebugPointForAllFEs()` or `disableDebugPointForAllBEs()` needs to be called afterward. + +### Concurrent Issue + +Enabled debug points affects FE or BE globally, which could cause other concurrent tests to fail unexpectly in your pull request.
+To avoid this, there's a convension that regression tests using debug points must be in directory regression-test/suites/fault_injection_p0,
+and their group name must be "nonConcurrent", as these regression tests will be executed serially by pull request workflow. + +### Examples + +```groovy +// .groovy file of the test case must be in regression-test/suites/fault_injection_p0 +// and the group name must be 'nonConcurrent' +suite('debugpoint_action', 'nonConcurrent') { + try { + // Activate debug point named "PublishVersionDaemon.stop_publish" in all FE + // and pass parameter "timeout" + // "execute" and "timeout" are pre-existing parameters, usage is mentioned above + GetDebugPoint().enableDebugPointForAllFEs('PublishVersionDaemon.stop_publish', [timeout:1]) + + // Activate debug point named "Tablet.build_tablet_report_info.version_miss" in all BE + // and pass parameter "tablet_id", "version_miss" and "timeout" + GetDebugPoint().enableDebugPointForAllBEs('Tablet.build_tablet_report_info.version_miss', + [tablet_id:'12345', version_miss:true, timeout:1]) + + // Test actions which will run into debug point and generate error + sql """CREATE TABLE tbl_1 (k1 INT, k2 INT) + DUPLICATE KEY (k1) + DISTRIBUTED BY HASH(k1) + BUCKETS 3 + PROPERTIES ("replication_allocation" = "tag.location.default: 1"); + """ + sql "INSERT INTO tbl_1 VALUES (1, 10)" + sql "INSERT INTO tbl_1 VALUES (2, 20)" + order_qt_select_1_1 'SELECT * FROM tbl_1' + + } finally { + // Deactivate debug points + GetDebugPoint().disableDebugPointForAllFEs('PublishVersionDaemon.stop_publish') + GetDebugPoint().disableDebugPointForAllBEs('Tablet.build_tablet_report_info.version_miss') + } +} +``` + + + diff --git a/docs/en/docs/admin-manual/http-actions/fe/profile-action.md b/docs/en/docs/admin-manual/http-actions/fe/profile-action.md index 30f623a4bfd3de..4b3b8c444f25fd 100644 --- a/docs/en/docs/admin-manual/http-actions/fe/profile-action.md +++ b/docs/en/docs/admin-manual/http-actions/fe/profile-action.md @@ -29,6 +29,7 @@ under the License. ## Request `GET /api/profile` +`GET /api/profile/text` ## Description @@ -119,6 +120,7 @@ Query: - BlockConvertTime: 97.539us - BlockSeekCount: 0 ``` +If it is a text interface, simply return the plain text content of the profile. ## Path parameters @@ -164,3 +166,24 @@ None "count": 0 } ``` +2. Get the query profile text of the specified query id + ``` + GET /api/profile/text?query_id=f732084bc8e74f39-8313581c9c3c0b58 + + Response: + Summary: + - Profile ID: 48bdf6d75dbb46c9-998b9c0368f4561f + - Task Type: QUERY + - Start Time: 2023-12-20 11:09:41 + - End Time: 2023-12-20 11:09:45 + - Total: 3s680ms + - Task State: EOF + - User: root + - Default Db: tpcds + - Sql Statement: with customer_total_return as + select sr_customer_sk as ctr_customer_sk + ,sr_store_sk as ctr_store_sk + ,sum(SR_FEE) as ctr_total_return + ... + ``` + diff --git a/docs/en/docs/sql-manual/sql-functions/table-functions/s3.md b/docs/en/docs/sql-manual/sql-functions/table-functions/s3.md index c482542595da71..e410bf396497eb 100644 --- a/docs/en/docs/sql-manual/sql-functions/table-functions/s3.md +++ b/docs/en/docs/sql-manual/sql-functions/table-functions/s3.md @@ -160,6 +160,16 @@ select * from s3( "region" = "ap-hongkong", "format" = "parquet", "use_path_style" = "false"); + +// The BOS on Baidu Cloud will use 'virtual-hosted style' compatible with the S3 protocol to access s3. +// BOS +select * from s3( + "uri" = "https://example-bucket.s3.bj.bcebos.com/your-folder/file.parquet", + "s3.access_key"= "ak", + "s3.secret_key" = "sk", + "s3.region" = "bj", + "format" = "parquet", + "use_path_style" = "false"); ``` Example of s3://: diff --git a/docs/zh-CN/docs/admin-manual/http-actions/fe/debug-point-action.md b/docs/zh-CN/docs/admin-manual/http-actions/fe/debug-point-action.md index a1c9a59a35b7e8..df68ac003c804f 100644 --- a/docs/zh-CN/docs/admin-manual/http-actions/fe/debug-point-action.md +++ b/docs/zh-CN/docs/admin-manual/http-actions/fe/debug-point-action.md @@ -26,9 +26,13 @@ under the License. # 代码打桩 -代码打桩是代码测试使用的。激活木桩后,可以执行木桩代码。木桩的名字是任意取的。 +代码打桩,是指在 FE 或 BE 源码中插入一段代码,当程序执行到这里时,可以改变程序的变量或行为,这样的一段代码称为一个`木桩`。 -FE 和 BE 都支持代码打桩。 +主要用于单元测试或回归测试,用来构造正常方法无法实现的异常。 + +每一个木桩都有一个名称,可以随便取名,可以通过一些机制控制木桩的开关,还可以向木桩传递参数。 + +FE 和 BE 都支持代码打桩,打桩完后要重新编译 BE 或 FE。 ## 木桩代码示例 @@ -54,8 +58,8 @@ BE 桩子示例代码 void Status foo() { // dbug_be_foo_do_nothing 是一个木桩名字, - // 打开这个木桩之后,DEBUG_EXECUTE_IF 将会执行宏参数中的代码块 - DEBUG_EXECUTE_IF("dbug_be_foo_do_nothing", { return Status.Nothing; }); + // 打开这个木桩之后,DBUG_EXECUTE_IF 将会执行宏参数中的代码块 + DBUG_EXECUTE_IF("dbug_be_foo_do_nothing", { return Status.Nothing; }); do_foo_action(); @@ -71,11 +75,12 @@ void Status foo() { ## 打开木桩 +打开总开关后,还需要通过向 FE 或 BE 发送 http 请求的方式,打开或关闭指定名称的木桩,只有这样当代码执行到这个木桩时,相关代码才会被执行。 ### API ``` - POST /api/debug_point/add/{debug_point_name}[?timeout=&execute=] +POST /api/debug_point/add/{debug_point_name}[?timeout=&execute=] ``` @@ -85,10 +90,10 @@ void Status foo() { 木桩名字。必填。 * `timeout` - 超时时间,单位为秒。超时之后,木桩失活。默认值-1表示永远不超时。可填。 + 超时时间,单位为秒。超时之后,木桩失活。默认值-1表示永远不超时。可选。 * `execute` - 木桩最大激活次数。默认值-1表示不限激活次数。可填。 + 木桩最大执行次数。默认值-1表示不限执行次数。可选。 ### Request body @@ -97,30 +102,109 @@ void Status foo() { ### Response - ``` - { - msg: "OK", - code: 0 - } - ``` +``` +{ + msg: "OK", + code: 0 +} +``` ### Examples -打开木桩 `foo`,最多激活5次。 +打开木桩 `foo`,最多执行5次。 - ``` - curl -X POST "http://127.0.0.1:8030/api/debug_point/add/foo?execute=5" +``` +curl -X POST "http://127.0.0.1:8030/api/debug_point/add/foo?execute=5" + +``` - ``` +## 向木桩传递参数 + +激活木桩时,除了前文所述的 timeout 和 execute,还可以传递其它自定义参数。
+一个参数是一个形如 key=value 的 key-value 对,在 url 的路径部分,紧跟在木桩名称后,以字符 '?' 开头。 + +### API + +``` +POST /api/debug_point/add/{debug_point_name}[?k1=v1&k2=v2&k3=v3...] +``` +* `k1=v1` + k1为参数名称,v1为参数值,多个参数用&分隔。 + +### Request body + +无 + +### Response + +``` +{ + msg: "OK", + code: 0 +} +``` + +### Examples + +假设 FE 在 fe.conf 中有配置 http_port=8030,则下面的请求激活 FE 中的木桩`foo`,并传递了两个参数 `percent` 和 `duration`: + +``` +curl -u root: -X POST "http://127.0.0.1:8030/api/debug_point/add/foo?percent=0.5&duration=3" +``` + +``` +注意: +1、在 FE 或 BE 的代码中,参数名和参数值都是字符串。 +2、在 FE 或 BE 的代码中和 http 请求中,参数名称和值都是大小写敏感的。 +3、发给 FE 或 BE 的 http 请求,路径部分格式是相同的,只是 IP 地址和端口号不同。 +``` + +### 在 FE 和 BE 代码中使用参数 + +激活 FE 中的木桩`OlapTableSink.write_random_choose_sink`并传递参数 `needCatchUp` 和 `sinkNum`: +>注意:可能需要用户名和密码 +``` +curl -u root: -X POST "http://127.0.0.1:8030/api/debug_point/add/OlapTableSink.write_random_choose_sink?needCatchUp=true&sinkNum=3" +``` + +在 FE 代码中使用木桩 OlapTableSink.write_random_choose_sink 的参数 `needCatchUp` 和 `sinkNum`: +```java +private void debugWriteRandomChooseSink(Tablet tablet, long version, Multimap bePathsMap) { + DebugPoint debugPoint = DebugPointUtil.getDebugPoint("OlapTableSink.write_random_choose_sink"); + if (debugPoint == null) { + return; + } + boolean needCatchup = debugPoint.param("needCatchUp", false); + int sinkNum = debugPoint.param("sinkNum", 0); + ... +} +``` + + +激活 BE 中的木桩`TxnManager.prepare_txn.random_failed`并传递参数 `percent`: +``` +curl -X POST "http://127.0.0.1:8040/api/debug_point/add/TxnManager.prepare_txn.random_failed?percent=0.7 +``` +在 BE 代码中使用木桩 `TxnManager.prepare_txn.random_failed` 的参数 `percent`: +```c++ +DBUG_EXECUTE_IF("TxnManager.prepare_txn.random_failed", + {if (rand() % 100 < (100 * dp->param("percent", 0.5))) { + LOG_WARNING("TxnManager.prepare_txn.random_failed random failed"); + return Status::InternalError("debug prepare txn random failed"); + }} +); +``` + + ## 关闭木桩 ### API ``` - POST /api/debug_point/remove/{debug_point_name} +POST /api/debug_point/remove/{debug_point_name} ``` @@ -137,10 +221,10 @@ void Status foo() { ### Response ``` - { - msg: "OK", - code: 0 - } +{ + msg: "OK", + code: 0 +} ``` ### Examples @@ -149,39 +233,94 @@ void Status foo() { 关闭木桩`foo`。 - ``` - curl -X POST "http://127.0.0.1:8030/api/debug_point/remove/foo" - - ``` +``` +curl -X POST "http://127.0.0.1:8030/api/debug_point/remove/foo" +``` ## 清除所有木桩 ### API ``` - POST /api/debug_point/clear +POST /api/debug_point/clear ``` - - ### Request body 无 ### Response - ``` - { - msg: "OK", - code: 0 - } - ``` +``` +{ + msg: "OK", + code: 0 +} +``` ### Examples 清除所有木桩。 - ``` - curl -X POST "http://127.0.0.1:8030/api/debug_point/clear" - ``` +``` +curl -X POST "http://127.0.0.1:8030/api/debug_point/clear" +``` + +## 在回归测试中使用木桩 + +> 提交PR时,社区 CI 系统默认开启 FE 和 BE 的`enable_debug_points`配置。 + +回归测试框架提供方法函数来开关指定的木桩,它们声明如下: + +```groovy +// 打开木桩,name 是木桩名称,params 是一个key-value列表,是传给木桩的参数 +def enableDebugPointForAllFEs(String name, Map params = null); +def enableDebugPointForAllBEs(String name, Map params = null); +// 关闭木桩,name 是木桩的名称 +def disableDebugPointForAllFEs(String name); +def disableDebugPointForAllFEs(String name); +``` +需要在调用测试 action 之前调用 `enableDebugPointForAllFEs()` 或 `enableDebugPointForAllBEs()` 来开启木桩,
+这样执行到木桩代码时,相关代码才会被执行,
+然后在调用测试 action 之后调用 `disableDebugPointForAllFEs()` 或 `disableDebugPointForAllBEs()` 来关闭木桩。 + +### 并发问题 + +FE 或 BE 中开启的木桩是全局生效的,同一个 Pull Request 中,并发跑的其它测试,可能会受影响而意外失败。 +为了避免这种情况,我们规定,使用木桩的回归测试,必须放在 regression-test/suites/fault_injection_p0 目录下, +且组名必须设置为 `nonConcurrent`,社区 CI 系统对于这些用例,会串行运行。 + +### Examples + +```groovy +// 测试用例的.groovy 文件必须放在 regression-test/suites/fault_injection_p0 目录下, +// 且组名设置为 'nonConcurrent' +suite('debugpoint_action', 'nonConcurrent') { + try { + // 打开所有FE中,名为 "PublishVersionDaemon.stop_publish" 的木桩 + // 传参数 timeout + // 与上面curl调用时一样,execute 是执行次数,timeout 是超时秒数 + GetDebugPoint().enableDebugPointForAllFEs('PublishVersionDaemon.stop_publish', [timeout:1]) + // 打开所有BE中,名为 "Tablet.build_tablet_report_info.version_miss" 的木桩 + // 传参数 tablet_id, version_miss 和 timeout + GetDebugPoint().enableDebugPointForAllBEs('Tablet.build_tablet_report_info.version_miss', + [tablet_id:'12345', version_miss:true, timeout:1]) + + // 测试用例,会触发木桩代码的执行 + sql """CREATE TABLE tbl_1 (k1 INT, k2 INT) + DUPLICATE KEY (k1) + DISTRIBUTED BY HASH(k1) + BUCKETS 3 + PROPERTIES ("replication_allocation" = "tag.location.default: 1"); + """ + sql "INSERT INTO tbl_1 VALUES (1, 10)" + sql "INSERT INTO tbl_1 VALUES (2, 20)" + order_qt_select_1_1 'SELECT * FROM tbl_1' + + } finally { + GetDebugPoint().disableDebugPointForAllFEs('PublishVersionDaemon.stop_publish') + GetDebugPoint().disableDebugPointForAllBEs('Tablet.build_tablet_report_info.version_miss') + } +} +``` diff --git a/docs/zh-CN/docs/admin-manual/http-actions/fe/profile-action.md b/docs/zh-CN/docs/admin-manual/http-actions/fe/profile-action.md index f02380d125b487..e12d94a0b98dde 100644 --- a/docs/zh-CN/docs/admin-manual/http-actions/fe/profile-action.md +++ b/docs/zh-CN/docs/admin-manual/http-actions/fe/profile-action.md @@ -29,6 +29,7 @@ under the License. ## Request `GET /api/profile` +`GET /api/profile/text` ## Description @@ -119,6 +120,7 @@ Query: - BlockConvertTime: 97.539us - BlockSeekCount: 0 ``` +如果为text接口,直接返回profile的纯文本内容 ## Path parameters @@ -164,3 +166,24 @@ Query: "count": 0 } ``` +2. 获取指定 query_id 的 query profile 的纯文本 + ``` + GET /api/profile/text?query_id=f732084bc8e74f39-8313581c9c3c0b58 + + Response: + Summary: + - Profile ID: 48bdf6d75dbb46c9-998b9c0368f4561f + - Task Type: QUERY + - Start Time: 2023-12-20 11:09:41 + - End Time: 2023-12-20 11:09:45 + - Total: 3s680ms + - Task State: EOF + - User: root + - Default Db: tpcds + - Sql Statement: with customer_total_return as + select sr_customer_sk as ctr_customer_sk + ,sr_store_sk as ctr_store_sk + ,sum(SR_FEE) as ctr_total_return + ... + ``` + diff --git a/docs/zh-CN/docs/sql-manual/sql-functions/table-functions/s3.md b/docs/zh-CN/docs/sql-manual/sql-functions/table-functions/s3.md index 5a9ffd60404b8c..5ee99684a90f65 100644 --- a/docs/zh-CN/docs/sql-manual/sql-functions/table-functions/s3.md +++ b/docs/zh-CN/docs/sql-manual/sql-functions/table-functions/s3.md @@ -160,6 +160,16 @@ select * from s3( "s3.region" = "ap-hongkong", "format" = "parquet", "use_path_style" = "false"); + +// 百度云bos采用兼容s3协议的virtual-hosted style方式访问s3。 +// BOS +select * from s3( + "uri" = "https://example-bucket.s3.bj.bcebos.com/your-folder/file.parquet", + "s3.access_key"= "ak", + "s3.secret_key" = "sk", + "s3.region" = "bj", + "format" = "parquet", + "use_path_style" = "false"); ``` s3:// 使用示例: diff --git a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/ProfileAction.java b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/ProfileAction.java index e5679e32caa5bd..4dcac417d572a3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/ProfileAction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/ProfileAction.java @@ -61,4 +61,22 @@ protected Object profile(HttpServletRequest request, HttpServletResponse respons result.put("profile", queryProfileStr); return ResponseEntityBuilder.ok(result); } + + @RequestMapping(path = "/api/profile/text", method = RequestMethod.GET) + protected Object profileText(HttpServletRequest request, HttpServletResponse response) { + executeCheckPassword(request, response); + checkGlobalAuth(ConnectContext.get().getCurrentUserIdentity(), PrivPredicate.ADMIN); + + String queryId = request.getParameter("query_id"); + if (Strings.isNullOrEmpty(queryId)) { + return "Missing query_id"; + } + + String queryProfileStr = ProfileManager.getInstance().getProfile(queryId); + if (queryProfileStr == null) { + return "query id " + queryId + " not found"; + } + + return queryProfileStr; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/BulkLoadJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/BulkLoadJob.java index d5b173d82fcae8..7a9f160b3f8d7b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/BulkLoadJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/BulkLoadJob.java @@ -67,6 +67,7 @@ import java.util.Optional; import java.util.Set; import java.util.concurrent.RejectedExecutionException; +import java.util.function.Predicate; import java.util.stream.Collectors; /** @@ -124,7 +125,7 @@ public static BulkLoadJob fromLoadStmt(LoadStmt stmt) throws DdlException { break; case SPARK: bulkLoadJob = new SparkLoadJob(db.getId(), stmt.getLabel().getLabelName(), stmt.getResourceDesc(), - stmt.getOrigStmt(), stmt.getUserInfo()); + stmt.getOrigStmt(), stmt.getUserInfo()); break; case MINI: case DELETE: @@ -209,7 +210,10 @@ public void onTaskFailed(long taskId, FailMsg failMsg) { if (loadTask == null) { return; } - if (loadTask.getRetryTime() <= 0) { + Predicate isTaskTimeout = + (LoadTask task) -> task instanceof LoadLoadingTask + && ((LoadLoadingTask) task).getLeftTimeMs() <= 0; + if (loadTask.getRetryTime() <= 0 || isTaskTimeout.test(loadTask)) { unprotectedExecuteCancel(failMsg, true); logFinalOperation(); return; diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadLoadingTask.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadLoadingTask.java index fa2eadcfa65464..97abfb0db906ca 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadLoadingTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadLoadingTask.java @@ -110,7 +110,7 @@ public LoadLoadingTask(Database db, OlapTable table, } public void init(TUniqueId loadId, List> fileStatusList, - int fileNum, UserIdentity userInfo) throws UserException { + int fileNum, UserIdentity userInfo) throws UserException { this.loadId = loadId; planner = new LoadingTaskPlanner(callback.getCallbackId(), txnId, db.getId(), table, brokerDesc, fileGroups, strictMode, isPartialUpdate, timezone, this.timeoutS, this.loadParallelism, this.sendBatchParallelism, @@ -154,23 +154,23 @@ private void executeOnce() throws Exception { * here we use exec_mem_limit to directly override the load_mem_limit property. */ curCoordinator.setLoadMemLimit(execMemLimit); - curCoordinator.setTimeout((int) (getLeftTimeMs() / 1000)); - curCoordinator.setMemTableOnSinkNode(enableMemTableOnSinkNode); + + long leftTimeMs = getLeftTimeMs(); + if (leftTimeMs <= 0) { + throw new LoadException("failed to execute loading task when timeout"); + } + int timeoutS = (int) (leftTimeMs / 1000); + curCoordinator.setTimeout(timeoutS); try { QeProcessorImpl.INSTANCE.registerQuery(loadId, curCoordinator); - actualExecute(curCoordinator); + actualExecute(curCoordinator, timeoutS); } finally { QeProcessorImpl.INSTANCE.unregisterQuery(loadId); } } - private void actualExecute(Coordinator curCoordinator) throws Exception { - int waitSecond = (int) (getLeftTimeMs() / 1000); - if (waitSecond <= 0) { - throw new LoadException("failed to execute plan when the left time is less than 0"); - } - + private void actualExecute(Coordinator curCoordinator, int waitSecond) throws Exception { if (LOG.isDebugEnabled()) { LOG.debug(new LogBuilder(LogKey.LOAD_JOB, callback.getCallbackId()) .add("task_id", signature) @@ -199,8 +199,8 @@ private void actualExecute(Coordinator curCoordinator) throws Exception { } } - private long getLeftTimeMs() { - return Math.max(jobDeadlineMs - System.currentTimeMillis(), 1000L); + public long getLeftTimeMs() { + return jobDeadlineMs - System.currentTimeMillis(); } private void createProfile(Coordinator coord) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/joinorder/hypergraph/HyperGraph.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/joinorder/hypergraph/HyperGraph.java index 8c678c0d2c9a2e..e534fc1fa84624 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/joinorder/hypergraph/HyperGraph.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/joinorder/hypergraph/HyperGraph.java @@ -52,6 +52,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Set; import java.util.stream.Collectors; import javax.annotation.Nullable; @@ -691,7 +692,7 @@ private boolean compareEdgeWithExpr(Edge t, Edge o, Map } int size = t.getExpressions().size(); for (int i = 0; i < size; i++) { - if (!expressionMap.get(t.getExpression(i)).equals(o.getExpression(i))) { + if (!Objects.equals(expressionMap.get(t.getExpression(i)), o.getExpression(i))) { return false; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index dcbfb9ef3f04e0..37b42c7123f75c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -616,6 +616,9 @@ private String getOriginSql(ParserRuleContext ctx) { @Override public MTMVRefreshTriggerInfo visitRefreshTrigger(RefreshTriggerContext ctx) { + if (ctx == null) { + return new MTMVRefreshTriggerInfo(RefreshTrigger.MANUAL); + } if (ctx.MANUAL() != null) { return new MTMVRefreshTriggerInfo(RefreshTrigger.MANUAL); } @@ -662,6 +665,9 @@ public RefreshMethod visitRefreshMethod(RefreshMethodContext ctx) { @Override public BuildMode visitBuildMode(BuildModeContext ctx) { + if (ctx == null) { + return BuildMode.IMMEDIATE; + } if (ctx.DEFERRED() != null) { return BuildMode.DEFERRED; } else if (ctx.IMMEDIATE() != null) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewRule.java index 6dcf1885563691..90ebe567c4628e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewRule.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewRule.java @@ -26,7 +26,6 @@ import org.apache.doris.catalog.TableIf; import org.apache.doris.common.AnalysisException; import org.apache.doris.mtmv.BaseTableInfo; -import org.apache.doris.mtmv.MTMVCache; import org.apache.doris.mtmv.MTMVPartitionInfo; import org.apache.doris.mtmv.MTMVUtil; import org.apache.doris.nereids.CascadesContext; @@ -104,13 +103,8 @@ protected List rewrite(Plan queryPlan, CascadesContext cascadesContext) { logger.debug(currentClassName + " this group is already rewritten so skip"); continue; } - MTMV mtmv = materializationContext.getMTMV(); - MTMVCache mtmvCache = getCacheFromMTMV(mtmv); - if (mtmvCache == null) { - logger.warn(currentClassName + " mv cache is null so return"); - return rewriteResults; - } - List viewStructInfos = extractStructInfo(mtmvCache.getLogicalPlan(), cascadesContext); + List viewStructInfos = extractStructInfo(materializationContext.getMvPlan(), + cascadesContext); if (viewStructInfos.size() > 1) { // view struct info should only have one logger.warn(currentClassName + " the num of view struct info is more then one so return"); @@ -200,7 +194,7 @@ protected List rewrite(Plan queryPlan, CascadesContext cascadesContext) { CascadesContext rewrittenPlanContext = CascadesContext.initContext(cascadesContext.getStatementContext(), rewrittenPlan, cascadesContext.getCurrentJobContext().getRequiredProperties()); - Rewriter.getWholeTreeRewriter(cascadesContext).execute(); + Rewriter.getWholeTreeRewriter(rewrittenPlanContext).execute(); rewrittenPlan = rewrittenPlanContext.getRewritePlan(); logger.debug(currentClassName + "rewrite by materialized view success"); rewriteResults.add(rewrittenPlan); @@ -289,17 +283,6 @@ protected boolean checkPartitionIsValid( && relatedTalbeValidSet.containsAll(relatedTableSelectedPartitionToCheck); } - private MTMVCache getCacheFromMTMV(MTMV mtmv) { - MTMVCache cache; - try { - cache = mtmv.getOrGenerateCache(); - } catch (AnalysisException analysisException) { - logger.warn(this.getClass().getSimpleName() + " get mtmv cache analysisException", analysisException); - return null; - } - return cache; - } - /** * Rewrite query by view, for aggregate or join rewriting should be different inherit class implementation */ diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializationContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializationContext.java index 19b1f5be8bb10a..153688ecc2d3dd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializationContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializationContext.java @@ -51,6 +51,8 @@ public class MaterializationContext { // generate form mv scan plan private ExpressionMapping mvExprToMvScanExprMapping; private boolean available = true; + // the mv plan from cache at present, record it to make sure query rewrite by mv is right when cache change. + private Plan mvPlan; /** * MaterializationContext, this contains necessary info for query rewriting by mv @@ -81,6 +83,8 @@ public MaterializationContext(MTMV mtmv, mtmvCache.getMvOutputExpressions(), mtmvCache.getLogicalPlan()), mvScanPlan.getExpressions()); + // copy the plan from cache, which the plan in cache may change + this.mvPlan = mtmvCache.getLogicalPlan(); } public Set getMatchedGroups() { @@ -119,6 +123,10 @@ public boolean isAvailable() { return available; } + public Plan getMvPlan() { + return mvPlan; + } + /** * MaterializationContext fromMaterializedView */ diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewFilterJoinRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewFilterJoinRule.java index 04d5546b0a988e..d1ffa522bb0756 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewFilterJoinRule.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewFilterJoinRule.java @@ -37,7 +37,7 @@ public class MaterializedViewFilterJoinRule extends AbstractMaterializedViewJoin @Override public List buildRules() { return ImmutableList.of( - logicalFilter(logicalJoin(any(), any())).thenApplyMulti(ctx -> { + logicalFilter(logicalJoin(any(), any())).thenApplyMultiNoThrow(ctx -> { LogicalFilter> root = ctx.root; return rewrite(root, ctx.cascadesContext); }).toRule(RuleType.MATERIALIZED_VIEW_FILTER_JOIN)); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewFilterProjectJoinRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewFilterProjectJoinRule.java index 4ebee36585d88c..2c9cca199de1d8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewFilterProjectJoinRule.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewFilterProjectJoinRule.java @@ -38,7 +38,7 @@ public class MaterializedViewFilterProjectJoinRule extends AbstractMaterializedV @Override public List buildRules() { return ImmutableList.of( - logicalFilter(logicalProject(logicalJoin(any(), any()))).thenApplyMulti(ctx -> { + logicalFilter(logicalProject(logicalJoin(any(), any()))).thenApplyMultiNoThrow(ctx -> { LogicalFilter>> root = ctx.root; return rewrite(root, ctx.cascadesContext); }).toRule(RuleType.MATERIALIZED_VIEW_FILTER_PROJECT_JOIN)); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewOnlyJoinRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewOnlyJoinRule.java index cf2b246cd1dfd0..a9f8333f4dfd24 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewOnlyJoinRule.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewOnlyJoinRule.java @@ -36,7 +36,7 @@ public class MaterializedViewOnlyJoinRule extends AbstractMaterializedViewJoinRu @Override public List buildRules() { return ImmutableList.of( - logicalJoin(any(), any()).thenApplyMulti(ctx -> { + logicalJoin(any(), any()).thenApplyMultiNoThrow(ctx -> { LogicalJoin root = ctx.root; return rewrite(root, ctx.cascadesContext); }).toRule(RuleType.MATERIALIZED_VIEW_ONLY_JOIN)); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewProjectFilterJoinRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewProjectFilterJoinRule.java index 442a07d06a7e4b..e97a5378b83165 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewProjectFilterJoinRule.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewProjectFilterJoinRule.java @@ -38,7 +38,7 @@ public class MaterializedViewProjectFilterJoinRule extends AbstractMaterializedV @Override public List buildRules() { return ImmutableList.of( - logicalProject(logicalFilter(logicalJoin(any(), any()))).thenApplyMulti(ctx -> { + logicalProject(logicalFilter(logicalJoin(any(), any()))).thenApplyMultiNoThrow(ctx -> { LogicalProject>> root = ctx.root; return rewrite(root, ctx.cascadesContext); }).toRule(RuleType.MATERIALIZED_VIEW_PROJECT_FILTER_JOIN)); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtils.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtils.java index e219bec04ba343..ca54c3768318cc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtils.java @@ -28,6 +28,7 @@ import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.SlotReference; import org.apache.doris.nereids.trees.expressions.WindowExpression; +import org.apache.doris.nereids.trees.plans.JoinType; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; import org.apache.doris.nereids.trees.plans.logical.LogicalCatalogRelation; @@ -45,6 +46,7 @@ import java.util.List; import java.util.Optional; import java.util.Set; +import java.util.stream.Collectors; /** * The common util for materialized view @@ -175,6 +177,28 @@ public Void visitLogicalFilter(LogicalFilter filter, IncrementCh @Override public Void visitLogicalJoin(LogicalJoin join, IncrementCheckerContext context) { + Plan left = join.child(0); + Set leftColumnSet = left.getOutputSet().stream() + .filter(slot -> slot instanceof SlotReference + && slot.isColumnFromTable()) + .map(slot -> ((SlotReference) slot).getColumn().get()) + .collect(Collectors.toSet()); + boolean useLeft = leftColumnSet.contains(context.getMvPartitionColumn().getColumn().get()); + JoinType joinType = join.getJoinType(); + if (joinType.isInnerJoin() || joinType.isCrossJoin()) { + context.setPctPossible(true); + } else if (joinType.isLeftJoin() + || joinType.isLefSemiJoin() + || joinType.isLeftAntiJoin()) { + context.setPctPossible(useLeft); + } else if (joinType.isRightJoin() + || joinType.isRightAntiJoin() + || joinType.isRightSemiJoin()) { + context.setPctPossible(!useLeft); + } else { + // un supported join type + context.setPctPossible(false); + } return visit(join, context); } @@ -272,6 +296,7 @@ private static final class IncrementCheckerContext { private boolean pctPossible = true; private TableIf relatedTable; private Column relatedTableColumn; + private boolean joinNullGenerateSide; public IncrementCheckerContext(SlotReference mvPartitionColumn) { this.mvPartitionColumn = mvPartitionColumn; @@ -304,6 +329,14 @@ public Column getRelatedTableColumn() { public void setRelatedTableColumn(Column relatedTableColumn) { this.relatedTableColumn = relatedTableColumn; } + + public boolean isJoinNullGenerateSide() { + return joinNullGenerateSide; + } + + public void setJoinNullGenerateSide(boolean joinNullGenerateSide) { + this.joinNullGenerateSide = joinNullGenerateSide; + } } /** diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/OneRangePartitionEvaluator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/OneRangePartitionEvaluator.java index 3e08404d0becf1..e6c2d92c8e2f9a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/OneRangePartitionEvaluator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/OneRangePartitionEvaluator.java @@ -396,11 +396,12 @@ public EvaluateRangeResult visitInPredicate(InPredicate inPredicate, EvaluateRan if (inPredicate.getCompareExpr() instanceof Slot && inPredicate.getOptions().stream().allMatch(Literal.class::isInstance)) { Slot slot = (Slot) inPredicate.getCompareExpr(); - ColumnRange unionLiteralRange = inPredicate.getOptions() - .stream() - .map(Literal.class::cast) - .map(ColumnRange::singleton) - .reduce(ColumnRange.empty(), ColumnRange::union); + ColumnRange unionLiteralRange = ColumnRange.empty(); + ColumnRange slotRange = result.childrenResult.get(0).columnRanges.get(slot); + for (Expression expr : inPredicate.getOptions()) { + unionLiteralRange = unionLiteralRange.union( + slotRange.intersect(ColumnRange.singleton((Literal) expr))); + } Map slotRanges = result.childrenResult.get(0).columnRanges; result = intersectSlotRange(result, slotRanges, slot, unionLiteralRange); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/PartitionPruner.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/PartitionPruner.java index cb4137ba5b6f35..6089cf31a612c2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/PartitionPruner.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/PartitionPruner.java @@ -113,6 +113,7 @@ public static List prune(List partitionSlots, Expression partitionPr partitionTableType)) .collect(ImmutableList.toImmutableList()); + partitionPredicate = OrToIn.INSTANCE.rewrite(partitionPredicate, null); PartitionPruner partitionPruner = new PartitionPruner(evaluators, partitionPredicate); //TODO: we keep default partition because it's too hard to prune it, we return false in canPrune(). return partitionPruner.prune(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalJoin.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalJoin.java index c771f0372e1744..773dbce5fc256e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalJoin.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalJoin.java @@ -423,8 +423,8 @@ public FunctionalDependencies computeFuncDeps(Supplier> outputSupplie // TODO: consider Null-safe hash condition when left and rigth is not nullable boolean isLeftUnique = left().getLogicalProperties() .getFunctionalDependencies().isUnique(keys.first); - boolean isRightUnique = left().getLogicalProperties() - .getFunctionalDependencies().isUnique(keys.first); + boolean isRightUnique = right().getLogicalProperties() + .getFunctionalDependencies().isUnique(keys.second); Builder fdBuilder = new Builder(); if (joinType.isInnerJoin()) { // inner join propagate uniforms slots diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/OlapScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/OlapScanNode.java index 8d7a5c0ab261e6..a300b9eab742c9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/OlapScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/OlapScanNode.java @@ -837,7 +837,8 @@ private void addScanRangeLocations(Partition partition, if (backend == null || !backend.isAlive()) { LOG.debug("backend {} not exists or is not alive for replica {}", replica.getBackendId(), replica.getId()); - errs.add(replica.getId() + "'s backend " + replica.getBackendId() + " does not exist or not alive"); + errs.add("replica " + replica.getId() + "'s backend " + replica.getBackendId() + + " does not exist or not alive"); continue; } if (!backend.isMixNode()) { @@ -875,7 +876,7 @@ private void addScanRangeLocations(Partition partition, } } if (tabletIsNull) { - throw new UserException(tabletId + " have no queryable replicas. err: " + throw new UserException("tablet " + tabletId + " has no queryable replicas. err: " + Joiner.on(", ").join(errs)); } TScanRange scanRange = new TScanRange(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/external/paimon/PaimonPredicateConverter.java b/fe/fe-core/src/main/java/org/apache/doris/planner/external/paimon/PaimonPredicateConverter.java index 9c997c03190419..cd35d1093ed084 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/external/paimon/PaimonPredicateConverter.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/external/paimon/PaimonPredicateConverter.java @@ -17,14 +17,15 @@ package org.apache.doris.planner.external.paimon; -import org.apache.doris.analysis.BinaryPredicate; import org.apache.doris.analysis.CastExpr; import org.apache.doris.analysis.CompoundPredicate; import org.apache.doris.analysis.Expr; +import org.apache.doris.analysis.FunctionCallExpr; import org.apache.doris.analysis.LiteralExpr; import org.apache.doris.analysis.SlotRef; import org.apache.doris.thrift.TExprOpcode; +import org.apache.paimon.data.BinaryString; import org.apache.paimon.predicate.Predicate; import org.apache.paimon.predicate.PredicateBuilder; import org.apache.paimon.types.DataField; @@ -84,10 +85,9 @@ public Predicate convertToPaimonExpr(Expr dorisExpr) { private Predicate binaryExprDesc(Expr dorisExpr) { TExprOpcode opcode = dorisExpr.getOpcode(); - BinaryPredicate bp = (BinaryPredicate) dorisExpr; // Make sure the col slot is always first - SlotRef slotRef = convertDorisExprToSlotRef(bp.getChild(0)); - LiteralExpr literalExpr = convertDorisExprToLiteralExpr(bp.getChild(1)); + SlotRef slotRef = convertDorisExprToSlotRef(dorisExpr.getChild(0)); + LiteralExpr literalExpr = convertDorisExprToLiteralExpr(dorisExpr.getChild(1)); if (slotRef == null || literalExpr == null) { return null; } @@ -113,6 +113,15 @@ private Predicate binaryExprDesc(Expr dorisExpr) { return builder.lessOrEqual(idx, value); case LT: return builder.lessThan(idx, value); + case INVALID_OPCODE: + if (dorisExpr instanceof FunctionCallExpr) { + String name = dorisExpr.getExprName().toLowerCase(); + String s = value.toString(); + if (name.equals("like") && !s.startsWith("%") && s.endsWith("%")) { + return builder.startsWith(idx, BinaryString.fromString(s.substring(0, s.length() - 1))); + } + } + return null; default: return null; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java index 7d7fdb1404e3b7..b9658bf34a5802 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java @@ -1422,7 +1422,7 @@ public void setEnableLeftZigZag(boolean enableLeftZigZag) { + "When enable_auto_sample is enabled, tables" + "larger than this value will automatically collect " + "statistics through sampling"}) - public long hugeTableLowerBoundSizeInBytes = 5L * 1024 * 1024 * 1024; + public long hugeTableLowerBoundSizeInBytes = 0; @VariableMgr.VarAttr(name = HUGE_TABLE_AUTO_ANALYZE_INTERVAL_IN_MILLIS, flag = VariableMgr.GLOBAL, description = {"控制对大表的自动ANALYZE的最小时间间隔," diff --git a/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java b/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java index 4f88d22c836f3c..75405854cabed2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java +++ b/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java @@ -114,6 +114,7 @@ import org.apache.doris.thrift.TCheckAuthResult; import org.apache.doris.thrift.TColumnDef; import org.apache.doris.thrift.TColumnDesc; +import org.apache.doris.thrift.TColumnInfo; import org.apache.doris.thrift.TCommitTxnRequest; import org.apache.doris.thrift.TCommitTxnResult; import org.apache.doris.thrift.TConfirmUnusedRemoteFilesRequest; @@ -240,7 +241,6 @@ import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.NoSuchElementException; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutionException; @@ -3280,40 +3280,38 @@ private TGetMetaResult getMetaImpl(TGetMetaRequest request, String clientIp) @Override public TGetColumnInfoResult getColumnInfo(TGetColumnInfoRequest request) { TGetColumnInfoResult result = new TGetColumnInfoResult(); - TStatus errorStatus = new TStatus(TStatusCode.RUNTIME_ERROR); + TStatus status = new TStatus(TStatusCode.OK); + result.setStatus(status); long dbId = request.getDbId(); long tableId = request.getTableId(); if (!Env.getCurrentEnv().isMaster()) { - errorStatus.setStatusCode(TStatusCode.NOT_MASTER); - errorStatus.addToErrorMsgs(NOT_MASTER_ERR_MSG); + status.setStatusCode(TStatusCode.NOT_MASTER); + status.addToErrorMsgs(NOT_MASTER_ERR_MSG); LOG.error("failed to getColumnInfo: {}", NOT_MASTER_ERR_MSG); return result; } - Database db = Env.getCurrentEnv().getInternalCatalog().getDbNullable(dbId); + Database db = Env.getCurrentInternalCatalog().getDbNullable(dbId); if (db == null) { - errorStatus.setErrorMsgs(Lists.newArrayList(String.format("dbId=%d is not exists", dbId))); - result.setStatus(errorStatus); + status.setStatusCode(TStatusCode.NOT_FOUND); + status.setErrorMsgs(Lists.newArrayList(String.format("dbId=%d is not exists", dbId))); return result; } - - Table table; - try { - table = db.getTable(tableId).get(); - } catch (NoSuchElementException e) { - errorStatus.setErrorMsgs( + Table table = db.getTableNullable(tableId); + if (table == null) { + status.setStatusCode(TStatusCode.NOT_FOUND); + status.setErrorMsgs( (Lists.newArrayList(String.format("dbId=%d tableId=%d is not exists", dbId, tableId)))); - result.setStatus(errorStatus); return result; } - StringBuilder sb = new StringBuilder(); + List columnsResult = Lists.newArrayList(); for (Column column : table.getBaseSchema(true)) { - sb.append(column.getName() + ":" + column.getUniqueId() + ","); + final TColumnInfo info = new TColumnInfo(); + info.setColumnName(column.getName()); + info.setColumnId(column.getUniqueId()); + columnsResult.add(info); } - String columnInfo = sb.toString(); - columnInfo = columnInfo.substring(0, columnInfo.length() - 1); - result.setStatus(new TStatus(TStatusCode.OK)); - result.setColumnInfo(columnInfo); + result.setColumns(columnsResult); return result; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/OlapAnalysisTask.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/OlapAnalysisTask.java index 50042e4610f492..e062e4eef850d2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/OlapAnalysisTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/OlapAnalysisTask.java @@ -61,7 +61,11 @@ public OlapAnalysisTask(AnalysisInfo info) { public void doExecute() throws Exception { Set partitionNames = info.colToPartitions.get(info.colName); - if (partitionNames.isEmpty()) { + if (partitionNames == null || partitionNames.isEmpty()) { + if (partitionNames == null) { + LOG.warn("Table {}.{}.{}, partitionNames for column {} is null. ColToPartitions:[{}]", + info.catalogId, info.dbId, info.tblId, info.colName, info.colToPartitions); + } StatsId statsId = new StatsId(concatColumnStatsId(), info.catalogId, info.dbId, info.tblId, info.indexId, info.colName, null); job.appendBuf(this, Arrays.asList(new ColStatsData(statsId))); diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticConstants.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticConstants.java index 111305b03ed8b6..3d6d2fe52aa937 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticConstants.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticConstants.java @@ -86,7 +86,7 @@ public class StatisticConstants { public static final int INSERT_MERGE_ITEM_COUNT = 200; public static final long HUGE_TABLE_DEFAULT_SAMPLE_ROWS = 4194304; - public static final long HUGE_TABLE_LOWER_BOUND_SIZE_IN_BYTES = 5L * 1024 * 1024 * 1024; + public static final long HUGE_TABLE_LOWER_BOUND_SIZE_IN_BYTES = 0; public static final long HUGE_TABLE_AUTO_ANALYZE_INTERVAL_IN_MILLIS = TimeUnit.HOURS.toMillis(12); diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsAutoCollector.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsAutoCollector.java index 244b1059d7e061..ee50471175d4f8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsAutoCollector.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsAutoCollector.java @@ -148,7 +148,7 @@ protected boolean skip(TableIf table) { protected void createAnalyzeJobForTbl(DatabaseIf db, List analysisInfos, TableIf table) { - AnalysisMethod analysisMethod = table.getDataSize(true) > StatisticsUtil.getHugeTableLowerBoundSizeInBytes() + AnalysisMethod analysisMethod = table.getDataSize(true) >= StatisticsUtil.getHugeTableLowerBoundSizeInBytes() ? AnalysisMethod.SAMPLE : AnalysisMethod.FULL; AnalysisInfo jobInfo = new AnalysisInfoBuilder() .setJobId(Env.getCurrentEnv().getNextId()) diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtilsTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtilsTest.java index 898b5843031865..c2af1dbdf4aa17 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtilsTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtilsTest.java @@ -186,7 +186,7 @@ public void getRelatedTableInfoUseRightTest() { + " (select * from " + " lineitem " + " where L_SHIPDATE in ('2017-01-30')) t1 " - + "left join " + + "right join " + " (select * from " + " orders " + " where O_ORDERDATE in ('2017-01-30')) t2 " @@ -207,6 +207,33 @@ public void getRelatedTableInfoUseRightTest() { }); } + @Test + public void getRelatedTableInfoUseNullGenerateSideTest() { + PlanChecker.from(connectContext) + .checkExplain("SELECT t1.L_SHIPDATE, t2.O_ORDERDATE, t1.L_QUANTITY, t2.O_ORDERSTATUS, " + + "count(distinct case when t1.L_SUPPKEY > 0 then t2.O_ORDERSTATUS else null end) as cnt_1 " + + "from " + + " (select * from " + + " lineitem " + + " where L_SHIPDATE in ('2017-01-30')) t1 " + + "left join " + + " (select * from " + + " orders " + + " where O_ORDERDATE in ('2017-01-30')) t2 " + + "on t1.L_ORDERKEY = t2.O_ORDERKEY " + + "group by " + + "t1.L_SHIPDATE, " + + "t2.O_ORDERDATE, " + + "t1.L_QUANTITY, " + + "t2.O_ORDERSTATUS;", + nereidsPlanner -> { + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan(); + Optional relatedTableInfo = + MaterializedViewUtils.getRelatedTableInfo("o_orderdate", rewrittenPlan); + Assertions.assertFalse(relatedTableInfo.isPresent()); + }); + } + @Test public void getRelatedTableInfoTestWithoutPartitionTest() { PlanChecker.from(connectContext) diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/EliminateJoinByFkTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/EliminateJoinByFkTest.java index 1faaca4f3f5550..721e94b144d05c 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/EliminateJoinByFkTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/EliminateJoinByFkTest.java @@ -107,7 +107,8 @@ void testNullWithPredicate() throws Exception { @Test void testMultiJoin() throws Exception { - addConstraint("Alter table foreign_null add constraint uk unique (id3)\n"); + addConstraint("Alter table foreign_null add constraint uk_id3 unique (id3)\n"); + addConstraint("Alter table foreign_not_null add constraint uk_id2 unique (id2)\n"); String sql = "select id1 from " + "foreign_null inner join foreign_not_null on id2 = id3\n" + "inner join pri on id1 = id3"; @@ -116,6 +117,7 @@ void testMultiJoin() throws Exception { .rewrite() .nonMatch(logicalOlapScan().when(scan -> scan.getTable().getName().equals("pri"))) .printlnTree(); - dropConstraint("Alter table foreign_null drop constraint uk\n"); + dropConstraint("Alter table foreign_null drop constraint uk_id3\n"); + dropConstraint("Alter table foreign_not_null drop constraint uk_id2"); } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/planner/ResourceTagQueryTest.java b/fe/fe-core/src/test/java/org/apache/doris/planner/ResourceTagQueryTest.java index baf5486195fae4..5afe9c3b39b8b6 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/planner/ResourceTagQueryTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/planner/ResourceTagQueryTest.java @@ -225,7 +225,7 @@ public void test() throws Exception { Assert.assertTrue(connectContext.isResourceTagsSet()); queryStr = "explain select * from test.tbl1"; String error = UtFrameUtils.getSQLPlanOrErrorMsg(connectContext, queryStr); - Assert.assertTrue(error.contains("have no queryable replicas")); + Assert.assertTrue(error.contains("no queryable replicas")); // set [0, 1, 2] backends' tag to zone1, so that at least 1 replica can be queried. // set tag for all backends. 0-2 to zone1, 4 and 5 to zone2 @@ -320,7 +320,7 @@ public void test() throws Exception { queryStr = "explain select * from test_prop.tbl2"; explainString = UtFrameUtils.getSQLPlanOrErrorMsg(connectContext, queryStr); System.out.println(explainString); - Assert.assertTrue(explainString.contains("have no queryable replicas")); + Assert.assertTrue(explainString.contains("no queryable replicas")); // The priority of table is higher than db,should same with table String createTableStr3 = "create table test_prop.tbl3\n" diff --git a/fe/fe-core/src/test/java/org/apache/doris/statistics/HMSAnalysisTaskTest.java b/fe/fe-core/src/test/java/org/apache/doris/statistics/HMSAnalysisTaskTest.java index a569a5cb06d9dc..12a1a9c046b4d0 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/statistics/HMSAnalysisTaskTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/statistics/HMSAnalysisTaskTest.java @@ -86,7 +86,7 @@ public void testAutoSampleSmallTable(@Mocked HMSExternalTable tableIf) new MockUp() { @Mock public long getDataSize(boolean singleReplica) { - return 1000; + return StatisticsUtil.getHugeTableLowerBoundSizeInBytes() - 1; } }; HMSAnalysisTask task = new HMSAnalysisTask(); diff --git a/fe/fe-core/src/test/java/org/apache/doris/statistics/OlapAnalysisTaskTest.java b/fe/fe-core/src/test/java/org/apache/doris/statistics/OlapAnalysisTaskTest.java index 5b27c79c8633ed..e0b5a4b047892e 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/statistics/OlapAnalysisTaskTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/statistics/OlapAnalysisTaskTest.java @@ -101,7 +101,7 @@ public void testSample3(@Mocked OlapTable tbl) { @Mock public long getDataSize(boolean singleReplica) { - return 1000; + return StatisticsUtil.getHugeTableLowerBoundSizeInBytes() - 1; } }; diff --git a/fe/fe-core/src/test/java/org/apache/doris/statistics/StatisticsAutoCollectorTest.java b/fe/fe-core/src/test/java/org/apache/doris/statistics/StatisticsAutoCollectorTest.java index 9f6d3db74b58a7..0b4b2203d0dc79 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/statistics/StatisticsAutoCollectorTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/statistics/StatisticsAutoCollectorTest.java @@ -346,7 +346,7 @@ public List getBaseSchema() { @Mock public long getDataSize(boolean singleReplica) { - return 1000; + return StatisticsUtil.getHugeTableLowerBoundSizeInBytes() - 1; } @Mock diff --git a/gensrc/thrift/FrontendService.thrift b/gensrc/thrift/FrontendService.thrift index d8a53d35662658..d672597a0db40f 100644 --- a/gensrc/thrift/FrontendService.thrift +++ b/gensrc/thrift/FrontendService.thrift @@ -1302,6 +1302,11 @@ struct TGetBackendMetaResult { 3: optional Types.TNetworkAddress master_address } +struct TColumnInfo { + 1: optional string columnName + 2: optional i64 columnId +} + struct TGetColumnInfoRequest { 1: optional i64 db_id 2: optional i64 table_id @@ -1309,7 +1314,7 @@ struct TGetColumnInfoRequest { struct TGetColumnInfoResult { 1: optional Status.TStatus status - 2: optional string column_info + 2: optional list columns } service FrontendService { diff --git a/regression-test/data/correctness_p0/test_null_aware_left_anti_join.out b/regression-test/data/correctness_p0/test_null_aware_left_anti_join.out index d33e4e2947f228..09d7d231709999 100644 --- a/regression-test/data/correctness_p0/test_null_aware_left_anti_join.out +++ b/regression-test/data/correctness_p0/test_null_aware_left_anti_join.out @@ -9,3 +9,10 @@ -- !select -- +-- !anti_emtpy_right -- +\N +1 +3 + +-- !semi_emtpy_right -- + diff --git a/regression-test/data/external_table_p0/paimon/test_paimon_catalog.out b/regression-test/data/external_table_p0/paimon/test_paimon_catalog.out index a2e4f2752c01c1..b0c9eff5b0a7f8 100644 --- a/regression-test/data/external_table_p0/paimon/test_paimon_catalog.out +++ b/regression-test/data/external_table_p0/paimon/test_paimon_catalog.out @@ -7,6 +7,22 @@ 2 2 3 4 5 6 7 8 9.1 10.1 11.10 2020-02-02 13str 14varchar a true aaaa 2023-12-21T10:02:32.747 10 20 30 40 50 60 70 80 90.1 100.1 110.10 2020-03-02 130str 140varchar b false bbbb 2023-12-21T10:02:37.527 +-- !predict_like_1 -- +1 2 3 4 5 6 7 8 9.1 10.1 11.10 2020-02-02 13str 14varchar a true aaaa 2023-08-13T09:32:38.530 +10 20 30 40 50 60 70 80 90.1 100.1 110.10 2020-03-02 130str 140varchar b false bbbb 2023-08-14T08:32:52.821 + +-- !predict_like_2 -- +1 2 3 4 5 6 7 8 9.1 10.1 11.10 2020-02-02 13str 14varchar a true aaaa 2023-08-13T09:32:38.530 +10 20 30 40 50 60 70 80 90.1 100.1 110.10 2020-03-02 130str 140varchar b false bbbb 2023-08-14T08:32:52.821 + +-- !predict_like_3 -- + +-- !predict_like_4 -- +10 20 30 40 50 60 70 80 90.1 100.1 110.10 2020-03-02 130str 140varchar b false bbbb 2023-08-14T08:32:52.821 + +-- !predict_like_5 -- +10 20 30 40 50 60 70 80 90.1 100.1 110.10 2020-03-02 130str 140varchar b false bbbb 2023-08-14T08:32:52.821 + -- !c1 -- 1 2 3 4 5 6 7 8 9.1 10.1 11.10 2020-02-02 13str 14varchar a true aaaa 2023-08-13T09:32:38.530 diff --git a/regression-test/data/external_table_p2/tvf/test_tvf_p2.out b/regression-test/data/external_table_p2/tvf/test_tvf_p2.out index 6a44b7322dcd07..53b454df858e2a 100644 --- a/regression-test/data/external_table_p2/tvf/test_tvf_p2.out +++ b/regression-test/data/external_table_p2/tvf/test_tvf_p2.out @@ -50,6 +50,18 @@ 32.024 64.0000 128.901468 32.024 64.0000 128.901468 2023-07-07 2023-07-07 2021-07-07T19:15:31.123456 2023-07-07 2023-07-07 2021-07-07T19:15:31.123456 32.689 64.2580 128.745382 32.689 64.2580 128.745382 2023-11-11 2023-11-11 2022-11-11T16:35:37.123456 2023-11-11 2023-11-11 2022-11-11T16:35:37.123456 +-- !wrong_page_header -- +53587 38687 2689589 99480 2971 1999262 218 386 5265 86 33.14 56.33 40.55 0.00 3487.30 2850.04 4844.38 69.74 0.00 3487.30 3557.04 637.26 +53587 47417 2689589 99480 2971 1999262 218 1216 5265 75 36.98 54.36 46.74 736.15 3505.50 2773.50 4077.00 138.46 736.15 2769.35 2907.81 -4.15 +53587 72713 2689589 99480 2971 1999262 218 1250 5265 15 54.97 85.75 75.46 0.00 1131.90 824.55 1286.25 56.59 0.00 1131.90 1188.49 307.35 +\N 124637 2689589 99480 \N \N 218 196 5265 \N 17.56 33.01 \N \N \N \N 3069.93 17.18 \N 245.52 \N \N +\N 132020 2689589 \N 2971 \N 218 \N 5265 \N \N 132.06 \N \N \N 8965.20 12281.58 255.41 \N \N 3448.10 \N +53587 132623 2689589 99480 2971 1999262 218 31 5265 74 62.58 90.74 77.12 0.00 5706.88 4630.92 6714.76 57.06 0.00 5706.88 5763.94 1075.96 +53587 298010 2689589 99480 2971 1999262 218 802 5265 59 83.41 159.31 70.09 0.00 4135.31 4921.19 9399.29 289.47 0.00 4135.31 4424.78 -785.88 +47821 4657 8720303 1194037 2971 4208705 574 656 5960 55 68.94 91.69 82.52 0.00 4538.60 3791.70 5042.95 408.47 0.00 4538.60 4947.07 746.90 +47821 50869 8720303 1194037 2971 4208705 574 1284 5960 77 2.87 2.92 0.37 0.00 28.49 220.99 224.84 0.00 0.00 28.49 28.49 -192.50 +47821 51494 8720303 1194037 2971 4208705 574 145 5960 18 69.14 94.72 14.20 0.00 255.60 1244.52 1704.96 7.66 0.00 255.60 263.26 -988.92 + -- !viewfs -- 25001 25001 25001 diff --git a/regression-test/data/inverted_index_p0/test_index_empty_string.out b/regression-test/data/inverted_index_p0/test_index_empty_string.out new file mode 100644 index 00000000000000..70b6b599194790 --- /dev/null +++ b/regression-test/data/inverted_index_p0/test_index_empty_string.out @@ -0,0 +1,7 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !sql -- +1 + +-- !sql -- +0 + diff --git a/regression-test/suites/correctness/test_in_null.groovy b/regression-test/suites/correctness/test_in_null.groovy index f1e08f38b1be90..04b0c9a4bb2681 100644 --- a/regression-test/suites/correctness/test_in_null.groovy +++ b/regression-test/suites/correctness/test_in_null.groovy @@ -19,7 +19,7 @@ suite("test_in_null") { sql """ DROP TABLE IF EXISTS db """ sql """ CREATE TABLE IF NOT EXISTS db( - `id` INT NULL , + `id` INT NULL ) ENGINE=OLAP DUPLICATE KEY(`id`) DISTRIBUTED BY HASH(`id`) BUCKETS 1 diff --git a/regression-test/suites/correctness_p0/test_many_inlineview.groovy b/regression-test/suites/correctness_p0/test_many_inlineview.groovy index 651b32411e8c97..810144947cbb8b 100644 --- a/regression-test/suites/correctness_p0/test_many_inlineview.groovy +++ b/regression-test/suites/correctness_p0/test_many_inlineview.groovy @@ -56,7 +56,7 @@ suite("test_many_inlineview") { sql """ CREATE TABLE `ods_drp_xpi_storemain` ( `fID` varchar(300) NOT NULL COMMENT 'ID', - `fStoreCode` varchar(150) NULL COMMENT '店铺编码', + `fStoreCode` varchar(150) NULL COMMENT '店铺编码' ) ENGINE=OLAP UNIQUE KEY(`fID`) DISTRIBUTED BY HASH(`fID`) BUCKETS 4 diff --git a/regression-test/suites/correctness_p0/test_null_aware_left_anti_join.groovy b/regression-test/suites/correctness_p0/test_null_aware_left_anti_join.groovy index f732b6bda585ea..6083290b2e5cf0 100644 --- a/regression-test/suites/correctness_p0/test_null_aware_left_anti_join.groovy +++ b/regression-test/suites/correctness_p0/test_null_aware_left_anti_join.groovy @@ -60,11 +60,21 @@ suite("test_null_aware_left_anti_join") { sql """ set parallel_pipeline_task_num=2; """ qt_select """ select ${tableName2}.k1 from ${tableName2} where k1 not in (select ${tableName1}.k1 from ${tableName1}) order by ${tableName2}.k1; """ - sql """ - drop table if exists ${tableName2}; + // In left anti join, if right side is empty, all rows(null included) of left should be output. + qt_anti_emtpy_right """ + select + * + from ${tableName1} t1 where k1 not in ( + select k1 from ${tableName2} t2 where t2.k1 > 2 + ) order by 1; """ - sql """ - drop table if exists ${tableName1}; + // In left semi join, if right side is empty, no row should be output. + qt_semi_emtpy_right """ + select + * + from ${tableName1} t1 where k1 in ( + select k1 from ${tableName2} t2 where t2.k1 > 2 + ) order by 1; """ } diff --git a/regression-test/suites/ddl_p0/test_ctas.groovy b/regression-test/suites/ddl_p0/test_ctas.groovy index 2aaf56ee5931a8..cf0ad869e923b4 100644 --- a/regression-test/suites/ddl_p0/test_ctas.groovy +++ b/regression-test/suites/ddl_p0/test_ctas.groovy @@ -259,7 +259,7 @@ suite("test_ctas") { `l_varchar` varchar(65533) NULL ) ENGINE=OLAP DUPLICATE KEY(`l_varchar`) - COMMENT 'OLAP\' + COMMENT 'OLAP' DISTRIBUTED BY HASH(`l_varchar`) BUCKETS 10 PROPERTIES ( "replication_allocation" = "tag.location.default: 1", diff --git a/regression-test/suites/delete_p0/test_delete.groovy b/regression-test/suites/delete_p0/test_delete.groovy index 9b1b8af64bc0e4..635762a3275880 100644 --- a/regression-test/suites/delete_p0/test_delete.groovy +++ b/regression-test/suites/delete_p0/test_delete.groovy @@ -255,7 +255,7 @@ suite("test_delete") { sql """ CREATE TABLE `dwd_pay` ( `tenant_id` int(11) DEFAULT NULL COMMENT '租户ID', - `pay_time` datetime DEFAULT NULL COMMENT '付款时间', + `pay_time` datetime DEFAULT NULL COMMENT '付款时间' ) ENGINE=OLAP DUPLICATE KEY(`tenant_id`) COMMENT "付款明细" @@ -308,7 +308,7 @@ suite("test_delete") { col_10 datetime, col_11 boolean, col_12 decimalv2(10,3), - col_8 string, + col_8 string ) ENGINE=OLAP duplicate KEY(`col_1`, col_2, col_3, col_4, col_5, col_6, col_7, col_9, col_10, col_11, col_12) COMMENT 'OLAP' diff --git a/regression-test/suites/external_table_p0/paimon/test_paimon_catalog.groovy b/regression-test/suites/external_table_p0/paimon/test_paimon_catalog.groovy index ee74abd7ed887c..c00ef1cf1be6a1 100644 --- a/regression-test/suites/external_table_p0/paimon/test_paimon_catalog.groovy +++ b/regression-test/suites/external_table_p0/paimon/test_paimon_catalog.groovy @@ -56,6 +56,11 @@ suite("test_paimon_catalog", "p0,external,doris,external_docker,external_docker_ if (enabled != null && enabled.equalsIgnoreCase("true")) { def all = """select * from all_table order by c1;""" def all_with_parquet = """select * from all_table_with_parquet order by c1;""" + def predict_like_1 = """select * from all_table where c13 like '%3%' order by c1""" + def predict_like_2 = """select * from all_table where c13 like '13%' order by c1""" + def predict_like_3 = """select * from all_table where c13 like '13' order by c1""" + def predict_like_4 = """select * from all_table where c13 like '130str' order by c1""" + def predict_like_5 = """select * from all_table where c13 like '130str%' order by c1""" def c1 = """select * from all_table where c1=1;""" def c2 = """select * from all_table where c2=2;""" def c3 = """select * from all_table where c3=3;""" @@ -175,6 +180,11 @@ suite("test_paimon_catalog", "p0,external,doris,external_docker,external_docker_ qt_all all qt_all_with_parquet all_with_parquet + qt_predict_like_1 predict_like_1 + qt_predict_like_2 predict_like_2 + qt_predict_like_3 predict_like_3 + qt_predict_like_4 predict_like_4 + qt_predict_like_5 predict_like_5 qt_c1 c1 qt_c2 c2 qt_c3 c3 diff --git a/regression-test/suites/external_table_p0/tvf/queries/test_queries_tvf.groovy b/regression-test/suites/external_table_p0/tvf/queries/test_queries_tvf.groovy index 95cd0d3aacabab..7594b4bd7b5618 100644 --- a/regression-test/suites/external_table_p0/tvf/queries/test_queries_tvf.groovy +++ b/regression-test/suites/external_table_p0/tvf/queries/test_queries_tvf.groovy @@ -22,7 +22,7 @@ suite("test_queries_tvf","p0,external,tvf,external_docker") { CREATE TABLE IF NOT EXISTS ${table_name} ( `user_id` LARGEINT NOT NULL COMMENT "用户id", `name` STRING COMMENT "用户名称", - `age` INT COMMENT "用户年龄", + `age` INT COMMENT "用户年龄" ) DISTRIBUTED BY HASH(user_id) PROPERTIES("replication_num" = "1"); """ diff --git a/regression-test/suites/external_table_p2/tvf/test_tvf_p2.groovy b/regression-test/suites/external_table_p2/tvf/test_tvf_p2.groovy index 39015b7b76f66a..96626af32b7ead 100644 --- a/regression-test/suites/external_table_p2/tvf/test_tvf_p2.groovy +++ b/regression-test/suites/external_table_p2/tvf/test_tvf_p2.groovy @@ -60,6 +60,12 @@ suite("test_tvf_p2", "p2,external,tvf,external_remote,external_remote_tvf") { "format" = "parquet"); """ + // test for wrong page header + qt_wrong_page_header """select * from hdfs( + "uri" = "hdfs://${nameNodeHost}:${hdfsPort}/catalog/tvf/parquet/wrong_page_header.parquet", + "format" = "parquet") order by ss_ticket_number,ss_item_sk limit 10; + """ + // viewfs qt_viewfs """select count(id), count(m1), count(m2) from hdfs( diff --git a/regression-test/suites/inverted_index_p0/test_index_empty_string.groovy b/regression-test/suites/inverted_index_p0/test_index_empty_string.groovy new file mode 100644 index 00000000000000..2cf1d844d2bca0 --- /dev/null +++ b/regression-test/suites/inverted_index_p0/test_index_empty_string.groovy @@ -0,0 +1,53 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + + +suite("test_index_empty_string", "p0"){ + def timeout = 60000 + def delta_time = 1000 + def alter_res = "null" + def useTime = 0 + + def indexTblName = "test_index_empty_string" + + sql "DROP TABLE IF EXISTS ${indexTblName}" + // create 1 replica table + sql """ + CREATE TABLE IF NOT EXISTS ${indexTblName}( + `id` int(11) NOT NULL, + `a` text NULL DEFAULT "", + `b` text NULL DEFAULT "", + INDEX a_idx(`a`) USING INVERTED COMMENT '', + INDEX b_idx(`b`) USING INVERTED PROPERTIES("parser" = "english") COMMENT '' + ) ENGINE=OLAP + DUPLICATE KEY(`id`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`id`) BUCKETS 1 + PROPERTIES( + "replication_allocation" = "tag.location.default: 1" + ); + """ + + sql """ + INSERT INTO $indexTblName VALUES + (1, '', '1'), + (2, '2', ''); + """ + + qt_sql "SELECT count() FROM $indexTblName WHERE a match '';" + qt_sql "SELECT count() FROM $indexTblName WHERE b match '';" +} diff --git a/regression-test/suites/load_p0/broker_load/ddl/dup_tbl_array.sql b/regression-test/suites/load_p0/broker_load/ddl/dup_tbl_array.sql index d9c31be6658ffe..36763c13b84ba9 100644 --- a/regression-test/suites/load_p0/broker_load/ddl/dup_tbl_array.sql +++ b/regression-test/suites/load_p0/broker_load/ddl/dup_tbl_array.sql @@ -33,7 +33,7 @@ CREATE TABLE dup_tbl_array kd13 array NOT NULL DEFAULT "[]", kd14 array NOT NULL DEFAULT "[]", kd15 array NOT NULL DEFAULT "[]", - kd16 array NOT NULL DEFAULT "[]", + kd16 array NOT NULL DEFAULT "[]" ) DUPLICATE KEY(k00) DISTRIBUTED BY HASH(k00) BUCKETS 32 diff --git a/regression-test/suites/load_p0/broker_load/ddl/dup_tbl_basic.sql b/regression-test/suites/load_p0/broker_load/ddl/dup_tbl_basic.sql index 0fa50ada8e42b1..cd0ea4b0e80cc0 100644 --- a/regression-test/suites/load_p0/broker_load/ddl/dup_tbl_basic.sql +++ b/regression-test/suites/load_p0/broker_load/ddl/dup_tbl_basic.sql @@ -48,7 +48,7 @@ CREATE TABLE dup_tbl_basic INDEX idx_ngrambf_k117 (`k17`) USING NGRAM_BF PROPERTIES("gram_size"="3", "bf_size"="256"), INDEX idx_bitmap_k104 (`k02`) USING BITMAP, - INDEX idx_bitmap_k110 (`kd01`) USING BITMAP, + INDEX idx_bitmap_k110 (`kd01`) USING BITMAP ) DUPLICATE KEY(k00) diff --git a/regression-test/suites/load_p0/broker_load/ddl/uniq_tbl_array.sql b/regression-test/suites/load_p0/broker_load/ddl/uniq_tbl_array.sql index 69e166aa523f2f..9e0d48e39d51e8 100644 --- a/regression-test/suites/load_p0/broker_load/ddl/uniq_tbl_array.sql +++ b/regression-test/suites/load_p0/broker_load/ddl/uniq_tbl_array.sql @@ -34,7 +34,7 @@ CREATE TABLE uniq_tbl_array kd14 array NOT NULL DEFAULT "[]", kd15 array NOT NULL DEFAULT "[]", kd16 array NOT NULL DEFAULT "[]", - kd17 array NOT NULL DEFAULT "[]", + kd17 array NOT NULL DEFAULT "[]" ) UNIQUE KEY(k00) DISTRIBUTED BY HASH(k00) BUCKETS 32 diff --git a/regression-test/suites/load_p0/http_stream/test_http_stream_json.groovy b/regression-test/suites/load_p0/http_stream/test_http_stream_json.groovy index 8a4710d0aa6224..a2ae83e0342d8c 100644 --- a/regression-test/suites/load_p0/http_stream/test_http_stream_json.groovy +++ b/regression-test/suites/load_p0/http_stream/test_http_stream_json.groovy @@ -27,7 +27,7 @@ suite("test_http_stream_json", "p0") { CREATE TABLE IF NOT EXISTS ${tableName1} ( id int, city VARCHAR(10), - code int, + code int ) DISTRIBUTED BY HASH(id) BUCKETS 1 PROPERTIES ( @@ -70,7 +70,7 @@ suite("test_http_stream_json", "p0") { CREATE TABLE IF NOT EXISTS ${tableName2} ( id int, city VARCHAR(10), - code int, + code int ) DISTRIBUTED BY HASH(id) BUCKETS 1 PROPERTIES ( @@ -113,7 +113,7 @@ suite("test_http_stream_json", "p0") { CREATE TABLE IF NOT EXISTS ${tableName3} ( id int, city VARCHAR(10), - code int, + code int ) DISTRIBUTED BY HASH(id) BUCKETS 1 PROPERTIES ( @@ -157,7 +157,7 @@ suite("test_http_stream_json", "p0") { CREATE TABLE IF NOT EXISTS ${tableName4} ( id int, city VARCHAR(10), - code int, + code int ) DISTRIBUTED BY HASH(id) BUCKETS 1 PROPERTIES ( diff --git a/regression-test/suites/load_p0/routine_load/ddl/dup_tbl_array_create.sql b/regression-test/suites/load_p0/routine_load/ddl/dup_tbl_array_create.sql index b0c3a2bd7f3424..4c86f6b7c33eb1 100644 --- a/regression-test/suites/load_p0/routine_load/ddl/dup_tbl_array_create.sql +++ b/regression-test/suites/load_p0/routine_load/ddl/dup_tbl_array_create.sql @@ -33,7 +33,7 @@ CREATE TABLE routine_load_dup_tbl_array kd13 array NOT NULL DEFAULT "[]", kd14 array NOT NULL DEFAULT "[]", kd15 array NOT NULL DEFAULT "[]", - kd16 array NOT NULL DEFAULT "[]", + kd16 array NOT NULL DEFAULT "[]" ) DUPLICATE KEY(k00) DISTRIBUTED BY HASH(k00) BUCKETS 32 diff --git a/regression-test/suites/load_p0/routine_load/ddl/dup_tbl_basic_create.sql b/regression-test/suites/load_p0/routine_load/ddl/dup_tbl_basic_create.sql index 85fa3790184754..561f3e66b1dce9 100644 --- a/regression-test/suites/load_p0/routine_load/ddl/dup_tbl_basic_create.sql +++ b/regression-test/suites/load_p0/routine_load/ddl/dup_tbl_basic_create.sql @@ -48,7 +48,7 @@ CREATE TABLE routine_load_dup_tbl_basic INDEX idx_ngrambf_k117 (`k17`) USING NGRAM_BF PROPERTIES("gram_size"="3", "bf_size"="256"), INDEX idx_bitmap_k104 (`k02`) USING BITMAP, - INDEX idx_bitmap_k110 (`kd01`) USING BITMAP, + INDEX idx_bitmap_k110 (`kd01`) USING BITMAP ) DUPLICATE KEY(k00) diff --git a/regression-test/suites/load_p0/routine_load/ddl/uniq_tbl_array_create.sql b/regression-test/suites/load_p0/routine_load/ddl/uniq_tbl_array_create.sql index c5b4f07ff87001..ea4a4b39eab592 100644 --- a/regression-test/suites/load_p0/routine_load/ddl/uniq_tbl_array_create.sql +++ b/regression-test/suites/load_p0/routine_load/ddl/uniq_tbl_array_create.sql @@ -34,7 +34,7 @@ CREATE TABLE routine_load_uniq_tbl_array kd14 array NOT NULL DEFAULT "[]", kd15 array NOT NULL DEFAULT "[]", kd16 array NOT NULL DEFAULT "[]", - kd17 array NOT NULL DEFAULT "[]", + kd17 array NOT NULL DEFAULT "[]" ) UNIQUE KEY(k00) DISTRIBUTED BY HASH(k00) BUCKETS 32 diff --git a/regression-test/suites/load_p0/stream_load/ddl/dup_tbl_array_create.sql b/regression-test/suites/load_p0/stream_load/ddl/dup_tbl_array_create.sql index 0277e3368cd216..61c471480d68e2 100644 --- a/regression-test/suites/load_p0/stream_load/ddl/dup_tbl_array_create.sql +++ b/regression-test/suites/load_p0/stream_load/ddl/dup_tbl_array_create.sql @@ -33,7 +33,7 @@ CREATE TABLE stream_load_dup_tbl_array kd13 array NOT NULL DEFAULT "[]", kd14 array NOT NULL DEFAULT "[]", kd15 array NOT NULL DEFAULT "[]", - kd16 array NOT NULL DEFAULT "[]", + kd16 array NOT NULL DEFAULT "[]" ) DUPLICATE KEY(k00) DISTRIBUTED BY HASH(k00) BUCKETS 32 diff --git a/regression-test/suites/load_p0/stream_load/ddl/dup_tbl_basic_create.sql b/regression-test/suites/load_p0/stream_load/ddl/dup_tbl_basic_create.sql index 57c5c468c838a1..f81ae5e20ea639 100644 --- a/regression-test/suites/load_p0/stream_load/ddl/dup_tbl_basic_create.sql +++ b/regression-test/suites/load_p0/stream_load/ddl/dup_tbl_basic_create.sql @@ -48,7 +48,7 @@ CREATE TABLE stream_load_dup_tbl_basic INDEX idx_ngrambf_k117 (`k17`) USING NGRAM_BF PROPERTIES("gram_size"="3", "bf_size"="256"), INDEX idx_bitmap_k104 (`k02`) USING BITMAP, - INDEX idx_bitmap_k110 (`kd01`) USING BITMAP, + INDEX idx_bitmap_k110 (`kd01`) USING BITMAP ) DUPLICATE KEY(k00) diff --git a/regression-test/suites/load_p0/stream_load/ddl/dup_tbl_basic_create_random_bucket.sql b/regression-test/suites/load_p0/stream_load/ddl/dup_tbl_basic_create_random_bucket.sql index cbd0fa7e71929a..9c86eb64745a6a 100644 --- a/regression-test/suites/load_p0/stream_load/ddl/dup_tbl_basic_create_random_bucket.sql +++ b/regression-test/suites/load_p0/stream_load/ddl/dup_tbl_basic_create_random_bucket.sql @@ -48,7 +48,7 @@ CREATE TABLE stream_load_dup_tbl_basic_random_bucket INDEX idx_ngrambf_k117 (`k17`) USING NGRAM_BF PROPERTIES("gram_size"="3", "bf_size"="256"), INDEX idx_bitmap_k104 (`k02`) USING BITMAP, - INDEX idx_bitmap_k110 (`kd01`) USING BITMAP, + INDEX idx_bitmap_k110 (`kd01`) USING BITMAP ) DUPLICATE KEY(k00) diff --git a/regression-test/suites/load_p0/stream_load/ddl/uniq_tbl_array_create.sql b/regression-test/suites/load_p0/stream_load/ddl/uniq_tbl_array_create.sql index c6ba1b69496409..41291df9948199 100644 --- a/regression-test/suites/load_p0/stream_load/ddl/uniq_tbl_array_create.sql +++ b/regression-test/suites/load_p0/stream_load/ddl/uniq_tbl_array_create.sql @@ -34,7 +34,7 @@ CREATE TABLE stream_load_uniq_tbl_array kd14 array NOT NULL DEFAULT "[]", kd15 array NOT NULL DEFAULT "[]", kd16 array NOT NULL DEFAULT "[]", - kd17 array NOT NULL DEFAULT "[]", + kd17 array NOT NULL DEFAULT "[]" ) UNIQUE KEY(k00) DISTRIBUTED BY HASH(k00) BUCKETS 32 diff --git a/regression-test/suites/load_p0/stream_load/test_group_commit_wal_limit.groovy b/regression-test/suites/load_p0/stream_load/test_group_commit_wal_limit.groovy index 68ccc3f853832a..24b66de04b86c6 100644 --- a/regression-test/suites/load_p0/stream_load/test_group_commit_wal_limit.groovy +++ b/regression-test/suites/load_p0/stream_load/test_group_commit_wal_limit.groovy @@ -71,7 +71,7 @@ suite("test_group_commit_wal_limit") { assertEquals(code, 0) out = process.text logger.info("out is " + out ) - assertTrue(out.contains('[INTERNAL_ERROR]Stream load size too large')) + assertTrue(out.contains('Stream load size too large')) // too lagre data case 1TB strBuilder = new StringBuilder() @@ -89,7 +89,7 @@ suite("test_group_commit_wal_limit") { assertEquals(code, 0) out = process.text logger.info("out is " + out ) - assertTrue(out.contains('[INTERNAL_ERROR]Stream load size too large')) + assertTrue(out.contains('Stream load size too large')) // httpload // normal case @@ -126,7 +126,7 @@ suite("test_group_commit_wal_limit") { assertEquals(code, 0) out = process.text logger.info("out is " + out ) - assertTrue(out.contains('[INTERNAL_ERROR]Http load size too large')) + assertTrue(out.contains('Http load size too large')) // too lagre data case 1TB strBuilder = new StringBuilder() @@ -144,5 +144,5 @@ suite("test_group_commit_wal_limit") { assertEquals(code, 0) out = process.text logger.info("out is " + out ) - assertTrue(out.contains('[INTERNAL_ERROR]Http load size too large')) + assertTrue(out.contains('Http load size too large')) } diff --git a/regression-test/suites/load_p0/stream_load/test_stream_load.groovy b/regression-test/suites/load_p0/stream_load/test_stream_load.groovy index 4e809ff2a5d4ff..d6b50fcae2ce42 100644 --- a/regression-test/suites/load_p0/stream_load/test_stream_load.groovy +++ b/regression-test/suites/load_p0/stream_load/test_stream_load.groovy @@ -1543,7 +1543,7 @@ suite("test_stream_load", "p0") { `c47` varchar(48) NULL, `c48` varchar(48) NULL, `c49` varchar(48) NULL, - `c50` varchar(48) NULL, + `c50` varchar(48) NULL ) ENGINE=OLAP DUPLICATE KEY(`c1`) COMMENT 'OLAP' diff --git a/regression-test/suites/load_p0/tvf/ddl/dup_tbl_array_tvf.sql b/regression-test/suites/load_p0/tvf/ddl/dup_tbl_array_tvf.sql index 4ddbba40631c0e..b6bc8aa4371eca 100644 --- a/regression-test/suites/load_p0/tvf/ddl/dup_tbl_array_tvf.sql +++ b/regression-test/suites/load_p0/tvf/ddl/dup_tbl_array_tvf.sql @@ -33,7 +33,7 @@ CREATE TABLE dup_tbl_array_tvf kd13 array NOT NULL DEFAULT "[]", kd14 array NOT NULL DEFAULT "[]", kd15 array NOT NULL DEFAULT "[]", - kd16 array NOT NULL DEFAULT "[]", + kd16 array NOT NULL DEFAULT "[]" ) DUPLICATE KEY(k00) DISTRIBUTED BY HASH(k00) BUCKETS 32 diff --git a/regression-test/suites/load_p0/tvf/ddl/dup_tbl_basic_tvf.sql b/regression-test/suites/load_p0/tvf/ddl/dup_tbl_basic_tvf.sql index 573807576c2593..95950d3b38858f 100644 --- a/regression-test/suites/load_p0/tvf/ddl/dup_tbl_basic_tvf.sql +++ b/regression-test/suites/load_p0/tvf/ddl/dup_tbl_basic_tvf.sql @@ -48,7 +48,7 @@ CREATE TABLE dup_tbl_basic_tvf INDEX idx_ngrambf_k117 (`k17`) USING NGRAM_BF PROPERTIES("gram_size"="3", "bf_size"="256"), INDEX idx_bitmap_k104 (`k02`) USING BITMAP, - INDEX idx_bitmap_k110 (`kd01`) USING BITMAP, + INDEX idx_bitmap_k110 (`kd01`) USING BITMAP ) DUPLICATE KEY(k00) diff --git a/regression-test/suites/load_p0/tvf/ddl/uniq_tbl_array_tvf.sql b/regression-test/suites/load_p0/tvf/ddl/uniq_tbl_array_tvf.sql index b38a1b0a3d38be..1cb4d3382073d3 100644 --- a/regression-test/suites/load_p0/tvf/ddl/uniq_tbl_array_tvf.sql +++ b/regression-test/suites/load_p0/tvf/ddl/uniq_tbl_array_tvf.sql @@ -34,7 +34,7 @@ CREATE TABLE uniq_tbl_array_tvf kd14 array NOT NULL DEFAULT "[]", kd15 array NOT NULL DEFAULT "[]", kd16 array NOT NULL DEFAULT "[]", - kd17 array NOT NULL DEFAULT "[]", + kd17 array NOT NULL DEFAULT "[]" ) UNIQUE KEY(k00) DISTRIBUTED BY HASH(k00) BUCKETS 32 diff --git a/regression-test/suites/mtmv_p0/test_build_mtmv.groovy b/regression-test/suites/mtmv_p0/test_build_mtmv.groovy index 882a7eff22e132..eb4560f8b06de7 100644 --- a/regression-test/suites/mtmv_p0/test_build_mtmv.groovy +++ b/regression-test/suites/mtmv_p0/test_build_mtmv.groovy @@ -152,6 +152,19 @@ suite("test_build_mtmv") { DROP MATERIALIZED VIEW ${mvName} """ + // use default value + sql """ + CREATE MATERIALIZED VIEW ${mvName} + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS + SELECT ${tableName}.username, ${tableNamePv}.pv FROM ${tableName}, ${tableNamePv} WHERE ${tableName}.id=${tableNamePv}.id; + """ + + sql """ + DROP MATERIALIZED VIEW ${mvName} + """ + // IMMEDIATE schedule interval sql """ CREATE MATERIALIZED VIEW ${mvName} diff --git a/regression-test/suites/nereids_rules_p0/mv/agg_with_roll_up/aggregate_with_roll_up.groovy b/regression-test/suites/nereids_rules_p0/mv/agg_with_roll_up/aggregate_with_roll_up.groovy index e67086deba0668..b56922d5db6470 100644 --- a/regression-test/suites/nereids_rules_p0/mv/agg_with_roll_up/aggregate_with_roll_up.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/agg_with_roll_up/aggregate_with_roll_up.groovy @@ -42,7 +42,11 @@ suite("aggregate_with_roll_up") { O_COMMENT VARCHAR(79) NOT NULL ) DUPLICATE KEY(o_orderkey, o_custkey) - PARTITION BY RANGE(o_orderdate) (PARTITION `day_2` VALUES LESS THAN ('2023-12-30')) + PARTITION BY RANGE(o_orderdate) ( + PARTITION `day_2` VALUES LESS THAN ('2023-12-9'), + PARTITION `day_3` VALUES LESS THAN ("2023-12-11"), + PARTITION `day_4` VALUES LESS THAN ("2023-12-30") + ) DISTRIBUTED BY HASH(o_orderkey) BUCKETS 3 PROPERTIES ( "replication_num" = "1" @@ -73,7 +77,10 @@ suite("aggregate_with_roll_up") { l_comment VARCHAR(44) NOT NULL ) DUPLICATE KEY(l_orderkey, l_partkey, l_suppkey, l_linenumber) - PARTITION BY RANGE(l_shipdate) (PARTITION `day_1` VALUES LESS THAN ('2023-12-30')) + PARTITION BY RANGE(l_shipdate) ( + PARTITION `day_1` VALUES LESS THAN ('2023-12-9'), + PARTITION `day_2` VALUES LESS THAN ("2023-12-11"), + PARTITION `day_3` VALUES LESS THAN ("2023-12-30")) DISTRIBUTED BY HASH(l_orderkey) BUCKETS 3 PROPERTIES ( "replication_num" = "1" @@ -144,6 +151,26 @@ suite("aggregate_with_roll_up") { } } + def check_rewrite_with_mv_partition = { mv_sql, query_sql, mv_name, partition_column -> + + sql """DROP MATERIALIZED VIEW IF EXISTS ${mv_name}""" + sql""" + CREATE MATERIALIZED VIEW ${mv_name} + BUILD IMMEDIATE REFRESH COMPLETE ON MANUAL + PARTITION BY (${partition_column}) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS ${mv_sql} + """ + + def job_name = getJobName(db, mv_name); + waitingMTMVTaskFinished(job_name) + explain { + sql("${query_sql}") + contains "(${mv_name})" + } + } + def check_rewrite_with_force_analyze = { mv_sql, query_sql, mv_name -> sql """DROP MATERIALIZED VIEW IF EXISTS ${mv_name}""" @@ -283,7 +310,7 @@ suite("aggregate_with_roll_up") { "l_partkey, " + "l_suppkey" order_qt_query15_0_before "${query15_0}" - check_rewrite(mv15_0, query15_0, "mv15_0") + check_rewrite_with_mv_partition(mv15_0, query15_0, "mv15_0", "l_shipdate") order_qt_query15_0_after "${query15_0}" sql """ DROP MATERIALIZED VIEW IF EXISTS mv15_0""" diff --git a/regression-test/suites/nereids_rules_p0/mv/agg_without_roll_up/aggregate_without_roll_up.groovy b/regression-test/suites/nereids_rules_p0/mv/agg_without_roll_up/aggregate_without_roll_up.groovy index b76763752fbd36..72e80eadbf93f0 100644 --- a/regression-test/suites/nereids_rules_p0/mv/agg_without_roll_up/aggregate_without_roll_up.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/agg_without_roll_up/aggregate_without_roll_up.groovy @@ -43,11 +43,15 @@ suite("aggregate_without_roll_up") { o_comment VARCHAR(79) NOT NULL ) DUPLICATE KEY(o_orderkey, o_custkey) - PARTITION BY RANGE(o_orderdate) (PARTITION `day_2` VALUES LESS THAN ('2023-12-30')) + PARTITION BY RANGE(o_orderdate) ( + PARTITION `day_2` VALUES LESS THAN ('2023-12-9'), + PARTITION `day_3` VALUES LESS THAN ("2023-12-11"), + PARTITION `day_4` VALUES LESS THAN ("2023-12-30") + ) DISTRIBUTED BY HASH(o_orderkey) BUCKETS 3 PROPERTIES ( "replication_num" = "1" - ) + ); """ sql """ @@ -74,7 +78,11 @@ suite("aggregate_without_roll_up") { l_comment VARCHAR(44) NOT NULL ) DUPLICATE KEY(l_orderkey, l_partkey, l_suppkey, l_linenumber) - PARTITION BY RANGE(l_shipdate) (PARTITION `day_1` VALUES LESS THAN ('2023-12-30')) + PARTITION BY RANGE(l_shipdate) ( + PARTITION `day_2` VALUES LESS THAN ('2023-12-9'), + PARTITION `day_3` VALUES LESS THAN ("2023-12-11"), + PARTITION `day_4` VALUES LESS THAN ("2023-12-30") + ) DISTRIBUTED BY HASH(l_orderkey) BUCKETS 3 PROPERTIES ( "replication_num" = "1" @@ -100,7 +108,8 @@ suite("aggregate_without_roll_up") { ) """ - sql """ insert into lineitem values + sql """ + insert into lineitem values (1, 2, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-08', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), (2, 4, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-09', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), (3, 2, 4, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-10', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), diff --git a/regression-test/suites/nereids_rules_p0/mv/join/inner/inner_join.groovy b/regression-test/suites/nereids_rules_p0/mv/join/inner/inner_join.groovy index d8025e0bc60b14..0dcf33a9d87eb2 100644 --- a/regression-test/suites/nereids_rules_p0/mv/join/inner/inner_join.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/join/inner/inner_join.groovy @@ -393,7 +393,7 @@ suite("inner_join") { sql """ DROP MATERIALIZED VIEW IF EXISTS mv6_0""" - // filter inside + inner + right + // filter inside + left + right def mv7_0 = "select l_shipdate, o_orderdate, l_partkey, l_suppkey " + "from lineitem " + "inner join (select * from orders where o_orderdate = '2023-12-08') t2 " + diff --git a/regression-test/suites/nereids_rules_p0/partition_prune/test_date_function_prune.groovy b/regression-test/suites/nereids_rules_p0/partition_prune/test_date_function_prune.groovy index 583bb48e83774e..965983596a00cc 100644 --- a/regression-test/suites/nereids_rules_p0/partition_prune/test_date_function_prune.groovy +++ b/regression-test/suites/nereids_rules_p0/partition_prune/test_date_function_prune.groovy @@ -25,7 +25,7 @@ suite("test_date_function_prune") { sql """ CREATE TABLE `dp` ( `node_name` varchar(100) NULL COMMENT '', - `date_time` datetime NULL COMMENT '', + `date_time` datetime NULL COMMENT '' ) ENGINE=OLAP DUPLICATE KEY(`node_name`) COMMENT '' diff --git a/regression-test/suites/partition_p0/auto_partition/test_auto_list_partition.groovy b/regression-test/suites/partition_p0/auto_partition/test_auto_list_partition.groovy index 76fd866d66fcc7..02a16da8884369 100644 --- a/regression-test/suites/partition_p0/auto_partition/test_auto_list_partition.groovy +++ b/regression-test/suites/partition_p0/auto_partition/test_auto_list_partition.groovy @@ -255,7 +255,7 @@ suite("test_auto_list_partition") { CREATE TABLE test_list_many_column2 ( id int not null, k largeint not null, - str varchar not null, + str varchar not null ) AUTO PARTITION BY LIST (`id`, `k`, `str`) ( @@ -276,7 +276,7 @@ suite("test_auto_list_partition") { CREATE TABLE test_list_many_column3 ( id int not null, k largeint not null, - str varchar not null, + str varchar not null ) AUTO PARTITION BY LIST (`id`, `k`, `str`) ( diff --git a/regression-test/suites/point_query_p0/test_rowstore.groovy b/regression-test/suites/point_query_p0/test_rowstore.groovy index e12849310457d7..c7c2b7636377e8 100644 --- a/regression-test/suites/point_query_p0/test_rowstore.groovy +++ b/regression-test/suites/point_query_p0/test_rowstore.groovy @@ -21,7 +21,7 @@ suite("test_rowstore", "p0") { sql """ CREATE TABLE IF NOT EXISTS ${tableName} ( `k1` int(11) NULL COMMENT "", - `k2` text NULL COMMENT "", + `k2` text NULL COMMENT "" ) ENGINE=OLAP UNIQUE KEY(`k1`) DISTRIBUTED BY HASH(`k1`) BUCKETS 1 diff --git a/regression-test/suites/query_p0/aggregate/array_agg.groovy b/regression-test/suites/query_p0/aggregate/array_agg.groovy index 1463eed6470cb8..ee4fcb9d455ad0 100644 --- a/regression-test/suites/query_p0/aggregate/array_agg.groovy +++ b/regression-test/suites/query_p0/aggregate/array_agg.groovy @@ -23,7 +23,7 @@ suite("array_agg") { CREATE TABLE `test_array_agg` ( `id` int(11) NOT NULL, `label_name` varchar(32) default null, - `value_field` string default null, + `value_field` string default null ) ENGINE=OLAP DUPLICATE KEY(`id`) COMMENT 'OLAP' diff --git a/regression-test/suites/query_p0/limit/test_map_select_with_limit.groovy b/regression-test/suites/query_p0/limit/test_map_select_with_limit.groovy index 43d340fce995eb..5a37b7b9bc1888 100644 --- a/regression-test/suites/query_p0/limit/test_map_select_with_limit.groovy +++ b/regression-test/suites/query_p0/limit/test_map_select_with_limit.groovy @@ -22,7 +22,7 @@ suite("test_map_select_with_limit", "query") { sql """ CREATE TABLE IF NOT EXISTS ${testTable} ( `k1` INT(11) NULL, - `k2` MAP NULL, + `k2` MAP NULL ) ENGINE=OLAP DUPLICATE KEY(`k1`) COMMENT 'OLAP' diff --git a/regression-test/suites/query_p0/sort/sort.groovy b/regression-test/suites/query_p0/sort/sort.groovy index a7f096b330ea05..7742bdcd7664a3 100644 --- a/regression-test/suites/query_p0/sort/sort.groovy +++ b/regression-test/suites/query_p0/sort/sort.groovy @@ -169,7 +169,7 @@ suite("sort") { `col_varchar` VARCHAR(10) NOT NULL, `col_varchar_null` VARCHAR(10) NULL, `col_char` CHAR(10) NOT NULL, - `col_char_null` CHAR(10) NULL, + `col_char_null` CHAR(10) NULL ) ENGINE=OLAP DUPLICATE KEY(`row_id`) DISTRIBUTED BY HASH(`row_id`) BUCKETS 1 diff --git a/regression-test/suites/query_p0/test_array_orderby_limit.groovy b/regression-test/suites/query_p0/test_array_orderby_limit.groovy index b63d936dfc699e..c573b1c7986a72 100644 --- a/regression-test/suites/query_p0/test_array_orderby_limit.groovy +++ b/regression-test/suites/query_p0/test_array_orderby_limit.groovy @@ -22,7 +22,7 @@ suite("test_array_char_orderby", "query") { sql """ CREATE TABLE IF NOT EXISTS ${testTable} ( `k1` INT(11) NULL, - `k2` array> NULL, + `k2` array> NULL ) ENGINE=OLAP DUPLICATE KEY(`k1`) COMMENT 'OLAP' diff --git a/regression-test/suites/show_p0/test_show_create_table_and_views.groovy b/regression-test/suites/show_p0/test_show_create_table_and_views.groovy index 89670aa434f680..ff98d09f9f4ca7 100644 --- a/regression-test/suites/show_p0/test_show_create_table_and_views.groovy +++ b/regression-test/suites/show_p0/test_show_create_table_and_views.groovy @@ -36,7 +36,7 @@ suite("test_show_create_table_and_views", "show") { CREATE TABLE ${dbName}.${tableName} ( `user_id` LARGEINT NOT NULL, `good_id` LARGEINT NOT NULL, - `cost` BIGINT SUM DEFAULT "0", + `cost` BIGINT SUM DEFAULT "0" ) AGGREGATE KEY(`user_id`, `good_id`) PARTITION BY RANGE(`good_id`) diff --git a/regression-test/suites/unique_with_mow_p0/partial_update/test_partial_update_native_insert_stmt_complex.groovy b/regression-test/suites/unique_with_mow_p0/partial_update/test_partial_update_native_insert_stmt_complex.groovy index 62dfed2fa70f63..ad143a04be3c94 100644 --- a/regression-test/suites/unique_with_mow_p0/partial_update/test_partial_update_native_insert_stmt_complex.groovy +++ b/regression-test/suites/unique_with_mow_p0/partial_update/test_partial_update_native_insert_stmt_complex.groovy @@ -72,6 +72,8 @@ suite("test_partial_update_native_insert_stmt_complex", "p0") { (5, 5, '5', 5.0, '2000-01-05');""" sql """insert into ${tbName3} values(1), (3), (5);""" + sql "sync;" + qt_tbl1 "select * from ${tbName1} order by id;" qt_tbl2 "select * from ${tbName2} order by id;" qt_tbl3 "select * from ${tbName3} order by id;" @@ -111,6 +113,8 @@ suite("test_partial_update_native_insert_stmt_complex", "p0") { (5, 5, '5', 5.0, '2000-01-05');""" sql """insert into ${tbName3} values(1), (3), (5);""" + sql "sync;" + qt_select_result "select ${tbName2}.id,1 from ${tbName2} inner join ${tbName3} on ${tbName2}.id = ${tbName3}.id order by ${tbName2}.id;" sql "set enable_unique_key_partial_update=true;"