diff --git a/.asf.yaml b/.asf.yaml index 9c192b25cd3eb02..ba63dcdcc09eecb 100644 --- a/.asf.yaml +++ b/.asf.yaml @@ -110,7 +110,6 @@ github: - shuke987 - wm1581066 - KassieZ - - gavinchou - yujun777 notifications: diff --git a/be/CMakeLists.txt b/be/CMakeLists.txt index fd1f2c1db4a4cd0..28b3fee115aab09 100644 --- a/be/CMakeLists.txt +++ b/be/CMakeLists.txt @@ -72,7 +72,6 @@ option(USE_LIBCPP "Use libc++" OFF) option(USE_MEM_TRACKER, "Use memory tracker" ON) option(USE_UNWIND "Use libunwind" ON) option(USE_JEMALLOC "Use jemalloc" ON) -option(USE_JEMALLOC_HOOK "Use jemalloc hook" ON) if (OS_MACOSX) set(GLIBC_COMPATIBILITY OFF) set(USE_LIBCPP ON) @@ -88,7 +87,6 @@ message(STATUS "GLIBC_COMPATIBILITY is ${GLIBC_COMPATIBILITY}") message(STATUS "USE_LIBCPP is ${USE_LIBCPP}") message(STATUS "USE_MEM_TRACKER is ${USE_MEM_TRACKER}") message(STATUS "USE_JEMALLOC is ${USE_JEMALLOC}") -message(STATUS "USE_JEMALLOC_HOOK is ${USE_JEMALLOC_HOOK}") message(STATUS "USE_UNWIND is ${USE_UNWIND}") message(STATUS "ENABLE_PCH is ${ENABLE_PCH}") @@ -348,9 +346,6 @@ endif() if (USE_JEMALLOC) add_definitions(-DUSE_JEMALLOC) endif() -if (USE_JEMALLOC_HOOK) - add_definitions(-DUSE_JEMALLOC_HOOK) -endif() # Compile with libunwind if (USE_UNWIND) diff --git a/be/src/agent/task_worker_pool.cpp b/be/src/agent/task_worker_pool.cpp index 10590a7faeae163..3b8d63f7a95dbe5 100644 --- a/be/src/agent/task_worker_pool.cpp +++ b/be/src/agent/task_worker_pool.cpp @@ -1365,6 +1365,7 @@ void update_s3_resource(const TStorageResource& param, io::RemoteFileSystemSPtr .region = param.s3_storage_param.region, .ak = param.s3_storage_param.ak, .sk = param.s3_storage_param.sk, + .token = param.s3_storage_param.token, .max_connections = param.s3_storage_param.max_conn, .request_timeout_ms = param.s3_storage_param.request_timeout_ms, .connect_timeout_ms = param.s3_storage_param.conn_timeout_ms, @@ -1384,6 +1385,7 @@ void update_s3_resource(const TStorageResource& param, io::RemoteFileSystemSPtr S3ClientConf conf { .ak = param.s3_storage_param.ak, .sk = param.s3_storage_param.sk, + .token = param.s3_storage_param.token, }; st = client->reset(conf); fs = std::move(existed_fs); diff --git a/be/src/common/config.cpp b/be/src/common/config.cpp index de1458c240dff32..0c257e48a8f4520 100644 --- a/be/src/common/config.cpp +++ b/be/src/common/config.cpp @@ -662,6 +662,9 @@ DEFINE_mInt64(storage_flood_stage_left_capacity_bytes, "1073741824"); // 1GB DEFINE_Int32(flush_thread_num_per_store, "6"); // number of thread for flushing memtable per store, for high priority load task DEFINE_Int32(high_priority_flush_thread_num_per_store, "6"); +// number of threads = min(flush_thread_num_per_store * num_store, +// max_flush_thread_num_per_cpu * num_cpu) +DEFINE_Int32(max_flush_thread_num_per_cpu, "4"); // config for tablet meta checkpoint DEFINE_mInt32(tablet_meta_checkpoint_min_new_rowsets_num, "10"); @@ -978,6 +981,7 @@ DEFINE_Bool(clear_file_cache, "false"); DEFINE_Bool(enable_file_cache_query_limit, "false"); DEFINE_mInt32(file_cache_enter_disk_resource_limit_mode_percent, "90"); DEFINE_mInt32(file_cache_exit_disk_resource_limit_mode_percent, "80"); +DEFINE_mBool(enable_read_cache_file_directly, "false"); DEFINE_mInt32(index_cache_entry_stay_time_after_lookup_s, "1800"); DEFINE_mInt32(inverted_index_cache_stale_sweep_time_sec, "600"); @@ -1211,11 +1215,19 @@ DEFINE_mInt32(thrift_client_open_num_tries, "1"); DEFINE_Bool(enable_index_compaction, "false"); +// http scheme in S3Client to use. E.g. http or https +DEFINE_String(s3_client_http_scheme, "http"); +DEFINE_Validator(s3_client_http_scheme, [](const std::string& config) -> bool { + return config == "http" || config == "https"; +}); + // enable injection point in regression-test DEFINE_mBool(enable_injection_point, "false"); DEFINE_mBool(ignore_schema_change_check, "false"); +DEFINE_mInt64(string_overflow_size, "4294967295"); // std::numic_limits::max() + // The min thread num for BufferedReaderPrefetchThreadPool DEFINE_Int64(num_buffered_reader_prefetch_thread_pool_min_thread, "16"); // The max thread num for BufferedReaderPrefetchThreadPool @@ -1224,6 +1236,8 @@ DEFINE_Int64(num_buffered_reader_prefetch_thread_pool_max_thread, "64"); DEFINE_Int64(num_s3_file_upload_thread_pool_min_thread, "16"); // The max thread num for S3FileUploadThreadPool DEFINE_Int64(num_s3_file_upload_thread_pool_max_thread, "64"); +// The max ratio for ttl cache's size +DEFINE_mInt64(max_ttl_cache_ratio, "90"); // clang-format off #ifdef BE_TEST @@ -1663,6 +1677,8 @@ Status set_fuzzy_configs() { ((distribution(*generator) % 2) == 0) ? "true" : "false"; fuzzy_field_and_value["enable_shrink_memory"] = ((distribution(*generator) % 2) == 0) ? "true" : "false"; + fuzzy_field_and_value["string_overflow_size"] = + ((distribution(*generator) % 2) == 0) ? "10" : "4294967295"; fmt::memory_buffer buf; for (auto& it : fuzzy_field_and_value) { diff --git a/be/src/common/config.h b/be/src/common/config.h index 4139d76b6bcb7a2..e1ec93ff63f7688 100644 --- a/be/src/common/config.h +++ b/be/src/common/config.h @@ -715,6 +715,9 @@ DECLARE_mInt64(storage_flood_stage_left_capacity_bytes); // 1GB DECLARE_Int32(flush_thread_num_per_store); // number of thread for flushing memtable per store, for high priority load task DECLARE_Int32(high_priority_flush_thread_num_per_store); +// number of threads = min(flush_thread_num_per_store * num_store, +// max_flush_thread_num_per_cpu * num_cpu) +DECLARE_Int32(max_flush_thread_num_per_cpu); // config for tablet meta checkpoint DECLARE_mInt32(tablet_meta_checkpoint_min_new_rowsets_num); @@ -1022,6 +1025,7 @@ DECLARE_Bool(clear_file_cache); DECLARE_Bool(enable_file_cache_query_limit); DECLARE_Int32(file_cache_enter_disk_resource_limit_mode_percent); DECLARE_Int32(file_cache_exit_disk_resource_limit_mode_percent); +DECLARE_mBool(enable_read_cache_file_directly); // inverted index searcher cache // cache entry stay time after lookup @@ -1290,11 +1294,17 @@ DECLARE_mInt64(hive_sink_max_file_size); // Retry the Open num_retries time waiting 100 milliseconds between retries. DECLARE_mInt32(thrift_client_open_num_tries); +// http scheme in S3Client to use. E.g. http or https +DECLARE_String(s3_client_http_scheme); + // enable injection point in regression-test DECLARE_mBool(enable_injection_point); DECLARE_mBool(ignore_schema_change_check); +/** Only use in fuzzy test **/ +DECLARE_mInt64(string_overflow_size); + // The min thread num for BufferedReaderPrefetchThreadPool DECLARE_Int64(num_buffered_reader_prefetch_thread_pool_min_thread); // The max thread num for BufferedReaderPrefetchThreadPool @@ -1303,6 +1313,8 @@ DECLARE_Int64(num_buffered_reader_prefetch_thread_pool_max_thread); DECLARE_Int64(num_s3_file_upload_thread_pool_min_thread); // The max thread num for S3FileUploadThreadPool DECLARE_Int64(num_s3_file_upload_thread_pool_max_thread); +// The max ratio for ttl cache's size +DECLARE_mInt64(max_ttl_cache_ratio); #ifdef BE_TEST // test s3 diff --git a/be/src/http/action/compaction_action.cpp b/be/src/http/action/compaction_action.cpp index f61edf92fc18042..43ad940db5e2c2c 100644 --- a/be/src/http/action/compaction_action.cpp +++ b/be/src/http/action/compaction_action.cpp @@ -193,6 +193,20 @@ Status CompactionAction::_handle_run_status_compaction(HttpRequest* req, std::st std::string compaction_type; bool run_status = false; + { + // Full compaction holds both base compaction lock and cumu compaction lock. + // So we can not judge if full compaction is running by check these two locks holding. + // Here, we use a variable 'is_full_compaction_running' to check if full compaction is running. + if (tablet->is_full_compaction_running()) { + msg = "compaction task for this tablet is running"; + compaction_type = "full"; + run_status = true; + *json_result = strings::Substitute(json_template, run_status, msg, tablet_id, + compaction_type); + return Status::OK(); + } + } + { // use try lock to check this tablet is running cumulative compaction std::unique_lock lock_cumulative(tablet->get_cumulative_compaction_lock(), diff --git a/be/src/io/cache/block_file_cache.cpp b/be/src/io/cache/block_file_cache.cpp index b41cb9f6a5fed9c..6a1c873966c3a92 100644 --- a/be/src/io/cache/block_file_cache.cpp +++ b/be/src/io/cache/block_file_cache.cpp @@ -623,7 +623,9 @@ BlockFileCache::FileBlockCell* BlockFileCache::add_cell(const UInt128Wrapper& ha auto& offsets = _files[hash]; DCHECK((context.expiration_time == 0 && context.cache_type != FileCacheType::TTL) || - (context.cache_type == FileCacheType::TTL && context.expiration_time != 0)); + (context.cache_type == FileCacheType::TTL && context.expiration_time != 0)) + << fmt::format("expiration time {}, cache type {}", context.expiration_time, + context.cache_type); FileCacheKey key; key.hash = hash; @@ -639,6 +641,7 @@ BlockFileCache::FileBlockCell* BlockFileCache::add_cell(const UInt128Wrapper& ha _key_to_time[hash] = context.expiration_time; _time_to_key.insert(std::make_pair(context.expiration_time, hash)); } + _cur_ttl_size += cell.size(); } auto [it, _] = offsets.insert(std::make_pair(offset, std::move(cell))); _cur_cache_size += size; @@ -695,6 +698,10 @@ const BlockFileCache::LRUQueue& BlockFileCache::get_queue(FileCacheType type) co bool BlockFileCache::try_reserve_for_ttl(size_t size, std::lock_guard& cache_lock) { size_t removed_size = 0; size_t cur_cache_size = _cur_cache_size; + auto limit = config::max_ttl_cache_ratio * _capacity; + if ((_cur_ttl_size + size) * 100 > limit) { + return false; + } auto is_overflow = [&] { return _disk_resource_limit_mode ? removed_size < size : cur_cache_size + size - removed_size > _capacity; @@ -1129,6 +1136,9 @@ void BlockFileCache::remove(FileBlockSPtr file_block, T& cache_lock, U& block_lo } } _cur_cache_size -= file_block->range().size(); + if (FileCacheType::TTL == type) { + _cur_ttl_size -= file_block->range().size(); + } auto& offsets = _files[hash]; offsets.erase(file_block->offset()); if (offsets.empty()) { @@ -1544,6 +1554,7 @@ std::string BlockFileCache::clear_file_cache_directly() { int64_t disposible_queue_size = _disposable_queue.get_elements_num(cache_lock); _files.clear(); _cur_cache_size = 0; + _cur_ttl_size = 0; _time_to_key.clear(); _key_to_time.clear(); _index_queue.clear(cache_lock); @@ -1560,6 +1571,22 @@ std::string BlockFileCache::clear_file_cache_directly() { return msg; } +std::map BlockFileCache::get_blocks_by_key(const UInt128Wrapper& hash) { + std::map offset_to_block; + std::lock_guard cache_lock(_mutex); + if (_files.contains(hash)) { + for (auto& [offset, cell] : _files[hash]) { + if (cell.file_block->state() == FileBlock::State::DOWNLOADED) { + offset_to_block.emplace(offset, cell.file_block); + use_cell(cell, nullptr, + need_to_move(cell.file_block->cache_type(), FileCacheType::DISPOSABLE), + cache_lock); + } + } + } + return offset_to_block; +} + template void BlockFileCache::remove(FileBlockSPtr file_block, std::lock_guard& cache_lock, std::lock_guard& block_lock); diff --git a/be/src/io/cache/block_file_cache.h b/be/src/io/cache/block_file_cache.h index 282148aa566dd3a..6f19095eace8d8e 100644 --- a/be/src/io/cache/block_file_cache.h +++ b/be/src/io/cache/block_file_cache.h @@ -94,7 +94,7 @@ class BlockFileCache { */ std::string clear_file_cache_async(); std::string clear_file_cache_directly(); - + std::map get_blocks_by_key(const UInt128Wrapper& hash); /// For debug. std::string dump_structure(const UInt128Wrapper& hash); @@ -394,6 +394,7 @@ class BlockFileCache { CachedFiles _files; QueryFileCacheContextMap _query_map; size_t _cur_cache_size = 0; + size_t _cur_ttl_size = 0; std::multimap _time_to_key; std::unordered_map _key_to_time; // The three queues are level queue. diff --git a/be/src/io/cache/block_file_cache_downloader.cpp b/be/src/io/cache/block_file_cache_downloader.cpp index 283605f23bed91d..f5a68f46d8f9590 100644 --- a/be/src/io/cache/block_file_cache_downloader.cpp +++ b/be/src/io/cache/block_file_cache_downloader.cpp @@ -316,12 +316,12 @@ void FileCacheBlockS3Downloader::download_file_cache_block(std::vectortablet_meta()->snapshot_rs_metas(); + auto id_to_rowset_meta_map = res.value()->tablet_meta()->snapshot_rs_metas(); if (auto iter = id_to_rowset_meta_map.find(meta.rowset_id()); iter != id_to_rowset_meta_map.end()) { UInt128Wrapper cache_key = BlockFileCache::hash(meta.file_name()); diff --git a/be/src/io/cache/cached_remote_file_reader.cpp b/be/src/io/cache/cached_remote_file_reader.cpp index d976ccb0df43806..9572180c8926ba0 100644 --- a/be/src/io/cache/cached_remote_file_reader.cpp +++ b/be/src/io/cache/cached_remote_file_reader.cpp @@ -34,6 +34,7 @@ #include "io/cache/block_file_cache_profile.h" #include "io/cache/file_block.h" #include "io/fs/file_reader.h" +#include "io/fs/local_file_system.h" #include "io/io_common.h" #include "util/bit_util.h" #include "util/doris_metrics.h" @@ -50,6 +51,9 @@ CachedRemoteFileReader::CachedRemoteFileReader(FileReaderSPtr remote_file_reader if (_is_doris_table) { _cache_hash = BlockFileCache::hash(path().filename().native()); _cache = FileCacheFactory::instance()->get_by_path(_cache_hash); + if (config::enable_read_cache_file_directly) { + _cache_file_readers = _cache->get_blocks_by_key(_cache_hash); + } } else { // Use path and modification time to build cache key std::string unique_path = fmt::format("{}:{}", path().native(), opts.mtime); @@ -69,6 +73,14 @@ CachedRemoteFileReader::CachedRemoteFileReader(FileReaderSPtr remote_file_reader } } +void CachedRemoteFileReader::_insert_file_reader(FileBlockSPtr file_block) { + if (config::enable_read_cache_file_directly) { + std::lock_guard lock(_mtx); + DCHECK(file_block->state() == FileBlock::State::DOWNLOADED); + _cache_file_readers.emplace(file_block->offset(), std::move(file_block)); + } +} + CachedRemoteFileReader::~CachedRemoteFileReader() { static_cast(close()); } @@ -110,6 +122,54 @@ Status CachedRemoteFileReader::read_at_impl(size_t offset, Slice result, size_t* return Status::OK(); } ReadStatistics stats; + auto defer_func = [&](int*) { + if (io_ctx->file_cache_stats) { + _update_state(stats, io_ctx->file_cache_stats); + io::FileCacheProfile::instance().update(io_ctx->file_cache_stats); + } + }; + std::unique_ptr defer((int*)0x01, std::move(defer_func)); + stats.bytes_read += bytes_req; + if (config::enable_read_cache_file_directly) { + // read directly + size_t need_read_size = bytes_req; + std::shared_lock lock(_mtx); + if (!_cache_file_readers.empty()) { + // find the last offset > offset. + auto iter = _cache_file_readers.upper_bound(offset); + if (iter != _cache_file_readers.begin()) { + iter--; + } + size_t cur_offset = offset; + while (need_read_size != 0 && iter != _cache_file_readers.end()) { + if (iter->second->offset() > cur_offset || + iter->second->range().right < cur_offset) { + break; + } + size_t file_offset = cur_offset - iter->second->offset(); + size_t reserve_bytes = + std::min(need_read_size, iter->second->range().size() - file_offset); + { + SCOPED_RAW_TIMER(&stats.local_read_timer); + if (!iter->second + ->read(Slice(result.data + (cur_offset - offset), reserve_bytes), + file_offset) + .ok()) { + break; + } + } + need_read_size -= reserve_bytes; + cur_offset += reserve_bytes; + iter++; + } + if (need_read_size == 0) { + *bytes_read = bytes_req; + stats.hit_cache = true; + return Status::OK(); + } + } + } + // read from cache or remote auto [align_left, align_size] = s_align_size(offset, bytes_req, size()); CacheContext cache_context(io_ctx); FileBlocksHolder holder = @@ -137,7 +197,6 @@ Status CachedRemoteFileReader::read_at_impl(size_t offset, Slice result, size_t* break; } } - stats.bytes_read += bytes_req; size_t empty_start = 0; size_t empty_end = 0; if (!empty_blocks.empty()) { @@ -164,6 +223,8 @@ Status CachedRemoteFileReader::read_at_impl(size_t offset, Slice result, size_t* } if (!st.ok()) { LOG_WARNING("Write data to file cache failed").error(st); + } else { + _insert_file_reader(block); } stats.bytes_write_into_file_cache += block_size; } @@ -243,11 +304,6 @@ Status CachedRemoteFileReader::read_at_impl(size_t offset, Slice result, size_t* current_offset = right + 1; } DCHECK(*bytes_read == bytes_req); - DorisMetrics::instance()->s3_bytes_read_total->increment(*bytes_read); - if (io_ctx->file_cache_stats) { - _update_state(stats, io_ctx->file_cache_stats); - io::FileCacheProfile::instance().update(io_ctx->file_cache_stats); - } return Status::OK(); } diff --git a/be/src/io/cache/cached_remote_file_reader.h b/be/src/io/cache/cached_remote_file_reader.h index 6cdba2c3dcc2a87..34fc53c7310f6f4 100644 --- a/be/src/io/cache/cached_remote_file_reader.h +++ b/be/src/io/cache/cached_remote_file_reader.h @@ -20,14 +20,17 @@ #include #include +#include #include #include #include #include "common/status.h" #include "io/cache/block_file_cache.h" +#include "io/cache/file_block.h" #include "io/cache/file_cache_common.h" #include "io/fs/file_reader.h" +#include "io/fs/file_reader_writer_fwd.h" #include "io/fs/file_system.h" #include "io/fs/path.h" #include "util/slice.h" @@ -60,10 +63,13 @@ class CachedRemoteFileReader final : public FileReader { const IOContext* io_ctx) override; private: + void _insert_file_reader(FileBlockSPtr file_block); bool _is_doris_table; FileReaderSPtr _remote_file_reader; UInt128Wrapper _cache_hash; BlockFileCache* _cache; + std::shared_mutex _mtx; + std::map _cache_file_readers; struct ReadStatistics { bool hit_cache = true; diff --git a/be/src/io/cache/file_block.cpp b/be/src/io/cache/file_block.cpp index 2efc26fb1a6a12e..5985aa95f7abdcd 100644 --- a/be/src/io/cache/file_block.cpp +++ b/be/src/io/cache/file_block.cpp @@ -30,6 +30,11 @@ namespace doris { namespace io { +std::ostream& operator<<(std::ostream& os, const FileBlock::State& value) { + os << FileBlock::state_to_string(value); + return os; +} + FileBlock::FileBlock(const FileCacheKey& key, size_t size, BlockFileCache* mgr, State download_state) : _block_range(key.offset, key.offset + size - 1), diff --git a/be/src/io/cache/file_block.h b/be/src/io/cache/file_block.h index dd4ef3757074ee4..b4044786dc73df0 100644 --- a/be/src/io/cache/file_block.h +++ b/be/src/io/cache/file_block.h @@ -41,6 +41,7 @@ class BlockFileCache; class FileBlock { friend struct FileBlocksHolder; friend class BlockFileCache; + friend class CachedRemoteFileReader; public: enum class State { @@ -154,6 +155,8 @@ class FileBlock { size_t _downloaded_size {0}; }; +extern std::ostream& operator<<(std::ostream& os, const FileBlock::State& value); + using FileBlockSPtr = std::shared_ptr; using FileBlocks = std::list; diff --git a/be/src/io/fs/broker_file_writer.h b/be/src/io/fs/broker_file_writer.h index 58bba9febd3db25..d6fce52a05c662a 100644 --- a/be/src/io/fs/broker_file_writer.h +++ b/be/src/io/fs/broker_file_writer.h @@ -33,7 +33,7 @@ namespace doris { class ExecEnv; namespace io { - +struct FileCacheAllocatorBuilder; class BrokerFileWriter final : public FileWriter { public: // Create and open file writer @@ -50,6 +50,7 @@ class BrokerFileWriter final : public FileWriter { const Path& path() const override { return _path; } size_t bytes_appended() const override { return _cur_offset; } bool closed() const override { return _closed; } + FileCacheAllocatorBuilder* cache_builder() const override { return nullptr; } private: Status _write(const uint8_t* buf, size_t buf_len, size_t* written_bytes); diff --git a/be/src/io/fs/file_writer.h b/be/src/io/fs/file_writer.h index 4feab99c09ff249..5d22dca60551c8e 100644 --- a/be/src/io/fs/file_writer.h +++ b/be/src/io/fs/file_writer.h @@ -28,6 +28,7 @@ namespace doris::io { class FileSystem; +struct FileCacheAllocatorBuilder; // Only affects remote file writers struct FileWriterOptions { @@ -62,6 +63,8 @@ class FileWriter { virtual size_t bytes_appended() const = 0; virtual bool closed() const = 0; + + virtual FileCacheAllocatorBuilder* cache_builder() const = 0; }; } // namespace doris::io diff --git a/be/src/io/fs/hdfs_file_reader.cpp b/be/src/io/fs/hdfs_file_reader.cpp index 358663b65d0dadc..a26448c90e2ce33 100644 --- a/be/src/io/fs/hdfs_file_reader.cpp +++ b/be/src/io/fs/hdfs_file_reader.cpp @@ -24,6 +24,8 @@ #include #include +#include "bvar/latency_recorder.h" +#include "bvar/reducer.h" #include "common/compiler_util.h" // IWYU pragma: keep #include "common/logging.h" #include "common/sync_point.h" @@ -33,6 +35,13 @@ #include "util/doris_metrics.h" namespace doris::io { + +bvar::Adder hdfs_bytes_read_total("hdfs_file_reader", "bytes_read"); +bvar::LatencyRecorder hdfs_bytes_per_read("hdfs_file_reader", "bytes_per_read"); // also QPS +bvar::PerSecond> hdfs_read_througthput("hdfs_file_reader", + "hdfs_read_throughput", + &hdfs_bytes_read_total); + namespace { Result get_file(const hdfsFS& fs, const Path& file, int64_t mtime, @@ -148,6 +157,8 @@ Status HdfsFileReader::read_at_impl(size_t offset, Slice result, size_t* bytes_r has_read += loop_read; } *bytes_read = has_read; + hdfs_bytes_read_total << *bytes_read; + hdfs_bytes_per_read << *bytes_read; return Status::OK(); } @@ -206,6 +217,8 @@ Status HdfsFileReader::read_at_impl(size_t offset, Slice result, size_t* bytes_r has_read += loop_read; } *bytes_read = has_read; + hdfs_bytes_read_total << *bytes_read; + hdfs_bytes_per_read << *bytes_read; return Status::OK(); } #endif diff --git a/be/src/io/fs/hdfs_file_writer.cpp b/be/src/io/fs/hdfs_file_writer.cpp index c596c0e290fe8c3..9ea66ca4da13ebc 100644 --- a/be/src/io/fs/hdfs_file_writer.cpp +++ b/be/src/io/fs/hdfs_file_writer.cpp @@ -159,9 +159,6 @@ void HdfsFileWriter::_write_into_local_file_cache() { size_t block_size = block->range().size(); size_t append_size = std::min(data_remain_size, block_size); if (block->state() == FileBlock::State::EMPTY) { - if (_index_offset != 0 && block->range().right >= _index_offset) { - static_cast(block->change_cache_type_self(FileCacheType::INDEX)); - } block->get_or_set_downloader(); if (block->is_downloader()) { Slice s(_batch_buffer.data() + pos, append_size); diff --git a/be/src/io/fs/hdfs_file_writer.h b/be/src/io/fs/hdfs_file_writer.h index 2ce865ecfedb20e..234835e083f4c56 100644 --- a/be/src/io/fs/hdfs_file_writer.h +++ b/be/src/io/fs/hdfs_file_writer.h @@ -51,6 +51,10 @@ class HdfsFileWriter final : public FileWriter { size_t bytes_appended() const override { return _bytes_appended; } bool closed() const override { return _closed; } + FileCacheAllocatorBuilder* cache_builder() const override { + return _cache_builder == nullptr ? nullptr : _cache_builder.get(); + } + private: // Flush buffered data into HDFS client and write local file cache if enabled // **Notice**: this would clear the underlying buffer @@ -83,7 +87,6 @@ class HdfsFileWriter final : public FileWriter { std::string _batch_buffer; }; BatchBuffer _batch_buffer; - size_t _index_offset = 0; }; } // namespace io diff --git a/be/src/io/fs/local_file_writer.h b/be/src/io/fs/local_file_writer.h index 4cd6712b04e1ef5..81ebb0ebd1fcb74 100644 --- a/be/src/io/fs/local_file_writer.h +++ b/be/src/io/fs/local_file_writer.h @@ -25,7 +25,7 @@ #include "util/slice.h" namespace doris::io { - +struct FileCacheAllocatorBuilder; class LocalFileWriter final : public FileWriter { public: LocalFileWriter(Path path, int fd, bool sync_data = true); @@ -38,6 +38,8 @@ class LocalFileWriter final : public FileWriter { size_t bytes_appended() const override; bool closed() const override { return _closed; } + FileCacheAllocatorBuilder* cache_builder() const override { return nullptr; } + private: void _abort(); Status _close(bool sync); diff --git a/be/src/io/fs/s3_file_bufferpool.cpp b/be/src/io/fs/s3_file_bufferpool.cpp index 9df60dc1fd30428..82493fa9514f10e 100644 --- a/be/src/io/fs/s3_file_bufferpool.cpp +++ b/be/src/io/fs/s3_file_bufferpool.cpp @@ -90,24 +90,6 @@ FileBuffer::~FileBuffer() { SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(ExecEnv::GetInstance()->s3_file_buffer_tracker()); _inner_data.reset(); } -/** - * 0. check if file cache holder allocated - * 1. update the cache's type to index cache - */ -void UploadFileBuffer::set_index_offset(size_t offset) { - _index_offset = offset; - if (_holder) { - bool change_to_index_cache = false; - for (auto iter = _holder->file_blocks.begin(); iter != _holder->file_blocks.end(); ++iter) { - if (iter == _cur_file_block) { - change_to_index_cache = true; - } - if (change_to_index_cache) { - static_cast((*iter)->change_cache_type_self(FileCacheType::INDEX)); - } - } - } -} /** * 0. when there is memory preserved, directly write data to buf @@ -222,9 +204,6 @@ void UploadFileBuffer::upload_to_local_file_cache(bool is_cancelled) { size_t block_size = block->range().size(); size_t append_size = std::min(data_remain_size, block_size); if (block->state() == FileBlock::State::EMPTY) { - if (_index_offset != 0 && block->range().right >= _index_offset) { - static_cast(block->change_cache_type_self(FileCacheType::INDEX)); - } block->get_or_set_downloader(); // Another thread may have started downloading due to a query // Just skip putting to cache from UploadFileBuffer @@ -279,7 +258,7 @@ Status FileBufferBuilder::build(std::shared_ptr* buf) { if (_type == BufferType::UPLOAD) { RETURN_IF_CATCH_EXCEPTION(*buf = std::make_shared( std::move(_upload_cb), std::move(state), _offset, - std::move(_alloc_holder_cb), _index_offset)); + std::move(_alloc_holder_cb))); return Status::OK(); } if (_type == BufferType::DOWNLOAD) { diff --git a/be/src/io/fs/s3_file_bufferpool.h b/be/src/io/fs/s3_file_bufferpool.h index 189f76969677592..15d0976df6b8808 100644 --- a/be/src/io/fs/s3_file_bufferpool.h +++ b/be/src/io/fs/s3_file_bufferpool.h @@ -151,18 +151,10 @@ struct DownloadFileBuffer final : public FileBuffer { struct UploadFileBuffer final : public FileBuffer { UploadFileBuffer(std::function upload_cb, OperationState state, - size_t offset, std::function alloc_holder, - size_t index_offset) + size_t offset, std::function alloc_holder) : FileBuffer(BufferType::UPLOAD, alloc_holder, offset, state), - _upload_to_remote(std::move(upload_cb)), - _index_offset(index_offset) {} + _upload_to_remote(std::move(upload_cb)) {} ~UploadFileBuffer() override = default; - /** - * set the index offset - * - * @param offset the index offset - */ - void set_index_offset(size_t offset); Status append_data(const Slice& s) override; /** * read the content from local file cache @@ -206,7 +198,6 @@ struct UploadFileBuffer final : public FileBuffer { FileBlocksHolderPtr _holder; decltype(_holder->file_blocks.begin()) _cur_file_block; size_t _append_offset {0}; - size_t _index_offset {0}; uint32_t _crc_value = 0; }; @@ -272,15 +263,6 @@ struct FileBufferBuilder { return *this; } /** - * set the index offset of the file buffer - * - * @param cb - */ - FileBufferBuilder& set_index_offset(size_t index_offset) { - _index_offset = index_offset; - return *this; - } - /** * set the callback which write the content into local file cache * * @param cb @@ -309,7 +291,6 @@ struct FileBufferBuilder { std::function _download; std::function _write_to_use_buffer; size_t _offset; - size_t _index_offset; }; } // namespace io } // namespace doris diff --git a/be/src/io/fs/s3_file_reader.cpp b/be/src/io/fs/s3_file_reader.cpp index 2bd40fbbf438611..68acbf47eb18a16 100644 --- a/be/src/io/fs/s3_file_reader.cpp +++ b/be/src/io/fs/s3_file_reader.cpp @@ -23,6 +23,7 @@ #include #include #include +#include #include #include #include @@ -43,6 +44,9 @@ bvar::Adder s3_file_reader_read_counter("s3_file_reader", "read_at"); bvar::Adder s3_file_reader_total("s3_file_reader", "total_num"); bvar::Adder s3_bytes_read_total("s3_file_reader", "bytes_read"); bvar::Adder s3_file_being_read("s3_file_reader", "file_being_read"); +bvar::LatencyRecorder s3_bytes_per_read("s3_file_reader", "bytes_per_read"); // also QPS +bvar::PerSecond> s3_read_througthput("s3_file_reader", "s3_read_throughput", + &s3_bytes_read_total); Result S3FileReader::create(std::shared_ptr client, std::string bucket, std::string key, @@ -125,6 +129,7 @@ Status S3FileReader::read_at_impl(size_t offset, Slice result, size_t* bytes_rea _path.native(), *bytes_read, bytes_req); } s3_bytes_read_total << *bytes_read; + s3_bytes_per_read << *bytes_read; s3_file_reader_read_counter << 1; DorisMetrics::instance()->s3_bytes_read_total->increment(*bytes_read); return Status::OK(); diff --git a/be/src/io/fs/s3_file_system.cpp b/be/src/io/fs/s3_file_system.cpp index 3d9f25686a88f82..dea3279350900d8 100644 --- a/be/src/io/fs/s3_file_system.cpp +++ b/be/src/io/fs/s3_file_system.cpp @@ -115,16 +115,21 @@ Status S3ClientHolder::init() { } Status S3ClientHolder::reset(const S3ClientConf& conf) { + S3ClientConf reset_conf; { std::shared_lock lock(_mtx); - if (conf.ak == _conf.ak && conf.sk == _conf.sk) { + if (conf.ak == _conf.ak && conf.sk == _conf.sk && conf.token == _conf.token) { return Status::OK(); // Same conf } + reset_conf = _conf; + reset_conf.ak = conf.ak; + reset_conf.sk = conf.sk; + reset_conf.token = conf.token; // Should check endpoint here? } - auto client = S3ClientFactory::instance().create(conf); + auto client = S3ClientFactory::instance().create(reset_conf); if (!client) { return Status::InternalError("failed to init s3 client with conf {}", conf.to_string()); } @@ -134,8 +139,7 @@ Status S3ClientHolder::reset(const S3ClientConf& conf) { { std::lock_guard lock(_mtx); _client = std::move(client); - _conf.ak = conf.ak; - _conf.sk = conf.sk; + _conf = std::move(reset_conf); } return Status::OK(); diff --git a/be/src/io/fs/s3_file_writer.cpp b/be/src/io/fs/s3_file_writer.cpp index 84487f496ac1e99..69202bd22fe98f8 100644 --- a/be/src/io/fs/s3_file_writer.cpp +++ b/be/src/io/fs/s3_file_writer.cpp @@ -87,16 +87,16 @@ S3FileWriter::S3FileWriter(std::shared_ptr client, std::strin : _path(fmt::format("s3://{}/{}", bucket, key)), _bucket(std::move(bucket)), _key(std::move(key)), - _client(std::move(client)), - _expiration_time(opts ? opts->file_cache_expiration : 0), - _is_cold_data(opts ? opts->is_cold_data : true), - _write_file_cache(opts ? opts->write_file_cache : false) { + _client(std::move(client)) { s3_file_writer_total << 1; s3_file_being_written << 1; Aws::Http::SetCompliantRfc3986Encoding(true); - if (config::enable_file_cache && _write_file_cache) { - _cache_hash = BlockFileCache::hash(_path.filename().native()); - _cache = FileCacheFactory::instance()->get_by_path(_cache_hash); + if (config::enable_file_cache && opts != nullptr && opts->write_file_cache) { + _cache_builder = std::make_unique(FileCacheAllocatorBuilder { + opts ? opts->is_cold_data : false, opts ? opts->file_cache_expiration : 0, + BlockFileCache::hash(_path.filename().native()), + FileCacheFactory::instance()->get_by_path( + BlockFileCache::hash(_path.filename().native()))}); } } @@ -264,7 +264,6 @@ Status S3FileWriter::appendv(const Slice* data, size_t data_cnt) { _upload_one_part(part_num, buf); }) .set_file_offset(_bytes_appended) - .set_index_offset(_index_offset) .set_sync_after_complete_task([this, part_num = _cur_part_num](Status s) { bool ret = false; if (!s.ok()) [[unlikely]] { @@ -282,22 +281,16 @@ Status S3FileWriter::appendv(const Slice* data, size_t data_cnt) { return ret; }) .set_is_cancelled([this]() { return _failed.load(); }); - if (_write_file_cache) { + if (_cache_builder != nullptr) { // We would load the data into file cache asynchronously which indicates // that this instance of S3FileWriter might have been destructed when we // try to do writing into file cache, so we make the lambda capture the variable // we need by value to extend their lifetime builder.set_allocate_file_blocks_holder( - [cache = _cache, k = _cache_hash, offset = _bytes_appended, - t = _expiration_time, cold = _is_cold_data]() -> FileBlocksHolderPtr { - CacheContext ctx; - ctx.cache_type = - t == 0 ? FileCacheType::NORMAL : FileCacheType::TTL; - ctx.expiration_time = t; - ctx.is_cold_data = cold; - auto holder = cache->get_or_set(k, offset, - config::s3_write_buffer_size, ctx); - return std::make_unique(std::move(holder)); + [builder = *_cache_builder, + offset = _bytes_appended]() -> FileBlocksHolderPtr { + return builder.allocate_cache_holder(offset, + config::s3_write_buffer_size); }); } RETURN_IF_ERROR(builder.build(&_pending_buf)); diff --git a/be/src/io/fs/s3_file_writer.h b/be/src/io/fs/s3_file_writer.h index a2c2ec0422a4f17..8f27b2023694445 100644 --- a/be/src/io/fs/s3_file_writer.h +++ b/be/src/io/fs/s3_file_writer.h @@ -58,6 +58,10 @@ class S3FileWriter final : public FileWriter { size_t bytes_appended() const override { return _bytes_appended; } bool closed() const override { return _closed; } + FileCacheAllocatorBuilder* cache_builder() const override { + return _cache_builder == nullptr ? nullptr : _cache_builder.get(); + } + private: Status _abort(); [[nodiscard]] std::string _dump_completed_part() const; @@ -73,15 +77,12 @@ class S3FileWriter final : public FileWriter { std::shared_ptr _client; std::string _upload_id; - size_t _index_offset {0}; // Current Part Num for CompletedPart int _cur_part_num = 1; std::mutex _completed_lock; std::vector> _completed_parts; - UInt128Wrapper _cache_hash; - BlockFileCache* _cache; // **Attention** call add_count() before submitting buf to async thread pool bthread::CountdownEvent _countdown_event {0}; @@ -92,9 +93,8 @@ class S3FileWriter final : public FileWriter { size_t _bytes_appended = 0; std::shared_ptr _pending_buf; - uint64_t _expiration_time; - bool _is_cold_data; - bool _write_file_cache; + std::unique_ptr + _cache_builder; // nullptr if disable write file cache }; } // namespace io diff --git a/be/src/io/fs/stream_sink_file_writer.h b/be/src/io/fs/stream_sink_file_writer.h index 2bd91075ad11a0a..4a0eb955c26171c 100644 --- a/be/src/io/fs/stream_sink_file_writer.h +++ b/be/src/io/fs/stream_sink_file_writer.h @@ -33,6 +33,7 @@ struct RowsetId; struct SegmentStatistics; namespace io { +struct FileCacheAllocatorBuilder; class StreamSinkFileWriter final : public FileWriter { public: StreamSinkFileWriter(std::vector> streams) @@ -57,6 +58,8 @@ class StreamSinkFileWriter final : public FileWriter { return dummy; } + FileCacheAllocatorBuilder* cache_builder() const override { return nullptr; } + private: std::vector> _streams; diff --git a/be/src/olap/data_dir.cpp b/be/src/olap/data_dir.cpp index d1b5185b7894ce2..da7302ef6054457 100644 --- a/be/src/olap/data_dir.cpp +++ b/be/src/olap/data_dir.cpp @@ -231,7 +231,7 @@ void DataDir::health_check() { // check disk if (_is_used) { Status res = _read_and_write_test_file(); - if (!res) { + if (!res && res.is()) { LOG(WARNING) << "store read/write test file occur IO Error. path=" << _path << ", err: " << res; _engine.add_broken_path(_path); diff --git a/be/src/olap/full_compaction.cpp b/be/src/olap/full_compaction.cpp index 0d6660ca54368ff..8a2712c38b55071 100644 --- a/be/src/olap/full_compaction.cpp +++ b/be/src/olap/full_compaction.cpp @@ -46,7 +46,9 @@ FullCompaction::FullCompaction(StorageEngine& engine, const TabletSharedPtr& tab : CompactionMixin(engine, tablet, "FullCompaction:" + std::to_string(tablet->tablet_id())) { } -FullCompaction::~FullCompaction() = default; +FullCompaction::~FullCompaction() { + tablet()->set_is_full_compaction_running(false); +} Status FullCompaction::prepare_compact() { if (!tablet()->init_succeeded()) { @@ -55,6 +57,7 @@ Status FullCompaction::prepare_compact() { std::unique_lock base_lock(tablet()->get_base_compaction_lock()); std::unique_lock cumu_lock(tablet()->get_cumulative_compaction_lock()); + tablet()->set_is_full_compaction_running(true); // 1. pick rowsets to compact RETURN_IF_ERROR(pick_rowsets_to_compact()); @@ -112,6 +115,7 @@ Status FullCompaction::modify_rowsets() { std::lock_guard rowset_update_wlock(tablet()->get_rowset_update_lock()); std::lock_guard meta_wlock(_tablet->get_header_lock()); RETURN_IF_ERROR(tablet()->modify_rowsets(output_rowsets, _input_rowsets, true)); + DBUG_EXECUTE_IF("FullCompaction.modify_rowsets.sleep", { sleep(5); }) tablet()->save_meta(); } return Status::OK(); diff --git a/be/src/olap/memtable_flush_executor.cpp b/be/src/olap/memtable_flush_executor.cpp index db14b9acaeeeaf9..b52a87dff07762e 100644 --- a/be/src/olap/memtable_flush_executor.cpp +++ b/be/src/olap/memtable_flush_executor.cpp @@ -202,15 +202,20 @@ void FlushToken::_flush_memtable(std::unique_ptr memtable_ptr, int32_t void MemTableFlushExecutor::init(int num_disk) { num_disk = std::max(1, num_disk); - size_t min_threads = std::max(1, config::flush_thread_num_per_store); - size_t max_threads = num_disk * min_threads; + int num_cpus = std::thread::hardware_concurrency(); + int min_threads = std::max(1, config::flush_thread_num_per_store); + int max_threads = num_cpus == 0 ? num_disk * min_threads + : std::min(num_disk * min_threads, + num_cpus * config::max_flush_thread_num_per_cpu); static_cast(ThreadPoolBuilder("MemTableFlushThreadPool") .set_min_threads(min_threads) .set_max_threads(max_threads) .build(&_flush_pool)); min_threads = std::max(1, config::high_priority_flush_thread_num_per_store); - max_threads = num_disk * min_threads; + max_threads = num_cpus == 0 ? num_disk * min_threads + : std::min(num_disk * min_threads, + num_cpus * config::max_flush_thread_num_per_cpu); static_cast(ThreadPoolBuilder("MemTableHighPriorityFlushThreadPool") .set_min_threads(min_threads) .set_max_threads(max_threads) diff --git a/be/src/olap/olap_server.cpp b/be/src/olap/olap_server.cpp index 73bd0e37f81db81..17ad3979faedfb8 100644 --- a/be/src/olap/olap_server.cpp +++ b/be/src/olap/olap_server.cpp @@ -918,11 +918,16 @@ bool StorageEngine::_push_tablet_into_submitted_compaction(TabletSharedPtr table .insert(tablet->tablet_id()) .second); break; - default: + case CompactionType::BASE_COMPACTION: already_existed = !(_tablet_submitted_base_compaction[tablet->data_dir()] .insert(tablet->tablet_id()) .second); break; + case CompactionType::FULL_COMPACTION: + already_existed = !(_tablet_submitted_full_compaction[tablet->data_dir()] + .insert(tablet->tablet_id()) + .second); + break; } return already_existed; } @@ -935,9 +940,12 @@ void StorageEngine::_pop_tablet_from_submitted_compaction(TabletSharedPtr tablet case CompactionType::CUMULATIVE_COMPACTION: removed = _tablet_submitted_cumu_compaction[tablet->data_dir()].erase(tablet->tablet_id()); break; - default: + case CompactionType::BASE_COMPACTION: removed = _tablet_submitted_base_compaction[tablet->data_dir()].erase(tablet->tablet_id()); break; + case CompactionType::FULL_COMPACTION: + removed = _tablet_submitted_full_compaction[tablet->data_dir()].erase(tablet->tablet_id()); + break; } if (removed == 1) { 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 e9956008f98179f..7774dc0c1dd95d3 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index_writer.cpp +++ b/be/src/olap/rowset/segment_v2/inverted_index_writer.cpp @@ -363,16 +363,15 @@ class InvertedIndexColumnWriterImpl : public InvertedIndexColumnWriter { LOG(ERROR) << "index writer is null in inverted index writer."; return Status::InternalError("index writer is null in inverted index writer"); } + size_t start_off = 0; for (int i = 0; i < count; ++i) { - // offsets[i+1] is now row element count - // [0, 3, 6] - // [10,20,30] [20,30,40], [30,40,50] - auto start_off = offsets[i]; - auto end_off = offsets[i + 1]; + // nullmap & value ptr-array may not from offsets[i] because olap_convertor make offsets accumulate from _base_offset which may not is 0, but nullmap & value in this segment is from 0, we only need + // every single array row element size to go through the nullmap & value ptr-array, and also can go through the every row in array to keep with _rid++ + auto array_elem_size = offsets[i + 1] - offsets[i]; // TODO(Amory).later we use object pool to avoid field creation lucene::document::Field* new_field = nullptr; CL_NS(analysis)::TokenStream* ts = nullptr; - for (auto j = start_off; j < end_off; ++j) { + for (auto j = start_off; j < start_off + array_elem_size; ++j) { if (null_map[j] == 1) { continue; } @@ -405,19 +404,22 @@ class InvertedIndexColumnWriterImpl : public InvertedIndexColumnWriter { _doc->add(*new_field); } } + start_off += array_elem_size; if (!_doc->getFields()->empty()) { // if this array is null, we just ignore to write inverted index RETURN_IF_ERROR(add_document()); _doc->clear(); _CLDELETE(ts); + } else { + RETURN_IF_ERROR(add_null_document()); } _rid++; } } else if constexpr (field_is_numeric_type(field_type)) { + size_t start_off = 0; for (int i = 0; i < count; ++i) { - auto start_off = offsets[i]; - auto end_off = offsets[i + 1]; - for (size_t j = start_off; j < end_off; ++j) { + auto array_elem_size = offsets[i + 1] - offsets[i]; + for (size_t j = start_off; j < start_off + array_elem_size; ++j) { if (null_map[j] == 1) { continue; } @@ -428,6 +430,7 @@ class InvertedIndexColumnWriterImpl : public InvertedIndexColumnWriter { _value_key_coder->full_encode_ascending(p, &new_value); _bkd_writer->add((const uint8_t*)new_value.c_str(), value_length, _rid); } + start_off += array_elem_size; _row_ids_seen_for_bkd++; _rid++; } diff --git a/be/src/olap/rowset/segment_v2/segment_writer.cpp b/be/src/olap/rowset/segment_v2/segment_writer.cpp index f6be1917e578408..7a83496b7fb4817 100644 --- a/be/src/olap/rowset/segment_v2/segment_writer.cpp +++ b/be/src/olap/rowset/segment_v2/segment_writer.cpp @@ -34,6 +34,8 @@ #include "common/status.h" #include "gutil/port.h" #include "inverted_index_fs_directory.h" +#include "io/cache/block_file_cache.h" +#include "io/cache/block_file_cache_factory.h" #include "io/fs/file_system.h" #include "io/fs/file_writer.h" #include "io/fs/local_file_system.h" @@ -1113,6 +1115,8 @@ Status SegmentWriter::finalize(uint64_t* segment_file_size, uint64_t* index_size } // write data RETURN_IF_ERROR(finalize_columns_data()); + // Get the index start before finalize_footer since this function would write new data. + uint64_t index_start = _file_writer->bytes_appended(); // write index RETURN_IF_ERROR(finalize_columns_index(index_size)); // write footer @@ -1122,6 +1126,17 @@ Status SegmentWriter::finalize(uint64_t* segment_file_size, uint64_t* index_size LOG(INFO) << "segment flush consumes a lot time_ns " << timer.elapsed_time() << ", segmemt_size " << *segment_file_size; } + // When the cache type is not ttl(expiration time == 0), the data should be split into normal cache queue + // and index cache queue + if (auto* cache_builder = _file_writer->cache_builder(); cache_builder != nullptr && + cache_builder->_expiration_time == 0 && + config::is_cloud_mode()) { + auto size = *index_size + *segment_file_size; + auto holder = cache_builder->allocate_cache_holder(index_start, size); + for (auto& segment : holder->file_blocks) { + static_cast(segment->change_cache_type_self(io::FileCacheType::INDEX)); + } + } return Status::OK(); } diff --git a/be/src/olap/storage_engine.cpp b/be/src/olap/storage_engine.cpp index d327f82ab3362ab..d549e17b1bffd8f 100644 --- a/be/src/olap/storage_engine.cpp +++ b/be/src/olap/storage_engine.cpp @@ -1396,6 +1396,30 @@ Status StorageEngine::get_compaction_status_json(std::string* result) { } root.AddMember(base_key, path_obj2, root.GetAllocator()); + // full + const std::string& full = "FullCompaction"; + rapidjson::Value full_key; + full_key.SetString(full.c_str(), full.length(), root.GetAllocator()); + rapidjson::Document path_obj3; + path_obj3.SetObject(); + for (auto& it : _tablet_submitted_full_compaction) { + const std::string& dir = it.first->path(); + rapidjson::Value path_key; + path_key.SetString(dir.c_str(), dir.length(), path_obj3.GetAllocator()); + + rapidjson::Document arr; + arr.SetArray(); + + for (auto& tablet_id : it.second) { + rapidjson::Value key; + const std::string& key_str = std::to_string(tablet_id); + key.SetString(key_str.c_str(), key_str.length(), path_obj3.GetAllocator()); + arr.PushBack(key, root.GetAllocator()); + } + path_obj3.AddMember(path_key, arr, path_obj3.GetAllocator()); + } + root.AddMember(full_key, path_obj3, root.GetAllocator()); + rapidjson::StringBuffer strbuf; rapidjson::PrettyWriter writer(strbuf); root.Accept(writer); diff --git a/be/src/olap/storage_engine.h b/be/src/olap/storage_engine.h index 0412a576e1641e1..63234047305a40b 100644 --- a/be/src/olap/storage_engine.h +++ b/be/src/olap/storage_engine.h @@ -441,6 +441,7 @@ class StorageEngine final : public BaseStorageEngine { // a tablet can do base and cumulative compaction at same time std::map> _tablet_submitted_cumu_compaction; std::map> _tablet_submitted_base_compaction; + std::map> _tablet_submitted_full_compaction; std::mutex _low_priority_task_nums_mutex; std::unordered_map _low_priority_task_nums; diff --git a/be/src/olap/tablet.cpp b/be/src/olap/tablet.cpp index 94889bbcf8f5d6e..358292463fc3fa1 100644 --- a/be/src/olap/tablet.cpp +++ b/be/src/olap/tablet.cpp @@ -1241,7 +1241,7 @@ void Tablet::get_compaction_status(std::string* json_result) { root.AddMember("last base failure time", base_value, root.GetAllocator()); rapidjson::Value full_value; format_str = ToStringFromUnixMillis(_last_full_compaction_failure_millis.load()); - base_value.SetString(format_str.c_str(), format_str.length(), root.GetAllocator()); + full_value.SetString(format_str.c_str(), format_str.length(), root.GetAllocator()); root.AddMember("last full failure time", full_value, root.GetAllocator()); rapidjson::Value cumu_success_value; format_str = ToStringFromUnixMillis(_last_cumu_compaction_success_millis.load()); diff --git a/be/src/olap/tablet.h b/be/src/olap/tablet.h index 41f67b898ce0d94..678a519cfaeb725 100644 --- a/be/src/olap/tablet.h +++ b/be/src/olap/tablet.h @@ -459,6 +459,10 @@ class Tablet final : public BaseTablet { void set_alter_failed(bool alter_failed) { _alter_failed = alter_failed; } bool is_alter_failed() { return _alter_failed; } + void set_is_full_compaction_running(bool is_full_compaction_running) { + _is_full_compaction_running = is_full_compaction_running; + } + inline bool is_full_compaction_running() const { return _is_full_compaction_running; } void clear_cache() override; private: @@ -573,6 +577,8 @@ class Tablet final : public BaseTablet { std::atomic _alter_failed = false; int64_t _io_error_times = 0; + + std::atomic_bool _is_full_compaction_running = false; }; inline CumulativeCompactionPolicy* Tablet::cumulative_compaction_policy() { diff --git a/be/src/pipeline/exec/aggregation_sink_operator.cpp b/be/src/pipeline/exec/aggregation_sink_operator.cpp index fd88b0d15218b06..d947dd55c36194f 100644 --- a/be/src/pipeline/exec/aggregation_sink_operator.cpp +++ b/be/src/pipeline/exec/aggregation_sink_operator.cpp @@ -19,16 +19,12 @@ #include -#include "pipeline/exec/distinct_streaming_aggregation_sink_operator.h" #include "pipeline/exec/operator.h" -#include "pipeline/exec/streaming_aggregation_sink_operator.h" #include "runtime/primitive_type.h" #include "vec/common/hash_table/hash.h" namespace doris::pipeline { -OPERATOR_CODE_GENERATOR(AggSinkOperator, StreamingOperator) - /// The minimum reduction factor (input rows divided by output rows) to grow hash tables /// in a streaming preaggregation, given that the hash tables are currently the given /// size or above. The sizes roughly correspond to hash table sizes where the bucket @@ -616,7 +612,7 @@ void AggSinkLocalState::_init_hash_method(const vectorized::VExprContextSPtrs& p } AggSinkOperatorX::AggSinkOperatorX(ObjectPool* pool, int operator_id, const TPlanNode& tnode, - const DescriptorTbl& descs) + const DescriptorTbl& descs, bool require_bucket_distribution) : DataSinkOperatorX(operator_id, tnode.node_id), _intermediate_tuple_id(tnode.agg_node.intermediate_tuple_id), _intermediate_tuple_desc(nullptr), @@ -629,9 +625,12 @@ AggSinkOperatorX::AggSinkOperatorX(ObjectPool* pool, int operator_id, const TPla _limit(tnode.limit), _have_conjuncts((tnode.__isset.vconjunct && !tnode.vconjunct.nodes.empty()) || (tnode.__isset.conjuncts && !tnode.conjuncts.empty())), - _partition_exprs(tnode.__isset.distribute_expr_lists ? tnode.distribute_expr_lists[0] - : std::vector {}), - _is_colocate(tnode.agg_node.__isset.is_colocate && tnode.agg_node.is_colocate), + _partition_exprs(require_bucket_distribution ? (tnode.__isset.distribute_expr_lists + ? tnode.distribute_expr_lists[0] + : std::vector {}) + : tnode.agg_node.grouping_exprs), + _is_colocate(tnode.agg_node.__isset.is_colocate && tnode.agg_node.is_colocate && + require_bucket_distribution), _agg_fn_output_row_descriptor(descs, tnode.row_tuples, tnode.nullable_tuples) {} Status AggSinkOperatorX::init(const TPlanNode& tnode, RuntimeState* state) { diff --git a/be/src/pipeline/exec/aggregation_sink_operator.h b/be/src/pipeline/exec/aggregation_sink_operator.h index b3ffa19d6db7912..ee31db3691759d7 100644 --- a/be/src/pipeline/exec/aggregation_sink_operator.h +++ b/be/src/pipeline/exec/aggregation_sink_operator.h @@ -23,26 +23,8 @@ #include "pipeline/pipeline_x/operator.h" #include "runtime/block_spill_manager.h" #include "runtime/exec_env.h" -#include "vec/exec/vaggregation_node.h" -namespace doris { -class ExecNode; - -namespace pipeline { - -class AggSinkOperatorBuilder final : public OperatorBuilder { -public: - AggSinkOperatorBuilder(int32_t, ExecNode*); - - OperatorPtr build_operator() override; - bool is_sink() const override { return true; } -}; - -class AggSinkOperator final : public StreamingOperator { -public: - AggSinkOperator(OperatorBuilderBase* operator_builder, ExecNode* node); - bool can_write() override { return true; } -}; +namespace doris::pipeline { class AggSinkOperatorX; @@ -143,7 +125,7 @@ class AggSinkLocalState : public PipelineXSinkLocalState { class AggSinkOperatorX final : public DataSinkOperatorX { public: AggSinkOperatorX(ObjectPool* pool, int operator_id, const TPlanNode& tnode, - const DescriptorTbl& descs); + const DescriptorTbl& descs, bool require_bucket_distribution); ~AggSinkOperatorX() override = default; Status init(const TDataSink& tsink) override { return Status::InternalError("{} should not init with TPlanNode", @@ -217,5 +199,4 @@ class AggSinkOperatorX final : public DataSinkOperatorX { RowDescriptor _agg_fn_output_row_descriptor; }; -} // namespace pipeline -} // namespace doris +} // namespace doris::pipeline diff --git a/be/src/pipeline/exec/aggregation_source_operator.cpp b/be/src/pipeline/exec/aggregation_source_operator.cpp index cff6f9fec42fe92..f53d96d71aa2ae1 100644 --- a/be/src/pipeline/exec/aggregation_source_operator.cpp +++ b/be/src/pipeline/exec/aggregation_source_operator.cpp @@ -22,13 +22,10 @@ #include "common/exception.h" #include "pipeline/exec/operator.h" -#include "pipeline/exec/streaming_aggregation_source_operator.h" #include "vec//utils/util.hpp" namespace doris::pipeline { -OPERATOR_CODE_GENERATOR(AggSourceOperator, SourceOperator) - AggLocalState::AggLocalState(RuntimeState* state, OperatorXBase* parent) : Base(state, parent), _get_results_timer(nullptr), diff --git a/be/src/pipeline/exec/aggregation_source_operator.h b/be/src/pipeline/exec/aggregation_source_operator.h index 1d1f564d41aa62f..d43ec7db0b550de 100644 --- a/be/src/pipeline/exec/aggregation_source_operator.h +++ b/be/src/pipeline/exec/aggregation_source_operator.h @@ -21,32 +21,12 @@ #include "common/status.h" #include "operator.h" #include "pipeline/pipeline_x/operator.h" -#include "vec/exec/vaggregation_node.h" namespace doris { -class ExecNode; class RuntimeState; namespace pipeline { -class AggSourceOperatorBuilder final : public OperatorBuilder { -public: - AggSourceOperatorBuilder(int32_t, ExecNode*); - - bool is_source() const override { return true; } - - OperatorPtr build_operator() override; -}; - -class AggSourceOperator final : public SourceOperator { -public: - AggSourceOperator(OperatorBuilderBase*, ExecNode*); - // if exec node split to: sink, source operator. the source operator - // should skip `alloc_resource()` function call, only sink operator - // call the function - Status open(RuntimeState*) override { return Status::OK(); } -}; - class AggSourceOperatorX; class AggLocalState final : public PipelineXLocalState { diff --git a/be/src/pipeline/exec/analytic_sink_operator.cpp b/be/src/pipeline/exec/analytic_sink_operator.cpp index a1d3384edc6ddeb..12c4e7634e71a6c 100644 --- a/be/src/pipeline/exec/analytic_sink_operator.cpp +++ b/be/src/pipeline/exec/analytic_sink_operator.cpp @@ -24,8 +24,6 @@ namespace doris::pipeline { -OPERATOR_CODE_GENERATOR(AnalyticSinkOperator, StreamingOperator) - Status AnalyticSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { RETURN_IF_ERROR(PipelineXSinkLocalState::init(state, info)); SCOPED_TIMER(exec_time_counter()); diff --git a/be/src/pipeline/exec/analytic_sink_operator.h b/be/src/pipeline/exec/analytic_sink_operator.h index 3ae4a7b5cff5ca2..0098c108e53d4e5 100644 --- a/be/src/pipeline/exec/analytic_sink_operator.h +++ b/be/src/pipeline/exec/analytic_sink_operator.h @@ -23,10 +23,8 @@ #include "operator.h" #include "pipeline/pipeline_x/dependency.h" #include "pipeline/pipeline_x/operator.h" -#include "vec/exec/vanalytic_eval_node.h" namespace doris { -class ExecNode; namespace pipeline { class AnalyticSinkOperatorBuilder final : public OperatorBuilder { diff --git a/be/src/pipeline/exec/analytic_source_operator.cpp b/be/src/pipeline/exec/analytic_source_operator.cpp index f6658583d4657ab..85995732fff3d41 100644 --- a/be/src/pipeline/exec/analytic_source_operator.cpp +++ b/be/src/pipeline/exec/analytic_source_operator.cpp @@ -24,8 +24,6 @@ namespace doris::pipeline { -OPERATOR_CODE_GENERATOR(AnalyticSourceOperator, SourceOperator) - AnalyticLocalState::AnalyticLocalState(RuntimeState* state, OperatorXBase* parent) : PipelineXLocalState(state, parent), _output_block_index(0), diff --git a/be/src/pipeline/exec/analytic_source_operator.h b/be/src/pipeline/exec/analytic_source_operator.h index 17a4d34ec739b3b..075053a486600c7 100644 --- a/be/src/pipeline/exec/analytic_source_operator.h +++ b/be/src/pipeline/exec/analytic_source_operator.h @@ -22,30 +22,12 @@ #include "common/status.h" #include "operator.h" #include "pipeline/pipeline_x/operator.h" -#include "vec/exec/vanalytic_eval_node.h" namespace doris { -class ExecNode; class RuntimeState; namespace pipeline { -class AnalyticSourceOperatorBuilder final : public OperatorBuilder { -public: - AnalyticSourceOperatorBuilder(int32_t, ExecNode*); - - bool is_source() const override { return true; } - - OperatorPtr build_operator() override; -}; - -class AnalyticSourceOperator final : public SourceOperator { -public: - AnalyticSourceOperator(OperatorBuilderBase*, ExecNode*); - - Status open(RuntimeState*) override { return Status::OK(); } -}; - class AnalyticSourceOperatorX; class AnalyticLocalState final : public PipelineXLocalState { public: diff --git a/be/src/pipeline/exec/assert_num_rows_operator.cpp b/be/src/pipeline/exec/assert_num_rows_operator.cpp index ef0efd3f86bf2b3..4a51002beff389f 100644 --- a/be/src/pipeline/exec/assert_num_rows_operator.cpp +++ b/be/src/pipeline/exec/assert_num_rows_operator.cpp @@ -22,10 +22,6 @@ namespace doris::pipeline { -OperatorPtr AssertNumRowsOperatorBuilder::build_operator() { - return std::make_shared(this, _node); -} - AssertNumRowsOperatorX::AssertNumRowsOperatorX(ObjectPool* pool, const TPlanNode& tnode, int operator_id, const DescriptorTbl& descs) : StreamingOperatorX(pool, tnode, operator_id, descs), diff --git a/be/src/pipeline/exec/assert_num_rows_operator.h b/be/src/pipeline/exec/assert_num_rows_operator.h index 4d6d835f815aa4c..07a90d6b4711449 100644 --- a/be/src/pipeline/exec/assert_num_rows_operator.h +++ b/be/src/pipeline/exec/assert_num_rows_operator.h @@ -19,25 +19,8 @@ #include "operator.h" #include "pipeline/pipeline_x/operator.h" -#include "vec/exec/vassert_num_rows_node.h" -namespace doris { - -namespace pipeline { - -class AssertNumRowsOperatorBuilder final : public OperatorBuilder { -public: - AssertNumRowsOperatorBuilder(int32_t id, ExecNode* node) - : OperatorBuilder(id, "AssertNumRowsOperator", node) {} - - OperatorPtr build_operator() override; -}; - -class AssertNumRowsOperator final : public StreamingOperator { -public: - AssertNumRowsOperator(OperatorBuilderBase* operator_builder, ExecNode* node) - : StreamingOperator(operator_builder, node) {} -}; +namespace doris::pipeline { class AssertNumRowsLocalState final : public PipelineXLocalState { public: @@ -70,5 +53,4 @@ class AssertNumRowsOperatorX final : public StreamingOperatorX { -public: - ConstValueOperatorBuilder(int32_t id, ExecNode* node) - : OperatorBuilder(id, "ConstValueOperator", node) {} - - OperatorPtr build_operator() override; - - bool is_source() const override { return true; } -}; - -class ConstValueOperator final : public SourceOperator { -public: - ConstValueOperator(OperatorBuilderBase* operator_builder, ExecNode* node) - : SourceOperator(operator_builder, node) {} - - bool can_read() override { return true; } -}; - -OperatorPtr ConstValueOperatorBuilder::build_operator() { - return std::make_shared(this, _node); -} - -} // namespace pipeline -} // namespace doris \ No newline at end of file diff --git a/be/src/pipeline/exec/datagen_operator.cpp b/be/src/pipeline/exec/datagen_operator.cpp index 4fbe21f71d5e32d..95b284c94b49502 100644 --- a/be/src/pipeline/exec/datagen_operator.cpp +++ b/be/src/pipeline/exec/datagen_operator.cpp @@ -23,7 +23,6 @@ #include "util/runtime_profile.h" #include "vec/exec/data_gen_functions/vdata_gen_function_inf.h" #include "vec/exec/data_gen_functions/vnumbers_tvf.h" -#include "vec/exec/vdata_gen_scan_node.h" namespace doris { class RuntimeState; @@ -31,19 +30,6 @@ class RuntimeState; namespace doris::pipeline { -OPERATOR_CODE_GENERATOR(DataGenOperator, SourceOperator) - -Status DataGenOperator::open(RuntimeState* state) { - RETURN_IF_ERROR(SourceOperator::open(state)); - return _node->open(state); -} - -Status DataGenOperator::close(RuntimeState* state) { - RETURN_IF_ERROR(SourceOperator::close(state)); - RETURN_IF_ERROR(_node->close(state)); - return Status::OK(); -} - DataGenSourceOperatorX::DataGenSourceOperatorX(ObjectPool* pool, const TPlanNode& tnode, int operator_id, const DescriptorTbl& descs) : OperatorX(pool, tnode, operator_id, descs), diff --git a/be/src/pipeline/exec/datagen_operator.h b/be/src/pipeline/exec/datagen_operator.h index af8eda179dac7c1..edcc85c8342e126 100644 --- a/be/src/pipeline/exec/datagen_operator.h +++ b/be/src/pipeline/exec/datagen_operator.h @@ -25,30 +25,11 @@ #include "vec/exec/vdata_gen_scan_node.h" namespace doris { -class ExecNode; class RuntimeState; } // namespace doris namespace doris::pipeline { -class DataGenOperatorBuilder : public OperatorBuilder { -public: - DataGenOperatorBuilder(int32_t id, ExecNode* exec_node); - bool is_source() const override { return true; } - OperatorPtr build_operator() override; -}; - -class DataGenOperator : public SourceOperator { -public: - DataGenOperator(OperatorBuilderBase* operator_builder, ExecNode* datagen_node); - - bool can_read() override { return true; } - - Status open(RuntimeState* state) override; - - Status close(RuntimeState* state) override; -}; - class DataGenSourceOperatorX; class DataGenLocalState final : public PipelineXLocalState<> { public: diff --git a/be/src/pipeline/exec/distinct_streaming_aggregation_sink_operator.cpp b/be/src/pipeline/exec/distinct_streaming_aggregation_sink_operator.cpp deleted file mode 100644 index 3cb18168dcb2157..000000000000000 --- a/be/src/pipeline/exec/distinct_streaming_aggregation_sink_operator.cpp +++ /dev/null @@ -1,100 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#include "distinct_streaming_aggregation_sink_operator.h" - -#include - -#include -#include - -#include "common/compiler_util.h" // IWYU pragma: keep -#include "pipeline/exec/data_queue.h" -#include "pipeline/exec/operator.h" -#include "vec/exec/distinct_vaggregation_node.h" -#include "vec/exec/vaggregation_node.h" - -namespace doris { -class ExecNode; -class RuntimeState; -} // namespace doris - -namespace doris::pipeline { - -DistinctStreamingAggSinkOperator::DistinctStreamingAggSinkOperator( - OperatorBuilderBase* operator_builder, ExecNode* agg_node, std::shared_ptr queue) - : StreamingOperator(operator_builder, agg_node), _data_queue(std::move(queue)) {} - -bool DistinctStreamingAggSinkOperator::can_write() { - // sink and source in diff threads - return _data_queue->has_enough_space_to_push(); -} - -Status DistinctStreamingAggSinkOperator::sink(RuntimeState* state, vectorized::Block* in_block, - SourceState source_state) { - if (in_block && in_block->rows() > 0) { - if (_output_block == nullptr) { - _output_block = _data_queue->get_free_block(); - } - RETURN_IF_ERROR( - _node->_distinct_pre_agg_with_serialized_key(in_block, _output_block.get())); - bool stop_emplace_flag = _node->is_stop_emplace_flag(); - // get enough data or reached limit rows, need push block to queue - if (!stop_emplace_flag && _node->limit() != -1 && - (_output_block->rows() + _output_distinct_rows) >= _node->limit()) { - auto limit_rows = _node->limit() - _output_distinct_rows; - _output_block->set_num_rows(limit_rows); - _output_distinct_rows += limit_rows; - _data_queue->push_block(std::move(_output_block)); - } else if (stop_emplace_flag || _output_block->rows() >= state->batch_size()) { - if (!stop_emplace_flag) { // if stop_emplace_flag = true, will be return rows directly, not get distinct - _output_distinct_rows += _output_block->rows(); - } - _data_queue->push_block(std::move(_output_block)); - } - } - - // reach limit or source finish - if ((UNLIKELY(source_state == SourceState::FINISHED)) || reached_limited_rows()) { - if (_output_block != nullptr) { //maybe the last block with eos - _output_distinct_rows += _output_block->rows(); - _data_queue->push_block(std::move(_output_block)); - } - _data_queue->set_finish(); - return Status::Error(""); - } - return Status::OK(); -} - -Status DistinctStreamingAggSinkOperator::close(RuntimeState* state) { - if (_data_queue && !_data_queue->is_finish()) { - // finish should be set, if not set here means error. - _data_queue->set_canceled(); - } - return StreamingOperator::close(state); -} - -DistinctStreamingAggSinkOperatorBuilder::DistinctStreamingAggSinkOperatorBuilder( - int32_t id, ExecNode* exec_node, std::shared_ptr queue) - : OperatorBuilder(id, "DistinctStreamingAggSinkOperator", exec_node), - _data_queue(std::move(queue)) {} - -OperatorPtr DistinctStreamingAggSinkOperatorBuilder::build_operator() { - return std::make_shared(this, _node, _data_queue); -} - -} // namespace doris::pipeline diff --git a/be/src/pipeline/exec/distinct_streaming_aggregation_sink_operator.h b/be/src/pipeline/exec/distinct_streaming_aggregation_sink_operator.h deleted file mode 100644 index c872a2b299e14cf..000000000000000 --- a/be/src/pipeline/exec/distinct_streaming_aggregation_sink_operator.h +++ /dev/null @@ -1,79 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#pragma once - -#include - -#include -#include - -#include "aggregation_sink_operator.h" -#include "common/status.h" -#include "operator.h" -#include "pipeline/exec/aggregation_sink_operator.h" -#include "pipeline/exec/aggregation_source_operator.h" -#include "util/runtime_profile.h" -#include "vec/core/block.h" -#include "vec/exec/distinct_vaggregation_node.h" -#include "vec/exec/vaggregation_node.h" - -namespace doris { -class ExecNode; -class RuntimeState; - -namespace pipeline { -class DataQueue; - -class DistinctStreamingAggSinkOperatorBuilder final - : public OperatorBuilder { -public: - DistinctStreamingAggSinkOperatorBuilder(int32_t, ExecNode*, std::shared_ptr); - - OperatorPtr build_operator() override; - - bool is_sink() const override { return true; } - bool is_source() const override { return false; } - -private: - std::shared_ptr _data_queue; -}; - -class DistinctStreamingAggSinkOperator final - : public StreamingOperator { -public: - DistinctStreamingAggSinkOperator(OperatorBuilderBase* operator_builder, ExecNode*, - std::shared_ptr); - - Status sink(RuntimeState* state, vectorized::Block* block, SourceState source_state) override; - - bool can_write() override; - - Status close(RuntimeState* state) override; - - bool reached_limited_rows() { - return _node->limit() != -1 && _output_distinct_rows >= _node->limit(); - } - -private: - int64_t _output_distinct_rows = 0; - std::shared_ptr _data_queue; - std::unique_ptr _output_block = vectorized::Block::create_unique(); -}; - -} // namespace pipeline -} // namespace doris diff --git a/be/src/pipeline/exec/distinct_streaming_aggregation_source_operator.cpp b/be/src/pipeline/exec/distinct_streaming_aggregation_source_operator.cpp deleted file mode 100644 index 5ab8bd30bc845ff..000000000000000 --- a/be/src/pipeline/exec/distinct_streaming_aggregation_source_operator.cpp +++ /dev/null @@ -1,91 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#include "distinct_streaming_aggregation_source_operator.h" - -#include - -#include "pipeline/exec/data_queue.h" -#include "pipeline/exec/operator.h" -#include "runtime/descriptors.h" -#include "util/runtime_profile.h" -#include "vec/core/block.h" -#include "vec/exec/distinct_vaggregation_node.h" -#include "vec/exec/vaggregation_node.h" - -namespace doris { -class ExecNode; -class RuntimeState; - -namespace pipeline { -DistinctStreamingAggSourceOperator::DistinctStreamingAggSourceOperator( - OperatorBuilderBase* templ, ExecNode* node, std::shared_ptr queue) - : SourceOperator(templ, node), _data_queue(std::move(queue)) {} - -bool DistinctStreamingAggSourceOperator::can_read() { - return _data_queue->has_data_or_finished(); -} - -Status DistinctStreamingAggSourceOperator::pull_data(RuntimeState* state, vectorized::Block* block, - bool* eos) { - std::unique_ptr agg_block; - RETURN_IF_ERROR(_data_queue->get_block_from_queue(&agg_block)); - if (agg_block != nullptr) { - block->swap(*agg_block); - agg_block->clear_column_data(block->columns()); - _data_queue->push_free_block(std::move(agg_block)); - } - if (_data_queue->data_exhausted()) { //the sink is eos or reached limit - *eos = true; - } - _node->_make_nullable_output_key(block); - if (_node->is_streaming_preagg() == false) { - // dispose the having clause, should not be execute in prestreaming agg - RETURN_IF_ERROR(vectorized::VExprContext::filter_block(_node->get_conjuncts(), block, - block->columns())); - } - - _node->add_num_rows_returned(block->rows()); - return Status::OK(); -} - -Status DistinctStreamingAggSourceOperator::get_block(RuntimeState* state, vectorized::Block* block, - SourceState& source_state) { - bool eos = false; - RETURN_IF_ERROR(_node->get_next_after_projects( - state, block, &eos, - std::bind(&DistinctStreamingAggSourceOperator::pull_data, this, std::placeholders::_1, - std::placeholders::_2, std::placeholders::_3))); - if (UNLIKELY(eos)) { - source_state = SourceState::FINISHED; - } else { - source_state = SourceState::DEPEND_ON_SOURCE; - } - return Status::OK(); -} - -DistinctStreamingAggSourceOperatorBuilder::DistinctStreamingAggSourceOperatorBuilder( - int32_t id, ExecNode* exec_node, std::shared_ptr queue) - : OperatorBuilder(id, "DistinctStreamingAggSourceOperator", exec_node), - _data_queue(std::move(queue)) {} - -OperatorPtr DistinctStreamingAggSourceOperatorBuilder::build_operator() { - return std::make_shared(this, _node, _data_queue); -} - -} // namespace pipeline -} // namespace doris diff --git a/be/src/pipeline/exec/distinct_streaming_aggregation_source_operator.h b/be/src/pipeline/exec/distinct_streaming_aggregation_source_operator.h deleted file mode 100644 index e8fd21310bbd7ba..000000000000000 --- a/be/src/pipeline/exec/distinct_streaming_aggregation_source_operator.h +++ /dev/null @@ -1,67 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. -#pragma once - -#include - -#include -#include - -#include "common/status.h" -#include "operator.h" -#include "pipeline/exec/aggregation_source_operator.h" -#include "vec/exec/distinct_vaggregation_node.h" -#include "vec/exec/vaggregation_node.h" - -namespace doris { -class ExecNode; -class RuntimeState; - -namespace vectorized { -class Block; -} // namespace vectorized -namespace pipeline { -class DataQueue; - -class DistinctStreamingAggSourceOperatorBuilder final - : public OperatorBuilder { -public: - DistinctStreamingAggSourceOperatorBuilder(int32_t, ExecNode*, std::shared_ptr); - - bool is_source() const override { return true; } - - OperatorPtr build_operator() override; - -private: - std::shared_ptr _data_queue; -}; - -class DistinctStreamingAggSourceOperator final - : public SourceOperator { -public: - DistinctStreamingAggSourceOperator(OperatorBuilderBase*, ExecNode*, std::shared_ptr); - bool can_read() override; - Status get_block(RuntimeState*, vectorized::Block*, SourceState& source_state) override; - Status open(RuntimeState*) override { return Status::OK(); } - Status pull_data(RuntimeState* state, vectorized::Block* output_block, bool* eos); - -private: - std::shared_ptr _data_queue; -}; - -} // namespace pipeline -} // namespace doris diff --git a/be/src/pipeline/exec/empty_set_operator.cpp b/be/src/pipeline/exec/empty_set_operator.cpp index 02dc80258031b6a..7233e46dfd1e527 100644 --- a/be/src/pipeline/exec/empty_set_operator.cpp +++ b/be/src/pipeline/exec/empty_set_operator.cpp @@ -23,8 +23,6 @@ namespace doris::pipeline { -OPERATOR_CODE_GENERATOR(EmptySetSourceOperator, SourceOperator) - Status EmptySetSourceOperatorX::get_block(RuntimeState* state, vectorized::Block* block, bool* eos) { *eos = true; diff --git a/be/src/pipeline/exec/empty_set_operator.h b/be/src/pipeline/exec/empty_set_operator.h index b65139fb985a5f6..3d5e9a72cf2bfec 100644 --- a/be/src/pipeline/exec/empty_set_operator.h +++ b/be/src/pipeline/exec/empty_set_operator.h @@ -21,27 +21,8 @@ #include "operator.h" #include "pipeline/pipeline_x/operator.h" -#include "vec/exec/vempty_set_node.h" -namespace doris { -class ExecNode; - -namespace pipeline { - -class EmptySetSourceOperatorBuilder final : public OperatorBuilder { -public: - EmptySetSourceOperatorBuilder(int32_t id, ExecNode* empty_set_node); - - bool is_source() const override { return true; } - - OperatorPtr build_operator() override; -}; - -class EmptySetSourceOperator final : public SourceOperator { -public: - EmptySetSourceOperator(OperatorBuilderBase* operator_builder, ExecNode* empty_set_node); - bool can_read() override { return true; } -}; +namespace doris::pipeline { class EmptySetLocalState final : public PipelineXLocalState { public: @@ -63,5 +44,4 @@ class EmptySetSourceOperatorX final : public OperatorX { [[nodiscard]] bool is_source() const override { return true; } }; -} // namespace pipeline -} // namespace doris +} // namespace doris::pipeline diff --git a/be/src/pipeline/exec/empty_source_operator.h b/be/src/pipeline/exec/empty_source_operator.h deleted file mode 100644 index b85d2b1a2ca9dc4..000000000000000 --- a/be/src/pipeline/exec/empty_source_operator.h +++ /dev/null @@ -1,89 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#pragma once - -#include - -#include - -#include "common/status.h" -#include "operator.h" -#include "runtime/descriptors.h" - -namespace doris { -class RuntimeState; - -namespace vectorized { -class Block; -} // namespace vectorized -} // namespace doris - -namespace doris::pipeline { - -class EmptySourceOperatorBuilder final : public OperatorBuilderBase { -public: - EmptySourceOperatorBuilder(int32_t id, const RowDescriptor& row_descriptor, ExecNode* exec_node) - : OperatorBuilderBase(id, "EmptySourceOperator"), - _row_descriptor(row_descriptor), - _exec_node(exec_node) {} - - bool is_source() const override { return true; } - - OperatorPtr build_operator() override; - - const RowDescriptor& row_desc() const override { return _row_descriptor; } - -private: - RowDescriptor _row_descriptor; - ExecNode* _exec_node = nullptr; -}; - -class EmptySourceOperator final : public OperatorBase { -public: - EmptySourceOperator(OperatorBuilderBase* builder, ExecNode* exec_node) - : OperatorBase(builder), _exec_node(exec_node) {} - - bool can_read() override { return true; } - bool is_pending_finish() const override { return false; } - - Status prepare(RuntimeState*) override { return Status::OK(); } - - Status open(RuntimeState*) override { return Status::OK(); } - - Status get_block(RuntimeState* /*runtime_state*/, vectorized::Block* /*block*/, - SourceState& result_state) override { - result_state = SourceState::FINISHED; - return Status::OK(); - } - - Status sink(RuntimeState*, vectorized::Block*, SourceState) override { return Status::OK(); } - - Status close(RuntimeState* state) override { - RETURN_IF_ERROR(_exec_node->close(state)); - return Status::OK(); - } - - [[nodiscard]] RuntimeProfile* get_runtime_profile() const override { - return _exec_node->runtime_profile(); - } - -private: - ExecNode* _exec_node = nullptr; -}; - -} // namespace doris::pipeline diff --git a/be/src/pipeline/exec/es_scan_operator.h b/be/src/pipeline/exec/es_scan_operator.h index cdbd6922454c116..c4e7772807181e3 100644 --- a/be/src/pipeline/exec/es_scan_operator.h +++ b/be/src/pipeline/exec/es_scan_operator.h @@ -25,10 +25,8 @@ #include "operator.h" #include "pipeline/exec/scan_operator.h" #include "pipeline/pipeline_x/operator.h" -#include "vec/exec/scan/vscan_node.h" namespace doris { -class ExecNode; namespace vectorized { class NewEsScanner; diff --git a/be/src/pipeline/exec/exchange_sink_operator.cpp b/be/src/pipeline/exec/exchange_sink_operator.cpp index 79a6ee0e74881fa..7db22c98f9f6b72 100644 --- a/be/src/pipeline/exec/exchange_sink_operator.cpp +++ b/be/src/pipeline/exec/exchange_sink_operator.cpp @@ -30,7 +30,6 @@ #include "pipeline/pipeline_x/local_exchange/local_exchange_sink_operator.h" #include "vec/columns/column_const.h" #include "vec/exprs/vexpr.h" -#include "vec/sink/vdata_stream_sender.h" namespace doris { class DataSink; @@ -38,58 +37,6 @@ class DataSink; namespace doris::pipeline { -ExchangeSinkOperatorBuilder::ExchangeSinkOperatorBuilder(int32_t id, DataSink* sink, - int mult_cast_id) - : DataSinkOperatorBuilder(id, "ExchangeSinkOperator", sink), _mult_cast_id(mult_cast_id) {} - -OperatorPtr ExchangeSinkOperatorBuilder::build_operator() { - return std::make_shared(this, _sink, _mult_cast_id); -} - -ExchangeSinkOperator::ExchangeSinkOperator(OperatorBuilderBase* operator_builder, DataSink* sink, - int mult_cast_id) - : DataSinkOperator(operator_builder, sink), _mult_cast_id(mult_cast_id) {} - -Status ExchangeSinkOperator::init(const TDataSink& tsink) { - // -1 means not the mult cast stream sender - if (_mult_cast_id == -1) { - _dest_node_id = tsink.stream_sink.dest_node_id; - } else { - _dest_node_id = tsink.multi_cast_stream_sink.sinks[_mult_cast_id].dest_node_id; - } - return Status::OK(); -} - -Status ExchangeSinkOperator::prepare(RuntimeState* state) { - _state = state; - PUniqueId id; - id.set_hi(_state->query_id().hi); - id.set_lo(_state->query_id().lo); - _sink_buffer = std::make_unique>( - id, _dest_node_id, _sink->_sender_id, _state->be_number(), state); - - RETURN_IF_ERROR(DataSinkOperator::prepare(state)); - _sink->register_pipeline_channels(_sink_buffer.get()); - return Status::OK(); -} - -bool ExchangeSinkOperator::can_write() { - return _sink_buffer->can_write() && _sink->channel_all_can_write(); -} - -bool ExchangeSinkOperator::is_pending_finish() const { - return _sink_buffer->is_pending_finish(); -} - -Status ExchangeSinkOperator::close(RuntimeState* state) { - RETURN_IF_ERROR(DataSinkOperator::close(state)); - if (_sink_buffer) { - _sink_buffer->update_profile(_sink->profile()); - _sink_buffer->close(); - } - return Status::OK(); -} - Status ExchangeSinkLocalState::serialize_block(vectorized::Block* src, PBlock* dest, int num_receivers) { return _parent->cast().serialize_block(*this, src, dest, num_receivers); @@ -221,8 +168,8 @@ Status ExchangeSinkLocalState::open(RuntimeState* state) { } if (_part_type == TPartitionType::HASH_PARTITIONED) { _partition_count = channels.size(); - _partitioner.reset( - new vectorized::Crc32HashPartitioner(channels.size())); + _partitioner.reset(new vectorized::Crc32HashPartitioner( + channels.size())); RETURN_IF_ERROR(_partitioner->init(p._texprs)); RETURN_IF_ERROR(_partitioner->prepare(state, p._row_desc)); _profile->add_info_string("Partitioner", @@ -269,8 +216,8 @@ Status ExchangeSinkLocalState::open(RuntimeState* state) { } else if (_part_type == TPartitionType::TABLE_SINK_HASH_PARTITIONED) { _partition_count = channels.size() * config::table_sink_partition_write_max_partition_nums_per_writer; - _partitioner.reset( - new vectorized::Crc32HashPartitioner(_partition_count)); + _partitioner.reset(new vectorized::Crc32HashPartitioner( + _partition_count)); _partition_function.reset(new HashPartitionFunction(_partitioner.get())); scale_writer_partitioning_exchanger.reset(new vectorized::ScaleWriterPartitioningExchanger< diff --git a/be/src/pipeline/exec/exchange_sink_operator.h b/be/src/pipeline/exec/exchange_sink_operator.h index f275365c0e85a35..aaa89d246be6562 100644 --- a/be/src/pipeline/exec/exchange_sink_operator.h +++ b/be/src/pipeline/exec/exchange_sink_operator.h @@ -35,36 +35,6 @@ class TDataSink; namespace pipeline { -class ExchangeSinkOperatorBuilder final - : public DataSinkOperatorBuilder { -public: - ExchangeSinkOperatorBuilder(int32_t id, DataSink* sink, int mult_cast_id = -1); - - OperatorPtr build_operator() override; - -private: - int _mult_cast_id = -1; -}; - -// Now local exchange is not supported since VDataStreamRecvr is considered as a pipeline broker. -class ExchangeSinkOperator final : public DataSinkOperator { -public: - ExchangeSinkOperator(OperatorBuilderBase* operator_builder, DataSink* sink, int mult_cast_id); - Status init(const TDataSink& tsink) override; - - Status prepare(RuntimeState* state) override; - bool can_write() override; - bool is_pending_finish() const override; - - Status close(RuntimeState* state) override; - -private: - std::unique_ptr> _sink_buffer = nullptr; - int _dest_node_id = -1; - RuntimeState* _state = nullptr; - int _mult_cast_id = -1; -}; - 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 a23dae6dd62dd14..1a891655ee8daa1 100644 --- a/be/src/pipeline/exec/exchange_source_operator.cpp +++ b/be/src/pipeline/exec/exchange_source_operator.cpp @@ -30,16 +30,6 @@ namespace doris::pipeline { -OPERATOR_CODE_GENERATOR(ExchangeSourceOperator, SourceOperator) - -bool ExchangeSourceOperator::can_read() { - return _node->_stream_recvr->ready_to_read(); -} - -bool ExchangeSourceOperator::is_pending_finish() const { - return false; -} - ExchangeLocalState::ExchangeLocalState(RuntimeState* state, OperatorXBase* parent) : Base(state, parent), num_rows_skipped(0), is_ready(false) {} diff --git a/be/src/pipeline/exec/exchange_source_operator.h b/be/src/pipeline/exec/exchange_source_operator.h index 6176ad5b7f71906..de761d8391c94c4 100644 --- a/be/src/pipeline/exec/exchange_source_operator.h +++ b/be/src/pipeline/exec/exchange_source_operator.h @@ -21,7 +21,6 @@ #include "operator.h" #include "pipeline/pipeline_x/operator.h" -#include "vec/exec/vexchange_node.h" namespace doris { class ExecNode; @@ -34,22 +33,6 @@ class Block; namespace doris::pipeline { -class ExchangeSourceOperatorBuilder final : public OperatorBuilder { -public: - ExchangeSourceOperatorBuilder(int32_t id, ExecNode* exec_node); - - bool is_source() const override { return true; } - - OperatorPtr build_operator() override; -}; - -class ExchangeSourceOperator final : public SourceOperator { -public: - ExchangeSourceOperator(OperatorBuilderBase*, ExecNode*); - bool can_read() override; - bool is_pending_finish() const override; -}; - class ExchangeSourceOperatorX; class ExchangeLocalState final : public PipelineXLocalState<> { ENABLE_FACTORY_CREATOR(ExchangeLocalState); diff --git a/be/src/pipeline/exec/file_scan_operator.h b/be/src/pipeline/exec/file_scan_operator.h index e59dd8055b29498..f5c4f194bc5767c 100644 --- a/be/src/pipeline/exec/file_scan_operator.h +++ b/be/src/pipeline/exec/file_scan_operator.h @@ -27,10 +27,8 @@ #include "pipeline/exec/scan_operator.h" #include "pipeline/pipeline_x/operator.h" #include "vec/exec/format/format_common.h" -#include "vec/exec/scan/vscan_node.h" namespace doris { -class ExecNode; namespace vectorized { class VFileScanner; } // namespace vectorized diff --git a/be/src/pipeline/exec/group_commit_block_sink_operator.cpp b/be/src/pipeline/exec/group_commit_block_sink_operator.cpp index 7c6abefc4a1902f..4e9969d957049d2 100644 --- a/be/src/pipeline/exec/group_commit_block_sink_operator.cpp +++ b/be/src/pipeline/exec/group_commit_block_sink_operator.cpp @@ -23,10 +23,6 @@ namespace doris::pipeline { -OperatorPtr GroupCommitBlockSinkOperatorBuilder::build_operator() { - return std::make_shared(this, _sink); -} - GroupCommitBlockSinkLocalState::~GroupCommitBlockSinkLocalState() { if (_load_block_queue) { _remove_estimated_wal_bytes(); diff --git a/be/src/pipeline/exec/group_commit_block_sink_operator.h b/be/src/pipeline/exec/group_commit_block_sink_operator.h index 9d8aa372d6ea3da..ad03d607c15d2d4 100644 --- a/be/src/pipeline/exec/group_commit_block_sink_operator.h +++ b/be/src/pipeline/exec/group_commit_block_sink_operator.h @@ -21,27 +21,7 @@ #include "pipeline/pipeline_x/operator.h" #include "vec/sink/group_commit_block_sink.h" -namespace doris { - -namespace pipeline { - -class GroupCommitBlockSinkOperatorBuilder final - : public DataSinkOperatorBuilder { -public: - GroupCommitBlockSinkOperatorBuilder(int32_t id, DataSink* sink) - : DataSinkOperatorBuilder(id, "GroupCommitBlockSinkOperator", sink) {} - - OperatorPtr build_operator() override; -}; - -class GroupCommitBlockSinkOperator final - : public DataSinkOperator { -public: - GroupCommitBlockSinkOperator(OperatorBuilderBase* operator_builder, DataSink* sink) - : DataSinkOperator(operator_builder, sink) {} - - bool can_write() override { return true; } // TODO: need use mem_limit -}; +namespace doris::pipeline { class GroupCommitBlockSinkOperatorX; class GroupCommitBlockSinkLocalState final : public PipelineXSinkLocalState { @@ -122,5 +102,4 @@ class GroupCommitBlockSinkOperatorX final TGroupCommitMode::type _group_commit_mode; }; -} // namespace pipeline -} // namespace doris \ No newline at end of file +} // namespace doris::pipeline diff --git a/be/src/pipeline/exec/hashjoin_build_sink.cpp b/be/src/pipeline/exec/hashjoin_build_sink.cpp index da3614e4479fcec..d583a827059111f 100644 --- a/be/src/pipeline/exec/hashjoin_build_sink.cpp +++ b/be/src/pipeline/exec/hashjoin_build_sink.cpp @@ -28,8 +28,6 @@ namespace doris::pipeline { -OPERATOR_CODE_GENERATOR(HashJoinBuildSink, StreamingOperator) - template struct Overload : Callables... { using Callables::operator()...; diff --git a/be/src/pipeline/exec/hashjoin_build_sink.h b/be/src/pipeline/exec/hashjoin_build_sink.h index 0998884c99bb814..18c922eb19d5ea9 100644 --- a/be/src/pipeline/exec/hashjoin_build_sink.h +++ b/be/src/pipeline/exec/hashjoin_build_sink.h @@ -22,12 +22,8 @@ #include "join_build_sink_operator.h" #include "operator.h" #include "pipeline/pipeline_x/operator.h" -#include "vec/exec/join/vhash_join_node.h" -namespace doris { -class ExecNode; - -namespace pipeline { +namespace doris::pipeline { class HashJoinBuildSinkBuilder final : public OperatorBuilder { public: @@ -187,5 +183,4 @@ class HashJoinBuildSinkOperatorX final const bool _need_local_merge; }; -} // namespace pipeline -} // namespace doris +} // namespace doris::pipeline diff --git a/be/src/pipeline/exec/hashjoin_probe_operator.cpp b/be/src/pipeline/exec/hashjoin_probe_operator.cpp index fc6f81f41902a2a..6e8e96e8d8a2ba9 100644 --- a/be/src/pipeline/exec/hashjoin_probe_operator.cpp +++ b/be/src/pipeline/exec/hashjoin_probe_operator.cpp @@ -22,10 +22,7 @@ #include "common/logging.h" #include "pipeline/exec/operator.h" -namespace doris { -namespace pipeline { - -OPERATOR_CODE_GENERATOR(HashJoinProbeOperator, StatefulOperator) +namespace doris::pipeline { HashJoinProbeLocalState::HashJoinProbeLocalState(RuntimeState* state, OperatorXBase* parent) : JoinProbeLocalState(state, parent) {} @@ -634,5 +631,4 @@ Status HashJoinProbeOperatorX::open(RuntimeState* state) { return Status::OK(); } -} // namespace pipeline -} // namespace doris +} // namespace doris::pipeline diff --git a/be/src/pipeline/exec/hashjoin_probe_operator.h b/be/src/pipeline/exec/hashjoin_probe_operator.h index 1b45a2a258eb07f..b5daefd735d9847 100644 --- a/be/src/pipeline/exec/hashjoin_probe_operator.h +++ b/be/src/pipeline/exec/hashjoin_probe_operator.h @@ -24,27 +24,10 @@ #include "pipeline/pipeline_x/operator.h" namespace doris { -class ExecNode; class RuntimeState; namespace pipeline { -class HashJoinProbeOperatorBuilder final : public OperatorBuilder { -public: - HashJoinProbeOperatorBuilder(int32_t, ExecNode*); - - OperatorPtr build_operator() override; -}; - -class HashJoinProbeOperator final : public StatefulOperator { -public: - HashJoinProbeOperator(OperatorBuilderBase*, ExecNode*); - // if exec node split to: sink, source operator. the source operator - // should skip `alloc_resource()` function call, only sink operator - // call the function - Status open(RuntimeState*) override { return Status::OK(); } -}; - class HashJoinProbeLocalState; using HashTableCtxVariants = std::variant< diff --git a/be/src/pipeline/exec/hive_table_sink_operator.cpp b/be/src/pipeline/exec/hive_table_sink_operator.cpp index 6b8eaa8c91e6354..b931d48e832fca2 100644 --- a/be/src/pipeline/exec/hive_table_sink_operator.cpp +++ b/be/src/pipeline/exec/hive_table_sink_operator.cpp @@ -21,10 +21,6 @@ namespace doris::pipeline { -OperatorPtr HiveTableSinkOperatorBuilder::build_operator() { - return std::make_shared(this, _sink); -} - Status HiveTableSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { RETURN_IF_ERROR(Base::init(state, info)); SCOPED_TIMER(exec_time_counter()); diff --git a/be/src/pipeline/exec/hive_table_sink_operator.h b/be/src/pipeline/exec/hive_table_sink_operator.h index 39b5df365674409..a489948268b9e46 100644 --- a/be/src/pipeline/exec/hive_table_sink_operator.h +++ b/be/src/pipeline/exec/hive_table_sink_operator.h @@ -21,26 +21,7 @@ #include "pipeline/pipeline_x/operator.h" #include "vec/sink/vhive_table_sink.h" -namespace doris { - -namespace pipeline { - -class HiveTableSinkOperatorBuilder final - : public DataSinkOperatorBuilder { -public: - HiveTableSinkOperatorBuilder(int32_t id, DataSink* sink) - : DataSinkOperatorBuilder(id, "HiveTableSinkOperator", sink) {} - - OperatorPtr build_operator() override; -}; - -class HiveTableSinkOperator final : public DataSinkOperator { -public: - HiveTableSinkOperator(OperatorBuilderBase* operator_builder, DataSink* sink) - : DataSinkOperator(operator_builder, sink) {} - - bool can_write() override { return _sink->can_write(); } -}; +namespace doris::pipeline { class HiveTableSinkOperatorX; @@ -111,5 +92,4 @@ class HiveTableSinkOperatorX final : public DataSinkOperatorX class JoinBuildSinkOperatorX; @@ -82,5 +79,4 @@ class JoinBuildSinkOperatorX : public DataSinkOperatorX { const std::vector _runtime_filter_descs; }; -} // namespace pipeline -} // namespace doris +} // namespace doris::pipeline diff --git a/be/src/pipeline/exec/join_probe_operator.h b/be/src/pipeline/exec/join_probe_operator.h index 679446147ef9ac5..228a4140b13c343 100644 --- a/be/src/pipeline/exec/join_probe_operator.h +++ b/be/src/pipeline/exec/join_probe_operator.h @@ -19,11 +19,8 @@ #include "operator.h" #include "pipeline/pipeline_x/operator.h" -#include "vec/exec/join/vjoin_node_base.h" -namespace doris { - -namespace pipeline { +namespace doris::pipeline { template class JoinProbeOperatorX; template @@ -127,5 +124,4 @@ class JoinProbeOperatorX : public StatefulOperatorX { const bool _use_specific_projections; }; -} // namespace pipeline -} // namespace doris +} // namespace doris::pipeline diff --git a/be/src/pipeline/exec/meta_scan_operator.h b/be/src/pipeline/exec/meta_scan_operator.h index e26af7dba5a6e18..440f489f2513e33 100644 --- a/be/src/pipeline/exec/meta_scan_operator.h +++ b/be/src/pipeline/exec/meta_scan_operator.h @@ -25,10 +25,8 @@ #include "operator.h" #include "pipeline/exec/scan_operator.h" #include "pipeline/pipeline_x/operator.h" -#include "vec/exec/scan/vscan_node.h" namespace doris { -class ExecNode; namespace vectorized { class NewOlapScanner; diff --git a/be/src/pipeline/exec/multi_cast_data_stream_sink.cpp b/be/src/pipeline/exec/multi_cast_data_stream_sink.cpp index de9cdeba04fae5f..e39f60b356ccd94 100644 --- a/be/src/pipeline/exec/multi_cast_data_stream_sink.cpp +++ b/be/src/pipeline/exec/multi_cast_data_stream_sink.cpp @@ -19,10 +19,6 @@ namespace doris::pipeline { -OperatorPtr MultiCastDataStreamSinkOperatorBuilder::build_operator() { - return std::make_shared(this, _sink); -} - std::string MultiCastDataStreamSinkLocalState::name_suffix() { auto& sinks = static_cast(_parent)->sink_node().sinks; std::string id_name = " (dst id : "; diff --git a/be/src/pipeline/exec/multi_cast_data_stream_sink.h b/be/src/pipeline/exec/multi_cast_data_stream_sink.h index b4886f089ef1083..1a18772af4fd12c 100644 --- a/be/src/pipeline/exec/multi_cast_data_stream_sink.h +++ b/be/src/pipeline/exec/multi_cast_data_stream_sink.h @@ -19,28 +19,9 @@ #include "operator.h" #include "pipeline/pipeline_x/operator.h" -#include "vec/sink/multi_cast_data_stream_sink.h" namespace doris::pipeline { -class MultiCastDataStreamSinkOperatorBuilder final - : public DataSinkOperatorBuilder { -public: - MultiCastDataStreamSinkOperatorBuilder(int32_t id, DataSink* sink) - : DataSinkOperatorBuilder(id, "MultiCastDataStreamSinkOperator", sink) {} - - OperatorPtr build_operator() override; -}; - -class MultiCastDataStreamSinkOperator final - : public DataSinkOperator { -public: - MultiCastDataStreamSinkOperator(OperatorBuilderBase* operator_builder, DataSink* sink) - : DataSinkOperator(operator_builder, sink) {} - - bool can_write() override { return _sink->can_write(); } -}; - class MultiCastDataStreamSinkOperatorX; class MultiCastDataStreamSinkLocalState final : public PipelineXSinkLocalState { diff --git a/be/src/pipeline/exec/multi_cast_data_stream_source.cpp b/be/src/pipeline/exec/multi_cast_data_stream_source.cpp index 90c809c535968d5..b72125abbb3eb4f 100644 --- a/be/src/pipeline/exec/multi_cast_data_stream_source.cpp +++ b/be/src/pipeline/exec/multi_cast_data_stream_source.cpp @@ -25,104 +25,6 @@ namespace doris::pipeline { -MultiCastDataStreamerSourceOperatorBuilder::MultiCastDataStreamerSourceOperatorBuilder( - int32_t id, const int consumer_id, std::shared_ptr& data_streamer, - const TDataStreamSink& sink) - : OperatorBuilderBase(id, "MultiCastDataStreamerSourceOperator"), - _consumer_id(consumer_id), - _multi_cast_data_streamer(data_streamer), - _t_data_stream_sink(sink) {} - -OperatorPtr MultiCastDataStreamerSourceOperatorBuilder::build_operator() { - return std::make_shared( - this, _consumer_id, _multi_cast_data_streamer, _t_data_stream_sink); -} - -const RowDescriptor& MultiCastDataStreamerSourceOperatorBuilder::row_desc() const { - return _multi_cast_data_streamer->row_desc(); -} - -MultiCastDataStreamerSourceOperator::MultiCastDataStreamerSourceOperator( - OperatorBuilderBase* operator_builder, const int consumer_id, - std::shared_ptr& data_streamer, const TDataStreamSink& sink) - : OperatorBase(operator_builder), - vectorized::RuntimeFilterConsumer(sink.dest_node_id, sink.runtime_filters, - data_streamer->row_desc(), _conjuncts), - _consumer_id(consumer_id), - _multi_cast_data_streamer(data_streamer), - _t_data_stream_sink(sink) {} - -Status MultiCastDataStreamerSourceOperator::prepare(doris::RuntimeState* state) { - RETURN_IF_ERROR(vectorized::RuntimeFilterConsumer::init(state)); - // init profile for runtime filter - RuntimeFilterConsumer::_init_profile(_multi_cast_data_streamer->profile()); - if (_t_data_stream_sink.__isset.output_exprs) { - RETURN_IF_ERROR(vectorized::VExpr::create_expr_trees(_t_data_stream_sink.output_exprs, - _output_expr_contexts)); - RETURN_IF_ERROR(vectorized::VExpr::prepare(_output_expr_contexts, state, row_desc())); - } - - if (_t_data_stream_sink.__isset.conjuncts) { - RETURN_IF_ERROR( - vectorized::VExpr::create_expr_trees(_t_data_stream_sink.conjuncts, _conjuncts)); - RETURN_IF_ERROR(vectorized::VExpr::prepare(_conjuncts, state, row_desc())); - } - return Status::OK(); -} - -Status MultiCastDataStreamerSourceOperator::open(doris::RuntimeState* state) { - if (_t_data_stream_sink.__isset.output_exprs) { - RETURN_IF_ERROR(vectorized::VExpr::open(_output_expr_contexts, state)); - } - if (_t_data_stream_sink.__isset.conjuncts) { - RETURN_IF_ERROR(vectorized::VExpr::open(_conjuncts, state)); - } - return _acquire_runtime_filter(false); -} - -bool MultiCastDataStreamerSourceOperator::runtime_filters_are_ready_or_timeout() { - return vectorized::RuntimeFilterConsumer::runtime_filters_are_ready_or_timeout(); -} - -bool MultiCastDataStreamerSourceOperator::can_read() { - return _multi_cast_data_streamer->can_read(_consumer_id); -} - -Status MultiCastDataStreamerSourceOperator::get_block(RuntimeState* state, vectorized::Block* block, - SourceState& source_state) { - bool eos = false; - vectorized::Block tmp_block; - vectorized::Block* output_block = block; - if (!_output_expr_contexts.empty()) { - output_block = &tmp_block; - } - _multi_cast_data_streamer->pull(_consumer_id, output_block, &eos); - - if (!_conjuncts.empty()) { - RETURN_IF_ERROR(vectorized::VExprContext::filter_block(_conjuncts, output_block, - output_block->columns())); - } - - if (!_output_expr_contexts.empty() && output_block->rows() > 0) { - RETURN_IF_ERROR(vectorized::VExprContext::get_output_block_after_execute_exprs( - _output_expr_contexts, *output_block, block, true)); - vectorized::materialize_block_inplace(*block); - } - if (eos) { - source_state = SourceState::FINISHED; - } - return Status::OK(); -} - -Status MultiCastDataStreamerSourceOperator::close(doris::RuntimeState* state) { - _multi_cast_data_streamer->close_sender(_consumer_id); - return OperatorBase::close(state); -} - -RuntimeProfile* MultiCastDataStreamerSourceOperator::get_runtime_profile() const { - return _multi_cast_data_streamer->profile(); -} - MultiCastDataStreamSourceLocalState::MultiCastDataStreamSourceLocalState(RuntimeState* state, OperatorXBase* parent) : Base(state, parent), diff --git a/be/src/pipeline/exec/multi_cast_data_stream_source.h b/be/src/pipeline/exec/multi_cast_data_stream_source.h index 8d14b4f266bec91..c3404a873c6a9a3 100644 --- a/be/src/pipeline/exec/multi_cast_data_stream_source.h +++ b/be/src/pipeline/exec/multi_cast_data_stream_source.h @@ -27,7 +27,6 @@ #include "vec/exec/runtime_filter_consumer.h" namespace doris { -class ExecNode; class RuntimeState; namespace vectorized { diff --git a/be/src/pipeline/exec/nested_loop_join_build_operator.cpp b/be/src/pipeline/exec/nested_loop_join_build_operator.cpp index 66612700fed37d9..09a3a976567aaed 100644 --- a/be/src/pipeline/exec/nested_loop_join_build_operator.cpp +++ b/be/src/pipeline/exec/nested_loop_join_build_operator.cpp @@ -20,11 +20,10 @@ #include #include "pipeline/exec/operator.h" +#include "vec/exec/join/vnested_loop_join_node.h" namespace doris::pipeline { -OPERATOR_CODE_GENERATOR(NestLoopJoinBuildOperator, StreamingOperator) - NestedLoopJoinBuildSinkLocalState::NestedLoopJoinBuildSinkLocalState(DataSinkOperatorXBase* parent, RuntimeState* state) : JoinBuildSinkLocalState( diff --git a/be/src/pipeline/exec/nested_loop_join_build_operator.h b/be/src/pipeline/exec/nested_loop_join_build_operator.h index da42e961f479a96..98377559f91b2e0 100644 --- a/be/src/pipeline/exec/nested_loop_join_build_operator.h +++ b/be/src/pipeline/exec/nested_loop_join_build_operator.h @@ -22,27 +22,8 @@ #include "operator.h" #include "pipeline/exec/join_build_sink_operator.h" #include "pipeline/pipeline_x/operator.h" -#include "vec/exec/join/vnested_loop_join_node.h" -namespace doris { -class ExecNode; - -namespace pipeline { - -class NestLoopJoinBuildOperatorBuilder final - : public OperatorBuilder { -public: - NestLoopJoinBuildOperatorBuilder(int32_t, ExecNode*); - - OperatorPtr build_operator() override; - bool is_sink() const override { return true; } -}; - -class NestLoopJoinBuildOperator final : public StreamingOperator { -public: - NestLoopJoinBuildOperator(OperatorBuilderBase* operator_builder, ExecNode* node); - bool can_write() override { return true; } -}; +namespace doris::pipeline { class NestedLoopJoinBuildSinkOperatorX; @@ -111,5 +92,4 @@ class NestedLoopJoinBuildSinkOperatorX final RowDescriptor _row_descriptor; }; -} // namespace pipeline -} // namespace doris +} // namespace doris::pipeline diff --git a/be/src/pipeline/exec/nested_loop_join_probe_operator.cpp b/be/src/pipeline/exec/nested_loop_join_probe_operator.cpp index c7afa2c399c249e..819e8f67616bf65 100644 --- a/be/src/pipeline/exec/nested_loop_join_probe_operator.cpp +++ b/be/src/pipeline/exec/nested_loop_join_probe_operator.cpp @@ -22,7 +22,6 @@ #include "pipeline/exec/operator.h" #include "vec/columns/column_filter_helper.h" #include "vec/core/block.h" -#include "vec/exec/join/vnested_loop_join_node.h" namespace doris { class RuntimeState; @@ -30,18 +29,6 @@ class RuntimeState; namespace doris::pipeline { -OPERATOR_CODE_GENERATOR(NestLoopJoinProbeOperator, StatefulOperator) - -Status NestLoopJoinProbeOperator::prepare(doris::RuntimeState* state) { - // just for speed up, the way is dangerous - _child_block = _node->get_left_block(); - return StatefulOperator::prepare(state); -} - -Status NestLoopJoinProbeOperator::close(doris::RuntimeState* state) { - return StatefulOperator::close(state); -} - NestedLoopJoinProbeLocalState::NestedLoopJoinProbeLocalState(RuntimeState* state, OperatorXBase* parent) : JoinProbeLocalState(state, diff --git a/be/src/pipeline/exec/nested_loop_join_probe_operator.h b/be/src/pipeline/exec/nested_loop_join_probe_operator.h index de9f11b437eece9..5c483a4348f87f2 100644 --- a/be/src/pipeline/exec/nested_loop_join_probe_operator.h +++ b/be/src/pipeline/exec/nested_loop_join_probe_operator.h @@ -24,33 +24,12 @@ #include "pipeline/exec/join_probe_operator.h" #include "pipeline/pipeline_x/operator.h" #include "util/simd/bits.h" -#include "vec/exec/join/vnested_loop_join_node.h" namespace doris { -class ExecNode; class RuntimeState; namespace pipeline { -class NestLoopJoinProbeOperatorBuilder final - : public OperatorBuilder { -public: - NestLoopJoinProbeOperatorBuilder(int32_t id, ExecNode* node); - - OperatorPtr build_operator() override; -}; - -class NestLoopJoinProbeOperator final : public StatefulOperator { -public: - NestLoopJoinProbeOperator(OperatorBuilderBase* operator_builder, ExecNode* node); - - Status prepare(RuntimeState* state) override; - - Status open(RuntimeState*) override { return Status::OK(); } - - Status close(RuntimeState* state) override; -}; - class NestedLoopJoinProbeOperatorX; class NestedLoopJoinProbeLocalState final : public JoinProbeLocalState { diff --git a/be/src/pipeline/exec/olap_scan_operator.h b/be/src/pipeline/exec/olap_scan_operator.h index 8f546826c88fdd3..f11cc6db0b982c4 100644 --- a/be/src/pipeline/exec/olap_scan_operator.h +++ b/be/src/pipeline/exec/olap_scan_operator.h @@ -25,10 +25,8 @@ #include "operator.h" #include "pipeline/exec/scan_operator.h" #include "pipeline/pipeline_x/operator.h" -#include "vec/exec/scan/vscan_node.h" namespace doris { -class ExecNode; namespace vectorized { class NewOlapScanner; diff --git a/be/src/pipeline/exec/olap_table_sink_operator.cpp b/be/src/pipeline/exec/olap_table_sink_operator.cpp index faffaf99c112ed5..60e6180469c764d 100644 --- a/be/src/pipeline/exec/olap_table_sink_operator.cpp +++ b/be/src/pipeline/exec/olap_table_sink_operator.cpp @@ -19,16 +19,8 @@ #include "common/status.h" -namespace doris { -class DataSink; -} // namespace doris - namespace doris::pipeline { -OperatorPtr OlapTableSinkOperatorBuilder::build_operator() { - return std::make_shared(this, _sink); -} - Status OlapTableSinkLocalState::close(RuntimeState* state, Status exec_status) { if (Base::_closed) { return Status::OK(); diff --git a/be/src/pipeline/exec/olap_table_sink_operator.h b/be/src/pipeline/exec/olap_table_sink_operator.h index 19c192160fd66ec..ad8bbab3ee9b18e 100644 --- a/be/src/pipeline/exec/olap_table_sink_operator.h +++ b/be/src/pipeline/exec/olap_table_sink_operator.h @@ -21,26 +21,7 @@ #include "pipeline/pipeline_x/operator.h" #include "vec/sink/volap_table_sink.h" -namespace doris { - -namespace pipeline { - -class OlapTableSinkOperatorBuilder final - : public DataSinkOperatorBuilder { -public: - OlapTableSinkOperatorBuilder(int32_t id, DataSink* sink) - : DataSinkOperatorBuilder(id, "OlapTableSinkOperator", sink) {} - - OperatorPtr build_operator() override; -}; - -class OlapTableSinkOperator final : public DataSinkOperator { -public: - OlapTableSinkOperator(OperatorBuilderBase* operator_builder, DataSink* sink) - : DataSinkOperator(operator_builder, sink) {} - - bool can_write() override { return _sink->can_write(); } -}; +namespace doris::pipeline { class OlapTableSinkOperatorX; @@ -102,5 +83,4 @@ class OlapTableSinkOperatorX final : public DataSinkOperatorX(this, _sink); -} - Status OlapTableSinkV2LocalState::close(RuntimeState* state, Status exec_status) { if (Base::_closed) { return Status::OK(); diff --git a/be/src/pipeline/exec/olap_table_sink_v2_operator.h b/be/src/pipeline/exec/olap_table_sink_v2_operator.h index 1fcd47162681193..da0700f8af33773 100644 --- a/be/src/pipeline/exec/olap_table_sink_v2_operator.h +++ b/be/src/pipeline/exec/olap_table_sink_v2_operator.h @@ -21,9 +21,7 @@ #include "pipeline/pipeline_x/operator.h" #include "vec/sink/volap_table_sink_v2.h" -namespace doris { - -namespace pipeline { +namespace doris::pipeline { class OlapTableSinkV2OperatorBuilder final : public DataSinkOperatorBuilder { @@ -104,5 +102,4 @@ class OlapTableSinkV2OperatorX final : public DataSinkOperatorX(this, _node); \ - } \ - \ - NAME::NAME(OperatorBuilderBase* operator_builder, ExecNode* node) \ - : SUBCLASS(operator_builder, node) {}; - namespace doris::pipeline { /** diff --git a/be/src/pipeline/exec/partition_sort_sink_operator.cpp b/be/src/pipeline/exec/partition_sort_sink_operator.cpp index abe2fde555e1647..77d9ca03e28a0b8 100644 --- a/be/src/pipeline/exec/partition_sort_sink_operator.cpp +++ b/be/src/pipeline/exec/partition_sort_sink_operator.cpp @@ -23,10 +23,6 @@ namespace doris::pipeline { -OperatorPtr PartitionSortSinkOperatorBuilder::build_operator() { - return std::make_shared(this, _node); -} - Status PartitionSortSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { RETURN_IF_ERROR(PipelineXSinkLocalState::init(state, info)); SCOPED_TIMER(exec_time_counter()); diff --git a/be/src/pipeline/exec/partition_sort_sink_operator.h b/be/src/pipeline/exec/partition_sort_sink_operator.h index 8602b096f516a27..4ac8a9d73f41a0d 100644 --- a/be/src/pipeline/exec/partition_sort_sink_operator.h +++ b/be/src/pipeline/exec/partition_sort_sink_operator.h @@ -24,31 +24,8 @@ #include "operator.h" #include "pipeline/pipeline_x/operator.h" #include "vec/common/sort/partition_sorter.h" -#include "vec/exec/vpartition_sort_node.h" -namespace doris { -class ExecNode; - -namespace pipeline { - -class PartitionSortSinkOperatorBuilder final - : public OperatorBuilder { -public: - PartitionSortSinkOperatorBuilder(int32_t id, ExecNode* sort_node) - : OperatorBuilder(id, "PartitionSortSinkOperator", sort_node) {} - - bool is_sink() const override { return true; } - - OperatorPtr build_operator() override; -}; - -class PartitionSortSinkOperator final : public StreamingOperator { -public: - PartitionSortSinkOperator(OperatorBuilderBase* operator_builder, ExecNode* sort_node) - : StreamingOperator(operator_builder, sort_node) {}; - - bool can_write() override { return true; } -}; +namespace doris::pipeline { class PartitionSortSinkOperatorX; class PartitionSortSinkLocalState : public PipelineXSinkLocalState { @@ -128,5 +105,4 @@ class PartitionSortSinkOperatorX final : public DataSinkOperatorX(this, _node); -} - Status PartitionSortSourceLocalState::init(RuntimeState* state, LocalStateInfo& info) { RETURN_IF_ERROR(PipelineXLocalState::init(state, info)); SCOPED_TIMER(exec_time_counter()); diff --git a/be/src/pipeline/exec/partition_sort_source_operator.h b/be/src/pipeline/exec/partition_sort_source_operator.h index 9d810db2039e700..5398a728e4ecf32 100644 --- a/be/src/pipeline/exec/partition_sort_source_operator.h +++ b/be/src/pipeline/exec/partition_sort_source_operator.h @@ -22,32 +22,12 @@ #include "common/status.h" #include "operator.h" #include "pipeline/pipeline_x/operator.h" -#include "vec/exec/vpartition_sort_node.h" namespace doris { -class ExecNode; class RuntimeState; namespace pipeline { -class PartitionSortSourceOperatorBuilder final - : public OperatorBuilder { -public: - PartitionSortSourceOperatorBuilder(int32_t id, ExecNode* sort_node) - : OperatorBuilder(id, "PartitionSortSourceOperator", sort_node) {} - - bool is_source() const override { return true; } - - OperatorPtr build_operator() override; -}; - -class PartitionSortSourceOperator final : public SourceOperator { -public: - PartitionSortSourceOperator(OperatorBuilderBase* operator_builder, ExecNode* sort_node) - : SourceOperator(operator_builder, sort_node) {} - Status open(RuntimeState*) override { return Status::OK(); } -}; - class PartitionSortSourceOperatorX; class PartitionSortSourceLocalState final : public PipelineXLocalState { diff --git a/be/src/pipeline/exec/partitioned_aggregation_sink_operator.cpp b/be/src/pipeline/exec/partitioned_aggregation_sink_operator.cpp index 78079a0ddf8c984..7eb09555aa8ab4c 100644 --- a/be/src/pipeline/exec/partitioned_aggregation_sink_operator.cpp +++ b/be/src/pipeline/exec/partitioned_aggregation_sink_operator.cpp @@ -122,9 +122,11 @@ void PartitionedAggSinkLocalState::update_profile(RuntimeProfile* child_profile) PartitionedAggSinkOperatorX::PartitionedAggSinkOperatorX(ObjectPool* pool, int operator_id, const TPlanNode& tnode, - const DescriptorTbl& descs) + const DescriptorTbl& descs, + bool require_bucket_distribution) : DataSinkOperatorX(operator_id, tnode.node_id) { - _agg_sink_operator = std::make_unique(pool, operator_id, tnode, descs); + _agg_sink_operator = std::make_unique(pool, operator_id, tnode, descs, + require_bucket_distribution); } Status PartitionedAggSinkOperatorX::init(const TPlanNode& tnode, RuntimeState* state) { diff --git a/be/src/pipeline/exec/partitioned_aggregation_sink_operator.h b/be/src/pipeline/exec/partitioned_aggregation_sink_operator.h index 1755cd866f270f8..1233f66b56294c6 100644 --- a/be/src/pipeline/exec/partitioned_aggregation_sink_operator.h +++ b/be/src/pipeline/exec/partitioned_aggregation_sink_operator.h @@ -294,7 +294,7 @@ class PartitionedAggSinkLocalState class PartitionedAggSinkOperatorX : public DataSinkOperatorX { public: PartitionedAggSinkOperatorX(ObjectPool* pool, int operator_id, const TPlanNode& tnode, - const DescriptorTbl& descs); + const DescriptorTbl& descs, bool require_bucket_distribution); ~PartitionedAggSinkOperatorX() override = default; Status init(const TDataSink& tsink) override { return Status::InternalError("{} should not init with TPlanNode", diff --git a/be/src/pipeline/exec/partitioned_aggregation_source_operator.h b/be/src/pipeline/exec/partitioned_aggregation_source_operator.h index eff1e7179c8d0dd..c1deb8af50db799 100644 --- a/be/src/pipeline/exec/partitioned_aggregation_source_operator.h +++ b/be/src/pipeline/exec/partitioned_aggregation_source_operator.h @@ -23,7 +23,6 @@ #include "pipeline/pipeline_x/operator.h" namespace doris { -class ExecNode; class RuntimeState; namespace pipeline { diff --git a/be/src/pipeline/exec/partitioned_hash_join_probe_operator.h b/be/src/pipeline/exec/partitioned_hash_join_probe_operator.h index 5bdc5278ffcf5e4..d650dd1590d2e0a 100644 --- a/be/src/pipeline/exec/partitioned_hash_join_probe_operator.h +++ b/be/src/pipeline/exec/partitioned_hash_join_probe_operator.h @@ -24,7 +24,6 @@ #include "pipeline/exec/hashjoin_build_sink.h" #include "pipeline/exec/hashjoin_probe_operator.h" #include "pipeline/exec/join_build_sink_operator.h" -#include "pipeline/pipeline_x/local_exchange/local_exchange_sink_operator.h" // LocalExchangeChannelIds #include "pipeline/pipeline_x/operator.h" #include "vec/runtime/partitioner.h" @@ -33,7 +32,7 @@ class RuntimeState; namespace pipeline { -using PartitionerType = vectorized::Crc32HashPartitioner; +using PartitionerType = vectorized::Crc32HashPartitioner; class PartitionedHashJoinProbeOperatorX; diff --git a/be/src/pipeline/exec/partitioned_hash_join_sink_operator.h b/be/src/pipeline/exec/partitioned_hash_join_sink_operator.h index 3f29e3093b6df01..9120392e1045549 100644 --- a/be/src/pipeline/exec/partitioned_hash_join_sink_operator.h +++ b/be/src/pipeline/exec/partitioned_hash_join_sink_operator.h @@ -24,17 +24,15 @@ #include "pipeline/exec/hashjoin_build_sink.h" #include "pipeline/exec/hashjoin_probe_operator.h" #include "pipeline/exec/join_build_sink_operator.h" -#include "pipeline/pipeline_x/local_exchange/local_exchange_sink_operator.h" // LocalExchangeChannelIds #include "pipeline/pipeline_x/operator.h" #include "vec/runtime/partitioner.h" namespace doris { -class ExecNode; class RuntimeState; namespace pipeline { -using PartitionerType = vectorized::Crc32HashPartitioner; +using PartitionerType = vectorized::Crc32HashPartitioner; class PartitionedHashJoinSinkOperatorX; diff --git a/be/src/pipeline/exec/repeat_operator.cpp b/be/src/pipeline/exec/repeat_operator.cpp index 42d009f0e76b7d7..fcf59f36543ba34 100644 --- a/be/src/pipeline/exec/repeat_operator.cpp +++ b/be/src/pipeline/exec/repeat_operator.cpp @@ -22,7 +22,6 @@ #include "common/logging.h" #include "pipeline/exec/operator.h" #include "vec/core/block.h" -#include "vec/exec/vrepeat_node.h" namespace doris { class RuntimeState; @@ -30,18 +29,6 @@ class RuntimeState; namespace doris::pipeline { -OPERATOR_CODE_GENERATOR(RepeatOperator, StatefulOperator) - -Status RepeatOperator::prepare(doris::RuntimeState* state) { - // just for speed up, the way is dangerous - _child_block = _node->get_child_block(); - return StatefulOperator::prepare(state); -} - -Status RepeatOperator::close(doris::RuntimeState* state) { - return StatefulOperator::close(state); -} - RepeatLocalState::RepeatLocalState(RuntimeState* state, OperatorXBase* parent) : Base(state, parent), _child_block(vectorized::Block::create_unique()), diff --git a/be/src/pipeline/exec/repeat_operator.h b/be/src/pipeline/exec/repeat_operator.h index 208b3d1e00565cc..922645d270d9153 100644 --- a/be/src/pipeline/exec/repeat_operator.h +++ b/be/src/pipeline/exec/repeat_operator.h @@ -21,30 +21,12 @@ #include "common/status.h" #include "pipeline/pipeline_x/operator.h" -#include "vec/exec/vrepeat_node.h" namespace doris { -class ExecNode; class RuntimeState; namespace pipeline { -class RepeatOperatorBuilder final : public OperatorBuilder { -public: - RepeatOperatorBuilder(int32_t id, ExecNode* repeat_node); - - OperatorPtr build_operator() override; -}; - -class RepeatOperator final : public StatefulOperator { -public: - RepeatOperator(OperatorBuilderBase* operator_builder, ExecNode* repeat_node); - - Status prepare(RuntimeState* state) override; - - Status close(RuntimeState* state) override; -}; - class RepeatOperatorX; class RepeatLocalState final : public PipelineXLocalState { diff --git a/be/src/pipeline/exec/result_file_sink_operator.cpp b/be/src/pipeline/exec/result_file_sink_operator.cpp index 65ad6599d5ddba9..f8cadad1df7e1d2 100644 --- a/be/src/pipeline/exec/result_file_sink_operator.cpp +++ b/be/src/pipeline/exec/result_file_sink_operator.cpp @@ -25,25 +25,9 @@ #include "runtime/buffer_control_block.h" #include "runtime/result_buffer_mgr.h" #include "vec/sink/vdata_stream_sender.h" -#include "vec/sink/vresult_file_sink.h" - -namespace doris { -class DataSink; -} // namespace doris namespace doris::pipeline { -ResultFileSinkOperatorBuilder::ResultFileSinkOperatorBuilder(int32_t id, DataSink* sink) - : DataSinkOperatorBuilder(id, "ResultSinkOperator", sink) {}; - -OperatorPtr ResultFileSinkOperatorBuilder::build_operator() { - return std::make_shared(this, _sink); -} - -ResultFileSinkOperator::ResultFileSinkOperator(OperatorBuilderBase* operator_builder, - DataSink* sink) - : DataSinkOperator(operator_builder, sink) {}; - ResultFileSinkLocalState::ResultFileSinkLocalState(DataSinkOperatorXBase* parent, RuntimeState* state) : AsyncWriterSink(parent, state), diff --git a/be/src/pipeline/exec/result_file_sink_operator.h b/be/src/pipeline/exec/result_file_sink_operator.h index 31b4b26206caafc..9dc91193510ce7b 100644 --- a/be/src/pipeline/exec/result_file_sink_operator.h +++ b/be/src/pipeline/exec/result_file_sink_operator.h @@ -23,25 +23,7 @@ #include "pipeline/pipeline_x/operator.h" #include "vec/sink/vresult_file_sink.h" -namespace doris { -class DataSink; - -namespace pipeline { - -class ResultFileSinkOperatorBuilder final - : public DataSinkOperatorBuilder { -public: - ResultFileSinkOperatorBuilder(int32_t id, DataSink* sink); - - OperatorPtr build_operator() override; -}; - -class ResultFileSinkOperator final : public DataSinkOperator { -public: - ResultFileSinkOperator(OperatorBuilderBase* operator_builder, DataSink* sink); - - bool can_write() override { return true; } -}; +namespace doris::pipeline { class ResultFileSinkOperatorX; class ResultFileSinkLocalState final @@ -127,5 +109,4 @@ class ResultFileSinkOperatorX final : public DataSinkOperatorX(this, _sink); -} - -ResultSinkOperator::ResultSinkOperator(OperatorBuilderBase* operator_builder, DataSink* sink) - : DataSinkOperator(operator_builder, sink) {}; - -bool ResultSinkOperator::can_write() { - return _sink->_sender->can_sink(); -} - Status ResultSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { RETURN_IF_ERROR(Base::init(state, info)); SCOPED_TIMER(exec_time_counter()); diff --git a/be/src/pipeline/exec/result_sink_operator.h b/be/src/pipeline/exec/result_sink_operator.h index aed9961a6d67716..71b8afce1719ec2 100644 --- a/be/src/pipeline/exec/result_sink_operator.h +++ b/be/src/pipeline/exec/result_sink_operator.h @@ -21,28 +21,12 @@ #include "operator.h" #include "pipeline/pipeline_x/operator.h" -#include "vec/sink/vresult_sink.h" namespace doris { -class DataSink; class PipBufferControlBlock; namespace pipeline { -class ResultSinkOperatorBuilder final : public DataSinkOperatorBuilder { -public: - ResultSinkOperatorBuilder(int32_t id, DataSink* sink); - - OperatorPtr build_operator() override; -}; - -class ResultSinkOperator final : public DataSinkOperator { -public: - ResultSinkOperator(OperatorBuilderBase* operator_builder, DataSink* sink); - - bool can_write() override; -}; - class ResultSinkLocalState final : public PipelineXSinkLocalState { ENABLE_FACTORY_CREATOR(ResultSinkLocalState); using Base = PipelineXSinkLocalState; diff --git a/be/src/pipeline/exec/scan_operator.cpp b/be/src/pipeline/exec/scan_operator.cpp index 9d32f0e25abf602..37da4d972883670 100644 --- a/be/src/pipeline/exec/scan_operator.cpp +++ b/be/src/pipeline/exec/scan_operator.cpp @@ -45,39 +45,6 @@ namespace doris::pipeline { -OPERATOR_CODE_GENERATOR(ScanOperator, SourceOperator) - -bool ScanOperator::can_read() { - if (!_node->_opened) { - return _node->_should_create_scanner || _node->ready_to_open(); - } else { - // If scanner meet any error, done == true - if (_node->_eos || _node->_scanner_ctx->done()) { - // _eos: need eos - // _scanner_ctx->done(): need finish - // _scanner_ctx->no_schedule(): should schedule _scanner_ctx - return true; - } else { - return _node->ready_to_read(); // there are some blocks to process - } - } -} - -bool ScanOperator::runtime_filters_are_ready_or_timeout() { - return _node->runtime_filters_are_ready_or_timeout(); -} - -std::string ScanOperator::debug_string() const { - fmt::memory_buffer debug_string_buffer; - 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, ", scanner ctx detail = {}", - _node->_scanner_ctx->debug_string()); - } - return fmt::to_string(debug_string_buffer); -} - #define RETURN_IF_PUSH_DOWN(stmt, status) \ if (pdt == vectorized::VScanNode::PushDownType::UNACCEPTABLE) { \ status = stmt; \ diff --git a/be/src/pipeline/exec/scan_operator.h b/be/src/pipeline/exec/scan_operator.h index 3ebccb58a8c604f..6b62af1eefd9acf 100644 --- a/be/src/pipeline/exec/scan_operator.h +++ b/be/src/pipeline/exec/scan_operator.h @@ -29,9 +29,6 @@ #include "runtime/descriptors.h" #include "vec/exec/scan/vscan_node.h" -namespace doris { -class ExecNode; -} // namespace doris namespace doris::vectorized { class ScannerDelegate; } @@ -39,24 +36,6 @@ class ScannerDelegate; namespace doris::pipeline { class PipScannerContext; -class ScanOperatorBuilder : public OperatorBuilder { -public: - ScanOperatorBuilder(int32_t id, ExecNode* exec_node); - bool is_source() const override { return true; } - OperatorPtr build_operator() override; -}; - -class ScanOperator : public SourceOperator { -public: - ScanOperator(OperatorBuilderBase* operator_builder, ExecNode* scan_node); - - bool can_read() override; // for source - - bool runtime_filters_are_ready_or_timeout() override; - - std::string debug_string() const override; -}; - class ScanLocalStateBase : public PipelineXLocalState<>, public vectorized::RuntimeFilterConsumer { public: ScanLocalStateBase(RuntimeState* state, OperatorXBase* parent) diff --git a/be/src/pipeline/exec/schema_scan_operator.cpp b/be/src/pipeline/exec/schema_scan_operator.cpp index 2d32e21d991960f..e4a3a8c1ca24681 100644 --- a/be/src/pipeline/exec/schema_scan_operator.cpp +++ b/be/src/pipeline/exec/schema_scan_operator.cpp @@ -24,7 +24,6 @@ #include "pipeline/exec/operator.h" #include "util/runtime_profile.h" #include "vec/data_types/data_type_factory.hpp" -#include "vec/exec/vschema_scan_node.h" namespace doris { class RuntimeState; @@ -32,18 +31,6 @@ class RuntimeState; namespace doris::pipeline { -OPERATOR_CODE_GENERATOR(SchemaScanOperator, SourceOperator) - -Status SchemaScanOperator::open(RuntimeState* state) { - return _node->open(state); -} - -Status SchemaScanOperator::close(RuntimeState* state) { - RETURN_IF_ERROR(SourceOperator::close(state)); - RETURN_IF_ERROR(_node->close(state)); - return Status::OK(); -} - Status SchemaScanLocalState::init(RuntimeState* state, LocalStateInfo& info) { RETURN_IF_ERROR(PipelineXLocalState<>::init(state, info)); diff --git a/be/src/pipeline/exec/schema_scan_operator.h b/be/src/pipeline/exec/schema_scan_operator.h index bd336132efb2b72..f07a0e854e70c07 100644 --- a/be/src/pipeline/exec/schema_scan_operator.h +++ b/be/src/pipeline/exec/schema_scan_operator.h @@ -25,30 +25,11 @@ #include "vec/exec/vschema_scan_node.h" namespace doris { -class ExecNode; class RuntimeState; } // namespace doris namespace doris::pipeline { -class SchemaScanOperatorBuilder : public OperatorBuilder { -public: - SchemaScanOperatorBuilder(int32_t id, ExecNode* exec_node); - bool is_source() const override { return true; } - OperatorPtr build_operator() override; -}; - -class SchemaScanOperator : public SourceOperator { -public: - SchemaScanOperator(OperatorBuilderBase* operator_builder, ExecNode* scan_node); - - bool can_read() override { return true; } - - Status open(RuntimeState* state) override; - - Status close(RuntimeState* state) override; -}; - class SchemaScanOperatorX; class SchemaScanLocalState final : public PipelineXLocalState<> { public: diff --git a/be/src/pipeline/exec/select_operator.h b/be/src/pipeline/exec/select_operator.h index 4fd929e323ba7dd..4bdc5a9e42d2987 100644 --- a/be/src/pipeline/exec/select_operator.h +++ b/be/src/pipeline/exec/select_operator.h @@ -21,24 +21,8 @@ #include "operator.h" #include "pipeline/pipeline_x/operator.h" -#include "vec/exec/vselect_node.h" -namespace doris { -class ExecNode; - -namespace pipeline { - -class SelectOperatorBuilder final : public OperatorBuilder { -public: - SelectOperatorBuilder(int32_t id, ExecNode* select_node); - - OperatorPtr build_operator() override; -}; - -class SelectOperator final : public StreamingOperator { -public: - SelectOperator(OperatorBuilderBase* operator_builder, ExecNode* select_node); -}; +namespace doris::pipeline { class SelectOperatorX; class SelectLocalState final : public PipelineXLocalState { @@ -72,5 +56,4 @@ class SelectOperatorX final : public StreamingOperatorX { [[nodiscard]] bool is_source() const override { return false; } }; -} // namespace pipeline -} // namespace doris +} // namespace doris::pipeline diff --git a/be/src/pipeline/exec/set_probe_sink_operator.cpp b/be/src/pipeline/exec/set_probe_sink_operator.cpp index 744c8b17e2257de..81c0cd463c1284a 100644 --- a/be/src/pipeline/exec/set_probe_sink_operator.cpp +++ b/be/src/pipeline/exec/set_probe_sink_operator.cpp @@ -23,10 +23,8 @@ #include "pipeline/exec/operator.h" #include "vec/common/hash_table/hash_table_set_probe.h" -#include "vec/exec/vset_operation_node.h" namespace doris { -class ExecNode; class RuntimeState; namespace vectorized { @@ -36,41 +34,6 @@ class Block; namespace doris::pipeline { -template -SetProbeSinkOperatorBuilder::SetProbeSinkOperatorBuilder(int32_t id, int child_id, - ExecNode* set_node) - : OperatorBuilder>(id, builder_name, set_node), - _child_id(child_id) {} - -template -OperatorPtr SetProbeSinkOperatorBuilder::build_operator() { - return std::make_shared>(this, _child_id, this->_node); -} - -template -SetProbeSinkOperator::SetProbeSinkOperator(OperatorBuilderBase* operator_builder, - int child_id, ExecNode* set_node) - : StreamingOperator>(operator_builder, - set_node), - _child_id(child_id) {} - -template -Status SetProbeSinkOperator::sink(RuntimeState* state, vectorized::Block* block, - SourceState source_state) { - return this->_node->sink_probe(state, _child_id, block, source_state == SourceState::FINISHED); -} - -template -bool SetProbeSinkOperator::can_write() { - DCHECK_GT(_child_id, 0); - return this->_node->is_child_finished(_child_id - 1); -} - -template class SetProbeSinkOperatorBuilder; -template class SetProbeSinkOperatorBuilder; -template class SetProbeSinkOperator; -template class SetProbeSinkOperator; - template Status SetProbeSinkOperatorX::init(const TPlanNode& tnode, RuntimeState* state) { DataSinkOperatorX>::_name = "SET_PROBE_SINK_OPERATOR"; diff --git a/be/src/pipeline/exec/set_probe_sink_operator.h b/be/src/pipeline/exec/set_probe_sink_operator.h index 9f80f03966b1f16..499eeee0d6205b0 100644 --- a/be/src/pipeline/exec/set_probe_sink_operator.h +++ b/be/src/pipeline/exec/set_probe_sink_operator.h @@ -22,10 +22,8 @@ #include "common/status.h" #include "operator.h" #include "pipeline/pipeline_x/operator.h" -#include "vec/exec/vset_operation_node.h" namespace doris { -class ExecNode; class RuntimeState; namespace vectorized { @@ -36,37 +34,6 @@ struct HashTableProbe; namespace pipeline { -template -class SetProbeSinkOperatorBuilder final - : public OperatorBuilder> { -private: - constexpr static auto builder_name = - is_intersect ? "IntersectProbeSinkOperator" : "ExceptProbeSinkOperator"; - -public: - SetProbeSinkOperatorBuilder(int32_t id, int child_id, ExecNode* set_node); - [[nodiscard]] bool is_sink() const override { return true; } - - OperatorPtr build_operator() override; - -private: - int _child_id; -}; - -template -class SetProbeSinkOperator : public StreamingOperator> { -public: - SetProbeSinkOperator(OperatorBuilderBase* operator_builder, int child_id, ExecNode* set_node); - - bool can_write() override; - - Status sink(RuntimeState* state, vectorized::Block* block, SourceState source_state) override; - Status open(RuntimeState* /*state*/) override { return Status::OK(); } - -private: - int _child_id; -}; - template class SetProbeSinkOperatorX; diff --git a/be/src/pipeline/exec/set_sink_operator.cpp b/be/src/pipeline/exec/set_sink_operator.cpp index 2042e3eb1a1d515..796174d080a88e2 100644 --- a/be/src/pipeline/exec/set_sink_operator.cpp +++ b/be/src/pipeline/exec/set_sink_operator.cpp @@ -22,34 +22,9 @@ #include "pipeline/exec/operator.h" #include "vec/common/hash_table/hash_table_set_build.h" #include "vec/core/materialize_block.h" -#include "vec/exec/vset_operation_node.h" - -namespace doris { -class ExecNode; -} // namespace doris namespace doris::pipeline { -template -SetSinkOperatorBuilder::SetSinkOperatorBuilder(int32_t id, ExecNode* set_node) - : OperatorBuilder>(id, builder_name, set_node) { -} - -template -OperatorPtr SetSinkOperatorBuilder::build_operator() { - return std::make_shared>(this, this->_node); -} - -template -SetSinkOperator::SetSinkOperator( - OperatorBuilderBase* builder, vectorized::VSetOperationNode* set_node) - : StreamingOperator>(builder, set_node) {} - -template class SetSinkOperatorBuilder; -template class SetSinkOperatorBuilder; -template class SetSinkOperator; -template class SetSinkOperator; - template Status SetSinkOperatorX::sink(RuntimeState* state, vectorized::Block* in_block, bool eos) { diff --git a/be/src/pipeline/exec/set_sink_operator.h b/be/src/pipeline/exec/set_sink_operator.h index 2a6bb63c02e8158..8894b8b15f2252f 100644 --- a/be/src/pipeline/exec/set_sink_operator.h +++ b/be/src/pipeline/exec/set_sink_operator.h @@ -22,10 +22,8 @@ #include "olap/olap_common.h" #include "operator.h" #include "pipeline/pipeline_x/operator.h" -#include "vec/exec/vset_operation_node.h" namespace doris { -class ExecNode; namespace vectorized { template @@ -34,32 +32,6 @@ struct HashTableBuild; namespace pipeline { -template -class SetSinkOperatorBuilder final - : public OperatorBuilder> { -private: - constexpr static auto builder_name = - is_intersect ? "IntersectSinkOperator" : "ExceptSinkOperator"; - -public: - SetSinkOperatorBuilder(int32_t id, ExecNode* set_node); - [[nodiscard]] bool is_sink() const override { return true; } - - OperatorPtr build_operator() override; -}; - -template -class SetSinkOperator : public StreamingOperator> { -public: - SetSinkOperator(OperatorBuilderBase* operator_builder, - vectorized::VSetOperationNode* set_node); - - bool can_write() override { return true; } - -private: - vectorized::VSetOperationNode* _set_node = nullptr; -}; - template class SetSinkOperatorX; diff --git a/be/src/pipeline/exec/set_source_operator.cpp b/be/src/pipeline/exec/set_source_operator.cpp index 88d38d325af0034..bdb844e70e80954 100644 --- a/be/src/pipeline/exec/set_source_operator.cpp +++ b/be/src/pipeline/exec/set_source_operator.cpp @@ -21,34 +21,9 @@ #include "common/status.h" #include "pipeline/exec/operator.h" -#include "vec/exec/vset_operation_node.h" - -namespace doris { -class ExecNode; -} // namespace doris namespace doris::pipeline { -template -SetSourceOperatorBuilder::SetSourceOperatorBuilder(int32_t id, ExecNode* set_node) - : OperatorBuilder>(id, builder_name, set_node) { -} - -template -OperatorPtr SetSourceOperatorBuilder::build_operator() { - return std::make_shared>(this, this->_node); -} - -template -SetSourceOperator::SetSourceOperator( - OperatorBuilderBase* builder, vectorized::VSetOperationNode* set_node) - : SourceOperator>(builder, set_node) {} - -template class SetSourceOperatorBuilder; -template class SetSourceOperatorBuilder; -template class SetSourceOperator; -template class SetSourceOperator; - template Status SetSourceLocalState::init(RuntimeState* state, LocalStateInfo& info) { RETURN_IF_ERROR(Base::init(state, info)); diff --git a/be/src/pipeline/exec/set_source_operator.h b/be/src/pipeline/exec/set_source_operator.h index 1c5cf162940b40f..94487507c26e3e3 100644 --- a/be/src/pipeline/exec/set_source_operator.h +++ b/be/src/pipeline/exec/set_source_operator.h @@ -22,37 +22,12 @@ #include "common/status.h" #include "operator.h" #include "pipeline/pipeline_x/operator.h" -#include "vec/exec/vset_operation_node.h" namespace doris { -class ExecNode; class RuntimeState; namespace pipeline { -template -class SetSourceOperatorBuilder - : public OperatorBuilder> { -private: - constexpr static auto builder_name = - is_intersect ? "IntersectSourceOperator" : "ExceptSourceOperator"; - -public: - SetSourceOperatorBuilder(int32_t id, ExecNode* set_node); - [[nodiscard]] bool is_source() const override { return true; } - - OperatorPtr build_operator() override; -}; - -template -class SetSourceOperator : public SourceOperator> { -public: - SetSourceOperator(OperatorBuilderBase* builder, - vectorized::VSetOperationNode* set_node); - - Status open(RuntimeState* /*state*/) override { return Status::OK(); } -}; - template class SetSourceOperatorX; diff --git a/be/src/pipeline/exec/sort_sink_operator.cpp b/be/src/pipeline/exec/sort_sink_operator.cpp index 91ae687510c985d..7c9f40d1b5956f1 100644 --- a/be/src/pipeline/exec/sort_sink_operator.cpp +++ b/be/src/pipeline/exec/sort_sink_operator.cpp @@ -26,8 +26,6 @@ namespace doris::pipeline { -OPERATOR_CODE_GENERATOR(SortSinkOperator, StreamingOperator) - Status SortSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { RETURN_IF_ERROR(Base::init(state, info)); SCOPED_TIMER(exec_time_counter()); diff --git a/be/src/pipeline/exec/sort_sink_operator.h b/be/src/pipeline/exec/sort_sink_operator.h index ad9c23401b4c69d..8298dc980b6c7d6 100644 --- a/be/src/pipeline/exec/sort_sink_operator.h +++ b/be/src/pipeline/exec/sort_sink_operator.h @@ -22,28 +22,8 @@ #include "operator.h" #include "pipeline/pipeline_x/operator.h" #include "vec/core/field.h" -#include "vec/exec/vsort_node.h" -namespace doris { -class ExecNode; - -namespace pipeline { - -class SortSinkOperatorBuilder final : public OperatorBuilder { -public: - SortSinkOperatorBuilder(int32_t id, ExecNode* sort_node); - - bool is_sink() const override { return true; } - - OperatorPtr build_operator() override; -}; - -class SortSinkOperator final : public StreamingOperator { -public: - SortSinkOperator(OperatorBuilderBase* operator_builder, ExecNode* sort_node); - - bool can_write() override { return true; } -}; +namespace doris::pipeline { enum class SortAlgorithm { HEAP_SORT, TOPN_SORT, FULL_SORT }; @@ -132,5 +112,4 @@ class SortSinkOperatorX final : public DataSinkOperatorX { const std::vector _partition_exprs; }; -} // namespace pipeline -} // namespace doris +} // namespace doris::pipeline diff --git a/be/src/pipeline/exec/sort_source_operator.cpp b/be/src/pipeline/exec/sort_source_operator.cpp index f7d6b6ae56c3c9b..34bfffb8d9fd2e2 100644 --- a/be/src/pipeline/exec/sort_source_operator.cpp +++ b/be/src/pipeline/exec/sort_source_operator.cpp @@ -23,8 +23,6 @@ namespace doris::pipeline { -OPERATOR_CODE_GENERATOR(SortSourceOperator, SourceOperator) - SortLocalState::SortLocalState(RuntimeState* state, OperatorXBase* parent) : PipelineXLocalState(state, parent) {} diff --git a/be/src/pipeline/exec/sort_source_operator.h b/be/src/pipeline/exec/sort_source_operator.h index 1af30bfd05f49f8..43c4934b977415c 100644 --- a/be/src/pipeline/exec/sort_source_operator.h +++ b/be/src/pipeline/exec/sort_source_operator.h @@ -22,29 +22,12 @@ #include "common/status.h" #include "operator.h" #include "pipeline/pipeline_x/operator.h" -#include "vec/exec/vsort_node.h" namespace doris { -class ExecNode; class RuntimeState; namespace pipeline { -class SortSourceOperatorBuilder final : public OperatorBuilder { -public: - SortSourceOperatorBuilder(int32_t id, ExecNode* sort_node); - - bool is_source() const override { return true; } - - OperatorPtr build_operator() override; -}; - -class SortSourceOperator final : public SourceOperator { -public: - SortSourceOperator(OperatorBuilderBase* operator_builder, ExecNode* sort_node); - Status open(RuntimeState*) override { return Status::OK(); } -}; - class SortSourceOperatorX; class SortLocalState final : public PipelineXLocalState { public: diff --git a/be/src/pipeline/exec/streaming_aggregation_sink_operator.cpp b/be/src/pipeline/exec/streaming_aggregation_sink_operator.cpp deleted file mode 100644 index 5805b209f4f7f8b..000000000000000 --- a/be/src/pipeline/exec/streaming_aggregation_sink_operator.cpp +++ /dev/null @@ -1,93 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#include "streaming_aggregation_sink_operator.h" - -#include - -#include - -#include "common/compiler_util.h" // IWYU pragma: keep -#include "pipeline/exec/data_queue.h" -#include "pipeline/exec/operator.h" -#include "vec/exec/vaggregation_node.h" - -namespace doris { -class ExecNode; -class RuntimeState; -} // namespace doris - -namespace doris::pipeline { - -StreamingAggSinkOperator::StreamingAggSinkOperator(OperatorBuilderBase* operator_builder, - ExecNode* agg_node, - std::shared_ptr queue) - : StreamingOperator(operator_builder, agg_node), _data_queue(std::move(queue)) {} - -Status StreamingAggSinkOperator::prepare(RuntimeState* state) { - RETURN_IF_ERROR(StreamingOperator::prepare(state)); - _queue_byte_size_counter = - ADD_COUNTER(_node->runtime_profile(), "MaxSizeInBlockQueue", TUnit::BYTES); - _queue_size_counter = ADD_COUNTER(_node->runtime_profile(), "MaxSizeOfBlockQueue", TUnit::UNIT); - return Status::OK(); -} - -bool StreamingAggSinkOperator::can_write() { - // sink and source in diff threads - return _data_queue->has_enough_space_to_push(); -} - -Status StreamingAggSinkOperator::sink(RuntimeState* state, vectorized::Block* in_block, - SourceState source_state) { - Status ret = Status::OK(); - if (in_block && in_block->rows() > 0) { - auto block_from_ctx = _data_queue->get_free_block(); - RETURN_IF_ERROR(_node->do_pre_agg(in_block, block_from_ctx.get())); - if (block_from_ctx->rows() == 0) { - _data_queue->push_free_block(std::move(block_from_ctx)); - } else { - _data_queue->push_block(std::move(block_from_ctx)); - } - } - - if (UNLIKELY(source_state == SourceState::FINISHED)) { - _data_queue->set_finish(); - } - return Status::OK(); -} - -Status StreamingAggSinkOperator::close(RuntimeState* state) { - if (_data_queue && !_data_queue->is_finish()) { - // finish should be set, if not set here means error. - _data_queue->set_canceled(); - } - if (_data_queue) { - COUNTER_SET(_queue_size_counter, _data_queue->max_size_of_queue()); - COUNTER_SET(_queue_byte_size_counter, _data_queue->max_bytes_in_queue()); - } - return StreamingOperator::close(state); -} - -StreamingAggSinkOperatorBuilder::StreamingAggSinkOperatorBuilder(int32_t id, ExecNode* exec_node, - std::shared_ptr queue) - : OperatorBuilder(id, "StreamingAggSinkOperator", exec_node), - _data_queue(std::move(queue)) {} - -OperatorPtr StreamingAggSinkOperatorBuilder::build_operator() { - return std::make_shared(this, _node, _data_queue); -} -} // namespace doris::pipeline diff --git a/be/src/pipeline/exec/streaming_aggregation_sink_operator.h b/be/src/pipeline/exec/streaming_aggregation_sink_operator.h deleted file mode 100644 index 99e94e2d5859571..000000000000000 --- a/be/src/pipeline/exec/streaming_aggregation_sink_operator.h +++ /dev/null @@ -1,76 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#pragma once - -#include - -#include - -#include "aggregation_sink_operator.h" -#include "aggregation_source_operator.h" -#include "common/status.h" -#include "operator.h" -#include "pipeline/pipeline_x/operator.h" -#include "util/runtime_profile.h" -#include "vec/core/block.h" -#include "vec/exec/vaggregation_node.h" - -namespace doris { -class ExecNode; -class RuntimeState; - -namespace pipeline { -class DataQueue; - -class StreamingAggSinkOperatorBuilder final : public OperatorBuilder { -public: - StreamingAggSinkOperatorBuilder(int32_t, ExecNode*, std::shared_ptr); - - OperatorPtr build_operator() override; - - bool is_sink() const override { return true; } - bool is_source() const override { return false; } - -private: - std::shared_ptr _data_queue; -}; - -class StreamingAggSinkOperator final : public StreamingOperator { -public: - StreamingAggSinkOperator(OperatorBuilderBase* operator_builder, ExecNode*, - std::shared_ptr); - - Status prepare(RuntimeState*) override; - - Status sink(RuntimeState* state, vectorized::Block* block, SourceState source_state) override; - - bool can_write() override; - - Status close(RuntimeState* state) override; - -private: - vectorized::Block _preagg_block = vectorized::Block(); - - RuntimeProfile::Counter* _queue_byte_size_counter = nullptr; - RuntimeProfile::Counter* _queue_size_counter = nullptr; - - std::shared_ptr _data_queue; -}; - -} // namespace pipeline -} // namespace doris diff --git a/be/src/pipeline/exec/streaming_aggregation_source_operator.cpp b/be/src/pipeline/exec/streaming_aggregation_source_operator.cpp deleted file mode 100644 index 9c9a014046a0433..000000000000000 --- a/be/src/pipeline/exec/streaming_aggregation_source_operator.cpp +++ /dev/null @@ -1,76 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#include "streaming_aggregation_source_operator.h" - -#include - -#include "pipeline/exec/data_queue.h" -#include "pipeline/exec/operator.h" -#include "runtime/descriptors.h" -#include "util/runtime_profile.h" -#include "vec/core/block.h" -#include "vec/exec/vaggregation_node.h" - -namespace doris { -class ExecNode; -class RuntimeState; - -namespace pipeline { - -StreamingAggSourceOperator::StreamingAggSourceOperator(OperatorBuilderBase* templ, ExecNode* node, - std::shared_ptr queue) - : SourceOperator(templ, node), _data_queue(std::move(queue)) {} - -bool StreamingAggSourceOperator::can_read() { - return _data_queue->has_data_or_finished(); -} - -Status StreamingAggSourceOperator::get_block(RuntimeState* state, vectorized::Block* block, - SourceState& source_state) { - bool eos = false; - if (!_data_queue->data_exhausted()) { - std::unique_ptr agg_block; - RETURN_IF_ERROR(_data_queue->get_block_from_queue(&agg_block)); - - if (_data_queue->data_exhausted()) { - RETURN_IF_ERROR(_node->pull(state, block, &eos)); - } else { - block->swap(*agg_block); - agg_block->clear_column_data(_node->row_desc().num_materialized_slots()); - _data_queue->push_free_block(std::move(agg_block)); - } - } else { - RETURN_IF_ERROR(_node->pull(state, block, &eos)); - } - - source_state = eos ? SourceState::FINISHED : SourceState::DEPEND_ON_SOURCE; - - return Status::OK(); -} - -StreamingAggSourceOperatorBuilder::StreamingAggSourceOperatorBuilder( - int32_t id, ExecNode* exec_node, std::shared_ptr queue) - : OperatorBuilder(id, "StreamingAggSourceOperator", exec_node), - _data_queue(std::move(queue)) {} - -OperatorPtr StreamingAggSourceOperatorBuilder::build_operator() { - return std::make_shared(this, _node, _data_queue); -} - -} // namespace pipeline -} // namespace doris diff --git a/be/src/pipeline/exec/streaming_aggregation_source_operator.h b/be/src/pipeline/exec/streaming_aggregation_source_operator.h deleted file mode 100644 index 89dbaab058ac824..000000000000000 --- a/be/src/pipeline/exec/streaming_aggregation_source_operator.h +++ /dev/null @@ -1,64 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. -#pragma once - -#include - -#include - -#include "common/status.h" -#include "operator.h" -#include "pipeline/exec/aggregation_source_operator.h" -#include "pipeline/pipeline_x/operator.h" -#include "vec/exec/vaggregation_node.h" - -namespace doris { -class ExecNode; -class RuntimeState; - -namespace vectorized { -class Block; -} // namespace vectorized -namespace pipeline { -class DataQueue; - -class StreamingAggSourceOperatorBuilder final - : public OperatorBuilder { -public: - StreamingAggSourceOperatorBuilder(int32_t, ExecNode*, std::shared_ptr); - - bool is_source() const override { return true; } - - OperatorPtr build_operator() override; - -private: - std::shared_ptr _data_queue; -}; - -class StreamingAggSourceOperator final : public SourceOperator { -public: - StreamingAggSourceOperator(OperatorBuilderBase*, ExecNode*, std::shared_ptr); - bool can_read() override; - Status get_block(RuntimeState*, vectorized::Block*, SourceState& source_state) override; - Status open(RuntimeState*) override { return Status::OK(); } - -private: - std::shared_ptr _data_queue; -}; - -} // namespace pipeline -} // namespace doris diff --git a/be/src/pipeline/exec/table_function_operator.cpp b/be/src/pipeline/exec/table_function_operator.cpp index 9256d1deb2b0723..cb547688595de6f 100644 --- a/be/src/pipeline/exec/table_function_operator.cpp +++ b/be/src/pipeline/exec/table_function_operator.cpp @@ -29,18 +29,6 @@ class RuntimeState; namespace doris::pipeline { -OPERATOR_CODE_GENERATOR(TableFunctionOperator, StatefulOperator) - -Status TableFunctionOperator::prepare(doris::RuntimeState* state) { - // just for speed up, the way is dangerous - _child_block = _node->get_child_block(); - return StatefulOperator::prepare(state); -} - -Status TableFunctionOperator::close(doris::RuntimeState* state) { - return StatefulOperator::close(state); -} - TableFunctionLocalState::TableFunctionLocalState(RuntimeState* state, OperatorXBase* parent) : PipelineXLocalState<>(state, parent), _child_block(vectorized::Block::create_unique()) {} diff --git a/be/src/pipeline/exec/table_function_operator.h b/be/src/pipeline/exec/table_function_operator.h index 8a7b7bd43d45d19..5b9457a474b79b2 100644 --- a/be/src/pipeline/exec/table_function_operator.h +++ b/be/src/pipeline/exec/table_function_operator.h @@ -25,28 +25,11 @@ #include "vec/exec/vtable_function_node.h" namespace doris { -class ExecNode; class RuntimeState; } // namespace doris namespace doris::pipeline { -class TableFunctionOperatorBuilder final : public OperatorBuilder { -public: - TableFunctionOperatorBuilder(int32_t id, ExecNode* node); - - OperatorPtr build_operator() override; -}; - -class TableFunctionOperator final : public StatefulOperator { -public: - TableFunctionOperator(OperatorBuilderBase* operator_builder, ExecNode* node); - - Status prepare(RuntimeState* state) override; - - Status close(RuntimeState* state) override; -}; - class TableFunctionOperatorX; class TableFunctionLocalState final : public PipelineXLocalState<> { public: diff --git a/be/src/pipeline/exec/table_sink_operator.h b/be/src/pipeline/exec/table_sink_operator.h deleted file mode 100644 index 46843c23f870b36..000000000000000 --- a/be/src/pipeline/exec/table_sink_operator.h +++ /dev/null @@ -1,49 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#pragma once - -#include "exec/data_sink.h" -#include "operator.h" - -namespace doris { - -namespace pipeline { - -// used for VMysqlTableSink, VJdbcTableSink and VOdbcTableSink. -class TableSinkOperatorBuilder final : public DataSinkOperatorBuilder { -public: - TableSinkOperatorBuilder(int32_t id, DataSink* sink) - : DataSinkOperatorBuilder(id, "TableSinkOperator", sink) {} - - OperatorPtr build_operator() override; -}; - -class TableSinkOperator final : public DataSinkOperator { -public: - TableSinkOperator(OperatorBuilderBase* operator_builder, DataSink* sink) - : DataSinkOperator(operator_builder, sink) {} - - bool can_write() override { return _sink->can_write(); } -}; - -OperatorPtr TableSinkOperatorBuilder::build_operator() { - return std::make_shared(this, _sink); -} - -} // namespace pipeline -} // namespace doris diff --git a/be/src/pipeline/exec/union_sink_operator.cpp b/be/src/pipeline/exec/union_sink_operator.cpp index 40344882a84e530..aa10468fc754d73 100644 --- a/be/src/pipeline/exec/union_sink_operator.cpp +++ b/be/src/pipeline/exec/union_sink_operator.cpp @@ -26,73 +26,8 @@ #include "runtime/runtime_state.h" #include "util/runtime_profile.h" -namespace doris { -class ExecNode; -} // namespace doris - namespace doris::pipeline { -UnionSinkOperatorBuilder::UnionSinkOperatorBuilder(int32_t id, int child_id, ExecNode* node, - std::shared_ptr queue) - : OperatorBuilder(id, "UnionSinkOperator", node), - _cur_child_id(child_id), - _data_queue(queue) {}; - -UnionSinkOperator::UnionSinkOperator(OperatorBuilderBase* operator_builder, int child_id, - ExecNode* node, std::shared_ptr queue) - : StreamingOperator(operator_builder, node), _cur_child_id(child_id), _data_queue(queue) {}; - -OperatorPtr UnionSinkOperatorBuilder::build_operator() { - return std::make_shared(this, _cur_child_id, _node, _data_queue); -} - -Status UnionSinkOperator::sink(RuntimeState* state, vectorized::Block* in_block, - SourceState source_state) { - if (_output_block == nullptr) { - _output_block = _data_queue->get_free_block(_cur_child_id); - } - - if (_cur_child_id < _node->get_first_materialized_child_idx()) { //pass_through - if (in_block->rows() > 0) { - _output_block->swap(*in_block); - _data_queue->push_block(std::move(_output_block), _cur_child_id); - } - } else if (_node->get_first_materialized_child_idx() != _node->children_count() && - _cur_child_id < _node->children_count()) { //need materialized - RETURN_IF_ERROR(this->_node->materialize_child_block(state, _cur_child_id, in_block, - _output_block.get())); - } else { - return Status::InternalError("maybe can't reach here, execute const expr: {}, {}, {}", - _cur_child_id, _node->get_first_materialized_child_idx(), - _node->children_count()); - } - - if (UNLIKELY(source_state == SourceState::FINISHED)) { - //if _cur_child_id eos, need check to push block - //Now here can't check _output_block rows, even it's row==0, also need push block - //because maybe sink is eos and queue have none data, if not push block - //the source can't can_read again and can't set source finished - if (_output_block) { - _data_queue->push_block(std::move(_output_block), _cur_child_id); - } - _data_queue->set_finish(_cur_child_id); - return Status::OK(); - } - // not eos and block rows is enough to output,so push block - if (_output_block && (_output_block->rows() >= state->batch_size())) { - _data_queue->push_block(std::move(_output_block), _cur_child_id); - } - return Status::OK(); -} - -Status UnionSinkOperator::close(RuntimeState* state) { - if (_data_queue && !_data_queue->is_finish(_cur_child_id)) { - // finish should be set, if not set here means error. - _data_queue->set_canceled(_cur_child_id); - } - return StreamingOperator::close(state); -} - Status UnionSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { RETURN_IF_ERROR(Base::init(state, info)); SCOPED_TIMER(exec_time_counter()); diff --git a/be/src/pipeline/exec/union_sink_operator.h b/be/src/pipeline/exec/union_sink_operator.h index 97b704078c63ecf..c11465da134b60f 100644 --- a/be/src/pipeline/exec/union_sink_operator.h +++ b/be/src/pipeline/exec/union_sink_operator.h @@ -25,47 +25,13 @@ #include "operator.h" #include "pipeline/pipeline_x/operator.h" #include "vec/core/block.h" -#include "vec/exec/vunion_node.h" namespace doris { -class ExecNode; class RuntimeState; namespace pipeline { class DataQueue; -class UnionSinkOperatorBuilder final : public OperatorBuilder { -public: - UnionSinkOperatorBuilder(int32_t id, int child_id, ExecNode* node, - std::shared_ptr queue); - - OperatorPtr build_operator() override; - - bool is_sink() const override { return true; } - -private: - int _cur_child_id; - std::shared_ptr _data_queue; -}; - -class UnionSinkOperator final : public StreamingOperator { -public: - UnionSinkOperator(OperatorBuilderBase* operator_builder, int child_id, ExecNode* node, - std::shared_ptr queue); - - bool can_write() override { return true; } - - Status sink(RuntimeState* state, vectorized::Block* in_block, - SourceState source_state) override; - - Status close(RuntimeState* state) override; - -private: - int _cur_child_id; - std::shared_ptr _data_queue; - std::unique_ptr _output_block; -}; - class UnionSinkOperatorX; class UnionSinkLocalState final : public PipelineXSinkLocalState { public: diff --git a/be/src/pipeline/exec/union_source_operator.cpp b/be/src/pipeline/exec/union_source_operator.cpp index 10f98a8d1cbc49b..6393ffcccd312b1 100644 --- a/be/src/pipeline/exec/union_source_operator.cpp +++ b/be/src/pipeline/exec/union_source_operator.cpp @@ -30,83 +30,10 @@ #include "vec/core/block.h" namespace doris { -class ExecNode; class RuntimeState; namespace pipeline { -UnionSourceOperatorBuilder::UnionSourceOperatorBuilder(int32_t id, ExecNode* node, - std::shared_ptr queue) - : OperatorBuilder(id, "UnionSourceOperator", node), _data_queue(queue) {}; - -OperatorPtr UnionSourceOperatorBuilder::build_operator() { - return std::make_shared(this, _node, _data_queue); -} - -UnionSourceOperator::UnionSourceOperator(OperatorBuilderBase* operator_builder, ExecNode* node, - std::shared_ptr queue) - : SourceOperator(operator_builder, node), - _data_queue(queue), - _need_read_for_const_expr(true) {}; - -bool UnionSourceOperator::_has_data() { - return _need_read_for_const_expr || _data_queue->remaining_has_data(); -} - -// we assumed it can read to process const expr, Although we don't know whether there is -// ,and queue have data, could read also -// The source operator's run dependences on Node's alloc_resource, which is called in Sink's open. -// So hang until SinkOperator was scheduled to open. -bool UnionSourceOperator::can_read() { - return _node->resource_allocated() && (_has_data() || _data_queue->is_all_finish()); -} - -Status UnionSourceOperator::pull_data(RuntimeState* state, vectorized::Block* block, bool* eos) { - // here we precess const expr firstly - if (_need_read_for_const_expr) { - if (_node->has_more_const(state)) { - RETURN_IF_ERROR(_node->get_next_const(state, block)); - } - _need_read_for_const_expr = _node->has_more_const(state); - } else { - std::unique_ptr output_block; - int child_idx = 0; - RETURN_IF_ERROR(_data_queue->get_block_from_queue(&output_block, &child_idx)); - if (!output_block) { - return Status::OK(); - } - block->swap(*output_block); - output_block->clear_column_data(_node->intermediate_row_desc().num_materialized_slots()); - _data_queue->push_free_block(std::move(output_block), child_idx); - } - - _node->reached_limit(block, eos); - return Status::OK(); -} - -Status UnionSourceOperator::get_block(RuntimeState* state, vectorized::Block* block, - SourceState& source_state) { - bool eos = false; - RETURN_IF_ERROR(_node->get_next_after_projects( - state, block, &eos, - std::bind(&UnionSourceOperator::pull_data, this, std::placeholders::_1, - std::placeholders::_2, std::placeholders::_3))); - //have executing const expr, queue have no data anymore, and child could be closed. - if (eos) { // reach limit - source_state = SourceState::FINISHED; - } else if (_has_data()) { - source_state = SourceState::MORE_DATA; - } else if (_data_queue->is_all_finish()) { - // Here, check the value of `_has_data(state)` again after `data_queue.is_all_finish()` is TRUE - // as there may be one or more blocks when `data_queue.is_all_finish()` is TRUE. - source_state = _has_data() ? SourceState::MORE_DATA : SourceState::FINISHED; - } else { - source_state = SourceState::DEPEND_ON_SOURCE; - } - - return Status::OK(); -} - Status UnionSourceLocalState::init(RuntimeState* state, LocalStateInfo& info) { RETURN_IF_ERROR(Base::init(state, info)); SCOPED_TIMER(exec_time_counter()); diff --git a/be/src/pipeline/exec/union_source_operator.h b/be/src/pipeline/exec/union_source_operator.h index 60530521ec0a822..ee1d35d73cff04f 100644 --- a/be/src/pipeline/exec/union_source_operator.h +++ b/be/src/pipeline/exec/union_source_operator.h @@ -23,10 +23,8 @@ #include "common/status.h" #include "operator.h" #include "pipeline/pipeline_x/operator.h" -#include "vec/exec/vunion_node.h" namespace doris { -class ExecNode; class RuntimeState; namespace vectorized { @@ -36,39 +34,6 @@ class Block; namespace pipeline { class DataQueue; -class UnionSourceOperatorBuilder final : public OperatorBuilder { -public: - UnionSourceOperatorBuilder(int32_t id, ExecNode* node, std::shared_ptr); - - bool is_source() const override { return true; } - - OperatorPtr build_operator() override; - -private: - std::shared_ptr _data_queue; -}; - -class UnionSourceOperator final : public SourceOperator { -public: - UnionSourceOperator(OperatorBuilderBase* operator_builder, ExecNode* node, - std::shared_ptr); - - // this operator in source open directly return, do this work in sink - Status open(RuntimeState* /*state*/) override { return Status::OK(); } - - Status get_block(RuntimeState* state, vectorized::Block* block, - SourceState& source_state) override; - bool can_read() override; - - Status pull_data(RuntimeState* state, vectorized::Block* output_block, bool* eos); - -private: - bool _has_data(); - - std::shared_ptr _data_queue; - bool _need_read_for_const_expr; -}; - class UnionSourceOperatorX; class UnionSourceLocalState final : public PipelineXLocalState { public: diff --git a/be/src/pipeline/pipeline.h b/be/src/pipeline/pipeline.h index ab5b7e36bc2ad59..1bab4f1fd5003b9 100644 --- a/be/src/pipeline/pipeline.h +++ b/be/src/pipeline/pipeline.h @@ -42,7 +42,7 @@ using PipelineId = uint32_t; class Pipeline : public std::enable_shared_from_this { friend class PipelineTask; friend class PipelineXTask; - friend class PipelineXFragmentContext; + friend class PipelineFragmentContext; public: Pipeline() = delete; diff --git a/be/src/pipeline/pipeline_fragment_context.cpp b/be/src/pipeline/pipeline_fragment_context.cpp index 3b094062074e34a..03de71f3c8d8298 100644 --- a/be/src/pipeline/pipeline_fragment_context.cpp +++ b/be/src/pipeline/pipeline_fragment_context.cpp @@ -48,46 +48,52 @@ #include "pipeline/exec/analytic_sink_operator.h" #include "pipeline/exec/analytic_source_operator.h" #include "pipeline/exec/assert_num_rows_operator.h" -#include "pipeline/exec/const_value_operator.h" -#include "pipeline/exec/data_queue.h" #include "pipeline/exec/datagen_operator.h" -#include "pipeline/exec/distinct_streaming_aggregation_sink_operator.h" -#include "pipeline/exec/distinct_streaming_aggregation_source_operator.h" +#include "pipeline/exec/distinct_streaming_aggregation_operator.h" #include "pipeline/exec/empty_set_operator.h" -#include "pipeline/exec/empty_source_operator.h" +#include "pipeline/exec/es_scan_operator.h" #include "pipeline/exec/exchange_sink_operator.h" #include "pipeline/exec/exchange_source_operator.h" +#include "pipeline/exec/file_scan_operator.h" #include "pipeline/exec/group_commit_block_sink_operator.h" #include "pipeline/exec/hashjoin_build_sink.h" #include "pipeline/exec/hashjoin_probe_operator.h" #include "pipeline/exec/hive_table_sink_operator.h" +#include "pipeline/exec/jdbc_scan_operator.h" +#include "pipeline/exec/jdbc_table_sink_operator.h" +#include "pipeline/exec/meta_scan_operator.h" #include "pipeline/exec/multi_cast_data_stream_sink.h" #include "pipeline/exec/multi_cast_data_stream_source.h" -#include "pipeline/exec/mysql_scan_operator.h" // IWYU pragma: keep #include "pipeline/exec/nested_loop_join_build_operator.h" #include "pipeline/exec/nested_loop_join_probe_operator.h" +#include "pipeline/exec/olap_scan_operator.h" #include "pipeline/exec/olap_table_sink_operator.h" #include "pipeline/exec/olap_table_sink_v2_operator.h" -#include "pipeline/exec/operator.h" #include "pipeline/exec/partition_sort_sink_operator.h" #include "pipeline/exec/partition_sort_source_operator.h" +#include "pipeline/exec/partitioned_aggregation_sink_operator.h" +#include "pipeline/exec/partitioned_aggregation_source_operator.h" +#include "pipeline/exec/partitioned_hash_join_probe_operator.h" +#include "pipeline/exec/partitioned_hash_join_sink_operator.h" #include "pipeline/exec/repeat_operator.h" #include "pipeline/exec/result_file_sink_operator.h" #include "pipeline/exec/result_sink_operator.h" -#include "pipeline/exec/scan_operator.h" #include "pipeline/exec/schema_scan_operator.h" #include "pipeline/exec/select_operator.h" -#include "pipeline/exec/set_probe_sink_operator.h" // IWYU pragma: keep -#include "pipeline/exec/set_sink_operator.h" // IWYU pragma: keep -#include "pipeline/exec/set_source_operator.h" // IWYU pragma: keep +#include "pipeline/exec/set_probe_sink_operator.h" +#include "pipeline/exec/set_sink_operator.h" +#include "pipeline/exec/set_source_operator.h" #include "pipeline/exec/sort_sink_operator.h" #include "pipeline/exec/sort_source_operator.h" -#include "pipeline/exec/streaming_aggregation_sink_operator.h" -#include "pipeline/exec/streaming_aggregation_source_operator.h" +#include "pipeline/exec/spill_sort_sink_operator.h" +#include "pipeline/exec/spill_sort_source_operator.h" +#include "pipeline/exec/streaming_aggregation_operator.h" #include "pipeline/exec/table_function_operator.h" -#include "pipeline/exec/table_sink_operator.h" #include "pipeline/exec/union_sink_operator.h" #include "pipeline/exec/union_source_operator.h" +#include "pipeline/pipeline_x/local_exchange/local_exchange_sink_operator.h" +#include "pipeline/pipeline_x/local_exchange/local_exchange_source_operator.h" +#include "pipeline/task_scheduler.h" #include "pipeline_task.h" #include "runtime/exec_env.h" #include "runtime/fragment_mgr.h" @@ -119,14 +125,11 @@ namespace doris::pipeline { bvar::Adder g_pipeline_tasks_count("doris_pipeline_tasks_count"); PipelineFragmentContext::PipelineFragmentContext( - const TUniqueId& query_id, const TUniqueId& instance_id, int fragment_id, int backend_num, - std::shared_ptr query_ctx, ExecEnv* exec_env, - const std::function& call_back, - report_status_callback report_status_cb) + const TUniqueId& query_id, const int fragment_id, std::shared_ptr query_ctx, + ExecEnv* exec_env, const std::function& call_back, + const report_status_callback& report_status_cb) : _query_id(query_id), - _fragment_instance_id(instance_id), _fragment_id(fragment_id), - _backend_num(backend_num), _exec_env(exec_env), _query_ctx(std::move(query_ctx)), _call_back(call_back), @@ -144,14 +147,19 @@ PipelineFragmentContext::~PipelineFragmentContext() { auto st = _query_ctx->exec_status(); _query_ctx.reset(); _tasks.clear(); - if (_runtime_state != nullptr) { - _call_back(_runtime_state.get(), &st); - _runtime_state.reset(); + if (!_task_runtime_states.empty()) { + for (auto& runtime_state : _task_runtime_states) { + _call_back(runtime_state.get(), &st); + runtime_state.reset(); + } } - _root_pipeline.reset(); _pipelines.clear(); _sink.reset(); - _multi_cast_stream_sink_senders.clear(); + _root_op.reset(); + _runtime_state.reset(); + _runtime_filter_states.clear(); + _runtime_filter_mgr_map.clear(); + _op_id_to_le_state.clear(); } bool PipelineFragmentContext::is_timeout(const VecDateTimeValue& now) const { @@ -171,26 +179,21 @@ bool PipelineFragmentContext::is_timeout(const VecDateTimeValue& now) const { void PipelineFragmentContext::cancel(const PPlanFragmentCancelReason& reason, const std::string& msg) { LOG_INFO("PipelineFragmentContext::cancel") - .tag("query_id", print_id(_query_ctx->query_id())) + .tag("query_id", print_id(_query_id)) .tag("fragment_id", _fragment_id) - .tag("instance_id", print_id(_runtime_state->fragment_instance_id())) - .tag("reason", PPlanFragmentCancelReason_Name(reason)) - .tag("message", msg); - // TODO(zhiqiang): may be not need to check if query is already cancelled. - // Dont cancel in this situation may lead to bug. For example, result sink node - // can not be cancelled if other fragments set the query_ctx cancelled, this will - // make result receiver on fe be stocked on rpc forever until timeout... - // We need a more detail discussion. - _query_ctx->cancel(msg, Status::Cancelled(msg)); + .tag("reason", reason) + .tag("error message", msg); + if (reason == PPlanFragmentCancelReason::TIMEOUT) { + LOG(WARNING) << "PipelineFragmentContext is cancelled due to timeout : " << debug_string(); + } + _query_ctx->cancel(msg, Status::Cancelled(msg), _fragment_id); if (reason == PPlanFragmentCancelReason::LIMIT_REACH) { _is_report_on_cancel = false; } else { - LOG(WARNING) << "PipelineFragmentContext " - << PrintInstanceStandardInfo(_query_id, _fragment_instance_id) - << " is canceled, cancel message: " << msg; + for (auto& id : _fragment_instance_ids) { + LOG(WARNING) << "PipelineFragmentContext cancel instance: " << print_id(id); + } } - - _runtime_state->set_process_status(_query_ctx->exec_status()); // Get pipe from new load stream manager and send cancel to it or the fragment may hang to wait read from pipe // For stream load the fragment's query_id == load id, it is set in FE. auto stream_load_ctx = _exec_env->new_load_stream_mgr()->get(_query_id); @@ -198,12 +201,14 @@ void PipelineFragmentContext::cancel(const PPlanFragmentCancelReason& reason, stream_load_ctx->pipe->cancel(msg); } - // must close stream_mgr to avoid dead lock in Exchange Node - // TODO bug llj fix this other instance will not cancel - _exec_env->vstream_mgr()->cancel(_fragment_instance_id, Status::Cancelled(msg)); // Cancel the result queue manager used by spark doris connector // TODO pipeline incomp // _exec_env->result_queue_mgr()->update_queue_status(id, Status::Aborted(msg)); + for (auto& tasks : _tasks) { + for (auto& task : tasks) { + task->clear_blocking_state(); + } + } } PipelinePtr PipelineFragmentContext::add_pipeline() { @@ -231,36 +236,33 @@ PipelinePtr PipelineFragmentContext::add_pipeline(PipelinePtr parent, int idx) { return pipeline; } -Status PipelineFragmentContext::prepare(const doris::TPipelineFragmentParams& request, size_t idx) { +Status PipelineFragmentContext::prepare(const doris::TPipelineFragmentParams& request) { if (_prepared) { return Status::InternalError("Already prepared"); } - if (request.__isset.query_options && request.query_options.__isset.execution_timeout) { - _timeout = request.query_options.execution_timeout; - } - const auto& local_params = request.local_params[idx]; + _num_instances = request.local_params.size(); + _total_instances = request.__isset.total_instances ? request.total_instances : _num_instances; _runtime_profile = std::make_unique("PipelineContext"); - _start_timer = ADD_TIMER(_runtime_profile, "StartTime"); - COUNTER_UPDATE(_start_timer, _fragment_watcher.elapsed_time()); _prepare_timer = ADD_TIMER(_runtime_profile, "PrepareTime"); SCOPED_TIMER(_prepare_timer); auto* fragment_context = this; - LOG_INFO("Preparing instance {}|{}, backend_num {}", print_id(_query_id), - print_id(local_params.fragment_instance_id), local_params.backend_num); + LOG_INFO("PipelineFragmentContext::prepare") + .tag("query_id", print_id(_query_id)) + .tag("fragment_id", _fragment_id) + .tag("pthread_id", (uintptr_t)pthread_self()); - // 1. init _runtime_state - _runtime_state = RuntimeState::create_unique( - local_params.fragment_instance_id, request.query_id, request.fragment_id, - request.query_options, _query_ctx->query_globals, _exec_env, _query_ctx.get()); + if (request.query_options.__isset.is_report_success) { + fragment_context->set_is_report_success(request.query_options.is_report_success); + } - _runtime_state->set_task_execution_context(shared_from_this()); + // 1. Set up the global runtime state. + _runtime_state = RuntimeState::create_unique(request.query_id, request.fragment_id, + request.query_options, _query_ctx->query_globals, + _exec_env, _query_ctx.get()); - // TODO should be combine with plan_fragment_executor.prepare funciton SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(_runtime_state->query_mem_tracker()); - _runtime_state->set_be_number(local_params.backend_num); - if (request.__isset.backend_id) { _runtime_state->set_backend_id(request.backend_id); } @@ -274,10 +276,6 @@ Status PipelineFragmentContext::prepare(const doris::TPipelineFragmentParams& re _runtime_state->set_load_job_id(request.load_job_id); } - if (request.query_options.__isset.is_report_success) { - fragment_context->set_is_report_success(request.query_options.is_report_success); - } - if (request.is_simplified_param) { _desc_tbl = _query_ctx->desc_tbl; } else { @@ -286,91 +284,58 @@ Status PipelineFragmentContext::prepare(const doris::TPipelineFragmentParams& re DescriptorTbl::create(_runtime_state->obj_pool(), request.desc_tbl, &_desc_tbl)); } _runtime_state->set_desc_tbl(_desc_tbl); - - // 2. Create ExecNode to build pipeline with PipelineFragmentContext - RETURN_IF_ERROR_OR_CATCH_EXCEPTION( - ExecNode::create_tree(_runtime_state.get(), _runtime_state->obj_pool(), - request.fragment.plan, *_desc_tbl, &_root_plan)); - - // Set senders of exchange nodes before pipeline build - std::vector exch_nodes; - _root_plan->collect_nodes(TPlanNodeType::EXCHANGE_NODE, &exch_nodes); - for (ExecNode* exch_node : exch_nodes) { - DCHECK_EQ(exch_node->type(), TPlanNodeType::EXCHANGE_NODE); - int num_senders = find_with_default(request.per_exch_num_senders, exch_node->id(), 0); - DCHECK_GT(num_senders, 0); - static_cast(exch_node)->set_num_senders(num_senders); - } - - // All prepare work do in exec node tree - RETURN_IF_ERROR(_root_plan->prepare(_runtime_state.get())); - // set scan ranges - std::vector scan_nodes; - std::vector no_scan_ranges; - _root_plan->collect_scan_nodes(&scan_nodes); - VLOG_CRITICAL << "query " << print_id(get_query_id()) - << " scan_nodes.size()=" << scan_nodes.size(); - VLOG_CRITICAL << "query " << print_id(get_query_id()) << " params.per_node_scan_ranges.size()=" - << local_params.per_node_scan_ranges.size(); - - // set scan range in ScanNode - for (auto& i : scan_nodes) { - // TODO(cmy): this "if...else" should be removed once all ScanNode are derived from VScanNode. - ExecNode* node = i; - if (typeid(*node) == typeid(vectorized::NewOlapScanNode) || - typeid(*node) == typeid(vectorized::NewFileScanNode) || - typeid(*node) == typeid(vectorized::NewOdbcScanNode) || - typeid(*node) == typeid(vectorized::NewEsScanNode) || - typeid(*node) == typeid(vectorized::VMetaScanNode) || - typeid(*node) == typeid(vectorized::NewJdbcScanNode)) { - auto* scan_node = static_cast(i); - auto scan_ranges = find_with_default(local_params.per_node_scan_ranges, scan_node->id(), - no_scan_ranges); - const bool shared_scan = - find_with_default(local_params.per_node_shared_scans, scan_node->id(), false); - scan_node->set_scan_ranges(_runtime_state.get(), scan_ranges); - scan_node->set_shared_scan(_runtime_state.get(), shared_scan); - } else { - auto* scan_node = static_cast(node); - auto scan_ranges = find_with_default(local_params.per_node_scan_ranges, scan_node->id(), - no_scan_ranges); - RETURN_IF_ERROR(scan_node->set_scan_ranges(_runtime_state.get(), scan_ranges)); - VLOG_CRITICAL << "query " << print_id(get_query_id()) - << " scan_node_id=" << scan_node->id() - << " size=" << scan_ranges.get().size(); - } - } - - _runtime_state->set_per_fragment_instance_idx(local_params.sender_id); _runtime_state->set_num_per_fragment_instances(request.num_senders); _runtime_state->set_load_stream_per_node(request.load_stream_per_node); _runtime_state->set_total_load_streams(request.total_load_streams); _runtime_state->set_num_local_sink(request.num_local_sink); - if (request.fragment.__isset.output_sink) { - // Here we build a DataSink object, which will be hold by DataSinkOperator - RETURN_IF_ERROR_OR_CATCH_EXCEPTION(DataSink::create_data_sink( - _runtime_state->obj_pool(), request.fragment.output_sink, - request.fragment.output_exprs, request, idx, _root_plan->row_desc(), - _runtime_state.get(), &_sink, *_desc_tbl)); + const auto& local_params = request.local_params[0]; + if (local_params.__isset.runtime_filter_params) { + _query_ctx->runtime_filter_mgr()->set_runtime_filter_params( + local_params.runtime_filter_params); + } + if (local_params.__isset.topn_filter_source_node_ids) { + _query_ctx->init_runtime_predicates(local_params.topn_filter_source_node_ids); + } else { + _query_ctx->init_runtime_predicates({0}); } - _root_pipeline = fragment_context->add_pipeline(); - _root_pipeline->set_is_root_pipeline(); - RETURN_IF_ERROR(_build_pipelines(_root_plan, _root_pipeline)); - if (_sink) { - // DataSinkOperator is builded here - RETURN_IF_ERROR(_create_sink(request.local_params[idx].sender_id, - request.fragment.output_sink, _runtime_state.get())); + _need_local_merge = request.__isset.parallel_instances; + + // 2. Build pipelines with operators in this fragment. + auto root_pipeline = add_pipeline(); + RETURN_IF_ERROR_OR_CATCH_EXCEPTION(_build_pipelines( + _runtime_state->obj_pool(), request, *_query_ctx->desc_tbl, &_root_op, root_pipeline)); + + // 3. Create sink operator + if (!request.fragment.__isset.output_sink) { + return Status::InternalError("No output sink in this fragment!"); } - RETURN_IF_ERROR(_build_pipeline_tasks(request)); - if (_sink) { - _runtime_state->runtime_profile()->add_child(_sink->profile(), true, nullptr); - _sink->profile()->add_child(_root_plan->runtime_profile(), true, nullptr); - } else { - _runtime_state->runtime_profile()->add_child(_root_plan->runtime_profile(), true, nullptr); + RETURN_IF_ERROR_OR_CATCH_EXCEPTION(_create_data_sink( + _runtime_state->obj_pool(), request.fragment.output_sink, request.fragment.output_exprs, + request, root_pipeline->output_row_desc(), _runtime_state.get(), *_desc_tbl, + root_pipeline->id())); + RETURN_IF_ERROR(_sink->init(request.fragment.output_sink)); + RETURN_IF_ERROR(root_pipeline->set_sink(_sink)); + + for (PipelinePtr& pipeline : _pipelines) { + DCHECK(pipeline->sink_x() != nullptr) << pipeline->operator_xs().size(); + RETURN_IF_ERROR(pipeline->sink_x()->set_child(pipeline->operator_xs().back())); + } + if (_enable_local_shuffle()) { + RETURN_IF_ERROR(_plan_local_exchange(request.num_buckets, + request.bucket_seq_to_instance_idx, + request.shuffle_idx_to_instance_idx)); } - _runtime_state->runtime_profile()->add_child(_runtime_profile.get(), true, nullptr); + + // 4. Initialize global states in pipelines. + for (PipelinePtr& pipeline : _pipelines) { + pipeline->children().clear(); + RETURN_IF_ERROR(pipeline->prepare(_runtime_state.get())); + } + + // 5. Build pipeline tasks and initialize local state. + RETURN_IF_ERROR(_build_pipeline_tasks(request)); _init_next_report_time(); @@ -381,29 +346,189 @@ Status PipelineFragmentContext::prepare(const doris::TPipelineFragmentParams& re Status PipelineFragmentContext::_build_pipeline_tasks( const doris::TPipelineFragmentParams& request) { _total_tasks = 0; - for (PipelinePtr& pipeline : _pipelines) { - // if sink - auto sink_operator = pipeline->get_sink_builder()->build_operator(); - // TODO pipeline 1 need to add new interface for exec node and operator - RETURN_IF_ERROR(sink_operator->init(request.fragment.output_sink)); - - RETURN_IF_ERROR(pipeline->build_operators()); - auto task = - std::make_unique(pipeline, _total_tasks++, _runtime_state.get(), - sink_operator, this, pipeline->pipeline_profile()); - RETURN_IF_ERROR(sink_operator->set_child(task->get_root())); - _tasks.emplace_back(std::move(task)); - _runtime_profile->add_child(pipeline->pipeline_profile(), true, nullptr); - } - g_pipeline_tasks_count << _total_tasks; - for (auto& task : _tasks) { - RETURN_IF_ERROR(task->prepare(_runtime_state.get())); + int target_size = request.local_params.size(); + _tasks.resize(target_size); + auto& pipeline_id_to_profile = _runtime_state->pipeline_id_to_profile(); + DCHECK(pipeline_id_to_profile.empty()); + pipeline_id_to_profile.resize(_pipelines.size()); + { + size_t pip_idx = 0; + for (auto& pipeline_profile : pipeline_id_to_profile) { + pipeline_profile = + std::make_unique("Pipeline : " + std::to_string(pip_idx)); + pip_idx++; + } } - // register the profile of child data stream sender - for (auto& sender : _multi_cast_stream_sink_senders) { - _sink->profile()->add_child(sender->profile(), true, nullptr); + for (size_t i = 0; i < target_size; i++) { + const auto& local_params = request.local_params[i]; + auto fragment_instance_id = local_params.fragment_instance_id; + _fragment_instance_ids.push_back(fragment_instance_id); + std::unique_ptr runtime_filter_mgr; + auto init_runtime_state = [&](std::unique_ptr& runtime_state) { + runtime_state->set_query_mem_tracker(_query_ctx->query_mem_tracker); + + runtime_state->set_task_execution_context(shared_from_this()); + runtime_state->set_be_number(local_params.backend_num); + + if (request.__isset.backend_id) { + runtime_state->set_backend_id(request.backend_id); + } + if (request.__isset.import_label) { + runtime_state->set_import_label(request.import_label); + } + if (request.__isset.db_name) { + runtime_state->set_db_name(request.db_name); + } + if (request.__isset.load_job_id) { + runtime_state->set_load_job_id(request.load_job_id); + } + + runtime_state->set_desc_tbl(_desc_tbl); + runtime_state->set_per_fragment_instance_idx(local_params.sender_id); + runtime_state->set_num_per_fragment_instances(request.num_senders); + runtime_state->resize_op_id_to_local_state(max_operator_id()); + runtime_state->set_max_operator_id(max_operator_id()); + runtime_state->set_load_stream_per_node(request.load_stream_per_node); + runtime_state->set_total_load_streams(request.total_load_streams); + runtime_state->set_num_local_sink(request.num_local_sink); + DCHECK(runtime_filter_mgr); + runtime_state->set_pipeline_x_runtime_filter_mgr(runtime_filter_mgr.get()); + }; + + auto filterparams = std::make_unique(); + + { + filterparams->runtime_filter_wait_infinitely = + _runtime_state->runtime_filter_wait_infinitely(); + filterparams->runtime_filter_wait_time_ms = + _runtime_state->runtime_filter_wait_time_ms(); + filterparams->enable_pipeline_exec = _runtime_state->enable_pipeline_x_exec(); + filterparams->execution_timeout = _runtime_state->execution_timeout(); + + filterparams->exec_env = ExecEnv::GetInstance(); + filterparams->query_id.set_hi(_runtime_state->query_id().hi); + filterparams->query_id.set_lo(_runtime_state->query_id().lo); + + filterparams->be_exec_version = _runtime_state->be_exec_version(); + filterparams->query_ctx = _query_ctx.get(); + } + + // build local_runtime_filter_mgr for each instance + runtime_filter_mgr = std::make_unique( + request.query_id, filterparams.get(), _query_ctx->query_mem_tracker); + + filterparams->runtime_filter_mgr = runtime_filter_mgr.get(); + + _runtime_filter_states.push_back(std::move(filterparams)); + std::map pipeline_id_to_task; + auto get_local_exchange_state = [&](PipelinePtr pipeline) + -> std::map, + std::shared_ptr>> { + std::map, + std::shared_ptr>> + le_state_map; + auto source_id = pipeline->operator_xs().front()->operator_id(); + if (auto iter = _op_id_to_le_state.find(source_id); iter != _op_id_to_le_state.end()) { + le_state_map.insert({source_id, iter->second}); + } + for (auto sink_to_source_id : pipeline->sink_x()->dests_id()) { + if (auto iter = _op_id_to_le_state.find(sink_to_source_id); + iter != _op_id_to_le_state.end()) { + le_state_map.insert({sink_to_source_id, iter->second}); + } + } + return le_state_map; + }; + auto get_task_runtime_state = [&](int task_id) -> RuntimeState* { + DCHECK(_task_runtime_states[task_id]); + return _task_runtime_states[task_id].get(); + }; + for (size_t pip_idx = 0; pip_idx < _pipelines.size(); pip_idx++) { + auto& pipeline = _pipelines[pip_idx]; + if (pipeline->need_to_create_task()) { + // build task runtime state + _task_runtime_states.push_back(RuntimeState::create_unique( + this, local_params.fragment_instance_id, request.query_id, + request.fragment_id, request.query_options, _query_ctx->query_globals, + _exec_env, _query_ctx.get())); + auto& task_runtime_state = _task_runtime_states.back(); + init_runtime_state(task_runtime_state); + auto cur_task_id = _total_tasks++; + task_runtime_state->set_task_id(cur_task_id); + task_runtime_state->set_task_num(pipeline->num_tasks()); + auto task = std::make_unique( + pipeline, cur_task_id, get_task_runtime_state(cur_task_id), this, + pipeline_id_to_profile[pip_idx].get(), get_local_exchange_state(pipeline), + i); + pipeline_id_to_task.insert({pipeline->id(), task.get()}); + _tasks[i].emplace_back(std::move(task)); + } + } + + /** + * Build DAG for pipeline tasks. + * For example, we have + * + * ExchangeSink (Pipeline1) JoinBuildSink (Pipeline2) + * \ / + * JoinProbeOperator1 (Pipeline1) JoinBuildSink (Pipeline3) + * \ / + * JoinProbeOperator2 (Pipeline1) + * + * In this fragment, we have three pipelines and pipeline 1 depends on pipeline 2 and pipeline 3. + * To build this DAG, `_dag` manage dependencies between pipelines by pipeline ID and + * `pipeline_id_to_task` is used to find the task by a unique pipeline ID. + * + * Finally, we have two upstream dependencies in Pipeline1 corresponding to JoinProbeOperator1 + * and JoinProbeOperator2. + */ + + // First, set up the parent profile,task runtime state + + auto prepare_and_set_parent_profile = [&](PipelineXTask* task, size_t pip_idx) { + DCHECK(pipeline_id_to_profile[pip_idx]); + RETURN_IF_ERROR( + task->prepare(local_params, request.fragment.output_sink, _query_ctx.get())); + return Status::OK(); + }; + + for (auto& _pipeline : _pipelines) { + if (pipeline_id_to_task.contains(_pipeline->id())) { + auto* task = pipeline_id_to_task[_pipeline->id()]; + DCHECK(task != nullptr); + + // if this task has upstream dependency, then record them. + if (_dag.find(_pipeline->id()) != _dag.end()) { + auto& deps = _dag[_pipeline->id()]; + for (auto& dep : deps) { + if (pipeline_id_to_task.contains(dep)) { + auto ss = pipeline_id_to_task[dep]->get_sink_shared_state(); + if (ss) { + task->inject_shared_state(ss); + } else { + pipeline_id_to_task[dep]->inject_shared_state( + task->get_source_shared_state()); + } + } + } + } + } + } + for (size_t pip_idx = 0; pip_idx < _pipelines.size(); pip_idx++) { + if (pipeline_id_to_task.contains(_pipelines[pip_idx]->id())) { + auto* task = pipeline_id_to_task[_pipelines[pip_idx]->id()]; + RETURN_IF_ERROR(prepare_and_set_parent_profile(task, pip_idx)); + } + } + { + std::lock_guard l(_state_map_lock); + _runtime_filter_mgr_map[fragment_instance_id] = std::move(runtime_filter_mgr); + } } + _pipeline_parent_map.clear(); + _dag.clear(); + _op_id_to_le_state.clear(); return Status::OK(); } @@ -467,8 +592,7 @@ void PipelineFragmentContext::trigger_report_if_necessary() { } VLOG_FILE << "Query " << print_id(this->get_query_id()) << " fragment " - << this->get_fragment_id() << " instance " - << print_id(this->get_fragment_instance_id()) << " profile:\n" + << this->get_fragment_id() << " profile:\n" << ss.str(); } auto st = send_report(false); @@ -481,265 +605,840 @@ void PipelineFragmentContext::trigger_report_if_necessary() { } // TODO: use virtual function to do abstruct -Status PipelineFragmentContext::_build_pipelines(ExecNode* node, PipelinePtr cur_pipe) { - auto node_type = node->type(); - switch (node_type) { - // for source - case TPlanNodeType::OLAP_SCAN_NODE: - case TPlanNodeType::JDBC_SCAN_NODE: - case TPlanNodeType::ODBC_SCAN_NODE: - case TPlanNodeType::FILE_SCAN_NODE: - case TPlanNodeType::META_SCAN_NODE: - case TPlanNodeType::GROUP_COMMIT_SCAN_NODE: - case TPlanNodeType::ES_HTTP_SCAN_NODE: - case TPlanNodeType::ES_SCAN_NODE: { - OperatorBuilderPtr operator_t = std::make_shared(node->id(), node); - RETURN_IF_ERROR(cur_pipe->add_operator(operator_t)); - break; - } - case TPlanNodeType::MYSQL_SCAN_NODE: { -#ifdef DORIS_WITH_MYSQL - OperatorBuilderPtr operator_t = - std::make_shared(node->id(), node); - RETURN_IF_ERROR(cur_pipe->add_operator(operator_t)); - break; -#else +Status PipelineFragmentContext::_build_pipelines(ObjectPool* pool, + const doris::TPipelineFragmentParams& request, + const DescriptorTbl& descs, OperatorXPtr* root, + PipelinePtr cur_pipe) { + if (request.fragment.plan.nodes.empty()) { + throw Exception(ErrorCode::INTERNAL_ERROR, "Invalid plan which has no plan node!"); + } + + int node_idx = 0; + + cur_pipe->_name.append(std::to_string(cur_pipe->id())); + + RETURN_IF_ERROR(_create_tree_helper(pool, request.fragment.plan.nodes, request, descs, nullptr, + &node_idx, root, cur_pipe, 0)); + + if (node_idx + 1 != request.fragment.plan.nodes.size()) { + // TODO: print thrift msg for diagnostic purposes. return Status::InternalError( - "Don't support MySQL table, you should rebuild Doris with WITH_MYSQL option ON"); -#endif + "Plan tree only partially reconstructed. Not all thrift nodes were used."); } - case TPlanNodeType::SCHEMA_SCAN_NODE: { - OperatorBuilderPtr operator_t = - std::make_shared(node->id(), node); - RETURN_IF_ERROR(cur_pipe->add_operator(operator_t)); + return Status::OK(); +} + +Status PipelineFragmentContext::_create_tree_helper(ObjectPool* pool, + const std::vector& tnodes, + const doris::TPipelineFragmentParams& request, + const DescriptorTbl& descs, OperatorXPtr parent, + int* node_idx, OperatorXPtr* root, + PipelinePtr& cur_pipe, int child_idx) { + // propagate error case + if (*node_idx >= tnodes.size()) { + // TODO: print thrift msg + return Status::InternalError( + "Failed to reconstruct plan tree from thrift. Node id: {}, number of nodes: {}", + *node_idx, tnodes.size()); + } + const TPlanNode& tnode = tnodes[*node_idx]; + + int num_children = tnodes[*node_idx].num_children; + OperatorXPtr op = nullptr; + RETURN_IF_ERROR(_create_operator(pool, tnodes[*node_idx], request, descs, op, cur_pipe, + parent == nullptr ? -1 : parent->node_id(), child_idx)); + + // assert(parent != nullptr || (node_idx == 0 && root_expr != nullptr)); + if (parent != nullptr) { + // add to parent's child(s) + RETURN_IF_ERROR(parent->set_child(op)); + } else { + *root = op; + } + + cur_pipe->_name.push_back('-'); + cur_pipe->_name.append(std::to_string(op->id())); + cur_pipe->_name.append(op->get_name()); + + // rely on that tnodes is preorder of the plan + for (int i = 0; i < num_children; i++) { + ++*node_idx; + RETURN_IF_ERROR(_create_tree_helper(pool, tnodes, request, descs, op, node_idx, nullptr, + cur_pipe, i)); + + // we are expecting a child, but have used all nodes + // this means we have been given a bad tree and must fail + if (*node_idx >= tnodes.size()) { + // TODO: print thrift msg + return Status::InternalError( + "Failed to reconstruct plan tree from thrift. Node id: {}, number of nodes: {}", + *node_idx, tnodes.size()); + } + } + + RETURN_IF_ERROR(op->init(tnode, _runtime_state.get())); + + return Status::OK(); +} + +void PipelineFragmentContext::_inherit_pipeline_properties( + const DataDistribution& data_distribution, PipelinePtr pipe_with_source, + PipelinePtr pipe_with_sink) { + pipe_with_sink->set_num_tasks(pipe_with_source->num_tasks()); + pipe_with_source->set_num_tasks(_num_instances); + pipe_with_source->set_data_distribution(data_distribution); +} + +Status PipelineFragmentContext::_add_local_exchange_impl( + int idx, ObjectPool* pool, PipelinePtr cur_pipe, PipelinePtr new_pip, + DataDistribution data_distribution, bool* do_local_exchange, int num_buckets, + const std::map& bucket_seq_to_instance_idx, + const std::map& shuffle_idx_to_instance_idx, + const bool ignore_data_hash_distribution) { + auto& operator_xs = cur_pipe->operator_xs(); + const auto downstream_pipeline_id = cur_pipe->id(); + auto local_exchange_id = next_operator_id(); + // 1. Create a new pipeline with local exchange sink. + DataSinkOperatorXPtr sink; + auto sink_id = next_sink_operator_id(); + const bool is_shuffled_hash_join = operator_xs.size() > idx + ? operator_xs[idx]->is_shuffled_hash_join() + : cur_pipe->sink_x()->is_shuffled_hash_join(); + sink.reset(new LocalExchangeSinkOperatorX( + sink_id, local_exchange_id, is_shuffled_hash_join ? _total_instances : _num_instances, + data_distribution.partition_exprs, bucket_seq_to_instance_idx)); + RETURN_IF_ERROR(new_pip->set_sink(sink)); + RETURN_IF_ERROR(new_pip->sink_x()->init(data_distribution.distribution_type, num_buckets, + is_shuffled_hash_join, shuffle_idx_to_instance_idx)); + + // 2. Create and initialize LocalExchangeSharedState. + auto shared_state = LocalExchangeSharedState::create_shared(_num_instances); + switch (data_distribution.distribution_type) { + case ExchangeType::HASH_SHUFFLE: + shared_state->exchanger = ShuffleExchanger::create_unique( + std::max(cur_pipe->num_tasks(), _num_instances), + is_shuffled_hash_join ? _total_instances : _num_instances); + break; + case ExchangeType::BUCKET_HASH_SHUFFLE: + shared_state->exchanger = BucketShuffleExchanger::create_unique( + std::max(cur_pipe->num_tasks(), _num_instances), _num_instances, num_buckets, + ignore_data_hash_distribution); break; + case ExchangeType::PASSTHROUGH: + shared_state->exchanger = + PassthroughExchanger::create_unique(cur_pipe->num_tasks(), _num_instances); + break; + case ExchangeType::BROADCAST: + shared_state->exchanger = + BroadcastExchanger::create_unique(cur_pipe->num_tasks(), _num_instances); + break; + case ExchangeType::PASS_TO_ONE: + shared_state->exchanger = + BroadcastExchanger::create_unique(cur_pipe->num_tasks(), _num_instances); + break; + case ExchangeType::ADAPTIVE_PASSTHROUGH: + shared_state->exchanger = + AdaptivePassthroughExchanger::create_unique(cur_pipe->num_tasks(), _num_instances); + break; + default: + return Status::InternalError("Unsupported local exchange type : " + + std::to_string((int)data_distribution.distribution_type)); + } + auto sink_dep = std::make_shared(sink_id, local_exchange_id, + "LOCAL_EXCHANGE_SINK_DEPENDENCY", true, + _runtime_state->get_query_ctx()); + sink_dep->set_shared_state(shared_state.get()); + shared_state->sink_deps.push_back(sink_dep); + _op_id_to_le_state.insert({local_exchange_id, {shared_state, sink_dep}}); + + // 3. Set two pipelines' operator list. For example, split pipeline [Scan - AggSink] to + // pipeline1 [Scan - LocalExchangeSink] and pipeline2 [LocalExchangeSource - AggSink]. + + // 3.1 Initialize new pipeline's operator list. + std::copy(operator_xs.begin(), operator_xs.begin() + idx, + std::inserter(new_pip->operator_xs(), new_pip->operator_xs().end())); + + // 3.2 Erase unused operators in previous pipeline. + operator_xs.erase(operator_xs.begin(), operator_xs.begin() + idx); + + // 4. Initialize LocalExchangeSource and insert it into this pipeline. + OperatorXPtr source_op; + source_op.reset(new LocalExchangeSourceOperatorX(pool, local_exchange_id)); + RETURN_IF_ERROR(source_op->set_child(new_pip->operator_xs().back())); + RETURN_IF_ERROR(source_op->init(data_distribution.distribution_type)); + if (!operator_xs.empty()) { + RETURN_IF_ERROR(operator_xs.front()->set_child(source_op)); + } + operator_xs.insert(operator_xs.begin(), source_op); + + shared_state->create_source_dependencies(source_op->operator_id(), source_op->node_id(), + _query_ctx.get()); + + // 5. Set children for two pipelines separately. + std::vector> new_children; + std::vector edges_with_source; + for (auto child : cur_pipe->children()) { + bool found = false; + for (auto op : new_pip->operator_xs()) { + if (child->sink_x()->node_id() == op->node_id()) { + new_pip->set_children(child); + found = true; + }; + } + if (!found) { + new_children.push_back(child); + edges_with_source.push_back(child->id()); + } } - case TPlanNodeType::EXCHANGE_NODE: { - OperatorBuilderPtr operator_t = - std::make_shared(node->id(), node); - RETURN_IF_ERROR(cur_pipe->add_operator(operator_t)); + new_children.push_back(new_pip); + edges_with_source.push_back(new_pip->id()); + + // 6. Set DAG for new pipelines. + if (!new_pip->children().empty()) { + std::vector edges_with_sink; + for (auto child : new_pip->children()) { + edges_with_sink.push_back(child->id()); + } + _dag.insert({new_pip->id(), edges_with_sink}); + } + cur_pipe->set_children(new_children); + _dag[downstream_pipeline_id] = edges_with_source; + RETURN_IF_ERROR(new_pip->sink_x()->set_child(new_pip->operator_xs().back())); + RETURN_IF_ERROR(cur_pipe->sink_x()->set_child(cur_pipe->operator_xs().back())); + + // 7. Inherit properties from current pipeline. + _inherit_pipeline_properties(data_distribution, cur_pipe, new_pip); + return Status::OK(); +} + +Status PipelineFragmentContext::_add_local_exchange( + int pip_idx, int idx, int node_id, ObjectPool* pool, PipelinePtr cur_pipe, + DataDistribution data_distribution, bool* do_local_exchange, int num_buckets, + const std::map& bucket_seq_to_instance_idx, + const std::map& shuffle_idx_to_instance_idx, + const bool ignore_data_distribution) { + DCHECK(_enable_local_shuffle()); + if (_num_instances <= 1) { + return Status::OK(); + } + + if (!cur_pipe->need_to_local_exchange(data_distribution)) { + return Status::OK(); + } + *do_local_exchange = true; + + auto& operator_xs = cur_pipe->operator_xs(); + auto total_op_num = operator_xs.size(); + auto new_pip = add_pipeline(cur_pipe, pip_idx + 1); + RETURN_IF_ERROR(_add_local_exchange_impl( + idx, pool, cur_pipe, new_pip, data_distribution, do_local_exchange, num_buckets, + bucket_seq_to_instance_idx, shuffle_idx_to_instance_idx, ignore_data_distribution)); + + CHECK(total_op_num + 1 == cur_pipe->operator_xs().size() + new_pip->operator_xs().size()) + << "total_op_num: " << total_op_num + << " cur_pipe->operator_xs().size(): " << cur_pipe->operator_xs().size() + << " new_pip->operator_xs().size(): " << new_pip->operator_xs().size(); + + // Add passthrough local exchanger if necessary + if (cur_pipe->num_tasks() > 1 && new_pip->num_tasks() == 1 && + Pipeline::is_hash_exchange(data_distribution.distribution_type)) { + RETURN_IF_ERROR(_add_local_exchange_impl( + new_pip->operator_xs().size(), pool, new_pip, add_pipeline(new_pip, pip_idx + 2), + DataDistribution(ExchangeType::PASSTHROUGH), do_local_exchange, num_buckets, + bucket_seq_to_instance_idx, shuffle_idx_to_instance_idx, ignore_data_distribution)); + } + return Status::OK(); +} + +Status PipelineFragmentContext::_plan_local_exchange( + int num_buckets, const std::map& bucket_seq_to_instance_idx, + const std::map& shuffle_idx_to_instance_idx) { + for (int pip_idx = _pipelines.size() - 1; pip_idx >= 0; pip_idx--) { + _pipelines[pip_idx]->init_data_distribution(); + // Set property if child pipeline is not join operator's child. + if (!_pipelines[pip_idx]->children().empty()) { + for (auto& child : _pipelines[pip_idx]->children()) { + if (child->sink_x()->node_id() == + _pipelines[pip_idx]->operator_xs().front()->node_id()) { + RETURN_IF_ERROR(_pipelines[pip_idx]->operator_xs().front()->set_child( + child->operator_xs().back())); + _pipelines[pip_idx]->set_data_distribution(child->data_distribution()); + } + } + } + + RETURN_IF_ERROR(_plan_local_exchange( + _pipelines[pip_idx]->operator_xs().front()->ignore_data_hash_distribution() + ? _num_instances + : num_buckets, + pip_idx, _pipelines[pip_idx], bucket_seq_to_instance_idx, + shuffle_idx_to_instance_idx, + _pipelines[pip_idx]->operator_xs().front()->ignore_data_hash_distribution())); + } + return Status::OK(); +} + +Status PipelineFragmentContext::_plan_local_exchange( + int num_buckets, int pip_idx, PipelinePtr pip, + const std::map& bucket_seq_to_instance_idx, + const std::map& shuffle_idx_to_instance_idx, + const bool ignore_data_hash_distribution) { + int idx = 1; + bool do_local_exchange = false; + do { + auto& ops = pip->operator_xs(); + do_local_exchange = false; + // Plan local exchange for each operator. + for (; idx < ops.size();) { + if (ops[idx]->required_data_distribution().need_local_exchange()) { + RETURN_IF_ERROR(_add_local_exchange( + pip_idx, idx, ops[idx]->node_id(), _runtime_state->obj_pool(), pip, + ops[idx]->required_data_distribution(), &do_local_exchange, num_buckets, + bucket_seq_to_instance_idx, shuffle_idx_to_instance_idx, + ignore_data_hash_distribution)); + } + if (do_local_exchange) { + // If local exchange is needed for current operator, we will split this pipeline to + // two pipelines by local exchange sink/source. And then we need to process remaining + // operators in this pipeline so we set idx to 2 (0 is local exchange source and 1 + // is current operator was already processed) and continue to plan local exchange. + idx = 2; + break; + } + idx++; + } + } while (do_local_exchange); + if (pip->sink_x()->required_data_distribution().need_local_exchange()) { + RETURN_IF_ERROR(_add_local_exchange( + pip_idx, idx, pip->sink_x()->node_id(), _runtime_state->obj_pool(), pip, + pip->sink_x()->required_data_distribution(), &do_local_exchange, num_buckets, + bucket_seq_to_instance_idx, shuffle_idx_to_instance_idx, + ignore_data_hash_distribution)); + } + return Status::OK(); +} + +Status PipelineFragmentContext::_create_data_sink(ObjectPool* pool, const TDataSink& thrift_sink, + const std::vector& output_exprs, + const TPipelineFragmentParams& params, + const RowDescriptor& row_desc, + RuntimeState* state, DescriptorTbl& desc_tbl, + PipelineId cur_pipeline_id) { + switch (thrift_sink.type) { + case TDataSinkType::DATA_STREAM_SINK: { + if (!thrift_sink.__isset.stream_sink) { + return Status::InternalError("Missing data stream sink."); + } + _sink.reset(new ExchangeSinkOperatorX(state, row_desc, next_sink_operator_id(), + thrift_sink.stream_sink, params.destinations)); break; } - case TPlanNodeType::EMPTY_SET_NODE: { - OperatorBuilderPtr operator_t = - std::make_shared(node->id(), node); - RETURN_IF_ERROR(cur_pipe->add_operator(operator_t)); + case TDataSinkType::RESULT_SINK: { + if (!thrift_sink.__isset.result_sink) { + return Status::InternalError("Missing data buffer sink."); + } + + // TODO: figure out good buffer size based on size of output row + _sink.reset(new ResultSinkOperatorX(next_sink_operator_id(), row_desc, output_exprs, + thrift_sink.result_sink)); break; } - case TPlanNodeType::DATA_GEN_SCAN_NODE: { - OperatorBuilderPtr operator_t = std::make_shared(node->id(), node); - RETURN_IF_ERROR(cur_pipe->add_operator(operator_t)); + case TDataSinkType::GROUP_COMMIT_OLAP_TABLE_SINK: + case TDataSinkType::OLAP_TABLE_SINK: { + if (state->query_options().enable_memtable_on_sink_node && + !_has_inverted_index_or_partial_update(thrift_sink.olap_table_sink) && + !config::is_cloud_mode()) { + _sink.reset(new OlapTableSinkV2OperatorX(pool, next_sink_operator_id(), row_desc, + output_exprs)); + } else { + _sink.reset(new OlapTableSinkOperatorX(pool, next_sink_operator_id(), row_desc, + output_exprs)); + } break; } - case TPlanNodeType::UNION_NODE: { - auto* union_node = assert_cast(node); - if (union_node->children_count() == 0 && - union_node->get_first_materialized_child_idx() == 0) { // only have const expr - OperatorBuilderPtr builder = - std::make_shared(node->id(), node); - RETURN_IF_ERROR(cur_pipe->add_operator(builder)); + case TDataSinkType::GROUP_COMMIT_BLOCK_SINK: { + DCHECK(thrift_sink.__isset.olap_table_sink); + _sink.reset(new GroupCommitBlockSinkOperatorX(next_sink_operator_id(), row_desc)); + break; + } + case TDataSinkType::HIVE_TABLE_SINK: { + if (!thrift_sink.__isset.hive_table_sink) { + return Status::InternalError("Missing hive table sink."); + } + _sink.reset( + new HiveTableSinkOperatorX(pool, next_sink_operator_id(), row_desc, output_exprs)); + break; + } + case TDataSinkType::JDBC_TABLE_SINK: { + if (!thrift_sink.__isset.jdbc_table_sink) { + return Status::InternalError("Missing data jdbc sink."); + } + if (config::enable_java_support) { + _sink.reset( + new JdbcTableSinkOperatorX(row_desc, next_sink_operator_id(), output_exprs)); } else { - int child_count = union_node->children_count(); - auto data_queue = std::make_shared(child_count); - data_queue->set_max_blocks_in_sub_queue(_runtime_state->data_queue_max_blocks()); - for (int child_id = 0; child_id < child_count; ++child_id) { - auto new_child_pipeline = add_pipeline(); - RETURN_IF_ERROR(_build_pipelines(union_node->child(child_id), new_child_pipeline)); - OperatorBuilderPtr child_sink_builder = std::make_shared( - union_node->id(), child_id, union_node, data_queue); - RETURN_IF_ERROR(new_child_pipeline->set_sink_builder(child_sink_builder)); - } - OperatorBuilderPtr source_builder = std::make_shared( - node->id(), union_node, data_queue); - RETURN_IF_ERROR(cur_pipe->add_operator(source_builder)); + return Status::InternalError( + "Jdbc table sink is not enabled, you can change be config " + "enable_java_support to true and restart be."); } break; } - case TPlanNodeType::AGGREGATION_NODE: { - auto* agg_node = static_cast(node); - auto new_pipe = add_pipeline(); - RETURN_IF_ERROR(_build_pipelines(node->child(0), new_pipe)); - if (agg_node->is_probe_expr_ctxs_empty() && agg_node->agg_output_desc()->slots().empty()) { - return Status::InternalError("Illegal aggregate node " + - std::to_string(agg_node->id()) + - ": group by and output is empty"); + case TDataSinkType::RESULT_FILE_SINK: { + if (!thrift_sink.__isset.result_file_sink) { + return Status::InternalError("Missing result file sink."); } - const int64_t data_queue_max_blocks = _runtime_state->data_queue_max_blocks(); - if (agg_node->is_aggregate_evaluators_empty() && !agg_node->is_probe_expr_ctxs_empty()) { - auto data_queue = std::make_shared(1); - data_queue->set_max_blocks_in_sub_queue(data_queue_max_blocks); - OperatorBuilderPtr pre_agg_sink = - std::make_shared(node->id(), agg_node, - data_queue); - RETURN_IF_ERROR(new_pipe->set_sink_builder(pre_agg_sink)); - - OperatorBuilderPtr pre_agg_source = - std::make_shared( - node->id(), agg_node, data_queue); - RETURN_IF_ERROR(cur_pipe->add_operator(pre_agg_source)); - } else if (agg_node->is_streaming_preagg() && !agg_node->is_probe_expr_ctxs_empty()) { - auto data_queue = std::make_shared(1); - data_queue->set_max_blocks_in_sub_queue(data_queue_max_blocks); - OperatorBuilderPtr pre_agg_sink = std::make_shared( - node->id(), agg_node, data_queue); - RETURN_IF_ERROR(new_pipe->set_sink_builder(pre_agg_sink)); - - OperatorBuilderPtr pre_agg_source = std::make_shared( - node->id(), agg_node, data_queue); - RETURN_IF_ERROR(cur_pipe->add_operator(pre_agg_source)); + // TODO: figure out good buffer size based on size of output row + // Result file sink is not the top sink + if (params.__isset.destinations && !params.destinations.empty()) { + _sink.reset(new ResultFileSinkOperatorX(next_sink_operator_id(), row_desc, + thrift_sink.result_file_sink, + params.destinations, output_exprs, desc_tbl)); } else { - OperatorBuilderPtr agg_sink = - std::make_shared(node->id(), agg_node); - RETURN_IF_ERROR(new_pipe->set_sink_builder(agg_sink)); - - OperatorBuilderPtr agg_source = - std::make_shared(node->id(), agg_node); - RETURN_IF_ERROR(cur_pipe->add_operator(agg_source)); + _sink.reset( + new ResultFileSinkOperatorX(next_sink_operator_id(), row_desc, output_exprs)); } break; } - case TPlanNodeType::SORT_NODE: { - auto new_pipeline = add_pipeline(); - RETURN_IF_ERROR(_build_pipelines(node->child(0), new_pipeline)); + case TDataSinkType::MULTI_CAST_DATA_STREAM_SINK: { + DCHECK(thrift_sink.__isset.multi_cast_stream_sink); + DCHECK_GT(thrift_sink.multi_cast_stream_sink.sinks.size(), 0); + // TODO: figure out good buffer size based on size of output row + auto sink_id = next_sink_operator_id(); + auto sender_size = thrift_sink.multi_cast_stream_sink.sinks.size(); + // one sink has multiple sources. + std::vector sources; + for (int i = 0; i < sender_size; ++i) { + auto source_id = next_operator_id(); + sources.push_back(source_id); + } - OperatorBuilderPtr sort_sink = std::make_shared(node->id(), node); - RETURN_IF_ERROR(new_pipeline->set_sink_builder(sort_sink)); + _sink.reset(new MultiCastDataStreamSinkOperatorX( + sink_id, sources, thrift_sink.multi_cast_stream_sink.sinks.size(), pool, + thrift_sink.multi_cast_stream_sink, row_desc)); + for (int i = 0; i < sender_size; ++i) { + auto new_pipeline = add_pipeline(); + RowDescriptor* _row_desc = nullptr; + { + const auto& tmp_row_desc = + !thrift_sink.multi_cast_stream_sink.sinks[i].output_exprs.empty() + ? RowDescriptor(state->desc_tbl(), + {thrift_sink.multi_cast_stream_sink.sinks[i] + .output_tuple_id}, + {false}) + : _sink->row_desc(); + _row_desc = pool->add(new RowDescriptor(tmp_row_desc)); + } + auto source_id = sources[i]; + OperatorXPtr source_op; + // 1. create and set the source operator of multi_cast_data_stream_source for new pipeline + source_op.reset(new MultiCastDataStreamerSourceOperatorX( + i, pool, thrift_sink.multi_cast_stream_sink.sinks[i], row_desc, source_id)); + RETURN_IF_ERROR(new_pipeline->add_operator(source_op)); + // 2. create and set sink operator of data stream sender for new pipeline + + DataSinkOperatorXPtr sink_op; + sink_op.reset( + new ExchangeSinkOperatorX(state, *_row_desc, next_sink_operator_id(), + thrift_sink.multi_cast_stream_sink.sinks[i], + thrift_sink.multi_cast_stream_sink.destinations[i])); + + RETURN_IF_ERROR(new_pipeline->set_sink(sink_op)); + { + TDataSink* t = pool->add(new TDataSink()); + t->stream_sink = thrift_sink.multi_cast_stream_sink.sinks[i]; + RETURN_IF_ERROR(sink_op->init(*t)); + } - OperatorBuilderPtr sort_source = - std::make_shared(node->id(), node); - RETURN_IF_ERROR(cur_pipe->add_operator(sort_source)); + // 3. set dependency dag + _dag[new_pipeline->id()].push_back(cur_pipeline_id); + } + if (sources.empty()) { + return Status::InternalError("size of sources must be greater than 0"); + } break; } - case TPlanNodeType::PARTITION_SORT_NODE: { - auto new_pipeline = add_pipeline(); - RETURN_IF_ERROR(_build_pipelines(node->child(0), new_pipeline)); - - OperatorBuilderPtr partition_sort_sink = - std::make_shared(node->id(), node); - RETURN_IF_ERROR(new_pipeline->set_sink_builder(partition_sort_sink)); + default: + return Status::InternalError("Unsuported sink type in pipeline: {}", thrift_sink.type); + } + return Status::OK(); +} - OperatorBuilderPtr partition_sort_source = - std::make_shared(node->id(), node); - RETURN_IF_ERROR(cur_pipe->add_operator(partition_sort_source)); +// NOLINTBEGIN(readability-function-size) +// NOLINTBEGIN(readability-function-cognitive-complexity) +Status PipelineFragmentContext::_create_operator(ObjectPool* pool, const TPlanNode& tnode, + const doris::TPipelineFragmentParams& request, + const DescriptorTbl& descs, OperatorXPtr& op, + PipelinePtr& cur_pipe, int parent_idx, + int child_idx) { + // We directly construct the operator from Thrift because the given array is in the order of preorder traversal. + // Therefore, here we need to use a stack-like structure. + _pipeline_parent_map.pop(cur_pipe, parent_idx, child_idx); + std::stringstream error_msg; + + switch (tnode.node_type) { + case TPlanNodeType::OLAP_SCAN_NODE: { + op.reset(new OlapScanOperatorX(pool, tnode, next_operator_id(), descs, _num_instances)); + RETURN_IF_ERROR(cur_pipe->add_operator(op)); + if (request.__isset.parallel_instances) { + cur_pipe->set_num_tasks(request.parallel_instances); + op->set_ignore_data_distribution(); + } break; } - case TPlanNodeType::ANALYTIC_EVAL_NODE: { - auto new_pipeline = add_pipeline(); - RETURN_IF_ERROR(_build_pipelines(node->child(0), new_pipeline)); - - OperatorBuilderPtr analytic_sink = - std::make_shared(node->id(), node); - RETURN_IF_ERROR(new_pipeline->set_sink_builder(analytic_sink)); - - OperatorBuilderPtr analytic_source = - std::make_shared(node->id(), node); - RETURN_IF_ERROR(cur_pipe->add_operator(analytic_source)); + case doris::TPlanNodeType::JDBC_SCAN_NODE: { + if (config::enable_java_support) { + op.reset(new JDBCScanOperatorX(pool, tnode, next_operator_id(), descs, _num_instances)); + RETURN_IF_ERROR(cur_pipe->add_operator(op)); + } else { + return Status::InternalError( + "Jdbc scan node is disabled, you can change be config enable_java_support " + "to true and restart be."); + } + if (request.__isset.parallel_instances) { + cur_pipe->set_num_tasks(request.parallel_instances); + op->set_ignore_data_distribution(); + } break; } - case TPlanNodeType::REPEAT_NODE: { - RETURN_IF_ERROR(_build_pipelines(node->child(0), cur_pipe)); - OperatorBuilderPtr builder = std::make_shared(node->id(), node); - RETURN_IF_ERROR(cur_pipe->add_operator(builder)); + case doris::TPlanNodeType::FILE_SCAN_NODE: { + op.reset(new FileScanOperatorX(pool, tnode, next_operator_id(), descs, _num_instances)); + RETURN_IF_ERROR(cur_pipe->add_operator(op)); + if (request.__isset.parallel_instances) { + cur_pipe->set_num_tasks(request.parallel_instances); + op->set_ignore_data_distribution(); + } break; } - case TPlanNodeType::ASSERT_NUM_ROWS_NODE: { - RETURN_IF_ERROR(_build_pipelines(node->child(0), cur_pipe)); - OperatorBuilderPtr builder = - std::make_shared(node->id(), node); - RETURN_IF_ERROR(cur_pipe->add_operator(builder)); + case TPlanNodeType::ES_SCAN_NODE: + case TPlanNodeType::ES_HTTP_SCAN_NODE: { + op.reset(new EsScanOperatorX(pool, tnode, next_operator_id(), descs, _num_instances)); + RETURN_IF_ERROR(cur_pipe->add_operator(op)); + if (request.__isset.parallel_instances) { + cur_pipe->set_num_tasks(request.parallel_instances); + op->set_ignore_data_distribution(); + } break; } - case TPlanNodeType::TABLE_FUNCTION_NODE: { - RETURN_IF_ERROR(_build_pipelines(node->child(0), cur_pipe)); - OperatorBuilderPtr builder = - std::make_shared(node->id(), node); - RETURN_IF_ERROR(cur_pipe->add_operator(builder)); + case TPlanNodeType::EXCHANGE_NODE: { + int num_senders = find_with_default(request.per_exch_num_senders, tnode.node_id, 0); + DCHECK_GT(num_senders, 0); + op.reset(new ExchangeSourceOperatorX(pool, tnode, next_operator_id(), descs, num_senders)); + RETURN_IF_ERROR(cur_pipe->add_operator(op)); + if (request.__isset.parallel_instances) { + op->set_ignore_data_distribution(); + cur_pipe->set_num_tasks(request.parallel_instances); + } + break; + } + case TPlanNodeType::AGGREGATION_NODE: { + if (tnode.agg_node.grouping_exprs.empty() && + descs.get_tuple_descriptor(tnode.agg_node.output_tuple_id)->slots().empty()) { + return Status::InternalError("Illegal aggregate node " + std::to_string(tnode.node_id) + + ": group by and output is empty"); + } + if (tnode.agg_node.aggregate_functions.empty() && !_runtime_state->enable_agg_spill() && + request.query_options.__isset.enable_distinct_streaming_aggregation && + request.query_options.enable_distinct_streaming_aggregation && + !tnode.agg_node.grouping_exprs.empty()) { + op.reset(new DistinctStreamingAggOperatorX(pool, next_operator_id(), tnode, descs)); + RETURN_IF_ERROR(cur_pipe->add_operator(op)); + } else if (tnode.agg_node.__isset.use_streaming_preaggregation && + tnode.agg_node.use_streaming_preaggregation && + !tnode.agg_node.grouping_exprs.empty()) { + op.reset(new StreamingAggOperatorX(pool, next_operator_id(), tnode, descs)); + RETURN_IF_ERROR(cur_pipe->add_operator(op)); + } else { + if (_runtime_state->enable_agg_spill() && !tnode.agg_node.grouping_exprs.empty()) { + op.reset(new PartitionedAggSourceOperatorX(pool, tnode, next_operator_id(), descs)); + } else { + op.reset(new AggSourceOperatorX(pool, tnode, next_operator_id(), descs)); + } + RETURN_IF_ERROR(cur_pipe->add_operator(op)); + + const auto downstream_pipeline_id = cur_pipe->id(); + if (_dag.find(downstream_pipeline_id) == _dag.end()) { + _dag.insert({downstream_pipeline_id, {}}); + } + cur_pipe = add_pipeline(cur_pipe); + _dag[downstream_pipeline_id].push_back(cur_pipe->id()); + + DataSinkOperatorXPtr sink; + if (_runtime_state->enable_agg_spill() && !tnode.agg_node.grouping_exprs.empty()) { + sink.reset(new PartitionedAggSinkOperatorX(pool, next_sink_operator_id(), tnode, + descs, _require_bucket_distribution)); + } else { + sink.reset(new AggSinkOperatorX(pool, next_sink_operator_id(), tnode, descs, + _require_bucket_distribution)); + } + sink->set_dests_id({op->operator_id()}); + RETURN_IF_ERROR(cur_pipe->set_sink(sink)); + RETURN_IF_ERROR(cur_pipe->sink_x()->init(tnode, _runtime_state.get())); + } break; } case TPlanNodeType::HASH_JOIN_NODE: { - auto* join_node = assert_cast(node); - auto new_pipe = add_pipeline(); - if (join_node->should_build_hash_table()) { - RETURN_IF_ERROR(_build_pipelines(node->child(1), new_pipe)); + const auto is_broadcast_join = tnode.hash_join_node.__isset.is_broadcast_join && + tnode.hash_join_node.is_broadcast_join; + const auto enable_join_spill = _runtime_state->enable_join_spill(); + if (enable_join_spill && !is_broadcast_join) { + auto tnode_ = tnode; + /// TODO: support rf in partitioned hash join + tnode_.runtime_filters.clear(); + const uint32_t partition_count = 32; + auto inner_probe_operator = + std::make_shared(pool, tnode_, 0, descs); + auto inner_sink_operator = std::make_shared( + pool, 0, tnode_, descs, _need_local_merge); + + RETURN_IF_ERROR(inner_probe_operator->init(tnode_, _runtime_state.get())); + RETURN_IF_ERROR(inner_sink_operator->init(tnode_, _runtime_state.get())); + + auto probe_operator = std::make_shared( + pool, tnode_, next_operator_id(), descs, partition_count); + probe_operator->set_inner_operators(inner_sink_operator, inner_probe_operator); + op = std::move(probe_operator); + RETURN_IF_ERROR(cur_pipe->add_operator(op)); + + const auto downstream_pipeline_id = cur_pipe->id(); + if (_dag.find(downstream_pipeline_id) == _dag.end()) { + _dag.insert({downstream_pipeline_id, {}}); + } + PipelinePtr build_side_pipe = add_pipeline(cur_pipe); + _dag[downstream_pipeline_id].push_back(build_side_pipe->id()); + + auto sink_operator = std::make_shared( + pool, next_sink_operator_id(), tnode_, descs, _need_local_merge, + partition_count); + sink_operator->set_inner_operators(inner_sink_operator, inner_probe_operator); + DataSinkOperatorXPtr sink = std::move(sink_operator); + 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())); + + _pipeline_parent_map.push(op->node_id(), cur_pipe); + _pipeline_parent_map.push(op->node_id(), build_side_pipe); + } else { + op.reset(new HashJoinProbeOperatorX(pool, tnode, next_operator_id(), descs)); + RETURN_IF_ERROR(cur_pipe->add_operator(op)); + + const auto downstream_pipeline_id = cur_pipe->id(); + if (_dag.find(downstream_pipeline_id) == _dag.end()) { + _dag.insert({downstream_pipeline_id, {}}); + } + PipelinePtr build_side_pipe = add_pipeline(cur_pipe); + _dag[downstream_pipeline_id].push_back(build_side_pipe->id()); + + DataSinkOperatorXPtr sink; + sink.reset(new HashJoinBuildSinkOperatorX(pool, next_sink_operator_id(), tnode, descs, + _need_local_merge)); + 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())); + + _pipeline_parent_map.push(op->node_id(), cur_pipe); + _pipeline_parent_map.push(op->node_id(), build_side_pipe); + } + break; + } + case TPlanNodeType::CROSS_JOIN_NODE: { + op.reset(new NestedLoopJoinProbeOperatorX(pool, tnode, next_operator_id(), descs)); + RETURN_IF_ERROR(cur_pipe->add_operator(op)); + + const auto downstream_pipeline_id = cur_pipe->id(); + if (_dag.find(downstream_pipeline_id) == _dag.end()) { + _dag.insert({downstream_pipeline_id, {}}); + } + PipelinePtr build_side_pipe = add_pipeline(cur_pipe); + _dag[downstream_pipeline_id].push_back(build_side_pipe->id()); + + DataSinkOperatorXPtr sink; + sink.reset(new NestedLoopJoinBuildSinkOperatorX(pool, next_sink_operator_id(), tnode, descs, + _need_local_merge)); + 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())); + _pipeline_parent_map.push(op->node_id(), cur_pipe); + _pipeline_parent_map.push(op->node_id(), build_side_pipe); + break; + } + case TPlanNodeType::UNION_NODE: { + int child_count = tnode.num_children; + op.reset(new UnionSourceOperatorX(pool, tnode, next_operator_id(), descs)); + RETURN_IF_ERROR(cur_pipe->add_operator(op)); + + const auto downstream_pipeline_id = cur_pipe->id(); + if (_dag.find(downstream_pipeline_id) == _dag.end()) { + _dag.insert({downstream_pipeline_id, {}}); + } + for (int i = 0; i < child_count; i++) { + PipelinePtr build_side_pipe = add_pipeline(cur_pipe); + _dag[downstream_pipeline_id].push_back(build_side_pipe->id()); + DataSinkOperatorXPtr sink; + sink.reset(new UnionSinkOperatorX(i, next_sink_operator_id(), pool, tnode, descs)); + 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())); + // preset children pipelines. if any pipeline found this as its father, will use the prepared pipeline to build. + _pipeline_parent_map.push(op->node_id(), build_side_pipe); + } + break; + } + case TPlanNodeType::SORT_NODE: { + if (_runtime_state->enable_sort_spill()) { + op.reset(new SpillSortSourceOperatorX(pool, tnode, next_operator_id(), descs)); } else { - OperatorBuilderPtr builder = std::make_shared( - node->child(1)->id(), node->child(1)->row_desc(), node->child(1)); - RETURN_IF_ERROR(new_pipe->add_operator(builder)); + op.reset(new SortSourceOperatorX(pool, tnode, next_operator_id(), descs)); } - OperatorBuilderPtr join_sink = - std::make_shared(node->id(), join_node); - RETURN_IF_ERROR(new_pipe->set_sink_builder(join_sink)); + RETURN_IF_ERROR(cur_pipe->add_operator(op)); - RETURN_IF_ERROR(_build_pipelines(node->child(0), cur_pipe)); - OperatorBuilderPtr join_source = - std::make_shared(node->id(), join_node); - RETURN_IF_ERROR(cur_pipe->add_operator(join_source)); + const auto downstream_pipeline_id = cur_pipe->id(); + if (_dag.find(downstream_pipeline_id) == _dag.end()) { + _dag.insert({downstream_pipeline_id, {}}); + } + cur_pipe = add_pipeline(cur_pipe); + _dag[downstream_pipeline_id].push_back(cur_pipe->id()); - cur_pipe->add_dependency(new_pipe); + DataSinkOperatorXPtr sink; + if (_runtime_state->enable_sort_spill()) { + sink.reset(new SpillSortSinkOperatorX(pool, next_sink_operator_id(), tnode, descs)); + } else { + sink.reset(new SortSinkOperatorX(pool, next_sink_operator_id(), tnode, descs)); + } + sink->set_dests_id({op->operator_id()}); + RETURN_IF_ERROR(cur_pipe->set_sink(sink)); + RETURN_IF_ERROR(cur_pipe->sink_x()->init(tnode, _runtime_state.get())); break; } - case TPlanNodeType::CROSS_JOIN_NODE: { - auto new_pipe = add_pipeline(); - RETURN_IF_ERROR(_build_pipelines(node->child(1), new_pipe)); - OperatorBuilderPtr join_sink = - std::make_shared(node->id(), node); - RETURN_IF_ERROR(new_pipe->set_sink_builder(join_sink)); + case doris::TPlanNodeType::PARTITION_SORT_NODE: { + op.reset(new PartitionSortSourceOperatorX(pool, tnode, next_operator_id(), descs)); + RETURN_IF_ERROR(cur_pipe->add_operator(op)); - RETURN_IF_ERROR(_build_pipelines(node->child(0), cur_pipe)); - OperatorBuilderPtr join_source = - std::make_shared(node->id(), node); - RETURN_IF_ERROR(cur_pipe->add_operator(join_source)); + const auto downstream_pipeline_id = cur_pipe->id(); + if (_dag.find(downstream_pipeline_id) == _dag.end()) { + _dag.insert({downstream_pipeline_id, {}}); + } + cur_pipe = add_pipeline(cur_pipe); + _dag[downstream_pipeline_id].push_back(cur_pipe->id()); + + DataSinkOperatorXPtr sink; + sink.reset(new PartitionSortSinkOperatorX(pool, next_sink_operator_id(), tnode, descs)); + sink->set_dests_id({op->operator_id()}); + RETURN_IF_ERROR(cur_pipe->set_sink(sink)); + RETURN_IF_ERROR(cur_pipe->sink_x()->init(tnode, _runtime_state.get())); + break; + } + case TPlanNodeType::ANALYTIC_EVAL_NODE: { + op.reset(new AnalyticSourceOperatorX(pool, tnode, next_operator_id(), descs)); + RETURN_IF_ERROR(cur_pipe->add_operator(op)); - cur_pipe->add_dependency(new_pipe); + const auto downstream_pipeline_id = cur_pipe->id(); + if (_dag.find(downstream_pipeline_id) == _dag.end()) { + _dag.insert({downstream_pipeline_id, {}}); + } + cur_pipe = add_pipeline(cur_pipe); + _dag[downstream_pipeline_id].push_back(cur_pipe->id()); + + DataSinkOperatorXPtr sink; + sink.reset(new AnalyticSinkOperatorX(pool, next_sink_operator_id(), tnode, descs)); + sink->set_dests_id({op->operator_id()}); + RETURN_IF_ERROR(cur_pipe->set_sink(sink)); + RETURN_IF_ERROR(cur_pipe->sink_x()->init(tnode, _runtime_state.get())); break; } case TPlanNodeType::INTERSECT_NODE: { - RETURN_IF_ERROR(_build_operators_for_set_operation_node(node, cur_pipe)); + RETURN_IF_ERROR(_build_operators_for_set_operation_node( + pool, tnode, descs, op, cur_pipe, parent_idx, child_idx)); break; } case TPlanNodeType::EXCEPT_NODE: { - RETURN_IF_ERROR(_build_operators_for_set_operation_node(node, cur_pipe)); + RETURN_IF_ERROR(_build_operators_for_set_operation_node( + pool, tnode, descs, op, cur_pipe, parent_idx, child_idx)); + break; + } + case TPlanNodeType::REPEAT_NODE: { + op.reset(new RepeatOperatorX(pool, tnode, next_operator_id(), descs)); + RETURN_IF_ERROR(cur_pipe->add_operator(op)); + break; + } + case TPlanNodeType::TABLE_FUNCTION_NODE: { + op.reset(new TableFunctionOperatorX(pool, tnode, next_operator_id(), descs)); + RETURN_IF_ERROR(cur_pipe->add_operator(op)); + break; + } + case TPlanNodeType::ASSERT_NUM_ROWS_NODE: { + op.reset(new AssertNumRowsOperatorX(pool, tnode, next_operator_id(), descs)); + RETURN_IF_ERROR(cur_pipe->add_operator(op)); + break; + } + case TPlanNodeType::EMPTY_SET_NODE: { + op.reset(new EmptySetSourceOperatorX(pool, tnode, next_operator_id(), descs)); + RETURN_IF_ERROR(cur_pipe->add_operator(op)); + break; + } + case TPlanNodeType::DATA_GEN_SCAN_NODE: { + op.reset(new DataGenSourceOperatorX(pool, tnode, next_operator_id(), descs)); + RETURN_IF_ERROR(cur_pipe->add_operator(op)); + break; + } + case TPlanNodeType::SCHEMA_SCAN_NODE: { + op.reset(new SchemaScanOperatorX(pool, tnode, next_operator_id(), descs)); + RETURN_IF_ERROR(cur_pipe->add_operator(op)); + break; + } + case TPlanNodeType::META_SCAN_NODE: { + op.reset(new MetaScanOperatorX(pool, tnode, next_operator_id(), descs)); + RETURN_IF_ERROR(cur_pipe->add_operator(op)); break; } case TPlanNodeType::SELECT_NODE: { - RETURN_IF_ERROR(_build_pipelines(node->child(0), cur_pipe)); - OperatorBuilderPtr builder = std::make_shared(node->id(), node); - RETURN_IF_ERROR(cur_pipe->add_operator(builder)); + op.reset(new SelectOperatorX(pool, tnode, next_operator_id(), descs)); + RETURN_IF_ERROR(cur_pipe->add_operator(op)); break; } default: - return Status::InternalError("Unsupported exec type in pipeline: {}", - print_plan_node_type(node_type)); + return Status::InternalError("Unsupported exec type in pipelineX: {}", + print_plan_node_type(tnode.node_type)); } + _require_bucket_distribution = true; + return Status::OK(); } +// NOLINTEND(readability-function-cognitive-complexity) +// NOLINTEND(readability-function-size) template -Status PipelineFragmentContext::_build_operators_for_set_operation_node(ExecNode* node, - PipelinePtr cur_pipe) { - auto build_pipeline = add_pipeline(); - RETURN_IF_ERROR(_build_pipelines(node->child(0), build_pipeline)); - OperatorBuilderPtr sink_builder = - std::make_shared>(node->id(), node); - RETURN_IF_ERROR(build_pipeline->set_sink_builder(sink_builder)); - std::vector all_pipelines; - all_pipelines.emplace_back(build_pipeline); - for (int child_id = 1; child_id < node->children_count(); ++child_id) { - auto probe_pipeline = add_pipeline(); - RETURN_IF_ERROR(_build_pipelines(node->child(child_id), probe_pipeline)); - OperatorBuilderPtr probe_sink_builder = - std::make_shared>(node->id(), child_id, - node); - RETURN_IF_ERROR(probe_pipeline->set_sink_builder(probe_sink_builder)); - //eg: These sinks must be completed one by one in order, child(1) must wait child(0) build finish - probe_pipeline->add_dependency(all_pipelines[child_id - 1]); - all_pipelines.emplace_back(probe_pipeline); - } - - OperatorBuilderPtr source_builder = - std::make_shared>(node->id(), node); - return cur_pipe->add_operator(source_builder); +Status PipelineFragmentContext::_build_operators_for_set_operation_node( + ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs, OperatorXPtr& op, + PipelinePtr& cur_pipe, int parent_idx, int child_idx) { + op.reset(new SetSourceOperatorX(pool, tnode, next_operator_id(), descs)); + RETURN_IF_ERROR(cur_pipe->add_operator(op)); + + const auto downstream_pipeline_id = cur_pipe->id(); + if (_dag.find(downstream_pipeline_id) == _dag.end()) { + _dag.insert({downstream_pipeline_id, {}}); + } + + for (int child_id = 0; child_id < tnode.num_children; child_id++) { + PipelinePtr probe_side_pipe = add_pipeline(cur_pipe); + _dag[downstream_pipeline_id].push_back(probe_side_pipe->id()); + + DataSinkOperatorXPtr sink; + if (child_id == 0) { + sink.reset(new SetSinkOperatorX(child_id, next_sink_operator_id(), pool, + tnode, descs)); + } else { + sink.reset(new SetProbeSinkOperatorX(child_id, next_sink_operator_id(), + pool, tnode, descs)); + } + sink->set_dests_id({op->operator_id()}); + RETURN_IF_ERROR(probe_side_pipe->set_sink(sink)); + RETURN_IF_ERROR(probe_side_pipe->sink_x()->init(tnode, _runtime_state.get())); + // prepare children pipelines. if any pipeline found this as its father, will use the prepared pipeline to build. + _pipeline_parent_map.push(op->node_id(), probe_side_pipe); + } + + return Status::OK(); } Status PipelineFragmentContext::submit() { @@ -752,23 +1451,16 @@ Status PipelineFragmentContext::submit() { Status st; auto* scheduler = _query_ctx->get_pipe_exec_scheduler(); for (auto& task : _tasks) { - st = scheduler->schedule_task(task.get()); - if (!st) { - { + for (auto& t : task) { + st = scheduler->schedule_task(t.get()); + if (!st) { std::lock_guard l(_status_lock); cancel(PPlanFragmentCancelReason::INTERNAL_ERROR, "submit context fail"); - } - { - // The fragment instance may contains 10 tasks, maybe 8 of them is in scheduler running - // and the 9th failed to add to scheduler, then it will update total_tasks. - // But the previous 8 tasks may finished and try to access total_tasks. So that - // has to use a lock to protect it. - std::lock_guard l(_task_mutex); _total_tasks = submit_tasks; + break; } - break; + submit_tasks++; } - submit_tasks++; } if (!st.ok()) { std::lock_guard l(_task_mutex); @@ -783,134 +1475,25 @@ Status PipelineFragmentContext::submit() { } void PipelineFragmentContext::close_sink() { - if (_sink) { - if (_prepared) { - static_cast( - _sink->close(_runtime_state.get(), Status::RuntimeError("prepare failed"))); - } else { - static_cast(_sink->close(_runtime_state.get(), Status::OK())); + for (auto& tasks : _tasks) { + auto& root_task = *tasks.begin(); + auto st = root_task->close_sink(_prepared ? Status::RuntimeError("prepare failed") + : Status::OK()); + if (!st.ok()) { + LOG_WARNING("PipelineFragmentContext::close_sink() error").tag("msg", st.msg()); } } } -void PipelineFragmentContext::close_if_prepare_failed(Status /*st*/) { - if (_tasks.empty()) { - if (_root_plan) { - static_cast(_root_plan->close(_runtime_state.get())); - } - if (_sink) { - static_cast( - _sink->close(_runtime_state.get(), Status::RuntimeError("prepare failed"))); - } - } +void PipelineFragmentContext::close_if_prepare_failed(Status st) { for (auto& task : _tasks) { - DCHECK(!task->is_pending_finish()); - WARN_IF_ERROR(task->close(Status::OK()), - fmt::format("Query {} closed since prepare failed", print_id(_query_id))); - close_a_pipeline(); - } -} - -// construct sink operator -Status PipelineFragmentContext::_create_sink(int sender_id, const TDataSink& thrift_sink, - RuntimeState* state) { - OperatorBuilderPtr sink_; - switch (thrift_sink.type) { - case TDataSinkType::DATA_STREAM_SINK: { - sink_ = std::make_shared(thrift_sink.stream_sink.dest_node_id, - _sink.get()); - break; - } - case TDataSinkType::RESULT_SINK: { - sink_ = std::make_shared(next_operator_builder_id(), - _sink.get()); - break; - } - case TDataSinkType::GROUP_COMMIT_OLAP_TABLE_SINK: - case TDataSinkType::OLAP_TABLE_SINK: { - DCHECK(thrift_sink.__isset.olap_table_sink); - if (state->query_options().enable_memtable_on_sink_node && - !_has_inverted_index_or_partial_update(thrift_sink.olap_table_sink) && - !config::is_cloud_mode()) { - sink_ = std::make_shared(next_operator_builder_id(), - _sink.get()); - } else { - sink_ = std::make_shared(next_operator_builder_id(), - _sink.get()); - } - break; - } - case TDataSinkType::GROUP_COMMIT_BLOCK_SINK: { - sink_ = std::make_shared(next_operator_builder_id(), - _sink.get()); - break; - } - case TDataSinkType::HIVE_TABLE_SINK: { - sink_ = std::make_shared(next_operator_builder_id(), - _sink.get()); - break; - } - case TDataSinkType::MYSQL_TABLE_SINK: - case TDataSinkType::JDBC_TABLE_SINK: - case TDataSinkType::ODBC_TABLE_SINK: - case TDataSinkType::RESULT_FILE_SINK: { - sink_ = std::make_shared( - thrift_sink.result_file_sink.dest_node_id, _sink.get()); - break; - } - case TDataSinkType::MULTI_CAST_DATA_STREAM_SINK: { - sink_ = std::make_shared(next_operator_builder_id(), - _sink.get()); - RETURN_IF_ERROR(_root_pipeline->set_sink_builder(sink_)); - - auto& multi_cast_data_streamer = - assert_cast(_sink.get()) - ->get_multi_cast_data_streamer(); - DCHECK_EQ(thrift_sink.multi_cast_stream_sink.sinks.size(), - thrift_sink.multi_cast_stream_sink.destinations.size()); - auto sender_size = thrift_sink.multi_cast_stream_sink.sinks.size(); - _multi_cast_stream_sink_senders.resize(sender_size); - for (int i = 0; i < sender_size; ++i) { - auto new_pipeline = add_pipeline(); - - auto row_desc = - !thrift_sink.multi_cast_stream_sink.sinks[i].output_exprs.empty() - ? RowDescriptor( - _runtime_state->desc_tbl(), - {thrift_sink.multi_cast_stream_sink.sinks[i].output_tuple_id}, - {false}) - : sink_->row_desc(); - // 1. create the data stream sender sink - _multi_cast_stream_sink_senders[i] = std::make_unique( - _runtime_state.get(), _runtime_state->obj_pool(), sender_id, row_desc, - thrift_sink.multi_cast_stream_sink.sinks[i], - thrift_sink.multi_cast_stream_sink.destinations[i]); - - // 2. create and set the source operator of multi_cast_data_stream_source for new pipeline - OperatorBuilderPtr source_op = - std::make_shared( - next_operator_builder_id(), i, multi_cast_data_streamer, - thrift_sink.multi_cast_stream_sink.sinks[i]); - static_cast(new_pipeline->add_operator(source_op)); - - // 3. create and set sink operator of data stream sender for new pipeline - OperatorBuilderPtr sink_op_builder = std::make_shared( - next_operator_builder_id(), _multi_cast_stream_sink_senders[i].get(), i); - static_cast(new_pipeline->set_sink_builder(sink_op_builder)); - - // 4. init and prepare the data_stream_sender of diff exchange - TDataSink t; - t.stream_sink = thrift_sink.multi_cast_stream_sink.sinks[i]; - RETURN_IF_ERROR(_multi_cast_stream_sink_senders[i]->init(t)); - RETURN_IF_ERROR(_multi_cast_stream_sink_senders[i]->prepare(state)); + for (auto& t : task) { + DCHECK(!t->is_pending_finish()); + WARN_IF_ERROR(t->close(st), "close_if_prepare_failed failed: "); + close_a_pipeline(); } - - return Status::OK(); - } - default: - return Status::InternalError("Unsuported sink type in pipeline: {}", thrift_sink.type); } - return _root_pipeline->set_sink_builder(sink_); + _query_ctx->cancel(st.to_string(), st, _fragment_id); } // If all pipeline tasks binded to the fragment instance are finished, then we could @@ -921,8 +1504,6 @@ void PipelineFragmentContext::_close_fragment_instance() { } Defer defer_op {[&]() { _is_fragment_instance_closed = true; }}; _runtime_profile->total_time_counter()->update(_fragment_watcher.elapsed_time()); - _runtime_state->runtime_profile()->total_time_counter()->update( - _fragment_watcher.elapsed_time()); static_cast(send_report(true)); if (_runtime_state->enable_profile()) { std::stringstream ss; @@ -938,9 +1519,15 @@ void PipelineFragmentContext::_close_fragment_instance() { _runtime_state->load_channel_profile()->pretty_print(&ss); } - LOG_INFO("Query {} fragment {} instance {} profile:\n {}", print_id(this->_query_id), - this->_fragment_id, print_id(this->get_fragment_instance_id()), ss.str()); + LOG_INFO("Query {} fragment {} profile:\n {}", print_id(this->_query_id), + this->_fragment_id, ss.str()); } + + if (_query_ctx->enable_profile()) { + _query_ctx->add_fragment_profile_x(_fragment_id, collect_realtime_profile_x(), + collect_realtime_load_channel_profile_x()); + } + // all submitted tasks done _exec_env->fragment_mgr()->remove_pipeline_context( std::dynamic_pointer_cast(shared_from_this())); @@ -976,31 +1563,100 @@ Status PipelineFragmentContext::send_report(bool done) { return Status::NeedSendAgain(""); } + std::vector runtime_states; + + for (auto& task_state : _task_runtime_states) { + runtime_states.push_back(task_state.get()); + } + + ReportStatusRequest req {true, + exec_status, + runtime_states, + nullptr, + _runtime_state->load_channel_profile(), + done || !exec_status.ok(), + _query_ctx->coord_addr, + _query_id, + _fragment_id, + TUniqueId(), + -1, + _runtime_state.get(), + [this](Status st) { return update_status(st); }, + [this](const PPlanFragmentCancelReason& reason, + const std::string& msg) { cancel(reason, msg); }}; + return _report_status_cb( - {false, - exec_status, - {}, - _runtime_state->enable_profile() ? _runtime_state->runtime_profile() : nullptr, - _runtime_state->enable_profile() ? _runtime_state->load_channel_profile() : nullptr, - done || !exec_status.ok(), - _query_ctx->coord_addr, - _query_id, - _fragment_id, - _fragment_instance_id, - _backend_num, - _runtime_state.get(), - [this](Status st) { return update_status(st); }, - [this](const PPlanFragmentCancelReason& reason, const std::string& msg) { - cancel(reason, msg); - }}, - std::dynamic_pointer_cast(shared_from_this())); + req, std::dynamic_pointer_cast(shared_from_this())); } std::string PipelineFragmentContext::debug_string() { fmt::memory_buffer debug_string_buffer; - fmt::format_to(debug_string_buffer, "PipelineFragmentContext Info: QueryId = {}\n", - print_id(_query_ctx->query_id())); + fmt::format_to(debug_string_buffer, "PipelineFragmentContext Info:\n"); + for (size_t j = 0; j < _tasks.size(); j++) { + fmt::format_to(debug_string_buffer, "Tasks in instance {}:\n", j); + for (size_t i = 0; i < _tasks[j].size(); i++) { + fmt::format_to(debug_string_buffer, "Task {}: {}\n", i, _tasks[j][i]->debug_string()); + } + } + return fmt::to_string(debug_string_buffer); } +std::vector> +PipelineFragmentContext::collect_realtime_profile_x() const { + std::vector> res; + DCHECK(_query_ctx->enable_pipeline_x_exec() == true) + << fmt::format("Query {} calling a pipeline X function, but its pipeline X is disabled", + print_id(this->_query_id)); + + // we do not have mutex to protect pipeline_id_to_profile + // so we need to make sure this funciton is invoked after fragment context + // has already been prepared. + if (!this->_prepared) { + std::string msg = + "Query " + print_id(this->_query_id) + " collecting profile, but its not prepared"; + DCHECK(false) << msg; + LOG_ERROR(msg); + return res; + } + + // pipeline_id_to_profile is initialized in prepare stage + for (auto& pipeline_profile : _runtime_state->pipeline_id_to_profile()) { + auto profile_ptr = std::make_shared(); + pipeline_profile->to_thrift(profile_ptr.get()); + res.push_back(profile_ptr); + } + + return res; +} + +std::shared_ptr +PipelineFragmentContext::collect_realtime_load_channel_profile_x() const { + // we do not have mutex to protect pipeline_id_to_profile + // so we need to make sure this funciton is invoked after fragment context + // has already been prepared. + if (!this->_prepared) { + std::string msg = + "Query " + print_id(this->_query_id) + " collecting profile, but its not prepared"; + DCHECK(false) << msg; + LOG_ERROR(msg); + return nullptr; + } + + for (auto& runtime_state : _task_runtime_states) { + if (runtime_state->runtime_profile() == nullptr) { + continue; + } + + auto tmp_load_channel_profile = std::make_shared(); + + runtime_state->runtime_profile()->to_thrift(tmp_load_channel_profile.get()); + this->_runtime_state->load_channel_profile()->update(*tmp_load_channel_profile); + } + + auto load_channel_profile = std::make_shared(); + this->_runtime_state->load_channel_profile()->to_thrift(load_channel_profile.get()); + return load_channel_profile; +} + } // namespace doris::pipeline diff --git a/be/src/pipeline/pipeline_fragment_context.h b/be/src/pipeline/pipeline_fragment_context.h index b9bfcb28f68af25..0c3af6733bccbab 100644 --- a/be/src/pipeline/pipeline_fragment_context.h +++ b/be/src/pipeline/pipeline_fragment_context.h @@ -31,7 +31,10 @@ #include "common/status.h" #include "pipeline/pipeline.h" +#include "pipeline/pipeline_fragment_context.h" #include "pipeline/pipeline_task.h" +#include "pipeline/pipeline_x/local_exchange/local_exchanger.h" +#include "pipeline/pipeline_x/pipeline_x_task.h" #include "runtime/query_context.h" #include "runtime/runtime_state.h" #include "runtime/task_execution_context.h" @@ -39,8 +42,6 @@ #include "util/stopwatch.hpp" namespace doris { -class ExecNode; -class DataSink; struct ReportStatusRequest; class ExecEnv; class RuntimeFilterMergeControllerEntity; @@ -49,6 +50,8 @@ class TPipelineFragmentParams; namespace pipeline { +class Dependency; + class PipelineFragmentContext : public TaskExecutionContext { public: ENABLE_FACTORY_CREATOR(PipelineFragmentContext); @@ -61,13 +64,15 @@ class PipelineFragmentContext : public TaskExecutionContext { using report_status_callback = std::function&&)>; PipelineFragmentContext() = default; - PipelineFragmentContext(const TUniqueId& query_id, const TUniqueId& instance_id, - int fragment_id, int backend_num, + PipelineFragmentContext(const TUniqueId& query_id, const int fragment_id, std::shared_ptr query_ctx, ExecEnv* exec_env, const std::function& call_back, - report_status_callback report_status_cb); + const report_status_callback& report_status_cb); + + ~PipelineFragmentContext(); - ~PipelineFragmentContext() override; + std::vector> collect_realtime_profile_x() const; + std::shared_ptr collect_realtime_load_channel_profile_x() const; bool is_timeout(const VecDateTimeValue& now) const; @@ -75,32 +80,23 @@ class PipelineFragmentContext : public TaskExecutionContext { PipelinePtr add_pipeline(PipelinePtr parent, int idx = -1); - TUniqueId get_fragment_instance_id() const { return _fragment_instance_id; } - RuntimeState* get_runtime_state() { return _runtime_state.get(); } QueryContext* get_query_ctx() { return _query_ctx.get(); } // should be protected by lock? [[nodiscard]] bool is_canceled() const { return _runtime_state->is_cancelled(); } - int32_t next_operator_builder_id() { return _next_operator_builder_id++; } - - Status prepare(const doris::TPipelineFragmentParams& request, size_t idx); - - virtual Status prepare(const doris::TPipelineFragmentParams& request) { - return Status::InternalError("Pipeline fragment context do not implement prepare"); - } + Status prepare(const doris::TPipelineFragmentParams& request); - virtual Status submit(); + Status submit(); - virtual void close_if_prepare_failed(Status st); - virtual void close_sink(); + void close_if_prepare_failed(Status st); + void close_sink(); void set_is_report_success(bool is_report_success) { _is_report_success = is_report_success; } - virtual void cancel( - const PPlanFragmentCancelReason& reason = PPlanFragmentCancelReason::INTERNAL_ERROR, - const std::string& msg = ""); + void cancel(const PPlanFragmentCancelReason& reason = PPlanFragmentCancelReason::INTERNAL_ERROR, + const std::string& msg = ""); // TODO: Support pipeline runtime filter @@ -110,10 +106,7 @@ class PipelineFragmentContext : public TaskExecutionContext { void close_a_pipeline(); - virtual void add_merge_controller_handler( - std::shared_ptr& handler) {} - - virtual Status send_report(bool); + Status send_report(bool); Status update_status(Status status) { std::lock_guard l(_status_lock); @@ -124,36 +117,94 @@ class PipelineFragmentContext : public TaskExecutionContext { } void trigger_report_if_necessary(); - virtual void instance_ids(std::vector& ins_ids) const { - ins_ids.resize(1); - ins_ids[0] = _fragment_instance_id; - } - virtual void instance_ids(std::vector& ins_ids) const { - ins_ids.resize(1); - ins_ids[0] = print_id(_fragment_instance_id); - } void refresh_next_report_time(); - virtual std::string debug_string(); + std::string debug_string(); uint64_t create_time() const { return _create_time; } -protected: - Status _create_sink(int sender_id, const TDataSink& t_data_sink, RuntimeState* state); - Status _build_pipelines(ExecNode*, PipelinePtr); - virtual Status _build_pipeline_tasks(const doris::TPipelineFragmentParams& request); + [[nodiscard]] int next_operator_id() { return _operator_id--; } + + [[nodiscard]] int max_operator_id() const { return _operator_id; } + + [[nodiscard]] int next_sink_operator_id() { return _sink_operator_id--; } + + [[nodiscard]] int max_sink_operator_id() const { return _sink_operator_id; } + + void instance_ids(std::vector& ins_ids) const { + ins_ids.resize(_fragment_instance_ids.size()); + for (size_t i = 0; i < _fragment_instance_ids.size(); i++) { + ins_ids[i] = _fragment_instance_ids[i]; + } + } + + void instance_ids(std::vector& ins_ids) const { + ins_ids.resize(_fragment_instance_ids.size()); + for (size_t i = 0; i < _fragment_instance_ids.size(); i++) { + ins_ids[i] = print_id(_fragment_instance_ids[i]); + } + } + + void add_merge_controller_handler( + std::shared_ptr& handler) { + _merge_controller_handlers.emplace_back(handler); + } + +private: + Status _build_pipelines(ObjectPool* pool, const doris::TPipelineFragmentParams& request, + const DescriptorTbl& descs, OperatorXPtr* root, PipelinePtr cur_pipe); + Status _create_tree_helper(ObjectPool* pool, const std::vector& tnodes, + const doris::TPipelineFragmentParams& request, + const DescriptorTbl& descs, OperatorXPtr parent, int* node_idx, + OperatorXPtr* root, PipelinePtr& cur_pipe, int child_idx); + + Status _create_operator(ObjectPool* pool, const TPlanNode& tnode, + const doris::TPipelineFragmentParams& request, + const DescriptorTbl& descs, OperatorXPtr& op, PipelinePtr& cur_pipe, + int parent_idx, int child_idx); template - Status _build_operators_for_set_operation_node(ExecNode*, PipelinePtr); - virtual void _close_fragment_instance(); + Status _build_operators_for_set_operation_node(ObjectPool* pool, const TPlanNode& tnode, + const DescriptorTbl& descs, OperatorXPtr& op, + PipelinePtr& cur_pipe, int parent_idx, + int child_idx); + + Status _create_data_sink(ObjectPool* pool, const TDataSink& thrift_sink, + const std::vector& output_exprs, + const TPipelineFragmentParams& params, const RowDescriptor& row_desc, + RuntimeState* state, DescriptorTbl& desc_tbl, + PipelineId cur_pipeline_id); + Status _plan_local_exchange(int num_buckets, + const std::map& bucket_seq_to_instance_idx, + const std::map& shuffle_idx_to_instance_idx); + Status _plan_local_exchange(int num_buckets, int pip_idx, PipelinePtr pip, + const std::map& bucket_seq_to_instance_idx, + const std::map& shuffle_idx_to_instance_idx, + const bool ignore_data_distribution); + void _inherit_pipeline_properties(const DataDistribution& data_distribution, + PipelinePtr pipe_with_source, PipelinePtr pipe_with_sink); + Status _add_local_exchange(int pip_idx, int idx, int node_id, ObjectPool* pool, + PipelinePtr cur_pipe, DataDistribution data_distribution, + bool* do_local_exchange, int num_buckets, + const std::map& bucket_seq_to_instance_idx, + const std::map& shuffle_idx_to_instance_idx, + const bool ignore_data_distribution); + Status _add_local_exchange_impl(int idx, ObjectPool* pool, PipelinePtr cur_pipe, + PipelinePtr new_pip, DataDistribution data_distribution, + bool* do_local_exchange, int num_buckets, + const std::map& bucket_seq_to_instance_idx, + const std::map& shuffle_idx_to_instance_idx, + const bool ignore_data_hash_distribution); + + bool _enable_local_shuffle() const { return _runtime_state->enable_local_shuffle(); } + + Status _build_pipeline_tasks(const doris::TPipelineFragmentParams& request); + void _close_fragment_instance(); void _init_next_report_time(); // Id of this query TUniqueId _query_id; - TUniqueId _fragment_instance_id; int _fragment_id; - int _backend_num; - ExecEnv* _exec_env = nullptr; std::atomic_bool _prepared = false; @@ -169,27 +220,16 @@ class PipelineFragmentContext : public TaskExecutionContext { // When submit fail, `_total_tasks` is equal to the number of tasks submitted. int _total_tasks = 0; - int32_t _next_operator_builder_id = 10000; - - PipelinePtr _root_pipeline; - std::unique_ptr _runtime_profile; bool _is_report_success = false; std::unique_ptr _runtime_state; - ExecNode* _root_plan = nullptr; // lives in _runtime_state->obj_pool() - // TODO: remove the _sink and _multi_cast_stream_sink_senders to set both - // of it in pipeline task not the fragment_context - std::unique_ptr _sink; - std::vector> _multi_cast_stream_sink_senders; - std::shared_ptr _query_ctx; QueryThreadContext _query_thread_context; MonotonicStopWatch _fragment_watcher; - RuntimeProfile::Counter* _start_timer = nullptr; RuntimeProfile::Counter* _prepare_timer = nullptr; std::function _call_back; @@ -212,10 +252,78 @@ class PipelineFragmentContext : public TaskExecutionContext { VecDateTimeValue _start_time; int _timeout = -1; -private: - std::vector> _tasks; + OperatorXPtr _root_op = nullptr; + // this is a [n * m] matrix. n is parallelism of pipeline engine and m is the number of pipelines. + std::vector>> _tasks; + + bool _need_local_merge = false; + + // It is used to manage the lifecycle of RuntimeFilterMergeController + std::vector> _merge_controller_handlers; + + // TODO: remove the _sink and _multi_cast_stream_sink_senders to set both + // of it in pipeline task not the fragment_context +#ifdef __clang__ +#pragma clang diagnostic push +#pragma clang diagnostic ignored "-Wshadow-field" +#endif + DataSinkOperatorXPtr _sink = nullptr; +#ifdef __clang__ +#pragma clang diagnostic pop +#endif + + // `_dag` manage dependencies between pipelines by pipeline ID. the indices will be blocked by members + std::map> _dag; + + // We use preorder traversal to create an operator tree. When we meet a join node, we should + // build probe operator and build operator in separate pipelines. To do this, we should build + // ProbeSide first, and use `_pipelines_to_build` to store which pipeline the build operator + // is in, so we can build BuildSide once we complete probe side. + struct pipeline_parent_map { + std::map> _build_side_pipelines; + void push(int parent_node_id, PipelinePtr pipeline) { + if (!_build_side_pipelines.contains(parent_node_id)) { + _build_side_pipelines.insert({parent_node_id, {pipeline}}); + } else { + _build_side_pipelines[parent_node_id].push_back(pipeline); + } + } + void pop(PipelinePtr& cur_pipe, int parent_node_id, int child_idx) { + if (!_build_side_pipelines.contains(parent_node_id)) { + return; + } + DCHECK(_build_side_pipelines.contains(parent_node_id)); + auto& child_pipeline = _build_side_pipelines[parent_node_id]; + DCHECK(child_idx < child_pipeline.size()); + cur_pipe = child_pipeline[child_idx]; + } + void clear() { _build_side_pipelines.clear(); } + } _pipeline_parent_map; + + std::mutex _state_map_lock; + + int _operator_id = 0; + int _sink_operator_id = 0; + std::map, std::shared_ptr>> + _op_id_to_le_state; + + // UniqueId -> runtime mgr + std::map> _runtime_filter_mgr_map; + + //Here are two types of runtime states: + // - _runtime state is at the Fragment level. + // - _task_runtime_states is at the task level, unique to each task. + + std::vector _fragment_instance_ids; + // Local runtime states for each task + std::vector> _task_runtime_states; + + std::vector> _runtime_filter_states; + // Total instance num running on all BEs + int _total_instances = -1; uint64_t _create_time; + bool _require_bucket_distribution = false; }; } // namespace pipeline } // namespace doris \ No newline at end of file diff --git a/be/src/pipeline/pipeline_task.cpp b/be/src/pipeline/pipeline_task.cpp index f31a39df31a79db..167ff01fe695ed6 100644 --- a/be/src/pipeline/pipeline_task.cpp +++ b/be/src/pipeline/pipeline_task.cpp @@ -391,39 +391,7 @@ void PipelineTask::set_state(PipelineTaskState state) { } std::string PipelineTask::debug_string() { - fmt::memory_buffer debug_string_buffer; - - fmt::format_to(debug_string_buffer, "QueryId: {}\n", print_id(query_context()->query_id())); - fmt::format_to(debug_string_buffer, "InstanceId: {}\n", - print_id(fragment_context()->get_fragment_instance_id())); - - fmt::format_to(debug_string_buffer, "RuntimeUsage: {}\n", - PrettyPrinter::print(get_runtime_ns(), TUnit::TIME_NS)); - { - std::stringstream profile_ss; - _fresh_profile_counter(); - _task_profile->pretty_print(&profile_ss, ""); - fmt::format_to(debug_string_buffer, "Profile: {}\n", profile_ss.str()); - } - fmt::format_to(debug_string_buffer, - "PipelineTask[this = {}, state = {}]\noperators: ", (void*)this, - get_state_name(_cur_state)); - for (size_t i = 0; i < _operators.size(); i++) { - fmt::format_to(debug_string_buffer, "\n{}{}", std::string(i * 2, ' '), - _operators[i]->debug_string()); - std::stringstream profile_ss; - _operators[i]->get_runtime_profile()->pretty_print(&profile_ss, std::string(i * 2, ' ')); - fmt::format_to(debug_string_buffer, "\n{}", profile_ss.str()); - } - fmt::format_to(debug_string_buffer, "\n{}{}", std::string(_operators.size() * 2, ' '), - _sink->debug_string()); - { - std::stringstream profile_ss; - _sink->get_runtime_profile()->pretty_print(&profile_ss, - std::string(_operators.size() * 2, ' ')); - fmt::format_to(debug_string_buffer, "\n{}", profile_ss.str()); - } - return fmt::to_string(debug_string_buffer); + return ""; } } // namespace doris::pipeline diff --git a/be/src/pipeline/pipeline_task.h b/be/src/pipeline/pipeline_task.h index b9a5cb06ff872d3..aabfc91af37a392 100644 --- a/be/src/pipeline/pipeline_task.h +++ b/be/src/pipeline/pipeline_task.h @@ -254,8 +254,6 @@ class PipelineTask { void set_parent_profile(RuntimeProfile* profile) { _parent_profile = profile; } - virtual bool is_pipelineX() const { return false; } - bool is_running() { return _running.load(); } void set_running(bool running) { _running = running; } diff --git a/be/src/pipeline/pipeline_x/local_exchange/local_exchange_sink_operator.h b/be/src/pipeline/pipeline_x/local_exchange/local_exchange_sink_operator.h index b3ecf29736fda69..db6662a221ad8ed 100644 --- a/be/src/pipeline/pipeline_x/local_exchange/local_exchange_sink_operator.h +++ b/be/src/pipeline/pipeline_x/local_exchange/local_exchange_sink_operator.h @@ -114,8 +114,8 @@ class LocalExchangeSinkOperatorX final : public DataSinkOperatorX(_num_partitions)); + _partitioner.reset(new vectorized::Crc32HashPartitioner( + _num_partitions)); RETURN_IF_ERROR(_partitioner->init(_texprs)); } else if (_type == ExchangeType::BUCKET_HASH_SHUFFLE) { _partitioner.reset(new vectorized::Crc32HashPartitioner( diff --git a/be/src/pipeline/pipeline_x/operator.cpp b/be/src/pipeline/pipeline_x/operator.cpp index d5afad15fa0ec0b..8b1fb15857107a1 100644 --- a/be/src/pipeline/pipeline_x/operator.cpp +++ b/be/src/pipeline/pipeline_x/operator.cpp @@ -70,8 +70,6 @@ #include "pipeline/exec/spill_sort_sink_operator.h" #include "pipeline/exec/spill_sort_source_operator.h" #include "pipeline/exec/streaming_aggregation_operator.h" -#include "pipeline/exec/streaming_aggregation_sink_operator.h" -#include "pipeline/exec/streaming_aggregation_source_operator.h" #include "pipeline/exec/table_function_operator.h" #include "pipeline/exec/union_sink_operator.h" #include "pipeline/exec/union_source_operator.h" diff --git a/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.cpp b/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.cpp deleted file mode 100644 index bf2c255a1274f06..000000000000000 --- a/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.cpp +++ /dev/null @@ -1,1516 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#include "pipeline_x_fragment_context.h" - -#include -#include -#include -#include -#include -#include -#include - -// IWYU pragma: no_include -#include // IWYU pragma: keep -#include -#include -#include -#include - -#include "cloud/config.h" -#include "common/config.h" -#include "common/logging.h" -#include "exec/data_sink.h" -#include "exec/exec_node.h" -#include "exec/scan_node.h" -#include "io/fs/stream_load_pipe.h" -#include "pipeline/exec/aggregation_sink_operator.h" -#include "pipeline/exec/aggregation_source_operator.h" -#include "pipeline/exec/analytic_sink_operator.h" -#include "pipeline/exec/analytic_source_operator.h" -#include "pipeline/exec/assert_num_rows_operator.h" -#include "pipeline/exec/datagen_operator.h" -#include "pipeline/exec/distinct_streaming_aggregation_operator.h" -#include "pipeline/exec/empty_set_operator.h" -#include "pipeline/exec/es_scan_operator.h" -#include "pipeline/exec/exchange_sink_operator.h" -#include "pipeline/exec/exchange_source_operator.h" -#include "pipeline/exec/file_scan_operator.h" -#include "pipeline/exec/group_commit_block_sink_operator.h" -#include "pipeline/exec/hashjoin_build_sink.h" -#include "pipeline/exec/hashjoin_probe_operator.h" -#include "pipeline/exec/hive_table_sink_operator.h" -#include "pipeline/exec/jdbc_scan_operator.h" -#include "pipeline/exec/jdbc_table_sink_operator.h" -#include "pipeline/exec/meta_scan_operator.h" -#include "pipeline/exec/multi_cast_data_stream_sink.h" -#include "pipeline/exec/multi_cast_data_stream_source.h" -#include "pipeline/exec/nested_loop_join_build_operator.h" -#include "pipeline/exec/nested_loop_join_probe_operator.h" -#include "pipeline/exec/olap_scan_operator.h" -#include "pipeline/exec/olap_table_sink_operator.h" -#include "pipeline/exec/olap_table_sink_v2_operator.h" -#include "pipeline/exec/partition_sort_sink_operator.h" -#include "pipeline/exec/partition_sort_source_operator.h" -#include "pipeline/exec/partitioned_aggregation_sink_operator.h" -#include "pipeline/exec/partitioned_aggregation_source_operator.h" -#include "pipeline/exec/partitioned_hash_join_probe_operator.h" -#include "pipeline/exec/partitioned_hash_join_sink_operator.h" -#include "pipeline/exec/repeat_operator.h" -#include "pipeline/exec/result_file_sink_operator.h" -#include "pipeline/exec/result_sink_operator.h" -#include "pipeline/exec/schema_scan_operator.h" -#include "pipeline/exec/select_operator.h" -#include "pipeline/exec/set_probe_sink_operator.h" -#include "pipeline/exec/set_sink_operator.h" -#include "pipeline/exec/set_source_operator.h" -#include "pipeline/exec/sort_sink_operator.h" -#include "pipeline/exec/sort_source_operator.h" -#include "pipeline/exec/spill_sort_sink_operator.h" -#include "pipeline/exec/spill_sort_source_operator.h" -#include "pipeline/exec/streaming_aggregation_operator.h" -#include "pipeline/exec/table_function_operator.h" -#include "pipeline/exec/union_sink_operator.h" -#include "pipeline/exec/union_source_operator.h" -#include "pipeline/pipeline_x/local_exchange/local_exchange_sink_operator.h" -#include "pipeline/pipeline_x/local_exchange/local_exchange_source_operator.h" -#include "pipeline/task_scheduler.h" -#include "runtime/exec_env.h" -#include "runtime/fragment_mgr.h" -#include "runtime/runtime_filter_mgr.h" -#include "runtime/runtime_state.h" -#include "runtime/stream_load/new_load_stream_mgr.h" -#include "runtime/stream_load/stream_load_context.h" -#include "runtime/thread_context.h" -#include "service/backend_options.h" -#include "util/container_util.hpp" -#include "util/debug_util.h" -#include "util/uid_util.h" -#include "vec/runtime/vdata_stream_mgr.h" - -namespace doris::pipeline { - -PipelineXFragmentContext::PipelineXFragmentContext( - const TUniqueId& query_id, const int fragment_id, std::shared_ptr query_ctx, - ExecEnv* exec_env, const std::function& call_back, - const report_status_callback& report_status_cb) - : PipelineFragmentContext(query_id, TUniqueId(), fragment_id, -1, query_ctx, exec_env, - call_back, report_status_cb) {} - -PipelineXFragmentContext::~PipelineXFragmentContext() { - // The memory released by the query end is recorded in the query mem tracker. - SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(_query_thread_context.query_mem_tracker); - auto st = _query_ctx->exec_status(); - _tasks.clear(); - if (!_task_runtime_states.empty()) { - for (auto& runtime_state : _task_runtime_states) { - _call_back(runtime_state.get(), &st); - runtime_state.reset(); - } - } - _runtime_state.reset(); - _runtime_filter_states.clear(); - _runtime_filter_mgr_map.clear(); - _op_id_to_le_state.clear(); -} - -void PipelineXFragmentContext::cancel(const PPlanFragmentCancelReason& reason, - const std::string& msg) { - { - std::lock_guard l(_task_mutex); - if (_closed_tasks == _total_tasks) { - // All tasks in this PipelineXFragmentContext already closed. - return; - } - } - LOG_INFO("PipelineXFragmentContext::cancel") - .tag("query_id", print_id(_query_id)) - .tag("fragment_id", _fragment_id) - .tag("reason", reason) - .tag("error message", msg); - if (reason == PPlanFragmentCancelReason::TIMEOUT) { - LOG(WARNING) << "PipelineXFragmentContext is cancelled due to timeout : " << debug_string(); - } - _query_ctx->cancel(msg, Status::Cancelled(msg), _fragment_id); - if (reason == PPlanFragmentCancelReason::LIMIT_REACH) { - _is_report_on_cancel = false; - } else { - for (auto& id : _fragment_instance_ids) { - LOG(WARNING) << "PipelineXFragmentContext cancel instance: " << print_id(id); - } - } - // Get pipe from new load stream manager and send cancel to it or the fragment may hang to wait read from pipe - // For stream load the fragment's query_id == load id, it is set in FE. - auto stream_load_ctx = _exec_env->new_load_stream_mgr()->get(_query_id); - if (stream_load_ctx != nullptr) { - stream_load_ctx->pipe->cancel(msg); - } - - // Cancel the result queue manager used by spark doris connector - // TODO pipeline incomp - // _exec_env->result_queue_mgr()->update_queue_status(id, Status::Aborted(msg)); - for (auto& tasks : _tasks) { - for (auto& task : tasks) { - if (task->is_finished()) { - continue; - } - task->clear_blocking_state(); - } - } -} - -Status PipelineXFragmentContext::prepare(const doris::TPipelineFragmentParams& request) { - if (_prepared) { - return Status::InternalError("Already prepared"); - } - _num_instances = request.local_params.size(); - _total_instances = request.__isset.total_instances ? request.total_instances : _num_instances; - _runtime_profile = std::make_unique("PipelineContext"); - _prepare_timer = ADD_TIMER(_runtime_profile, "PrepareTime"); - SCOPED_TIMER(_prepare_timer); - - auto* fragment_context = this; - - LOG_INFO("PipelineXFragmentContext::prepare") - .tag("query_id", print_id(_query_id)) - .tag("fragment_id", _fragment_id) - .tag("pthread_id", (uintptr_t)pthread_self()); - - if (request.query_options.__isset.is_report_success) { - fragment_context->set_is_report_success(request.query_options.is_report_success); - } - - // 1. Set up the global runtime state. - _runtime_state = RuntimeState::create_unique(request.query_id, request.fragment_id, - request.query_options, _query_ctx->query_globals, - _exec_env, _query_ctx.get()); - - SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(_runtime_state->query_mem_tracker()); - if (request.__isset.backend_id) { - _runtime_state->set_backend_id(request.backend_id); - } - if (request.__isset.import_label) { - _runtime_state->set_import_label(request.import_label); - } - if (request.__isset.db_name) { - _runtime_state->set_db_name(request.db_name); - } - if (request.__isset.load_job_id) { - _runtime_state->set_load_job_id(request.load_job_id); - } - - if (request.is_simplified_param) { - _desc_tbl = _query_ctx->desc_tbl; - } else { - DCHECK(request.__isset.desc_tbl); - RETURN_IF_ERROR( - DescriptorTbl::create(_runtime_state->obj_pool(), request.desc_tbl, &_desc_tbl)); - } - _runtime_state->set_desc_tbl(_desc_tbl); - _runtime_state->set_num_per_fragment_instances(request.num_senders); - _runtime_state->set_load_stream_per_node(request.load_stream_per_node); - _runtime_state->set_total_load_streams(request.total_load_streams); - _runtime_state->set_num_local_sink(request.num_local_sink); - - const auto& local_params = request.local_params[0]; - if (local_params.__isset.runtime_filter_params) { - _query_ctx->runtime_filter_mgr()->set_runtime_filter_params( - local_params.runtime_filter_params); - } - if (local_params.__isset.topn_filter_source_node_ids) { - _query_ctx->init_runtime_predicates(local_params.topn_filter_source_node_ids); - } else { - _query_ctx->init_runtime_predicates({0}); - } - - _need_local_merge = request.__isset.parallel_instances; - - // 2. Build pipelines with operators in this fragment. - auto root_pipeline = add_pipeline(); - RETURN_IF_ERROR_OR_CATCH_EXCEPTION(_build_pipelines( - _runtime_state->obj_pool(), request, *_query_ctx->desc_tbl, &_root_op, root_pipeline)); - - // 3. Create sink operator - if (!request.fragment.__isset.output_sink) { - return Status::InternalError("No output sink in this fragment!"); - } - RETURN_IF_ERROR_OR_CATCH_EXCEPTION(_create_data_sink( - _runtime_state->obj_pool(), request.fragment.output_sink, request.fragment.output_exprs, - request, root_pipeline->output_row_desc(), _runtime_state.get(), *_desc_tbl, - root_pipeline->id())); - RETURN_IF_ERROR(_sink->init(request.fragment.output_sink)); - RETURN_IF_ERROR(root_pipeline->set_sink(_sink)); - - for (PipelinePtr& pipeline : _pipelines) { - DCHECK(pipeline->sink_x() != nullptr) << pipeline->operator_xs().size(); - RETURN_IF_ERROR(pipeline->sink_x()->set_child(pipeline->operator_xs().back())); - } - if (_enable_local_shuffle()) { - RETURN_IF_ERROR(_plan_local_exchange(request.num_buckets, - request.bucket_seq_to_instance_idx, - request.shuffle_idx_to_instance_idx)); - } - - // 4. Initialize global states in pipelines. - for (PipelinePtr& pipeline : _pipelines) { - pipeline->children().clear(); - RETURN_IF_ERROR(pipeline->prepare(_runtime_state.get())); - } - - // 5. Build pipeline tasks and initialize local state. - RETURN_IF_ERROR(_build_pipeline_tasks(request)); - - _init_next_report_time(); - - _prepared = true; - return Status::OK(); -} - -Status PipelineXFragmentContext::_plan_local_exchange( - int num_buckets, const std::map& bucket_seq_to_instance_idx, - const std::map& shuffle_idx_to_instance_idx) { - for (int pip_idx = _pipelines.size() - 1; pip_idx >= 0; pip_idx--) { - _pipelines[pip_idx]->init_data_distribution(); - // Set property if child pipeline is not join operator's child. - if (!_pipelines[pip_idx]->children().empty()) { - for (auto& child : _pipelines[pip_idx]->children()) { - if (child->sink_x()->node_id() == - _pipelines[pip_idx]->operator_xs().front()->node_id()) { - RETURN_IF_ERROR(_pipelines[pip_idx]->operator_xs().front()->set_child( - child->operator_xs().back())); - _pipelines[pip_idx]->set_data_distribution(child->data_distribution()); - } - } - } - - RETURN_IF_ERROR(_plan_local_exchange( - _pipelines[pip_idx]->operator_xs().front()->ignore_data_hash_distribution() - ? _num_instances - : num_buckets, - pip_idx, _pipelines[pip_idx], bucket_seq_to_instance_idx, - shuffle_idx_to_instance_idx, - _pipelines[pip_idx]->operator_xs().front()->ignore_data_hash_distribution())); - } - return Status::OK(); -} - -Status PipelineXFragmentContext::_plan_local_exchange( - int num_buckets, int pip_idx, PipelinePtr pip, - const std::map& bucket_seq_to_instance_idx, - const std::map& shuffle_idx_to_instance_idx, - const bool ignore_data_hash_distribution) { - int idx = 1; - bool do_local_exchange = false; - do { - auto& ops = pip->operator_xs(); - do_local_exchange = false; - // Plan local exchange for each operator. - for (; idx < ops.size();) { - if (ops[idx]->required_data_distribution().need_local_exchange()) { - RETURN_IF_ERROR(_add_local_exchange( - pip_idx, idx, ops[idx]->node_id(), _runtime_state->obj_pool(), pip, - ops[idx]->required_data_distribution(), &do_local_exchange, num_buckets, - bucket_seq_to_instance_idx, shuffle_idx_to_instance_idx, - ignore_data_hash_distribution)); - } - if (do_local_exchange) { - // If local exchange is needed for current operator, we will split this pipeline to - // two pipelines by local exchange sink/source. And then we need to process remaining - // operators in this pipeline so we set idx to 2 (0 is local exchange source and 1 - // is current operator was already processed) and continue to plan local exchange. - idx = 2; - break; - } - idx++; - } - } while (do_local_exchange); - if (pip->sink_x()->required_data_distribution().need_local_exchange()) { - RETURN_IF_ERROR(_add_local_exchange( - pip_idx, idx, pip->sink_x()->node_id(), _runtime_state->obj_pool(), pip, - pip->sink_x()->required_data_distribution(), &do_local_exchange, num_buckets, - bucket_seq_to_instance_idx, shuffle_idx_to_instance_idx, - ignore_data_hash_distribution)); - } - return Status::OK(); -} - -Status PipelineXFragmentContext::_create_data_sink(ObjectPool* pool, const TDataSink& thrift_sink, - const std::vector& output_exprs, - const TPipelineFragmentParams& params, - const RowDescriptor& row_desc, - RuntimeState* state, DescriptorTbl& desc_tbl, - PipelineId cur_pipeline_id) { - switch (thrift_sink.type) { - case TDataSinkType::DATA_STREAM_SINK: { - if (!thrift_sink.__isset.stream_sink) { - return Status::InternalError("Missing data stream sink."); - } - _sink.reset(new ExchangeSinkOperatorX(state, row_desc, next_sink_operator_id(), - thrift_sink.stream_sink, params.destinations)); - break; - } - case TDataSinkType::RESULT_SINK: { - if (!thrift_sink.__isset.result_sink) { - return Status::InternalError("Missing data buffer sink."); - } - - // TODO: figure out good buffer size based on size of output row - _sink.reset(new ResultSinkOperatorX(next_sink_operator_id(), row_desc, output_exprs, - thrift_sink.result_sink)); - break; - } - case TDataSinkType::GROUP_COMMIT_OLAP_TABLE_SINK: - case TDataSinkType::OLAP_TABLE_SINK: { - if (state->query_options().enable_memtable_on_sink_node && - !_has_inverted_index_or_partial_update(thrift_sink.olap_table_sink) && - !config::is_cloud_mode()) { - _sink.reset(new OlapTableSinkV2OperatorX(pool, next_sink_operator_id(), row_desc, - output_exprs)); - } else { - _sink.reset(new OlapTableSinkOperatorX(pool, next_sink_operator_id(), row_desc, - output_exprs)); - } - break; - } - case TDataSinkType::GROUP_COMMIT_BLOCK_SINK: { - DCHECK(thrift_sink.__isset.olap_table_sink); - _sink.reset(new GroupCommitBlockSinkOperatorX(next_sink_operator_id(), row_desc)); - break; - } - case TDataSinkType::HIVE_TABLE_SINK: { - if (!thrift_sink.__isset.hive_table_sink) { - return Status::InternalError("Missing hive table sink."); - } - _sink.reset( - new HiveTableSinkOperatorX(pool, next_sink_operator_id(), row_desc, output_exprs)); - break; - } - case TDataSinkType::JDBC_TABLE_SINK: { - if (!thrift_sink.__isset.jdbc_table_sink) { - return Status::InternalError("Missing data jdbc sink."); - } - if (config::enable_java_support) { - _sink.reset( - new JdbcTableSinkOperatorX(row_desc, next_sink_operator_id(), output_exprs)); - } else { - return Status::InternalError( - "Jdbc table sink is not enabled, you can change be config " - "enable_java_support to true and restart be."); - } - break; - } - case TDataSinkType::RESULT_FILE_SINK: { - if (!thrift_sink.__isset.result_file_sink) { - return Status::InternalError("Missing result file sink."); - } - - // TODO: figure out good buffer size based on size of output row - // Result file sink is not the top sink - if (params.__isset.destinations && !params.destinations.empty()) { - _sink.reset(new ResultFileSinkOperatorX(next_sink_operator_id(), row_desc, - thrift_sink.result_file_sink, - params.destinations, output_exprs, desc_tbl)); - } else { - _sink.reset( - new ResultFileSinkOperatorX(next_sink_operator_id(), row_desc, output_exprs)); - } - break; - } - case TDataSinkType::MULTI_CAST_DATA_STREAM_SINK: { - DCHECK(thrift_sink.__isset.multi_cast_stream_sink); - DCHECK_GT(thrift_sink.multi_cast_stream_sink.sinks.size(), 0); - // TODO: figure out good buffer size based on size of output row - auto sink_id = next_sink_operator_id(); - auto sender_size = thrift_sink.multi_cast_stream_sink.sinks.size(); - // one sink has multiple sources. - std::vector sources; - for (int i = 0; i < sender_size; ++i) { - auto source_id = next_operator_id(); - sources.push_back(source_id); - } - - _sink.reset(new MultiCastDataStreamSinkOperatorX( - sink_id, sources, thrift_sink.multi_cast_stream_sink.sinks.size(), pool, - thrift_sink.multi_cast_stream_sink, row_desc)); - for (int i = 0; i < sender_size; ++i) { - auto new_pipeline = add_pipeline(); - RowDescriptor* _row_desc = nullptr; - { - const auto& tmp_row_desc = - !thrift_sink.multi_cast_stream_sink.sinks[i].output_exprs.empty() - ? RowDescriptor(state->desc_tbl(), - {thrift_sink.multi_cast_stream_sink.sinks[i] - .output_tuple_id}, - {false}) - : _sink->row_desc(); - _row_desc = pool->add(new RowDescriptor(tmp_row_desc)); - } - auto source_id = sources[i]; - OperatorXPtr source_op; - // 1. create and set the source operator of multi_cast_data_stream_source for new pipeline - source_op.reset(new MultiCastDataStreamerSourceOperatorX( - i, pool, thrift_sink.multi_cast_stream_sink.sinks[i], row_desc, source_id)); - RETURN_IF_ERROR(new_pipeline->add_operator(source_op)); - // 2. create and set sink operator of data stream sender for new pipeline - - DataSinkOperatorXPtr sink_op; - sink_op.reset( - new ExchangeSinkOperatorX(state, *_row_desc, next_sink_operator_id(), - thrift_sink.multi_cast_stream_sink.sinks[i], - thrift_sink.multi_cast_stream_sink.destinations[i])); - - RETURN_IF_ERROR(new_pipeline->set_sink(sink_op)); - { - TDataSink* t = pool->add(new TDataSink()); - t->stream_sink = thrift_sink.multi_cast_stream_sink.sinks[i]; - RETURN_IF_ERROR(sink_op->init(*t)); - } - - // 3. set dependency dag - _dag[new_pipeline->id()].push_back(cur_pipeline_id); - } - if (sources.empty()) { - return Status::InternalError("size of sources must be greater than 0"); - } - break; - } - default: - return Status::InternalError("Unsuported sink type in pipeline: {}", thrift_sink.type); - } - return Status::OK(); -} - -Status PipelineXFragmentContext::_build_pipeline_tasks( - const doris::TPipelineFragmentParams& request) { - _total_tasks = 0; - int target_size = request.local_params.size(); - _tasks.resize(target_size); - auto& pipeline_id_to_profile = _runtime_state->pipeline_id_to_profile(); - DCHECK(pipeline_id_to_profile.empty()); - pipeline_id_to_profile.resize(_pipelines.size()); - { - size_t pip_idx = 0; - for (auto& pipeline_profile : pipeline_id_to_profile) { - pipeline_profile = - std::make_unique("Pipeline : " + std::to_string(pip_idx)); - pip_idx++; - } - } - - for (size_t i = 0; i < target_size; i++) { - const auto& local_params = request.local_params[i]; - auto fragment_instance_id = local_params.fragment_instance_id; - _fragment_instance_ids.push_back(fragment_instance_id); - std::unique_ptr runtime_filter_mgr; - auto init_runtime_state = [&](std::unique_ptr& runtime_state) { - runtime_state->set_query_mem_tracker(_query_ctx->query_mem_tracker); - - runtime_state->set_task_execution_context(shared_from_this()); - runtime_state->set_be_number(local_params.backend_num); - - if (request.__isset.backend_id) { - runtime_state->set_backend_id(request.backend_id); - } - if (request.__isset.import_label) { - runtime_state->set_import_label(request.import_label); - } - if (request.__isset.db_name) { - runtime_state->set_db_name(request.db_name); - } - if (request.__isset.load_job_id) { - runtime_state->set_load_job_id(request.load_job_id); - } - - runtime_state->set_desc_tbl(_desc_tbl); - runtime_state->set_per_fragment_instance_idx(local_params.sender_id); - runtime_state->set_num_per_fragment_instances(request.num_senders); - runtime_state->resize_op_id_to_local_state(max_operator_id()); - runtime_state->set_max_operator_id(max_operator_id()); - runtime_state->set_load_stream_per_node(request.load_stream_per_node); - runtime_state->set_total_load_streams(request.total_load_streams); - runtime_state->set_num_local_sink(request.num_local_sink); - DCHECK(runtime_filter_mgr); - runtime_state->set_pipeline_x_runtime_filter_mgr(runtime_filter_mgr.get()); - }; - - auto filterparams = std::make_unique(); - - { - filterparams->runtime_filter_wait_infinitely = - _runtime_state->runtime_filter_wait_infinitely(); - filterparams->runtime_filter_wait_time_ms = - _runtime_state->runtime_filter_wait_time_ms(); - filterparams->enable_pipeline_exec = _runtime_state->enable_pipeline_x_exec(); - filterparams->execution_timeout = _runtime_state->execution_timeout(); - - filterparams->exec_env = ExecEnv::GetInstance(); - filterparams->query_id.set_hi(_runtime_state->query_id().hi); - filterparams->query_id.set_lo(_runtime_state->query_id().lo); - - filterparams->be_exec_version = _runtime_state->be_exec_version(); - filterparams->query_ctx = _query_ctx.get(); - } - - // build local_runtime_filter_mgr for each instance - runtime_filter_mgr = std::make_unique( - request.query_id, filterparams.get(), _query_ctx->query_mem_tracker); - - filterparams->runtime_filter_mgr = runtime_filter_mgr.get(); - - _runtime_filter_states.push_back(std::move(filterparams)); - std::map pipeline_id_to_task; - auto get_local_exchange_state = [&](PipelinePtr pipeline) - -> std::map, - std::shared_ptr>> { - std::map, - std::shared_ptr>> - le_state_map; - auto source_id = pipeline->operator_xs().front()->operator_id(); - if (auto iter = _op_id_to_le_state.find(source_id); iter != _op_id_to_le_state.end()) { - le_state_map.insert({source_id, iter->second}); - } - for (auto sink_to_source_id : pipeline->sink_x()->dests_id()) { - if (auto iter = _op_id_to_le_state.find(sink_to_source_id); - iter != _op_id_to_le_state.end()) { - le_state_map.insert({sink_to_source_id, iter->second}); - } - } - return le_state_map; - }; - auto get_task_runtime_state = [&](int task_id) -> RuntimeState* { - DCHECK(_task_runtime_states[task_id]); - return _task_runtime_states[task_id].get(); - }; - for (size_t pip_idx = 0; pip_idx < _pipelines.size(); pip_idx++) { - auto& pipeline = _pipelines[pip_idx]; - if (pipeline->need_to_create_task()) { - // build task runtime state - _task_runtime_states.push_back(RuntimeState::create_unique( - this, local_params.fragment_instance_id, request.query_id, - request.fragment_id, request.query_options, _query_ctx->query_globals, - _exec_env, _query_ctx.get())); - auto& task_runtime_state = _task_runtime_states.back(); - init_runtime_state(task_runtime_state); - auto cur_task_id = _total_tasks++; - task_runtime_state->set_task_id(cur_task_id); - task_runtime_state->set_task_num(pipeline->num_tasks()); - auto task = std::make_unique( - pipeline, cur_task_id, get_task_runtime_state(cur_task_id), this, - pipeline_id_to_profile[pip_idx].get(), get_local_exchange_state(pipeline), - i); - pipeline_id_to_task.insert({pipeline->id(), task.get()}); - _tasks[i].emplace_back(std::move(task)); - } - } - - /** - * Build DAG for pipeline tasks. - * For example, we have - * - * ExchangeSink (Pipeline1) JoinBuildSink (Pipeline2) - * \ / - * JoinProbeOperator1 (Pipeline1) JoinBuildSink (Pipeline3) - * \ / - * JoinProbeOperator2 (Pipeline1) - * - * In this fragment, we have three pipelines and pipeline 1 depends on pipeline 2 and pipeline 3. - * To build this DAG, `_dag` manage dependencies between pipelines by pipeline ID and - * `pipeline_id_to_task` is used to find the task by a unique pipeline ID. - * - * Finally, we have two upstream dependencies in Pipeline1 corresponding to JoinProbeOperator1 - * and JoinProbeOperator2. - */ - - // First, set up the parent profile,task runtime state - - auto prepare_and_set_parent_profile = [&](PipelineXTask* task, size_t pip_idx) { - DCHECK(pipeline_id_to_profile[pip_idx]); - RETURN_IF_ERROR( - task->prepare(local_params, request.fragment.output_sink, _query_ctx.get())); - return Status::OK(); - }; - - for (auto& _pipeline : _pipelines) { - if (pipeline_id_to_task.contains(_pipeline->id())) { - auto* task = pipeline_id_to_task[_pipeline->id()]; - DCHECK(task != nullptr); - - // if this task has upstream dependency, then record them. - if (_dag.find(_pipeline->id()) != _dag.end()) { - auto& deps = _dag[_pipeline->id()]; - for (auto& dep : deps) { - if (pipeline_id_to_task.contains(dep)) { - auto ss = pipeline_id_to_task[dep]->get_sink_shared_state(); - if (ss) { - task->inject_shared_state(ss); - } else { - pipeline_id_to_task[dep]->inject_shared_state( - task->get_source_shared_state()); - } - } - } - } - } - } - for (size_t pip_idx = 0; pip_idx < _pipelines.size(); pip_idx++) { - if (pipeline_id_to_task.contains(_pipelines[pip_idx]->id())) { - auto* task = pipeline_id_to_task[_pipelines[pip_idx]->id()]; - RETURN_IF_ERROR(prepare_and_set_parent_profile(task, pip_idx)); - } - } - { - std::lock_guard l(_state_map_lock); - _runtime_filter_mgr_map[fragment_instance_id] = std::move(runtime_filter_mgr); - } - } - _pipeline_parent_map.clear(); - _dag.clear(); - _op_id_to_le_state.clear(); - - return Status::OK(); -} - -Status PipelineXFragmentContext::_build_pipelines(ObjectPool* pool, - const doris::TPipelineFragmentParams& request, - const DescriptorTbl& descs, OperatorXPtr* root, - PipelinePtr cur_pipe) { - if (request.fragment.plan.nodes.empty()) { - throw Exception(ErrorCode::INTERNAL_ERROR, "Invalid plan which has no plan node!"); - } - - int node_idx = 0; - - cur_pipe->_name.append(std::to_string(cur_pipe->id())); - - RETURN_IF_ERROR(_create_tree_helper(pool, request.fragment.plan.nodes, request, descs, nullptr, - &node_idx, root, cur_pipe, 0)); - - if (node_idx + 1 != request.fragment.plan.nodes.size()) { - // TODO: print thrift msg for diagnostic purposes. - return Status::InternalError( - "Plan tree only partially reconstructed. Not all thrift nodes were used."); - } - - return Status::OK(); -} - -Status PipelineXFragmentContext::_create_tree_helper(ObjectPool* pool, - const std::vector& tnodes, - const doris::TPipelineFragmentParams& request, - const DescriptorTbl& descs, - OperatorXPtr parent, int* node_idx, - OperatorXPtr* root, PipelinePtr& cur_pipe, - int child_idx) { - // propagate error case - if (*node_idx >= tnodes.size()) { - // TODO: print thrift msg - return Status::InternalError( - "Failed to reconstruct plan tree from thrift. Node id: {}, number of nodes: {}", - *node_idx, tnodes.size()); - } - const TPlanNode& tnode = tnodes[*node_idx]; - - int num_children = tnodes[*node_idx].num_children; - OperatorXPtr op = nullptr; - RETURN_IF_ERROR(_create_operator(pool, tnodes[*node_idx], request, descs, op, cur_pipe, - parent == nullptr ? -1 : parent->node_id(), child_idx)); - - // assert(parent != nullptr || (node_idx == 0 && root_expr != nullptr)); - if (parent != nullptr) { - // add to parent's child(s) - RETURN_IF_ERROR(parent->set_child(op)); - } else { - *root = op; - } - - cur_pipe->_name.push_back('-'); - cur_pipe->_name.append(std::to_string(op->id())); - cur_pipe->_name.append(op->get_name()); - - // rely on that tnodes is preorder of the plan - for (int i = 0; i < num_children; i++) { - ++*node_idx; - RETURN_IF_ERROR(_create_tree_helper(pool, tnodes, request, descs, op, node_idx, nullptr, - cur_pipe, i)); - - // we are expecting a child, but have used all nodes - // this means we have been given a bad tree and must fail - if (*node_idx >= tnodes.size()) { - // TODO: print thrift msg - return Status::InternalError( - "Failed to reconstruct plan tree from thrift. Node id: {}, number of nodes: {}", - *node_idx, tnodes.size()); - } - } - - RETURN_IF_ERROR(op->init(tnode, _runtime_state.get())); - - return Status::OK(); -} - -void PipelineXFragmentContext::_inherit_pipeline_properties( - const DataDistribution& data_distribution, PipelinePtr pipe_with_source, - PipelinePtr pipe_with_sink) { - pipe_with_sink->set_num_tasks(pipe_with_source->num_tasks()); - pipe_with_source->set_num_tasks(_num_instances); - pipe_with_source->set_data_distribution(data_distribution); -} - -Status PipelineXFragmentContext::_add_local_exchange_impl( - int idx, ObjectPool* pool, PipelinePtr cur_pipe, PipelinePtr new_pip, - DataDistribution data_distribution, bool* do_local_exchange, int num_buckets, - const std::map& bucket_seq_to_instance_idx, - const std::map& shuffle_idx_to_instance_idx, - const bool ignore_data_hash_distribution) { - auto& operator_xs = cur_pipe->operator_xs(); - const auto downstream_pipeline_id = cur_pipe->id(); - auto local_exchange_id = next_operator_id(); - // 1. Create a new pipeline with local exchange sink. - DataSinkOperatorXPtr sink; - auto sink_id = next_sink_operator_id(); - const bool is_shuffled_hash_join = operator_xs.size() > idx - ? operator_xs[idx]->is_shuffled_hash_join() - : cur_pipe->sink_x()->is_shuffled_hash_join(); - sink.reset(new LocalExchangeSinkOperatorX( - sink_id, local_exchange_id, is_shuffled_hash_join ? _total_instances : _num_instances, - data_distribution.partition_exprs, bucket_seq_to_instance_idx)); - RETURN_IF_ERROR(new_pip->set_sink(sink)); - RETURN_IF_ERROR(new_pip->sink_x()->init(data_distribution.distribution_type, num_buckets, - is_shuffled_hash_join, shuffle_idx_to_instance_idx)); - - // 2. Create and initialize LocalExchangeSharedState. - auto shared_state = LocalExchangeSharedState::create_shared(_num_instances); - switch (data_distribution.distribution_type) { - case ExchangeType::HASH_SHUFFLE: - shared_state->exchanger = ShuffleExchanger::create_unique( - std::max(cur_pipe->num_tasks(), _num_instances), - is_shuffled_hash_join ? _total_instances : _num_instances); - break; - case ExchangeType::BUCKET_HASH_SHUFFLE: - shared_state->exchanger = BucketShuffleExchanger::create_unique( - std::max(cur_pipe->num_tasks(), _num_instances), _num_instances, num_buckets, - ignore_data_hash_distribution); - break; - case ExchangeType::PASSTHROUGH: - shared_state->exchanger = - PassthroughExchanger::create_unique(cur_pipe->num_tasks(), _num_instances); - break; - case ExchangeType::BROADCAST: - shared_state->exchanger = - BroadcastExchanger::create_unique(cur_pipe->num_tasks(), _num_instances); - break; - case ExchangeType::PASS_TO_ONE: - shared_state->exchanger = - BroadcastExchanger::create_unique(cur_pipe->num_tasks(), _num_instances); - break; - case ExchangeType::ADAPTIVE_PASSTHROUGH: - shared_state->exchanger = - AdaptivePassthroughExchanger::create_unique(cur_pipe->num_tasks(), _num_instances); - break; - default: - return Status::InternalError("Unsupported local exchange type : " + - std::to_string((int)data_distribution.distribution_type)); - } - auto sink_dep = std::make_shared(sink_id, local_exchange_id, - "LOCAL_EXCHANGE_SINK_DEPENDENCY", true, - _runtime_state->get_query_ctx()); - sink_dep->set_shared_state(shared_state.get()); - shared_state->sink_deps.push_back(sink_dep); - _op_id_to_le_state.insert({local_exchange_id, {shared_state, sink_dep}}); - - // 3. Set two pipelines' operator list. For example, split pipeline [Scan - AggSink] to - // pipeline1 [Scan - LocalExchangeSink] and pipeline2 [LocalExchangeSource - AggSink]. - - // 3.1 Initialize new pipeline's operator list. - std::copy(operator_xs.begin(), operator_xs.begin() + idx, - std::inserter(new_pip->operator_xs(), new_pip->operator_xs().end())); - - // 3.2 Erase unused operators in previous pipeline. - operator_xs.erase(operator_xs.begin(), operator_xs.begin() + idx); - - // 4. Initialize LocalExchangeSource and insert it into this pipeline. - OperatorXPtr source_op; - source_op.reset(new LocalExchangeSourceOperatorX(pool, local_exchange_id)); - RETURN_IF_ERROR(source_op->set_child(new_pip->operator_xs().back())); - RETURN_IF_ERROR(source_op->init(data_distribution.distribution_type)); - if (!operator_xs.empty()) { - RETURN_IF_ERROR(operator_xs.front()->set_child(source_op)); - } - operator_xs.insert(operator_xs.begin(), source_op); - - shared_state->create_source_dependencies(source_op->operator_id(), source_op->node_id(), - _query_ctx.get()); - - // 5. Set children for two pipelines separately. - std::vector> new_children; - std::vector edges_with_source; - for (auto child : cur_pipe->children()) { - bool found = false; - for (auto op : new_pip->operator_xs()) { - if (child->sink_x()->node_id() == op->node_id()) { - new_pip->set_children(child); - found = true; - }; - } - if (!found) { - new_children.push_back(child); - edges_with_source.push_back(child->id()); - } - } - new_children.push_back(new_pip); - edges_with_source.push_back(new_pip->id()); - - // 6. Set DAG for new pipelines. - if (!new_pip->children().empty()) { - std::vector edges_with_sink; - for (auto child : new_pip->children()) { - edges_with_sink.push_back(child->id()); - } - _dag.insert({new_pip->id(), edges_with_sink}); - } - cur_pipe->set_children(new_children); - _dag[downstream_pipeline_id] = edges_with_source; - RETURN_IF_ERROR(new_pip->sink_x()->set_child(new_pip->operator_xs().back())); - RETURN_IF_ERROR(cur_pipe->sink_x()->set_child(cur_pipe->operator_xs().back())); - - // 7. Inherit properties from current pipeline. - _inherit_pipeline_properties(data_distribution, cur_pipe, new_pip); - return Status::OK(); -} - -Status PipelineXFragmentContext::_add_local_exchange( - int pip_idx, int idx, int node_id, ObjectPool* pool, PipelinePtr cur_pipe, - DataDistribution data_distribution, bool* do_local_exchange, int num_buckets, - const std::map& bucket_seq_to_instance_idx, - const std::map& shuffle_idx_to_instance_idx, - const bool ignore_data_distribution) { - DCHECK(_enable_local_shuffle()); - if (_num_instances <= 1) { - return Status::OK(); - } - - if (!cur_pipe->need_to_local_exchange(data_distribution)) { - return Status::OK(); - } - *do_local_exchange = true; - - auto& operator_xs = cur_pipe->operator_xs(); - auto total_op_num = operator_xs.size(); - auto new_pip = add_pipeline(cur_pipe, pip_idx + 1); - RETURN_IF_ERROR(_add_local_exchange_impl( - idx, pool, cur_pipe, new_pip, data_distribution, do_local_exchange, num_buckets, - bucket_seq_to_instance_idx, shuffle_idx_to_instance_idx, ignore_data_distribution)); - - CHECK(total_op_num + 1 == cur_pipe->operator_xs().size() + new_pip->operator_xs().size()) - << "total_op_num: " << total_op_num - << " cur_pipe->operator_xs().size(): " << cur_pipe->operator_xs().size() - << " new_pip->operator_xs().size(): " << new_pip->operator_xs().size(); - - // Add passthrough local exchanger if necessary - if (cur_pipe->num_tasks() > 1 && new_pip->num_tasks() == 1 && - Pipeline::is_hash_exchange(data_distribution.distribution_type)) { - RETURN_IF_ERROR(_add_local_exchange_impl( - new_pip->operator_xs().size(), pool, new_pip, add_pipeline(new_pip, pip_idx + 2), - DataDistribution(ExchangeType::PASSTHROUGH), do_local_exchange, num_buckets, - bucket_seq_to_instance_idx, shuffle_idx_to_instance_idx, ignore_data_distribution)); - } - return Status::OK(); -} - -// NOLINTBEGIN(readability-function-size) -// NOLINTBEGIN(readability-function-cognitive-complexity) -Status PipelineXFragmentContext::_create_operator(ObjectPool* pool, const TPlanNode& tnode, - const doris::TPipelineFragmentParams& request, - const DescriptorTbl& descs, OperatorXPtr& op, - PipelinePtr& cur_pipe, int parent_idx, - int child_idx) { - // We directly construct the operator from Thrift because the given array is in the order of preorder traversal. - // Therefore, here we need to use a stack-like structure. - _pipeline_parent_map.pop(cur_pipe, parent_idx, child_idx); - std::stringstream error_msg; - - switch (tnode.node_type) { - case TPlanNodeType::OLAP_SCAN_NODE: { - op.reset(new OlapScanOperatorX(pool, tnode, next_operator_id(), descs, _num_instances)); - RETURN_IF_ERROR(cur_pipe->add_operator(op)); - if (request.__isset.parallel_instances) { - cur_pipe->set_num_tasks(request.parallel_instances); - op->set_ignore_data_distribution(); - } - break; - } - case doris::TPlanNodeType::JDBC_SCAN_NODE: { - if (config::enable_java_support) { - op.reset(new JDBCScanOperatorX(pool, tnode, next_operator_id(), descs, _num_instances)); - RETURN_IF_ERROR(cur_pipe->add_operator(op)); - } else { - return Status::InternalError( - "Jdbc scan node is disabled, you can change be config enable_java_support " - "to true and restart be."); - } - if (request.__isset.parallel_instances) { - cur_pipe->set_num_tasks(request.parallel_instances); - op->set_ignore_data_distribution(); - } - break; - } - case doris::TPlanNodeType::FILE_SCAN_NODE: { - op.reset(new FileScanOperatorX(pool, tnode, next_operator_id(), descs, _num_instances)); - RETURN_IF_ERROR(cur_pipe->add_operator(op)); - if (request.__isset.parallel_instances) { - cur_pipe->set_num_tasks(request.parallel_instances); - op->set_ignore_data_distribution(); - } - break; - } - case TPlanNodeType::ES_SCAN_NODE: - case TPlanNodeType::ES_HTTP_SCAN_NODE: { - op.reset(new EsScanOperatorX(pool, tnode, next_operator_id(), descs, _num_instances)); - RETURN_IF_ERROR(cur_pipe->add_operator(op)); - if (request.__isset.parallel_instances) { - cur_pipe->set_num_tasks(request.parallel_instances); - op->set_ignore_data_distribution(); - } - break; - } - case TPlanNodeType::EXCHANGE_NODE: { - int num_senders = find_with_default(request.per_exch_num_senders, tnode.node_id, 0); - DCHECK_GT(num_senders, 0); - op.reset(new ExchangeSourceOperatorX(pool, tnode, next_operator_id(), descs, num_senders)); - RETURN_IF_ERROR(cur_pipe->add_operator(op)); - if (request.__isset.parallel_instances) { - op->set_ignore_data_distribution(); - cur_pipe->set_num_tasks(request.parallel_instances); - } - break; - } - case TPlanNodeType::AGGREGATION_NODE: { - if (tnode.agg_node.grouping_exprs.empty() && - descs.get_tuple_descriptor(tnode.agg_node.output_tuple_id)->slots().empty()) { - return Status::InternalError("Illegal aggregate node " + std::to_string(tnode.node_id) + - ": group by and output is empty"); - } - if (tnode.agg_node.aggregate_functions.empty() && !_runtime_state->enable_agg_spill() && - request.query_options.__isset.enable_distinct_streaming_aggregation && - request.query_options.enable_distinct_streaming_aggregation && - !tnode.agg_node.grouping_exprs.empty()) { - op.reset(new DistinctStreamingAggOperatorX(pool, next_operator_id(), tnode, descs)); - RETURN_IF_ERROR(cur_pipe->add_operator(op)); - } else if (tnode.agg_node.__isset.use_streaming_preaggregation && - tnode.agg_node.use_streaming_preaggregation && - !tnode.agg_node.grouping_exprs.empty()) { - op.reset(new StreamingAggOperatorX(pool, next_operator_id(), tnode, descs)); - RETURN_IF_ERROR(cur_pipe->add_operator(op)); - } else { - if (_runtime_state->enable_agg_spill() && !tnode.agg_node.grouping_exprs.empty()) { - op.reset(new PartitionedAggSourceOperatorX(pool, tnode, next_operator_id(), descs)); - } else { - op.reset(new AggSourceOperatorX(pool, tnode, next_operator_id(), descs)); - } - RETURN_IF_ERROR(cur_pipe->add_operator(op)); - - const auto downstream_pipeline_id = cur_pipe->id(); - if (_dag.find(downstream_pipeline_id) == _dag.end()) { - _dag.insert({downstream_pipeline_id, {}}); - } - cur_pipe = add_pipeline(cur_pipe); - _dag[downstream_pipeline_id].push_back(cur_pipe->id()); - - DataSinkOperatorXPtr sink; - if (_runtime_state->enable_agg_spill() && !tnode.agg_node.grouping_exprs.empty()) { - sink.reset(new PartitionedAggSinkOperatorX(pool, next_sink_operator_id(), tnode, - descs)); - } else { - sink.reset(new AggSinkOperatorX(pool, next_sink_operator_id(), tnode, descs)); - } - sink->set_dests_id({op->operator_id()}); - RETURN_IF_ERROR(cur_pipe->set_sink(sink)); - RETURN_IF_ERROR(cur_pipe->sink_x()->init(tnode, _runtime_state.get())); - } - break; - } - case TPlanNodeType::HASH_JOIN_NODE: { - const auto is_broadcast_join = tnode.hash_join_node.__isset.is_broadcast_join && - tnode.hash_join_node.is_broadcast_join; - const auto enable_join_spill = _runtime_state->enable_join_spill(); - if (enable_join_spill && !is_broadcast_join) { - auto tnode_ = tnode; - /// TODO: support rf in partitioned hash join - tnode_.runtime_filters.clear(); - const uint32_t partition_count = 32; - auto inner_probe_operator = - std::make_shared(pool, tnode_, 0, descs); - auto inner_sink_operator = std::make_shared( - pool, 0, tnode_, descs, _need_local_merge); - - RETURN_IF_ERROR(inner_probe_operator->init(tnode_, _runtime_state.get())); - RETURN_IF_ERROR(inner_sink_operator->init(tnode_, _runtime_state.get())); - - auto probe_operator = std::make_shared( - pool, tnode_, next_operator_id(), descs, partition_count); - probe_operator->set_inner_operators(inner_sink_operator, inner_probe_operator); - op = std::move(probe_operator); - RETURN_IF_ERROR(cur_pipe->add_operator(op)); - - const auto downstream_pipeline_id = cur_pipe->id(); - if (_dag.find(downstream_pipeline_id) == _dag.end()) { - _dag.insert({downstream_pipeline_id, {}}); - } - PipelinePtr build_side_pipe = add_pipeline(cur_pipe); - _dag[downstream_pipeline_id].push_back(build_side_pipe->id()); - - auto sink_operator = std::make_shared( - pool, next_sink_operator_id(), tnode_, descs, _need_local_merge, - partition_count); - sink_operator->set_inner_operators(inner_sink_operator, inner_probe_operator); - DataSinkOperatorXPtr sink = std::move(sink_operator); - 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())); - - _pipeline_parent_map.push(op->node_id(), cur_pipe); - _pipeline_parent_map.push(op->node_id(), build_side_pipe); - } else { - op.reset(new HashJoinProbeOperatorX(pool, tnode, next_operator_id(), descs)); - RETURN_IF_ERROR(cur_pipe->add_operator(op)); - - const auto downstream_pipeline_id = cur_pipe->id(); - if (_dag.find(downstream_pipeline_id) == _dag.end()) { - _dag.insert({downstream_pipeline_id, {}}); - } - PipelinePtr build_side_pipe = add_pipeline(cur_pipe); - _dag[downstream_pipeline_id].push_back(build_side_pipe->id()); - - DataSinkOperatorXPtr sink; - sink.reset(new HashJoinBuildSinkOperatorX(pool, next_sink_operator_id(), tnode, descs, - _need_local_merge)); - 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())); - - _pipeline_parent_map.push(op->node_id(), cur_pipe); - _pipeline_parent_map.push(op->node_id(), build_side_pipe); - } - break; - } - case TPlanNodeType::CROSS_JOIN_NODE: { - op.reset(new NestedLoopJoinProbeOperatorX(pool, tnode, next_operator_id(), descs)); - RETURN_IF_ERROR(cur_pipe->add_operator(op)); - - const auto downstream_pipeline_id = cur_pipe->id(); - if (_dag.find(downstream_pipeline_id) == _dag.end()) { - _dag.insert({downstream_pipeline_id, {}}); - } - PipelinePtr build_side_pipe = add_pipeline(cur_pipe); - _dag[downstream_pipeline_id].push_back(build_side_pipe->id()); - - DataSinkOperatorXPtr sink; - sink.reset(new NestedLoopJoinBuildSinkOperatorX(pool, next_sink_operator_id(), tnode, descs, - _need_local_merge)); - 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())); - _pipeline_parent_map.push(op->node_id(), cur_pipe); - _pipeline_parent_map.push(op->node_id(), build_side_pipe); - break; - } - case TPlanNodeType::UNION_NODE: { - int child_count = tnode.num_children; - op.reset(new UnionSourceOperatorX(pool, tnode, next_operator_id(), descs)); - RETURN_IF_ERROR(cur_pipe->add_operator(op)); - - const auto downstream_pipeline_id = cur_pipe->id(); - if (_dag.find(downstream_pipeline_id) == _dag.end()) { - _dag.insert({downstream_pipeline_id, {}}); - } - for (int i = 0; i < child_count; i++) { - PipelinePtr build_side_pipe = add_pipeline(cur_pipe); - _dag[downstream_pipeline_id].push_back(build_side_pipe->id()); - DataSinkOperatorXPtr sink; - sink.reset(new UnionSinkOperatorX(i, next_sink_operator_id(), pool, tnode, descs)); - 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())); - // preset children pipelines. if any pipeline found this as its father, will use the prepared pipeline to build. - _pipeline_parent_map.push(op->node_id(), build_side_pipe); - } - break; - } - case TPlanNodeType::SORT_NODE: { - if (_runtime_state->enable_sort_spill()) { - op.reset(new SpillSortSourceOperatorX(pool, tnode, next_operator_id(), descs)); - } else { - op.reset(new SortSourceOperatorX(pool, tnode, next_operator_id(), descs)); - } - RETURN_IF_ERROR(cur_pipe->add_operator(op)); - - const auto downstream_pipeline_id = cur_pipe->id(); - if (_dag.find(downstream_pipeline_id) == _dag.end()) { - _dag.insert({downstream_pipeline_id, {}}); - } - cur_pipe = add_pipeline(cur_pipe); - _dag[downstream_pipeline_id].push_back(cur_pipe->id()); - - DataSinkOperatorXPtr sink; - if (_runtime_state->enable_sort_spill()) { - sink.reset(new SpillSortSinkOperatorX(pool, next_sink_operator_id(), tnode, descs)); - } else { - sink.reset(new SortSinkOperatorX(pool, next_sink_operator_id(), tnode, descs)); - } - sink->set_dests_id({op->operator_id()}); - RETURN_IF_ERROR(cur_pipe->set_sink(sink)); - RETURN_IF_ERROR(cur_pipe->sink_x()->init(tnode, _runtime_state.get())); - break; - } - case doris::TPlanNodeType::PARTITION_SORT_NODE: { - op.reset(new PartitionSortSourceOperatorX(pool, tnode, next_operator_id(), descs)); - RETURN_IF_ERROR(cur_pipe->add_operator(op)); - - const auto downstream_pipeline_id = cur_pipe->id(); - if (_dag.find(downstream_pipeline_id) == _dag.end()) { - _dag.insert({downstream_pipeline_id, {}}); - } - cur_pipe = add_pipeline(cur_pipe); - _dag[downstream_pipeline_id].push_back(cur_pipe->id()); - - DataSinkOperatorXPtr sink; - sink.reset(new PartitionSortSinkOperatorX(pool, next_sink_operator_id(), tnode, descs)); - sink->set_dests_id({op->operator_id()}); - RETURN_IF_ERROR(cur_pipe->set_sink(sink)); - RETURN_IF_ERROR(cur_pipe->sink_x()->init(tnode, _runtime_state.get())); - break; - } - case TPlanNodeType::ANALYTIC_EVAL_NODE: { - op.reset(new AnalyticSourceOperatorX(pool, tnode, next_operator_id(), descs)); - RETURN_IF_ERROR(cur_pipe->add_operator(op)); - - const auto downstream_pipeline_id = cur_pipe->id(); - if (_dag.find(downstream_pipeline_id) == _dag.end()) { - _dag.insert({downstream_pipeline_id, {}}); - } - cur_pipe = add_pipeline(cur_pipe); - _dag[downstream_pipeline_id].push_back(cur_pipe->id()); - - DataSinkOperatorXPtr sink; - sink.reset(new AnalyticSinkOperatorX(pool, next_sink_operator_id(), tnode, descs)); - sink->set_dests_id({op->operator_id()}); - RETURN_IF_ERROR(cur_pipe->set_sink(sink)); - RETURN_IF_ERROR(cur_pipe->sink_x()->init(tnode, _runtime_state.get())); - break; - } - case TPlanNodeType::INTERSECT_NODE: { - RETURN_IF_ERROR(_build_operators_for_set_operation_node( - pool, tnode, descs, op, cur_pipe, parent_idx, child_idx)); - break; - } - case TPlanNodeType::EXCEPT_NODE: { - RETURN_IF_ERROR(_build_operators_for_set_operation_node( - pool, tnode, descs, op, cur_pipe, parent_idx, child_idx)); - break; - } - case TPlanNodeType::REPEAT_NODE: { - op.reset(new RepeatOperatorX(pool, tnode, next_operator_id(), descs)); - RETURN_IF_ERROR(cur_pipe->add_operator(op)); - break; - } - case TPlanNodeType::TABLE_FUNCTION_NODE: { - op.reset(new TableFunctionOperatorX(pool, tnode, next_operator_id(), descs)); - RETURN_IF_ERROR(cur_pipe->add_operator(op)); - break; - } - case TPlanNodeType::ASSERT_NUM_ROWS_NODE: { - op.reset(new AssertNumRowsOperatorX(pool, tnode, next_operator_id(), descs)); - RETURN_IF_ERROR(cur_pipe->add_operator(op)); - break; - } - case TPlanNodeType::EMPTY_SET_NODE: { - op.reset(new EmptySetSourceOperatorX(pool, tnode, next_operator_id(), descs)); - RETURN_IF_ERROR(cur_pipe->add_operator(op)); - break; - } - case TPlanNodeType::DATA_GEN_SCAN_NODE: { - op.reset(new DataGenSourceOperatorX(pool, tnode, next_operator_id(), descs)); - RETURN_IF_ERROR(cur_pipe->add_operator(op)); - break; - } - case TPlanNodeType::SCHEMA_SCAN_NODE: { - op.reset(new SchemaScanOperatorX(pool, tnode, next_operator_id(), descs)); - RETURN_IF_ERROR(cur_pipe->add_operator(op)); - break; - } - case TPlanNodeType::META_SCAN_NODE: { - op.reset(new MetaScanOperatorX(pool, tnode, next_operator_id(), descs)); - RETURN_IF_ERROR(cur_pipe->add_operator(op)); - break; - } - case TPlanNodeType::SELECT_NODE: { - op.reset(new SelectOperatorX(pool, tnode, next_operator_id(), descs)); - RETURN_IF_ERROR(cur_pipe->add_operator(op)); - break; - } - default: - return Status::InternalError("Unsupported exec type in pipelineX: {}", - print_plan_node_type(tnode.node_type)); - } - - return Status::OK(); -} -// NOLINTEND(readability-function-cognitive-complexity) -// NOLINTEND(readability-function-size) - -template -Status PipelineXFragmentContext::_build_operators_for_set_operation_node( - ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs, OperatorXPtr& op, - PipelinePtr& cur_pipe, int parent_idx, int child_idx) { - op.reset(new SetSourceOperatorX(pool, tnode, next_operator_id(), descs)); - RETURN_IF_ERROR(cur_pipe->add_operator(op)); - - const auto downstream_pipeline_id = cur_pipe->id(); - if (_dag.find(downstream_pipeline_id) == _dag.end()) { - _dag.insert({downstream_pipeline_id, {}}); - } - - for (int child_id = 0; child_id < tnode.num_children; child_id++) { - PipelinePtr probe_side_pipe = add_pipeline(cur_pipe); - _dag[downstream_pipeline_id].push_back(probe_side_pipe->id()); - - DataSinkOperatorXPtr sink; - if (child_id == 0) { - sink.reset(new SetSinkOperatorX(child_id, next_sink_operator_id(), pool, - tnode, descs)); - } else { - sink.reset(new SetProbeSinkOperatorX(child_id, next_sink_operator_id(), - pool, tnode, descs)); - } - sink->set_dests_id({op->operator_id()}); - RETURN_IF_ERROR(probe_side_pipe->set_sink(sink)); - RETURN_IF_ERROR(probe_side_pipe->sink_x()->init(tnode, _runtime_state.get())); - // prepare children pipelines. if any pipeline found this as its father, will use the prepared pipeline to build. - _pipeline_parent_map.push(op->node_id(), probe_side_pipe); - } - - return Status::OK(); -} - -Status PipelineXFragmentContext::submit() { - if (_submitted) { - return Status::InternalError("submitted"); - } - _submitted = true; - - int submit_tasks = 0; - Status st; - auto* scheduler = _query_ctx->get_pipe_exec_scheduler(); - for (auto& task : _tasks) { - for (auto& t : task) { - st = scheduler->schedule_task(t.get()); - if (!st) { - std::lock_guard l(_status_lock); - cancel(PPlanFragmentCancelReason::INTERNAL_ERROR, "submit context fail"); - _total_tasks = submit_tasks; - break; - } - submit_tasks++; - } - } - if (!st.ok()) { - std::lock_guard l(_task_mutex); - if (_closed_tasks == _total_tasks) { - _close_fragment_instance(); - } - return Status::InternalError("Submit pipeline failed. err = {}, BE: {}", st.to_string(), - BackendOptions::get_localhost()); - } else { - return st; - } -} - -void PipelineXFragmentContext::close_sink() { - for (auto& tasks : _tasks) { - auto& root_task = *tasks.begin(); - auto st = root_task->close_sink(_prepared ? Status::RuntimeError("prepare failed") - : Status::OK()); - if (!st.ok()) { - LOG_WARNING("PipelineXFragmentContext::close_sink() error").tag("msg", st.msg()); - } - } -} - -void PipelineXFragmentContext::close_if_prepare_failed(Status st) { - for (auto& task : _tasks) { - for (auto& t : task) { - DCHECK(!t->is_pending_finish()); - WARN_IF_ERROR(t->close(st), "close_if_prepare_failed failed: "); - close_a_pipeline(); - } - } - _query_ctx->cancel(st.to_string(), st, _fragment_id); -} - -void PipelineXFragmentContext::_close_fragment_instance() { - if (_is_fragment_instance_closed) { - return; - } - Defer defer_op {[&]() { _is_fragment_instance_closed = true; }}; - _runtime_profile->total_time_counter()->update(_fragment_watcher.elapsed_time()); - static_cast(send_report(true)); - if (_runtime_state->enable_profile()) { - std::stringstream ss; - // Compute the _local_time_percent before pretty_print the runtime_profile - // Before add this operation, the print out like that: - // UNION_NODE (id=0):(Active: 56.720us, non-child: 00.00%) - // After add the operation, the print out like that: - // UNION_NODE (id=0):(Active: 56.720us, non-child: 82.53%) - // We can easily know the exec node execute time without child time consumed. - _runtime_state->runtime_profile()->compute_time_in_profile(); - _runtime_state->runtime_profile()->pretty_print(&ss); - if (_runtime_state->load_channel_profile()) { - _runtime_state->load_channel_profile()->pretty_print(&ss); - } - - LOG_INFO("Query {} fragment {} profile:\n {}", print_id(this->_query_id), - this->_fragment_id, ss.str()); - } - - if (_query_ctx->enable_profile()) { - _query_ctx->add_fragment_profile_x(_fragment_id, collect_realtime_profile_x(), - collect_realtime_load_channel_profile_x()); - } - - // all submitted tasks done - _exec_env->fragment_mgr()->remove_pipeline_context( - std::dynamic_pointer_cast(shared_from_this())); -} - -Status PipelineXFragmentContext::send_report(bool done) { - Status exec_status = Status::OK(); - { - std::lock_guard l(_status_lock); - exec_status = _query_ctx->exec_status(); - } - - // If plan is done successfully, but _is_report_success is false, - // no need to send report. - if (!_is_report_success && done && exec_status.ok()) { - return Status::NeedSendAgain(""); - } - - // If both _is_report_success and _is_report_on_cancel are false, - // which means no matter query is success or failed, no report is needed. - // This may happen when the query limit reached and - // a internal cancellation being processed - if (!_is_report_success && !_is_report_on_cancel) { - return Status::NeedSendAgain(""); - } - - std::vector runtime_states; - - for (auto& task_state : _task_runtime_states) { - runtime_states.push_back(task_state.get()); - } - - ReportStatusRequest req {true, - exec_status, - runtime_states, - nullptr, - _runtime_state->load_channel_profile(), - done || !exec_status.ok(), - _query_ctx->coord_addr, - _query_id, - _fragment_id, - TUniqueId(), - _backend_num, - _runtime_state.get(), - [this](Status st) { return update_status(st); }, - [this](const PPlanFragmentCancelReason& reason, - const std::string& msg) { cancel(reason, msg); }}; - - return _report_status_cb( - req, std::dynamic_pointer_cast(shared_from_this())); -} - -std::vector> -PipelineXFragmentContext::collect_realtime_profile_x() const { - std::vector> res; - DCHECK(_query_ctx->enable_pipeline_x_exec() == true) - << fmt::format("Query {} calling a pipeline X function, but its pipeline X is disabled", - print_id(this->_query_id)); - - // we do not have mutex to protect pipeline_id_to_profile - // so we need to make sure this funciton is invoked after fragment context - // has already been prepared. - if (!this->_prepared) { - std::string msg = - "Query " + print_id(this->_query_id) + " collecting profile, but its not prepared"; - DCHECK(false) << msg; - LOG_ERROR(msg); - return res; - } - - // pipeline_id_to_profile is initialized in prepare stage - for (auto& pipeline_profile : _runtime_state->pipeline_id_to_profile()) { - auto profile_ptr = std::make_shared(); - pipeline_profile->to_thrift(profile_ptr.get()); - res.push_back(profile_ptr); - } - - return res; -} - -std::shared_ptr -PipelineXFragmentContext::collect_realtime_load_channel_profile_x() const { - // we do not have mutex to protect pipeline_id_to_profile - // so we need to make sure this funciton is invoked after fragment context - // has already been prepared. - if (!this->_prepared) { - std::string msg = - "Query " + print_id(this->_query_id) + " collecting profile, but its not prepared"; - DCHECK(false) << msg; - LOG_ERROR(msg); - return nullptr; - } - - for (auto& runtime_state : _task_runtime_states) { - if (runtime_state->runtime_profile() == nullptr) { - continue; - } - - auto tmp_load_channel_profile = std::make_shared(); - - runtime_state->runtime_profile()->to_thrift(tmp_load_channel_profile.get()); - this->_runtime_state->load_channel_profile()->update(*tmp_load_channel_profile); - } - - auto load_channel_profile = std::make_shared(); - this->_runtime_state->load_channel_profile()->to_thrift(load_channel_profile.get()); - return load_channel_profile; -} - -std::string PipelineXFragmentContext::debug_string() { - fmt::memory_buffer debug_string_buffer; - fmt::format_to(debug_string_buffer, "PipelineXFragmentContext Info:\n"); - for (size_t j = 0; j < _tasks.size(); j++) { - fmt::format_to(debug_string_buffer, "Tasks in instance {}:\n", j); - for (size_t i = 0; i < _tasks[j].size(); i++) { - fmt::format_to(debug_string_buffer, "Task {}: {}\n", i, _tasks[j][i]->debug_string()); - } - } - - return fmt::to_string(debug_string_buffer); -} -} // namespace doris::pipeline diff --git a/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.h b/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.h deleted file mode 100644 index 31febc0d8aaf4da..000000000000000 --- a/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.h +++ /dev/null @@ -1,245 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#pragma once - -#include -#include -#include -#include -#include - -#include -#include -#include -#include -#include -#include -#include -#include - -#include "common/status.h" -#include "pipeline/pipeline.h" -#include "pipeline/pipeline_fragment_context.h" -#include "pipeline/pipeline_task.h" -#include "pipeline/pipeline_x/local_exchange/local_exchanger.h" -#include "pipeline/pipeline_x/pipeline_x_task.h" -#include "runtime/query_context.h" -#include "runtime/runtime_state.h" -#include "util/runtime_profile.h" -#include "util/stopwatch.hpp" - -namespace doris { -class ExecNode; -class DataSink; -struct ReportStatusRequest; -class ExecEnv; -class RuntimeFilterMergeControllerEntity; -class TDataSink; -class TPipelineFragmentParams; - -namespace pipeline { -class Dependency; - -class PipelineXFragmentContext : public PipelineFragmentContext { -public: - // Callback to report execution status of plan fragment. - // 'profile' is the cumulative profile, 'done' indicates whether the execution - // is done or still continuing. - // Note: this does not take a const RuntimeProfile&, because it might need to call - // functions like PrettyPrint() or to_thrift(), neither of which is const - // because they take locks. - PipelineXFragmentContext(const TUniqueId& query_id, const int fragment_id, - std::shared_ptr query_ctx, ExecEnv* exec_env, - const std::function& call_back, - const report_status_callback& report_status_cb); - - ~PipelineXFragmentContext() override; - - void instance_ids(std::vector& ins_ids) const override { - ins_ids.resize(_fragment_instance_ids.size()); - for (size_t i = 0; i < _fragment_instance_ids.size(); i++) { - ins_ids[i] = _fragment_instance_ids[i]; - } - } - - void instance_ids(std::vector& ins_ids) const override { - ins_ids.resize(_fragment_instance_ids.size()); - for (size_t i = 0; i < _fragment_instance_ids.size(); i++) { - ins_ids[i] = print_id(_fragment_instance_ids[i]); - } - } - - void add_merge_controller_handler( - std::shared_ptr& handler) override { - _merge_controller_handlers.emplace_back(handler); - } - - // bool is_canceled() const { return _runtime_state->is_cancelled(); } - - // Prepare global information including global states and the unique operator tree shared by all pipeline tasks. - Status prepare(const doris::TPipelineFragmentParams& request) override; - - Status submit() override; - - void close_if_prepare_failed(Status st) override; - void close_sink() override; - - void cancel(const PPlanFragmentCancelReason& reason = PPlanFragmentCancelReason::INTERNAL_ERROR, - const std::string& msg = "") override; - - Status send_report(bool) override; - - [[nodiscard]] int next_operator_id() { return _operator_id--; } - - [[nodiscard]] int max_operator_id() const { return _operator_id; } - - [[nodiscard]] int next_sink_operator_id() { return _sink_operator_id--; } - - [[nodiscard]] int max_sink_operator_id() const { return _sink_operator_id; } - - std::vector> collect_realtime_profile_x() const; - std::shared_ptr collect_realtime_load_channel_profile_x() const; - - std::string debug_string() override; - -private: - void _close_fragment_instance() override; - Status _build_pipeline_tasks(const doris::TPipelineFragmentParams& request) override; - Status _add_local_exchange(int pip_idx, int idx, int node_id, ObjectPool* pool, - PipelinePtr cur_pipe, DataDistribution data_distribution, - bool* do_local_exchange, int num_buckets, - const std::map& bucket_seq_to_instance_idx, - const std::map& shuffle_idx_to_instance_idx, - const bool ignore_data_distribution); - void _inherit_pipeline_properties(const DataDistribution& data_distribution, - PipelinePtr pipe_with_source, PipelinePtr pipe_with_sink); - Status _add_local_exchange_impl(int idx, ObjectPool* pool, PipelinePtr cur_pipe, - PipelinePtr new_pipe, DataDistribution data_distribution, - bool* do_local_exchange, int num_buckets, - const std::map& bucket_seq_to_instance_idx, - const std::map& shuffle_idx_to_instance_idx, - const bool ignore_data_distribution); - - [[nodiscard]] Status _build_pipelines(ObjectPool* pool, - const doris::TPipelineFragmentParams& request, - const DescriptorTbl& descs, OperatorXPtr* root, - PipelinePtr cur_pipe); - Status _create_tree_helper(ObjectPool* pool, const std::vector& tnodes, - const doris::TPipelineFragmentParams& request, - const DescriptorTbl& descs, OperatorXPtr parent, int* node_idx, - OperatorXPtr* root, PipelinePtr& cur_pipe, int child_idx); - - Status _create_operator(ObjectPool* pool, const TPlanNode& tnode, - const doris::TPipelineFragmentParams& request, - const DescriptorTbl& descs, OperatorXPtr& op, PipelinePtr& cur_pipe, - int parent_idx, int child_idx); - template - Status _build_operators_for_set_operation_node(ObjectPool* pool, const TPlanNode& tnode, - const DescriptorTbl& descs, OperatorXPtr& op, - PipelinePtr& cur_pipe, int parent_idx, - int child_idx); - - Status _create_data_sink(ObjectPool* pool, const TDataSink& thrift_sink, - const std::vector& output_exprs, - const TPipelineFragmentParams& params, const RowDescriptor& row_desc, - RuntimeState* state, DescriptorTbl& desc_tbl, - PipelineId cur_pipeline_id); - Status _plan_local_exchange(int num_buckets, - const std::map& bucket_seq_to_instance_idx, - const std::map& shuffle_idx_to_instance_idx); - Status _plan_local_exchange(int num_buckets, int pip_idx, PipelinePtr pip, - const std::map& bucket_seq_to_instance_idx, - const std::map& shuffle_idx_to_instance_idx, - const bool ignore_data_distribution); - - bool _enable_local_shuffle() const { return _runtime_state->enable_local_shuffle(); } - - OperatorXPtr _root_op = nullptr; - // this is a [n * m] matrix. n is parallelism of pipeline engine and m is the number of pipelines. - std::vector>> _tasks; - - bool _need_local_merge = false; - - // It is used to manage the lifecycle of RuntimeFilterMergeController - std::vector> _merge_controller_handlers; - - // TODO: remove the _sink and _multi_cast_stream_sink_senders to set both - // of it in pipeline task not the fragment_context -#ifdef __clang__ -#pragma clang diagnostic push -#pragma clang diagnostic ignored "-Wshadow-field" -#endif - DataSinkOperatorXPtr _sink = nullptr; -#ifdef __clang__ -#pragma clang diagnostic pop -#endif - - // `_dag` manage dependencies between pipelines by pipeline ID. the indices will be blocked by members - std::map> _dag; - - // We use preorder traversal to create an operator tree. When we meet a join node, we should - // build probe operator and build operator in separate pipelines. To do this, we should build - // ProbeSide first, and use `_pipelines_to_build` to store which pipeline the build operator - // is in, so we can build BuildSide once we complete probe side. - struct pipeline_parent_map { - std::map> _build_side_pipelines; - void push(int parent_node_id, PipelinePtr pipeline) { - if (!_build_side_pipelines.contains(parent_node_id)) { - _build_side_pipelines.insert({parent_node_id, {pipeline}}); - } else { - _build_side_pipelines[parent_node_id].push_back(pipeline); - } - } - void pop(PipelinePtr& cur_pipe, int parent_node_id, int child_idx) { - if (!_build_side_pipelines.contains(parent_node_id)) { - return; - } - DCHECK(_build_side_pipelines.contains(parent_node_id)); - auto& child_pipeline = _build_side_pipelines[parent_node_id]; - DCHECK(child_idx < child_pipeline.size()); - cur_pipe = child_pipeline[child_idx]; - } - void clear() { _build_side_pipelines.clear(); } - } _pipeline_parent_map; - - std::mutex _state_map_lock; - - int _operator_id = 0; - int _sink_operator_id = 0; - std::map, std::shared_ptr>> - _op_id_to_le_state; - - // UniqueId -> runtime mgr - std::map> _runtime_filter_mgr_map; - - //Here are two types of runtime states: - // - _runtime state is at the Fragment level. - // - _task_runtime_states is at the task level, unique to each task. - - std::vector _fragment_instance_ids; - // Local runtime states for each task - std::vector> _task_runtime_states; - - std::vector> _runtime_filter_states; - - // Total instance num running on all BEs - int _total_instances = -1; -}; - -} // namespace pipeline -} // namespace doris diff --git a/be/src/pipeline/pipeline_x/pipeline_x_task.cpp b/be/src/pipeline/pipeline_x/pipeline_x_task.cpp index 6ffc5571b4e563e..e8957318a6d66b0 100644 --- a/be/src/pipeline/pipeline_x/pipeline_x_task.cpp +++ b/be/src/pipeline/pipeline_x/pipeline_x_task.cpp @@ -29,8 +29,8 @@ #include "pipeline/exec/operator.h" #include "pipeline/exec/scan_operator.h" #include "pipeline/pipeline.h" +#include "pipeline/pipeline_fragment_context.h" #include "pipeline/task_queue.h" -#include "pipeline_x_fragment_context.h" #include "runtime/descriptors.h" #include "runtime/query_context.h" #include "runtime/thread_context.h" diff --git a/be/src/pipeline/pipeline_x/pipeline_x_task.h b/be/src/pipeline/pipeline_x/pipeline_x_task.h index 1f3dd9c3b71f5f9..c1a72836fd87207 100644 --- a/be/src/pipeline/pipeline_x/pipeline_x_task.h +++ b/be/src/pipeline/pipeline_x/pipeline_x_task.h @@ -126,8 +126,6 @@ class PipelineXTask : public PipelineTask { return _op_shared_states[id].get(); } - bool is_pipelineX() const override { return true; } - void wake_up(); DataSinkOperatorXPtr sink() const { return _sink; } diff --git a/be/src/pipeline/task_scheduler.cpp b/be/src/pipeline/task_scheduler.cpp index 8981a7e621c463b..0461999d1852605 100644 --- a/be/src/pipeline/task_scheduler.cpp +++ b/be/src/pipeline/task_scheduler.cpp @@ -48,156 +48,6 @@ namespace doris::pipeline { -BlockedTaskScheduler::BlockedTaskScheduler(std::string name) - : _name(std::move(name)), _started(false), _shutdown(false) {} - -Status BlockedTaskScheduler::start() { - LOG(INFO) << "BlockedTaskScheduler start"; - RETURN_IF_ERROR(Thread::create( - "BlockedTaskScheduler", _name, [this]() { this->_schedule(); }, &_thread)); - while (!this->_started.load()) { - std::this_thread::sleep_for(std::chrono::milliseconds(5)); - } - LOG(INFO) << "BlockedTaskScheduler started"; - return Status::OK(); -} - -void BlockedTaskScheduler::shutdown() { - LOG(INFO) << "Start shutdown BlockedTaskScheduler"; - if (!this->_shutdown) { - this->_shutdown = true; - if (_thread) { - _task_cond.notify_one(); - _thread->join(); - } - } -} - -Status BlockedTaskScheduler::add_blocked_task(PipelineTask* task) { - if (this->_shutdown) { - return Status::InternalError("BlockedTaskScheduler shutdown"); - } - std::unique_lock lock(_task_mutex); - if (task->is_pipelineX()) { - // put this task into current dependency's blocking queue and wait for event notification - // instead of using a separate BlockedTaskScheduler. - task->set_running(false); - return Status::OK(); - } - _blocked_tasks.push_back(task); - _task_cond.notify_one(); - task->set_running(false); - return Status::OK(); -} - -void BlockedTaskScheduler::_schedule() { - _started.store(true); - std::list local_blocked_tasks; - int empty_times = 0; - - while (!_shutdown) { - { - std::unique_lock lock(this->_task_mutex); - local_blocked_tasks.splice(local_blocked_tasks.end(), _blocked_tasks); - if (local_blocked_tasks.empty()) { - while (!_shutdown.load() && _blocked_tasks.empty()) { - _task_cond.wait_for(lock, std::chrono::milliseconds(10)); - } - - if (_shutdown.load()) { - break; - } - - DCHECK(!_blocked_tasks.empty()); - local_blocked_tasks.splice(local_blocked_tasks.end(), _blocked_tasks); - } - } - - auto origin_local_block_tasks_size = local_blocked_tasks.size(); - auto iter = local_blocked_tasks.begin(); - VecDateTimeValue now = VecDateTimeValue::local_time(); - while (iter != local_blocked_tasks.end()) { - auto* task = *iter; - auto state = task->get_state(); - task->log_detail_if_need(); - if (state == PipelineTaskState::PENDING_FINISH) { - // should cancel or should finish - if (task->is_pending_finish()) { - VLOG_DEBUG << "Task pending" << task->debug_string(); - iter++; - } else { - _make_task_run(local_blocked_tasks, iter, PipelineTaskState::PENDING_FINISH); - } - } else if (task->query_context()->is_cancelled()) { - _make_task_run(local_blocked_tasks, iter); - } else if (task->query_context()->is_timeout(now)) { - LOG(WARNING) << "Timeout, query_id=" << print_id(task->query_context()->query_id()) - << ", instance_id=" << print_id(task->instance_id()) - << ", task info: " << task->debug_string(); - - task->query_context()->cancel("", Status::Cancelled("")); - _make_task_run(local_blocked_tasks, iter); - } else if (state == PipelineTaskState::BLOCKED_FOR_DEPENDENCY) { - if (task->has_dependency()) { - iter++; - } else { - _make_task_run(local_blocked_tasks, iter); - } - } else if (state == PipelineTaskState::BLOCKED_FOR_SOURCE) { - if (task->source_can_read()) { - _make_task_run(local_blocked_tasks, iter); - } else { - iter++; - } - } else if (state == PipelineTaskState::BLOCKED_FOR_RF) { - if (task->runtime_filters_are_ready_or_timeout()) { - _make_task_run(local_blocked_tasks, iter); - } else { - iter++; - } - } else if (state == PipelineTaskState::BLOCKED_FOR_SINK) { - if (task->sink_can_write()) { - _make_task_run(local_blocked_tasks, iter); - } else { - iter++; - } - } else { - // TODO: DCHECK the state - _make_task_run(local_blocked_tasks, iter); - } - } - - if (origin_local_block_tasks_size == 0 || - local_blocked_tasks.size() == origin_local_block_tasks_size) { - empty_times += 1; - } else { - empty_times = 0; - } - - if (empty_times != 0 && (empty_times & (EMPTY_TIMES_TO_YIELD - 1)) == 0) { -#ifdef __x86_64__ - _mm_pause(); -#else - sched_yield(); -#endif - } - if (empty_times == EMPTY_TIMES_TO_YIELD * 10) { - empty_times = 0; - sched_yield(); - } - } - LOG(INFO) << "BlockedTaskScheduler schedule thread stop"; -} - -void BlockedTaskScheduler::_make_task_run(std::list& local_tasks, - std::list::iterator& task_itr, - PipelineTaskState t_state) { - auto* task = *task_itr; - task->set_state(t_state); - local_tasks.erase(task_itr++); - static_cast(task->get_task_queue()->push_back(task)); -} - TaskScheduler::~TaskScheduler() { stop(); LOG(INFO) << "Task scheduler " << _name << " shutdown"; @@ -241,13 +91,8 @@ void _close_task(PipelineTask* task, PipelineTaskState state, Status exec_status // for pending finish now. So that could call close directly. Status status = task->close(exec_status); if (!status.ok() && state != PipelineTaskState::CANCELED) { - if (task->is_pipelineX()) { //should call fragment context cancel, in it will call query context cancel - task->fragment_context()->cancel(PPlanFragmentCancelReason::INTERNAL_ERROR, - std::string(status.msg())); - } else { - task->query_context()->cancel(status.to_string(), - Status::Cancelled(status.to_string())); - } + task->fragment_context()->cancel(PPlanFragmentCancelReason::INTERNAL_ERROR, + std::string(status.msg())); state = PipelineTaskState::CANCELED; } task->set_state(state); @@ -264,7 +109,7 @@ void TaskScheduler::_do_work(size_t index) { if (!task) { continue; } - if (task->is_pipelineX() && task->is_running()) { + if (task->is_running()) { static_cast(_task_queue->push_back(task, index)); continue; } @@ -278,8 +123,6 @@ void TaskScheduler::_do_work(size_t index) { // If the state is PENDING_FINISH, then the task is come from blocked queue, its is_pending_finish // has to return false. The task is finished and need to close now. if (state == PipelineTaskState::PENDING_FINISH) { - DCHECK(task->is_pipelineX() || !task->is_pending_finish()) - << "must not pending close " << task->debug_string(); Status exec_status = fragment_ctx->get_query_ctx()->exec_status(); _close_task(task, canceled ? PipelineTaskState::CANCELED : PipelineTaskState::FINISHED, exec_status); @@ -301,13 +144,8 @@ void TaskScheduler::_do_work(size_t index) { continue; } - if (task->is_pipelineX()) { - task->set_state(PipelineTaskState::RUNNABLE); - } + task->set_state(PipelineTaskState::RUNNABLE); - DCHECK(task->is_pipelineX() || task->get_state() == PipelineTaskState::RUNNABLE) - << "state:" << get_state_name(task->get_state()) - << " task: " << task->debug_string(); // task exec bool eos = false; auto status = Status::OK(); @@ -354,11 +192,9 @@ void TaskScheduler::_do_work(size_t index) { continue; } else if (!status.ok()) { task->set_eos_time(); - LOG(WARNING) << fmt::format( - "Pipeline task failed. query_id: {} reason: {}", - PrintInstanceStandardInfo(task->query_context()->query_id(), - task->fragment_context()->get_fragment_instance_id()), - status.to_string()); + LOG(WARNING) << fmt::format("Pipeline task failed. query_id: {} reason: {}", + print_id(task->query_context()->query_id()), + status.to_string()); // Print detail informations below when you debugging here. // // LOG(WARNING)<< "task:\n"<debug_string(); @@ -375,35 +211,21 @@ void TaskScheduler::_do_work(size_t index) { task->set_eos_time(); // TODO: pipeline parallel need to wait the last task finish to call finalize // and find_p_dependency - VLOG_DEBUG << fmt::format( - "Try close task: {}, fragment_ctx->is_canceled(): {}", - PrintInstanceStandardInfo(task->query_context()->query_id(), - task->fragment_context()->get_fragment_instance_id()), - fragment_ctx->is_canceled()); - if (task->is_pipelineX()) { - // is pending finish will add the task to dependency's blocking queue, and then the task will be - // added to running queue when dependency is ready. - if (task->is_pending_finish()) { - // Only meet eos, should set task to PENDING_FINISH state - task->set_state(PipelineTaskState::PENDING_FINISH); - task->set_running(false); - } else { - // Close the task directly? - Status exec_status = fragment_ctx->get_query_ctx()->exec_status(); - _close_task( - task, - canceled ? PipelineTaskState::CANCELED : PipelineTaskState::FINISHED, - exec_status); - } - } else { + VLOG_DEBUG << fmt::format("Try close task: {}, fragment_ctx->is_canceled(): {}", + print_id(task->query_context()->query_id()), + fragment_ctx->is_canceled()); + // is pending finish will add the task to dependency's blocking queue, and then the task will be + // added to running queue when dependency is ready. + if (task->is_pending_finish()) { // Only meet eos, should set task to PENDING_FINISH state - // pipeline is ok, because it will check is pending finish, and if it is ready, it will be invoked. task->set_state(PipelineTaskState::PENDING_FINISH); task->set_running(false); - // After the task is added to the block queue, it maybe run by another thread - // and the task maybe released in the other thread. And will core at - // task set running. - static_cast(_blocked_task_scheduler->add_blocked_task(task)); + } else { + // Close the task directly? + Status exec_status = fragment_ctx->get_query_ctx()->exec_status(); + _close_task(task, + canceled ? PipelineTaskState::CANCELED : PipelineTaskState::FINISHED, + exec_status); } continue; } @@ -414,7 +236,7 @@ void TaskScheduler::_do_work(size_t index) { case PipelineTaskState::BLOCKED_FOR_SINK: case PipelineTaskState::BLOCKED_FOR_RF: case PipelineTaskState::BLOCKED_FOR_DEPENDENCY: - static_cast(_blocked_task_scheduler->add_blocked_task(task)); + task->set_running(false); break; case PipelineTaskState::RUNNABLE: task->set_running(false); diff --git a/be/src/pipeline/task_scheduler.h b/be/src/pipeline/task_scheduler.h index 5bbf85fad452fd2..8e513748203e214 100644 --- a/be/src/pipeline/task_scheduler.h +++ b/be/src/pipeline/task_scheduler.h @@ -44,41 +44,11 @@ class TaskQueue; namespace doris::pipeline { -class BlockedTaskScheduler { -public: - explicit BlockedTaskScheduler(std::string name); - - ~BlockedTaskScheduler() = default; - - Status start(); - void shutdown(); - Status add_blocked_task(PipelineTask* task); - -private: - std::mutex _task_mutex; - std::string _name; - std::condition_variable _task_cond; - std::list _blocked_tasks; - - scoped_refptr _thread; - std::atomic _started; - std::atomic _shutdown; - - static constexpr auto EMPTY_TIMES_TO_YIELD = 64; - - void _schedule(); - void _make_task_run(std::list& local_tasks, - std::list::iterator& task_itr, - PipelineTaskState state = PipelineTaskState::RUNNABLE); -}; - class TaskScheduler { public: - TaskScheduler(ExecEnv* exec_env, std::shared_ptr b_scheduler, - std::shared_ptr task_queue, std::string name, + TaskScheduler(ExecEnv* exec_env, std::shared_ptr task_queue, std::string name, CgroupCpuCtl* cgroup_cpu_ctl) : _task_queue(std::move(task_queue)), - _blocked_task_scheduler(std::move(b_scheduler)), _shutdown(false), _name(name), _cgroup_cpu_ctl(cgroup_cpu_ctl) {} @@ -97,7 +67,6 @@ class TaskScheduler { std::unique_ptr _fix_thread_pool; std::shared_ptr _task_queue; std::vector>> _markers; - std::shared_ptr _blocked_task_scheduler; std::atomic _shutdown; std::string _name; CgroupCpuCtl* _cgroup_cpu_ctl = nullptr; diff --git a/be/src/runtime/CMakeLists.txt b/be/src/runtime/CMakeLists.txt index 3bfec93edfc0833..a0b3b799a764cb5 100644 --- a/be/src/runtime/CMakeLists.txt +++ b/be/src/runtime/CMakeLists.txt @@ -25,7 +25,7 @@ set(EXECUTABLE_OUTPUT_PATH "${BUILD_DIR}/src/runtime") file(GLOB_RECURSE RUNTIME_FILES CONFIGURE_DEPENDS *.cpp *.cc) -if (NOT USE_JEMALLOC OR NOT USE_MEM_TRACKER OR NOT USE_JEMALLOC_HOOK) +if (NOT USE_JEMALLOC OR NOT USE_MEM_TRACKER) list(REMOVE_ITEM RUNTIME_FILES ${CMAKE_CURRENT_SOURCE_DIR}/memory/jemalloc_hook.cpp) endif() diff --git a/be/src/runtime/exec_env.h b/be/src/runtime/exec_env.h index eb3e6f8a89c81bf..1cf0aea8349d947 100644 --- a/be/src/runtime/exec_env.h +++ b/be/src/runtime/exec_env.h @@ -45,7 +45,6 @@ class DeltaWriterV2Pool; } // namespace vectorized namespace pipeline { class TaskScheduler; -class BlockedTaskScheduler; struct RuntimeFilterTimerQueue; } // namespace pipeline class WorkloadGroupMgr; @@ -305,10 +304,6 @@ class ExecEnv { } std::shared_ptr get_dummy_lru_cache() { return _dummy_lru_cache; } - std::shared_ptr get_global_block_scheduler() { - return _global_block_scheduler; - } - pipeline::RuntimeFilterTimerQueue* runtime_filter_timer_queue() { return _runtime_filter_timer_queue; } @@ -453,11 +448,6 @@ class ExecEnv { TabletHotspot* _tablet_hotspot; CloudWarmUpManager* _cloud_warm_up_manager; - // used for query with group cpu hard limit - std::shared_ptr _global_block_scheduler; - // used for query without workload group - std::shared_ptr _without_group_block_scheduler; - pipeline::RuntimeFilterTimerQueue* _runtime_filter_timer_queue = nullptr; WorkloadSchedPolicyMgr* _workload_sched_mgr = nullptr; diff --git a/be/src/runtime/exec_env_init.cpp b/be/src/runtime/exec_env_init.cpp index 5a7e39cf158c419..ae36757b10a37c1 100644 --- a/be/src/runtime/exec_env_init.cpp +++ b/be/src/runtime/exec_env_init.cpp @@ -364,15 +364,10 @@ Status ExecEnv::init_pipeline_task_scheduler() { LOG_INFO("pipeline executors_size set ").tag("size", executors_size); // TODO pipeline workload group combie two blocked schedulers. auto t_queue = std::make_shared(executors_size); - _without_group_block_scheduler = - std::make_shared("PipeNoGSchePool"); - _without_group_task_scheduler = new pipeline::TaskScheduler( - this, _without_group_block_scheduler, t_queue, "PipeNoGSchePool", nullptr); + _without_group_task_scheduler = + new pipeline::TaskScheduler(this, t_queue, "PipeNoGSchePool", nullptr); RETURN_IF_ERROR(_without_group_task_scheduler->start()); - RETURN_IF_ERROR(_without_group_block_scheduler->start()); - _global_block_scheduler = std::make_shared("PipeGBlockSche"); - RETURN_IF_ERROR(_global_block_scheduler->start()); _runtime_filter_timer_queue = new doris::pipeline::RuntimeFilterTimerQueue(); _runtime_filter_timer_queue->run(); return Status::OK(); @@ -627,10 +622,8 @@ void ExecEnv::destroy() { // stop workload scheduler SAFE_STOP(_workload_sched_mgr); // stop pipline step 1, non-cgroup execution - SAFE_SHUTDOWN(_without_group_block_scheduler.get()); SAFE_STOP(_without_group_task_scheduler); // stop pipline step 2, cgroup execution - SAFE_SHUTDOWN(_global_block_scheduler.get()); SAFE_STOP(_workload_group_manager); SAFE_STOP(_external_scan_context_mgr); diff --git a/be/src/runtime/fragment_mgr.cpp b/be/src/runtime/fragment_mgr.cpp index c4605f1de54590e..7dbf1571e50737d 100644 --- a/be/src/runtime/fragment_mgr.cpp +++ b/be/src/runtime/fragment_mgr.cpp @@ -42,7 +42,6 @@ #include #include "common/status.h" -#include "pipeline/pipeline_x/pipeline_x_fragment_context.h" // IWYU pragma: no_include #include // IWYU pragma: keep #include @@ -845,7 +844,7 @@ Status FragmentMgr::exec_plan_fragment(const TPipelineFragmentParams& params, _setup_shared_hashtable_for_broadcast_join(params, query_ctx.get()); int64_t duration_ns = 0; std::shared_ptr context = - std::make_shared( + std::make_shared( query_ctx->query_id(), params.fragment_id, query_ctx, _exec_env, cb, std::bind(std::mem_fn(&FragmentMgr::trigger_pipeline_context_report), this, std::placeholders::_1, std::placeholders::_2)); @@ -892,7 +891,7 @@ Status FragmentMgr::exec_plan_fragment(const TPipelineFragmentParams& params, g_fragment_last_active_time.set_value(now); std::lock_guard lock(_lock); std::vector ins_ids; - reinterpret_cast(context.get())->instance_ids(ins_ids); + context->instance_ids(ins_ids); // TODO: simplify this mapping for (const auto& ins_id : ins_ids) { _pipeline_map.insert({ins_id, context}); @@ -1029,8 +1028,7 @@ void FragmentMgr::cancel_worker() { for (auto& pipeline_itr : _pipeline_map) { if (pipeline_itr.second->is_timeout(now)) { std::vector ins_ids; - reinterpret_cast(pipeline_itr.second.get()) - ->instance_ids(ins_ids); + pipeline_itr.second->instance_ids(ins_ids); for (auto& ins_id : ins_ids) { to_cancel.push_back(ins_id); } diff --git a/be/src/runtime/fragment_mgr.h b/be/src/runtime/fragment_mgr.h index 25b555f4fe8d60a..5b70be6d8a5c41b 100644 --- a/be/src/runtime/fragment_mgr.h +++ b/be/src/runtime/fragment_mgr.h @@ -51,7 +51,6 @@ extern bvar::Status g_fragment_last_active_time; namespace pipeline { class PipelineFragmentContext; -class PipelineXFragmentContext; } // namespace pipeline class QueryContext; class ExecEnv; @@ -104,7 +103,7 @@ class FragmentMgr : public RestMonitorIface { void cancel_instance(const TUniqueId& instance_id, const PPlanFragmentCancelReason& reason, const std::string& msg = ""); // Cancel fragment (only pipelineX). - // {query id fragment} -> PipelineXFragmentContext + // {query id fragment} -> PipelineFragmentContext void cancel_fragment(const TUniqueId& query_id, int32_t fragment_id, const PPlanFragmentCancelReason& reason, const std::string& msg = ""); diff --git a/be/src/runtime/load_stream_mgr.cpp b/be/src/runtime/load_stream_mgr.cpp index 7d9de12d0782ea7..1964cf257e38e0b 100644 --- a/be/src/runtime/load_stream_mgr.cpp +++ b/be/src/runtime/load_stream_mgr.cpp @@ -37,9 +37,13 @@ LoadStreamMgr::LoadStreamMgr(uint32_t segment_file_writer_thread_num, : _num_threads(segment_file_writer_thread_num), _heavy_work_pool(heavy_work_pool), _light_work_pool(light_work_pool) { + uint32_t num_cpu = std::thread::hardware_concurrency(); + uint32_t thread_num = num_cpu == 0 ? segment_file_writer_thread_num + : std::min(segment_file_writer_thread_num, + num_cpu * config::max_flush_thread_num_per_cpu); static_cast(ThreadPoolBuilder("SegmentFileWriterThreadPool") - .set_min_threads(segment_file_writer_thread_num) - .set_max_threads(segment_file_writer_thread_num) + .set_min_threads(thread_num) + .set_max_threads(thread_num) .build(&_file_writer_thread_pool)); } diff --git a/be/src/runtime/query_context.cpp b/be/src/runtime/query_context.cpp index 081d8ca1f590edb..c521c855515952c 100644 --- a/be/src/runtime/query_context.cpp +++ b/be/src/runtime/query_context.cpp @@ -33,7 +33,6 @@ #include "olap/olap_common.h" #include "pipeline/pipeline_fragment_context.h" #include "pipeline/pipeline_x/dependency.h" -#include "pipeline/pipeline_x/pipeline_x_fragment_context.h" #include "runtime/exec_env.h" #include "runtime/fragment_mgr.h" #include "runtime/runtime_query_statistics_mgr.h" @@ -430,20 +429,16 @@ QueryContext::_collect_realtime_query_profile_x() const { for (auto& [fragment_id, fragment_ctx_wptr] : _fragment_id_to_pipeline_ctx) { if (auto fragment_ctx = fragment_ctx_wptr.lock()) { - // In theory, cast result can not be nullptr since we have checked the pipeline X engine above - std::shared_ptr fragment_ctx_x = - std::dynamic_pointer_cast(fragment_ctx); - - if (fragment_ctx_x == nullptr) { + if (fragment_ctx == nullptr) { std::string msg = - fmt::format("PipelineXFragmentContext is nullptr, query {} fragment_id: {}", + fmt::format("PipelineFragmentContext is nullptr, query {} fragment_id: {}", print_id(_query_id), fragment_id); LOG_ERROR(msg); DCHECK(false) << msg; continue; } - auto profile = fragment_ctx_x->collect_realtime_profile_x(); + auto profile = fragment_ctx->collect_realtime_profile_x(); if (profile.empty()) { std::string err_msg = fmt::format( diff --git a/be/src/runtime/query_statistics.cpp b/be/src/runtime/query_statistics.cpp index de9507041803194..551f518f22f2b0a 100644 --- a/be/src/runtime/query_statistics.cpp +++ b/be/src/runtime/query_statistics.cpp @@ -32,6 +32,10 @@ void QueryStatistics::merge(const QueryStatistics& other) { cpu_nanos += other.cpu_nanos.load(std::memory_order_relaxed); shuffle_send_bytes += other.shuffle_send_bytes.load(std::memory_order_relaxed); shuffle_send_rows += other.shuffle_send_rows.load(std::memory_order_relaxed); + _scan_bytes_from_local_storage += + other._scan_bytes_from_local_storage.load(std::memory_order_relaxed); + _scan_bytes_from_remote_storage += + other._scan_bytes_from_remote_storage.load(std::memory_order_relaxed); int64_t other_peak_mem = other.max_peak_memory_bytes.load(std::memory_order_relaxed); if (other_peak_mem > this->max_peak_memory_bytes) { @@ -51,6 +55,8 @@ void QueryStatistics::to_pb(PQueryStatistics* statistics) { statistics->set_cpu_ms(cpu_nanos / NANOS_PER_MILLIS); statistics->set_returned_rows(returned_rows); statistics->set_max_peak_memory_bytes(max_peak_memory_bytes); + statistics->set_scan_bytes_from_remote_storage(_scan_bytes_from_remote_storage); + statistics->set_scan_bytes_from_local_storage(_scan_bytes_from_local_storage); } void QueryStatistics::to_thrift(TQueryStatistics* statistics) const { @@ -64,12 +70,16 @@ void QueryStatistics::to_thrift(TQueryStatistics* statistics) const { current_used_memory_bytes.load(std::memory_order_relaxed)); statistics->__set_shuffle_send_bytes(shuffle_send_bytes.load(std::memory_order_relaxed)); statistics->__set_shuffle_send_rows(shuffle_send_rows.load(std::memory_order_relaxed)); + statistics->__set_scan_bytes_from_remote_storage(_scan_bytes_from_remote_storage); + statistics->__set_scan_bytes_from_local_storage(_scan_bytes_from_local_storage); } void QueryStatistics::from_pb(const PQueryStatistics& statistics) { scan_rows = statistics.scan_rows(); scan_bytes = statistics.scan_bytes(); cpu_nanos = statistics.cpu_ms() * NANOS_PER_MILLIS; + _scan_bytes_from_local_storage = statistics.scan_bytes_from_local_storage(); + _scan_bytes_from_remote_storage = statistics.scan_bytes_from_remote_storage(); } void QueryStatistics::merge(QueryStatisticsRecvr* recvr) { diff --git a/be/src/runtime/query_statistics.h b/be/src/runtime/query_statistics.h index a9f6e192ec00d46..0a1c5c9f7ba33f3 100644 --- a/be/src/runtime/query_statistics.h +++ b/be/src/runtime/query_statistics.h @@ -73,6 +73,14 @@ class QueryStatistics { this->shuffle_send_rows.fetch_add(delta_rows, std::memory_order_relaxed); } + void add_scan_bytes_from_local_storage(int64_t scan_bytes_from_local_storage) { + this->_scan_bytes_from_local_storage += scan_bytes_from_local_storage; + } + + void add_scan_bytes_from_remote_storage(int64_t scan_bytes_from_remote_storage) { + this->_scan_bytes_from_remote_storage += scan_bytes_from_remote_storage; + } + void set_returned_rows(int64_t num_rows) { this->returned_rows = num_rows; } void set_max_peak_memory_bytes(int64_t max_peak_memory_bytes) { @@ -95,6 +103,8 @@ class QueryStatistics { cpu_nanos.store(0, std::memory_order_relaxed); shuffle_send_bytes.store(0, std::memory_order_relaxed); shuffle_send_rows.store(0, std::memory_order_relaxed); + _scan_bytes_from_local_storage.store(0); + _scan_bytes_from_remote_storage.store(0); returned_rows = 0; max_peak_memory_bytes.store(0, std::memory_order_relaxed); @@ -120,6 +130,8 @@ class QueryStatistics { std::atomic scan_rows; std::atomic scan_bytes; std::atomic cpu_nanos; + std::atomic _scan_bytes_from_local_storage; + std::atomic _scan_bytes_from_remote_storage; // number rows returned by query. // only set once by result sink when closing. int64_t returned_rows; diff --git a/be/src/runtime/runtime_state.cpp b/be/src/runtime/runtime_state.cpp index 2713ee441dd0dfa..4df90fd798af24b 100644 --- a/be/src/runtime/runtime_state.cpp +++ b/be/src/runtime/runtime_state.cpp @@ -141,7 +141,7 @@ RuntimeState::RuntimeState(const TUniqueId& instance_id, const TUniqueId& query_ query_id, RuntimeFilterParamsContext::create(this), _query_mem_tracker)); } -RuntimeState::RuntimeState(pipeline::PipelineXFragmentContext*, const TUniqueId& instance_id, +RuntimeState::RuntimeState(pipeline::PipelineFragmentContext*, const TUniqueId& instance_id, const TUniqueId& query_id, int32_t fragment_id, const TQueryOptions& query_options, const TQueryGlobals& query_globals, ExecEnv* exec_env, QueryContext* ctx) diff --git a/be/src/runtime/runtime_state.h b/be/src/runtime/runtime_state.h index b266b76778f6722..025d9517c28ef4c 100644 --- a/be/src/runtime/runtime_state.h +++ b/be/src/runtime/runtime_state.h @@ -49,7 +49,7 @@ class IRuntimeFilter; namespace pipeline { class PipelineXLocalStateBase; class PipelineXSinkLocalStateBase; -class PipelineXFragmentContext; +class PipelineFragmentContext; class PipelineXTask; } // namespace pipeline @@ -76,7 +76,7 @@ class RuntimeState { ExecEnv* exec_env, QueryContext* ctx); // for only use in pipelineX - RuntimeState(pipeline::PipelineXFragmentContext*, const TUniqueId& instance_id, + RuntimeState(pipeline::PipelineFragmentContext*, const TUniqueId& instance_id, const TUniqueId& query_id, int32 fragment_id, const TQueryOptions& query_options, const TQueryGlobals& query_globals, ExecEnv* exec_env, QueryContext* ctx); @@ -662,7 +662,7 @@ class RuntimeState { // runtime filter std::unique_ptr _runtime_filter_mgr; - // owned by PipelineXFragmentContext + // owned by PipelineFragmentContext RuntimeFilterMgr* _pipeline_x_runtime_filter_mgr = nullptr; // Data stream receivers created by a plan fragment are gathered here to make sure diff --git a/be/src/runtime/stream_load/stream_load_executor.cpp b/be/src/runtime/stream_load/stream_load_executor.cpp index 720c2e868987e79..58621c77a2a1159 100644 --- a/be/src/runtime/stream_load/stream_load_executor.cpp +++ b/be/src/runtime/stream_load/stream_load_executor.cpp @@ -78,38 +78,25 @@ Status StreamLoadExecutor::execute_plan_fragment(std::shared_ptrexec_env()->new_load_stream_mgr()->remove(ctx->id); ctx->commit_infos = std::move(state->tablet_commit_infos()); - if (status->ok()) { - ctx->number_total_rows = state->num_rows_load_total(); - ctx->number_loaded_rows = state->num_rows_load_success(); - ctx->number_filtered_rows = state->num_rows_load_filtered(); - ctx->number_unselected_rows = state->num_rows_load_unselected(); - - int64_t num_selected_rows = ctx->number_total_rows - ctx->number_unselected_rows; - if (!ctx->group_commit && num_selected_rows > 0 && - (double)ctx->number_filtered_rows / num_selected_rows > ctx->max_filter_ratio) { - // NOTE: Do not modify the error message here, for historical reasons, - // some users may rely on this error message. - *status = Status::DataQualityError("too many filtered rows"); - } - if (ctx->number_filtered_rows > 0 && !state->get_error_log_file_path().empty()) { - ctx->error_url = to_load_error_http_path(state->get_error_log_file_path()); - } + ctx->number_total_rows = state->num_rows_load_total(); + ctx->number_loaded_rows = state->num_rows_load_success(); + ctx->number_filtered_rows = state->num_rows_load_filtered(); + ctx->number_unselected_rows = state->num_rows_load_unselected(); + int64_t num_selected_rows = ctx->number_total_rows - ctx->number_unselected_rows; + if (!ctx->group_commit && num_selected_rows > 0 && + (double)ctx->number_filtered_rows / num_selected_rows > ctx->max_filter_ratio) { + // NOTE: Do not modify the error message here, for historical reasons, + // some users may rely on this error message. + *status = Status::DataQualityError("too many filtered rows"); + } + if (ctx->number_filtered_rows > 0 && !state->get_error_log_file_path().empty()) { + ctx->error_url = to_load_error_http_path(state->get_error_log_file_path()); + } - if (status->ok()) { - DorisMetrics::instance()->stream_receive_bytes_total->increment(ctx->receive_bytes); - DorisMetrics::instance()->stream_load_rows_total->increment( - ctx->number_loaded_rows); - } + if (status->ok()) { + DorisMetrics::instance()->stream_receive_bytes_total->increment(ctx->receive_bytes); + DorisMetrics::instance()->stream_load_rows_total->increment(ctx->number_loaded_rows); } else { - if (ctx->group_commit) { - ctx->number_total_rows = state->num_rows_load_total(); - ctx->number_loaded_rows = state->num_rows_load_success(); - ctx->number_filtered_rows = state->num_rows_load_filtered(); - ctx->number_unselected_rows = state->num_rows_load_unselected(); - if (ctx->number_filtered_rows > 0 && !state->get_error_log_file_path().empty()) { - ctx->error_url = to_load_error_http_path(state->get_error_log_file_path()); - } - } LOG(WARNING) << "fragment execute failed" << ", err_msg=" << status->to_string() << ", " << ctx->brief(); // cancel body_sink, make sender known it diff --git a/be/src/runtime/workload_group/workload_group.cpp b/be/src/runtime/workload_group/workload_group.cpp index c82346f040ec829..05e38b973c6057c 100644 --- a/be/src/runtime/workload_group/workload_group.cpp +++ b/be/src/runtime/workload_group/workload_group.cpp @@ -361,9 +361,8 @@ void WorkloadGroup::upsert_task_scheduler(WorkloadGroupInfo* tg_info, ExecEnv* e } auto task_queue = std::make_shared(executors_size); std::unique_ptr pipeline_task_scheduler = - std::make_unique( - exec_env, exec_env->get_global_block_scheduler(), std::move(task_queue), - "Pipe_" + tg_name, cg_cpu_ctl_ptr); + std::make_unique(exec_env, std::move(task_queue), + "Pipe_" + tg_name, cg_cpu_ctl_ptr); Status ret = pipeline_task_scheduler->start(); if (ret.ok()) { _task_sched = std::move(pipeline_task_scheduler); diff --git a/be/src/service/doris_main.cpp b/be/src/service/doris_main.cpp index 731e09c6be9fc2d..ca7fca1ce62de3b 100644 --- a/be/src/service/doris_main.cpp +++ b/be/src/service/doris_main.cpp @@ -169,8 +169,9 @@ auto instruction_fail_to_string(InstructionFail fail) { case InstructionFail::ARM_NEON: ret("ARM_NEON"); } - LOG(FATAL) << "__builtin_unreachable"; - __builtin_unreachable(); + + LOG(ERROR) << "Unrecognized instruction fail value." << std::endl; + exit(-1); } sigjmp_buf jmpbuf; @@ -439,6 +440,7 @@ int main(int argc, char** argv) { it = paths.erase(it); } else { LOG(ERROR) << "read write test file failed, path=" << it->path; + // if only one disk and the disk is full, also need exit because rocksdb will open failed exit(-1); } } else { diff --git a/be/src/util/s3_util.cpp b/be/src/util/s3_util.cpp index 05fabf08489f469..3563ab4a609a5cf 100644 --- a/be/src/util/s3_util.cpp +++ b/be/src/util/s3_util.cpp @@ -201,6 +201,11 @@ std::shared_ptr S3ClientFactory::create(const S3ClientConf& s if (s3_conf.connect_timeout_ms > 0) { aws_config.connectTimeoutMs = s3_conf.connect_timeout_ms; } + + if (config::s3_client_http_scheme == "http") { + aws_config.scheme = Aws::Http::Scheme::HTTP; + } + aws_config.retryStrategy = std::make_shared(config::max_s3_client_retry); std::shared_ptr new_client; diff --git a/be/src/util/s3_util.h b/be/src/util/s3_util.h index 62d8b4f1d64e7dc..46226b793591a41 100644 --- a/be/src/util/s3_util.h +++ b/be/src/util/s3_util.h @@ -132,8 +132,8 @@ struct S3Conf { cloud::ObjectStoreInfoPB::Provider provider; std::string to_string() const { - return fmt::format("(bucket={}, prefix={}, client_conf={})", bucket, prefix, - client_conf.to_string()); + return fmt::format("(bucket={}, prefix={}, client_conf={}, sse_enabled={})", bucket, prefix, + client_conf.to_string(), sse_enabled); } }; diff --git a/be/src/vec/aggregate_functions/aggregate_function_simple_factory.cpp b/be/src/vec/aggregate_functions/aggregate_function_simple_factory.cpp index 00597b212befd08..d95d0ce6ccb90da 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_simple_factory.cpp +++ b/be/src/vec/aggregate_functions/aggregate_function_simple_factory.cpp @@ -40,6 +40,7 @@ void register_aggregate_function_count(AggregateFunctionSimpleFactory& factory); void register_aggregate_function_count_by_enum(AggregateFunctionSimpleFactory& factory); void register_aggregate_function_HLL_union_agg(AggregateFunctionSimpleFactory& factory); void register_aggregate_function_uniq(AggregateFunctionSimpleFactory& factory); +void register_aggregate_function_uniq_distribute_key(AggregateFunctionSimpleFactory& factory); void register_aggregate_function_bit(AggregateFunctionSimpleFactory& factory); void register_aggregate_function_bitmap(AggregateFunctionSimpleFactory& factory); void register_aggregate_function_quantile_state(AggregateFunctionSimpleFactory& factory); @@ -80,6 +81,7 @@ AggregateFunctionSimpleFactory& AggregateFunctionSimpleFactory::instance() { register_aggregate_function_count(instance); register_aggregate_function_count_by_enum(instance); register_aggregate_function_uniq(instance); + register_aggregate_function_uniq_distribute_key(instance); register_aggregate_function_bit(instance); register_aggregate_function_bitmap(instance); register_aggregate_function_group_array_intersect(instance); diff --git a/be/src/vec/aggregate_functions/aggregate_function_uniq.h b/be/src/vec/aggregate_functions/aggregate_function_uniq.h index 2e8855134ebd314..58abd3842c21b27 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_uniq.h +++ b/be/src/vec/aggregate_functions/aggregate_function_uniq.h @@ -75,7 +75,7 @@ struct AggregateFunctionUniqExactData { Set set; - static String get_name() { return "uniqExact"; } + static String get_name() { return "multi_distinct"; } }; namespace detail { diff --git a/be/src/vec/aggregate_functions/aggregate_function_uniq_distribute_key.cpp b/be/src/vec/aggregate_functions/aggregate_function_uniq_distribute_key.cpp new file mode 100644 index 000000000000000..3bf979483b527c9 --- /dev/null +++ b/be/src/vec/aggregate_functions/aggregate_function_uniq_distribute_key.cpp @@ -0,0 +1,73 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "vec/aggregate_functions/aggregate_function_uniq_distribute_key.h" + +#include + +#include "vec/aggregate_functions/aggregate_function_simple_factory.h" +#include "vec/aggregate_functions/factory_helpers.h" +#include "vec/aggregate_functions/helpers.h" + +namespace doris::vectorized { + +template