From 378b2f21e22b11e8b912b077f2032e63c1ca0c81 Mon Sep 17 00:00:00 2001 From: Mryange Date: Tue, 26 Nov 2024 15:41:38 +0800 Subject: [PATCH 001/399] [opt](exec)lazy deserialize pblock in VDataStreamRecvr::SenderQueue (#44378) ### What problem does this PR solve? Previously, for a `pblock` (serialized block), the block would be deserialized immediately after receiving the RPC request and then placed into the `data_queue`. This approach caused significant time consumption during RPC processing due to the deserialization process, impacting overall performance. The new approach defers deserialization until `getBlock` is called. This has the following advantages: 1. Reduces time spent during the RPC handling phase. 2. Memory allocation for deserialization happens within the execution thread, improving cache locality and reducing contention on memory resources. ### Check List (For Author) - Test - [ ] Regression test - [ ] Unit Test - [ ] Manual test (add detailed scripts or steps below) - [x] No need to test or manual test. Explain why: - [x] This is a refactor/code format and no logic has been changed. - [ ] Previous test can cover this change. - [ ] No code files have been changed. - [ ] Other reason - Behavior changed: - [x] No. - [ ] Yes. - Does this need documentation? - [x] No. - [ ] Yes. ### Check List (For Reviewer who merge this PR) - [ ] Confirm the release note - [ ] Confirm test cases - [ ] Confirm document - [ ] Add branch pick label --- be/src/vec/runtime/vdata_stream_mgr.cpp | 11 +++- be/src/vec/runtime/vdata_stream_recvr.cpp | 68 ++++++++++------------- be/src/vec/runtime/vdata_stream_recvr.h | 49 +++++++++++++--- 3 files changed, 76 insertions(+), 52 deletions(-) diff --git a/be/src/vec/runtime/vdata_stream_mgr.cpp b/be/src/vec/runtime/vdata_stream_mgr.cpp index 382a6d0e6e3ece..7dad3d2c86705e 100644 --- a/be/src/vec/runtime/vdata_stream_mgr.cpp +++ b/be/src/vec/runtime/vdata_stream_mgr.cpp @@ -18,10 +18,12 @@ #include "vec/runtime/vdata_stream_mgr.h" #include +#include #include #include #include +#include #include #include #include @@ -141,9 +143,12 @@ Status VDataStreamMgr::transmit_block(const PTransmitDataParams* request, bool eos = request->eos(); if (request->has_block()) { - RETURN_IF_ERROR(recvr->add_block( - request->block(), request->sender_id(), request->be_number(), request->packet_seq(), - eos ? nullptr : done, wait_for_worker, cpu_time_stop_watch.elapsed_time())); + std::unique_ptr pblock_ptr { + const_cast(request)->release_block()}; + RETURN_IF_ERROR(recvr->add_block(std::move(pblock_ptr), request->sender_id(), + request->be_number(), request->packet_seq(), + eos ? nullptr : done, wait_for_worker, + cpu_time_stop_watch.elapsed_time())); } if (eos) { diff --git a/be/src/vec/runtime/vdata_stream_recvr.cpp b/be/src/vec/runtime/vdata_stream_recvr.cpp index b48b9f780b8754..81e4e1cd5f037e 100644 --- a/be/src/vec/runtime/vdata_stream_recvr.cpp +++ b/be/src/vec/runtime/vdata_stream_recvr.cpp @@ -69,7 +69,6 @@ VDataStreamRecvr::SenderQueue::~SenderQueue() { } Status VDataStreamRecvr::SenderQueue::get_batch(Block* block, bool* eos) { - std::lock_guard l(_lock); // protect _block_queue #ifndef NDEBUG if (!_is_cancelled && _block_queue.empty() && _num_remaining_senders > 0) { throw doris::Exception(ErrorCode::INTERNAL_ERROR, @@ -79,25 +78,33 @@ Status VDataStreamRecvr::SenderQueue::get_batch(Block* block, bool* eos) { _debug_string_info()); } #endif - return _inner_get_batch_without_lock(block, eos); -} + BlockItem block_item; + { + std::lock_guard l(_lock); + //check and get block_item from data_queue + if (_is_cancelled) { + RETURN_IF_ERROR(_cancel_status); + return Status::Cancelled("Cancelled"); + } -Status VDataStreamRecvr::SenderQueue::_inner_get_batch_without_lock(Block* block, bool* eos) { - if (_is_cancelled) { - RETURN_IF_ERROR(_cancel_status); - return Status::Cancelled("Cancelled"); - } + if (_block_queue.empty()) { + DCHECK_EQ(_num_remaining_senders, 0); + *eos = true; + return Status::OK(); + } - if (_block_queue.empty()) { - DCHECK_EQ(_num_remaining_senders, 0); - *eos = true; - return Status::OK(); + DCHECK(!_block_queue.empty()); + block_item = std::move(_block_queue.front()); + _block_queue.pop_front(); } - - DCHECK(!_block_queue.empty()); - auto [next_block, block_byte_size] = std::move(_block_queue.front()); - _block_queue.pop_front(); + BlockUPtr next_block; + RETURN_IF_ERROR(block_item.get_block(next_block)); + size_t block_byte_size = block_item.block_byte_size(); + COUNTER_UPDATE(_recvr->_deserialize_row_batch_timer, block_item.deserialize_time()); + COUNTER_UPDATE(_recvr->_decompress_timer, block->get_decompress_time()); + COUNTER_UPDATE(_recvr->_decompress_bytes, block->get_decompressed_bytes()); _recvr->_parent->memory_used_counter()->update(-(int64_t)block_byte_size); + std::lock_guard l(_lock); sub_blocks_memory_usage(block_byte_size); _record_debug_info(); if (_block_queue.empty() && _source_dependency) { @@ -133,7 +140,7 @@ void VDataStreamRecvr::SenderQueue::try_set_dep_ready_without_lock() { } } -Status VDataStreamRecvr::SenderQueue::add_block(const PBlock& pblock, int be_number, +Status VDataStreamRecvr::SenderQueue::add_block(std::unique_ptr pblock, int be_number, int64_t packet_seq, ::google::protobuf::Closure** done, const int64_t wait_for_worker, @@ -163,30 +170,12 @@ Status VDataStreamRecvr::SenderQueue::add_block(const PBlock& pblock, int be_num } } - BlockUPtr block = nullptr; - int64_t deserialize_time = 0; - { - SCOPED_RAW_TIMER(&deserialize_time); - block = Block::create_unique(); - RETURN_IF_ERROR_OR_CATCH_EXCEPTION(block->deserialize(pblock)); - } - - const auto rows = block->rows(); - if (rows == 0) { - return Status::OK(); - } - auto block_byte_size = block->allocated_bytes(); - VLOG_ROW << "added #rows=" << rows << " batch_size=" << block_byte_size << "\n"; - std::lock_guard l(_lock); if (_is_cancelled) { return Status::OK(); } - COUNTER_UPDATE(_recvr->_deserialize_row_batch_timer, deserialize_time); - COUNTER_UPDATE(_recvr->_decompress_timer, block->get_decompress_time()); - COUNTER_UPDATE(_recvr->_decompress_bytes, block->get_decompressed_bytes()); - COUNTER_UPDATE(_recvr->_rows_produced_counter, rows); + const auto block_byte_size = pblock->ByteSizeLong(); COUNTER_UPDATE(_recvr->_blocks_produced_counter, 1); if (_recvr->_max_wait_worker_time->value() < wait_for_worker) { _recvr->_max_wait_worker_time->set(wait_for_worker); @@ -196,7 +185,7 @@ Status VDataStreamRecvr::SenderQueue::add_block(const PBlock& pblock, int be_num _recvr->_max_find_recvr_time->set((int64_t)time_to_find_recvr); } - _block_queue.emplace_back(std::move(block), block_byte_size); + _block_queue.emplace_back(std::move(pblock), block_byte_size); COUNTER_UPDATE(_recvr->_remote_bytes_received_counter, block_byte_size); _record_debug_info(); try_set_dep_ready_without_lock(); @@ -370,7 +359,6 @@ VDataStreamRecvr::VDataStreamRecvr(VDataStreamMgr* stream_mgr, pipeline::Exchang _first_batch_wait_total_timer = ADD_TIMER(_profile, "FirstBatchArrivalWaitTime"); _decompress_timer = ADD_TIMER(_profile, "DecompressTime"); _decompress_bytes = ADD_COUNTER(_profile, "DecompressBytes", TUnit::BYTES); - _rows_produced_counter = ADD_COUNTER(_profile, "RowsProduced", TUnit::UNIT); _blocks_produced_counter = ADD_COUNTER(_profile, "BlocksProduced", TUnit::UNIT); _max_wait_worker_time = ADD_COUNTER(_profile, "MaxWaitForWorkerTime", TUnit::UNIT); _max_wait_to_process_time = ADD_COUNTER(_profile, "MaxWaitToProcessTime", TUnit::UNIT); @@ -401,13 +389,13 @@ Status VDataStreamRecvr::create_merger(const VExprContextSPtrs& ordering_expr, return Status::OK(); } -Status VDataStreamRecvr::add_block(const PBlock& pblock, int sender_id, int be_number, +Status VDataStreamRecvr::add_block(std::unique_ptr pblock, int sender_id, int be_number, int64_t packet_seq, ::google::protobuf::Closure** done, const int64_t wait_for_worker, const uint64_t time_to_find_recvr) { SCOPED_ATTACH_TASK(_query_thread_context); int use_sender_id = _is_merging ? sender_id : 0; - return _sender_queues[use_sender_id]->add_block(pblock, be_number, packet_seq, done, + return _sender_queues[use_sender_id]->add_block(std::move(pblock), be_number, packet_seq, done, wait_for_worker, time_to_find_recvr); } diff --git a/be/src/vec/runtime/vdata_stream_recvr.h b/be/src/vec/runtime/vdata_stream_recvr.h index 08fb004f3b1a8e..1639366c8b83d6 100644 --- a/be/src/vec/runtime/vdata_stream_recvr.h +++ b/be/src/vec/runtime/vdata_stream_recvr.h @@ -18,6 +18,7 @@ #pragma once #include +#include #include #include @@ -84,9 +85,9 @@ class VDataStreamRecvr : public HasTaskExecutionCtx { std::vector sender_queues() const { return _sender_queues; } - Status add_block(const PBlock& pblock, int sender_id, int be_number, int64_t packet_seq, - ::google::protobuf::Closure** done, const int64_t wait_for_worker, - const uint64_t time_to_find_recvr); + Status add_block(std::unique_ptr pblock, int sender_id, int be_number, + int64_t packet_seq, ::google::protobuf::Closure** done, + const int64_t wait_for_worker, const uint64_t time_to_find_recvr); void add_block(Block* block, int sender_id, bool use_move); @@ -157,8 +158,6 @@ class VDataStreamRecvr : public HasTaskExecutionCtx { RuntimeProfile::Counter* _decompress_timer = nullptr; RuntimeProfile::Counter* _decompress_bytes = nullptr; - // Number of rows received - RuntimeProfile::Counter* _rows_produced_counter = nullptr; // Number of blocks received RuntimeProfile::Counter* _blocks_produced_counter = nullptr; RuntimeProfile::Counter* _max_wait_worker_time = nullptr; @@ -181,7 +180,7 @@ class VDataStreamRecvr::SenderQueue { Status get_batch(Block* next_block, bool* eos); - Status add_block(const PBlock& pblock, int be_number, int64_t packet_seq, + Status add_block(std::unique_ptr pblock, int be_number, int64_t packet_seq, ::google::protobuf::Closure** done, const int64_t wait_for_worker, const uint64_t time_to_find_recvr); @@ -205,8 +204,6 @@ class VDataStreamRecvr::SenderQueue { protected: friend class pipeline::ExchangeLocalState; - Status _inner_get_batch_without_lock(Block* block, bool* eos); - void try_set_dep_ready_without_lock(); // To record information about several variables in the event of a DCHECK failure. @@ -260,7 +257,41 @@ class VDataStreamRecvr::SenderQueue { Status _cancel_status; int _num_remaining_senders; std::unique_ptr _queue_mem_tracker; - std::list> _block_queue; + + // `BlockItem` is used in `_block_queue` to handle both local and remote exchange blocks. + // For local exchange blocks, `BlockUPtr` is used directly without any modification. + // For remote exchange blocks, the `pblock` is stored in `BlockItem`. + // When `getBlock` is called, the `pblock` is deserialized into a usable block. + struct BlockItem { + Status get_block(BlockUPtr& block) { + if (!_block) { + DCHECK(_pblock); + SCOPED_RAW_TIMER(&_deserialize_time); + _block = Block::create_unique(); + RETURN_IF_ERROR_OR_CATCH_EXCEPTION(_block->deserialize(*_pblock)); + } + block.swap(_block); + _block.reset(); + return Status::OK(); + } + + size_t block_byte_size() const { return _block_byte_size; } + int64_t deserialize_time() const { return _deserialize_time; } + BlockItem() = default; + BlockItem(BlockUPtr&& block, size_t block_byte_size) + : _block(std::move(block)), _block_byte_size(block_byte_size) {} + + BlockItem(std::unique_ptr&& pblock, size_t block_byte_size) + : _block(nullptr), _pblock(std::move(pblock)), _block_byte_size(block_byte_size) {} + + private: + BlockUPtr _block; + std::unique_ptr _pblock; + size_t _block_byte_size = 0; + int64_t _deserialize_time = 0; + }; + + std::list _block_queue; // sender_id std::unordered_set _sender_eos_set; From 3358156cf14531e9a9573bbcc965e608d05cdae7 Mon Sep 17 00:00:00 2001 From: Vallish Pai Date: Tue, 26 Nov 2024 13:29:42 +0530 Subject: [PATCH 002/399] [Enhancement] (nereids)implement showLoadProfileCommand in nereids (#44440) Issue Number: close #42773 --- .../org/apache/doris/nereids/DorisParser.g4 | 2 +- .../nereids/parser/LogicalPlanBuilder.java | 8 +++ .../doris/nereids/trees/plans/PlanType.java | 1 + .../commands/ShowLoadProfileCommand.java | 57 +++++++++++++++++++ .../trees/plans/visitor/CommandVisitor.java | 5 ++ .../show/test_show_commands_nereids.groovy | 2 +- 6 files changed, 73 insertions(+), 2 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowLoadProfileCommand.java diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index 47214c8c271af6..30b924ed878a5c 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -211,6 +211,7 @@ supportedShowStatement | SHOW DELETE ((FROM | IN) database=multipartIdentifier)? #showDelete | SHOW ALL? GRANTS #showGrants | SHOW GRANTS FOR userIdentify #showGrantsForUser + | SHOW LOAD PROFILE loadIdPath=STRING_LITERAL #showLoadProfile | SHOW VIEW (FROM |IN) tableName=multipartIdentifier ((FROM | IN) database=identifier)? #showView @@ -330,7 +331,6 @@ unsupportedShowStatement ((FROM | IN) database=multipartIdentifier)? #showIndex | SHOW TRANSACTION ((FROM | IN) database=multipartIdentifier)? wildWhere? #showTransaction | SHOW QUERY PROFILE queryIdPath=STRING_LITERAL #showQueryProfile - | SHOW LOAD PROFILE loadIdPath=STRING_LITERAL #showLoadProfile | SHOW CACHE HOTSPOT tablePath=STRING_LITERAL #showCacheHotSpot | SHOW ENCRYPTKEYS ((FROM | IN) database=multipartIdentifier)? wildWhere? #showEncryptKeys | SHOW SYNC JOB ((FROM | IN) database=multipartIdentifier)? #showSyncJob diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index 5d31b284e15562..b77c141e2fd695 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -216,6 +216,7 @@ import org.apache.doris.nereids.DorisParser.ShowGrantsContext; import org.apache.doris.nereids.DorisParser.ShowGrantsForUserContext; import org.apache.doris.nereids.DorisParser.ShowLastInsertContext; +import org.apache.doris.nereids.DorisParser.ShowLoadProfileContext; import org.apache.doris.nereids.DorisParser.ShowPartitionIdContext; import org.apache.doris.nereids.DorisParser.ShowPluginsContext; import org.apache.doris.nereids.DorisParser.ShowPrivilegesContext; @@ -483,6 +484,7 @@ import org.apache.doris.nereids.trees.plans.commands.ShowFrontendsCommand; import org.apache.doris.nereids.trees.plans.commands.ShowGrantsCommand; import org.apache.doris.nereids.trees.plans.commands.ShowLastInsertCommand; +import org.apache.doris.nereids.trees.plans.commands.ShowLoadProfileCommand; import org.apache.doris.nereids.trees.plans.commands.ShowPartitionIdCommand; import org.apache.doris.nereids.trees.plans.commands.ShowPluginsCommand; import org.apache.doris.nereids.trees.plans.commands.ShowPrivilegesCommand; @@ -4136,10 +4138,16 @@ public Object visitRefreshCatalog(RefreshCatalogContext ctx) { throw new AnalysisException("catalog name can not be null"); } + @Override public LogicalPlan visitShowLastInsert(ShowLastInsertContext ctx) { return new ShowLastInsertCommand(); } + @Override + public LogicalPlan visitShowLoadProfile(ShowLoadProfileContext ctx) { + return new ShowLoadProfileCommand(ctx.loadIdPath.getText()); + } + @Override public LogicalPlan visitShowGrants(ShowGrantsContext ctx) { boolean all = (ctx.ALL() != null) ? true : false; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index f36af26e89273e..e649a905b4a0c7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -196,6 +196,7 @@ public enum PlanType { SHOW_FRONTENDS_COMMAND, SHOW_GRANTS_COMMAND, SHOW_LAST_INSERT_COMMAND, + SHOW_LOAD_PROFILE_COMMAND, SHOW_PARTITIONID_COMMAND, SHOW_PROC_COMMAND, SHOW_PLUGINS_COMMAND, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowLoadProfileCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowLoadProfileCommand.java new file mode 100644 index 00000000000000..5320671ad7794d --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowLoadProfileCommand.java @@ -0,0 +1,57 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.commands; + +import org.apache.doris.catalog.Env; +import org.apache.doris.common.Config; +import org.apache.doris.common.UserException; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.ShowResultSet; +import org.apache.doris.qe.StmtExecutor; + +/** + * show load profile command + */ +public class ShowLoadProfileCommand extends ShowCommand { + private String loadIdPath; + + /** + * constructor + */ + public ShowLoadProfileCommand(String path) { + super(PlanType.SHOW_LOAD_PROFILE_COMMAND); + this.loadIdPath = path; + } + + @Override + public ShowResultSet doRun(ConnectContext ctx, StmtExecutor executor) throws Exception { + String selfHost = Env.getCurrentEnv().getSelfNode().getHost(); + int httpPort = Config.http_port; + String terminalMsg = String.format( + "try visit http://%s:%d/QueryProfile/%s, show query/load profile syntax is a deprecated feature", + selfHost, httpPort, this.loadIdPath); + throw new UserException(terminalMsg); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitShowLoadProfileCommand(this, context); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index 4383ebc01a0b9e..1c2a49b1b4e757 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -75,6 +75,7 @@ import org.apache.doris.nereids.trees.plans.commands.ShowFrontendsCommand; import org.apache.doris.nereids.trees.plans.commands.ShowGrantsCommand; import org.apache.doris.nereids.trees.plans.commands.ShowLastInsertCommand; +import org.apache.doris.nereids.trees.plans.commands.ShowLoadProfileCommand; import org.apache.doris.nereids.trees.plans.commands.ShowPartitionIdCommand; import org.apache.doris.nereids.trees.plans.commands.ShowPluginsCommand; import org.apache.doris.nereids.trees.plans.commands.ShowPrivilegesCommand; @@ -402,6 +403,10 @@ default R visitShowBrokerCommand(ShowBrokerCommand showBrokerCommand, C context) return visitCommand(showBrokerCommand, context); } + default R visitShowLoadProfileCommand(ShowLoadProfileCommand showLoadProfileCommand, C context) { + return visitCommand(showLoadProfileCommand, context); + } + default R visitDropRoleCommand(DropRoleCommand dropRoleCommand, C context) { return visitCommand(dropRoleCommand, context); } diff --git a/regression-test/suites/nereids_p0/show/test_show_commands_nereids.groovy b/regression-test/suites/nereids_p0/show/test_show_commands_nereids.groovy index e994e22ec99709..998033deff9863 100644 --- a/regression-test/suites/nereids_p0/show/test_show_commands_nereids.groovy +++ b/regression-test/suites/nereids_p0/show/test_show_commands_nereids.groovy @@ -26,5 +26,5 @@ suite("test_show_commands_nereids") { checkNereidsExecute("""show backends;""") checkNereidsExecute("""show whitelist;""") checkNereidsExecute("""show triggers;""") - + checkNereidsExecute("""show load profile \"\\";""") } From 8b9809e1c02c942f6fffa52e18bbb8b35ee21829 Mon Sep 17 00:00:00 2001 From: Vallish Pai Date: Tue, 26 Nov 2024 13:31:39 +0530 Subject: [PATCH 003/399] [Enhancement] (nereids)implement showSmallFilesCommand in nereids (#44453) Issue Number: close #42770 --- .../org/apache/doris/nereids/DorisParser.g4 | 4 +- .../nereids/parser/LogicalPlanBuilder.java | 12 +++ .../doris/nereids/trees/plans/PlanType.java | 1 + .../plans/commands/ShowSmallFilesCommand.java | 91 +++++++++++++++++++ .../trees/plans/visitor/CommandVisitor.java | 5 + .../auth_call/test_ddl_file_auth.groovy | 2 + 6 files changed, 113 insertions(+), 2 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowSmallFilesCommand.java diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index 30b924ed878a5c..f4394fa02c60f3 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -221,7 +221,8 @@ supportedShowStatement | SHOW ROLES #showRoles | SHOW PARTITION partitionId=INTEGER_VALUE #showPartitionId | SHOW PRIVILEGES #showPrivileges - | SHOW PROC path=STRING_LITERAL #showProc + | SHOW PROC path=STRING_LITERAL #showProc + | SHOW FILE ((FROM | IN) database=multipartIdentifier)? #showSmallFiles | SHOW STORAGE? ENGINES #showStorageEngines | SHOW CREATE CATALOG name=identifier #showCreateCatalog | SHOW SQL_BLOCK_RULE (FOR ruleName=identifier)? #showSqlBlockRule @@ -325,7 +326,6 @@ unsupportedShowStatement ((FROM | IN) database=multipartIdentifier)? wildWhere? #showFunctions | SHOW GLOBAL FULL? FUNCTIONS wildWhere? #showGlobalFunctions | SHOW TYPECAST ((FROM | IN) database=multipartIdentifier)? #showTypeCast - | SHOW FILE ((FROM | IN) database=multipartIdentifier)? #showSmallFiles | SHOW (KEY | KEYS | INDEX | INDEXES) (FROM |IN) tableName=multipartIdentifier ((FROM | IN) database=multipartIdentifier)? #showIndex diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index b77c141e2fd695..ab4ecb44fd4425 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -224,6 +224,7 @@ import org.apache.doris.nereids.DorisParser.ShowProcedureStatusContext; import org.apache.doris.nereids.DorisParser.ShowRepositoriesContext; import org.apache.doris.nereids.DorisParser.ShowRolesContext; +import org.apache.doris.nereids.DorisParser.ShowSmallFilesContext; import org.apache.doris.nereids.DorisParser.ShowSqlBlockRuleContext; import org.apache.doris.nereids.DorisParser.ShowStorageEnginesContext; import org.apache.doris.nereids.DorisParser.ShowTableIdContext; @@ -492,6 +493,7 @@ import org.apache.doris.nereids.trees.plans.commands.ShowProcedureStatusCommand; import org.apache.doris.nereids.trees.plans.commands.ShowRepositoriesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowRolesCommand; +import org.apache.doris.nereids.trees.plans.commands.ShowSmallFilesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowSqlBlockRuleCommand; import org.apache.doris.nereids.trees.plans.commands.ShowStorageEnginesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowTableIdCommand; @@ -4232,6 +4234,16 @@ public LogicalPlan visitShowPlugins(ShowPluginsContext ctx) { return new ShowPluginsCommand(); } + @Override + public LogicalPlan visitShowSmallFiles(ShowSmallFilesContext ctx) { + String dbName = null; + if (ctx.database != null) { + List nameParts = visitMultipartIdentifier(ctx.database); + dbName = nameParts.get(0); // only one entry possible + } + return new ShowSmallFilesCommand(dbName); + } + @Override public LogicalPlan visitShowSqlBlockRule(ShowSqlBlockRuleContext ctx) { String ruleName = null; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index e649a905b4a0c7..8698fd9855a9de 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -203,6 +203,7 @@ public enum PlanType { SHOW_PRIVILEGES_COMMAND, SHOW_REPOSITORIES_COMMAND, SHOW_ROLE_COMMAND, + SHOW_SMALL_FILES_COMMAND, SHOW_STORAGE_ENGINES_COMMAND, SHOW_TABLE_ID_COMMAND, SHOW_TRIGGERS_COMMAND, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowSmallFilesCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowSmallFilesCommand.java new file mode 100644 index 00000000000000..608fc8d4eff59d --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowSmallFilesCommand.java @@ -0,0 +1,91 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.commands; + +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.ScalarType; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.DdlException; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.datasource.InternalCatalog; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.ShowResultSet; +import org.apache.doris.qe.ShowResultSetMetaData; +import org.apache.doris.qe.StmtExecutor; + +import com.google.common.base.Strings; + +import java.util.List; + +/** + * show small file command + */ +public class ShowSmallFilesCommand extends ShowCommand { + private static final ShowResultSetMetaData META_DATA = + ShowResultSetMetaData.builder() + .addColumn(new Column("Id", ScalarType.createVarchar(32))) + .addColumn(new Column("DbName", ScalarType.createVarchar(256))) + .addColumn(new Column("Catalog", ScalarType.createVarchar(32))) + .addColumn(new Column("FileName", ScalarType.createVarchar(16))) + .addColumn(new Column("FileSize", ScalarType.createVarchar(16))) + .addColumn(new Column("IsContent", ScalarType.createVarchar(16))) + .addColumn(new Column("MD5", ScalarType.createVarchar(16))) + .build(); + private String dbName; // if empty we will use current db; + + /** + * constructor + */ + public ShowSmallFilesCommand(String dbName) { + super(PlanType.SHOW_SMALL_FILES_COMMAND); + this.dbName = dbName; + } + + @Override + public ShowResultSet doRun(ConnectContext ctx, StmtExecutor executor) throws Exception { + if (Strings.isNullOrEmpty(dbName)) { + dbName = ctx.getDatabase(); + if (Strings.isNullOrEmpty(dbName)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_NO_DB_ERROR); + } + } + if (!Env.getCurrentEnv().getAccessManager() + .checkDbPriv(ConnectContext.get(), InternalCatalog.INTERNAL_CATALOG_NAME, dbName, PrivPredicate.SHOW)) { + ErrorReport.reportAnalysisException( + ErrorCode.ERR_DBACCESS_DENIED_ERROR, ConnectContext.get().getQualifiedUser(), dbName); + } + + List> results; + try { + results = Env.getCurrentEnv().getSmallFileMgr().getInfo(dbName); + } catch (DdlException e) { + throw new AnalysisException(e.getMessage()); + } + return new ShowResultSet(META_DATA, results); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitShowSmallFilesCommand(this, context); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index 1c2a49b1b4e757..2e7108613c2297 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -83,6 +83,7 @@ import org.apache.doris.nereids.trees.plans.commands.ShowProcedureStatusCommand; import org.apache.doris.nereids.trees.plans.commands.ShowRepositoriesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowRolesCommand; +import org.apache.doris.nereids.trees.plans.commands.ShowSmallFilesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowSqlBlockRuleCommand; import org.apache.doris.nereids.trees.plans.commands.ShowStorageEnginesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowTableIdCommand; @@ -330,6 +331,10 @@ default R visitShowCreateTableCommand(ShowCreateTableCommand showCreateTableComm return visitCommand(showCreateTableCommand, context); } + default R visitShowSmallFilesCommand(ShowSmallFilesCommand showSmallFilesCommand, C context) { + return visitCommand(showSmallFilesCommand, context); + } + default R visitShowSqlBlockRuleCommand(ShowSqlBlockRuleCommand showblockruleCommand, C context) { return visitCommand(showblockruleCommand, context); } diff --git a/regression-test/suites/auth_call/test_ddl_file_auth.groovy b/regression-test/suites/auth_call/test_ddl_file_auth.groovy index 77ca5e6703f977..99b71eabddf140 100644 --- a/regression-test/suites/auth_call/test_ddl_file_auth.groovy +++ b/regression-test/suites/auth_call/test_ddl_file_auth.groovy @@ -76,6 +76,8 @@ suite("test_ddl_file_auth","p0,auth_call") { "catalog" = "internal" );""" sql """use ${dbName}""" + checkNereidsExecute("SHOW FILE;") + checkNereidsExecute("SHOW FILE FROM ${dbName};") def res = sql """SHOW FILE FROM ${dbName};""" assertTrue(res.size() == 1) From 78717e32bca0b9220f8c8e8dd86416ebd916daeb Mon Sep 17 00:00:00 2001 From: Vallish Pai Date: Tue, 26 Nov 2024 13:32:44 +0530 Subject: [PATCH 004/399] [Enhancement] (nereids)implement DropWorkloadGroupCommand in nereids (#44482) Issue Number: close #42621 --- .../org/apache/doris/nereids/DorisParser.g4 | 2 +- .../nereids/parser/LogicalPlanBuilder.java | 7 +++ .../doris/nereids/trees/plans/PlanType.java | 1 + .../commands/DropWorkloadGroupCommand.java | 58 +++++++++++++++++++ .../trees/plans/visitor/CommandVisitor.java | 5 ++ .../workloadgroup/WorkloadGroupMgr.java | 7 ++- .../test_nereids_workload_test.out | 6 ++ .../test_nereids_workload_test.groovy | 27 +++++++++ 8 files changed, 110 insertions(+), 3 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropWorkloadGroupCommand.java create mode 100644 regression-test/data/workload_manager_p0/test_nereids_workload_test.out create mode 100644 regression-test/suites/workload_manager_p0/test_nereids_workload_test.groovy diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index f4394fa02c60f3..c5a3b7d6efb0e8 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -200,6 +200,7 @@ supportedDropStatement | DROP ROLE (IF EXISTS)? name=identifier #dropRole | DROP SQL_BLOCK_RULE (IF EXISTS)? identifierSeq #dropSqlBlockRule | DROP USER (IF EXISTS)? userIdentify #dropUser + | DROP WORKLOAD GROUP (IF EXISTS)? name=identifierOrText #dropWorkloadGroup ; supportedShowStatement @@ -671,7 +672,6 @@ unsupportedDropStatement ((FROM | IN) database=identifier)? properties=propertyClause #dropFile | DROP INDEX (IF EXISTS)? name=identifier ON tableName=multipartIdentifier #dropIndex | DROP RESOURCE (IF EXISTS)? name=identifierOrText #dropResource - | DROP WORKLOAD GROUP (IF EXISTS)? name=identifierOrText #dropWorkloadGroup | DROP WORKLOAD POLICY (IF EXISTS)? name=identifierOrText #dropWorkloadPolicy | DROP ENCRYPTKEY (IF EXISTS)? name=multipartIdentifier #dropEncryptkey | DROP ROW POLICY (IF EXISTS)? policyName=identifier diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index ab4ecb44fd4425..537ff71320b1e1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -104,6 +104,7 @@ import org.apache.doris.nereids.DorisParser.DropRoleContext; import org.apache.doris.nereids.DorisParser.DropSqlBlockRuleContext; import org.apache.doris.nereids.DorisParser.DropUserContext; +import org.apache.doris.nereids.DorisParser.DropWorkloadGroupContext; import org.apache.doris.nereids.DorisParser.ElementAtContext; import org.apache.doris.nereids.DorisParser.ExceptContext; import org.apache.doris.nereids.DorisParser.ExceptOrReplaceContext; @@ -453,6 +454,7 @@ import org.apache.doris.nereids.trees.plans.commands.DropRoleCommand; import org.apache.doris.nereids.trees.plans.commands.DropSqlBlockRuleCommand; import org.apache.doris.nereids.trees.plans.commands.DropUserCommand; +import org.apache.doris.nereids.trees.plans.commands.DropWorkloadGroupCommand; import org.apache.doris.nereids.trees.plans.commands.ExplainCommand; import org.apache.doris.nereids.trees.plans.commands.ExplainCommand.ExplainLevel; import org.apache.doris.nereids.trees.plans.commands.ExportCommand; @@ -4374,6 +4376,11 @@ public LogicalPlan visitDropUser(DropUserContext ctx) { return new DropUserCommand(userIdent, ctx.EXISTS() != null); } + @Override + public LogicalPlan visitDropWorkloadGroup(DropWorkloadGroupContext ctx) { + return new DropWorkloadGroupCommand(ctx.name.getText(), ctx.EXISTS() != null); + } + @Override public LogicalPlan visitShowTableId(ShowTableIdContext ctx) { long tableId = -1; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index 8698fd9855a9de..8ce5caac661c92 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -184,6 +184,7 @@ public enum PlanType { EXECUTE_COMMAND, DROP_SQL_BLOCK_RULE_COMMAND, DROP_USER_COMMAND, + DROP_WORKLOAD_GROUP_NAME, SHOW_BACKENDS_COMMAND, SHOW_BLOCK_RULE_COMMAND, SHOW_BROKER_COMMAND, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropWorkloadGroupCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropWorkloadGroupCommand.java new file mode 100644 index 00000000000000..694fc1411db127 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropWorkloadGroupCommand.java @@ -0,0 +1,58 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.commands; + +import org.apache.doris.catalog.Env; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.StmtExecutor; + +/** + * drop workload group command + */ +public class DropWorkloadGroupCommand extends DropCommand { + private final boolean ifExists; + private final String workloadGroupName; + + /** + * constructor + */ + public DropWorkloadGroupCommand(String workloadGroupName, boolean ifExists) { + super(PlanType.DROP_WORKLOAD_GROUP_NAME); + this.workloadGroupName = workloadGroupName; + this.ifExists = ifExists; + } + + @Override + public void doRun(ConnectContext ctx, StmtExecutor executor) throws Exception { + // check auth + if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, "ADMIN"); + } + Env.getCurrentEnv().getWorkloadGroupMgr().dropWorkloadGroup(workloadGroupName, ifExists); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitDropWorkloadGroupCommand(this, context); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index 2e7108613c2297..9e451fac4f1371 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -44,6 +44,7 @@ import org.apache.doris.nereids.trees.plans.commands.DropRoleCommand; import org.apache.doris.nereids.trees.plans.commands.DropSqlBlockRuleCommand; import org.apache.doris.nereids.trees.plans.commands.DropUserCommand; +import org.apache.doris.nereids.trees.plans.commands.DropWorkloadGroupCommand; import org.apache.doris.nereids.trees.plans.commands.ExplainCommand; import org.apache.doris.nereids.trees.plans.commands.ExportCommand; import org.apache.doris.nereids.trees.plans.commands.LoadCommand; @@ -424,6 +425,10 @@ default R visitDropUserCommand(DropUserCommand dropUserCommand, C context) { return visitCommand(dropUserCommand, context); } + default R visitDropWorkloadGroupCommand(DropWorkloadGroupCommand dropWorkloadGroupCommand, C context) { + return visitCommand(dropWorkloadGroupCommand, context); + } + default R visitShowTableIdCommand(ShowTableIdCommand showTableIdCommand, C context) { return visitCommand(showTableIdCommand, context); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/resource/workloadgroup/WorkloadGroupMgr.java b/fe/fe-core/src/main/java/org/apache/doris/resource/workloadgroup/WorkloadGroupMgr.java index 5e61f48bec6f2e..b4c25524c3a8fa 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/resource/workloadgroup/WorkloadGroupMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/resource/workloadgroup/WorkloadGroupMgr.java @@ -492,7 +492,10 @@ public void alterWorkloadGroup(AlterWorkloadGroupStmt stmt) throws DdlException } public void dropWorkloadGroup(DropWorkloadGroupStmt stmt) throws DdlException { - String workloadGroupName = stmt.getWorkloadGroupName(); + dropWorkloadGroup(stmt.getWorkloadGroupName(), stmt.isIfExists()); + } + + public void dropWorkloadGroup(String workloadGroupName, boolean ifExists) throws DdlException { if (DEFAULT_GROUP_NAME.equals(workloadGroupName) || INTERNAL_GROUP_NAME.equals(workloadGroupName)) { throw new DdlException("Dropping workload group " + workloadGroupName + " is not allowed"); } @@ -521,7 +524,7 @@ public void dropWorkloadGroup(DropWorkloadGroupStmt stmt) throws DdlException { writeLock(); try { if (!nameToWorkloadGroup.containsKey(workloadGroupName)) { - if (stmt.isIfExists()) { + if (ifExists) { return; } throw new DdlException("workload group " + workloadGroupName + " does not exist"); diff --git a/regression-test/data/workload_manager_p0/test_nereids_workload_test.out b/regression-test/data/workload_manager_p0/test_nereids_workload_test.out new file mode 100644 index 00000000000000..c188c28c35c3d4 --- /dev/null +++ b/regression-test/data/workload_manager_p0/test_nereids_workload_test.out @@ -0,0 +1,6 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !check_workload_check1 -- +test_nereids_wg1 + +-- !check_workload_check2 -- + diff --git a/regression-test/suites/workload_manager_p0/test_nereids_workload_test.groovy b/regression-test/suites/workload_manager_p0/test_nereids_workload_test.groovy new file mode 100644 index 00000000000000..3f7c55f5c99cf9 --- /dev/null +++ b/regression-test/suites/workload_manager_p0/test_nereids_workload_test.groovy @@ -0,0 +1,27 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("test_nereids_workload_test") { + sql "drop workload group if exists test_nereids_wg1;" + sql "drop workload group if exists test_nereids_wg2;" + sql "create workload group test_nereids_wg1 properties('cpu_share'='1024');" + sql "create workload group test_nereids_wg2 properties('cpu_share'='1024');" + qt_check_workload_check1("select NAME from information_schema.workload_groups where NAME='test_nereids_wg1';") + checkNereidsExecute("drop workload group test_nereids_wg1;") + qt_check_workload_check2("select NAME from information_schema.workload_groups where NAME='test_nereids_wg1';") + checkNereidsExecute("drop workload group if exists test_nereids_wg2;") +} \ No newline at end of file From 1b8f7624e7a6a9c12c549157880ed81f698aa55d Mon Sep 17 00:00:00 2001 From: Vallish Pai Date: Tue, 26 Nov 2024 13:34:15 +0530 Subject: [PATCH 005/399] [Enhancement] (nereids)implement alterWorkloadGroupCommand in nereids (#44500) Issue Number: close #42793 --- .../org/apache/doris/nereids/DorisParser.g4 | 4 +- .../nereids/parser/LogicalPlanBuilder.java | 10 +++ .../doris/nereids/trees/plans/PlanType.java | 1 + .../commands/AlterWorkloadGroupCommand.java | 81 +++++++++++++++++++ .../trees/plans/visitor/CommandVisitor.java | 5 ++ .../workloadgroup/WorkloadGroupMgr.java | 6 +- .../test_nereids_workload_alter_test.out | 7 ++ .../test_nereids_workload_alter_test.groovy | 25 ++++++ 8 files changed, 135 insertions(+), 4 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterWorkloadGroupCommand.java create mode 100644 regression-test/data/workload_manager_p0/test_nereids_workload_alter_test.out create mode 100644 regression-test/suites/workload_manager_p0/test_nereids_workload_alter_test.groovy diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index c5a3b7d6efb0e8..c662ee295b6f49 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -193,6 +193,8 @@ supportedAlterStatement AS query #alterView | ALTER STORAGE VAULT name=multipartIdentifier properties=propertyClause #alterStorageVault | ALTER ROLE role=identifier commentSpec #alterRole + | ALTER WORKLOAD GROUP name=identifierOrText + properties=propertyClause? #alterWorkloadGroup ; supportedDropStatement @@ -558,8 +560,6 @@ unsupportedAlterStatement | ALTER RESOURCE name=identifierOrText properties=propertyClause? #alterResource | ALTER COLOCATE GROUP name=multipartIdentifier SET LEFT_PAREN propertyItemList RIGHT_PAREN #alterColocateGroup - | ALTER WORKLOAD GROUP name=identifierOrText - properties=propertyClause? #alterWorkloadGroup | ALTER WORKLOAD POLICY name=identifierOrText properties=propertyClause? #alterWorkloadPolicy | ALTER ROUTINE LOAD FOR name=multipartIdentifier properties=propertyClause? diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index 537ff71320b1e1..dc0fe2aa4a10d6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -56,6 +56,7 @@ import org.apache.doris.nereids.DorisParser.AlterRoleContext; import org.apache.doris.nereids.DorisParser.AlterStorageVaultContext; import org.apache.doris.nereids.DorisParser.AlterViewContext; +import org.apache.doris.nereids.DorisParser.AlterWorkloadGroupContext; import org.apache.doris.nereids.DorisParser.ArithmeticBinaryContext; import org.apache.doris.nereids.DorisParser.ArithmeticUnaryContext; import org.apache.doris.nereids.DorisParser.ArrayLiteralContext; @@ -430,6 +431,7 @@ import org.apache.doris.nereids.trees.plans.commands.AlterRoleCommand; import org.apache.doris.nereids.trees.plans.commands.AlterStorageVaultCommand; import org.apache.doris.nereids.trees.plans.commands.AlterViewCommand; +import org.apache.doris.nereids.trees.plans.commands.AlterWorkloadGroupCommand; import org.apache.doris.nereids.trees.plans.commands.CallCommand; import org.apache.doris.nereids.trees.plans.commands.CancelJobTaskCommand; import org.apache.doris.nereids.trees.plans.commands.CancelMTMVTaskCommand; @@ -4298,12 +4300,20 @@ public LogicalPlan visitShowCreateMaterializedView(ShowCreateMaterializedViewCon return new ShowCreateMaterializedViewCommand(stripQuotes(ctx.mvName.getText()), new TableNameInfo(nameParts)); } + @Override + public LogicalPlan visitAlterWorkloadGroup(AlterWorkloadGroupContext ctx) { + Map properties = ctx.propertyClause() != null + ? Maps.newHashMap(visitPropertyClause(ctx.propertyClause())) : Maps.newHashMap(); + return new AlterWorkloadGroupCommand(ctx.name.getText(), properties); + } + @Override public LogicalPlan visitAlterRole(AlterRoleContext ctx) { String comment = visitCommentSpec(ctx.commentSpec()); return new AlterRoleCommand(ctx.role.getText(), comment); } + @Override public LogicalPlan visitShowFrontends(ShowFrontendsContext ctx) { String detail = (ctx.name != null) ? ctx.name.getText() : null; return new ShowFrontendsCommand(detail); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index 8ce5caac661c92..04551da904bfa9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -170,6 +170,7 @@ public enum PlanType { ALTER_ROLE_COMMAND, ALTER_VIEW_COMMAND, ALTER_STORAGE_VAULT, + ALTER_WORKLOAD_GROUP_COMMAND, DROP_CATALOG_RECYCLE_BIN_COMMAND, UNSET_VARIABLE_COMMAND, UNSET_DEFAULT_STORAGE_VAULT_COMMAND, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterWorkloadGroupCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterWorkloadGroupCommand.java new file mode 100644 index 00000000000000..8cceaa20fdce9a --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterWorkloadGroupCommand.java @@ -0,0 +1,81 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.commands; + +import org.apache.doris.catalog.Env; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.StmtExecutor; +import org.apache.doris.resource.workloadgroup.WorkloadGroup; +import org.apache.doris.resource.workloadgroup.WorkloadGroupMgr; + +import org.apache.commons.lang3.StringUtils; + +import java.util.Map; + +/** + * alter workload group command + */ +public class AlterWorkloadGroupCommand extends AlterCommand { + private final String workloadGroupName; + private final Map properties; + + /** + * constructor + */ + public AlterWorkloadGroupCommand(String workloadGroupName, Map properties) { + super(PlanType.ALTER_WORKLOAD_GROUP_COMMAND); + this.workloadGroupName = workloadGroupName; + this.properties = properties; + } + + @Override + public void doRun(ConnectContext ctx, StmtExecutor executor) throws Exception { + // check auth + if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, "ADMIN"); + } + + if (properties == null || properties.isEmpty()) { + throw new AnalysisException("Workload Group properties can't be empty"); + } + + if (properties.containsKey(WorkloadGroup.INTERNAL_TYPE)) { + throw new AnalysisException(WorkloadGroup.INTERNAL_TYPE + " can not be create or modified "); + } + String tagStr = properties.get(WorkloadGroup.TAG); + if (!StringUtils.isEmpty(tagStr) && (WorkloadGroupMgr.DEFAULT_GROUP_NAME.equals(workloadGroupName) + || WorkloadGroupMgr.INTERNAL_GROUP_NAME.equals(workloadGroupName))) { + throw new AnalysisException( + WorkloadGroupMgr.INTERNAL_GROUP_NAME + " and " + WorkloadGroupMgr.DEFAULT_GROUP_NAME + + " group can not set tag"); + } + + Env.getCurrentEnv().getWorkloadGroupMgr().alterWorkloadGroup(workloadGroupName, properties); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitAlterWorkloadGroupCommand(this, context); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index 9e451fac4f1371..13eeef182641fb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -22,6 +22,7 @@ import org.apache.doris.nereids.trees.plans.commands.AlterMTMVCommand; import org.apache.doris.nereids.trees.plans.commands.AlterRoleCommand; import org.apache.doris.nereids.trees.plans.commands.AlterViewCommand; +import org.apache.doris.nereids.trees.plans.commands.AlterWorkloadGroupCommand; import org.apache.doris.nereids.trees.plans.commands.CallCommand; import org.apache.doris.nereids.trees.plans.commands.CancelJobTaskCommand; import org.apache.doris.nereids.trees.plans.commands.CancelMTMVTaskCommand; @@ -377,6 +378,10 @@ default R visitAlterRoleCommand(AlterRoleCommand alterRoleCommand, C context) { return visitCommand(alterRoleCommand, context); } + default R visitAlterWorkloadGroupCommand(AlterWorkloadGroupCommand alterWorkloadGroupCommand, C context) { + return visitCommand(alterWorkloadGroupCommand, context); + } + default R visitCleanAllProfileCommand(CleanAllProfileCommand cleanAllProfileCommand, C context) { return visitCommand(cleanAllProfileCommand, context); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/resource/workloadgroup/WorkloadGroupMgr.java b/fe/fe-core/src/main/java/org/apache/doris/resource/workloadgroup/WorkloadGroupMgr.java index b4c25524c3a8fa..22cbcf4a0a1502 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/resource/workloadgroup/WorkloadGroupMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/resource/workloadgroup/WorkloadGroupMgr.java @@ -464,8 +464,10 @@ private void checkGlobalUnlock(WorkloadGroup newWg, WorkloadGroup oldWg) throws } public void alterWorkloadGroup(AlterWorkloadGroupStmt stmt) throws DdlException { - String workloadGroupName = stmt.getWorkloadGroupName(); - Map properties = stmt.getProperties(); + alterWorkloadGroup(stmt.getWorkloadGroupName(), stmt.getProperties()); + } + + public void alterWorkloadGroup(String workloadGroupName, Map properties) throws DdlException { if (properties.size() == 0) { throw new DdlException("alter workload group should contain at least one property"); } diff --git a/regression-test/data/workload_manager_p0/test_nereids_workload_alter_test.out b/regression-test/data/workload_manager_p0/test_nereids_workload_alter_test.out new file mode 100644 index 00000000000000..897cef4c230a51 --- /dev/null +++ b/regression-test/data/workload_manager_p0/test_nereids_workload_alter_test.out @@ -0,0 +1,7 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !check_workload_check1 -- +1024 10 + +-- !check_workload_check2 -- +20 8 + diff --git a/regression-test/suites/workload_manager_p0/test_nereids_workload_alter_test.groovy b/regression-test/suites/workload_manager_p0/test_nereids_workload_alter_test.groovy new file mode 100644 index 00000000000000..b3f600b3f186b4 --- /dev/null +++ b/regression-test/suites/workload_manager_p0/test_nereids_workload_alter_test.groovy @@ -0,0 +1,25 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("test_nereids_workload_test") { + sql "drop workload group if exists test_nereids_alter_wg1;" + sql "create workload group test_nereids_alter_wg1 properties('cpu_share'='1024', 'scan_thread_num'='10');" + qt_check_workload_check1("select CPU_SHARE,scan_thread_num from information_schema.workload_groups where NAME='test_nereids_alter_wg1';") + checkNereidsExecute("Alter workload group test_nereids_alter_wg1 properties('cpu_share'='20', 'scan_thread_num'='8');") + qt_check_workload_check2("select CPU_SHARE,scan_thread_num from information_schema.workload_groups where NAME='test_nereids_alter_wg1';") + checkNereidsExecute("drop workload group if exists test_nereids_alter_wg1;") +} \ No newline at end of file From 95fdd45600b4357a0ea8b6a0d1c0ac6182dd0500 Mon Sep 17 00:00:00 2001 From: Vallish Pai Date: Tue, 26 Nov 2024 13:35:46 +0530 Subject: [PATCH 006/399] [Enhancement] (nereids)implement DropEncryptKeyCommand in nereids (#44506) Issue Number: close #42623 --- .../org/apache/doris/nereids/DorisParser.g4 | 2 +- .../apache/doris/analysis/EncryptKeyName.java | 25 +++++++ .../nereids/parser/LogicalPlanBuilder.java | 9 +++ .../doris/nereids/trees/plans/PlanType.java | 1 + .../plans/commands/DropEncryptkeyCommand.java | 65 +++++++++++++++++++ .../trees/plans/visitor/CommandVisitor.java | 5 ++ .../nereids_p0/test_nereids_encrypt_test.out | 8 +++ .../test_nereids_encrypt_test.groovy | 29 +++++++++ 8 files changed, 143 insertions(+), 1 deletion(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropEncryptkeyCommand.java create mode 100644 regression-test/data/nereids_p0/test_nereids_encrypt_test.out create mode 100644 regression-test/suites/nereids_p0/test_nereids_encrypt_test.groovy diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index c662ee295b6f49..932f644fca4e7a 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -199,6 +199,7 @@ supportedAlterStatement supportedDropStatement : DROP CATALOG RECYCLE BIN WHERE idType=STRING_LITERAL EQ id=INTEGER_VALUE #dropCatalogRecycleBin + | DROP ENCRYPTKEY (IF EXISTS)? name=multipartIdentifier #dropEncryptkey | DROP ROLE (IF EXISTS)? name=identifier #dropRole | DROP SQL_BLOCK_RULE (IF EXISTS)? identifierSeq #dropSqlBlockRule | DROP USER (IF EXISTS)? userIdentify #dropUser @@ -673,7 +674,6 @@ unsupportedDropStatement | DROP INDEX (IF EXISTS)? name=identifier ON tableName=multipartIdentifier #dropIndex | DROP RESOURCE (IF EXISTS)? name=identifierOrText #dropResource | DROP WORKLOAD POLICY (IF EXISTS)? name=identifierOrText #dropWorkloadPolicy - | DROP ENCRYPTKEY (IF EXISTS)? name=multipartIdentifier #dropEncryptkey | DROP ROW POLICY (IF EXISTS)? policyName=identifier ON tableName=multipartIdentifier (FOR (userIdentify | ROLE roleName=identifier))? #dropRowPolicy diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/EncryptKeyName.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/EncryptKeyName.java index 155f1753bf4a34..e76bfb7e5b20a2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/EncryptKeyName.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/EncryptKeyName.java @@ -25,6 +25,7 @@ import org.apache.doris.common.io.Text; import org.apache.doris.common.io.Writable; import org.apache.doris.persist.gson.GsonUtils; +import org.apache.doris.qe.ConnectContext; import com.google.common.base.Strings; import com.google.gson.annotations.SerializedName; @@ -34,6 +35,7 @@ import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; +import java.util.List; import java.util.Objects; public class EncryptKeyName implements Writable { @@ -52,6 +54,19 @@ public EncryptKeyName(String db, String keyName) { } } + /** + * EncryptKeyName + * @param parts like [db1,keyName] or [keyName] + */ + public EncryptKeyName(List parts) { + int size = parts.size(); + keyName = parts.get(size - 1); + keyName = keyName.toLowerCase(); + if (size >= 2) { + db = parts.get(size - 2); + } + } + public EncryptKeyName(String keyName) { this.db = null; this.keyName = keyName.toLowerCase(); @@ -67,6 +82,16 @@ public void analyze(Analyzer analyzer) throws AnalysisException { } } + public void analyze(ConnectContext ctx) throws AnalysisException { + FeNameFormat.checkCommonName("EncryptKey", keyName); + if (db == null) { + db = ctx.getDatabase(); + if (Strings.isNullOrEmpty(db)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_NO_DB_ERROR); + } + } + } + public String getDb() { return db; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index dc0fe2aa4a10d6..320043e12b8b53 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -20,6 +20,7 @@ import org.apache.doris.analysis.ArithmeticExpr.Operator; import org.apache.doris.analysis.BrokerDesc; import org.apache.doris.analysis.ColumnNullableType; +import org.apache.doris.analysis.EncryptKeyName; import org.apache.doris.analysis.PassVar; import org.apache.doris.analysis.SetType; import org.apache.doris.analysis.StorageBackend; @@ -100,6 +101,7 @@ import org.apache.doris.nereids.DorisParser.DereferenceContext; import org.apache.doris.nereids.DorisParser.DropCatalogRecycleBinContext; import org.apache.doris.nereids.DorisParser.DropConstraintContext; +import org.apache.doris.nereids.DorisParser.DropEncryptkeyContext; import org.apache.doris.nereids.DorisParser.DropMTMVContext; import org.apache.doris.nereids.DorisParser.DropProcedureContext; import org.apache.doris.nereids.DorisParser.DropRoleContext; @@ -450,6 +452,7 @@ import org.apache.doris.nereids.trees.plans.commands.DropCatalogRecycleBinCommand; import org.apache.doris.nereids.trees.plans.commands.DropCatalogRecycleBinCommand.IdType; import org.apache.doris.nereids.trees.plans.commands.DropConstraintCommand; +import org.apache.doris.nereids.trees.plans.commands.DropEncryptkeyCommand; import org.apache.doris.nereids.trees.plans.commands.DropJobCommand; import org.apache.doris.nereids.trees.plans.commands.DropMTMVCommand; import org.apache.doris.nereids.trees.plans.commands.DropProcedureCommand; @@ -4375,6 +4378,12 @@ public LogicalPlan visitDropRole(DropRoleContext ctx) { return new DropRoleCommand(ctx.name.getText(), ctx.EXISTS() != null); } + @Override + public LogicalPlan visitDropEncryptkey(DropEncryptkeyContext ctx) { + List nameParts = visitMultipartIdentifier(ctx.name); + return new DropEncryptkeyCommand(new EncryptKeyName(nameParts), ctx.EXISTS() != null); + } + @Override public LogicalPlan visitDropSqlBlockRule(DropSqlBlockRuleContext ctx) { return new DropSqlBlockRuleCommand(visitIdentifierSeq(ctx.identifierSeq()), ctx.EXISTS() != null); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index 04551da904bfa9..1f7b838edfd905 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -172,6 +172,7 @@ public enum PlanType { ALTER_STORAGE_VAULT, ALTER_WORKLOAD_GROUP_COMMAND, DROP_CATALOG_RECYCLE_BIN_COMMAND, + DROP_ENCRYPTKEY_COMMAND, UNSET_VARIABLE_COMMAND, UNSET_DEFAULT_STORAGE_VAULT_COMMAND, UNSUPPORTED_COMMAND, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropEncryptkeyCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropEncryptkeyCommand.java new file mode 100644 index 00000000000000..4c163d342d696b --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropEncryptkeyCommand.java @@ -0,0 +1,65 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.commands; + +import org.apache.doris.analysis.EncryptKeyName; +import org.apache.doris.catalog.Database; +import org.apache.doris.catalog.EncryptKeySearchDesc; +import org.apache.doris.catalog.Env; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.StmtExecutor; + +/** + * drop encrypt key command + */ +public class DropEncryptkeyCommand extends DropCommand { + private final boolean ifExists; + private final EncryptKeyName encryptKeyName; + + /** + * constructor + */ + public DropEncryptkeyCommand(EncryptKeyName encryptKeyName, boolean ifExists) { + super(PlanType.DROP_ENCRYPTKEY_COMMAND); + this.encryptKeyName = encryptKeyName; + this.ifExists = ifExists; + } + + @Override + public void doRun(ConnectContext ctx, StmtExecutor executor) throws Exception { + // check operation privilege + if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, "ADMIN"); + } + // analyze encryptkey name + encryptKeyName.analyze(ctx); + EncryptKeySearchDesc encryptKeySearchDesc = new EncryptKeySearchDesc(encryptKeyName); + Database db = Env.getCurrentInternalCatalog().getDbOrDdlException(encryptKeyName.getDb()); + db.dropEncryptKey(encryptKeySearchDesc, ifExists); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitDropEncryptKeyCommand(this, context); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index 13eeef182641fb..6b801524fb295f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -39,6 +39,7 @@ import org.apache.doris.nereids.trees.plans.commands.DeleteFromUsingCommand; import org.apache.doris.nereids.trees.plans.commands.DropCatalogRecycleBinCommand; import org.apache.doris.nereids.trees.plans.commands.DropConstraintCommand; +import org.apache.doris.nereids.trees.plans.commands.DropEncryptkeyCommand; import org.apache.doris.nereids.trees.plans.commands.DropJobCommand; import org.apache.doris.nereids.trees.plans.commands.DropMTMVCommand; import org.apache.doris.nereids.trees.plans.commands.DropProcedureCommand; @@ -422,6 +423,10 @@ default R visitDropRoleCommand(DropRoleCommand dropRoleCommand, C context) { return visitCommand(dropRoleCommand, context); } + default R visitDropEncryptKeyCommand(DropEncryptkeyCommand dropEncryptkeyCommand, C context) { + return visitCommand(dropEncryptkeyCommand, context); + } + default R visitDropSqlBlockRuleCommand(DropSqlBlockRuleCommand dropSqlBlockRuleCommand, C context) { return visitCommand(dropSqlBlockRuleCommand, context); } diff --git a/regression-test/data/nereids_p0/test_nereids_encrypt_test.out b/regression-test/data/nereids_p0/test_nereids_encrypt_test.out new file mode 100644 index 00000000000000..7984f572e8f26f --- /dev/null +++ b/regression-test/data/nereids_p0/test_nereids_encrypt_test.out @@ -0,0 +1,8 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !check_encrypt_1 -- +test_nereids_encrypt_test_db.test_nereids_encrypt_test_key ABCD123456789 + +-- !check_encrypt_2 -- + +-- !check_encrypt_3 -- + diff --git a/regression-test/suites/nereids_p0/test_nereids_encrypt_test.groovy b/regression-test/suites/nereids_p0/test_nereids_encrypt_test.groovy new file mode 100644 index 00000000000000..2fab616580faaa --- /dev/null +++ b/regression-test/suites/nereids_p0/test_nereids_encrypt_test.groovy @@ -0,0 +1,29 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. +suite("test_nereids_encrypt_test") { + def dbName="test_nereids_encrypt_test_db" + def encryptkeyName="test_nereids_encrypt_test_key" + sql """ create database IF NOT EXISTS ${dbName}; """ + sql """ use ${dbName}; """ + checkNereidsExecute("drop encryptkey if exists ${encryptkeyName}") + sql """CREATE ENCRYPTKEY ${encryptkeyName} AS "ABCD123456789";""" + qt_check_encrypt_1("SHOW ENCRYPTKEYS FROM ${dbName}") + checkNereidsExecute("drop encryptkey ${encryptkeyName}") + qt_check_encrypt_2("SHOW ENCRYPTKEYS FROM ${dbName}") + checkNereidsExecute("drop encryptkey if exists ${encryptkeyName}") + qt_check_encrypt_3("SHOW ENCRYPTKEYS FROM ${dbName}") +} \ No newline at end of file From 8751287549245e5bd6ac8adbe21d75af6c9063ed Mon Sep 17 00:00:00 2001 From: Calvin Kirs Date: Tue, 26 Nov 2024 17:03:04 +0800 Subject: [PATCH 007/399] [chore](github)Update .asf.yaml (#44614) --- .asf.yaml | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/.asf.yaml b/.asf.yaml index 3892aca2eddb77..f4c21ce4b0f3c9 100644 --- a/.asf.yaml +++ b/.asf.yaml @@ -56,8 +56,7 @@ github: - cloud_p0 (Doris Cloud Regression) - FE UT (Doris FE UT) - BE UT (Doris BE UT) - - Build Broker - - ShellCheck + - Build Broker - Build Third Party Libraries (Linux) - Build Third Party Libraries (macOS) - Build Third Party Libraries (macOS-arm64) @@ -80,7 +79,6 @@ github: - Clang Formatter - CheckStyle - Build Broker - - ShellCheck - Build Third Party Libraries (Linux) - Build Third Party Libraries (macOS) - FE UT (Doris FE UT) @@ -103,7 +101,6 @@ github: - Clang Formatter - CheckStyle - Build Broker - - ShellCheck - Build Third Party Libraries (Linux) - Build Third Party Libraries (macOS) - COMPILE (DORIS_COMPILE) From aa15ca692a1cc010235e15571fac29699d26a367 Mon Sep 17 00:00:00 2001 From: Mryange Date: Tue, 26 Nov 2024 17:32:12 +0800 Subject: [PATCH 008/399] [fix](DECIMAL) error DECIMAL cat to BOOLEAN (#44326) In the past, there were issues with converting `double` and `decimal` to `boolean`. For example, a `double` value like 0.13 would first be cast to `uint8`, resulting in 0. Then, it would be converted to `bool`, yielding 0 (incorrect, as the expected result is 1). Similarly, `decimal` values were directly cast to `uint8`, leading to non-0/1 values for `bool`. This issue arises because Doris internally uses `uint8` to represent `boolean`. before ``` mysql> select cast(40.123 as BOOLEAN); +-------------------------+ | cast(40.123 as BOOLEAN) | +-------------------------+ | 40 | +-------------------------+ ``` now ``` mysql> select cast(40.123 as BOOLEAN); +-------------------------+ | cast(40.123 as BOOLEAN) | +-------------------------+ | 1 | +-------------------------+ ``` --- be/src/vec/data_types/data_type_decimal.h | 19 ++++--- be/src/vec/functions/function_cast.h | 43 ++++++++++----- .../test_cast_decimalv3_as_bool.out | 17 ++++++ .../test_cast_decimalv3_as_bool.groovy | 55 +++++++++++++++++++ .../test_case_function_null.groovy | 18 ++++-- 5 files changed, 124 insertions(+), 28 deletions(-) create mode 100644 regression-test/data/correctness/test_cast_decimalv3_as_bool.out create mode 100644 regression-test/suites/correctness/test_cast_decimalv3_as_bool.groovy diff --git a/be/src/vec/data_types/data_type_decimal.h b/be/src/vec/data_types/data_type_decimal.h index a44390cd58d86a..74655ff6ee8bec 100644 --- a/be/src/vec/data_types/data_type_decimal.h +++ b/be/src/vec/data_types/data_type_decimal.h @@ -460,15 +460,20 @@ void convert_from_decimals(RealTo* dst, const RealFrom* src, UInt32 precicion_fr MaxFieldType multiplier = DataTypeDecimal::get_scale_multiplier(scale_from); FromDataType from_data_type(precicion_from, scale_from); for (size_t i = 0; i < size; i++) { - auto tmp = static_cast(src[i]).value / multiplier.value; - if constexpr (narrow_integral) { - if (tmp < min_result.value || tmp > max_result.value) { - THROW_DECIMAL_CONVERT_OVERFLOW_EXCEPTION(from_data_type.to_string(src[i]), - from_data_type.get_name(), - OrigToDataType {}.get_name()); + // uint8_t now use as boolean in doris + if constexpr (std::is_same_v) { + dst[i] = static_cast(src[i]).value != 0; + } else { + auto tmp = static_cast(src[i]).value / multiplier.value; + if constexpr (narrow_integral) { + if (tmp < min_result.value || tmp > max_result.value) { + THROW_DECIMAL_CONVERT_OVERFLOW_EXCEPTION(from_data_type.to_string(src[i]), + from_data_type.get_name(), + OrigToDataType {}.get_name()); + } } + dst[i] = tmp; } - dst[i] = tmp; } } diff --git a/be/src/vec/functions/function_cast.h b/be/src/vec/functions/function_cast.h index 5de820dfa3a24f..7027a4e9c622bc 100644 --- a/be/src/vec/functions/function_cast.h +++ b/be/src/vec/functions/function_cast.h @@ -258,6 +258,21 @@ struct ConvertImpl { using FromFieldType = typename FromDataType::FieldType; using ToFieldType = typename ToDataType::FieldType; + // `static_cast_set` is introduced to wrap `static_cast` and handle special cases. + // Doris uses `uint8` to represent boolean values internally. + // Directly `static_cast` to `uint8` can result in non-0/1 values, + // To address this, `static_cast_set` performs an additional check: + // For `uint8` types, it explicitly uses `static_cast` to ensure + // the result is either 0 or 1. + static void static_cast_set(ToFieldType& to, const FromFieldType& from) { + // uint8_t now use as boolean in doris + if constexpr (std::is_same_v) { + to = static_cast(from); + } else { + to = static_cast(from); + } + } + template static Status execute(FunctionContext* context, Block& block, const ColumnNumbers& arguments, uint32_t result, size_t input_rows_count, @@ -377,8 +392,9 @@ struct ConvertImpl { } else if constexpr (IsDateTimeV2Type) { DataTypeDateTimeV2::cast_from_date(vec_from[i], vec_to[i]); } else { - vec_to[i] = - reinterpret_cast(vec_from[i]).to_int64(); + static_cast_set( + vec_to[i], + reinterpret_cast(vec_from[i]).to_int64()); } } } else if constexpr (IsTimeV2Type) { @@ -409,13 +425,16 @@ struct ConvertImpl { } } else { if constexpr (IsDateTimeV2Type) { - vec_to[i] = reinterpret_cast&>( - vec_from[i]) - .to_int64(); + static_cast_set( + vec_to[i], + reinterpret_cast&>( + vec_from[i]) + .to_int64()); } else { - vec_to[i] = reinterpret_cast&>( - vec_from[i]) - .to_int64(); + static_cast_set(vec_to[i], + reinterpret_cast&>( + vec_from[i]) + .to_int64()); } } } @@ -441,16 +460,10 @@ struct ConvertImpl { } } else { for (size_t i = 0; i < size; ++i) { - vec_to[i] = static_cast(vec_from[i]); + static_cast_set(vec_to[i], vec_from[i]); } } } - // TODO: support boolean cast more reasonable - if constexpr (std::is_same_v) { - for (int i = 0; i < size; ++i) { - vec_to[i] = static_cast(vec_to[i]); - } - } block.replace_by_position(result, std::move(col_to)); } else { diff --git a/regression-test/data/correctness/test_cast_decimalv3_as_bool.out b/regression-test/data/correctness/test_cast_decimalv3_as_bool.out new file mode 100644 index 00000000000000..4f41130b00bafd --- /dev/null +++ b/regression-test/data/correctness/test_cast_decimalv3_as_bool.out @@ -0,0 +1,17 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !select1 -- +0.000 13131.213132100 0E-16 +0.000 2131231.231000000 2.3323000E-9 +3.141 0E-9 123123.2131231231322130 + +-- !select2 -- +false true false +false true true +true false true + +-- !select3 -- +true 1 true false + +-- !select3 -- +true 1 true false + diff --git a/regression-test/suites/correctness/test_cast_decimalv3_as_bool.groovy b/regression-test/suites/correctness/test_cast_decimalv3_as_bool.groovy new file mode 100644 index 00000000000000..768da493251e6c --- /dev/null +++ b/regression-test/suites/correctness/test_cast_decimalv3_as_bool.groovy @@ -0,0 +1,55 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("test_cast_decimalv3_as_bool") { + sql """ DROP TABLE IF EXISTS cast_decimalv3_as_bool """ + sql """ + CREATE TABLE IF NOT EXISTS cast_decimalv3_as_bool ( + `id` int(11) , + `k1` decimalv3(9,3) , + `k2` decimalv3(18,9) , + `k3` decimalv3(38,16) , + ) + UNIQUE KEY(`id`) + DISTRIBUTED BY HASH(`id`) BUCKETS 10 + PROPERTIES ( + "enable_unique_key_merge_on_write" = "true", + "replication_num" = "1" + ); + """ + sql """ + set enable_nereids_planner=true,enable_fold_constant_by_be = false + """ + sql """ + INSERT INTO cast_decimalv3_as_bool VALUES + (1,0.00001,13131.2131321,0.000000000000000000), + (2,0.00000,2131231.231,0.0000000023323), + (3,3.141414,0.0000000000,123123.213123123132213); + """ + qt_select1 """ + select k1,k2,k3 from cast_decimalv3_as_bool order by id + """ + qt_select2 """ + select cast(k1 as boolean), cast(k2 as boolean) , cast(k3 as boolean) from cast_decimalv3_as_bool order by id + """ + qt_select3""" + select cast(3.00001 as boolean), cast(cast(3.00001 as boolean) as int),cast(0.001 as boolean),cast(0.000 as boolean); + """ + qt_select3""" + select cast(cast(3.00001 as double)as boolean), cast(cast(cast(3.00001 as double) as boolean) as int),cast(cast(0.001 as double) as boolean),cast(cast(0.000 as double) as boolean); + """ +} \ No newline at end of file diff --git a/regression-test/suites/query_p0/sql_functions/case_function/test_case_function_null.groovy b/regression-test/suites/query_p0/sql_functions/case_function/test_case_function_null.groovy index 5138db6e73b4ad..a91c86b5f48908 100644 --- a/regression-test/suites/query_p0/sql_functions/case_function/test_case_function_null.groovy +++ b/regression-test/suites/query_p0/sql_functions/case_function/test_case_function_null.groovy @@ -185,10 +185,11 @@ suite("test_case_function_null", "query,p0,arrow_flight_sql") { c2, c1; """ - + // There is a behavior change. The 0.4cast boolean used to be 0 in the past, but now it has changed to 1. + // Therefore, we need to update the case accordingly. qt_sql_case1 """ SELECT SUM( - CASE (((NULL BETWEEN NULL AND NULL)) and (CAST(0.4716 AS BOOLEAN))) + CASE (((NULL BETWEEN NULL AND NULL)) and (CAST(0.0 AS BOOLEAN))) WHEN ((CAST('-1530390546' AS VARCHAR)) LIKE ('-1678299490')) THEN (- (+ case_null2.c0)) WHEN CASE (NULL IN (NULL)) @@ -197,9 +198,10 @@ suite("test_case_function_null", "query,p0,arrow_flight_sql") { END) FROM case_null2; """ - + // There is a behavior change. The 0.4cast boolean used to be 0 in the past, but now it has changed to 1. + // Therefore, we need to update the case accordingly. qt_sql_case2 """ - SELECT SUM(CASE (((NULL BETWEEN NULL AND NULL)) and (CAST(0.4716 AS BOOLEAN))) + SELECT SUM(CASE (((NULL BETWEEN NULL AND NULL)) and (CAST(0.0 AS BOOLEAN))) WHEN ((CAST('-1530390546' AS VARCHAR)) LIKE ('-1678299490')) THEN (- (+ case_null2.c0)) END) @@ -209,9 +211,11 @@ suite("test_case_function_null", "query,p0,arrow_flight_sql") { sql "SET experimental_enable_nereids_planner=true" sql "SET enable_fallback_to_original_planner=false" + // There is a behavior change. The 0.4cast boolean used to be 0 in the past, but now it has changed to 1. + // Therefore, we need to update the case accordingly. qt_sql_case1 """ SELECT SUM( - CASE (((NULL BETWEEN NULL AND NULL)) and (CAST(0.4716 AS BOOLEAN))) + CASE (((NULL BETWEEN NULL AND NULL)) and (CAST(0.0 AS BOOLEAN))) WHEN ((CAST('-1530390546' AS VARCHAR)) LIKE ('-1678299490')) THEN (- (+ case_null2.c0)) WHEN CASE (NULL IN (NULL)) @@ -221,8 +225,10 @@ suite("test_case_function_null", "query,p0,arrow_flight_sql") { FROM case_null2; """ + // There is a behavior change. The 0.4cast boolean used to be 0 in the past, but now it has changed to 1. + // Therefore, we need to update the case accordingly. qt_sql_case2 """ - SELECT SUM(CASE (((NULL BETWEEN NULL AND NULL)) and (CAST(0.4716 AS BOOLEAN))) + SELECT SUM(CASE (((NULL BETWEEN NULL AND NULL)) and (CAST(0.0 AS BOOLEAN))) WHEN ((CAST('-1530390546' AS VARCHAR)) LIKE ('-1678299490')) THEN (- (+ case_null2.c0)) END) From 4d48597487b35d9bece381aefae1736a9016c9f1 Mon Sep 17 00:00:00 2001 From: Sun Chenyang Date: Tue, 26 Nov 2024 17:35:42 +0800 Subject: [PATCH 009/399] [fix](inverted index) fix close error on inverted index writer (#44601) Related PR: #44205 fix close error on inverted index writer --- be/src/olap/rowset/segment_v2/inverted_index_writer.cpp | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) 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 08cebaa07d6670..86a8f89e4c94e4 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index_writer.cpp +++ b/be/src/olap/rowset/segment_v2/inverted_index_writer.cpp @@ -667,8 +667,13 @@ class InvertedIndexColumnWriterImpl : public InvertedIndexColumnWriter { FINALLY_CLOSE(meta_out); FINALLY_CLOSE(data_out); FINALLY_CLOSE(index_out); - FINALLY_CLOSE(_dir); - FINALLY_CLOSE(_index_writer); + if constexpr (field_is_numeric_type(field_type)) { + FINALLY_CLOSE(_dir); + } else if constexpr (field_is_slice_type(field_type)) { + FINALLY_CLOSE(_index_writer); + // After closing the _index_writer, it needs to be reset to null to prevent issues of not closing it or closing it multiple times. + _index_writer.reset(); + } }) return Status::OK(); From 14a8bac3cad8000fe96e8676e7ef04675c412716 Mon Sep 17 00:00:00 2001 From: qiye Date: Tue, 26 Nov 2024 19:20:47 +0800 Subject: [PATCH 010/399] [fix](build index)Fix non-master nodes cannot see the latest build index job status (#44544) Problem Summary: Non-master nodes cannot see the latest build index job status, this will cause the case `test_build_index_with_clone_fault_injection` unstable. We add forward_to_master in show build index statement to get the latest job status. ### Release note Fix non-master nodes cannot see the latest build index job status --- .../java/org/apache/doris/alter/IndexChangeJob.java | 1 + .../java/org/apache/doris/alter/SchemaChangeJobV2.java | 2 +- .../org/apache/doris/analysis/ShowBuildIndexStmt.java | 10 ++++++++++ .../test_build_index_with_clone_fault.groovy | 1 + 4 files changed, 13 insertions(+), 1 deletion(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/IndexChangeJob.java b/fe/fe-core/src/main/java/org/apache/doris/alter/IndexChangeJob.java index a51cdb01699e23..e00587bee16d98 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/IndexChangeJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/IndexChangeJob.java @@ -348,6 +348,7 @@ protected void runWaitingTxnJob() throws AlterCancelException { olapTable.readUnlock(); } this.jobState = JobState.RUNNING; + // DO NOT write edit log here, tasks will be sent again if FE restart or master changed. LOG.info("transfer inverted index job {} state to {}", jobId, this.jobState); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java index 522433dfe898da..58e3e05e214a9d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java @@ -556,7 +556,7 @@ protected void runWaitingTxnJob() throws AlterCancelException { this.jobState = JobState.RUNNING; - // DO NOT write edit log here, tasks will be send again if FE restart or master changed. + // DO NOT write edit log here, tasks will be sent again if FE restart or master changed. LOG.info("transfer schema change job {} state to {}", jobId, this.jobState); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowBuildIndexStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowBuildIndexStmt.java index 944365fb0d48df..9914bb76400a41 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowBuildIndexStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowBuildIndexStmt.java @@ -29,6 +29,7 @@ import org.apache.doris.common.proc.ProcNodeInterface; import org.apache.doris.common.proc.ProcService; import org.apache.doris.common.util.OrderByPair; +import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.ShowResultSetMetaData; import com.google.common.base.Strings; @@ -224,4 +225,13 @@ public ShowResultSetMetaData getMetaData() { return builder.build(); } + + @Override + public RedirectStatus getRedirectStatus() { + if (ConnectContext.get().getSessionVariable().getForwardToMaster()) { + return RedirectStatus.FORWARD_NO_SYNC; + } else { + return RedirectStatus.NO_FORWARD; + } + } } diff --git a/regression-test/suites/fault_injection_p0/test_build_index_with_clone_fault.groovy b/regression-test/suites/fault_injection_p0/test_build_index_with_clone_fault.groovy index ec175792d48de3..998c8fa98619e0 100644 --- a/regression-test/suites/fault_injection_p0/test_build_index_with_clone_fault.groovy +++ b/regression-test/suites/fault_injection_p0/test_build_index_with_clone_fault.groovy @@ -58,6 +58,7 @@ suite("test_build_index_with_clone_fault_injection", "nonConcurrent"){ while (attempt < maxRetries) { def show_build_index = sql_return_maparray("show build index where TableName = \"${tbl}\" ORDER BY JobId DESC LIMIT 1") if (show_build_index && show_build_index.size() > 0) { + logger.info("show build index result: ${show_build_index}") def currentState = show_build_index[0].State def currentMsg = show_build_index[0].Msg if ((currentState == expectedState && currentMsg == expectedMsg) || currentState == "FINISHED") { From 0066bcdb26f08e2e5945a4a67a91814aae657814 Mon Sep 17 00:00:00 2001 From: zfr95 Date: Tue, 26 Nov 2024 20:36:16 +0800 Subject: [PATCH 011/399] [fix](auth)fix stream load information to observer (#44632) --- .../suites/auth_call/test_dml_stream_load_auth.groovy | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/regression-test/suites/auth_call/test_dml_stream_load_auth.groovy b/regression-test/suites/auth_call/test_dml_stream_load_auth.groovy index 240fdde6f6926d..30f01c0e72d9c5 100644 --- a/regression-test/suites/auth_call/test_dml_stream_load_auth.groovy +++ b/regression-test/suites/auth_call/test_dml_stream_load_auth.groovy @@ -59,7 +59,7 @@ suite("test_dml_stream_load_auth","p0,auth_call") { def path_file = "${context.file.parent}/../../data/auth_call/stream_load_data.csv" def load_path = "${context.file.parent}/../../data/auth_call/stream_load_cm.sh" - def cm = """curl --location-trusted -u ${user}:${pwd} -H "column_separator:," -T ${path_file} http://${sql_ip}:${http_port}/api/${dbName}/${tableName}/_stream_load""" + def cm = """curl -v --location-trusted -u ${user}:${pwd} -H "column_separator:," -T ${path_file} http://${sql_ip}:${http_port}/api/${dbName}/${tableName}/_stream_load""" logger.info("cm: " + cm) write_to_file(load_path, cm) cm = "bash " + load_path From bce9b69c7da60d62101516612d34707763b77bc0 Mon Sep 17 00:00:00 2001 From: Gabriel Date: Tue, 26 Nov 2024 20:42:39 +0800 Subject: [PATCH 012/399] [fix](coordinator) Fix wrong bucket assignment (#44603) follow-up : #44459 --- .../src/main/java/org/apache/doris/qe/Coordinator.java | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java index 61ecaf7fc8b359..3a6f6e4f84069d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java @@ -2766,17 +2766,18 @@ private void assignScanRanges(PlanFragmentId fragmentId, int parallelExecInstanc .addAll(nodeScanRange.getValue()); } } - params.instanceExecParams.add(instanceParam); + List instanceExecParams = new ArrayList<>(); + instanceExecParams.add(instanceParam); for (int i = 1; i < parallelExecInstanceNum; i++) { - params.instanceExecParams.add(new FInstanceExecParam( + instanceExecParams.add(new FInstanceExecParam( null, addressScanRange.getKey(), 0, params)); } int index = 0; for (Pair>> nodeScanRangeMap : scanRange) { - params.instanceExecParams.get(index % params.instanceExecParams.size()) - .addBucketSeq(nodeScanRangeMap.first); + instanceExecParams.get(index % instanceExecParams.size()).addBucketSeq(nodeScanRangeMap.first); index++; } + params.instanceExecParams.addAll(instanceExecParams); } else { int expectedInstanceNum = 1; if (parallelExecInstanceNum > 1) { From d3714bf55619fffed08250687f614dcaea271458 Mon Sep 17 00:00:00 2001 From: meiyi Date: Tue, 26 Nov 2024 22:55:56 +0800 Subject: [PATCH 013/399] [fix](cloud) cloud mode support txn load for mow tables (#41932) --- .../cloud_engine_calc_delete_bitmap_task.cpp | 106 ++++++++-- .../cloud_engine_calc_delete_bitmap_task.h | 9 +- be/src/cloud/cloud_tablet.cpp | 6 +- be/src/cloud/cloud_tablet.h | 3 +- be/src/olap/base_tablet.cpp | 60 ++++-- be/src/olap/base_tablet.h | 15 +- be/src/olap/calc_delete_bitmap_executor.cpp | 6 +- be/src/olap/calc_delete_bitmap_executor.h | 3 +- be/src/olap/tablet.cpp | 2 +- be/src/olap/tablet.h | 3 +- be/src/olap/tablet_meta.h | 1 + be/src/olap/txn_manager.h | 6 + cloud/src/meta-service/meta_service_txn.cpp | 181 ++++++++---------- .../java/org/apache/doris/common/Config.java | 3 + .../CloudGlobalTransactionMgr.java | 161 +++++++++++----- .../doris/transaction/TransactionEntry.java | 9 - gensrc/thrift/AgentService.thrift | 1 + .../insert_p0/transaction/txn_insert.groovy | 11 +- .../txn_insert_concurrent_insert_mow.groovy | 135 +++++++++++++ ..._insert_concurrent_insert_aggregate.groovy | 2 +- ..._insert_concurrent_insert_duplicate.groovy | 2 +- .../txn_insert_concurrent_insert_mor.groovy | 2 +- .../txn_insert_concurrent_insert_mow.groovy | 13 +- .../txn_insert_concurrent_insert_ud.groovy | 7 +- ...txn_insert_concurrent_insert_update.groovy | 2 +- .../txn_insert_with_schema_change.groovy | 2 +- 26 files changed, 511 insertions(+), 240 deletions(-) create mode 100644 regression-test/suites/insert_p0/transaction/txn_insert_concurrent_insert_mow.groovy rename regression-test/suites/insert_p2/{ => transaction}/txn_insert_concurrent_insert_aggregate.groovy (99%) rename regression-test/suites/insert_p2/{ => transaction}/txn_insert_concurrent_insert_duplicate.groovy (99%) rename regression-test/suites/insert_p2/{ => transaction}/txn_insert_concurrent_insert_mor.groovy (99%) rename regression-test/suites/insert_p2/{ => transaction}/txn_insert_concurrent_insert_mow.groovy (95%) rename regression-test/suites/insert_p2/{ => transaction}/txn_insert_concurrent_insert_ud.groovy (98%) rename regression-test/suites/insert_p2/{ => transaction}/txn_insert_concurrent_insert_update.groovy (99%) rename regression-test/suites/insert_p2/{ => transaction}/txn_insert_with_schema_change.groovy (99%) diff --git a/be/src/cloud/cloud_engine_calc_delete_bitmap_task.cpp b/be/src/cloud/cloud_engine_calc_delete_bitmap_task.cpp index 6abc3958650ef6..7391449b73f1b1 100644 --- a/be/src/cloud/cloud_engine_calc_delete_bitmap_task.cpp +++ b/be/src/cloud/cloud_engine_calc_delete_bitmap_task.cpp @@ -75,7 +75,7 @@ Status CloudEngineCalcDeleteBitmapTask::execute() { for (size_t i = 0; i < partition.tablet_ids.size(); i++) { auto tablet_id = partition.tablet_ids[i]; auto tablet_calc_delete_bitmap_ptr = std::make_shared( - _engine, this, tablet_id, transaction_id, version); + _engine, this, tablet_id, transaction_id, version, partition.sub_txn_ids); if (has_compaction_stats) { tablet_calc_delete_bitmap_ptr->set_compaction_stats( partition.base_compaction_cnts[i], partition.cumulative_compaction_cnts[i], @@ -107,12 +107,13 @@ Status CloudEngineCalcDeleteBitmapTask::execute() { CloudTabletCalcDeleteBitmapTask::CloudTabletCalcDeleteBitmapTask( CloudStorageEngine& engine, CloudEngineCalcDeleteBitmapTask* engine_task, int64_t tablet_id, - int64_t transaction_id, int64_t version) + int64_t transaction_id, int64_t version, const std::vector& sub_txn_ids) : _engine(engine), _engine_calc_delete_bitmap_task(engine_task), _tablet_id(tablet_id), _transaction_id(transaction_id), - _version(version) { + _version(version), + _sub_txn_ids(sub_txn_ids) { _mem_tracker = MemTrackerLimiter::create_shared( MemTrackerLimiter::Type::OTHER, fmt::format("CloudTabletCalcDeleteBitmapTask#_transaction_id={}", _transaction_id)); @@ -189,6 +190,60 @@ Status CloudTabletCalcDeleteBitmapTask::handle() const { return error_st; } + int64_t t3 = MonotonicMicros(); + Status status; + if (_sub_txn_ids.empty()) { + status = _handle_rowset(tablet, _version); + } else { + std::stringstream ss; + for (const auto& sub_txn_id : _sub_txn_ids) { + ss << sub_txn_id << ", "; + } + LOG(INFO) << "start calc delete bitmap for txn_id=" << _transaction_id << ", sub_txn_ids=[" + << ss.str() << "], table_id=" << tablet->table_id() + << ", partition_id=" << tablet->partition_id() << ", tablet_id=" << _tablet_id + << ", start_version=" << _version; + std::vector invisible_rowsets; + DeleteBitmapPtr tablet_delete_bitmap = + std::make_shared(tablet->tablet_meta()->delete_bitmap()); + for (int i = 0; i < _sub_txn_ids.size(); ++i) { + int64_t sub_txn_id = _sub_txn_ids[i]; + int64_t version = _version + i; + LOG(INFO) << "start calc delete bitmap for txn_id=" << _transaction_id + << ", sub_txn_id=" << sub_txn_id << ", table_id=" << tablet->table_id() + << ", partition_id=" << tablet->partition_id() << ", tablet_id=" << _tablet_id + << ", start_version=" << _version << ", cur_version=" << version; + status = _handle_rowset(tablet, version, sub_txn_id, &invisible_rowsets, + tablet_delete_bitmap); + if (!status.ok()) { + LOG(INFO) << "failed to calculate delete bitmap on tablet" + << ", table_id=" << tablet->table_id() + << ", transaction_id=" << _transaction_id << ", sub_txn_id=" << sub_txn_id + << ", tablet_id=" << tablet->tablet_id() << ", start version=" << _version + << ", cur_version=" << version << ", status=" << status; + return status; + } + DCHECK(invisible_rowsets.size() == i + 1); + } + } + auto total_update_delete_bitmap_time_us = MonotonicMicros() - t3; + LOG(INFO) << "calculate delete bitmap successfully on tablet" + << ", table_id=" << tablet->table_id() << ", transaction_id=" << _transaction_id + << ", tablet_id=" << tablet->tablet_id() + << ", get_tablet_time_us=" << get_tablet_time_us + << ", sync_rowset_time_us=" << sync_rowset_time_us + << ", total_update_delete_bitmap_time_us=" << total_update_delete_bitmap_time_us + << ", res=" << status; + return status; +} + +Status CloudTabletCalcDeleteBitmapTask::_handle_rowset( + std::shared_ptr tablet, int64_t version, int64_t sub_txn_id, + std::vector* invisible_rowsets, + DeleteBitmapPtr tablet_delete_bitmap) const { + int64_t transaction_id = sub_txn_id == -1 ? _transaction_id : sub_txn_id; + std::string txn_str = "txn_id=" + std::to_string(_transaction_id) + + (sub_txn_id == -1 ? "" : ", sub_txn_id=" + std::to_string(sub_txn_id)); RowsetSharedPtr rowset; DeleteBitmapPtr delete_bitmap; RowsetIdUnorderedSet rowset_ids; @@ -197,59 +252,76 @@ Status CloudTabletCalcDeleteBitmapTask::handle() const { int64_t txn_expiration; TxnPublishInfo previous_publish_info; Status status = _engine.txn_delete_bitmap_cache().get_tablet_txn_info( - _transaction_id, _tablet_id, &rowset, &delete_bitmap, &rowset_ids, &txn_expiration, + transaction_id, _tablet_id, &rowset, &delete_bitmap, &rowset_ids, &txn_expiration, &partial_update_info, &publish_status, &previous_publish_info); if (status != Status::OK()) { - LOG(WARNING) << "failed to get tablet txn info. tablet_id=" << _tablet_id - << ", txn_id=" << _transaction_id << ", status=" << status; + LOG(WARNING) << "failed to get tablet txn info. tablet_id=" << _tablet_id << ", " << txn_str + << ", status=" << status; _engine_calc_delete_bitmap_task->add_error_tablet_id(_tablet_id, status); return status; } int64_t t3 = MonotonicMicros(); - rowset->set_version(Version(_version, _version)); + rowset->set_version(Version(version, version)); TabletTxnInfo txn_info; txn_info.rowset = rowset; txn_info.delete_bitmap = delete_bitmap; txn_info.rowset_ids = rowset_ids; txn_info.partial_update_info = partial_update_info; txn_info.publish_status = publish_status; - txn_info.publish_info = {.publish_version = _version, + txn_info.publish_info = {.publish_version = version, .base_compaction_cnt = _ms_base_compaction_cnt, .cumulative_compaction_cnt = _ms_cumulative_compaction_cnt, .cumulative_point = _ms_cumulative_point}; auto update_delete_bitmap_time_us = 0; if (txn_info.publish_status && (*(txn_info.publish_status) == PublishStatus::SUCCEED) && - _version == previous_publish_info.publish_version && + version == previous_publish_info.publish_version && _ms_base_compaction_cnt == previous_publish_info.base_compaction_cnt && _ms_cumulative_compaction_cnt == previous_publish_info.cumulative_compaction_cnt && _ms_cumulative_point == previous_publish_info.cumulative_point) { // if version or compaction stats can't match, it means that this is a retry and there are // compaction or other loads finished successfully on the same tablet. So the previous publish // is stale and we should re-calculate the delete bitmap - LOG(INFO) << "tablet=" << _tablet_id << ",txn=" << _transaction_id + LOG(INFO) << "tablet=" << _tablet_id << ", " << txn_str << ",publish_status=SUCCEED,not need to recalculate and update delete_bitmap."; } else { - status = CloudTablet::update_delete_bitmap(tablet, &txn_info, _transaction_id, - txn_expiration); + if (invisible_rowsets == nullptr) { + status = CloudTablet::update_delete_bitmap(tablet, &txn_info, transaction_id, + txn_expiration); + } else { + txn_info.is_txn_load = true; + txn_info.invisible_rowsets = *invisible_rowsets; + txn_info.lock_id = _transaction_id; + txn_info.next_visible_version = _version; + status = CloudTablet::update_delete_bitmap(tablet, &txn_info, transaction_id, + txn_expiration, tablet_delete_bitmap); + } update_delete_bitmap_time_us = MonotonicMicros() - t3; } if (status != Status::OK()) { LOG(WARNING) << "failed to calculate delete bitmap. rowset_id=" << rowset->rowset_id() - << ", tablet_id=" << _tablet_id << ", txn_id=" << _transaction_id - << ", status=" << status; + << ", tablet_id=" << _tablet_id << ", " << txn_str << ", status=" << status; _engine_calc_delete_bitmap_task->add_error_tablet_id(_tablet_id, status); return status; } _engine_calc_delete_bitmap_task->add_succ_tablet_id(_tablet_id); LOG(INFO) << "calculate delete bitmap successfully on tablet" - << ", table_id=" << tablet->table_id() << ", transaction_id=" << _transaction_id + << ", table_id=" << tablet->table_id() << ", " << txn_str << ", tablet_id=" << tablet->tablet_id() << ", num_rows=" << rowset->num_rows() - << ", get_tablet_time_us=" << get_tablet_time_us - << ", sync_rowset_time_us=" << sync_rowset_time_us << ", update_delete_bitmap_time_us=" << update_delete_bitmap_time_us << ", res=" << status; + if (invisible_rowsets != nullptr) { + invisible_rowsets->push_back(rowset); + // see CloudTablet::save_delete_bitmap + auto dm = txn_info.delete_bitmap->delete_bitmap; + for (auto it = dm.begin(); it != dm.end(); ++it) { + if (std::get<1>(it->first) != DeleteBitmap::INVALID_SEGMENT_ID) { + tablet_delete_bitmap->merge( + {std::get<0>(it->first), std::get<1>(it->first), version}, it->second); + } + } + } return status; } diff --git a/be/src/cloud/cloud_engine_calc_delete_bitmap_task.h b/be/src/cloud/cloud_engine_calc_delete_bitmap_task.h index e3733d3e696ff8..c70a9cfa3903ba 100644 --- a/be/src/cloud/cloud_engine_calc_delete_bitmap_task.h +++ b/be/src/cloud/cloud_engine_calc_delete_bitmap_task.h @@ -34,7 +34,8 @@ class CloudTabletCalcDeleteBitmapTask { public: CloudTabletCalcDeleteBitmapTask(CloudStorageEngine& engine, CloudEngineCalcDeleteBitmapTask* engine_task, int64_t tablet_id, - int64_t transaction_id, int64_t version); + int64_t transaction_id, int64_t version, + const std::vector& sub_txn_ids); ~CloudTabletCalcDeleteBitmapTask() = default; void set_compaction_stats(int64_t ms_base_compaction_cnt, int64_t ms_cumulative_compaction_cnt, @@ -43,12 +44,18 @@ class CloudTabletCalcDeleteBitmapTask { Status handle() const; private: + Status _handle_rowset(std::shared_ptr tablet, int64_t version, + int64_t sub_txn_id = -1, + std::vector* invisible_rowsets = nullptr, + DeleteBitmapPtr tablet_delete_bitmap = nullptr) const; + CloudStorageEngine& _engine; CloudEngineCalcDeleteBitmapTask* _engine_calc_delete_bitmap_task; int64_t _tablet_id; int64_t _transaction_id; int64_t _version; + std::vector _sub_txn_ids; int64_t _ms_base_compaction_cnt {-1}; int64_t _ms_cumulative_compaction_cnt {-1}; diff --git a/be/src/cloud/cloud_tablet.cpp b/be/src/cloud/cloud_tablet.cpp index c88b073e96494a..267c204c0e608e 100644 --- a/be/src/cloud/cloud_tablet.cpp +++ b/be/src/cloud/cloud_tablet.cpp @@ -690,7 +690,8 @@ CalcDeleteBitmapExecutor* CloudTablet::calc_delete_bitmap_executor() { Status CloudTablet::save_delete_bitmap(const TabletTxnInfo* txn_info, int64_t txn_id, DeleteBitmapPtr delete_bitmap, RowsetWriter* rowset_writer, - const RowsetIdUnorderedSet& cur_rowset_ids) { + const RowsetIdUnorderedSet& cur_rowset_ids, + int64_t lock_id) { RowsetSharedPtr rowset = txn_info->rowset; int64_t cur_version = rowset->start_version(); // update delete bitmap info, in order to avoid recalculation when trying again @@ -714,8 +715,9 @@ Status CloudTablet::save_delete_bitmap(const TabletTxnInfo* txn_info, int64_t tx } } + auto ms_lock_id = lock_id == -1 ? txn_id : lock_id; RETURN_IF_ERROR(_engine.meta_mgr().update_delete_bitmap( - *this, txn_id, COMPACTION_DELETE_BITMAP_LOCK_ID, new_delete_bitmap.get())); + *this, ms_lock_id, COMPACTION_DELETE_BITMAP_LOCK_ID, new_delete_bitmap.get())); // store the delete bitmap with sentinel marks in txn_delete_bitmap_cache because if the txn is retried for some reason, // it will use the delete bitmap from txn_delete_bitmap_cache when re-calculating the delete bitmap, during which it will do diff --git a/be/src/cloud/cloud_tablet.h b/be/src/cloud/cloud_tablet.h index 0fde2f5b1d93ff..80038e569ba2fc 100644 --- a/be/src/cloud/cloud_tablet.h +++ b/be/src/cloud/cloud_tablet.h @@ -170,7 +170,8 @@ class CloudTablet final : public BaseTablet { Status save_delete_bitmap(const TabletTxnInfo* txn_info, int64_t txn_id, DeleteBitmapPtr delete_bitmap, RowsetWriter* rowset_writer, - const RowsetIdUnorderedSet& cur_rowset_ids) override; + const RowsetIdUnorderedSet& cur_rowset_ids, + int64_t lock_id = -1) override; Status calc_delete_bitmap_for_compaction(const std::vector& input_rowsets, const RowsetSharedPtr& output_rowset, diff --git a/be/src/olap/base_tablet.cpp b/be/src/olap/base_tablet.cpp index 2e70e4586cc768..a499a27b07f6e2 100644 --- a/be/src/olap/base_tablet.cpp +++ b/be/src/olap/base_tablet.cpp @@ -450,7 +450,8 @@ Status BaseTablet::lookup_row_key(const Slice& encoded_key, TabletSchema* latest RowLocation* row_location, uint32_t version, std::vector>& segment_caches, RowsetSharedPtr* rowset, bool with_rowid, - std::string* encoded_seq_value, OlapReaderStatistics* stats) { + std::string* encoded_seq_value, OlapReaderStatistics* stats, + DeleteBitmapPtr delete_bitmap) { SCOPED_BVAR_LATENCY(g_tablet_lookup_rowkey_latency); size_t seq_col_length = 0; // use the latest tablet schema to decide if the tablet has sequence column currently @@ -467,6 +468,8 @@ Status BaseTablet::lookup_row_key(const Slice& encoded_key, TabletSchema* latest Slice(encoded_key.get_data(), encoded_key.get_size() - seq_col_length - rowid_length); RowLocation loc; + auto tablet_delete_bitmap = + delete_bitmap == nullptr ? _tablet_meta->delete_bitmap_ptr() : delete_bitmap; for (size_t i = 0; i < specified_rowsets.size(); i++) { const auto& rs = specified_rowsets[i]; const auto& segments_key_bounds = rs->rowset_meta()->get_segments_key_bounds(); @@ -501,7 +504,7 @@ Status BaseTablet::lookup_row_key(const Slice& encoded_key, TabletSchema* latest if (!s.ok() && !s.is()) { return s; } - if (s.ok() && _tablet_meta->delete_bitmap().contains_agg_without_cache( + if (s.ok() && tablet_delete_bitmap->contains_agg_without_cache( {loc.rowset_id, loc.segment_id, version}, loc.row_id)) { // if has sequence col, we continue to compare the sequence_id of // all rowsets, util we find an existing key. @@ -535,7 +538,8 @@ Status BaseTablet::calc_delete_bitmap(const BaseTabletSPtr& tablet, RowsetShared const std::vector& segments, const std::vector& specified_rowsets, DeleteBitmapPtr delete_bitmap, int64_t end_version, - CalcDeleteBitmapToken* token, RowsetWriter* rowset_writer) { + CalcDeleteBitmapToken* token, RowsetWriter* rowset_writer, + DeleteBitmapPtr tablet_delete_bitmap) { auto rowset_id = rowset->rowset_id(); if (specified_rowsets.empty() || segments.empty()) { LOG(INFO) << "skip to construct delete bitmap tablet: " << tablet->tablet_id() @@ -548,10 +552,11 @@ Status BaseTablet::calc_delete_bitmap(const BaseTabletSPtr& tablet, RowsetShared const auto& seg = segment; if (token != nullptr) { RETURN_IF_ERROR(token->submit(tablet, rowset, seg, specified_rowsets, end_version, - delete_bitmap, rowset_writer)); + delete_bitmap, rowset_writer, tablet_delete_bitmap)); } else { RETURN_IF_ERROR(tablet->calc_segment_delete_bitmap( - rowset, segment, specified_rowsets, delete_bitmap, end_version, rowset_writer)); + rowset, segment, specified_rowsets, delete_bitmap, end_version, rowset_writer, + tablet_delete_bitmap)); } } @@ -562,7 +567,8 @@ Status BaseTablet::calc_segment_delete_bitmap(RowsetSharedPtr rowset, const segment_v2::SegmentSharedPtr& seg, const std::vector& specified_rowsets, DeleteBitmapPtr delete_bitmap, int64_t end_version, - RowsetWriter* rowset_writer) { + RowsetWriter* rowset_writer, + DeleteBitmapPtr tablet_delete_bitmap) { OlapStopWatch watch; auto rowset_id = rowset->rowset_id(); Version dummy_version(end_version + 1, end_version + 1); @@ -676,9 +682,16 @@ Status BaseTablet::calc_segment_delete_bitmap(RowsetSharedPtr rowset, } RowsetSharedPtr rowset_find; - auto st = lookup_row_key(key, rowset_schema.get(), true, specified_rowsets, &loc, - cast_set(dummy_version.first - 1), segment_caches, - &rowset_find); + Status st = Status::OK(); + if (tablet_delete_bitmap == nullptr) { + st = lookup_row_key(key, rowset_schema.get(), true, specified_rowsets, &loc, + cast_set(dummy_version.first - 1), segment_caches, + &rowset_find); + } else { + st = lookup_row_key(key, rowset_schema.get(), true, specified_rowsets, &loc, + cast_set(dummy_version.first - 1), segment_caches, + &rowset_find, true, nullptr, nullptr, tablet_delete_bitmap); + } bool expected_st = st.ok() || st.is() || st.is(); // It's a defensive DCHECK, we need to exclude some common errors to avoid core-dump // while stress test @@ -1351,7 +1364,8 @@ Status BaseTablet::check_delete_bitmap_correctness(DeleteBitmapPtr delete_bitmap } Status BaseTablet::update_delete_bitmap(const BaseTabletSPtr& self, TabletTxnInfo* txn_info, - int64_t txn_id, int64_t txn_expiration) { + int64_t txn_id, int64_t txn_expiration, + DeleteBitmapPtr tablet_delete_bitmap) { SCOPED_BVAR_LATENCY(g_tablet_update_delete_bitmap_latency); RowsetIdUnorderedSet cur_rowset_ids; RowsetIdUnorderedSet rowset_ids_to_add; @@ -1380,6 +1394,8 @@ Status BaseTablet::update_delete_bitmap(const BaseTabletSPtr& self, TabletTxnInf auto t1 = watch.get_elapse_time_us(); { + int64_t next_visible_version = txn_info->is_txn_load ? txn_info->next_visible_version + : txn_info->rowset->start_version(); std::shared_lock meta_rlock(self->_meta_lock); // tablet is under alter process. The delete bitmap will be calculated after conversion. if (self->tablet_state() == TABLET_NOTREADY) { @@ -1387,7 +1403,7 @@ Status BaseTablet::update_delete_bitmap(const BaseTabletSPtr& self, TabletTxnInf << self->tablet_id(); return Status::OK(); } - RETURN_IF_ERROR(self->get_all_rs_id_unlocked(cur_version - 1, &cur_rowset_ids)); + RETURN_IF_ERROR(self->get_all_rs_id_unlocked(next_visible_version - 1, &cur_rowset_ids)); } auto t2 = watch.get_elapse_time_us(); @@ -1402,6 +1418,15 @@ Status BaseTablet::update_delete_bitmap(const BaseTabletSPtr& self, TabletTxnInf std::shared_lock meta_rlock(self->_meta_lock); specified_rowsets = self->get_rowset_by_ids(&rowset_ids_to_add); } + if (txn_info->is_txn_load) { + for (auto invisible_rowset : txn_info->invisible_rowsets) { + specified_rowsets.emplace_back(invisible_rowset); + } + std::sort(specified_rowsets.begin(), specified_rowsets.end(), + [](RowsetSharedPtr& lhs, RowsetSharedPtr& rhs) { + return lhs->end_version() > rhs->end_version(); + }); + } auto t3 = watch.get_elapse_time_us(); // If a rowset is produced by compaction before the commit phase of the partial update load @@ -1446,7 +1471,8 @@ Status BaseTablet::update_delete_bitmap(const BaseTabletSPtr& self, TabletTxnInf auto token = self->calc_delete_bitmap_executor()->create_token(); // set rowset_writer to nullptr to skip the alignment process RETURN_IF_ERROR(calc_delete_bitmap(self, rowset, segments, rowsets_skip_alignment, - delete_bitmap, cur_version - 1, token.get(), nullptr)); + delete_bitmap, cur_version - 1, token.get(), nullptr, + tablet_delete_bitmap)); RETURN_IF_ERROR(token->wait()); } @@ -1454,13 +1480,14 @@ Status BaseTablet::update_delete_bitmap(const BaseTabletSPtr& self, TabletTxnInf // Otherwise, it will be submitted to the thread pool for calculation. if (segments.size() <= 1) { RETURN_IF_ERROR(calc_delete_bitmap(self, rowset, segments, specified_rowsets, delete_bitmap, - cur_version - 1, nullptr, transient_rs_writer.get())); + cur_version - 1, nullptr, transient_rs_writer.get(), + tablet_delete_bitmap)); } else { auto token = self->calc_delete_bitmap_executor()->create_token(); RETURN_IF_ERROR(calc_delete_bitmap(self, rowset, segments, specified_rowsets, delete_bitmap, - cur_version - 1, token.get(), - transient_rs_writer.get())); + cur_version - 1, token.get(), transient_rs_writer.get(), + tablet_delete_bitmap)); RETURN_IF_ERROR(token->wait()); } @@ -1511,8 +1538,9 @@ Status BaseTablet::update_delete_bitmap(const BaseTabletSPtr& self, TabletTxnInf segments.begin(), segments.end(), 0, [](size_t sum, const segment_v2::SegmentSharedPtr& s) { return sum += s->num_rows(); }); auto t5 = watch.get_elapse_time_us(); + int64_t lock_id = txn_info->is_txn_load ? txn_info->lock_id : -1; RETURN_IF_ERROR(self->save_delete_bitmap(txn_info, txn_id, delete_bitmap, - transient_rs_writer.get(), cur_rowset_ids)); + transient_rs_writer.get(), cur_rowset_ids, lock_id)); LOG(INFO) << "[Publish] construct delete bitmap tablet: " << self->tablet_id() << ", rowset_ids to add: " << rowset_ids_to_add.size() << ", rowset_ids to del: " << rowset_ids_to_del.size() diff --git a/be/src/olap/base_tablet.h b/be/src/olap/base_tablet.h index b6fc953e46082e..f961f4c49eedd6 100644 --- a/be/src/olap/base_tablet.h +++ b/be/src/olap/base_tablet.h @@ -156,7 +156,8 @@ class BaseTablet { std::vector>& segment_caches, RowsetSharedPtr* rowset = nullptr, bool with_rowid = true, std::string* encoded_seq_value = nullptr, - OlapReaderStatistics* stats = nullptr); + OlapReaderStatistics* stats = nullptr, + DeleteBitmapPtr tablet_delete_bitmap = nullptr); // calc delete bitmap when flush memtable, use a fake version to calc // For example, cur max version is 5, and we use version 6 to calc but @@ -169,13 +170,15 @@ class BaseTablet { const std::vector& specified_rowsets, DeleteBitmapPtr delete_bitmap, int64_t version, CalcDeleteBitmapToken* token, - RowsetWriter* rowset_writer = nullptr); + RowsetWriter* rowset_writer = nullptr, + DeleteBitmapPtr tablet_delete_bitmap = nullptr); Status calc_segment_delete_bitmap(RowsetSharedPtr rowset, const segment_v2::SegmentSharedPtr& seg, const std::vector& specified_rowsets, DeleteBitmapPtr delete_bitmap, int64_t end_version, - RowsetWriter* rowset_writer); + RowsetWriter* rowset_writer, + DeleteBitmapPtr tablet_delete_bitmap = nullptr); Status calc_delete_bitmap_between_segments( RowsetSharedPtr rowset, const std::vector& segments, @@ -235,11 +238,13 @@ class BaseTablet { int64_t txn_expiration = 0) = 0; static Status update_delete_bitmap(const BaseTabletSPtr& self, TabletTxnInfo* txn_info, - int64_t txn_id, int64_t txn_expiration = 0); + int64_t txn_id, int64_t txn_expiration = 0, + DeleteBitmapPtr tablet_delete_bitmap = nullptr); virtual Status save_delete_bitmap(const TabletTxnInfo* txn_info, int64_t txn_id, DeleteBitmapPtr delete_bitmap, RowsetWriter* rowset_writer, - const RowsetIdUnorderedSet& cur_rowset_ids) = 0; + const RowsetIdUnorderedSet& cur_rowset_ids, + int64_t lock_id = -1) = 0; virtual CalcDeleteBitmapExecutor* calc_delete_bitmap_executor() = 0; void calc_compaction_output_rowset_delete_bitmap( diff --git a/be/src/olap/calc_delete_bitmap_executor.cpp b/be/src/olap/calc_delete_bitmap_executor.cpp index 3983dc0a98642a..e45f9801f68ba4 100644 --- a/be/src/olap/calc_delete_bitmap_executor.cpp +++ b/be/src/olap/calc_delete_bitmap_executor.cpp @@ -34,7 +34,8 @@ Status CalcDeleteBitmapToken::submit(BaseTabletSPtr tablet, RowsetSharedPtr cur_ const segment_v2::SegmentSharedPtr& cur_segment, const std::vector& target_rowsets, int64_t end_version, DeleteBitmapPtr delete_bitmap, - RowsetWriter* rowset_writer) { + RowsetWriter* rowset_writer, + DeleteBitmapPtr tablet_delete_bitmap) { { std::shared_lock rlock(_lock); RETURN_IF_ERROR(_status); @@ -44,7 +45,8 @@ Status CalcDeleteBitmapToken::submit(BaseTabletSPtr tablet, RowsetSharedPtr cur_ return _thread_token->submit_func([=, this]() { SCOPED_ATTACH_TASK(_query_thread_context); auto st = tablet->calc_segment_delete_bitmap(cur_rowset, cur_segment, target_rowsets, - delete_bitmap, end_version, rowset_writer); + delete_bitmap, end_version, rowset_writer, + tablet_delete_bitmap); if (!st.ok()) { LOG(WARNING) << "failed to calc segment delete bitmap, tablet_id: " << tablet->tablet_id() << " rowset: " << cur_rowset->rowset_id() diff --git a/be/src/olap/calc_delete_bitmap_executor.h b/be/src/olap/calc_delete_bitmap_executor.h index fa1e79b7feaa19..288108b04971df 100644 --- a/be/src/olap/calc_delete_bitmap_executor.h +++ b/be/src/olap/calc_delete_bitmap_executor.h @@ -52,7 +52,8 @@ class CalcDeleteBitmapToken { Status submit(BaseTabletSPtr tablet, RowsetSharedPtr cur_rowset, const segment_v2::SegmentSharedPtr& cur_segment, const std::vector& target_rowsets, int64_t end_version, - DeleteBitmapPtr delete_bitmap, RowsetWriter* rowset_writer); + DeleteBitmapPtr delete_bitmap, RowsetWriter* rowset_writer, + DeleteBitmapPtr tablet_delete_bitmap); // wait all tasks in token to be completed. Status wait(); diff --git a/be/src/olap/tablet.cpp b/be/src/olap/tablet.cpp index 8a0e23e75b86ad..0d04984d0e06ba 100644 --- a/be/src/olap/tablet.cpp +++ b/be/src/olap/tablet.cpp @@ -2490,7 +2490,7 @@ CalcDeleteBitmapExecutor* Tablet::calc_delete_bitmap_executor() { Status Tablet::save_delete_bitmap(const TabletTxnInfo* txn_info, int64_t txn_id, DeleteBitmapPtr delete_bitmap, RowsetWriter* rowset_writer, - const RowsetIdUnorderedSet& cur_rowset_ids) { + const RowsetIdUnorderedSet& cur_rowset_ids, int64_t lock_id) { RowsetSharedPtr rowset = txn_info->rowset; int64_t cur_version = rowset->start_version(); diff --git a/be/src/olap/tablet.h b/be/src/olap/tablet.h index f5866c67641581..0b7d758ab8fd88 100644 --- a/be/src/olap/tablet.h +++ b/be/src/olap/tablet.h @@ -417,7 +417,8 @@ class Tablet final : public BaseTablet { CalcDeleteBitmapExecutor* calc_delete_bitmap_executor() override; Status save_delete_bitmap(const TabletTxnInfo* txn_info, int64_t txn_id, DeleteBitmapPtr delete_bitmap, RowsetWriter* rowset_writer, - const RowsetIdUnorderedSet& cur_rowset_ids) override; + const RowsetIdUnorderedSet& cur_rowset_ids, + int64_t lock_id = -1) override; void merge_delete_bitmap(const DeleteBitmap& delete_bitmap); bool check_all_rowset_segment(); diff --git a/be/src/olap/tablet_meta.h b/be/src/olap/tablet_meta.h index d56e529e42bf4b..fb0895604a19fe 100644 --- a/be/src/olap/tablet_meta.h +++ b/be/src/olap/tablet_meta.h @@ -236,6 +236,7 @@ class TabletMeta : public MetadataAdder { static void init_column_from_tcolumn(uint32_t unique_id, const TColumn& tcolumn, ColumnPB* column); + DeleteBitmapPtr delete_bitmap_ptr() { return _delete_bitmap; } DeleteBitmap& delete_bitmap() { return *_delete_bitmap; } bool enable_unique_key_merge_on_write() const { return _enable_unique_key_merge_on_write; } diff --git a/be/src/olap/txn_manager.h b/be/src/olap/txn_manager.h index 88ee97c5f6a3b9..1994dec949414f 100644 --- a/be/src/olap/txn_manager.h +++ b/be/src/olap/txn_manager.h @@ -87,6 +87,12 @@ struct TabletTxnInfo { std::shared_ptr publish_status; TxnPublishInfo publish_info; + // for cloud only, used to calculate delete bitmap for txn load + bool is_txn_load = false; + std::vector invisible_rowsets; + int64_t lock_id; + int64_t next_visible_version; + TxnState state {TxnState::PREPARED}; TabletTxnInfo() = default; diff --git a/cloud/src/meta-service/meta_service_txn.cpp b/cloud/src/meta-service/meta_service_txn.cpp index 32f6b56f51af4c..58930f6edfc63b 100644 --- a/cloud/src/meta-service/meta_service_txn.cpp +++ b/cloud/src/meta-service/meta_service_txn.cpp @@ -912,6 +912,69 @@ void update_tablet_stats(const StatsTabletKeyInfo& info, const TabletStats& stat } } +// process mow table, check lock and remove pending key +void process_mow_when_commit_txn( + const CommitTxnRequest* request, const std::string& instance_id, MetaServiceCode& code, + std::string& msg, std::unique_ptr& txn, + std::unordered_map>& table_id_tablet_ids) { + int64_t txn_id = request->txn_id(); + std::stringstream ss; + std::vector lock_keys; + lock_keys.reserve(request->mow_table_ids().size()); + for (auto table_id : request->mow_table_ids()) { + lock_keys.push_back(meta_delete_bitmap_update_lock_key({instance_id, table_id, -1})); + } + std::vector> lock_values; + TxnErrorCode err = txn->batch_get(&lock_values, lock_keys); + if (err != TxnErrorCode::TXN_OK) { + ss << "failed to get delete bitmap update lock key info, instance_id=" << instance_id + << " err=" << err; + msg = ss.str(); + code = cast_as(err); + LOG(WARNING) << msg << " txn_id=" << txn_id; + return; + } + size_t total_locks = lock_keys.size(); + for (size_t i = 0; i < total_locks; i++) { + int64_t table_id = request->mow_table_ids(i); + // When the key does not exist, it means the lock has been acquired + // by another transaction and successfully committed. + if (!lock_values[i].has_value()) { + ss << "get delete bitmap update lock info, lock is expired" + << " table_id=" << table_id << " key=" << hex(lock_keys[i]); + code = MetaServiceCode::LOCK_EXPIRED; + msg = ss.str(); + LOG(WARNING) << msg << " txn_id=" << txn_id; + return; + } + + DeleteBitmapUpdateLockPB lock_info; + if (!lock_info.ParseFromString(lock_values[i].value())) [[unlikely]] { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = "failed to parse DeleteBitmapUpdateLockPB"; + LOG(WARNING) << msg << " txn_id=" << txn_id; + return; + } + if (lock_info.lock_id() != request->txn_id()) { + msg = "lock is expired"; + code = MetaServiceCode::LOCK_EXPIRED; + return; + } + txn->remove(lock_keys[i]); + LOG(INFO) << "xxx remove delete bitmap lock, lock_key=" << hex(lock_keys[i]) + << " txn_id=" << txn_id; + + for (auto tablet_id : table_id_tablet_ids[table_id]) { + std::string pending_key = meta_pending_delete_bitmap_key({instance_id, tablet_id}); + txn->remove(pending_key); + LOG(INFO) << "xxx remove delete bitmap pending key, pending_key=" << hex(pending_key) + << " txn_id=" << txn_id; + } + } + lock_keys.clear(); + lock_values.clear(); +} + /** * 0. Extract txn_id from request * 1. Get db id from TxnKv with txn_id @@ -1173,61 +1236,11 @@ void commit_txn_immediately( stats.num_segs += i.num_segments(); } // for tmp_rowsets_meta - // process mow table, check lock and remove pending key - std::vector lock_keys; - lock_keys.reserve(request->mow_table_ids().size()); - for (auto table_id : request->mow_table_ids()) { - lock_keys.push_back(meta_delete_bitmap_update_lock_key({instance_id, table_id, -1})); - } - std::vector> lock_values; - err = txn->batch_get(&lock_values, lock_keys); - if (err != TxnErrorCode::TXN_OK) { - ss << "failed to get delete bitmap update lock key info, instance_id=" << instance_id - << " err=" << err; - msg = ss.str(); - code = cast_as(err); - LOG(WARNING) << msg << " txn_id=" << txn_id; + process_mow_when_commit_txn(request, instance_id, code, msg, txn, table_id_tablet_ids); + if (code != MetaServiceCode::OK) { + LOG(WARNING) << "process mow failed, txn_id=" << txn_id << " code=" << code; return; } - size_t total_locks = lock_keys.size(); - for (size_t i = 0; i < total_locks; i++) { - int64_t table_id = request->mow_table_ids(i); - // When the key does not exist, it means the lock has been acquired - // by another transaction and successfully committed. - if (!lock_values[i].has_value()) { - ss << "get delete bitmap update lock info, lock is expired" - << " table_id=" << table_id << " key=" << hex(lock_keys[i]); - code = MetaServiceCode::LOCK_EXPIRED; - msg = ss.str(); - LOG(WARNING) << msg << " txn_id=" << txn_id; - return; - } - - DeleteBitmapUpdateLockPB lock_info; - if (!lock_info.ParseFromString(lock_values[i].value())) [[unlikely]] { - code = MetaServiceCode::PROTOBUF_PARSE_ERR; - msg = "failed to parse DeleteBitmapUpdateLockPB"; - LOG(WARNING) << msg << " txn_id=" << txn_id; - return; - } - if (lock_info.lock_id() != request->txn_id()) { - msg = "lock is expired"; - code = MetaServiceCode::LOCK_EXPIRED; - return; - } - txn->remove(lock_keys[i]); - LOG(INFO) << "xxx remove delete bitmap lock, lock_key=" << hex(lock_keys[i]) - << " txn_id=" << txn_id; - - for (auto tablet_id : table_id_tablet_ids[table_id]) { - std::string pending_key = meta_pending_delete_bitmap_key({instance_id, tablet_id}); - txn->remove(pending_key); - LOG(INFO) << "xxx remove delete bitmap pending key, pending_key=" - << hex(pending_key) << " txn_id=" << txn_id; - } - } - lock_keys.clear(); - lock_values.clear(); // Save rowset meta for (auto& i : rowsets) { @@ -1810,62 +1823,12 @@ void commit_txn_eventually( response->add_versions(i.second + 1); } - // process mow table, check lock and remove pending key - std::vector lock_keys; - lock_keys.reserve(request->mow_table_ids().size()); - for (auto table_id : request->mow_table_ids()) { - lock_keys.push_back(meta_delete_bitmap_update_lock_key({instance_id, table_id, -1})); - } - std::vector> lock_values; - err = txn->batch_get(&lock_values, lock_keys); - if (err != TxnErrorCode::TXN_OK) { - ss << "failed to get delete bitmap update lock key info, instance_id=" << instance_id - << " err=" << err; - msg = ss.str(); - code = cast_as(err); - LOG(WARNING) << msg << " txn_id=" << txn_id; + process_mow_when_commit_txn(request, instance_id, code, msg, txn, table_id_tablet_ids); + if (code != MetaServiceCode::OK) { + LOG(WARNING) << "process mow failed, txn_id=" << txn_id << " code=" << code; return; } - for (size_t i = 0; i < lock_keys.size(); i++) { - int64_t table_id = request->mow_table_ids(i); - // When the key does not exist, it means the lock has been acquired - // by another transaction and successfully committed. - if (!lock_values[i].has_value()) { - ss << "get delete bitmap update lock info, lock is expired" - << " table_id=" << table_id << " key=" << hex(lock_keys[i]); - code = MetaServiceCode::LOCK_EXPIRED; - msg = ss.str(); - LOG(WARNING) << msg << " txn_id=" << txn_id; - return; - } - - DeleteBitmapUpdateLockPB lock_info; - if (!lock_info.ParseFromString(lock_values[i].value())) [[unlikely]] { - code = MetaServiceCode::PROTOBUF_PARSE_ERR; - msg = "failed to parse DeleteBitmapUpdateLockPB"; - LOG(WARNING) << msg << " txn_id=" << txn_id; - return; - } - if (lock_info.lock_id() != request->txn_id()) { - msg = "lock is expired"; - code = MetaServiceCode::LOCK_EXPIRED; - return; - } - txn->remove(lock_keys[i]); - LOG(INFO) << "xxx remove delete bitmap lock, lock_key=" << hex(lock_keys[i]) - << " txn_id=" << txn_id; - - for (auto tablet_id : table_id_tablet_ids[table_id]) { - std::string pending_key = meta_pending_delete_bitmap_key({instance_id, tablet_id}); - txn->remove(pending_key); - LOG(INFO) << "xxx remove delete bitmap pending key, pending_key=" - << hex(pending_key) << " txn_id=" << txn_id; - } - } - lock_keys.clear(); - lock_values.clear(); - // Save table versions for (auto& i : table_id_tablet_ids) { std::string ver_key = table_version_key({instance_id, db_id, i.first}); @@ -2282,6 +2245,12 @@ void commit_txn_with_sub_txn(const CommitTxnRequest* request, CommitTxnResponse* } // for tmp_rowsets_meta } + process_mow_when_commit_txn(request, instance_id, code, msg, txn, table_id_tablet_ids); + if (code != MetaServiceCode::OK) { + LOG(WARNING) << "process mow failed, txn_id=" << txn_id << " code=" << code; + return; + } + // Save rowset meta for (auto& i : rowsets) { size_t rowset_size = i.first.size() + i.second.size(); diff --git a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java index be0390db584ca9..72e6438999c396 100644 --- a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java +++ b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java @@ -3188,6 +3188,9 @@ public static int metaServiceRpcRetryTimes() { @ConfField(mutable = true, description = {"存算分离模式下calculate delete bitmap task 超时时间,默认15s"}) public static int calculate_delete_bitmap_task_timeout_seconds = 15; + @ConfField(mutable = true, description = {"存算分离模式下事务导入calculate delete bitmap task 超时时间,默认300s"}) + public static int calculate_delete_bitmap_task_timeout_seconds_for_transaction_load = 300; + @ConfField(mutable = true, description = {"存算分离模式下commit阶段等锁超时时间,默认5s"}) public static int try_commit_lock_timeout_seconds = 5; diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/CloudGlobalTransactionMgr.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/CloudGlobalTransactionMgr.java index 1a80058759bba9..3c56f7dc56a180 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/CloudGlobalTransactionMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/CloudGlobalTransactionMgr.java @@ -99,6 +99,7 @@ import org.apache.doris.thrift.TCalcDeleteBitmapPartitionInfo; import org.apache.doris.thrift.TStatus; import org.apache.doris.thrift.TStatusCode; +import org.apache.doris.thrift.TTabletCommitInfo; import org.apache.doris.thrift.TTaskType; import org.apache.doris.thrift.TUniqueId; import org.apache.doris.thrift.TWaitingTxnStatusRequest; @@ -492,7 +493,7 @@ private void commitTransaction(long dbId, List tableList, long transactio transactionState.getTransactionStatus().toString()); } } - calcDeleteBitmapForMow(dbId, mowTableList, transactionId, tabletCommitInfos); + calcDeleteBitmapForMow(dbId, mowTableList, transactionId, tabletCommitInfos, null); } CommitTxnRequest.Builder builder = CommitTxnRequest.newBuilder(); @@ -529,12 +530,17 @@ private void commitTransaction(long dbId, List
tableList, long transactio } final CommitTxnRequest commitTxnRequest = builder.build(); + executeCommitTxnRequest(commitTxnRequest, transactionId, is2PC, txnCommitAttachment); + } + + private void executeCommitTxnRequest(CommitTxnRequest commitTxnRequest, long transactionId, boolean is2PC, + TxnCommitAttachment txnCommitAttachment) throws UserException { boolean txnOperated = false; TransactionState txnState = null; TxnStateChangeCallback cb = null; long callbackId = 0L; try { - txnState = commitTxn(commitTxnRequest, transactionId, is2PC, dbId, tableList); + txnState = commitTxn(commitTxnRequest, transactionId, is2PC); txnOperated = true; if (DebugPointUtil.isEnable("CloudGlobalTransactionMgr.commitTransaction.timeout")) { throw new UserException(InternalErrorCode.DELETE_BITMAP_LOCK_ERR, @@ -558,8 +564,8 @@ private void commitTransaction(long dbId, List
tableList, long transactio } } - private TransactionState commitTxn(CommitTxnRequest commitTxnRequest, long transactionId, boolean is2PC, long dbId, - List
tableList) throws UserException { + private TransactionState commitTxn(CommitTxnRequest commitTxnRequest, long transactionId, boolean is2PC) + throws UserException { CommitTxnResponse commitTxnResponse = null; TransactionState txnState = null; int retryTime = 0; @@ -648,9 +654,9 @@ private List getMowTableList(List
tableList, List tableList, long transactionId, - List tabletCommitInfos) + List tabletCommitInfos, List subTransactionStates) throws UserException { - Map>> backendToPartitionTablets = Maps.newHashMap(); + Map>> backendToPartitionTablets = Maps.newHashMap(); Map partitions = Maps.newHashMap(); Map> tableToPartitions = Maps.newHashMap(); Map> tableToTabletList = Maps.newHashMap(); @@ -661,6 +667,8 @@ private void calcDeleteBitmapForMow(long dbId, List tableList, long t throw new UserException("The partition info is empty, table may be dropped, txnid=" + transactionId); } + Map> partitionToSubTxnIds = getPartitionSubTxnIds(subTransactionStates, tableToTabletList, + tabletToTabletMeta); Map baseCompactionCnts = Maps.newHashMap(); Map cumulativeCompactionCnts = Maps.newHashMap(); Map cumulativePoints = Maps.newHashMap(); @@ -670,9 +678,11 @@ private void calcDeleteBitmapForMow(long dbId, List tableList, long t Map> backendToPartitionInfos = getCalcDeleteBitmapInfo( backendToPartitionTablets, partitionVersions, baseCompactionCnts, cumulativeCompactionCnts, - cumulativePoints); + cumulativePoints, partitionToSubTxnIds); try { - sendCalcDeleteBitmaptask(dbId, transactionId, backendToPartitionInfos); + sendCalcDeleteBitmaptask(dbId, transactionId, backendToPartitionInfos, + subTransactionStates == null ? Config.calculate_delete_bitmap_task_timeout_seconds + : Config.calculate_delete_bitmap_task_timeout_seconds_for_transaction_load); } catch (UserException e) { LOG.warn("failed to sendCalcDeleteBitmaptask for txn=" + transactionId + ",exception=" + e.getMessage()); removeDeleteBitmapUpdateLock(tableToPartitions, transactionId); @@ -680,11 +690,34 @@ private void calcDeleteBitmapForMow(long dbId, List tableList, long t } } + private Map> getPartitionSubTxnIds(List subTransactionStates, + Map> tableToTabletList, Map tabletToTabletMeta) { + if (subTransactionStates == null) { + return null; + } + Map> partitionToSubTxnIds = Maps.newHashMap(); + for (SubTransactionState subTransactionState : subTransactionStates) { + if (!tableToTabletList.containsKey(subTransactionState.getTable().getId())) { + // skip non mow table + continue; + } + for (TTabletCommitInfo ci : subTransactionState.getTabletCommitInfos()) { + TabletMeta tabletMeta = tabletToTabletMeta.get(ci.getTabletId()); + long partitionId = tabletMeta.getPartitionId(); + List subTxnIds = partitionToSubTxnIds.computeIfAbsent(partitionId, k -> Lists.newArrayList()); + if (!subTxnIds.contains(subTransactionState.getSubTransactionId())) { + subTxnIds.add(subTransactionState.getSubTransactionId()); + } + } + } + return partitionToSubTxnIds; + } + private void getPartitionInfo(List tableList, List tabletCommitInfos, Map> tableToParttions, Map partitions, - Map>> backendToPartitionTablets, + Map>> backendToPartitionTablets, Map> tableToTabletList, Map tabletToTabletMeta) { Map tableMap = Maps.newHashMap(); @@ -697,18 +730,22 @@ private void getPartitionInfo(List tableList, TabletInvertedIndex tabletInvertedIndex = Env.getCurrentEnv().getTabletInvertedIndex(); List tabletMetaList = tabletInvertedIndex.getTabletMetaList(tabletIds); for (int i = 0; i < tabletMetaList.size(); i++) { + long tabletId = tabletIds.get(i); + if (tabletToTabletMeta.containsKey(tabletId)) { + continue; + } TabletMeta tabletMeta = tabletMetaList.get(i); long tableId = tabletMeta.getTableId(); if (!tableMap.containsKey(tableId)) { continue; } - tabletToTabletMeta.put(tabletIds.get(i), tabletMeta); + tabletToTabletMeta.put(tabletId, tabletMeta); - if (!tableToTabletList.containsKey(tableId)) { - tableToTabletList.put(tableId, Lists.newArrayList()); + List tableTabletIds = tableToTabletList.computeIfAbsent(tableId, k -> Lists.newArrayList()); + if (!tableTabletIds.contains(tabletId)) { + tableTabletIds.add(tabletId); } - tableToTabletList.get(tableId).add(tabletIds.get(i)); long partitionId = tabletMeta.getPartitionId(); long backendId = tabletCommitInfos.get(i).getBackendId(); @@ -721,11 +758,11 @@ private void getPartitionInfo(List tableList, if (!backendToPartitionTablets.containsKey(backendId)) { backendToPartitionTablets.put(backendId, Maps.newHashMap()); } - Map> partitionToTablets = backendToPartitionTablets.get(backendId); + Map> partitionToTablets = backendToPartitionTablets.get(backendId); if (!partitionToTablets.containsKey(partitionId)) { - partitionToTablets.put(partitionId, Lists.newArrayList()); + partitionToTablets.put(partitionId, Sets.newHashSet()); } - partitionToTablets.get(partitionId).add(tabletIds.get(i)); + partitionToTablets.get(partitionId).add(tabletId); partitions.putIfAbsent(partitionId, tableMap.get(tableId).getPartition(partitionId)); } } @@ -741,18 +778,18 @@ private Map getPartitionVersions(Map partitionMap) } private Map> getCalcDeleteBitmapInfo( - Map>> backendToPartitionTablets, Map partitionVersions, + Map>> backendToPartitionTablets, Map partitionVersions, Map baseCompactionCnts, Map cumulativeCompactionCnts, - Map cumulativePoints) { + Map cumulativePoints, Map> partitionToSubTxnIds) { Map> backendToPartitionInfos = Maps.newHashMap(); - for (Map.Entry>> entry : backendToPartitionTablets.entrySet()) { + for (Map.Entry>> entry : backendToPartitionTablets.entrySet()) { List partitionInfos = Lists.newArrayList(); - for (Map.Entry> partitionToTablets : entry.getValue().entrySet()) { + for (Map.Entry> partitionToTablets : entry.getValue().entrySet()) { Long partitionId = partitionToTablets.getKey(); - List tabletList = partitionToTablets.getValue(); + Set tabletList = partitionToTablets.getValue(); TCalcDeleteBitmapPartitionInfo partitionInfo = new TCalcDeleteBitmapPartitionInfo(partitionId, partitionVersions.get(partitionId), - tabletList); + Lists.newArrayList(tabletList)); if (!baseCompactionCnts.isEmpty() && !cumulativeCompactionCnts.isEmpty() && !cumulativePoints.isEmpty()) { List reqBaseCompactionCnts = Lists.newArrayList(); @@ -766,6 +803,13 @@ private Map> getCalcDeleteBitmapInfo( partitionInfo.setBaseCompactionCnts(reqBaseCompactionCnts); partitionInfo.setCumulativeCompactionCnts(reqCumulativeCompactionCnts); partitionInfo.setCumulativePoints(reqCumulativePoints); + if (partitionToSubTxnIds != null) { + List subTxnIds = partitionToSubTxnIds.get(partitionId); + if (subTxnIds != null && !subTxnIds.isEmpty()) { + partitionInfo.setSubTxnIds(subTxnIds); + LOG.debug("partitionId: {}, subTxnIds: {}", partitionId, subTxnIds); + } + } } partitionInfos.add(partitionInfo); } @@ -926,8 +970,8 @@ private void removeDeleteBitmapUpdateLock(Map> tableToParttions, } private void sendCalcDeleteBitmaptask(long dbId, long transactionId, - Map> backendToPartitionInfos) - throws UserException { + Map> backendToPartitionInfos, + long calculateDeleteBitmapTaskTimeoutSeconds) throws UserException { if (backendToPartitionInfos.isEmpty()) { return; } @@ -948,13 +992,14 @@ private void sendCalcDeleteBitmaptask(long dbId, long transactionId, // not check return value, because the add will success AgentTaskQueue.addTask(task); batchTask.addTask(task); - LOG.info("send calculate delete bitmap task to be {}, txn_id {}", entry.getKey(), transactionId); + LOG.info("send calculate delete bitmap task to be {}, txn_id {}, partitionInfos={}", entry.getKey(), + transactionId, entry.getValue()); } AgentTaskExecutor.submit(batchTask); boolean ok; try { - ok = countDownLatch.await(Config.calculate_delete_bitmap_task_timeout_seconds, TimeUnit.SECONDS); + ok = countDownLatch.await(calculateDeleteBitmapTaskTimeoutSeconds, TimeUnit.SECONDS); } catch (InterruptedException e) { LOG.warn("InterruptedException: ", e); ok = false; @@ -1043,14 +1088,42 @@ public boolean commitAndPublishTransaction(DatabaseIf db, long transactionId, "disable_load_job is set to true, all load jobs are not allowed"); } LOG.info("try to commit transaction, txnId: {}, subTxnStates: {}", transactionId, subTransactionStates); + + Preconditions.checkState(db instanceof Database); + List tableIdList = subTransactionStates.stream().map(s -> s.getTable().getId()).distinct() + .collect(Collectors.toList()); + List
tableList = ((Database) db).getTablesOnIdOrderOrThrowException(tableIdList); + beforeCommitTransaction(tableList, transactionId, timeoutMillis); + try { + commitTransactionWithSubTxns(db.getId(), tableList, transactionId, subTransactionStates); + } finally { + decreaseWaitingLockCount(tableList); + MetaLockUtils.commitUnlockTables(tableList); + } + return true; + } + + private void commitTransactionWithSubTxns(long dbId, List
tableList, long transactionId, + List subTransactionStates) throws UserException { + List tabletCommitInfos = subTransactionStates.stream().map( + SubTransactionState::getTabletCommitInfos).flatMap(Collection::stream) + .map(c -> new TabletCommitInfo(c.getTabletId(), c.getBackendId())).collect(Collectors.toList()); + List mowTableList = getMowTableList(tableList, tabletCommitInfos); + if (!mowTableList.isEmpty()) { + calcDeleteBitmapForMow(dbId, mowTableList, transactionId, tabletCommitInfos, subTransactionStates); + } + cleanSubTransactions(transactionId); CommitTxnRequest.Builder builder = CommitTxnRequest.newBuilder(); - builder.setDbId(db.getId()) + builder.setDbId(dbId) .setTxnId(transactionId) .setIs2Pc(false) .setCloudUniqueId(Config.cloud_unique_id) .setIsTxnLoad(true) .setEnableTxnLazyCommit(Config.enable_cloud_txn_lazy_commit); + for (OlapTable olapTable : mowTableList) { + builder.addMowTableIds(olapTable.getId()); + } // add sub txn infos for (SubTransactionState subTransactionState : subTransactionStates) { builder.addSubTxnInfos(SubTxnInfo.newBuilder().setSubTxnId(subTransactionState.getSubTransactionId()) @@ -1064,31 +1137,12 @@ public boolean commitAndPublishTransaction(DatabaseIf db, long transactionId, } final CommitTxnRequest commitTxnRequest = builder.build(); - TransactionState txnState = null; - boolean txnOperated = false; - try { - txnState = commitTxn(commitTxnRequest, transactionId, false, db.getId(), - subTransactionStates.stream().map(SubTransactionState::getTable) - .collect(Collectors.toList())); - txnOperated = true; - } finally { - if (txnState != null) { - TxnStateChangeCallback cb = callbackFactory.getCallback(txnState.getCallbackId()); - if (cb != null) { - LOG.info("commitTxn, run txn callback, transactionId:{} callbackId:{}, txnState:{}", - txnState.getTransactionId(), txnState.getCallbackId(), txnState); - cb.afterCommitted(txnState, txnOperated); - cb.afterVisible(txnState, txnOperated); - } - } - } - return true; + executeCommitTxnRequest(commitTxnRequest, transactionId, false, null); } - @Override - public boolean commitAndPublishTransaction(DatabaseIf db, List
tableList, long transactionId, - List tabletCommitInfos, long timeoutMillis, - TxnCommitAttachment txnCommitAttachment) throws UserException { + // add some log and get commit lock, mainly used for mow tables + private void beforeCommitTransaction(List
tableList, long transactionId, long timeoutMillis) + throws UserException { for (int i = 0; i < tableList.size(); i++) { long tableId = tableList.get(i).getId(); LOG.info("start commit txn=" + transactionId + ",table=" + tableId); @@ -1107,6 +1161,13 @@ public boolean commitAndPublishTransaction(DatabaseIf db, List
tableList, "get table cloud commit lock timeout, tableList=(" + StringUtils.join(tableList, ",") + ")"); } + } + + @Override + public boolean commitAndPublishTransaction(DatabaseIf db, List
tableList, long transactionId, + List tabletCommitInfos, long timeoutMillis, + TxnCommitAttachment txnCommitAttachment) throws UserException { + beforeCommitTransaction(tableList, transactionId, timeoutMillis); try { commitTransaction(db.getId(), tableList, transactionId, tabletCommitInfos, txnCommitAttachment); } finally { diff --git a/fe/fe-core/src/main/java/org/apache/doris/transaction/TransactionEntry.java b/fe/fe-core/src/main/java/org/apache/doris/transaction/TransactionEntry.java index 25c4ff4b3b2493..8a75dd0d0c2303 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/transaction/TransactionEntry.java +++ b/fe/fe-core/src/main/java/org/apache/doris/transaction/TransactionEntry.java @@ -21,8 +21,6 @@ import org.apache.doris.catalog.Database; import org.apache.doris.catalog.DatabaseIf; import org.apache.doris.catalog.Env; -import org.apache.doris.catalog.KeysType; -import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.Table; import org.apache.doris.catalog.TableIf; import org.apache.doris.cloud.transaction.CloudGlobalTransactionMgr; @@ -206,13 +204,6 @@ public long beginTransaction(TableIf table, SubTransactionType subTransactionTyp throw new AnalysisException( "Transaction insert can not insert into values and insert into select at the same time"); } - if (Config.isCloudMode()) { - OlapTable olapTable = (OlapTable) table; - if (olapTable.getKeysType() == KeysType.UNIQUE_KEYS && olapTable.getEnableUniqueKeyMergeOnWrite()) { - throw new UserException( - "Transaction load is not supported for merge on write unique keys table in cloud mode"); - } - } DatabaseIf database = table.getDatabase(); if (!isTransactionBegan) { long timeoutSecond = ConnectContext.get().getExecTimeout(); diff --git a/gensrc/thrift/AgentService.thrift b/gensrc/thrift/AgentService.thrift index abffd176ef8ea7..fdbf4483bf8c4c 100644 --- a/gensrc/thrift/AgentService.thrift +++ b/gensrc/thrift/AgentService.thrift @@ -440,6 +440,7 @@ struct TCalcDeleteBitmapPartitionInfo { 4: optional list base_compaction_cnts 5: optional list cumulative_compaction_cnts 6: optional list cumulative_points + 7: optional list sub_txn_ids } struct TCalcDeleteBitmapRequest { diff --git a/regression-test/suites/insert_p0/transaction/txn_insert.groovy b/regression-test/suites/insert_p0/transaction/txn_insert.groovy index 6653c05740e3b1..a4868ca6b11ffe 100644 --- a/regression-test/suites/insert_p0/transaction/txn_insert.groovy +++ b/regression-test/suites/insert_p0/transaction/txn_insert.groovy @@ -584,19 +584,10 @@ suite("txn_insert") { } catch (Exception e) { logger.info("exception: " + e) sql """ rollback """ - if (isCloudMode()) { - assertTrue(e.getMessage().contains("Transaction load is not supported for merge on write unique keys table in cloud mode")) - } else { - assertTrue(false, "should not reach here") - } + assertTrue(false, "should not reach here") } } - // the following cases are not supported in cloud mode - if (isCloudMode()) { - break - } - // 16. update stmt(mow table) if (use_nereids_planner) { def ut_table = "txn_insert_ut" diff --git a/regression-test/suites/insert_p0/transaction/txn_insert_concurrent_insert_mow.groovy b/regression-test/suites/insert_p0/transaction/txn_insert_concurrent_insert_mow.groovy new file mode 100644 index 00000000000000..8e6de4dd9e9a38 --- /dev/null +++ b/regression-test/suites/insert_p0/transaction/txn_insert_concurrent_insert_mow.groovy @@ -0,0 +1,135 @@ +// 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. + +import java.sql.Connection +import java.sql.DriverManager +import java.sql.Statement +import java.util.concurrent.TimeUnit +import java.util.concurrent.CompletableFuture + +suite("txn_insert_concurrent_insert_mow") { + def tableName = "txn_insert_concurrent_insert_mow" + List errors = new ArrayList<>() + + for (int i = 0; i < 3; i++) { + def table_name = "${tableName}_${i}" + sql """ drop table if exists ${table_name} """ + sql """ + CREATE TABLE IF NOT EXISTS ${table_name} ( + L_ORDERKEY INTEGER NOT NULL, + L_PARTKEY INTEGER NOT NULL, + L_SUPPKEY INTEGER NOT NULL, + L_LINENUMBER INTEGER NOT NULL, + L_QUANTITY DECIMAL(15,2) NOT NULL, + L_EXTENDEDPRICE DECIMAL(15,2) NOT NULL, + L_DISCOUNT DECIMAL(15,2) NOT NULL, + L_TAX DECIMAL(15,2) NOT NULL, + L_RETURNFLAG CHAR(1) NOT NULL, + L_LINESTATUS CHAR(1) NOT NULL, + L_SHIPDATE DATE NOT NULL, + L_COMMITDATE DATE NOT NULL, + L_RECEIPTDATE DATE NOT NULL, + L_SHIPINSTRUCT CHAR(25) NOT NULL, + L_SHIPMODE CHAR(10) NOT NULL, + L_COMMENT VARCHAR(44) NOT NULL + ) + UNIQUE KEY(L_ORDERKEY, L_PARTKEY, L_SUPPKEY, L_LINENUMBER) + DISTRIBUTED BY HASH(L_ORDERKEY) BUCKETS 3 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + if (i == 0) { + continue + } + + streamLoad { + table table_name + set 'column_separator', '|' + set 'compress_type', 'GZ' + set 'columns', "l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag,l_linestatus, l_shipdate,l_commitdate,l_receiptdate,l_shipinstruct,l_shipmode,l_comment,temp" + file """${getS3Url()}/regression/tpch/sf0.1/lineitem.tbl.gz""" + + time 10000 // limit inflight 10s + check { result, exception, startTime, endTime -> + if (exception != null) { + throw exception + } + log.info("Stream load result: ${result}".toString()) + def json = parseJson(result) + assertEquals("success", json.Status.toLowerCase()) + assertEquals(json.NumberTotalRows, json.NumberLoadedRows) + assertTrue(json.NumberLoadedRows > 0 && json.LoadBytes > 0) + } + } + } + sql """ sync """ + + def dbName = "regression_test_insert_p0_transaction" + def url = getServerPrepareJdbcUrl(context.config.jdbcUrl, dbName).replace("&useServerPrepStmts=true", "") + "&useLocalSessionState=true" + logger.info("url: ${url}") + + def sqls = [ + "begin", + "insert into ${tableName}_0 select * from ${tableName}_1 where L_ORDERKEY < 30000;", + "insert into ${tableName}_1 select * from ${tableName}_2 where L_ORDERKEY > 500000;", + "insert into ${tableName}_0 select * from ${tableName}_2 where L_ORDERKEY < 30000;", + "commit" + ] + def txn_insert = { -> + try (Connection conn = DriverManager.getConnection(url, context.config.jdbcUser, context.config.jdbcPassword); + Statement stmt = conn.createStatement()) { + for (def sql : sqls) { + logger.info(Thread.currentThread().getName() + " execute sql: " + sql) + stmt.execute(sql) + } + logger.info("finish txn insert for " + Thread.currentThread().getName()) + } catch (Throwable e) { + logger.error("txn insert failed", e) + errors.add("txn insert failed " + e.getMessage()) + } + } + + List> futures = new ArrayList<>() + for (int i = 0; i < 10; i++) { + CompletableFuture future = CompletableFuture.runAsync(txn_insert) + futures.add(future) + } + CompletableFuture[] futuresArray = futures.toArray(new CompletableFuture[0]) + CompletableFuture.allOf(futuresArray).get(10, TimeUnit.MINUTES) + sql """ sync """ + + logger.info("error num: " + errors.size() + ", errors: " + errors) + + def result = sql """ select count() from ${tableName}_0 """ + logger.info("result: ${result}") + assertEquals(30209, result[0][0]) + result = sql """ select count() from ${tableName}_1 """ + logger.info("result: ${result}") + assertEquals(600572, result[0][0]) + + def db_name = "regression_test_insert_p0_transaction" + def tables = sql """ show tables from $db_name """ + logger.info("tables: $tables") + for (def table_info : tables) { + def table_name = table_info[0] + if (table_name.startsWith(tableName)) { + check_table_version_continuous(db_name, table_name) + } + } +} diff --git a/regression-test/suites/insert_p2/txn_insert_concurrent_insert_aggregate.groovy b/regression-test/suites/insert_p2/transaction/txn_insert_concurrent_insert_aggregate.groovy similarity index 99% rename from regression-test/suites/insert_p2/txn_insert_concurrent_insert_aggregate.groovy rename to regression-test/suites/insert_p2/transaction/txn_insert_concurrent_insert_aggregate.groovy index 1fbe18bf212c47..fa57c5501187bc 100644 --- a/regression-test/suites/insert_p2/txn_insert_concurrent_insert_aggregate.groovy +++ b/regression-test/suites/insert_p2/transaction/txn_insert_concurrent_insert_aggregate.groovy @@ -81,7 +81,7 @@ suite("txn_insert_concurrent_insert_aggregate") { } sql """ sync """ - def dbName = "regression_test_insert_p2" + def dbName = "regression_test_insert_p2_transaction" def url = getServerPrepareJdbcUrl(context.config.jdbcUrl, dbName).replace("&useServerPrepStmts=true", "") + "&useLocalSessionState=true" logger.info("url: ${url}") diff --git a/regression-test/suites/insert_p2/txn_insert_concurrent_insert_duplicate.groovy b/regression-test/suites/insert_p2/transaction/txn_insert_concurrent_insert_duplicate.groovy similarity index 99% rename from regression-test/suites/insert_p2/txn_insert_concurrent_insert_duplicate.groovy rename to regression-test/suites/insert_p2/transaction/txn_insert_concurrent_insert_duplicate.groovy index 048a07fb817fff..e771078f1fbecb 100644 --- a/regression-test/suites/insert_p2/txn_insert_concurrent_insert_duplicate.groovy +++ b/regression-test/suites/insert_p2/transaction/txn_insert_concurrent_insert_duplicate.groovy @@ -81,7 +81,7 @@ suite("txn_insert_concurrent_insert_duplicate") { } sql """ sync """ - def dbName = "regression_test_insert_p2" + def dbName = "regression_test_insert_p2_transaction" def url = getServerPrepareJdbcUrl(context.config.jdbcUrl, dbName).replace("&useServerPrepStmts=true", "") + "&useLocalSessionState=true" logger.info("url: ${url}") diff --git a/regression-test/suites/insert_p2/txn_insert_concurrent_insert_mor.groovy b/regression-test/suites/insert_p2/transaction/txn_insert_concurrent_insert_mor.groovy similarity index 99% rename from regression-test/suites/insert_p2/txn_insert_concurrent_insert_mor.groovy rename to regression-test/suites/insert_p2/transaction/txn_insert_concurrent_insert_mor.groovy index c67119328ef7a6..418992835d0551 100644 --- a/regression-test/suites/insert_p2/txn_insert_concurrent_insert_mor.groovy +++ b/regression-test/suites/insert_p2/transaction/txn_insert_concurrent_insert_mor.groovy @@ -82,7 +82,7 @@ suite("txn_insert_concurrent_insert_mor") { } sql """ sync """ - def dbName = "regression_test_insert_p2" + def dbName = "regression_test_insert_p2_transaction" def url = getServerPrepareJdbcUrl(context.config.jdbcUrl, dbName).replace("&useServerPrepStmts=true", "") + "&useLocalSessionState=true" logger.info("url: ${url}") diff --git a/regression-test/suites/insert_p2/txn_insert_concurrent_insert_mow.groovy b/regression-test/suites/insert_p2/transaction/txn_insert_concurrent_insert_mow.groovy similarity index 95% rename from regression-test/suites/insert_p2/txn_insert_concurrent_insert_mow.groovy rename to regression-test/suites/insert_p2/transaction/txn_insert_concurrent_insert_mow.groovy index f8a971db75ee2f..4d6e297cac918b 100644 --- a/regression-test/suites/insert_p2/txn_insert_concurrent_insert_mow.groovy +++ b/regression-test/suites/insert_p2/transaction/txn_insert_concurrent_insert_mow.groovy @@ -22,11 +22,6 @@ import java.util.concurrent.TimeUnit import java.util.concurrent.CompletableFuture suite("txn_insert_concurrent_insert_mow") { - if (isCloudMode()) { - logger.info("cloud txn load does not support mow") - return - } - def tableName = "txn_insert_concurrent_insert_mow" List errors = new ArrayList<>() @@ -85,8 +80,12 @@ suite("txn_insert_concurrent_insert_mow") { } } sql """ sync """ + def t2_row_count = 6001215 + def result = sql """ select count() from ${tableName}_2 """ + logger.info("${tableName}_2 row count: ${result}, expected: ${t2_row_count}") + assertEquals(t2_row_count, result[0][0] as int) - def dbName = "regression_test_insert_p2" + def dbName = "regression_test_insert_p2_transaction" def url = getServerPrepareJdbcUrl(context.config.jdbcUrl, dbName).replace("&useServerPrepStmts=true", "") + "&useLocalSessionState=true" logger.info("url: ${url}") @@ -156,7 +155,7 @@ suite("txn_insert_concurrent_insert_mow") { logger.info("error num: " + errors.size() + ", errors: " + errors) def t0_row_count = 6001215 - def result = sql """ select count() from ${tableName}_0 """ + result = sql """ select count() from ${tableName}_0 """ logger.info("${tableName}_0 row count: ${result}, expected: ${t0_row_count}") def t1_row_count = 2999666 diff --git a/regression-test/suites/insert_p2/txn_insert_concurrent_insert_ud.groovy b/regression-test/suites/insert_p2/transaction/txn_insert_concurrent_insert_ud.groovy similarity index 98% rename from regression-test/suites/insert_p2/txn_insert_concurrent_insert_ud.groovy rename to regression-test/suites/insert_p2/transaction/txn_insert_concurrent_insert_ud.groovy index a524703f9ef99f..d0b27641c64a99 100644 --- a/regression-test/suites/insert_p2/txn_insert_concurrent_insert_ud.groovy +++ b/regression-test/suites/insert_p2/transaction/txn_insert_concurrent_insert_ud.groovy @@ -23,11 +23,6 @@ import java.util.concurrent.CompletableFuture // test update and delete command suite("txn_insert_concurrent_insert_ud") { - if (isCloudMode()) { - logger.info("cloud txn load does not support mow") - return - } - def tableName = "txn_insert_concurrent_insert_ud" List errors = new ArrayList<>() @@ -88,7 +83,7 @@ suite("txn_insert_concurrent_insert_ud") { } sql """ sync """ - def dbName = "regression_test_insert_p2" + def dbName = "regression_test_insert_p2_transaction" def url = getServerPrepareJdbcUrl(context.config.jdbcUrl, dbName).replace("&useServerPrepStmts=true", "") + "&useLocalSessionState=true" logger.info("url: ${url}") diff --git a/regression-test/suites/insert_p2/txn_insert_concurrent_insert_update.groovy b/regression-test/suites/insert_p2/transaction/txn_insert_concurrent_insert_update.groovy similarity index 99% rename from regression-test/suites/insert_p2/txn_insert_concurrent_insert_update.groovy rename to regression-test/suites/insert_p2/transaction/txn_insert_concurrent_insert_update.groovy index b467a87de8201c..eba69918660caf 100644 --- a/regression-test/suites/insert_p2/txn_insert_concurrent_insert_update.groovy +++ b/regression-test/suites/insert_p2/transaction/txn_insert_concurrent_insert_update.groovy @@ -84,7 +84,7 @@ suite("txn_insert_concurrent_insert_update") { } sql """ sync """ - def dbName = "regression_test_insert_p2" + def dbName = "regression_test_insert_p2_transaction" def url = getServerPrepareJdbcUrl(context.config.jdbcUrl, dbName).replace("&useServerPrepStmts=true", "") + "&useLocalSessionState=true" logger.info("url: ${url}") diff --git a/regression-test/suites/insert_p2/txn_insert_with_schema_change.groovy b/regression-test/suites/insert_p2/transaction/txn_insert_with_schema_change.groovy similarity index 99% rename from regression-test/suites/insert_p2/txn_insert_with_schema_change.groovy rename to regression-test/suites/insert_p2/transaction/txn_insert_with_schema_change.groovy index 56692b68d3730d..34b859b5e80d0a 100644 --- a/regression-test/suites/insert_p2/txn_insert_with_schema_change.groovy +++ b/regression-test/suites/insert_p2/transaction/txn_insert_with_schema_change.groovy @@ -24,7 +24,7 @@ import java.util.concurrent.TimeUnit // schema change and modify replica num suite("txn_insert_with_schema_change") { def tableName = "txn_insert_with_schema_change" - def dbName = "regression_test_insert_p2" + def dbName = "regression_test_insert_p2_transaction" def url = getServerPrepareJdbcUrl(context.config.jdbcUrl, dbName).replace("&useServerPrepStmts=true", "") + "&useLocalSessionState=true" logger.info("url: ${url}") From 267a8cc9ea9e25cf66d7e1b0be645cf75e5100ce Mon Sep 17 00:00:00 2001 From: meiyi Date: Tue, 26 Nov 2024 23:08:44 +0800 Subject: [PATCH 014/399] [fix](cluster key) add cluster key case (#44488) --- .../java/org/apache/doris/catalog/Table.java | 5 +- .../translator/PhysicalPlanTranslator.java | 2 +- .../data/nereids_p0/delete/delete_cte_ck.out | 29 +++++ .../nereids_p0/delete/delete_cte_ck.groovy | 123 ++++++++++++++++++ 4 files changed, 156 insertions(+), 3 deletions(-) create mode 100644 regression-test/data/nereids_p0/delete/delete_cte_ck.out create mode 100644 regression-test/suites/nereids_p0/delete/delete_cte_ck.groovy diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java index d85d98a8ea550f..ed37a936e10819 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java @@ -40,6 +40,7 @@ import com.google.common.base.Preconditions; import com.google.common.base.Strings; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; @@ -393,7 +394,7 @@ public TableType getType() { } public List getFullSchema() { - return fullSchema; + return ImmutableList.copyOf(fullSchema); } // should override in subclass if necessary @@ -403,7 +404,7 @@ public List getBaseSchema() { public List getBaseSchema(boolean full) { if (full) { - return fullSchema; + return ImmutableList.copyOf(fullSchema); } else { return fullSchema.stream().filter(Column::isVisible).collect(Collectors.toList()); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java index 654ccc8ca1155a..56ae65ec722941 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java @@ -2695,7 +2695,7 @@ private boolean checkPushSort(SortNode sortNode, OlapTable olapTable) { if (sortExprs.size() > olapTable.getDataSortInfo().getColNum()) { return false; } - List sortKeyColumns = olapTable.getFullSchema(); + List sortKeyColumns = new ArrayList<>(olapTable.getFullSchema()); if (olapTable.getEnableUniqueKeyMergeOnWrite()) { Map clusterKeyMap = new TreeMap<>(); for (Column column : olapTable.getFullSchema()) { diff --git a/regression-test/data/nereids_p0/delete/delete_cte_ck.out b/regression-test/data/nereids_p0/delete/delete_cte_ck.out new file mode 100644 index 00000000000000..2734e5b4a33b33 --- /dev/null +++ b/regression-test/data/nereids_p0/delete/delete_cte_ck.out @@ -0,0 +1,29 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !sql -- +1 \N 2 1 1.0 \N +1 10 1 1 1.0 2000-01-01 +2 \N 4 2 2.0 \N +2 20 2 2 2.0 2000-01-02 +3 \N 6 3 3.0 \N +3 30 3 3 3.0 2000-01-03 + +-- !sql -- +2 \N 4 2 2.0 \N +2 20 2 2 2.0 2000-01-02 +3 \N 6 3 3.0 \N +3 30 3 3 3.0 2000-01-03 + +-- !sql -- +1 \N 2 1 1.0 \N +1 10 1 1 1.0 2000-01-01 +2 \N 4 2 2.0 \N +2 20 2 2 2.0 2000-01-02 +3 \N 6 3 3.0 \N +3 30 3 3 3.0 2000-01-03 + +-- !sql -- +2 \N 4 2 2.0 \N +2 20 2 2 2.0 2000-01-02 +3 \N 6 3 3.0 \N +3 30 3 3 3.0 2000-01-03 + diff --git a/regression-test/suites/nereids_p0/delete/delete_cte_ck.groovy b/regression-test/suites/nereids_p0/delete/delete_cte_ck.groovy new file mode 100644 index 00000000000000..2e9bd1f8fb3fa9 --- /dev/null +++ b/regression-test/suites/nereids_p0/delete/delete_cte_ck.groovy @@ -0,0 +1,123 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite('nereids_delete_cte_ck') { + + String db = context.config.getDbNameByFile(context.file) + sql "select 1;" // to create database + + for (def use_row_store : [false, true]) { + logger.info("current params: use_row_store: ${use_row_store}") + + connect(user = context.config.jdbcUser, password = context.config.jdbcPassword, url = context.config.jdbcUrl) { + sql "use ${db};" + + def t1 = 't1_cte_ck' + def t2 = 't2_cte_ck' + def t3 = 't3_cte_ck' + + sql "drop table if exists ${t1}" + sql """ + create table ${t1} ( + id int, + id1 int, + c1 bigint, + c2 string, + c3 double, + c4 date + ) unique key (id, id1) + CLUSTER BY (`id1`, `id`, `c4`) + distributed by hash(id, id1) + properties( + "replication_num"="1", + "enable_unique_key_merge_on_write" = "true", + "store_row_column" = "${use_row_store}"); """ + + sql "drop table if exists ${t2}" + sql """ + create table ${t2} ( + id int, + c1 bigint, + c2 string, + c3 double, + c4 date + ) unique key (id) + CLUSTER BY (`c4`, `c1`) + distributed by hash(id) + properties( + "replication_num"="1" + ); + """ + + sql "drop table if exists ${t3}" + sql """ + create table ${t3} ( + id int + ) distributed by hash(id) + properties( + "replication_num"="1" + ); + """ + + sql """ + INSERT INTO ${t1} VALUES + (1, 10, 1, '1', 1.0, '2000-01-01'), + (2, 20, 2, '2', 2.0, '2000-01-02'), + (3, 30, 3, '3', 3.0, '2000-01-03'); + """ + + sql """ + + INSERT INTO ${t2} VALUES + (1, 10, '10', 10.0, '2000-01-10'), + (2, 20, '20', 20.0, '2000-01-20'), + (3, 30, '30', 30.0, '2000-01-30'), + (4, 4, '4', 4.0, '2000-01-04'), + (5, 5, '5', 5.0, '2000-01-05'); + """ + + sql """ + INSERT INTO ${t3} VALUES + (1), + (4), + (5); + """ + + sql "set enable_nereids_planner=true" + sql "set enable_fallback_to_original_planner=false" + sql "set enable_nereids_dml=true" + + sql "insert into ${t1}(id, c1, c2, c3) select id, c1 * 2, c2, c3 from ${t1}" + sql "insert into ${t2}(id, c1, c2, c3) select id, c1, c2 * 2, c3 from ${t2}" + sql "insert into ${t2}(c1, c3) select c1 + 1, c3 + 1 from (select id, c1, c3 from ${t1} order by id, c1 limit 10) ${t1}, ${t3}" + + qt_sql "select * from ${t1} order by id, id1" + + sql """ + with cte as (select * from ${t3}) + delete from ${t1} + using ${t2} join cte on ${t2}.id = cte.id + where ${t1}.id = ${t2}.id; + """ + + qt_sql "select * from ${t1} order by id, id1" + + // test cte != null and using == null + sql """with cte as (select 1) delete from ${t1} where ${t1}.id = 100""" + } + } +} \ No newline at end of file From 4d66933819192b664b840f1770099e558066473b Mon Sep 17 00:00:00 2001 From: walter Date: Tue, 26 Nov 2024 23:25:48 +0800 Subject: [PATCH 015/399] [enhancement](backup) Add config backup_handler_update_interval_millis (#44628) --- .../src/main/java/org/apache/doris/common/Config.java | 6 ++++++ .../main/java/org/apache/doris/backup/BackupHandler.java | 2 +- 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java index 72e6438999c396..9650f73be50adc 100644 --- a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java +++ b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java @@ -1634,6 +1634,12 @@ public class Config extends ConfigBase { @ConfField(mutable = true, masterOnly = true) public static boolean ignore_backup_not_support_table_type = false; + /** + * A internal config, to control the update interval of backup handler. Only used to speed up tests. + */ + @ConfField(mutable = false) + public static long backup_handler_update_interval_millis = 3000; + /** * Control the default max num of the instance for a user. */ diff --git a/fe/fe-core/src/main/java/org/apache/doris/backup/BackupHandler.java b/fe/fe-core/src/main/java/org/apache/doris/backup/BackupHandler.java index 7cad2f985001bc..d70544add98747 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/backup/BackupHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/backup/BackupHandler.java @@ -122,7 +122,7 @@ public BackupHandler() { } public BackupHandler(Env env) { - super("backupHandler", 3000L); + super("backupHandler", Config.backup_handler_update_interval_millis); this.env = env; } From 0a61efaa2fe1efee7ef3a117e62c680695c7be69 Mon Sep 17 00:00:00 2001 From: walter Date: Tue, 26 Nov 2024 23:26:10 +0800 Subject: [PATCH 016/399] [fix](backup) Automatic adapt upload/download snapshot batch size (#44560) The original `backup_upload_task_num_per_be` and `restore_download_task_num_per_be` would not adapt the different num of snapshots, a large `UploadTask` or `DownloadTask` might exceed the threshold of `thrift_max_message_size` and failed forever. This PR changes these options to adapt the number of snapshots automatically. --- .../java/org/apache/doris/common/Config.java | 13 ++++--- .../org/apache/doris/backup/BackupJob.java | 17 +++------ .../org/apache/doris/backup/RestoreJob.java | 37 ++++++------------- 3 files changed, 25 insertions(+), 42 deletions(-) diff --git a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java index 9650f73be50adc..66dbff5abed8e6 100644 --- a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java +++ b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java @@ -2771,16 +2771,17 @@ public class Config extends ConfigBase { public static String nereids_trace_log_dir = System.getenv("LOG_DIR") + "/nereids_trace"; @ConfField(mutable = true, masterOnly = true, description = { - "备份过程中,分配给每个be的upload任务最大个数,默认值为3个。", - "The max number of upload tasks assigned to each be during the backup process, the default value is 3." + "备份过程中,一个 upload 任务上传的快照数量上限,默认值为10个", + "The max number of snapshots assigned to a upload task during the backup process, the default value is 10." }) - public static int backup_upload_task_num_per_be = 3; + public static int backup_upload_snapshot_batch_size = 10; @ConfField(mutable = true, masterOnly = true, description = { - "恢复过程中,分配给每个be的download任务最大个数,默认值为3个。", - "The max number of download tasks assigned to each be during the restore process, the default value is 3." + "恢复过程中,一个 download 任务下载的快照数量上限,默认值为10个", + "The max number of snapshots assigned to a download task during the restore process, " + + "the default value is 10." }) - public static int restore_download_task_num_per_be = 3; + public static int restore_download_snapshot_batch_size = 10; @ConfField(mutable = true, masterOnly = true, description = { "备份恢复过程中,单次 RPC 分配给每个be的任务最大个数,默认值为10000个。", diff --git a/fe/fe-core/src/main/java/org/apache/doris/backup/BackupJob.java b/fe/fe-core/src/main/java/org/apache/doris/backup/BackupJob.java index 478e8902d7d8c4..621a2b1d9f7d29 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/backup/BackupJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/backup/BackupJob.java @@ -751,13 +751,10 @@ private void uploadSnapshot() { for (Long beId : beToSnapshots.keySet()) { List infos = beToSnapshots.get(beId); int totalNum = infos.size(); - int batchNum = totalNum; - if (Config.backup_upload_task_num_per_be > 0) { - batchNum = Math.min(totalNum, Config.backup_upload_task_num_per_be); - } // each task contains several upload sub tasks - int taskNumPerBatch = Math.max(totalNum / batchNum, 1); - LOG.info("backend {} has {} batch, total {} tasks, {}", beId, batchNum, totalNum, this); + int taskNumPerBatch = Config.backup_upload_snapshot_batch_size; + LOG.info("backend {} has total {} snapshots, per task batch size {}, {}", + beId, totalNum, taskNumPerBatch, this); List brokers = Lists.newArrayList(); Status st = repo.getBrokerAddress(beId, env, brokers); @@ -768,12 +765,10 @@ private void uploadSnapshot() { Preconditions.checkState(brokers.size() == 1); // allot tasks - int index = 0; - for (int batch = 0; batch < batchNum; batch++) { + for (int index = 0; index < totalNum; index += taskNumPerBatch) { Map srcToDest = Maps.newHashMap(); - int currentBatchTaskNum = (batch == batchNum - 1) ? totalNum - index : taskNumPerBatch; - for (int j = 0; j < currentBatchTaskNum; j++) { - SnapshotInfo info = infos.get(index++); + for (int j = 0; j < taskNumPerBatch && index + j < totalNum; j++) { + SnapshotInfo info = infos.get(index + j); String src = info.getTabletPath(); String dest = repo.getRepoTabletPathBySnapshotInfo(label, info); if (dest == null) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java b/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java index aef09bd78beb8f..0b5b0cb03390f3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java @@ -1687,16 +1687,10 @@ private void downloadRemoteSnapshots() { for (Long beId : beToSnapshots.keySet()) { List beSnapshotInfos = beToSnapshots.get(beId); int totalNum = beSnapshotInfos.size(); - int batchNum = totalNum; - if (Config.restore_download_task_num_per_be > 0) { - batchNum = Math.min(totalNum, Config.restore_download_task_num_per_be); - } // each task contains several upload sub tasks - int taskNumPerBatch = Math.max(totalNum / batchNum, 1); - if (LOG.isDebugEnabled()) { - LOG.debug("backend {} has {} batch, total {} tasks, {}", - beId, batchNum, totalNum, this); - } + int taskNumPerBatch = Config.restore_download_snapshot_batch_size; + LOG.info("backend {} has total {} snapshots, per task batch size {}, {}", + beId, totalNum, taskNumPerBatch, this); List brokerAddrs = null; brokerAddrs = Lists.newArrayList(); @@ -1708,12 +1702,10 @@ private void downloadRemoteSnapshots() { Preconditions.checkState(brokerAddrs.size() == 1); // allot tasks - int index = 0; - for (int batch = 0; batch < batchNum; batch++) { + for (int index = 0; index < totalNum; index += taskNumPerBatch) { Map srcToDest = Maps.newHashMap(); - int currentBatchTaskNum = (batch == batchNum - 1) ? totalNum - index : taskNumPerBatch; - for (int j = 0; j < currentBatchTaskNum; j++) { - SnapshotInfo info = beSnapshotInfos.get(index++); + for (int j = 0; j < taskNumPerBatch && index + j < totalNum; j++) { + SnapshotInfo info = beSnapshotInfos.get(index + j); Table tbl = db.getTableNullable(info.getTblId()); if (tbl == null) { status = new Status(ErrCode.NOT_FOUND, "restored table " @@ -1847,22 +1839,17 @@ private void downloadLocalSnapshots() { for (Long beId : beToSnapshots.keySet()) { List beSnapshotInfos = beToSnapshots.get(beId); int totalNum = beSnapshotInfos.size(); - int batchNum = totalNum; - if (Config.restore_download_task_num_per_be > 0) { - batchNum = Math.min(totalNum, Config.restore_download_task_num_per_be); - } // each task contains several upload sub tasks - int taskNumPerBatch = Math.max(totalNum / batchNum, 1); + int taskNumPerBatch = Config.restore_download_snapshot_batch_size; + LOG.info("backend {} has total {} snapshots, per task batch size {}, {}", + beId, totalNum, taskNumPerBatch, this); // allot tasks - int index = 0; - for (int batch = 0; batch < batchNum; batch++) { + for (int index = 0; index < totalNum; index += taskNumPerBatch) { List remoteTabletSnapshots = Lists.newArrayList(); - int currentBatchTaskNum = (batch == batchNum - 1) ? totalNum - index : taskNumPerBatch; - for (int j = 0; j < currentBatchTaskNum; j++) { + for (int j = 0; j < taskNumPerBatch && index + j < totalNum; j++) { TRemoteTabletSnapshot remoteTabletSnapshot = new TRemoteTabletSnapshot(); - - SnapshotInfo info = beSnapshotInfos.get(index++); + SnapshotInfo info = beSnapshotInfos.get(index + j); Table tbl = db.getTableNullable(info.getTblId()); if (tbl == null) { status = new Status(ErrCode.NOT_FOUND, "restored table " From 7560a78fcbc083d35c9b49981133999b0662ba93 Mon Sep 17 00:00:00 2001 From: Kaijie Chen Date: Tue, 26 Nov 2024 23:38:50 +0800 Subject: [PATCH 017/399] [cleanup](load) remove bvar "g_memtable_input_block_allocated_size" (#44304) ### What problem does this PR solve? Bvar "g_memtable_input_block_allocated_size" is no longer needed after #40912. Calling `MutableBlock::allocated_bytes()` in `Memtable::insert()` has some performance penalties. So we should remove it. --- be/src/olap/memtable.cpp | 6 ------ 1 file changed, 6 deletions(-) diff --git a/be/src/olap/memtable.cpp b/be/src/olap/memtable.cpp index 233c59f0910871..5db3d89378bcc2 100644 --- a/be/src/olap/memtable.cpp +++ b/be/src/olap/memtable.cpp @@ -43,7 +43,6 @@ namespace doris { bvar::Adder g_memtable_cnt("memtable_cnt"); -bvar::Adder g_memtable_input_block_allocated_size("memtable_input_block_allocated_size"); using namespace ErrorCode; @@ -151,7 +150,6 @@ MemTable::~MemTable() { << _mem_tracker->consumption(); } } - g_memtable_input_block_allocated_size << -_input_mutable_block.allocated_bytes(); g_memtable_cnt << -1; if (_keys_type != KeysType::DUP_KEYS) { for (auto it = _row_in_blocks.begin(); it != _row_in_blocks.end(); it++) { @@ -222,11 +220,8 @@ Status MemTable::insert(const vectorized::Block* input_block, auto num_rows = row_idxs.size(); size_t cursor_in_mutableblock = _input_mutable_block.rows(); - auto block_size0 = _input_mutable_block.allocated_bytes(); RETURN_IF_ERROR(_input_mutable_block.add_rows(input_block, row_idxs.data(), row_idxs.data() + num_rows, &_column_offset)); - auto block_size1 = _input_mutable_block.allocated_bytes(); - g_memtable_input_block_allocated_size << block_size1 - block_size0; for (int i = 0; i < num_rows; i++) { _row_in_blocks.emplace_back(new RowInBlock {cursor_in_mutableblock + i}); } @@ -631,7 +626,6 @@ Status MemTable::_to_block(std::unique_ptr* res) { } RETURN_IF_ERROR(_sort_by_cluster_keys()); } - g_memtable_input_block_allocated_size << -_input_mutable_block.allocated_bytes(); _input_mutable_block.clear(); // After to block, all data in arena is saved in the block _arena.reset(); From 053e5935576f017b4cbcdb7425902c7cb329080b Mon Sep 17 00:00:00 2001 From: Kaijie Chen Date: Tue, 26 Nov 2024 23:40:07 +0800 Subject: [PATCH 018/399] [fix](load) restore "use_new_load_scan_node" in property map for compatibility (#44285) `LoadStmt#checkProperties()` will throw DdlException when encounter unknown properties. So we have to keep the deprecated property keys in the map for compatibility. --- .../java/org/apache/doris/analysis/LoadStmt.java | 9 +++++++++ .../broker_load/test_s3_load_properties.groovy | 13 +++++++++++++ 2 files changed, 22 insertions(+) diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/LoadStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/LoadStmt.java index 3a2efe3c98c902..9ceec9415e3cfb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/LoadStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/LoadStmt.java @@ -99,6 +99,9 @@ public class LoadStmt extends DdlStmt implements NotFallbackInParser { public static final String PRIORITY = "priority"; public static final String LOAD_TO_SINGLE_TABLET = "load_to_single_tablet"; + // deprecated, keeping this property to make LoadStmt#checkProperties() happy + public static final String USE_NEW_LOAD_SCAN_NODE = "use_new_load_scan_node"; + // for load data from Baidu Object Store(BOS) public static final String BOS_ENDPOINT = "bos_endpoint"; public static final String BOS_ACCESSKEY = "bos_accesskey"; @@ -221,6 +224,12 @@ public class LoadStmt extends DdlStmt implements NotFallbackInParser { return Boolean.valueOf(s); } }) + .put(USE_NEW_LOAD_SCAN_NODE, new Function() { + @Override + public @Nullable Boolean apply(@Nullable String s) { + return Boolean.valueOf(s); + } + }) .put(KEY_SKIP_LINES, new Function() { @Override public @Nullable Integer apply(@Nullable String s) { diff --git a/regression-test/suites/load_p2/broker_load/test_s3_load_properties.groovy b/regression-test/suites/load_p2/broker_load/test_s3_load_properties.groovy index 9d73256d38ccd2..fdc6971f475fac 100644 --- a/regression-test/suites/load_p2/broker_load/test_s3_load_properties.groovy +++ b/regression-test/suites/load_p2/broker_load/test_s3_load_properties.groovy @@ -187,6 +187,19 @@ suite("test_s3_load_properties", "p2") { // "", "", "", "","").addProperties("skip_lines", "10")) // } + /* ========================================================== deprecated properties ========================================================== */ + for (String table : basicTables) { + attributesList.add(new LoadAttributes("s3://${s3BucketName}/regression/load/data/basic_data.csv", + "${table}", "LINES TERMINATED BY \"\n\"", "COLUMNS TERMINATED BY \"|\"", "FORMAT AS \"CSV\"", "(k00,k01,k02,k03,k04,k05,k06,k07,k08,k09,k10,k11,k12,k13,k14,k15,k16,k17,k18)", + "", "", "", "", "")).addProperties("use_new_load_scan_node", "true") + } + + for (String table : basicTables) { + attributesList.add(new LoadAttributes("s3://${s3BucketName}/regression/load/data/basic_data.csv", + "${table}", "LINES TERMINATED BY \"\n\"", "COLUMNS TERMINATED BY \"|\"", "FORMAT AS \"CSV\"", "(k00,k01,k02,k03,k04,k05,k06,k07,k08,k09,k10,k11,k12,k13,k14,k15,k16,k17,k18)", + "", "", "", "", "")).addProperties("use_new_load_scan_node", "false") + } + /* ========================================================== wrong column sep ========================================================== */ for (String table : basicTables) { attributesList.add(new LoadAttributes("s3://${s3BucketName}/regression/load/data/basic_data.csv", From 09daf0684f69758a2b49c17464593aa0e74bd80c Mon Sep 17 00:00:00 2001 From: huanghaibin Date: Tue, 26 Nov 2024 23:46:31 +0800 Subject: [PATCH 019/399] [fix](cloud-mow)Fix some implicit conversion loses when complie mow code (#44410) Related PR: #40204 Problem Summary: There is some implicit conversion loses like uint64_t to uint32_t on mow code, this pr fix it. --- be/src/cloud/cloud_engine_calc_delete_bitmap_task.cpp | 2 +- be/src/cloud/cloud_tablet.cpp | 4 ++-- be/src/olap/tablet_meta.cpp | 2 +- cloud/src/meta-service/meta_service.cpp | 2 +- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/be/src/cloud/cloud_engine_calc_delete_bitmap_task.cpp b/be/src/cloud/cloud_engine_calc_delete_bitmap_task.cpp index 7391449b73f1b1..91611d20c6270b 100644 --- a/be/src/cloud/cloud_engine_calc_delete_bitmap_task.cpp +++ b/be/src/cloud/cloud_engine_calc_delete_bitmap_task.cpp @@ -273,7 +273,7 @@ Status CloudTabletCalcDeleteBitmapTask::_handle_rowset( .base_compaction_cnt = _ms_base_compaction_cnt, .cumulative_compaction_cnt = _ms_cumulative_compaction_cnt, .cumulative_point = _ms_cumulative_point}; - auto update_delete_bitmap_time_us = 0; + int64_t update_delete_bitmap_time_us = 0; if (txn_info.publish_status && (*(txn_info.publish_status) == PublishStatus::SUCCEED) && version == previous_publish_info.publish_version && _ms_base_compaction_cnt == previous_publish_info.base_compaction_cnt && diff --git a/be/src/cloud/cloud_tablet.cpp b/be/src/cloud/cloud_tablet.cpp index 267c204c0e608e..ea86f3b40ff1dc 100644 --- a/be/src/cloud/cloud_tablet.cpp +++ b/be/src/cloud/cloud_tablet.cpp @@ -397,8 +397,8 @@ int CloudTablet::delete_expired_stale_rowsets() { } for (int64_t path_id : path_ids) { - int start_version = -1; - int end_version = -1; + int64_t start_version = -1; + int64_t end_version = -1; // delete stale versions in version graph auto version_path = _timestamped_version_tracker.fetch_and_delete_path_by_id(path_id); for (auto& v_ts : version_path->timestamped_versions()) { diff --git a/be/src/olap/tablet_meta.cpp b/be/src/olap/tablet_meta.cpp index 9a27b95dbcd446..0570aff349c583 100644 --- a/be/src/olap/tablet_meta.cpp +++ b/be/src/olap/tablet_meta.cpp @@ -668,7 +668,7 @@ void TabletMeta::init_from_pb(const TabletMetaPB& tablet_meta_pb) { RowsetId rst_id; rst_id.init(tablet_meta_pb.delete_bitmap().rowset_ids(i)); auto seg_id = tablet_meta_pb.delete_bitmap().segment_ids(i); - uint32_t ver = tablet_meta_pb.delete_bitmap().versions(i); + auto ver = tablet_meta_pb.delete_bitmap().versions(i); auto bitmap = tablet_meta_pb.delete_bitmap().segment_delete_bitmaps(i).data(); delete_bitmap().delete_bitmap[{rst_id, seg_id, ver}] = roaring::Roaring::read(bitmap); } diff --git a/cloud/src/meta-service/meta_service.cpp b/cloud/src/meta-service/meta_service.cpp index 4f374832925dd7..974b3091a73e95 100644 --- a/cloud/src/meta-service/meta_service.cpp +++ b/cloud/src/meta-service/meta_service.cpp @@ -1748,7 +1748,7 @@ void MetaServiceImpl::update_delete_bitmap(google::protobuf::RpcController* cont } RPC_RATE_LIMIT(update_delete_bitmap) - uint32_t fdb_txn_size = 0; + uint64_t fdb_txn_size = 0; auto table_id = request->table_id(); auto tablet_id = request->tablet_id(); From 54f87af876d28005d690a95ae48f6a4a7da30863 Mon Sep 17 00:00:00 2001 From: Gavin Chou Date: Tue, 26 Nov 2024 23:55:10 +0800 Subject: [PATCH 020/399] [opt](log) Detail log for file cache resource limit mode (#44094) --- be/src/io/cache/block_file_cache.cpp | 31 ++++++++++++++-------------- 1 file changed, 16 insertions(+), 15 deletions(-) diff --git a/be/src/io/cache/block_file_cache.cpp b/be/src/io/cache/block_file_cache.cpp index ebcbe9135daa35..528ace90e3e7d7 100644 --- a/be/src/io/cache/block_file_cache.cpp +++ b/be/src/io/cache/block_file_cache.cpp @@ -1678,13 +1678,16 @@ void BlockFileCache::check_disk_resource_limit() { LOG_ERROR("").tag("file cache path", _cache_base_path).tag("error", strerror(errno)); return; } - auto [capacity_percentage, inode_percentage] = percent; - auto inode_is_insufficient = [](const int& inode_percentage) { - return inode_percentage >= config::file_cache_enter_disk_resource_limit_mode_percent; + auto [space_percentage, inode_percentage] = percent; + auto is_insufficient = [](const int& percentage) { + return percentage >= config::file_cache_enter_disk_resource_limit_mode_percent; }; - DCHECK(capacity_percentage >= 0 && capacity_percentage <= 100); - DCHECK(inode_percentage >= 0 && inode_percentage <= 100); - // ATTN: due to that can be change, so if its invalid, set it to default value + DCHECK_GE(space_percentage, 0); + DCHECK_LE(space_percentage, 100); + DCHECK_GE(inode_percentage, 0); + DCHECK_LE(inode_percentage, 100); + // ATTN: due to that can be changed dynamically, set it to default value if it's invalid + // FIXME: reject with config validator if (config::file_cache_enter_disk_resource_limit_mode_percent <= config::file_cache_exit_disk_resource_limit_mode_percent) { LOG_WARNING("config error, set to default value") @@ -1693,23 +1696,21 @@ void BlockFileCache::check_disk_resource_limit() { config::file_cache_enter_disk_resource_limit_mode_percent = 90; config::file_cache_exit_disk_resource_limit_mode_percent = 80; } - if (capacity_percentage >= config::file_cache_enter_disk_resource_limit_mode_percent || - inode_is_insufficient(inode_percentage)) { + if (is_insufficient(space_percentage) || is_insufficient(inode_percentage)) { _disk_resource_limit_mode = true; _disk_limit_mode_metrics->set_value(1); } else if (_disk_resource_limit_mode && - (capacity_percentage < config::file_cache_exit_disk_resource_limit_mode_percent) && + (space_percentage < config::file_cache_exit_disk_resource_limit_mode_percent) && (inode_percentage < config::file_cache_exit_disk_resource_limit_mode_percent)) { _disk_resource_limit_mode = false; _disk_limit_mode_metrics->set_value(0); } if (_disk_resource_limit_mode) { - // log per mins - LOG_EVERY_N(WARNING, 3) << "file cache background thread space percent=" - << capacity_percentage << " inode percent=" << inode_percentage - << " is inode insufficient=" - << inode_is_insufficient(inode_percentage) - << " mode run in resource limit"; + LOG(WARNING) << "file_cache=" << get_base_path() << " space_percent=" << space_percentage + << " inode_percent=" << inode_percentage + << " is_space_insufficient=" << is_insufficient(space_percentage) + << " is_inode_insufficient=" << is_insufficient(inode_percentage) + << " mode run in resource limit"; } } From 95d19858cbd5f953dfbe06cbb47359a6adfdd769 Mon Sep 17 00:00:00 2001 From: Vallish Pai Date: Wed, 27 Nov 2024 07:05:23 +0530 Subject: [PATCH 021/399] [Enhancement] (nereids)implement DropWorkloadPolicyCommand in nereids (#44490) Issue Number: close #42622 --- .../org/apache/doris/nereids/DorisParser.g4 | 2 +- .../nereids/parser/LogicalPlanBuilder.java | 7 +++ .../doris/nereids/trees/plans/PlanType.java | 1 + .../commands/DropWorkloadPolicyCommand.java | 61 +++++++++++++++++++ .../trees/plans/visitor/CommandVisitor.java | 5 ++ .../WorkloadSchedPolicyMgr.java | 7 ++- .../test_nereids_workload_policy_test.out | 6 ++ .../test_nereids_workload_policy_test.groovy | 32 ++++++++++ 8 files changed, 118 insertions(+), 3 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropWorkloadPolicyCommand.java create mode 100644 regression-test/data/workload_manager_p0/test_nereids_workload_policy_test.out create mode 100644 regression-test/suites/workload_manager_p0/test_nereids_workload_policy_test.groovy diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index 932f644fca4e7a..b1fdda33646e13 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -204,6 +204,7 @@ supportedDropStatement | DROP SQL_BLOCK_RULE (IF EXISTS)? identifierSeq #dropSqlBlockRule | DROP USER (IF EXISTS)? userIdentify #dropUser | DROP WORKLOAD GROUP (IF EXISTS)? name=identifierOrText #dropWorkloadGroup + | DROP WORKLOAD POLICY (IF EXISTS)? name=identifierOrText #dropWorkloadPolicy ; supportedShowStatement @@ -673,7 +674,6 @@ unsupportedDropStatement ((FROM | IN) database=identifier)? properties=propertyClause #dropFile | DROP INDEX (IF EXISTS)? name=identifier ON tableName=multipartIdentifier #dropIndex | DROP RESOURCE (IF EXISTS)? name=identifierOrText #dropResource - | DROP WORKLOAD POLICY (IF EXISTS)? name=identifierOrText #dropWorkloadPolicy | DROP ROW POLICY (IF EXISTS)? policyName=identifier ON tableName=multipartIdentifier (FOR (userIdentify | ROLE roleName=identifier))? #dropRowPolicy diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index 320043e12b8b53..216cd60135b423 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -108,6 +108,7 @@ import org.apache.doris.nereids.DorisParser.DropSqlBlockRuleContext; import org.apache.doris.nereids.DorisParser.DropUserContext; import org.apache.doris.nereids.DorisParser.DropWorkloadGroupContext; +import org.apache.doris.nereids.DorisParser.DropWorkloadPolicyContext; import org.apache.doris.nereids.DorisParser.ElementAtContext; import org.apache.doris.nereids.DorisParser.ExceptContext; import org.apache.doris.nereids.DorisParser.ExceptOrReplaceContext; @@ -460,6 +461,7 @@ import org.apache.doris.nereids.trees.plans.commands.DropSqlBlockRuleCommand; import org.apache.doris.nereids.trees.plans.commands.DropUserCommand; import org.apache.doris.nereids.trees.plans.commands.DropWorkloadGroupCommand; +import org.apache.doris.nereids.trees.plans.commands.DropWorkloadPolicyCommand; import org.apache.doris.nereids.trees.plans.commands.ExplainCommand; import org.apache.doris.nereids.trees.plans.commands.ExplainCommand.ExplainLevel; import org.apache.doris.nereids.trees.plans.commands.ExportCommand; @@ -4400,6 +4402,11 @@ public LogicalPlan visitDropWorkloadGroup(DropWorkloadGroupContext ctx) { return new DropWorkloadGroupCommand(ctx.name.getText(), ctx.EXISTS() != null); } + @Override + public LogicalPlan visitDropWorkloadPolicy(DropWorkloadPolicyContext ctx) { + return new DropWorkloadPolicyCommand(ctx.name.getText(), ctx.EXISTS() != null); + } + @Override public LogicalPlan visitShowTableId(ShowTableIdContext ctx) { long tableId = -1; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index 1f7b838edfd905..0b45dde83d288f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -187,6 +187,7 @@ public enum PlanType { DROP_SQL_BLOCK_RULE_COMMAND, DROP_USER_COMMAND, DROP_WORKLOAD_GROUP_NAME, + DROP_WORKLOAD_POLICY_COMMAND, SHOW_BACKENDS_COMMAND, SHOW_BLOCK_RULE_COMMAND, SHOW_BROKER_COMMAND, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropWorkloadPolicyCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropWorkloadPolicyCommand.java new file mode 100644 index 00000000000000..96a3c0c69cf9ae --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropWorkloadPolicyCommand.java @@ -0,0 +1,61 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.commands; + +import org.apache.doris.catalog.Env; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.common.FeNameFormat; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.StmtExecutor; + +/** + * drop workload policy command + */ +public class DropWorkloadPolicyCommand extends DropCommand { + private final boolean ifExists; + private final String workloadPolicy; + + /** + * constructor + */ + public DropWorkloadPolicyCommand(String workloadPolicy, boolean ifExists) { + super(PlanType.DROP_WORKLOAD_POLICY_COMMAND); + this.workloadPolicy = workloadPolicy; + this.ifExists = ifExists; + } + + @Override + public void doRun(ConnectContext ctx, StmtExecutor executor) throws Exception { + // check auth + if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, "ADMIN"); + } + + FeNameFormat.checkWorkloadSchedPolicyName(workloadPolicy); + Env.getCurrentEnv().getWorkloadSchedPolicyMgr().dropWorkloadSchedPolicy(workloadPolicy, ifExists); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitDropWorkloadPolicyCommand(this, context); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index 6b801524fb295f..4893ef41be8e44 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -47,6 +47,7 @@ import org.apache.doris.nereids.trees.plans.commands.DropSqlBlockRuleCommand; import org.apache.doris.nereids.trees.plans.commands.DropUserCommand; import org.apache.doris.nereids.trees.plans.commands.DropWorkloadGroupCommand; +import org.apache.doris.nereids.trees.plans.commands.DropWorkloadPolicyCommand; import org.apache.doris.nereids.trees.plans.commands.ExplainCommand; import org.apache.doris.nereids.trees.plans.commands.ExportCommand; import org.apache.doris.nereids.trees.plans.commands.LoadCommand; @@ -439,6 +440,10 @@ default R visitDropWorkloadGroupCommand(DropWorkloadGroupCommand dropWorkloadGro return visitCommand(dropWorkloadGroupCommand, context); } + default R visitDropWorkloadPolicyCommand(DropWorkloadPolicyCommand dropWorkloadPolicyCommand, C context) { + return visitCommand(dropWorkloadPolicyCommand, context); + } + default R visitShowTableIdCommand(ShowTableIdCommand showTableIdCommand, C context) { return visitCommand(showTableIdCommand, context); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/resource/workloadschedpolicy/WorkloadSchedPolicyMgr.java b/fe/fe-core/src/main/java/org/apache/doris/resource/workloadschedpolicy/WorkloadSchedPolicyMgr.java index 3879dd83b9adfb..715bdfc5e1ee6d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/resource/workloadschedpolicy/WorkloadSchedPolicyMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/resource/workloadschedpolicy/WorkloadSchedPolicyMgr.java @@ -476,12 +476,15 @@ public void alterWorkloadSchedPolicy(AlterWorkloadSchedPolicyStmt alterStmt) thr } public void dropWorkloadSchedPolicy(DropWorkloadSchedPolicyStmt dropStmt) throws UserException { + dropWorkloadSchedPolicy(dropStmt.getPolicyName(), dropStmt.isIfExists()); + } + + public void dropWorkloadSchedPolicy(String policyName, boolean isExists) throws UserException { writeLock(); try { - String policyName = dropStmt.getPolicyName(); WorkloadSchedPolicy schedPolicy = nameToPolicy.get(policyName); if (schedPolicy == null) { - if (dropStmt.isIfExists()) { + if (isExists) { return; } else { throw new UserException("workload schedule policy " + policyName + " not exists"); diff --git a/regression-test/data/workload_manager_p0/test_nereids_workload_policy_test.out b/regression-test/data/workload_manager_p0/test_nereids_workload_policy_test.out new file mode 100644 index 00000000000000..8eead5743138a6 --- /dev/null +++ b/regression-test/data/workload_manager_p0/test_nereids_workload_policy_test.out @@ -0,0 +1,6 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !check_workload_policy_check1 -- +test_nereids_worklod_policy1 + +-- !check_workload_policy_check2 -- + diff --git a/regression-test/suites/workload_manager_p0/test_nereids_workload_policy_test.groovy b/regression-test/suites/workload_manager_p0/test_nereids_workload_policy_test.groovy new file mode 100644 index 00000000000000..c3b93f9ffa28d1 --- /dev/null +++ b/regression-test/suites/workload_manager_p0/test_nereids_workload_policy_test.groovy @@ -0,0 +1,32 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("test_nereids_workload_policy_test") { + sql "drop workload policy if exists test_nereids_worklod_policy1;" + sql "create workload policy test_nereids_worklod_policy1 " + + "conditions(username='root') " + + "actions(set_session_variable 'workload_group=normal') " + + "properties( " + + "'enabled' = 'false', " + + "'priority'='10' " + + ");" + qt_check_workload_policy_check1("select NAME from information_schema.workload_policy where NAME='test_nereids_worklod_policy1';") + checkNereidsExecute("drop workload policy test_nereids_worklod_policy1;") + checkNereidsExecute("drop workload policy if exists test_nereids_worklod_policy1;") + qt_check_workload_policy_check2("select NAME from information_schema.workload_policy where NAME='test_nereids_worklod_policy1';") + +} \ No newline at end of file From bf67d440f3f34ab35e2095a82944dcece9654f21 Mon Sep 17 00:00:00 2001 From: Vallish Pai Date: Wed, 27 Nov 2024 07:08:17 +0530 Subject: [PATCH 022/399] [Enhancement] (nereids)implement alterWorkloadPolicyCommand in nereids (#44501) Issue Number: close #42794 --- .../org/apache/doris/nereids/DorisParser.g4 | 4 +- .../nereids/parser/LogicalPlanBuilder.java | 9 +++ .../doris/nereids/trees/plans/PlanType.java | 1 + .../commands/AlterWorkloadPolicyCommand.java | 65 +++++++++++++++++++ .../trees/plans/visitor/CommandVisitor.java | 5 ++ .../WorkloadSchedPolicyMgr.java | 6 +- ...test_nereids_workloadpolicy_alter_test.out | 7 ++ ...t_nereids_workloadpolicy_alter_test.groovy | 32 +++++++++ 8 files changed, 125 insertions(+), 4 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterWorkloadPolicyCommand.java create mode 100644 regression-test/data/workload_manager_p0/test_nereids_workloadpolicy_alter_test.out create mode 100644 regression-test/suites/workload_manager_p0/test_nereids_workloadpolicy_alter_test.groovy diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index b1fdda33646e13..777a0e2d7d7fa6 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -195,6 +195,8 @@ supportedAlterStatement | ALTER ROLE role=identifier commentSpec #alterRole | ALTER WORKLOAD GROUP name=identifierOrText properties=propertyClause? #alterWorkloadGroup + | ALTER WORKLOAD POLICY name=identifierOrText + properties=propertyClause? #alterWorkloadPolicy ; supportedDropStatement @@ -562,8 +564,6 @@ unsupportedAlterStatement | ALTER RESOURCE name=identifierOrText properties=propertyClause? #alterResource | ALTER COLOCATE GROUP name=multipartIdentifier SET LEFT_PAREN propertyItemList RIGHT_PAREN #alterColocateGroup - | ALTER WORKLOAD POLICY name=identifierOrText - properties=propertyClause? #alterWorkloadPolicy | ALTER ROUTINE LOAD FOR name=multipartIdentifier properties=propertyClause? (FROM type=identifier LEFT_PAREN propertyItemList RIGHT_PAREN)? #alterRoutineLoad | ALTER SQL_BLOCK_RULE name=identifier properties=propertyClause? #alterSqlBlockRule diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index 216cd60135b423..94e499277ca221 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -58,6 +58,7 @@ import org.apache.doris.nereids.DorisParser.AlterStorageVaultContext; import org.apache.doris.nereids.DorisParser.AlterViewContext; import org.apache.doris.nereids.DorisParser.AlterWorkloadGroupContext; +import org.apache.doris.nereids.DorisParser.AlterWorkloadPolicyContext; import org.apache.doris.nereids.DorisParser.ArithmeticBinaryContext; import org.apache.doris.nereids.DorisParser.ArithmeticUnaryContext; import org.apache.doris.nereids.DorisParser.ArrayLiteralContext; @@ -435,6 +436,7 @@ import org.apache.doris.nereids.trees.plans.commands.AlterStorageVaultCommand; import org.apache.doris.nereids.trees.plans.commands.AlterViewCommand; import org.apache.doris.nereids.trees.plans.commands.AlterWorkloadGroupCommand; +import org.apache.doris.nereids.trees.plans.commands.AlterWorkloadPolicyCommand; import org.apache.doris.nereids.trees.plans.commands.CallCommand; import org.apache.doris.nereids.trees.plans.commands.CancelJobTaskCommand; import org.apache.doris.nereids.trees.plans.commands.CancelMTMVTaskCommand; @@ -4312,6 +4314,13 @@ public LogicalPlan visitAlterWorkloadGroup(AlterWorkloadGroupContext ctx) { return new AlterWorkloadGroupCommand(ctx.name.getText(), properties); } + @Override + public LogicalPlan visitAlterWorkloadPolicy(AlterWorkloadPolicyContext ctx) { + Map properties = ctx.propertyClause() != null + ? Maps.newHashMap(visitPropertyClause(ctx.propertyClause())) : Maps.newHashMap(); + return new AlterWorkloadPolicyCommand(ctx.name.getText(), properties); + } + @Override public LogicalPlan visitAlterRole(AlterRoleContext ctx) { String comment = visitCommentSpec(ctx.commentSpec()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index 0b45dde83d288f..7fa4db085848eb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -171,6 +171,7 @@ public enum PlanType { ALTER_VIEW_COMMAND, ALTER_STORAGE_VAULT, ALTER_WORKLOAD_GROUP_COMMAND, + ALTER_WORKLOAD_POLICY_COMMAND, DROP_CATALOG_RECYCLE_BIN_COMMAND, DROP_ENCRYPTKEY_COMMAND, UNSET_VARIABLE_COMMAND, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterWorkloadPolicyCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterWorkloadPolicyCommand.java new file mode 100644 index 00000000000000..42e5250d5b62e7 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterWorkloadPolicyCommand.java @@ -0,0 +1,65 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.commands; + +import org.apache.doris.catalog.Env; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.StmtExecutor; + +import java.util.Map; + +/** + * alter workload policy command + */ +public class AlterWorkloadPolicyCommand extends AlterCommand { + private final String workloadPolicyName; + private final Map properties; + + /** + * constructor + */ + public AlterWorkloadPolicyCommand(String workloadPolicyName, Map properties) { + super(PlanType.ALTER_WORKLOAD_POLICY_COMMAND); + this.workloadPolicyName = workloadPolicyName; + this.properties = properties; + } + + @Override + public void doRun(ConnectContext ctx, StmtExecutor executor) throws Exception { + // check auth + if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, "ADMIN"); + } + + if (properties == null || properties.isEmpty()) { + throw new AnalysisException("properties can't be null when alter workload schedule policy"); + } + Env.getCurrentEnv().getWorkloadSchedPolicyMgr().alterWorkloadSchedPolicy(workloadPolicyName, properties); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitAlterWorkloadPolicyCommand(this, context); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index 4893ef41be8e44..a564cb1c140208 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -23,6 +23,7 @@ import org.apache.doris.nereids.trees.plans.commands.AlterRoleCommand; import org.apache.doris.nereids.trees.plans.commands.AlterViewCommand; import org.apache.doris.nereids.trees.plans.commands.AlterWorkloadGroupCommand; +import org.apache.doris.nereids.trees.plans.commands.AlterWorkloadPolicyCommand; import org.apache.doris.nereids.trees.plans.commands.CallCommand; import org.apache.doris.nereids.trees.plans.commands.CancelJobTaskCommand; import org.apache.doris.nereids.trees.plans.commands.CancelMTMVTaskCommand; @@ -384,6 +385,10 @@ default R visitAlterWorkloadGroupCommand(AlterWorkloadGroupCommand alterWorkload return visitCommand(alterWorkloadGroupCommand, context); } + default R visitAlterWorkloadPolicyCommand(AlterWorkloadPolicyCommand alterWorkloadPolicyCommand, C context) { + return visitCommand(alterWorkloadPolicyCommand, context); + } + default R visitCleanAllProfileCommand(CleanAllProfileCommand cleanAllProfileCommand, C context) { return visitCommand(cleanAllProfileCommand, context); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/resource/workloadschedpolicy/WorkloadSchedPolicyMgr.java b/fe/fe-core/src/main/java/org/apache/doris/resource/workloadschedpolicy/WorkloadSchedPolicyMgr.java index 715bdfc5e1ee6d..43f9cd2a794b0f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/resource/workloadschedpolicy/WorkloadSchedPolicyMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/resource/workloadschedpolicy/WorkloadSchedPolicyMgr.java @@ -456,15 +456,17 @@ private void checkProperties(Map properties, List wgIdList } public void alterWorkloadSchedPolicy(AlterWorkloadSchedPolicyStmt alterStmt) throws UserException { + alterWorkloadSchedPolicy(alterStmt.getPolicyName(), alterStmt.getProperties()); + } + + public void alterWorkloadSchedPolicy(String policyName, Map properties) throws UserException { writeLock(); try { - String policyName = alterStmt.getPolicyName(); WorkloadSchedPolicy policy = nameToPolicy.get(policyName); if (policy == null) { throw new UserException("can not find workload schedule policy " + policyName); } - Map properties = alterStmt.getProperties(); List wgIdList = new ArrayList<>(); checkProperties(properties, wgIdList); policy.updatePropertyIfNotNull(properties, wgIdList); diff --git a/regression-test/data/workload_manager_p0/test_nereids_workloadpolicy_alter_test.out b/regression-test/data/workload_manager_p0/test_nereids_workloadpolicy_alter_test.out new file mode 100644 index 00000000000000..ca42544b6bd6e4 --- /dev/null +++ b/regression-test/data/workload_manager_p0/test_nereids_workloadpolicy_alter_test.out @@ -0,0 +1,7 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !check_workload_policy_check1 -- +test_nereids_alter_worklod_policy1 10 + +-- !check_workload_policy_check2 -- +test_nereids_alter_worklod_policy1 17 + diff --git a/regression-test/suites/workload_manager_p0/test_nereids_workloadpolicy_alter_test.groovy b/regression-test/suites/workload_manager_p0/test_nereids_workloadpolicy_alter_test.groovy new file mode 100644 index 00000000000000..e696f09587fd83 --- /dev/null +++ b/regression-test/suites/workload_manager_p0/test_nereids_workloadpolicy_alter_test.groovy @@ -0,0 +1,32 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("test_nereids_workloadpolicy_alter_test") { + sql "drop workload policy if exists test_nereids_alter_worklod_policy1;" + sql "create workload policy test_nereids_alter_worklod_policy1 " + + "conditions(username='root') " + + "actions(set_session_variable 'workload_group=normal') " + + "properties( " + + "'enabled' = 'false', " + + "'priority'='10');" + + qt_check_workload_policy_check1("select NAME,PRIORITY from information_schema.workload_policy where NAME='test_nereids_alter_worklod_policy1';") + checkNereidsExecute("alter workload policy test_nereids_alter_worklod_policy1 properties('priority'='17');") + qt_check_workload_policy_check2("select NAME,PRIORITY from information_schema.workload_policy where NAME='test_nereids_alter_worklod_policy1';") + sql "drop workload policy if exists test_nereids_alter_worklod_policy1;" + +} \ No newline at end of file From 9c640d89156331995a2ad359ffe5c958a63b488f Mon Sep 17 00:00:00 2001 From: Vallish Pai Date: Wed, 27 Nov 2024 07:23:42 +0530 Subject: [PATCH 023/399] [Enhancement] (nereids)implement showEventsCommand in nereids (#44499) Issue Number: close #42732 --- .../org/apache/doris/nereids/DorisParser.g4 | 2 +- .../nereids/parser/LogicalPlanBuilder.java | 7 ++ .../doris/nereids/trees/plans/PlanType.java | 1 + .../plans/commands/ShowEventsCommand.java | 70 +++++++++++++++++++ .../trees/plans/visitor/CommandVisitor.java | 5 ++ .../show/test_show_commands_nereids.groovy | 1 + 6 files changed, 85 insertions(+), 1 deletion(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowEventsCommand.java diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index 777a0e2d7d7fa6..2d68e01b78761a 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -214,6 +214,7 @@ supportedShowStatement | SHOW AUTHORS #showAuthors | SHOW BROKER #showBroker | SHOW DYNAMIC PARTITION TABLES ((FROM | IN) database=multipartIdentifier)? #showDynamicPartition + | SHOW EVENTS ((FROM | IN) database=multipartIdentifier)? wildWhere? #showEvents | SHOW LAST INSERT #showLastInsert | SHOW DELETE ((FROM | IN) database=multipartIdentifier)? #showDelete | SHOW ALL? GRANTS #showGrants @@ -287,7 +288,6 @@ unsupportedShowStatement | SHOW FULL? VIEWS ((FROM | IN) database=multipartIdentifier)? wildWhere? #showViews | SHOW FULL? PROCESSLIST #showProcessList | SHOW (GLOBAL | SESSION | LOCAL)? STATUS wildWhere? #showStatus - | SHOW EVENTS ((FROM | IN) database=multipartIdentifier)? wildWhere? #showEvents | SHOW CREATE VIEW name=multipartIdentifier #showCreateView | SHOW CREATE MATERIALIZED VIEW name=multipartIdentifier #showMaterializedView | SHOW CREATE (DATABASE | SCHEMA) name=multipartIdentifier #showCreateDatabase diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index 94e499277ca221..ae0921a1732419 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -218,6 +218,7 @@ import org.apache.doris.nereids.DorisParser.ShowCreateTableContext; import org.apache.doris.nereids.DorisParser.ShowDeleteContext; import org.apache.doris.nereids.DorisParser.ShowDynamicPartitionContext; +import org.apache.doris.nereids.DorisParser.ShowEventsContext; import org.apache.doris.nereids.DorisParser.ShowFrontendsContext; import org.apache.doris.nereids.DorisParser.ShowGrantsContext; import org.apache.doris.nereids.DorisParser.ShowGrantsForUserContext; @@ -493,6 +494,7 @@ import org.apache.doris.nereids.trees.plans.commands.ShowCreateTableCommand; import org.apache.doris.nereids.trees.plans.commands.ShowDeleteCommand; import org.apache.doris.nereids.trees.plans.commands.ShowDynamicPartitionCommand; +import org.apache.doris.nereids.trees.plans.commands.ShowEventsCommand; import org.apache.doris.nereids.trees.plans.commands.ShowFrontendsCommand; import org.apache.doris.nereids.trees.plans.commands.ShowGrantsCommand; import org.apache.doris.nereids.trees.plans.commands.ShowLastInsertCommand; @@ -4013,6 +4015,11 @@ public LogicalPlan visitShowAuthors(ShowAuthorsContext ctx) { return new ShowAuthorsCommand(); } + @Override + public LogicalPlan visitShowEvents(ShowEventsContext ctx) { + return new ShowEventsCommand(); + } + @Override public LogicalPlan visitShowConfig(ShowConfigContext ctx) { ShowConfigCommand command; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index 7fa4db085848eb..79a4ff31c1fc20 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -198,6 +198,7 @@ public enum PlanType { SHOW_CREATE_TABLE_COMMAND, SHOW_DELETE_COMMAND, SHOW_DYNAMIC_PARTITION_COMMAND, + SHOW_EVENTS_COMMAND, SHOW_FRONTENDS_COMMAND, SHOW_GRANTS_COMMAND, SHOW_LAST_INSERT_COMMAND, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowEventsCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowEventsCommand.java new file mode 100644 index 00000000000000..02b516601f40fc --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowEventsCommand.java @@ -0,0 +1,70 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.commands; + +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.ScalarType; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.ShowResultSet; +import org.apache.doris.qe.ShowResultSetMetaData; +import org.apache.doris.qe.StmtExecutor; + +import com.google.common.collect.Lists; + +import java.util.List; + +/** + * show events + */ +public class ShowEventsCommand extends ShowCommand { + private static final ShowResultSetMetaData META_DATA = + ShowResultSetMetaData.builder() + .addColumn(new Column("Db", ScalarType.createVarchar(20))) + .addColumn(new Column("Name", ScalarType.createVarchar(30))) + .addColumn(new Column("Definer", ScalarType.createVarchar(20))) + .addColumn(new Column("Time", ScalarType.createVarchar(20))) + .addColumn(new Column("Type", ScalarType.createVarchar(20))) + .addColumn(new Column("Execute at", ScalarType.createVarchar(20))) + .addColumn(new Column("Interval value", ScalarType.createVarchar(30))) + .addColumn(new Column("Interval field", ScalarType.createVarchar(30))) + .addColumn(new Column("Status", ScalarType.createVarchar(30))) + .addColumn(new Column("Ends", ScalarType.createVarchar(30))) + .addColumn(new Column("Status", ScalarType.createVarchar(30))) + .addColumn(new Column("Originator", ScalarType.createVarchar(30))) + .addColumn(new Column("character_set_client", ScalarType.createVarchar(30))) + .addColumn(new Column("collation_connection", ScalarType.createVarchar(30))) + .addColumn(new Column("Database Collation", ScalarType.createVarchar(30))) + .build(); + + public ShowEventsCommand() { + super(PlanType.SHOW_EVENTS_COMMAND); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitShowEventsCommand(this, context); + } + + @Override + public ShowResultSet doRun(ConnectContext ctx, StmtExecutor executor) throws Exception { + List> rowSet = Lists.newArrayList(); + return new ShowResultSet(META_DATA, rowSet); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index a564cb1c140208..c0f86d8dfa45ba 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -77,6 +77,7 @@ import org.apache.doris.nereids.trees.plans.commands.ShowCreateTableCommand; import org.apache.doris.nereids.trees.plans.commands.ShowDeleteCommand; import org.apache.doris.nereids.trees.plans.commands.ShowDynamicPartitionCommand; +import org.apache.doris.nereids.trees.plans.commands.ShowEventsCommand; import org.apache.doris.nereids.trees.plans.commands.ShowFrontendsCommand; import org.apache.doris.nereids.trees.plans.commands.ShowGrantsCommand; import org.apache.doris.nereids.trees.plans.commands.ShowLastInsertCommand; @@ -457,6 +458,10 @@ default R visitSyncCommand(SyncCommand syncCommand, C context) { return visitCommand(syncCommand, context); } + default R visitShowEventsCommand(ShowEventsCommand showEventsCommand, C context) { + return visitCommand(showEventsCommand, context); + } + default R visitShowDeleteCommand(ShowDeleteCommand showDeleteCommand, C context) { return visitCommand(showDeleteCommand, context); } diff --git a/regression-test/suites/nereids_p0/show/test_show_commands_nereids.groovy b/regression-test/suites/nereids_p0/show/test_show_commands_nereids.groovy index 998033deff9863..84dcbc2323fa02 100644 --- a/regression-test/suites/nereids_p0/show/test_show_commands_nereids.groovy +++ b/regression-test/suites/nereids_p0/show/test_show_commands_nereids.groovy @@ -26,5 +26,6 @@ suite("test_show_commands_nereids") { checkNereidsExecute("""show backends;""") checkNereidsExecute("""show whitelist;""") checkNereidsExecute("""show triggers;""") + checkNereidsExecute("""show events;""") checkNereidsExecute("""show load profile \"\\";""") } From aca0eaad6859242aa33a3929e0f901473844ab23 Mon Sep 17 00:00:00 2001 From: xzj7019 Date: Wed, 27 Nov 2024 10:28:01 +0800 Subject: [PATCH 024/399] [fix](nereids) fix ptopN push down under multi winexprs with partial forbidden type (#44617) Related PR: #38393 Problem Summary: In the previous pr which supporting multi win expr ptopN pushdown, it handled partial forbidden type unexpectly and will lead some case to push down the pTopN wrongly. plan before fixing: explain shape plan select * from (select row_number() over(partition by c1, c2 order by c3) as rn, sum(c2) over(order by c2 range between unbounded preceding and unbounded following) as sw from push_down_multi_predicate_through_window_t) t where rn <= 1 and sw <= 1; +------------------------------------------------------------------------------------+ | Explain String(Nereids Planner) | +------------------------------------------------------------------------------------+ | PhysicalResultSink | | --PhysicalProject | | ----filter((rn <= 1) and (sw <= 1)) | | ------PhysicalWindow | | --------PhysicalQuickSort[MERGE_SORT] | | ----------PhysicalDistribute[DistributionSpecGather] | | ------------PhysicalQuickSort[LOCAL_SORT] | | --------------PhysicalWindow | | ----------------PhysicalQuickSort[LOCAL_SORT] | | ------------------PhysicalDistribute[DistributionSpecHash] | | --------------------PhysicalPartitionTopN | | ----------------------PhysicalOlapScan[push_down_multi_predicate_through_window_t] | +------------------------------------------------------------------------------------+ plan after fixing: explain shape plan select * from (select row_number() over(partition by c1, c2 order by c3) as rn, sum(c2) over(order by c2 range between unbounded preceding and unbounded following) as sw from push_down_multi_predicate_through_window_t) t where rn <= 1 and sw <= 1; +----------------------------------------------------------------------------------+ | Explain String(Nereids Planner) | +----------------------------------------------------------------------------------+ | PhysicalResultSink | | --PhysicalProject | | ----filter((rn <= 1) and (sw <= 1)) | | ------PhysicalWindow | | --------PhysicalQuickSort[MERGE_SORT] | | ----------PhysicalDistribute[DistributionSpecGather] | | ------------PhysicalQuickSort[LOCAL_SORT] | | --------------PhysicalWindow | | ----------------PhysicalQuickSort[LOCAL_SORT] | | ------------------PhysicalDistribute[DistributionSpecHash] | | --------------------PhysicalOlapScan[push_down_multi_predicate_through_window_t] | +----------------------------------------------------------------------------------+ --- .../trees/plans/logical/LogicalWindow.java | 10 ++--- .../nereids_syntax_p0/window_function.out | 6 +++ ...sh_down_multi_filter_through_window.groovy | 20 ++++++++++ .../nereids_syntax_p0/window_function.groovy | 39 +++++++++++++++++++ 4 files changed, 70 insertions(+), 5 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalWindow.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalWindow.java index 6cd8389349a007..5ee70afbf5ff86 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalWindow.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalWindow.java @@ -218,7 +218,7 @@ && child(0).child(0) instanceof LogicalPartitionTopN)) { for (NamedExpression windowExpr : windowExpressions) { if (windowExpr == null || windowExpr.children().size() != 1 || !(windowExpr.child(0) instanceof WindowExpression)) { - continue; + return null; } WindowExpression windowFunc = (WindowExpression) windowExpr.child(0); @@ -226,12 +226,12 @@ && child(0).child(0) instanceof LogicalPartitionTopN)) { if (!(windowFunc.getFunction() instanceof RowNumber || windowFunc.getFunction() instanceof Rank || windowFunc.getFunction() instanceof DenseRank)) { - continue; + return null; } // Check the partition key and order key. if (windowFunc.getPartitionKeys().isEmpty() && windowFunc.getOrderKeys().isEmpty()) { - continue; + return null; } // Check the window type and window frame. @@ -240,10 +240,10 @@ && child(0).child(0) instanceof LogicalPartitionTopN)) { WindowFrame frame = windowFrame.get(); if (!(frame.getLeftBoundary().getFrameBoundType() == WindowFrame.FrameBoundType.UNBOUNDED_PRECEDING && frame.getRightBoundary().getFrameBoundType() == WindowFrame.FrameBoundType.CURRENT_ROW)) { - continue; + return null; } } else { - continue; + return null; } // Check filter conditions. diff --git a/regression-test/data/nereids_syntax_p0/window_function.out b/regression-test/data/nereids_syntax_p0/window_function.out index 4ec92fc61ad46c..38eba68274e18f 100644 --- a/regression-test/data/nereids_syntax_p0/window_function.out +++ b/regression-test/data/nereids_syntax_p0/window_function.out @@ -561,3 +561,9 @@ \N \N +-- !multi_winf1 -- +1 c + +-- !multi_winf2 -- +1 35 + diff --git a/regression-test/suites/nereids_rules_p0/push_down_filter_through_window/push_down_multi_filter_through_window.groovy b/regression-test/suites/nereids_rules_p0/push_down_filter_through_window/push_down_multi_filter_through_window.groovy index d808d30f8ebdfe..015a6e7fae7d47 100644 --- a/regression-test/suites/nereids_rules_p0/push_down_filter_through_window/push_down_multi_filter_through_window.groovy +++ b/regression-test/suites/nereids_rules_p0/push_down_filter_through_window/push_down_multi_filter_through_window.groovy @@ -157,4 +157,24 @@ suite("push_down_multi_filter_through_window") { sql ("select * from (select row_number() over(partition by c1, c2 order by c3) as rn, rank() over(partition by c1 order by c3) as rk from push_down_multi_predicate_through_window_t) t where rn <= 1 or rk <= 1;") notContains "VPartitionTopN" } + + explain { + sql ("select * from (select row_number() over(partition by c1, c2 order by c3) as rn, sum(c2) over(order by c2 range between unbounded preceding and unbounded following) as sw from push_down_multi_predicate_through_window_t) t where rn <= 1 and sw <= 1;") + notContains "VPartitionTopN" + } + + explain { + sql ("select * from (select sum(c2) over(order by c2 range between unbounded preceding and unbounded following) as sw, row_number() over(partition by c1, c2 order by c3) as rn from push_down_multi_predicate_through_window_t) t where rn <= 1 and sw <= 1;") + notContains "VPartitionTopN" + } + + explain { + sql ("select * from (select row_number() over(partition by c1, c2 order by c3 rows between unbounded preceding and current row) as rn, sum(c2) over(order by c2) as sw from push_down_multi_predicate_through_window_t) t where rn <= 1 and sw <= 1;") + notContains "VPartitionTopN" + } + + explain { + sql ("select * from (select sum(c2) over(order by c2) as sw, row_number() over(partition by c1, c2 order by c3 rows between unbounded preceding and current row) as rn from push_down_multi_predicate_through_window_t) t where rn <= 1 and sw <= 1;") + notContains "VPartitionTopN" + } } diff --git a/regression-test/suites/nereids_syntax_p0/window_function.groovy b/regression-test/suites/nereids_syntax_p0/window_function.groovy index 9a427d1019887d..31cb425fcf7a44 100644 --- a/regression-test/suites/nereids_syntax_p0/window_function.groovy +++ b/regression-test/suites/nereids_syntax_p0/window_function.groovy @@ -240,4 +240,43 @@ suite("window_function") { """ qt_sql """ select LAST_VALUE(col_tinyint_undef_signed_not_null) over (partition by col_double_undef_signed_not_null, col_int_undef_signed, (col_float_undef_signed_not_null - col_int_undef_signed), round_bankers(col_int_undef_signed) order by pk rows between unbounded preceding and 4 preceding) AS col_alias56089 from table_200_undef_partitions2_keys3_properties4_distributed_by53 order by col_alias56089; """ + + order_qt_multi_winf1 """ + select * + from ( + select + row_number() over(partition by c1 order by c2) rn, + lead(c2, 2, '') over(partition by c1 order by c2) + from ( + select 1 as c1, 'a' as c2 + union all + select 1 as c1, 'b' as c2 + union all + select 1 as c1, 'c' as c2 + union all + select 1 as c1, 'd' as c2 + union all + select 1 as c1, 'e' as c2 + )t + )a where rn=1 + """ + order_qt_multi_winf2 """ + select * + from ( + select + row_number() over(partition by c1 order by c2) rn, + sum(c2) over(order by c2 range between unbounded preceding and unbounded following) + from ( + select 1 as c1, 5 as c2 + union all + select 1 as c1, 6 as c2 + union all + select 1 as c1, 7 as c2 + union all + select 1 as c1, 8 as c2 + union all + select 1 as c1, 9 as c2 + )t + )a where rn=1 + """ } From df5bfe8f66b2fb50285c1c0253abca207f4d7801 Mon Sep 17 00:00:00 2001 From: zhiqiang Date: Wed, 27 Nov 2024 10:37:29 +0800 Subject: [PATCH 025/399] [fix](hyperscan) Fix hyper scan fall back to re2 (#44547) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * Core modification When hyper scan failed, we should not set_error in FunctionContext. Since set_error will try cancel query, but actually we want to fall back to re2 in this case. * Some refactor Rename FunctionRegexp so that we can distinguish regexp match with regexp_extract. * Reproduce ```cpp SELECT * FROM regexp_test_chinese WHERE city REGEXP "^上海|^北京" ORDER BY id; ``` Note, the `|` in above sql is a Chinese character. --- be/src/vec/functions/function_regexp.cpp | 15 ++++---- be/src/vec/functions/like.cpp | 21 +++++------ be/src/vec/functions/like.h | 4 +-- .../string_functions/test_regexp_chinese.out | 3 ++ .../test_regexp_chinese.groovy | 35 +++++++++++++++++++ 5 files changed, 57 insertions(+), 21 deletions(-) create mode 100644 regression-test/data/query_p0/sql_functions/string_functions/test_regexp_chinese.out create mode 100644 regression-test/suites/query_p0/sql_functions/string_functions/test_regexp_chinese.groovy diff --git a/be/src/vec/functions/function_regexp.cpp b/be/src/vec/functions/function_regexp.cpp index ae508120cf9ee8..f740cac273c7cd 100644 --- a/be/src/vec/functions/function_regexp.cpp +++ b/be/src/vec/functions/function_regexp.cpp @@ -383,12 +383,13 @@ struct RegexpExtractAllImpl { } }; +// template FunctionRegexpFunctionality is used for regexp_xxxx series functions, not for regexp match. template -class FunctionRegexp : public IFunction { +class FunctionRegexpFunctionality : public IFunction { public: static constexpr auto name = Impl::name; - static FunctionPtr create() { return std::make_shared(); } + static FunctionPtr create() { return std::make_shared(); } String get_name() const override { return name; } @@ -486,11 +487,11 @@ class FunctionRegexp : public IFunction { }; void register_function_regexp_extract(SimpleFunctionFactory& factory) { - factory.register_function>(); - factory.register_function>>(); - factory.register_function>>(); - factory.register_function>(); - factory.register_function>(); + factory.register_function>(); + factory.register_function>>(); + factory.register_function>>(); + factory.register_function>(); + factory.register_function>(); } } // namespace doris::vectorized diff --git a/be/src/vec/functions/like.cpp b/be/src/vec/functions/like.cpp index 631ebb8dbe65f2..d727ba6e850f04 100644 --- a/be/src/vec/functions/like.cpp +++ b/be/src/vec/functions/like.cpp @@ -486,12 +486,10 @@ Status FunctionLikeBase::hs_prepare(FunctionContext* context, const char* expres if (res != HS_SUCCESS) { *database = nullptr; - if (context) { - context->set_error("hs_compile regex pattern error"); - } - return Status::RuntimeError("hs_compile regex pattern error:" + - std::string(compile_err->message)); + std::string error_message = compile_err->message; hs_free_compile_error(compile_err); + // Do not call FunctionContext::set_error here, since we do not want to cancel the query here. + return Status::RuntimeError("hs_compile regex pattern error:" + error_message); } hs_free_compile_error(compile_err); @@ -499,10 +497,8 @@ Status FunctionLikeBase::hs_prepare(FunctionContext* context, const char* expres hs_free_database(*database); *database = nullptr; *scratch = nullptr; - if (context) { - context->set_error("hs_alloc_scratch allocate scratch space error"); - } - return Status::RuntimeError("hs_alloc_scratch allocate scratch space error"); + // Do not call FunctionContext::set_error here, since we do not want to cancel the query here. + return Status::RuntimeError("hs_alloc_scratch allocate scratch space error"); } return Status::OK(); @@ -934,7 +930,8 @@ Status FunctionLike::open(FunctionContext* context, FunctionContext::FunctionSta return Status::OK(); } -Status FunctionRegexp::open(FunctionContext* context, FunctionContext::FunctionStateScope scope) { +Status FunctionRegexpLike::open(FunctionContext* context, + FunctionContext::FunctionStateScope scope) { if (scope != FunctionContext::THREAD_LOCAL) { return Status::OK(); } @@ -1001,8 +998,8 @@ void register_function_like(SimpleFunctionFactory& factory) { } void register_function_regexp(SimpleFunctionFactory& factory) { - factory.register_function(); - factory.register_alias(FunctionRegexp::name, FunctionRegexp::alias); + factory.register_function(); + factory.register_alias(FunctionRegexpLike::name, FunctionRegexpLike::alias); } } // namespace doris::vectorized diff --git a/be/src/vec/functions/like.h b/be/src/vec/functions/like.h index 229341e7bd13d1..fed26718603575 100644 --- a/be/src/vec/functions/like.h +++ b/be/src/vec/functions/like.h @@ -280,12 +280,12 @@ class FunctionLike : public FunctionLikeBase { static void remove_escape_character(std::string* search_string); }; -class FunctionRegexp : public FunctionLikeBase { +class FunctionRegexpLike : public FunctionLikeBase { public: static constexpr auto name = "regexp"; static constexpr auto alias = "rlike"; - static FunctionPtr create() { return std::make_shared(); } + static FunctionPtr create() { return std::make_shared(); } String get_name() const override { return name; } diff --git a/regression-test/data/query_p0/sql_functions/string_functions/test_regexp_chinese.out b/regression-test/data/query_p0/sql_functions/string_functions/test_regexp_chinese.out new file mode 100644 index 00000000000000..27bb4af4d8a44d --- /dev/null +++ b/regression-test/data/query_p0/sql_functions/string_functions/test_regexp_chinese.out @@ -0,0 +1,3 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !sql_regexp -- + diff --git a/regression-test/suites/query_p0/sql_functions/string_functions/test_regexp_chinese.groovy b/regression-test/suites/query_p0/sql_functions/string_functions/test_regexp_chinese.groovy new file mode 100644 index 00000000000000..900a0a04610f9f --- /dev/null +++ b/regression-test/suites/query_p0/sql_functions/string_functions/test_regexp_chinese.groovy @@ -0,0 +1,35 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("test_regexp_chinese") { + sql "DROP TABLE IF EXISTS regexp_test_chinese;" + sql """ + CREATE TABLE regexp_test_chinese ( + id int NULL DEFAULT "0", + city varchar(50) NOT NULL DEFAULT "" + ) DISTRIBUTED BY HASH(id) BUCKETS 5 properties("replication_num" = "1"); + """ + + sql """ + INSERT INTO regexp_test_chinese VALUES(1, "上海"),(2, "深圳"),(3, "上海测试"), (4, "北京测试"); + """ + + qt_sql_regexp """ + SELECT * FROM regexp_test_chinese WHERE city REGEXP "^上海|^北京" ORDER BY id; + """ +} + From 7e5a380e1e9f8cb1ef552e25e293c4459e663abb Mon Sep 17 00:00:00 2001 From: zhiqiang Date: Wed, 27 Nov 2024 10:40:18 +0800 Subject: [PATCH 026/399] [chore](regression-test) Remove some flaky profile test. (#42396) As title --- ..._pipeline_task_serial_read_on_limit.groovy | 172 ------------------ .../query_profile/scanner_profile.groovy | 108 ----------- .../test_execute_by_frontend.groovy | 90 --------- 3 files changed, 370 deletions(-) delete mode 100644 regression-test/suites/query_profile/adaptive_pipeline_task_serial_read_on_limit.groovy delete mode 100644 regression-test/suites/query_profile/scanner_profile.groovy delete mode 100644 regression-test/suites/query_profile/test_execute_by_frontend.groovy diff --git a/regression-test/suites/query_profile/adaptive_pipeline_task_serial_read_on_limit.groovy b/regression-test/suites/query_profile/adaptive_pipeline_task_serial_read_on_limit.groovy deleted file mode 100644 index 87279f65157409..00000000000000 --- a/regression-test/suites/query_profile/adaptive_pipeline_task_serial_read_on_limit.groovy +++ /dev/null @@ -1,172 +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. - -import groovy.json.JsonOutput -import groovy.json.JsonSlurper -import groovy.json.StringEscapeUtils - - -def getProfileList = { - def dst = 'http://' + context.config.feHttpAddress - def conn = new URL(dst + "/rest/v1/query_profile").openConnection() - conn.setRequestMethod("GET") - def encoding = Base64.getEncoder().encodeToString((context.config.feHttpUser + ":" + - (context.config.feHttpPassword == null ? "" : context.config.feHttpPassword)).getBytes("UTF-8")) - conn.setRequestProperty("Authorization", "Basic ${encoding}") - return conn.getInputStream().getText() -} - - -def getProfile = { id -> - def dst = 'http://' + context.config.feHttpAddress - def conn = new URL(dst + "/api/profile/text/?query_id=$id").openConnection() - conn.setRequestMethod("GET") - def encoding = Base64.getEncoder().encodeToString((context.config.feHttpUser + ":" + - (context.config.feHttpPassword == null ? "" : context.config.feHttpPassword)).getBytes("UTF-8")) - conn.setRequestProperty("Authorization", "Basic ${encoding}") - return conn.getInputStream().getText() -} - -suite('adaptive_pipeline_task_serial_read_on_limit') { - sql """ - DROP TABLE IF EXISTS adaptive_pipeline_task_serial_read_on_limit; - """ - sql """ - CREATE TABLE if not exists `adaptive_pipeline_task_serial_read_on_limit` ( - `id` INT, - `name` varchar(32) - ) ENGINE=OLAP - DISTRIBUTED BY HASH(`id`) BUCKETS 5 - PROPERTIES ( - "replication_allocation" = "tag.location.default: 1" - ); - """ - - // Insert data to table - sql """ - insert into adaptive_pipeline_task_serial_read_on_limit values - (1, "A"),(2, "B"),(3, "C"),(4, "D"),(5,"E"),(6,"F"),(7,"G"),(8,"H"),(9,"K"); - """ - sql """ - insert into adaptive_pipeline_task_serial_read_on_limit values - (10, "A"),(20, "B"),(30, "C"),(40, "D"),(50,"E"),(60,"F"),(70,"G"),(80,"H"),(90,"K"); - """ - sql """ - insert into adaptive_pipeline_task_serial_read_on_limit values - (101, "A"),(201, "B"),(301, "C"),(401, "D"),(501,"E"),(601,"F"),(701,"G"),(801,"H"),(901,"K"); - """ - sql """ - insert into adaptive_pipeline_task_serial_read_on_limit values - (1010, "A"),(2010, "B"),(3010, "C"),(4010, "D"),(5010,"E"),(6010,"F"),(7010,"G"),(8010,"H"),(9010,"K"); - """ - - def uuidString = UUID.randomUUID().toString() - sql "set enable_profile=true" - // set parallel_pipeline_task_num to 1 so that only one scan node, - // and we can check MaxScannerThreadNum in profile. - sql "set parallel_pipeline_task_num=1;" - // no limit, MaxScannerThreadNum = TabletNum - sql """ - select "no_limit_1_${uuidString}", * from adaptive_pipeline_task_serial_read_on_limit; - """ - sql "set parallel_pipeline_task_num=0;" - // With Limit, MaxScannerThreadNum = 1 - sql """ - select "with_limit_1_${uuidString}", * from adaptive_pipeline_task_serial_read_on_limit limit 10000; - """ - // With Limit, but bigger then adaptive_pipeline_task_serial_read_on_limit, MaxScannerThreadNum = TabletNum - sql """ - select "with_limit_2_${uuidString}", * from adaptive_pipeline_task_serial_read_on_limit limit 10001; - """ - sql """ - set enable_adaptive_pipeline_task_serial_read_on_limit=false; - """ - sql "set parallel_pipeline_task_num=1;" - // Forbid the strategy, with limit, MaxScannerThreadNum = TabletNum - sql """ - select "not_enable_limit_${uuidString}", * from adaptive_pipeline_task_serial_read_on_limit limit 100; - """ - - sql "set parallel_pipeline_task_num=0;" - - // Enable the strategy, with limit 20, MaxScannerThreadNum = 1 - sql """ - set enable_adaptive_pipeline_task_serial_read_on_limit=true; - """ - sql """ - set adaptive_pipeline_task_serial_read_on_limit=20; - """ - sql """ - select "modify_to_20_${uuidString}", * from adaptive_pipeline_task_serial_read_on_limit limit 15; - """ - - sql "set enable_profile=false" - - Thread.sleep(5) - - def wholeString = getProfileList() - List profileData = new JsonSlurper().parseText(wholeString).data.rows - String queryIdNoLimit1 = ""; - String queryIdWithLimit1 = ""; - String queryIdWithLimit2 = ""; - String queryIDNotEnableLimit = ""; - String queryIdModifyTo20 = ""; - - logger.info("{}", uuidString) - - for (def profileItem in profileData) { - if (profileItem["Sql Statement"].toString().contains("no_limit_1_${uuidString}")) { - queryIdNoLimit1 = profileItem["Profile ID"].toString() - logger.info("profileItem: {}", profileItem) - } - if (profileItem["Sql Statement"].toString().contains("with_limit_1_${uuidString}")) { - queryIdWithLimit1 = profileItem["Profile ID"].toString() - logger.info("profileItem: {}", profileItem) - } - if (profileItem["Sql Statement"].toString().contains("with_limit_2_${uuidString}")) { - queryIdWithLimit2 = profileItem["Profile ID"].toString() - logger.info("profileItem: {}", profileItem) - } - if (profileItem["Sql Statement"].toString().contains("not_enable_limit_${uuidString}")) { - queryIDNotEnableLimit = profileItem["Profile ID"].toString() - logger.info("profileItem: {}", profileItem) - } - if (profileItem["Sql Statement"].toString().contains("modify_to_20_${uuidString}")) { - queryIdModifyTo20 = profileItem["Profile ID"].toString() - logger.info("profileItem: {}", profileItem) - } - } - - logger.info("queryIdWithLimit1_${uuidString}: {}", queryIdWithLimit1) - logger.info("queryIdModifyTo20_${uuidString}: {}", queryIdModifyTo20) - - assertTrue(queryIdWithLimit1 != "") - assertTrue(queryIdModifyTo20 != "") - - def String profileWithLimit1 = getProfile(queryIdWithLimit1).toString() - def String profileModifyTo20 = getProfile(queryIdModifyTo20).toString() - - if (!profileWithLimit1.contains("- MaxScannerThreadNum: 1")) { - logger.info("profileWithLimit1:\n{}", profileWithLimit1) - } - assertTrue(profileWithLimit1.contains("- MaxScannerThreadNum: 1")) - - if (!profileModifyTo20.contains("- MaxScannerThreadNum: 1")) { - logger.info("profileModifyTo20:\n{}", profileModifyTo20) - } - assertTrue(profileModifyTo20.contains("- MaxScannerThreadNum: 1")) -} \ No newline at end of file diff --git a/regression-test/suites/query_profile/scanner_profile.groovy b/regression-test/suites/query_profile/scanner_profile.groovy deleted file mode 100644 index 75ae6a5ab65a9b..00000000000000 --- a/regression-test/suites/query_profile/scanner_profile.groovy +++ /dev/null @@ -1,108 +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. - -import groovy.json.JsonOutput -import groovy.json.JsonSlurper -import groovy.json.StringEscapeUtils - - -def getProfileList = { - def dst = 'http://' + context.config.feHttpAddress - def conn = new URL(dst + "/rest/v1/query_profile").openConnection() - conn.setRequestMethod("GET") - def encoding = Base64.getEncoder().encodeToString((context.config.feHttpUser + ":" + - (context.config.feHttpPassword == null ? "" : context.config.feHttpPassword)).getBytes("UTF-8")) - conn.setRequestProperty("Authorization", "Basic ${encoding}") - return conn.getInputStream().getText() -} - - -def getProfile = { id -> - def dst = 'http://' + context.config.feHttpAddress - def conn = new URL(dst + "/api/profile/text/?query_id=$id").openConnection() - conn.setRequestMethod("GET") - def encoding = Base64.getEncoder().encodeToString((context.config.feHttpUser + ":" + - (context.config.feHttpPassword == null ? "" : context.config.feHttpPassword)).getBytes("UTF-8")) - conn.setRequestProperty("Authorization", "Basic ${encoding}") - return conn.getInputStream().getText() -} - -suite('scanner_profile') { - sql """ - DROP TABLE IF EXISTS scanner_profile; - """ - sql """ - CREATE TABLE if not exists `scanner_profile` ( - `id` INT, - `name` varchar(32) - ) ENGINE=OLAP - DISTRIBUTED BY HASH(`id`) BUCKETS 10 - PROPERTIES ( - "replication_allocation" = "tag.location.default: 1" - ); - """ - - // Insert data to table - sql """ - insert into scanner_profile values - (1, "A"),(2, "B"),(3, "C"),(4, "D"),(5,"E"),(6,"F"),(7,"G"),(8,"H"),(9,"K"); - """ - sql """ - insert into scanner_profile values - (10, "A"),(20, "B"),(30, "C"),(40, "D"),(50,"E"),(60,"F"),(70,"G"),(80,"H"),(90,"K"); - """ - sql """ - insert into scanner_profile values - (101, "A"),(201, "B"),(301, "C"),(401, "D"),(501,"E"),(601,"F"),(701,"G"),(801,"H"),(901,"K"); - """ - sql """ - insert into scanner_profile values - (1010, "A"),(2010, "B"),(3010, "C"),(4010, "D"),(5010,"E"),(6010,"F"),(7010,"G"),(8010,"H"),(9010,"K"); - """ - - def uuidString = UUID.randomUUID().toString() - sql "set enable_profile=true" - // With Limit, MaxScannerThreadNum = 1 - sql """ - select "with_limit_1_${uuidString}", * from scanner_profile limit 10; - """ - - def wholeString = getProfileList() - List profileData = new JsonSlurper().parseText(wholeString).data.rows - String queryIdWithLimit1 = ""; - - - logger.info("{}", uuidString) - - for (def profileItem in profileData) { - if (profileItem["Sql Statement"].toString().contains("with_limit_1_${uuidString}")) { - queryIdWithLimit1 = profileItem["Profile ID"].toString() - logger.info("profileItem: {}", profileItem) - } - } - - logger.info("queryIdWithLimit1_${uuidString}: {}", queryIdWithLimit1) - - assertTrue(queryIdWithLimit1 != "") - - // Sleep 5 seconds to make sure profile collection is done - Thread.sleep(5000) - - def String profileWithLimit1 = getProfile(queryIdWithLimit1).toString() - logger.info("query profile {}", profileWithLimit1) - assertTrue(profileWithLimit1.contains("- PeakRunningScanner: 1")) -} \ No newline at end of file diff --git a/regression-test/suites/query_profile/test_execute_by_frontend.groovy b/regression-test/suites/query_profile/test_execute_by_frontend.groovy deleted file mode 100644 index 40435e02d99870..00000000000000 --- a/regression-test/suites/query_profile/test_execute_by_frontend.groovy +++ /dev/null @@ -1,90 +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. - -import groovy.json.JsonOutput -import groovy.json.JsonSlurper -import groovy.json.StringEscapeUtils - -def getProfileList = { - def dst = 'http://' + context.config.feHttpAddress - def conn = new URL(dst + "/rest/v1/query_profile").openConnection() - conn.setRequestMethod("GET") - def encoding = Base64.getEncoder().encodeToString((context.config.feHttpUser + ":" + - (context.config.feHttpPassword == null ? "" : context.config.feHttpPassword)).getBytes("UTF-8")) - conn.setRequestProperty("Authorization", "Basic ${encoding}") - return conn.getInputStream().getText() -} - -def getProfile = { id -> - def dst = 'http://' + context.config.feHttpAddress - def conn = new URL(dst + "/api/profile/text/?query_id=$id").openConnection() - conn.setRequestMethod("GET") - def encoding = Base64.getEncoder().encodeToString((context.config.feHttpUser + ":" + - (context.config.feHttpPassword == null ? "" : context.config.feHttpPassword)).getBytes("UTF-8")) - conn.setRequestProperty("Authorization", "Basic ${encoding}") - // set conn parameters - - return conn.getInputStream().getText() - } - -suite('test_execute_by_frontend') { - sql """ - CREATE TABLE if not exists `test_execute_by_frontend` ( - `id` INT, - `name` varchar(32) - )ENGINE=OLAP - UNIQUE KEY(`id`) - DISTRIBUTED BY HASH(`id`) BUCKETS 1 - PROPERTIES ( - "replication_allocation" = "tag.location.default: 1" - ); - """ - - sql "set enable_profile=true" - def simpleSql1 = "select * from test_execute_by_frontend" - sql "${simpleSql1}" - simpleSql2 = """select cast("1" as Int)""" - sql "${simpleSql2}" - def isRecorded = false - def wholeString = getProfileList() - List profileData = new JsonSlurper().parseText(wholeString).data.rows - String queryId1 = ""; - String queryId2 = ""; - - for (final def profileItem in profileData) { - if (profileItem["Sql Statement"].toString() == simpleSql1) { - isRecorded = true - queryId1 = profileItem["Profile ID"].toString() - assertEquals("internal", profileItem["Default Catalog"].toString()) - } - if (profileItem["Sql Statement"].toString() == simpleSql2) { - queryId2 = profileItem["Profile ID"].toString() - } - } - - assertTrue(isRecorded) - - String profileContent1 = getProfile(queryId1) - def executionProfileIdx1 = profileContent1.indexOf("Executed By Frontend: true") - assertTrue(executionProfileIdx1 > 0) - String profileContent2 = getProfile(queryId2) - def executionProfileIdx2 = profileContent2.indexOf("Executed By Frontend: true") - assertTrue(executionProfileIdx2 > 0) - - sql """ SET enable_profile = false """ - sql """ DROP TABLE IF EXISTS test_execute_by_frontend """ -} \ No newline at end of file From db05c2860aa81ab1ee8f08f838859b73d00213f7 Mon Sep 17 00:00:00 2001 From: Calvin Kirs Date: Wed, 27 Nov 2024 10:43:41 +0800 Subject: [PATCH 027/399] [chore](github)Update .asf.yaml (#44651) --- .asf.yaml | 1 - 1 file changed, 1 deletion(-) diff --git a/.asf.yaml b/.asf.yaml index f4c21ce4b0f3c9..7a7d845e4c9bb0 100644 --- a/.asf.yaml +++ b/.asf.yaml @@ -125,7 +125,6 @@ github: - FE UT (Doris FE UT) - BE UT (Doris BE UT) - Build Broker - - ShellCheck - Build Third Party Libraries (Linux) - Build Third Party Libraries (macOS) - COMPILE (DORIS_COMPILE) From 23df40b73a10078f4fa5cb5bfc8d66763b514451 Mon Sep 17 00:00:00 2001 From: Xinyi Zou Date: Wed, 27 Nov 2024 10:44:07 +0800 Subject: [PATCH 028/399] [fix](memory) Fix compatibility with CgroupV2 (#44579) ### What problem does this PR solve? Problem Summary: 1. Fix `memory.max` value is `max`. 2. Fix `memory_usage` minus `slab_reclaimable`, part of "slab" that might be reclaimed. Note, CgroupV2 MemAvailable is usually a little smaller than Process MemAvailable. Process `MemAvailable = MemFree - LowWaterMark + (PageCache - min(PageCache / 2, LowWaterMark))`, from `MemAvailable` in `/proc/meminfo`, calculated by OS. CgroupV2 `MemAvailable = cgroup_mem_limit - cgroup_mem_usage`, `cgroup_mem_usage = memory.current - inactive_file - slab_reclaimable`, in fact, there seems to be some memory that can be reused in `cgroup_mem_usage`. ref: https://arthurchiao.art/blog/cgroupv2-zh/ --- be/src/common/cgroup_memory_ctl.cpp | 23 +++++++++++++++++++---- be/src/util/mem_info.cpp | 18 +++++++++++++++--- 2 files changed, 34 insertions(+), 7 deletions(-) diff --git a/be/src/common/cgroup_memory_ctl.cpp b/be/src/common/cgroup_memory_ctl.cpp index a29432bdb4ede5..dddcbd50338d82 100644 --- a/be/src/common/cgroup_memory_ctl.cpp +++ b/be/src/common/cgroup_memory_ctl.cpp @@ -27,6 +27,7 @@ #include "common/status.h" #include "util/cgroup_util.h" +#include "util/error_util.h" namespace doris { @@ -84,14 +85,23 @@ struct CgroupsV2Reader : CGroupMemoryCtl::ICgroupsReader { : _mount_file_dir(std::move(mount_file_dir)) {} Status read_memory_limit(int64_t* value) override { - RETURN_IF_ERROR(CGroupUtil::read_int_line_from_cgroup_file((_mount_file_dir / "memory.max"), - value)); + std::filesystem::path file_path = _mount_file_dir / "memory.max"; + std::string line; + std::ifstream file_stream(file_path, std::ios::in); + getline(file_stream, line); + if (file_stream.fail() || file_stream.bad()) { + return Status::CgroupError("Error reading {}: {}", file_path.string(), + get_str_err_msg()); + } + if (line == "max") { + *value = std::numeric_limits::max(); + return Status::OK(); + } + RETURN_IF_ERROR(CGroupUtil::read_int_line_from_cgroup_file(file_path, value)); return Status::OK(); } Status read_memory_usage(int64_t* value) override { - // memory.current contains a single number - // the reason why we subtract it described here: https://github.com/ClickHouse/ClickHouse/issues/64652#issuecomment-2149630667 RETURN_IF_ERROR(CGroupUtil::read_int_line_from_cgroup_file( (_mount_file_dir / "memory.current"), value)); std::unordered_map metrics_map; @@ -100,7 +110,12 @@ struct CgroupsV2Reader : CGroupMemoryCtl::ICgroupsReader { if (*value < metrics_map["inactive_file"]) { return Status::CgroupError("CgroupsV2Reader read_memory_usage negative memory usage"); } + // the reason why we subtract inactive_file described here: + // https://github.com/ClickHouse/ClickHouse/issues/64652#issuecomment-2149630667 *value -= metrics_map["inactive_file"]; + // Part of "slab" that might be reclaimed, such as dentries and inodes. + // https://arthurchiao.art/blog/cgroupv2-zh/ + *value -= metrics_map["slab_reclaimable"]; return Status::OK(); } diff --git a/be/src/util/mem_info.cpp b/be/src/util/mem_info.cpp index 36579452db3f85..fe9cf84b2aed54 100644 --- a/be/src/util/mem_info.cpp +++ b/be/src/util/mem_info.cpp @@ -197,9 +197,10 @@ void MemInfo::refresh_proc_meminfo() { _s_cgroup_mem_limit = std::numeric_limits::max(); // find cgroup limit failed, wait 300s, 1000 * 100ms. _s_cgroup_mem_refresh_wait_times = -3000; - LOG(INFO) << "Refresh cgroup memory limit failed, refresh again after 300s, cgroup " - "mem limit: " - << _s_cgroup_mem_limit; + LOG(WARNING) + << "Refresh cgroup memory limit failed, refresh again after 300s, cgroup " + "mem limit: " + << _s_cgroup_mem_limit << ", " << status; } else { _s_cgroup_mem_limit = cgroup_mem_limit; // wait 10s, 100 * 100ms, avoid too frequently. @@ -209,12 +210,17 @@ void MemInfo::refresh_proc_meminfo() { _s_cgroup_mem_refresh_wait_times++; } + // cgroup mem limit is refreshed every 10 seconds, + // cgroup mem usage is refreshed together with memInfo every time, which is very frequent. if (_s_cgroup_mem_limit != std::numeric_limits::max()) { int64_t cgroup_mem_usage; auto status = CGroupMemoryCtl::find_cgroup_mem_usage(&cgroup_mem_usage); if (!status.ok()) { _s_cgroup_mem_usage = std::numeric_limits::min(); _s_cgroup_mem_refresh_state = false; + LOG_EVERY_N(WARNING, 500) + << "Refresh cgroup memory usage failed, cgroup mem limit: " + << _s_cgroup_mem_limit << ", " << status; } else { _s_cgroup_mem_usage = cgroup_mem_usage; _s_cgroup_mem_refresh_state = true; @@ -279,6 +285,12 @@ void MemInfo::refresh_proc_meminfo() { mem_available = _mem_info_bytes["MemAvailable"]; } if (_s_cgroup_mem_refresh_state) { + // Note, CgroupV2 MemAvailable is usually a little smaller than Process MemAvailable. + // Process `MemAvailable = MemFree - LowWaterMark + (PageCache - min(PageCache / 2, LowWaterMark))`, + // from `MemAvailable` in `/proc/meminfo`, calculated by OS. + // CgroupV2 `MemAvailable = cgroup_mem_limit - cgroup_mem_usage`, + // `cgroup_mem_usage = memory.current - inactive_file - slab_reclaimable`, in fact, + // there seems to be some memory that can be reused in `cgroup_mem_usage`. if (mem_available < 0) { mem_available = _s_cgroup_mem_limit - _s_cgroup_mem_usage; } else { From 6d57e3866c538801355eea26a7259db58b464699 Mon Sep 17 00:00:00 2001 From: minghong Date: Wed, 27 Nov 2024 10:51:35 +0800 Subject: [PATCH 029/399] [opt](nereids) optimize push limit to agg (#44042) ### What problem does this PR solve? Pr #34853 introduced PushTopnToAgg rule. But there is a limitation that Topn(limit) should output all group by keys. This pr removes this limitation by using the first group by key as order key. --- .../rules/rewrite/LimitAggToTopNAgg.java | 61 ++++++++++-- .../plans/physical/PhysicalHashAggregate.java | 4 +- .../rules/rewrite/EliminateSortTest.java | 7 +- .../nereids_hint_tpcds_p0/shape/query23.out | 63 ++++++------ .../nereids_hint_tpcds_p0/shape/query32.out | 43 ++++---- .../nereids_hint_tpcds_p0/shape/query38.out | 91 ++++++++--------- .../nereids_hint_tpcds_p0/shape/query97.out | 61 ++++++------ .../shape/query23.out | 63 ++++++------ .../shape/query32.out | 43 ++++---- .../shape/query38.out | 91 ++++++++--------- .../shape/query97.out | 61 ++++++------ .../constraints/query23.out | 63 ++++++------ .../noStatsRfPrune/query23.out | 63 ++++++------ .../noStatsRfPrune/query32.out | 43 ++++---- .../noStatsRfPrune/query38.out | 91 ++++++++--------- .../noStatsRfPrune/query97.out | 57 +++++------ .../no_stats_shape/query23.out | 63 ++++++------ .../no_stats_shape/query32.out | 43 ++++---- .../no_stats_shape/query38.out | 91 ++++++++--------- .../no_stats_shape/query97.out | 57 +++++------ .../rf_prune/query23.out | 63 ++++++------ .../rf_prune/query32.out | 43 ++++---- .../rf_prune/query38.out | 91 ++++++++--------- .../rf_prune/query97.out | 57 +++++------ .../shape/query23.out | 63 ++++++------ .../shape/query32.out | 43 ++++---- .../shape/query38.out | 91 ++++++++--------- .../shape/query97.out | 57 +++++------ .../shape/query23.out | 63 ++++++------ .../shape/query32.out | 43 ++++---- .../shape/query38.out | 97 ++++++++++--------- .../shape/query97.out | 57 +++++------ .../tpcds_sf100/constraints/query23.out | 63 ++++++------ .../tpcds_sf100/noStatsRfPrune/query23.out | 63 ++++++------ .../tpcds_sf100/noStatsRfPrune/query32.out | 43 ++++---- .../tpcds_sf100/noStatsRfPrune/query38.out | 91 ++++++++--------- .../tpcds_sf100/noStatsRfPrune/query97.out | 57 +++++------ .../tpcds_sf100/no_stats_shape/query23.out | 63 ++++++------ .../tpcds_sf100/no_stats_shape/query32.out | 43 ++++---- .../tpcds_sf100/no_stats_shape/query38.out | 91 ++++++++--------- .../tpcds_sf100/no_stats_shape/query97.out | 57 +++++------ .../tpcds_sf100/rf_prune/query23.out | 63 ++++++------ .../tpcds_sf100/rf_prune/query32.out | 43 ++++---- .../tpcds_sf100/rf_prune/query38.out | 91 ++++++++--------- .../tpcds_sf100/rf_prune/query97.out | 57 +++++------ .../tpcds_sf100/shape/query23.out | 63 ++++++------ .../tpcds_sf100/shape/query32.out | 43 ++++---- .../tpcds_sf100/shape/query38.out | 91 ++++++++--------- .../tpcds_sf100/shape/query97.out | 57 +++++------ .../tpcds_sf1000/shape/query23.out | 63 ++++++------ .../tpcds_sf1000/shape/query32.out | 43 ++++---- .../tpcds_sf1000/shape/query38.out | 91 ++++++++--------- .../tpcds_sf1000/shape/query97.out | 61 ++++++------ .../tpch/push_topn_to_agg.groovy | 4 +- 54 files changed, 1679 insertions(+), 1589 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/LimitAggToTopNAgg.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/LimitAggToTopNAgg.java index 040bd9c10f3f3e..dfa1230a8f8f0e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/LimitAggToTopNAgg.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/LimitAggToTopNAgg.java @@ -21,6 +21,8 @@ import org.apache.doris.nereids.rules.Rule; import org.apache.doris.nereids.rules.RuleType; import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.expressions.SlotReference; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; import org.apache.doris.nereids.trees.plans.logical.LogicalLimit; @@ -32,6 +34,7 @@ import com.google.common.collect.Lists; import java.util.List; +import java.util.Optional; import java.util.stream.Collectors; /** @@ -53,7 +56,11 @@ public List buildRules() { >= limit.getLimit() + limit.getOffset()) .then(limit -> { LogicalAggregate agg = limit.child(); - List orderKeys = generateOrderKeyByGroupKey(agg); + Optional orderKeysOpt = tryGenerateOrderKeyByTheFirstGroupKey(agg); + if (!orderKeysOpt.isPresent()) { + return null; + } + List orderKeys = Lists.newArrayList(orderKeysOpt.get()); return new LogicalTopN<>(orderKeys, limit.getLimit(), limit.getOffset(), agg); }).toRule(RuleType.LIMIT_AGG_TO_TOPN_AGG), //limit->project->agg to topn->project->agg @@ -62,12 +69,47 @@ public List buildRules() { && ConnectContext.get().getSessionVariable().pushTopnToAgg && ConnectContext.get().getSessionVariable().topnOptLimitThreshold >= limit.getLimit() + limit.getOffset()) - .when(limit -> outputAllGroupKeys(limit, limit.child().child())) .then(limit -> { LogicalProject project = limit.child(); - LogicalAggregate agg = (LogicalAggregate) project.child(); - List orderKeys = generateOrderKeyByGroupKey(agg); - return new LogicalTopN<>(orderKeys, limit.getLimit(), limit.getOffset(), project); + LogicalAggregate agg + = (LogicalAggregate) project.child(); + Optional orderKeysOpt = tryGenerateOrderKeyByTheFirstGroupKey(agg); + if (!orderKeysOpt.isPresent()) { + return null; + } + List orderKeys = Lists.newArrayList(orderKeysOpt.get()); + Plan result; + + if (outputAllGroupKeys(limit, agg)) { + result = new LogicalTopN<>(orderKeys, limit.getLimit(), + limit.getOffset(), project); + } else { + // add the first group by key to topn, and prune this key by upper project + // topn order keys are prefix of group by keys + // refer to PushTopnToAgg.tryGenerateOrderKeyByGroupKeyAndTopnKey() + Expression firstGroupByKey = agg.getGroupByExpressions().get(0); + if (!(firstGroupByKey instanceof SlotReference)) { + return null; + } + boolean shouldPruneFirstGroupByKey = true; + if (project.getOutputs().contains(firstGroupByKey)) { + shouldPruneFirstGroupByKey = false; + } else { + List bottomProjections = Lists.newArrayList(project.getProjects()); + bottomProjections.add((SlotReference) firstGroupByKey); + project = project.withProjects(bottomProjections); + } + LogicalTopN topn = new LogicalTopN<>(orderKeys, limit.getLimit(), + limit.getOffset(), project); + if (shouldPruneFirstGroupByKey) { + List limitOutput = limit.getOutput().stream() + .map(e -> (NamedExpression) e).collect(Collectors.toList()); + result = new LogicalProject<>(limitOutput, topn); + } else { + result = topn; + } + } + return result; }).toRule(RuleType.LIMIT_AGG_TO_TOPN_AGG), // topn -> agg: add all group key to sort key, if sort key is prefix of group key logicalTopN(logicalAggregate()) @@ -111,9 +153,10 @@ private boolean outputAllGroupKeys(LogicalLimit limit, LogicalAggregate agg) { return limit.getOutputSet().containsAll(agg.getGroupByExpressions()); } - private List generateOrderKeyByGroupKey(LogicalAggregate agg) { - return agg.getGroupByExpressions().stream() - .map(key -> new OrderKey(key, true, false)) - .collect(Collectors.toList()); + private Optional tryGenerateOrderKeyByTheFirstGroupKey(LogicalAggregate agg) { + if (agg.getGroupByExpressions().isEmpty()) { + return Optional.empty(); + } + return Optional.of(new OrderKey(agg.getGroupByExpressions().get(0), true, false)); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalHashAggregate.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalHashAggregate.java index 2a78b063a973f3..11baf2137ab5d8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalHashAggregate.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalHashAggregate.java @@ -206,8 +206,8 @@ public String toString() { "groupByExpr", groupByExpressions, "outputExpr", outputExpressions, "partitionExpr", partitionExpressions, - "requireProperties", requireProperties, - "topnOpt", topnPushInfo != null + "topnFilter", topnPushInfo != null, + "topnPushDown", getMutableState(MutableState.KEY_PUSH_TOPN_TO_AGG).isPresent() ); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/EliminateSortTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/EliminateSortTest.java index ac1ffa97ef9e7d..cef9bac4ed90a7 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/EliminateSortTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/EliminateSortTest.java @@ -165,8 +165,7 @@ void testSortLimit() { PlanChecker.from(connectContext).disableNereidsRules("PRUNE_EMPTY_PARTITION") .analyze("select count(*) from (select * from student order by id) t limit 1") .rewrite() - // there is no topn below agg - .matches(logicalTopN(logicalAggregate(logicalProject(logicalOlapScan())))); + .nonMatch(logicalTopN()); PlanChecker.from(connectContext) .disableNereidsRules("PRUNE_EMPTY_PARTITION") .analyze("select count(*) from (select * from student order by id limit 1) t") @@ -184,8 +183,6 @@ void testSortLimit() { .analyze("select count(*) from " + "(select * from student order by id) t1 left join student t2 on t1.id = t2.id limit 1") .rewrite() - .matches(logicalTopN(logicalAggregate(logicalProject(logicalJoin( - logicalProject(logicalOlapScan()), - logicalProject(logicalOlapScan())))))); + .nonMatch(logicalTopN()); } } diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query23.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query23.out index 5e2eafb60ffe8d..45d7a47eb7e46a 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query23.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query23.out @@ -46,35 +46,36 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------filter(d_year IN (2000, 2001, 2002, 2003)) ----------------------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink -------PhysicalTopN[GATHER_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecGather] -------------hashAgg[LOCAL] ---------------PhysicalUnion -----------------PhysicalProject -------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF5 cs_item_sk->[item_sk] ---------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF5 ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() build RFs:RF4 c_customer_sk->[cs_bill_customer_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_moy = 7) and (date_dim.d_year = 2000)) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) -----------------PhysicalProject -------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF8 ws_item_sk->[item_sk] ---------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF8 ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((web_sales.ws_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() build RFs:RF7 c_customer_sk->[ws_bill_customer_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ws_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] apply RFs: RF6 RF7 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_moy = 7) and (date_dim.d_year = 2000)) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) +------PhysicalLimit[GLOBAL] +--------PhysicalLimit[LOCAL] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecGather] +--------------hashAgg[LOCAL] +----------------PhysicalUnion +------------------PhysicalProject +--------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF5 cs_item_sk->[item_sk] +----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF5 +----------------------PhysicalProject +------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() build RFs:RF4 c_customer_sk->[cs_bill_customer_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_moy = 7) and (date_dim.d_year = 2000)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) +------------------PhysicalProject +--------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF8 ws_item_sk->[item_sk] +----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF8 +----------------------PhysicalProject +------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((web_sales.ws_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() build RFs:RF7 c_customer_sk->[ws_bill_customer_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ws_sold_date_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[web_sales] apply RFs: RF6 RF7 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_moy = 7) and (date_dim.d_year = 2000)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query32.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query32.out index 152d2f05e77185..52b5030de53cbc 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query32.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query32.out @@ -1,27 +1,28 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_32 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[LOCAL] -----------PhysicalProject -------------filter((cast(cs_ext_discount_amt as DECIMALV3(38, 5)) > (1.3 * avg(cast(cs_ext_discount_amt as DECIMALV3(9, 4))) OVER(PARTITION BY i_item_sk)))) ---------------PhysicalWindow -----------------PhysicalQuickSort[LOCAL_SORT] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 -----------------------------PhysicalProject -------------------------------filter((item.i_manufact_id = 722)) ---------------------------------PhysicalOlapScan[item] -------------------------PhysicalProject ---------------------------filter((date_dim.d_date <= '2001-06-07') and (date_dim.d_date >= '2001-03-09')) -----------------------------PhysicalOlapScan[date_dim] +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[LOCAL] +------------PhysicalProject +--------------filter((cast(cs_ext_discount_amt as DECIMALV3(38, 5)) > (1.3 * avg(cast(cs_ext_discount_amt as DECIMALV3(9, 4))) OVER(PARTITION BY i_item_sk)))) +----------------PhysicalWindow +------------------PhysicalQuickSort[LOCAL_SORT] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 +------------------------------PhysicalProject +--------------------------------filter((item.i_manufact_id = 722)) +----------------------------------PhysicalOlapScan[item] +--------------------------PhysicalProject +----------------------------filter((date_dim.d_date <= '2001-06-07') and (date_dim.d_date >= '2001-03-09')) +------------------------------PhysicalOlapScan[date_dim] Hint log: Used: leading(catalog_sales item date_dim ) diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query38.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query38.out index 5ae805bdffe325..dc794c95f97b2f 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query38.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query38.out @@ -1,49 +1,50 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_38 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[LOCAL] -----------PhysicalProject -------------PhysicalIntersect ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ws_bill_customer_sk] -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq <= 1200) and (date_dim.d_month_seq >= 1189)) ---------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalOlapScan[customer] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq <= 1200) and (date_dim.d_month_seq >= 1189)) ---------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalOlapScan[customer] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF5 c_customer_sk->[ss_customer_sk] -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[store_sales] apply RFs: RF4 RF5 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq <= 1200) and (date_dim.d_month_seq >= 1189)) ---------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalOlapScan[customer] +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[LOCAL] +------------PhysicalProject +--------------PhysicalIntersect +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ws_bill_customer_sk] +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_month_seq <= 1200) and (date_dim.d_month_seq >= 1189)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalOlapScan[customer] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_month_seq <= 1200) and (date_dim.d_month_seq >= 1189)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalOlapScan[customer] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF5 c_customer_sk->[ss_customer_sk] +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[store_sales] apply RFs: RF4 RF5 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_month_seq <= 1200) and (date_dim.d_month_seq >= 1189)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query97.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query97.out index ffa14a9727df17..66b55c629d658a 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query97.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query97.out @@ -1,36 +1,37 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_97 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[LOCAL] -----------PhysicalProject -------------hashJoin[FULL_OUTER_JOIN colocated] hashCondition=((ssci.customer_sk = csci.customer_sk) and (ssci.item_sk = csci.item_sk)) otherCondition=() ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] ---------------------------PhysicalProject -----------------------------filter(( not ss_sold_date_sk IS NULL)) -------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 ---------------------------PhysicalProject -----------------------------filter((date_dim.d_month_seq <= 1210) and (date_dim.d_month_seq >= 1199)) -------------------------------PhysicalOlapScan[date_dim] ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] ---------------------------PhysicalProject -----------------------------filter(( not cs_sold_date_sk IS NULL)) -------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 ---------------------------PhysicalProject -----------------------------filter((date_dim.d_month_seq <= 1210) and (date_dim.d_month_seq >= 1199)) -------------------------------PhysicalOlapScan[date_dim] +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[LOCAL] +------------PhysicalProject +--------------hashJoin[FULL_OUTER_JOIN colocated] hashCondition=((ssci.customer_sk = csci.customer_sk) and (ssci.item_sk = csci.item_sk)) otherCondition=() +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +----------------------------PhysicalProject +------------------------------filter(( not ss_sold_date_sk IS NULL)) +--------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_month_seq <= 1210) and (date_dim.d_month_seq >= 1199)) +--------------------------------PhysicalOlapScan[date_dim] +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] +----------------------------PhysicalProject +------------------------------filter(( not cs_sold_date_sk IS NULL)) +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_month_seq <= 1210) and (date_dim.d_month_seq >= 1199)) +--------------------------------PhysicalOlapScan[date_dim] Hint log: Used: leading(store_sales date_dim ) leading(ssci csci ) diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query23.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query23.out index 5e2eafb60ffe8d..45d7a47eb7e46a 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query23.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query23.out @@ -46,35 +46,36 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------filter(d_year IN (2000, 2001, 2002, 2003)) ----------------------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink -------PhysicalTopN[GATHER_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecGather] -------------hashAgg[LOCAL] ---------------PhysicalUnion -----------------PhysicalProject -------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF5 cs_item_sk->[item_sk] ---------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF5 ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() build RFs:RF4 c_customer_sk->[cs_bill_customer_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_moy = 7) and (date_dim.d_year = 2000)) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) -----------------PhysicalProject -------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF8 ws_item_sk->[item_sk] ---------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF8 ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((web_sales.ws_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() build RFs:RF7 c_customer_sk->[ws_bill_customer_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ws_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] apply RFs: RF6 RF7 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_moy = 7) and (date_dim.d_year = 2000)) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) +------PhysicalLimit[GLOBAL] +--------PhysicalLimit[LOCAL] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecGather] +--------------hashAgg[LOCAL] +----------------PhysicalUnion +------------------PhysicalProject +--------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF5 cs_item_sk->[item_sk] +----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF5 +----------------------PhysicalProject +------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() build RFs:RF4 c_customer_sk->[cs_bill_customer_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_moy = 7) and (date_dim.d_year = 2000)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) +------------------PhysicalProject +--------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF8 ws_item_sk->[item_sk] +----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF8 +----------------------PhysicalProject +------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((web_sales.ws_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() build RFs:RF7 c_customer_sk->[ws_bill_customer_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ws_sold_date_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[web_sales] apply RFs: RF6 RF7 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_moy = 7) and (date_dim.d_year = 2000)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query32.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query32.out index 0389bfeea82673..c413940debc6e7 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query32.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query32.out @@ -1,25 +1,26 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_32 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[LOCAL] -----------PhysicalProject -------------filter((cast(cs_ext_discount_amt as DECIMALV3(38, 5)) > (1.3 * avg(cast(cs_ext_discount_amt as DECIMALV3(9, 4))) OVER(PARTITION BY i_item_sk)))) ---------------PhysicalWindow -----------------PhysicalQuickSort[LOCAL_SORT] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 -----------------------------PhysicalProject -------------------------------filter((item.i_manufact_id = 722)) ---------------------------------PhysicalOlapScan[item] -------------------------PhysicalProject ---------------------------filter((date_dim.d_date <= '2001-06-07') and (date_dim.d_date >= '2001-03-09')) -----------------------------PhysicalOlapScan[date_dim] +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[LOCAL] +------------PhysicalProject +--------------filter((cast(cs_ext_discount_amt as DECIMALV3(38, 5)) > (1.3 * avg(cast(cs_ext_discount_amt as DECIMALV3(9, 4))) OVER(PARTITION BY i_item_sk)))) +----------------PhysicalWindow +------------------PhysicalQuickSort[LOCAL_SORT] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 +------------------------------PhysicalProject +--------------------------------filter((item.i_manufact_id = 722)) +----------------------------------PhysicalOlapScan[item] +--------------------------PhysicalProject +----------------------------filter((date_dim.d_date <= '2001-06-07') and (date_dim.d_date >= '2001-03-09')) +------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query38.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query38.out index 5ae805bdffe325..dc794c95f97b2f 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query38.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query38.out @@ -1,49 +1,50 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_38 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[LOCAL] -----------PhysicalProject -------------PhysicalIntersect ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ws_bill_customer_sk] -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq <= 1200) and (date_dim.d_month_seq >= 1189)) ---------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalOlapScan[customer] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq <= 1200) and (date_dim.d_month_seq >= 1189)) ---------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalOlapScan[customer] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF5 c_customer_sk->[ss_customer_sk] -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[store_sales] apply RFs: RF4 RF5 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq <= 1200) and (date_dim.d_month_seq >= 1189)) ---------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalOlapScan[customer] +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[LOCAL] +------------PhysicalProject +--------------PhysicalIntersect +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ws_bill_customer_sk] +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_month_seq <= 1200) and (date_dim.d_month_seq >= 1189)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalOlapScan[customer] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_month_seq <= 1200) and (date_dim.d_month_seq >= 1189)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalOlapScan[customer] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF5 c_customer_sk->[ss_customer_sk] +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[store_sales] apply RFs: RF4 RF5 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_month_seq <= 1200) and (date_dim.d_month_seq >= 1189)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query97.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query97.out index ca6f63d5e6bdf0..d3a845763241f7 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query97.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query97.out @@ -1,34 +1,35 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_97 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[LOCAL] -----------PhysicalProject -------------hashJoin[FULL_OUTER_JOIN colocated] hashCondition=((ssci.customer_sk = csci.customer_sk) and (ssci.item_sk = csci.item_sk)) otherCondition=() ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] ---------------------------PhysicalProject -----------------------------filter(( not ss_sold_date_sk IS NULL)) -------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 ---------------------------PhysicalProject -----------------------------filter((date_dim.d_month_seq <= 1210) and (date_dim.d_month_seq >= 1199)) -------------------------------PhysicalOlapScan[date_dim] ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] ---------------------------PhysicalProject -----------------------------filter(( not cs_sold_date_sk IS NULL)) -------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 ---------------------------PhysicalProject -----------------------------filter((date_dim.d_month_seq <= 1210) and (date_dim.d_month_seq >= 1199)) -------------------------------PhysicalOlapScan[date_dim] +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[LOCAL] +------------PhysicalProject +--------------hashJoin[FULL_OUTER_JOIN colocated] hashCondition=((ssci.customer_sk = csci.customer_sk) and (ssci.item_sk = csci.item_sk)) otherCondition=() +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +----------------------------PhysicalProject +------------------------------filter(( not ss_sold_date_sk IS NULL)) +--------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_month_seq <= 1210) and (date_dim.d_month_seq >= 1199)) +--------------------------------PhysicalOlapScan[date_dim] +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] +----------------------------PhysicalProject +------------------------------filter(( not cs_sold_date_sk IS NULL)) +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_month_seq <= 1210) and (date_dim.d_month_seq >= 1199)) +--------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/constraints/query23.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/constraints/query23.out index 85217462d1a5ed..923ee0e2966746 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/constraints/query23.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/constraints/query23.out @@ -46,35 +46,36 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------filter(d_year IN (2000, 2001, 2002, 2003)) ----------------------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink -------PhysicalTopN[GATHER_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecGather] -------------hashAgg[LOCAL] ---------------PhysicalUnion -----------------PhysicalProject -------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF5 item_sk->[cs_item_sk] ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() build RFs:RF4 c_customer_sk->[cs_bill_customer_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 RF5 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) ---------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -----------------PhysicalProject -------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF8 item_sk->[ws_item_sk] ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((web_sales.ws_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() build RFs:RF7 c_customer_sk->[ws_bill_customer_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ws_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] apply RFs: RF6 RF7 RF8 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) ---------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------PhysicalLimit[GLOBAL] +--------PhysicalLimit[LOCAL] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecGather] +--------------hashAgg[LOCAL] +----------------PhysicalUnion +------------------PhysicalProject +--------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF5 item_sk->[cs_item_sk] +----------------------PhysicalProject +------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() build RFs:RF4 c_customer_sk->[cs_bill_customer_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 RF5 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) +----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------PhysicalProject +--------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF8 item_sk->[ws_item_sk] +----------------------PhysicalProject +------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((web_sales.ws_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() build RFs:RF7 c_customer_sk->[ws_bill_customer_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ws_sold_date_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[web_sales] apply RFs: RF6 RF7 RF8 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) +----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query23.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query23.out index aa0bb3fa70d0e7..c5d202bfee2bc3 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query23.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query23.out @@ -46,35 +46,36 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------filter(d_year IN (2000, 2001, 2002, 2003)) ----------------------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink -------PhysicalTopN[GATHER_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecGather] -------------hashAgg[LOCAL] ---------------PhysicalUnion -----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = frequent_ss_items.item_sk)) otherCondition=() -------------------------PhysicalProject ---------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF5 -----------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------------PhysicalProject -----------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) -------------------------PhysicalOlapScan[date_dim] -----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF8 d_date_sk->[ws_sold_date_sk] ---------------------PhysicalProject -----------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF7 ws_item_sk->[item_sk] -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF7 -------------------------PhysicalProject ---------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] apply RFs: RF8 -----------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) ---------------------PhysicalProject -----------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) -------------------------PhysicalOlapScan[date_dim] +------PhysicalLimit[GLOBAL] +--------PhysicalLimit[LOCAL] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecGather] +--------------hashAgg[LOCAL] +----------------PhysicalUnion +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] +----------------------PhysicalProject +------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = frequent_ss_items.item_sk)) otherCondition=() +--------------------------PhysicalProject +----------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF5 +------------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------PhysicalProject +------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) +--------------------------PhysicalOlapScan[date_dim] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF8 d_date_sk->[ws_sold_date_sk] +----------------------PhysicalProject +------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF7 ws_item_sk->[item_sk] +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF7 +--------------------------PhysicalProject +----------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[web_sales] apply RFs: RF8 +------------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) +----------------------PhysicalProject +------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) +--------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query32.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query32.out index 15393efb582c73..7992f57d1c87b3 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query32.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query32.out @@ -1,25 +1,26 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_32 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[LOCAL] -----------PhysicalProject -------------filter((cast(cs_ext_discount_amt as DECIMALV3(38, 5)) > (1.3 * avg(cast(cs_ext_discount_amt as DECIMALV3(9, 4))) OVER(PARTITION BY i_item_sk)))) ---------------PhysicalWindow -----------------PhysicalQuickSort[LOCAL_SORT] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 -----------------------------PhysicalProject -------------------------------filter((item.i_manufact_id = 29)) ---------------------------------PhysicalOlapScan[item] -------------------------PhysicalProject ---------------------------filter((date_dim.d_date <= '1999-04-07') and (date_dim.d_date >= '1999-01-07')) -----------------------------PhysicalOlapScan[date_dim] +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[LOCAL] +------------PhysicalProject +--------------filter((cast(cs_ext_discount_amt as DECIMALV3(38, 5)) > (1.3 * avg(cast(cs_ext_discount_amt as DECIMALV3(9, 4))) OVER(PARTITION BY i_item_sk)))) +----------------PhysicalWindow +------------------PhysicalQuickSort[LOCAL_SORT] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 +------------------------------PhysicalProject +--------------------------------filter((item.i_manufact_id = 29)) +----------------------------------PhysicalOlapScan[item] +--------------------------PhysicalProject +----------------------------filter((date_dim.d_date <= '1999-04-07') and (date_dim.d_date >= '1999-01-07')) +------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query38.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query38.out index 304fa6dd35c5f6..b0bda7f927c138 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query38.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query38.out @@ -1,49 +1,50 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_38 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[LOCAL] -----------PhysicalProject -------------PhysicalIntersect ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) ---------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalOlapScan[customer] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) ---------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalOlapScan[customer] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[store_sales] apply RFs: RF4 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) ---------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalOlapScan[customer] +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[LOCAL] +------------PhysicalProject +--------------PhysicalIntersect +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalOlapScan[customer] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalOlapScan[customer] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[store_sales] apply RFs: RF4 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query97.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query97.out index 404a9e7e9e92a1..4ebfd5abc0eb1c 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query97.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query97.out @@ -1,32 +1,33 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_97 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[LOCAL] -----------PhysicalProject -------------hashJoin[FULL_OUTER_JOIN colocated] hashCondition=((ssci.customer_sk = csci.customer_sk) and (ssci.item_sk = csci.item_sk)) otherCondition=() ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 ---------------------------PhysicalProject -----------------------------filter((date_dim.d_month_seq <= 1225) and (date_dim.d_month_seq >= 1214)) -------------------------------PhysicalOlapScan[date_dim] ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 ---------------------------PhysicalProject -----------------------------filter((date_dim.d_month_seq <= 1225) and (date_dim.d_month_seq >= 1214)) -------------------------------PhysicalOlapScan[date_dim] +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[LOCAL] +------------PhysicalProject +--------------hashJoin[FULL_OUTER_JOIN colocated] hashCondition=((ssci.customer_sk = csci.customer_sk) and (ssci.item_sk = csci.item_sk)) otherCondition=() +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_month_seq <= 1225) and (date_dim.d_month_seq >= 1214)) +--------------------------------PhysicalOlapScan[date_dim] +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_month_seq <= 1225) and (date_dim.d_month_seq >= 1214)) +--------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query23.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query23.out index 26e1f5089288a1..a65c74fa0b81dd 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query23.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query23.out @@ -46,35 +46,36 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------filter(d_year IN (2000, 2001, 2002, 2003)) ----------------------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink -------PhysicalTopN[GATHER_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecGather] -------------hashAgg[LOCAL] ---------------PhysicalUnion -----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF4 item_sk->[cs_item_sk] -------------------------PhysicalProject ---------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 RF5 -----------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------------PhysicalProject -----------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) -------------------------PhysicalOlapScan[date_dim] -----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF8 d_date_sk->[ws_sold_date_sk] ---------------------PhysicalProject -----------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF7 ws_item_sk->[item_sk] -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF7 -------------------------PhysicalProject ---------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() build RFs:RF6 c_customer_sk->[ws_bill_customer_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] apply RFs: RF6 RF8 -----------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) ---------------------PhysicalProject -----------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) -------------------------PhysicalOlapScan[date_dim] +------PhysicalLimit[GLOBAL] +--------PhysicalLimit[LOCAL] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecGather] +--------------hashAgg[LOCAL] +----------------PhysicalUnion +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] +----------------------PhysicalProject +------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF4 item_sk->[cs_item_sk] +--------------------------PhysicalProject +----------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 RF5 +------------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------PhysicalProject +------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) +--------------------------PhysicalOlapScan[date_dim] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF8 d_date_sk->[ws_sold_date_sk] +----------------------PhysicalProject +------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF7 ws_item_sk->[item_sk] +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF7 +--------------------------PhysicalProject +----------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() build RFs:RF6 c_customer_sk->[ws_bill_customer_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[web_sales] apply RFs: RF6 RF8 +------------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) +----------------------PhysicalProject +------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) +--------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query32.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query32.out index 15393efb582c73..7992f57d1c87b3 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query32.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query32.out @@ -1,25 +1,26 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_32 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[LOCAL] -----------PhysicalProject -------------filter((cast(cs_ext_discount_amt as DECIMALV3(38, 5)) > (1.3 * avg(cast(cs_ext_discount_amt as DECIMALV3(9, 4))) OVER(PARTITION BY i_item_sk)))) ---------------PhysicalWindow -----------------PhysicalQuickSort[LOCAL_SORT] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 -----------------------------PhysicalProject -------------------------------filter((item.i_manufact_id = 29)) ---------------------------------PhysicalOlapScan[item] -------------------------PhysicalProject ---------------------------filter((date_dim.d_date <= '1999-04-07') and (date_dim.d_date >= '1999-01-07')) -----------------------------PhysicalOlapScan[date_dim] +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[LOCAL] +------------PhysicalProject +--------------filter((cast(cs_ext_discount_amt as DECIMALV3(38, 5)) > (1.3 * avg(cast(cs_ext_discount_amt as DECIMALV3(9, 4))) OVER(PARTITION BY i_item_sk)))) +----------------PhysicalWindow +------------------PhysicalQuickSort[LOCAL_SORT] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 +------------------------------PhysicalProject +--------------------------------filter((item.i_manufact_id = 29)) +----------------------------------PhysicalOlapScan[item] +--------------------------PhysicalProject +----------------------------filter((date_dim.d_date <= '1999-04-07') and (date_dim.d_date >= '1999-01-07')) +------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query38.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query38.out index fbc5dd43162046..2bc9d9fecbca40 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query38.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query38.out @@ -1,49 +1,50 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_38 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[LOCAL] -----------PhysicalProject -------------PhysicalIntersect ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ws_bill_customer_sk] -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) ---------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalOlapScan[customer] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) ---------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalOlapScan[customer] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF5 c_customer_sk->[ss_customer_sk] -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[store_sales] apply RFs: RF4 RF5 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) ---------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalOlapScan[customer] +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[LOCAL] +------------PhysicalProject +--------------PhysicalIntersect +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ws_bill_customer_sk] +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalOlapScan[customer] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalOlapScan[customer] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF5 c_customer_sk->[ss_customer_sk] +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[store_sales] apply RFs: RF4 RF5 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query97.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query97.out index 404a9e7e9e92a1..4ebfd5abc0eb1c 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query97.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query97.out @@ -1,32 +1,33 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_97 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[LOCAL] -----------PhysicalProject -------------hashJoin[FULL_OUTER_JOIN colocated] hashCondition=((ssci.customer_sk = csci.customer_sk) and (ssci.item_sk = csci.item_sk)) otherCondition=() ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 ---------------------------PhysicalProject -----------------------------filter((date_dim.d_month_seq <= 1225) and (date_dim.d_month_seq >= 1214)) -------------------------------PhysicalOlapScan[date_dim] ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 ---------------------------PhysicalProject -----------------------------filter((date_dim.d_month_seq <= 1225) and (date_dim.d_month_seq >= 1214)) -------------------------------PhysicalOlapScan[date_dim] +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[LOCAL] +------------PhysicalProject +--------------hashJoin[FULL_OUTER_JOIN colocated] hashCondition=((ssci.customer_sk = csci.customer_sk) and (ssci.item_sk = csci.item_sk)) otherCondition=() +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_month_seq <= 1225) and (date_dim.d_month_seq >= 1214)) +--------------------------------PhysicalOlapScan[date_dim] +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_month_seq <= 1225) and (date_dim.d_month_seq >= 1214)) +--------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query23.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query23.out index 1343c4f4fb0214..6103f4eb74d6ab 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query23.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query23.out @@ -46,35 +46,36 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------filter(d_year IN (2000, 2001, 2002, 2003)) ----------------------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink -------PhysicalTopN[GATHER_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecGather] -------------hashAgg[LOCAL] ---------------PhysicalUnion -----------------PhysicalProject -------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF5 cs_item_sk->[item_sk] ---------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF5 ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() build RFs:RF4 c_customer_sk->[cs_bill_customer_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) -----------------PhysicalProject -------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF8 ws_item_sk->[item_sk] ---------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF8 ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((web_sales.ws_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() build RFs:RF7 c_customer_sk->[ws_bill_customer_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ws_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] apply RFs: RF6 RF7 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) +------PhysicalLimit[GLOBAL] +--------PhysicalLimit[LOCAL] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecGather] +--------------hashAgg[LOCAL] +----------------PhysicalUnion +------------------PhysicalProject +--------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF5 cs_item_sk->[item_sk] +----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF5 +----------------------PhysicalProject +------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() build RFs:RF4 c_customer_sk->[cs_bill_customer_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) +------------------PhysicalProject +--------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF8 ws_item_sk->[item_sk] +----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF8 +----------------------PhysicalProject +------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((web_sales.ws_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() build RFs:RF7 c_customer_sk->[ws_bill_customer_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ws_sold_date_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[web_sales] apply RFs: RF6 RF7 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query32.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query32.out index 15393efb582c73..7992f57d1c87b3 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query32.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query32.out @@ -1,25 +1,26 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_32 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[LOCAL] -----------PhysicalProject -------------filter((cast(cs_ext_discount_amt as DECIMALV3(38, 5)) > (1.3 * avg(cast(cs_ext_discount_amt as DECIMALV3(9, 4))) OVER(PARTITION BY i_item_sk)))) ---------------PhysicalWindow -----------------PhysicalQuickSort[LOCAL_SORT] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 -----------------------------PhysicalProject -------------------------------filter((item.i_manufact_id = 29)) ---------------------------------PhysicalOlapScan[item] -------------------------PhysicalProject ---------------------------filter((date_dim.d_date <= '1999-04-07') and (date_dim.d_date >= '1999-01-07')) -----------------------------PhysicalOlapScan[date_dim] +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[LOCAL] +------------PhysicalProject +--------------filter((cast(cs_ext_discount_amt as DECIMALV3(38, 5)) > (1.3 * avg(cast(cs_ext_discount_amt as DECIMALV3(9, 4))) OVER(PARTITION BY i_item_sk)))) +----------------PhysicalWindow +------------------PhysicalQuickSort[LOCAL_SORT] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 +------------------------------PhysicalProject +--------------------------------filter((item.i_manufact_id = 29)) +----------------------------------PhysicalOlapScan[item] +--------------------------PhysicalProject +----------------------------filter((date_dim.d_date <= '1999-04-07') and (date_dim.d_date >= '1999-01-07')) +------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query38.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query38.out index 802779104ed84f..26d52e9fae9c40 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query38.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query38.out @@ -1,49 +1,50 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_38 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[LOCAL] -----------PhysicalProject -------------PhysicalIntersect ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) ---------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalOlapScan[customer] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) ---------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalOlapScan[customer] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[store_sales] apply RFs: RF4 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) ---------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalOlapScan[customer] +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[LOCAL] +------------PhysicalProject +--------------PhysicalIntersect +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalOlapScan[customer] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalOlapScan[customer] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[store_sales] apply RFs: RF4 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query97.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query97.out index 404a9e7e9e92a1..4ebfd5abc0eb1c 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query97.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query97.out @@ -1,32 +1,33 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_97 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[LOCAL] -----------PhysicalProject -------------hashJoin[FULL_OUTER_JOIN colocated] hashCondition=((ssci.customer_sk = csci.customer_sk) and (ssci.item_sk = csci.item_sk)) otherCondition=() ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 ---------------------------PhysicalProject -----------------------------filter((date_dim.d_month_seq <= 1225) and (date_dim.d_month_seq >= 1214)) -------------------------------PhysicalOlapScan[date_dim] ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 ---------------------------PhysicalProject -----------------------------filter((date_dim.d_month_seq <= 1225) and (date_dim.d_month_seq >= 1214)) -------------------------------PhysicalOlapScan[date_dim] +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[LOCAL] +------------PhysicalProject +--------------hashJoin[FULL_OUTER_JOIN colocated] hashCondition=((ssci.customer_sk = csci.customer_sk) and (ssci.item_sk = csci.item_sk)) otherCondition=() +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_month_seq <= 1225) and (date_dim.d_month_seq >= 1214)) +--------------------------------PhysicalOlapScan[date_dim] +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_month_seq <= 1225) and (date_dim.d_month_seq >= 1214)) +--------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query23.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query23.out index 91ab82284bc473..0d8b21c19639e9 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query23.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query23.out @@ -46,35 +46,36 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------filter(d_year IN (2000, 2001, 2002, 2003)) ----------------------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink -------PhysicalTopN[GATHER_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecGather] -------------hashAgg[LOCAL] ---------------PhysicalUnion -----------------PhysicalProject -------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF5 cs_item_sk->[item_sk] ---------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF5 ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() build RFs:RF4 c_customer_sk->[cs_bill_customer_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) -----------------PhysicalProject -------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF8 ws_item_sk->[item_sk] ---------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF8 ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((web_sales.ws_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() build RFs:RF7 c_customer_sk->[ws_bill_customer_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ws_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] apply RFs: RF6 RF7 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) +------PhysicalLimit[GLOBAL] +--------PhysicalLimit[LOCAL] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecGather] +--------------hashAgg[LOCAL] +----------------PhysicalUnion +------------------PhysicalProject +--------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF5 cs_item_sk->[item_sk] +----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF5 +----------------------PhysicalProject +------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() build RFs:RF4 c_customer_sk->[cs_bill_customer_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) +------------------PhysicalProject +--------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF8 ws_item_sk->[item_sk] +----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF8 +----------------------PhysicalProject +------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((web_sales.ws_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() build RFs:RF7 c_customer_sk->[ws_bill_customer_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ws_sold_date_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[web_sales] apply RFs: RF6 RF7 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query32.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query32.out index 15393efb582c73..7992f57d1c87b3 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query32.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query32.out @@ -1,25 +1,26 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_32 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[LOCAL] -----------PhysicalProject -------------filter((cast(cs_ext_discount_amt as DECIMALV3(38, 5)) > (1.3 * avg(cast(cs_ext_discount_amt as DECIMALV3(9, 4))) OVER(PARTITION BY i_item_sk)))) ---------------PhysicalWindow -----------------PhysicalQuickSort[LOCAL_SORT] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 -----------------------------PhysicalProject -------------------------------filter((item.i_manufact_id = 29)) ---------------------------------PhysicalOlapScan[item] -------------------------PhysicalProject ---------------------------filter((date_dim.d_date <= '1999-04-07') and (date_dim.d_date >= '1999-01-07')) -----------------------------PhysicalOlapScan[date_dim] +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[LOCAL] +------------PhysicalProject +--------------filter((cast(cs_ext_discount_amt as DECIMALV3(38, 5)) > (1.3 * avg(cast(cs_ext_discount_amt as DECIMALV3(9, 4))) OVER(PARTITION BY i_item_sk)))) +----------------PhysicalWindow +------------------PhysicalQuickSort[LOCAL_SORT] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 +------------------------------PhysicalProject +--------------------------------filter((item.i_manufact_id = 29)) +----------------------------------PhysicalOlapScan[item] +--------------------------PhysicalProject +----------------------------filter((date_dim.d_date <= '1999-04-07') and (date_dim.d_date >= '1999-01-07')) +------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query38.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query38.out index 9a6c130acd1eaf..ef2051eb2d000c 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query38.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query38.out @@ -1,49 +1,50 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_38 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[LOCAL] -----------PhysicalProject -------------PhysicalIntersect ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ws_bill_customer_sk] -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) ---------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalOlapScan[customer] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) ---------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalOlapScan[customer] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF5 c_customer_sk->[ss_customer_sk] -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[store_sales] apply RFs: RF4 RF5 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) ---------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalOlapScan[customer] +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[LOCAL] +------------PhysicalProject +--------------PhysicalIntersect +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ws_bill_customer_sk] +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalOlapScan[customer] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalOlapScan[customer] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF5 c_customer_sk->[ss_customer_sk] +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[store_sales] apply RFs: RF4 RF5 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query97.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query97.out index 404a9e7e9e92a1..4ebfd5abc0eb1c 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query97.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query97.out @@ -1,32 +1,33 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_97 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[LOCAL] -----------PhysicalProject -------------hashJoin[FULL_OUTER_JOIN colocated] hashCondition=((ssci.customer_sk = csci.customer_sk) and (ssci.item_sk = csci.item_sk)) otherCondition=() ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 ---------------------------PhysicalProject -----------------------------filter((date_dim.d_month_seq <= 1225) and (date_dim.d_month_seq >= 1214)) -------------------------------PhysicalOlapScan[date_dim] ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 ---------------------------PhysicalProject -----------------------------filter((date_dim.d_month_seq <= 1225) and (date_dim.d_month_seq >= 1214)) -------------------------------PhysicalOlapScan[date_dim] +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[LOCAL] +------------PhysicalProject +--------------hashJoin[FULL_OUTER_JOIN colocated] hashCondition=((ssci.customer_sk = csci.customer_sk) and (ssci.item_sk = csci.item_sk)) otherCondition=() +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_month_seq <= 1225) and (date_dim.d_month_seq >= 1214)) +--------------------------------PhysicalOlapScan[date_dim] +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_month_seq <= 1225) and (date_dim.d_month_seq >= 1214)) +--------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query23.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query23.out index 5b81b4629a48e0..eb873d6b08706a 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query23.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query23.out @@ -48,35 +48,36 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------------------filter(d_year IN (2000, 2001, 2002, 2003)) ------------------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink -------PhysicalTopN[GATHER_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecGather] -------------hashAgg[LOCAL] ---------------PhysicalUnion -----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[cs_sold_date_sk] ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() build RFs:RF6 c_customer_sk->[cs_bill_customer_sk] -------------------------PhysicalProject ---------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF5 item_sk->[cs_item_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF5 RF6 RF7 -----------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) ---------------------PhysicalProject -----------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2000)) -------------------------PhysicalOlapScan[date_dim] -----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[ws_sold_date_sk] ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() build RFs:RF9 c_customer_sk->[ws_bill_customer_sk] -------------------------PhysicalProject ---------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF8 item_sk->[ws_item_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] apply RFs: RF8 RF9 RF10 -----------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) ---------------------PhysicalProject -----------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2000)) -------------------------PhysicalOlapScan[date_dim] +------PhysicalLimit[GLOBAL] +--------PhysicalLimit[LOCAL] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecGather] +--------------hashAgg[LOCAL] +----------------PhysicalUnion +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[cs_sold_date_sk] +----------------------PhysicalProject +------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() build RFs:RF6 c_customer_sk->[cs_bill_customer_sk] +--------------------------PhysicalProject +----------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF5 item_sk->[cs_item_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF5 RF6 RF7 +------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) +----------------------PhysicalProject +------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2000)) +--------------------------PhysicalOlapScan[date_dim] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[ws_sold_date_sk] +----------------------PhysicalProject +------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() build RFs:RF9 c_customer_sk->[ws_bill_customer_sk] +--------------------------PhysicalProject +----------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF8 item_sk->[ws_item_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[web_sales] apply RFs: RF8 RF9 RF10 +------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) +----------------------PhysicalProject +------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2000)) +--------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query32.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query32.out index a9acbffc271bf6..0a251a341c24df 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query32.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query32.out @@ -1,25 +1,26 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_32 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[LOCAL] -----------PhysicalProject -------------filter((cast(cs_ext_discount_amt as DECIMALV3(38, 5)) > (1.3 * avg(cast(cs_ext_discount_amt as DECIMALV3(9, 4))) OVER(PARTITION BY i_item_sk)))) ---------------PhysicalWindow -----------------PhysicalQuickSort[LOCAL_SORT] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 -----------------------------PhysicalProject -------------------------------filter((item.i_manufact_id = 66)) ---------------------------------PhysicalOlapScan[item] -------------------------PhysicalProject ---------------------------filter((date_dim.d_date <= '2002-06-27') and (date_dim.d_date >= '2002-03-29')) -----------------------------PhysicalOlapScan[date_dim] +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[LOCAL] +------------PhysicalProject +--------------filter((cast(cs_ext_discount_amt as DECIMALV3(38, 5)) > (1.3 * avg(cast(cs_ext_discount_amt as DECIMALV3(9, 4))) OVER(PARTITION BY i_item_sk)))) +----------------PhysicalWindow +------------------PhysicalQuickSort[LOCAL_SORT] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 +------------------------------PhysicalProject +--------------------------------filter((item.i_manufact_id = 66)) +----------------------------------PhysicalOlapScan[item] +--------------------------PhysicalProject +----------------------------filter((date_dim.d_date <= '2002-06-27') and (date_dim.d_date >= '2002-03-29')) +------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query38.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query38.out index 08d22c95f4ff40..ce87ec57d57863 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query38.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query38.out @@ -1,52 +1,53 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_38 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[LOCAL] -----------PhysicalProject -------------PhysicalIntersect ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF0 c_customer_sk->[ws_bill_customer_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer] -------------------------PhysicalProject ---------------------------filter((date_dim.d_month_seq <= 1197) and (date_dim.d_month_seq >= 1186)) -----------------------------PhysicalOlapScan[date_dim] ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[cs_bill_customer_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer] -------------------------PhysicalProject ---------------------------filter((date_dim.d_month_seq <= 1197) and (date_dim.d_month_seq >= 1186)) -----------------------------PhysicalOlapScan[date_dim] ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[ss_sold_date_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF4 c_customer_sk->[ss_customer_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[store_sales] apply RFs: RF4 RF5 -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer] -------------------------PhysicalProject ---------------------------filter((date_dim.d_month_seq <= 1197) and (date_dim.d_month_seq >= 1186)) -----------------------------PhysicalOlapScan[date_dim] +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[LOCAL] +------------PhysicalProject +--------------PhysicalIntersect +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF0 c_customer_sk->[ws_bill_customer_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[customer] +--------------------------PhysicalProject +----------------------------filter((date_dim.d_month_seq <= 1197) and (date_dim.d_month_seq >= 1186)) +------------------------------PhysicalOlapScan[date_dim] +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[cs_bill_customer_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[customer] +--------------------------PhysicalProject +----------------------------filter((date_dim.d_month_seq <= 1197) and (date_dim.d_month_seq >= 1186)) +------------------------------PhysicalOlapScan[date_dim] +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[ss_sold_date_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF4 c_customer_sk->[ss_customer_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[store_sales] apply RFs: RF4 RF5 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[customer] +--------------------------PhysicalProject +----------------------------filter((date_dim.d_month_seq <= 1197) and (date_dim.d_month_seq >= 1186)) +------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query97.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query97.out index 199f8d20f62131..0960d491884815 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query97.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query97.out @@ -1,32 +1,33 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_97 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[LOCAL] -----------PhysicalProject -------------hashJoin[FULL_OUTER_JOIN colocated] hashCondition=((ssci.customer_sk = csci.customer_sk) and (ssci.item_sk = csci.item_sk)) otherCondition=() ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 ---------------------------PhysicalProject -----------------------------filter((date_dim.d_month_seq <= 1201) and (date_dim.d_month_seq >= 1190)) -------------------------------PhysicalOlapScan[date_dim] ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 ---------------------------PhysicalProject -----------------------------filter((date_dim.d_month_seq <= 1201) and (date_dim.d_month_seq >= 1190)) -------------------------------PhysicalOlapScan[date_dim] +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[LOCAL] +------------PhysicalProject +--------------hashJoin[FULL_OUTER_JOIN colocated] hashCondition=((ssci.customer_sk = csci.customer_sk) and (ssci.item_sk = csci.item_sk)) otherCondition=() +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_month_seq <= 1201) and (date_dim.d_month_seq >= 1190)) +--------------------------------PhysicalOlapScan[date_dim] +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_month_seq <= 1201) and (date_dim.d_month_seq >= 1190)) +--------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/constraints/query23.out b/regression-test/data/new_shapes_p0/tpcds_sf100/constraints/query23.out index 85217462d1a5ed..923ee0e2966746 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/constraints/query23.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/constraints/query23.out @@ -46,35 +46,36 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------filter(d_year IN (2000, 2001, 2002, 2003)) ----------------------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink -------PhysicalTopN[GATHER_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecGather] -------------hashAgg[LOCAL] ---------------PhysicalUnion -----------------PhysicalProject -------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF5 item_sk->[cs_item_sk] ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() build RFs:RF4 c_customer_sk->[cs_bill_customer_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 RF5 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) ---------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -----------------PhysicalProject -------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF8 item_sk->[ws_item_sk] ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((web_sales.ws_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() build RFs:RF7 c_customer_sk->[ws_bill_customer_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ws_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] apply RFs: RF6 RF7 RF8 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) ---------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------PhysicalLimit[GLOBAL] +--------PhysicalLimit[LOCAL] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecGather] +--------------hashAgg[LOCAL] +----------------PhysicalUnion +------------------PhysicalProject +--------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF5 item_sk->[cs_item_sk] +----------------------PhysicalProject +------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() build RFs:RF4 c_customer_sk->[cs_bill_customer_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 RF5 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) +----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------PhysicalProject +--------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF8 item_sk->[ws_item_sk] +----------------------PhysicalProject +------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((web_sales.ws_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() build RFs:RF7 c_customer_sk->[ws_bill_customer_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ws_sold_date_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[web_sales] apply RFs: RF6 RF7 RF8 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) +----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query23.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query23.out index aa0bb3fa70d0e7..c5d202bfee2bc3 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query23.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query23.out @@ -46,35 +46,36 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------filter(d_year IN (2000, 2001, 2002, 2003)) ----------------------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink -------PhysicalTopN[GATHER_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecGather] -------------hashAgg[LOCAL] ---------------PhysicalUnion -----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = frequent_ss_items.item_sk)) otherCondition=() -------------------------PhysicalProject ---------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF5 -----------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------------PhysicalProject -----------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) -------------------------PhysicalOlapScan[date_dim] -----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF8 d_date_sk->[ws_sold_date_sk] ---------------------PhysicalProject -----------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF7 ws_item_sk->[item_sk] -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF7 -------------------------PhysicalProject ---------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] apply RFs: RF8 -----------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) ---------------------PhysicalProject -----------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) -------------------------PhysicalOlapScan[date_dim] +------PhysicalLimit[GLOBAL] +--------PhysicalLimit[LOCAL] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecGather] +--------------hashAgg[LOCAL] +----------------PhysicalUnion +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] +----------------------PhysicalProject +------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = frequent_ss_items.item_sk)) otherCondition=() +--------------------------PhysicalProject +----------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF5 +------------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------PhysicalProject +------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) +--------------------------PhysicalOlapScan[date_dim] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF8 d_date_sk->[ws_sold_date_sk] +----------------------PhysicalProject +------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF7 ws_item_sk->[item_sk] +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF7 +--------------------------PhysicalProject +----------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[web_sales] apply RFs: RF8 +------------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) +----------------------PhysicalProject +------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) +--------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query32.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query32.out index 15393efb582c73..7992f57d1c87b3 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query32.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query32.out @@ -1,25 +1,26 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_32 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[LOCAL] -----------PhysicalProject -------------filter((cast(cs_ext_discount_amt as DECIMALV3(38, 5)) > (1.3 * avg(cast(cs_ext_discount_amt as DECIMALV3(9, 4))) OVER(PARTITION BY i_item_sk)))) ---------------PhysicalWindow -----------------PhysicalQuickSort[LOCAL_SORT] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 -----------------------------PhysicalProject -------------------------------filter((item.i_manufact_id = 29)) ---------------------------------PhysicalOlapScan[item] -------------------------PhysicalProject ---------------------------filter((date_dim.d_date <= '1999-04-07') and (date_dim.d_date >= '1999-01-07')) -----------------------------PhysicalOlapScan[date_dim] +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[LOCAL] +------------PhysicalProject +--------------filter((cast(cs_ext_discount_amt as DECIMALV3(38, 5)) > (1.3 * avg(cast(cs_ext_discount_amt as DECIMALV3(9, 4))) OVER(PARTITION BY i_item_sk)))) +----------------PhysicalWindow +------------------PhysicalQuickSort[LOCAL_SORT] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 +------------------------------PhysicalProject +--------------------------------filter((item.i_manufact_id = 29)) +----------------------------------PhysicalOlapScan[item] +--------------------------PhysicalProject +----------------------------filter((date_dim.d_date <= '1999-04-07') and (date_dim.d_date >= '1999-01-07')) +------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query38.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query38.out index 304fa6dd35c5f6..b0bda7f927c138 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query38.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query38.out @@ -1,49 +1,50 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_38 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[LOCAL] -----------PhysicalProject -------------PhysicalIntersect ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) ---------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalOlapScan[customer] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) ---------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalOlapScan[customer] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[store_sales] apply RFs: RF4 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) ---------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalOlapScan[customer] +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[LOCAL] +------------PhysicalProject +--------------PhysicalIntersect +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalOlapScan[customer] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalOlapScan[customer] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[store_sales] apply RFs: RF4 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query97.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query97.out index 404a9e7e9e92a1..4ebfd5abc0eb1c 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query97.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query97.out @@ -1,32 +1,33 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_97 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[LOCAL] -----------PhysicalProject -------------hashJoin[FULL_OUTER_JOIN colocated] hashCondition=((ssci.customer_sk = csci.customer_sk) and (ssci.item_sk = csci.item_sk)) otherCondition=() ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 ---------------------------PhysicalProject -----------------------------filter((date_dim.d_month_seq <= 1225) and (date_dim.d_month_seq >= 1214)) -------------------------------PhysicalOlapScan[date_dim] ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 ---------------------------PhysicalProject -----------------------------filter((date_dim.d_month_seq <= 1225) and (date_dim.d_month_seq >= 1214)) -------------------------------PhysicalOlapScan[date_dim] +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[LOCAL] +------------PhysicalProject +--------------hashJoin[FULL_OUTER_JOIN colocated] hashCondition=((ssci.customer_sk = csci.customer_sk) and (ssci.item_sk = csci.item_sk)) otherCondition=() +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_month_seq <= 1225) and (date_dim.d_month_seq >= 1214)) +--------------------------------PhysicalOlapScan[date_dim] +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_month_seq <= 1225) and (date_dim.d_month_seq >= 1214)) +--------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query23.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query23.out index 26e1f5089288a1..a65c74fa0b81dd 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query23.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query23.out @@ -46,35 +46,36 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------filter(d_year IN (2000, 2001, 2002, 2003)) ----------------------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink -------PhysicalTopN[GATHER_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecGather] -------------hashAgg[LOCAL] ---------------PhysicalUnion -----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF4 item_sk->[cs_item_sk] -------------------------PhysicalProject ---------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 RF5 -----------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------------PhysicalProject -----------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) -------------------------PhysicalOlapScan[date_dim] -----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF8 d_date_sk->[ws_sold_date_sk] ---------------------PhysicalProject -----------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF7 ws_item_sk->[item_sk] -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF7 -------------------------PhysicalProject ---------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() build RFs:RF6 c_customer_sk->[ws_bill_customer_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] apply RFs: RF6 RF8 -----------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) ---------------------PhysicalProject -----------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) -------------------------PhysicalOlapScan[date_dim] +------PhysicalLimit[GLOBAL] +--------PhysicalLimit[LOCAL] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecGather] +--------------hashAgg[LOCAL] +----------------PhysicalUnion +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] +----------------------PhysicalProject +------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF4 item_sk->[cs_item_sk] +--------------------------PhysicalProject +----------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 RF5 +------------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------PhysicalProject +------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) +--------------------------PhysicalOlapScan[date_dim] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF8 d_date_sk->[ws_sold_date_sk] +----------------------PhysicalProject +------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF7 ws_item_sk->[item_sk] +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF7 +--------------------------PhysicalProject +----------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() build RFs:RF6 c_customer_sk->[ws_bill_customer_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[web_sales] apply RFs: RF6 RF8 +------------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) +----------------------PhysicalProject +------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) +--------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query32.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query32.out index 15393efb582c73..7992f57d1c87b3 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query32.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query32.out @@ -1,25 +1,26 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_32 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[LOCAL] -----------PhysicalProject -------------filter((cast(cs_ext_discount_amt as DECIMALV3(38, 5)) > (1.3 * avg(cast(cs_ext_discount_amt as DECIMALV3(9, 4))) OVER(PARTITION BY i_item_sk)))) ---------------PhysicalWindow -----------------PhysicalQuickSort[LOCAL_SORT] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 -----------------------------PhysicalProject -------------------------------filter((item.i_manufact_id = 29)) ---------------------------------PhysicalOlapScan[item] -------------------------PhysicalProject ---------------------------filter((date_dim.d_date <= '1999-04-07') and (date_dim.d_date >= '1999-01-07')) -----------------------------PhysicalOlapScan[date_dim] +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[LOCAL] +------------PhysicalProject +--------------filter((cast(cs_ext_discount_amt as DECIMALV3(38, 5)) > (1.3 * avg(cast(cs_ext_discount_amt as DECIMALV3(9, 4))) OVER(PARTITION BY i_item_sk)))) +----------------PhysicalWindow +------------------PhysicalQuickSort[LOCAL_SORT] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 +------------------------------PhysicalProject +--------------------------------filter((item.i_manufact_id = 29)) +----------------------------------PhysicalOlapScan[item] +--------------------------PhysicalProject +----------------------------filter((date_dim.d_date <= '1999-04-07') and (date_dim.d_date >= '1999-01-07')) +------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query38.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query38.out index fbc5dd43162046..2bc9d9fecbca40 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query38.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query38.out @@ -1,49 +1,50 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_38 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[LOCAL] -----------PhysicalProject -------------PhysicalIntersect ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ws_bill_customer_sk] -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) ---------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalOlapScan[customer] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) ---------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalOlapScan[customer] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF5 c_customer_sk->[ss_customer_sk] -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[store_sales] apply RFs: RF4 RF5 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) ---------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalOlapScan[customer] +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[LOCAL] +------------PhysicalProject +--------------PhysicalIntersect +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ws_bill_customer_sk] +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalOlapScan[customer] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalOlapScan[customer] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF5 c_customer_sk->[ss_customer_sk] +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[store_sales] apply RFs: RF4 RF5 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query97.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query97.out index 404a9e7e9e92a1..4ebfd5abc0eb1c 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query97.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query97.out @@ -1,32 +1,33 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_97 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[LOCAL] -----------PhysicalProject -------------hashJoin[FULL_OUTER_JOIN colocated] hashCondition=((ssci.customer_sk = csci.customer_sk) and (ssci.item_sk = csci.item_sk)) otherCondition=() ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 ---------------------------PhysicalProject -----------------------------filter((date_dim.d_month_seq <= 1225) and (date_dim.d_month_seq >= 1214)) -------------------------------PhysicalOlapScan[date_dim] ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 ---------------------------PhysicalProject -----------------------------filter((date_dim.d_month_seq <= 1225) and (date_dim.d_month_seq >= 1214)) -------------------------------PhysicalOlapScan[date_dim] +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[LOCAL] +------------PhysicalProject +--------------hashJoin[FULL_OUTER_JOIN colocated] hashCondition=((ssci.customer_sk = csci.customer_sk) and (ssci.item_sk = csci.item_sk)) otherCondition=() +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_month_seq <= 1225) and (date_dim.d_month_seq >= 1214)) +--------------------------------PhysicalOlapScan[date_dim] +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_month_seq <= 1225) and (date_dim.d_month_seq >= 1214)) +--------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query23.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query23.out index 1343c4f4fb0214..6103f4eb74d6ab 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query23.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query23.out @@ -46,35 +46,36 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------filter(d_year IN (2000, 2001, 2002, 2003)) ----------------------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink -------PhysicalTopN[GATHER_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecGather] -------------hashAgg[LOCAL] ---------------PhysicalUnion -----------------PhysicalProject -------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF5 cs_item_sk->[item_sk] ---------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF5 ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() build RFs:RF4 c_customer_sk->[cs_bill_customer_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) -----------------PhysicalProject -------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF8 ws_item_sk->[item_sk] ---------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF8 ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((web_sales.ws_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() build RFs:RF7 c_customer_sk->[ws_bill_customer_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ws_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] apply RFs: RF6 RF7 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) +------PhysicalLimit[GLOBAL] +--------PhysicalLimit[LOCAL] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecGather] +--------------hashAgg[LOCAL] +----------------PhysicalUnion +------------------PhysicalProject +--------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF5 cs_item_sk->[item_sk] +----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF5 +----------------------PhysicalProject +------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() build RFs:RF4 c_customer_sk->[cs_bill_customer_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) +------------------PhysicalProject +--------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF8 ws_item_sk->[item_sk] +----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF8 +----------------------PhysicalProject +------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((web_sales.ws_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() build RFs:RF7 c_customer_sk->[ws_bill_customer_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ws_sold_date_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[web_sales] apply RFs: RF6 RF7 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query32.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query32.out index 15393efb582c73..7992f57d1c87b3 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query32.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query32.out @@ -1,25 +1,26 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_32 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[LOCAL] -----------PhysicalProject -------------filter((cast(cs_ext_discount_amt as DECIMALV3(38, 5)) > (1.3 * avg(cast(cs_ext_discount_amt as DECIMALV3(9, 4))) OVER(PARTITION BY i_item_sk)))) ---------------PhysicalWindow -----------------PhysicalQuickSort[LOCAL_SORT] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 -----------------------------PhysicalProject -------------------------------filter((item.i_manufact_id = 29)) ---------------------------------PhysicalOlapScan[item] -------------------------PhysicalProject ---------------------------filter((date_dim.d_date <= '1999-04-07') and (date_dim.d_date >= '1999-01-07')) -----------------------------PhysicalOlapScan[date_dim] +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[LOCAL] +------------PhysicalProject +--------------filter((cast(cs_ext_discount_amt as DECIMALV3(38, 5)) > (1.3 * avg(cast(cs_ext_discount_amt as DECIMALV3(9, 4))) OVER(PARTITION BY i_item_sk)))) +----------------PhysicalWindow +------------------PhysicalQuickSort[LOCAL_SORT] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 +------------------------------PhysicalProject +--------------------------------filter((item.i_manufact_id = 29)) +----------------------------------PhysicalOlapScan[item] +--------------------------PhysicalProject +----------------------------filter((date_dim.d_date <= '1999-04-07') and (date_dim.d_date >= '1999-01-07')) +------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query38.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query38.out index 802779104ed84f..26d52e9fae9c40 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query38.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query38.out @@ -1,49 +1,50 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_38 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[LOCAL] -----------PhysicalProject -------------PhysicalIntersect ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) ---------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalOlapScan[customer] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) ---------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalOlapScan[customer] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[store_sales] apply RFs: RF4 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) ---------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalOlapScan[customer] +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[LOCAL] +------------PhysicalProject +--------------PhysicalIntersect +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalOlapScan[customer] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalOlapScan[customer] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[store_sales] apply RFs: RF4 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query97.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query97.out index 404a9e7e9e92a1..4ebfd5abc0eb1c 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query97.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query97.out @@ -1,32 +1,33 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_97 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[LOCAL] -----------PhysicalProject -------------hashJoin[FULL_OUTER_JOIN colocated] hashCondition=((ssci.customer_sk = csci.customer_sk) and (ssci.item_sk = csci.item_sk)) otherCondition=() ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 ---------------------------PhysicalProject -----------------------------filter((date_dim.d_month_seq <= 1225) and (date_dim.d_month_seq >= 1214)) -------------------------------PhysicalOlapScan[date_dim] ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 ---------------------------PhysicalProject -----------------------------filter((date_dim.d_month_seq <= 1225) and (date_dim.d_month_seq >= 1214)) -------------------------------PhysicalOlapScan[date_dim] +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[LOCAL] +------------PhysicalProject +--------------hashJoin[FULL_OUTER_JOIN colocated] hashCondition=((ssci.customer_sk = csci.customer_sk) and (ssci.item_sk = csci.item_sk)) otherCondition=() +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_month_seq <= 1225) and (date_dim.d_month_seq >= 1214)) +--------------------------------PhysicalOlapScan[date_dim] +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_month_seq <= 1225) and (date_dim.d_month_seq >= 1214)) +--------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query23.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query23.out index 91ab82284bc473..0d8b21c19639e9 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query23.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query23.out @@ -46,35 +46,36 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------filter(d_year IN (2000, 2001, 2002, 2003)) ----------------------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink -------PhysicalTopN[GATHER_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecGather] -------------hashAgg[LOCAL] ---------------PhysicalUnion -----------------PhysicalProject -------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF5 cs_item_sk->[item_sk] ---------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF5 ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() build RFs:RF4 c_customer_sk->[cs_bill_customer_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) -----------------PhysicalProject -------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF8 ws_item_sk->[item_sk] ---------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF8 ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((web_sales.ws_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() build RFs:RF7 c_customer_sk->[ws_bill_customer_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ws_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] apply RFs: RF6 RF7 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) +------PhysicalLimit[GLOBAL] +--------PhysicalLimit[LOCAL] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecGather] +--------------hashAgg[LOCAL] +----------------PhysicalUnion +------------------PhysicalProject +--------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF5 cs_item_sk->[item_sk] +----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF5 +----------------------PhysicalProject +------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() build RFs:RF4 c_customer_sk->[cs_bill_customer_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) +------------------PhysicalProject +--------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF8 ws_item_sk->[item_sk] +----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF8 +----------------------PhysicalProject +------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((web_sales.ws_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() build RFs:RF7 c_customer_sk->[ws_bill_customer_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ws_sold_date_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[web_sales] apply RFs: RF6 RF7 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query32.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query32.out index 15393efb582c73..7992f57d1c87b3 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query32.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query32.out @@ -1,25 +1,26 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_32 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[LOCAL] -----------PhysicalProject -------------filter((cast(cs_ext_discount_amt as DECIMALV3(38, 5)) > (1.3 * avg(cast(cs_ext_discount_amt as DECIMALV3(9, 4))) OVER(PARTITION BY i_item_sk)))) ---------------PhysicalWindow -----------------PhysicalQuickSort[LOCAL_SORT] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 -----------------------------PhysicalProject -------------------------------filter((item.i_manufact_id = 29)) ---------------------------------PhysicalOlapScan[item] -------------------------PhysicalProject ---------------------------filter((date_dim.d_date <= '1999-04-07') and (date_dim.d_date >= '1999-01-07')) -----------------------------PhysicalOlapScan[date_dim] +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[LOCAL] +------------PhysicalProject +--------------filter((cast(cs_ext_discount_amt as DECIMALV3(38, 5)) > (1.3 * avg(cast(cs_ext_discount_amt as DECIMALV3(9, 4))) OVER(PARTITION BY i_item_sk)))) +----------------PhysicalWindow +------------------PhysicalQuickSort[LOCAL_SORT] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 +------------------------------PhysicalProject +--------------------------------filter((item.i_manufact_id = 29)) +----------------------------------PhysicalOlapScan[item] +--------------------------PhysicalProject +----------------------------filter((date_dim.d_date <= '1999-04-07') and (date_dim.d_date >= '1999-01-07')) +------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query38.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query38.out index 9a6c130acd1eaf..ef2051eb2d000c 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query38.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query38.out @@ -1,49 +1,50 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_38 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[LOCAL] -----------PhysicalProject -------------PhysicalIntersect ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ws_bill_customer_sk] -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) ---------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalOlapScan[customer] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) ---------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalOlapScan[customer] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF5 c_customer_sk->[ss_customer_sk] -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[store_sales] apply RFs: RF4 RF5 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) ---------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalOlapScan[customer] +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[LOCAL] +------------PhysicalProject +--------------PhysicalIntersect +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ws_bill_customer_sk] +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalOlapScan[customer] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalOlapScan[customer] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF5 c_customer_sk->[ss_customer_sk] +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[store_sales] apply RFs: RF4 RF5 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query97.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query97.out index 404a9e7e9e92a1..4ebfd5abc0eb1c 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query97.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query97.out @@ -1,32 +1,33 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_97 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[LOCAL] -----------PhysicalProject -------------hashJoin[FULL_OUTER_JOIN colocated] hashCondition=((ssci.customer_sk = csci.customer_sk) and (ssci.item_sk = csci.item_sk)) otherCondition=() ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 ---------------------------PhysicalProject -----------------------------filter((date_dim.d_month_seq <= 1225) and (date_dim.d_month_seq >= 1214)) -------------------------------PhysicalOlapScan[date_dim] ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 ---------------------------PhysicalProject -----------------------------filter((date_dim.d_month_seq <= 1225) and (date_dim.d_month_seq >= 1214)) -------------------------------PhysicalOlapScan[date_dim] +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[LOCAL] +------------PhysicalProject +--------------hashJoin[FULL_OUTER_JOIN colocated] hashCondition=((ssci.customer_sk = csci.customer_sk) and (ssci.item_sk = csci.item_sk)) otherCondition=() +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_month_seq <= 1225) and (date_dim.d_month_seq >= 1214)) +--------------------------------PhysicalOlapScan[date_dim] +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_month_seq <= 1225) and (date_dim.d_month_seq >= 1214)) +--------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query23.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query23.out index 5e2eafb60ffe8d..45d7a47eb7e46a 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query23.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query23.out @@ -46,35 +46,36 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------filter(d_year IN (2000, 2001, 2002, 2003)) ----------------------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink -------PhysicalTopN[GATHER_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecGather] -------------hashAgg[LOCAL] ---------------PhysicalUnion -----------------PhysicalProject -------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF5 cs_item_sk->[item_sk] ---------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF5 ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() build RFs:RF4 c_customer_sk->[cs_bill_customer_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_moy = 7) and (date_dim.d_year = 2000)) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) -----------------PhysicalProject -------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF8 ws_item_sk->[item_sk] ---------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF8 ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((web_sales.ws_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() build RFs:RF7 c_customer_sk->[ws_bill_customer_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ws_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] apply RFs: RF6 RF7 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_moy = 7) and (date_dim.d_year = 2000)) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) +------PhysicalLimit[GLOBAL] +--------PhysicalLimit[LOCAL] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecGather] +--------------hashAgg[LOCAL] +----------------PhysicalUnion +------------------PhysicalProject +--------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF5 cs_item_sk->[item_sk] +----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF5 +----------------------PhysicalProject +------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() build RFs:RF4 c_customer_sk->[cs_bill_customer_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_moy = 7) and (date_dim.d_year = 2000)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) +------------------PhysicalProject +--------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF8 ws_item_sk->[item_sk] +----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF8 +----------------------PhysicalProject +------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((web_sales.ws_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() build RFs:RF7 c_customer_sk->[ws_bill_customer_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ws_sold_date_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[web_sales] apply RFs: RF6 RF7 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_moy = 7) and (date_dim.d_year = 2000)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query32.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query32.out index 0389bfeea82673..c413940debc6e7 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query32.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query32.out @@ -1,25 +1,26 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_32 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[LOCAL] -----------PhysicalProject -------------filter((cast(cs_ext_discount_amt as DECIMALV3(38, 5)) > (1.3 * avg(cast(cs_ext_discount_amt as DECIMALV3(9, 4))) OVER(PARTITION BY i_item_sk)))) ---------------PhysicalWindow -----------------PhysicalQuickSort[LOCAL_SORT] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 -----------------------------PhysicalProject -------------------------------filter((item.i_manufact_id = 722)) ---------------------------------PhysicalOlapScan[item] -------------------------PhysicalProject ---------------------------filter((date_dim.d_date <= '2001-06-07') and (date_dim.d_date >= '2001-03-09')) -----------------------------PhysicalOlapScan[date_dim] +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[LOCAL] +------------PhysicalProject +--------------filter((cast(cs_ext_discount_amt as DECIMALV3(38, 5)) > (1.3 * avg(cast(cs_ext_discount_amt as DECIMALV3(9, 4))) OVER(PARTITION BY i_item_sk)))) +----------------PhysicalWindow +------------------PhysicalQuickSort[LOCAL_SORT] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 +------------------------------PhysicalProject +--------------------------------filter((item.i_manufact_id = 722)) +----------------------------------PhysicalOlapScan[item] +--------------------------PhysicalProject +----------------------------filter((date_dim.d_date <= '2001-06-07') and (date_dim.d_date >= '2001-03-09')) +------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query38.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query38.out index 5ae805bdffe325..dc794c95f97b2f 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query38.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query38.out @@ -1,49 +1,50 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_38 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[LOCAL] -----------PhysicalProject -------------PhysicalIntersect ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ws_bill_customer_sk] -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq <= 1200) and (date_dim.d_month_seq >= 1189)) ---------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalOlapScan[customer] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq <= 1200) and (date_dim.d_month_seq >= 1189)) ---------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalOlapScan[customer] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF5 c_customer_sk->[ss_customer_sk] -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[store_sales] apply RFs: RF4 RF5 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq <= 1200) and (date_dim.d_month_seq >= 1189)) ---------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalOlapScan[customer] +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[LOCAL] +------------PhysicalProject +--------------PhysicalIntersect +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ws_bill_customer_sk] +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_month_seq <= 1200) and (date_dim.d_month_seq >= 1189)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalOlapScan[customer] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_month_seq <= 1200) and (date_dim.d_month_seq >= 1189)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalOlapScan[customer] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF5 c_customer_sk->[ss_customer_sk] +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[store_sales] apply RFs: RF4 RF5 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_month_seq <= 1200) and (date_dim.d_month_seq >= 1189)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query97.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query97.out index ca6f63d5e6bdf0..d3a845763241f7 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query97.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query97.out @@ -1,34 +1,35 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_97 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[LOCAL] -----------PhysicalProject -------------hashJoin[FULL_OUTER_JOIN colocated] hashCondition=((ssci.customer_sk = csci.customer_sk) and (ssci.item_sk = csci.item_sk)) otherCondition=() ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] ---------------------------PhysicalProject -----------------------------filter(( not ss_sold_date_sk IS NULL)) -------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 ---------------------------PhysicalProject -----------------------------filter((date_dim.d_month_seq <= 1210) and (date_dim.d_month_seq >= 1199)) -------------------------------PhysicalOlapScan[date_dim] ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] ---------------------------PhysicalProject -----------------------------filter(( not cs_sold_date_sk IS NULL)) -------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 ---------------------------PhysicalProject -----------------------------filter((date_dim.d_month_seq <= 1210) and (date_dim.d_month_seq >= 1199)) -------------------------------PhysicalOlapScan[date_dim] +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[LOCAL] +------------PhysicalProject +--------------hashJoin[FULL_OUTER_JOIN colocated] hashCondition=((ssci.customer_sk = csci.customer_sk) and (ssci.item_sk = csci.item_sk)) otherCondition=() +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +----------------------------PhysicalProject +------------------------------filter(( not ss_sold_date_sk IS NULL)) +--------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_month_seq <= 1210) and (date_dim.d_month_seq >= 1199)) +--------------------------------PhysicalOlapScan[date_dim] +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] +----------------------------PhysicalProject +------------------------------filter(( not cs_sold_date_sk IS NULL)) +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_month_seq <= 1210) and (date_dim.d_month_seq >= 1199)) +--------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/suites/nereids_tpch_p0/tpch/push_topn_to_agg.groovy b/regression-test/suites/nereids_tpch_p0/tpch/push_topn_to_agg.groovy index 36460f06e32d56..631656a6b1921b 100644 --- a/regression-test/suites/nereids_tpch_p0/tpch/push_topn_to_agg.groovy +++ b/regression-test/suites/nereids_tpch_p0/tpch/push_topn_to_agg.groovy @@ -68,10 +68,10 @@ suite("push_topn_to_agg") { contains("sortByGroupKey:true") } - // group key is not output of limit, deny opt + // use group key as sort key to enable topn-push opt explain { sql "select sum(o_shippriority) from orders group by o_clerk limit 14; " - contains("sortByGroupKey:false") + contains("sortByGroupKey:true") } // group key is part of output of limit, apply opt From d3276d99cefeb2f2310dcd613ee563547d40ab22 Mon Sep 17 00:00:00 2001 From: LiBinfeng Date: Wed, 27 Nov 2024 10:51:44 +0800 Subject: [PATCH 030/399] [Improvement](Nereids) Support create routine load command (#43930) Issue Number: close #42805 --- .../org/apache/doris/nereids/DorisParser.g4 | 25 +- .../doris/analysis/CreateRoutineLoadStmt.java | 46 ++ .../org/apache/doris/analysis/Separator.java | 5 + .../nereids/parser/LogicalPlanBuilder.java | 122 +++++ .../doris/nereids/trees/plans/PlanType.java | 3 +- .../commands/info/CreateRoutineLoadInfo.java | 517 ++++++++++++++++++ .../plans/commands/info/LabelNameInfo.java | 108 ++++ .../load/CreateRoutineLoadCommand.java | 90 +++ .../plans/commands/load/LoadColumnClause.java | 35 ++ .../plans/commands/load/LoadColumnDesc.java | 45 ++ .../commands/load/LoadDeleteOnClause.java | 35 ++ .../commands/load/LoadPartitionNames.java | 70 +++ .../plans/commands/load/LoadProperty.java | 36 ++ .../plans/commands/load/LoadSeparator.java | 33 ++ .../commands/load/LoadSequenceClause.java | 33 ++ .../plans/commands/load/LoadWhereClause.java | 42 ++ .../trees/plans/visitor/CommandVisitor.java | 5 + .../test_routine_load_condition.groovy | 4 +- .../routine_load/test_routine_load.groovy | 6 +- 19 files changed, 1240 insertions(+), 20 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateRoutineLoadInfo.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/LabelNameInfo.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/load/CreateRoutineLoadCommand.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/load/LoadColumnClause.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/load/LoadColumnDesc.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/load/LoadDeleteOnClause.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/load/LoadPartitionNames.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/load/LoadProperty.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/load/LoadSeparator.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/load/LoadSequenceClause.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/load/LoadWhereClause.java diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index 2d68e01b78761a..d7ed0598d19ddb 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -58,13 +58,11 @@ statementBase | supportedUnsetStatement #supportedUnsetStatementAlias | supportedRefreshStatement #supportedRefreshStatementAlias | supportedShowStatement #supportedShowStatementAlias + | supportedLoadStatement #supportedLoadStatementAlias | supportedRecoverStatement #supportedRecoverStatementAlias - | supportedLoadStatement #supportedLoadfStatementAlias | unsupportedStatement #unsupported ; - - unsupportedStatement : unsupportedUseStatement | unsupportedDmlStatement @@ -242,11 +240,12 @@ supportedShowStatement | SHOW TABLE tableId=INTEGER_VALUE #showTableId | SHOW WHITELIST #showWhitelist | SHOW TABLETS BELONG - tabletIds+=INTEGER_VALUE (COMMA tabletIds+=INTEGER_VALUE)* #showTabletsBelong + tabletIds+=INTEGER_VALUE (COMMA tabletIds+=INTEGER_VALUE)* #showTabletsBelong ; supportedLoadStatement : SYNC #sync + | createRoutineLoad #createRoutineLoadAlias ; unsupportedOtherStatement @@ -358,6 +357,14 @@ unsupportedShowStatement | SHOW WARM UP JOB wildWhere? #showWarmUpJob ; +createRoutineLoad + : CREATE ROUTINE LOAD label=multipartIdentifier (ON table=identifier)? + (WITH (APPEND | DELETE | MERGE))? + (loadProperty (COMMA loadProperty)*)? propertyClause? FROM type=identifier + LEFT_PAREN customProperties=propertyItemList RIGHT_PAREN + commentSpec? + ; + unsupportedLoadStatement : LOAD mysqlDataDesc (PROPERTIES LEFT_PAREN properties=propertyItemList RIGHT_PAREN)? @@ -369,11 +376,6 @@ unsupportedLoadStatement | STOP SYNC JOB name=multipartIdentifier #stopDataSyncJob | RESUME SYNC JOB name=multipartIdentifier #resumeDataSyncJob | PAUSE SYNC JOB name=multipartIdentifier #pauseDataSyncJob - | CREATE ROUTINE LOAD label=multipartIdentifier (ON table=identifier)? - (WITH (APPEND | DELETE | MERGE))? - (loadProperty (COMMA loadProperty)*)? propertyClause? FROM type=identifier - LEFT_PAREN customProperties=propertyItemList RIGHT_PAREN - commentSpec? #createRoutineLoadJob | PAUSE ROUTINE LOAD FOR label=multipartIdentifier #pauseRoutineLoad | PAUSE ALL ROUTINE LOAD #pauseAllRoutineLoad | RESUME ROUTINE LOAD FOR label=multipartIdentifier #resumeRoutineLoad @@ -383,11 +385,6 @@ unsupportedLoadStatement | SHOW ROUTINE LOAD TASK ((FROM | IN) database=identifier)? wildWhere? #showRoutineLoadTask | SHOW ALL? CREATE ROUTINE LOAD FOR label=multipartIdentifier #showCreateRoutineLoad | SHOW CREATE LOAD FOR label=multipartIdentifier #showCreateLoad - | importSequenceStatement #importSequenceStatementAlias - | importPrecedingFilterStatement #importPrecedingFilterStatementAlias - | importWhereStatement #importWhereStatementAlias - | importDeleteOnStatement #importDeleteOnStatementAlias - | importColumnsStatement #importColumnsStatementAlias ; loadProperty diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateRoutineLoadStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateRoutineLoadStmt.java index a16259dcdb98d2..27379ccd28d098 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateRoutineLoadStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateRoutineLoadStmt.java @@ -34,6 +34,7 @@ import org.apache.doris.load.routineload.RoutineLoadDataSourcePropertyFactory; import org.apache.doris.load.routineload.RoutineLoadJob; import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.OriginStatement; import com.google.common.base.Strings; import com.google.common.collect.ImmutableSet; @@ -229,6 +230,51 @@ public CreateRoutineLoadStmt(LabelName labelName, String tableName, List loadPropertyList, OriginStatement origStmt, UserIdentity userIdentity, + Map jobProperties, String typeName, RoutineLoadDesc routineLoadDesc, + int desireTaskConcurrentNum, long maxErrorNum, double maxFilterRatio, long maxBatchIntervalS, + long maxBatchRows, long maxBatchSizeBytes, long execMemLimit, int sendBatchParallelism, String timezone, + String format, String jsonPaths, String jsonRoot, byte enclose, byte escape, long workloadGroupId, + boolean loadToSingleTablet, boolean strictMode, boolean isPartialUpdate, boolean stripOuterArray, + boolean numAsString, boolean fuzzyParse, AbstractDataSourceProperties dataSourceProperties) { + this.labelName = labelName; + this.dbName = dbName; + this.name = name; + this.tableName = tableName; + this.loadPropertyList = loadPropertyList; + this.setOrigStmt(origStmt); + this.setUserInfo(userIdentity); + this.jobProperties = jobProperties; + this.typeName = typeName; + this.routineLoadDesc = routineLoadDesc; + this.desiredConcurrentNum = desireTaskConcurrentNum; + this.maxErrorNum = maxErrorNum; + this.maxFilterRatio = maxFilterRatio; + this.maxBatchIntervalS = maxBatchIntervalS; + this.maxBatchRows = maxBatchRows; + this.maxBatchSizeBytes = maxBatchSizeBytes; + this.execMemLimit = execMemLimit; + this.sendBatchParallelism = sendBatchParallelism; + this.timezone = timezone; + this.format = format; + this.jsonPaths = jsonPaths; + this.jsonRoot = jsonRoot; + this.enclose = enclose; + this.escape = escape; + this.workloadGroupId = workloadGroupId; + this.loadToSingleTablet = loadToSingleTablet; + this.strictMode = strictMode; + this.isPartialUpdate = isPartialUpdate; + this.stripOuterArray = stripOuterArray; + this.numAsString = numAsString; + this.fuzzyParse = fuzzyParse; + this.dataSourceProperties = dataSourceProperties; + } + public String getName() { return name; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/Separator.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/Separator.java index 3a5731944a6b95..86027ea9ceac5b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/Separator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/Separator.java @@ -34,6 +34,11 @@ public Separator(String separator) { this.separator = null; } + public Separator(String separator, String oriSeparator) { + this.oriSeparator = oriSeparator; + this.separator = separator; + } + public String getOriSeparator() { return oriSeparator; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index ae0921a1732419..6e86b4073f5792 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -87,6 +87,7 @@ import org.apache.doris.nereids.DorisParser.ConstantContext; import org.apache.doris.nereids.DorisParser.CreateMTMVContext; import org.apache.doris.nereids.DorisParser.CreateProcedureContext; +import org.apache.doris.nereids.DorisParser.CreateRoutineLoadContext; import org.apache.doris.nereids.DorisParser.CreateRowPolicyContext; import org.apache.doris.nereids.DorisParser.CreateTableContext; import org.apache.doris.nereids.DorisParser.CreateTableLikeContext; @@ -126,6 +127,12 @@ import org.apache.doris.nereids.DorisParser.IdentifierContext; import org.apache.doris.nereids.DorisParser.IdentifierListContext; import org.apache.doris.nereids.DorisParser.IdentifierSeqContext; +import org.apache.doris.nereids.DorisParser.ImportColumnsContext; +import org.apache.doris.nereids.DorisParser.ImportDeleteOnContext; +import org.apache.doris.nereids.DorisParser.ImportPartitionsContext; +import org.apache.doris.nereids.DorisParser.ImportPrecedingFilterContext; +import org.apache.doris.nereids.DorisParser.ImportSequenceContext; +import org.apache.doris.nereids.DorisParser.ImportWhereContext; import org.apache.doris.nereids.DorisParser.InPartitionDefContext; import org.apache.doris.nereids.DorisParser.IndexDefContext; import org.apache.doris.nereids.DorisParser.IndexDefsContext; @@ -141,6 +148,7 @@ import org.apache.doris.nereids.DorisParser.LateralViewContext; import org.apache.doris.nereids.DorisParser.LessThanPartitionDefContext; import org.apache.doris.nereids.DorisParser.LimitClauseContext; +import org.apache.doris.nereids.DorisParser.LoadPropertyContext; import org.apache.doris.nereids.DorisParser.LogicalBinaryContext; import org.apache.doris.nereids.DorisParser.LogicalNotContext; import org.apache.doris.nereids.DorisParser.MapLiteralContext; @@ -193,6 +201,7 @@ import org.apache.doris.nereids.DorisParser.SelectClauseContext; import org.apache.doris.nereids.DorisParser.SelectColumnClauseContext; import org.apache.doris.nereids.DorisParser.SelectHintContext; +import org.apache.doris.nereids.DorisParser.SeparatorContext; import org.apache.doris.nereids.DorisParser.SetCharsetContext; import org.apache.doris.nereids.DorisParser.SetCollateContext; import org.apache.doris.nereids.DorisParser.SetDefaultStorageVaultContext; @@ -532,6 +541,7 @@ import org.apache.doris.nereids.trees.plans.commands.info.ColumnDefinition; import org.apache.doris.nereids.trees.plans.commands.info.CreateJobInfo; import org.apache.doris.nereids.trees.plans.commands.info.CreateMTMVInfo; +import org.apache.doris.nereids.trees.plans.commands.info.CreateRoutineLoadInfo; import org.apache.doris.nereids.trees.plans.commands.info.CreateTableInfo; import org.apache.doris.nereids.trees.plans.commands.info.CreateTableLikeInfo; import org.apache.doris.nereids.trees.plans.commands.info.CreateViewInfo; @@ -544,6 +554,7 @@ import org.apache.doris.nereids.trees.plans.commands.info.GeneratedColumnDesc; import org.apache.doris.nereids.trees.plans.commands.info.InPartition; import org.apache.doris.nereids.trees.plans.commands.info.IndexDefinition; +import org.apache.doris.nereids.trees.plans.commands.info.LabelNameInfo; import org.apache.doris.nereids.trees.plans.commands.info.LessThanPartition; import org.apache.doris.nereids.trees.plans.commands.info.MTMVPartitionDefinition; import org.apache.doris.nereids.trees.plans.commands.info.PartitionDefinition; @@ -568,6 +579,15 @@ import org.apache.doris.nereids.trees.plans.commands.insert.BatchInsertIntoTableCommand; import org.apache.doris.nereids.trees.plans.commands.insert.InsertIntoTableCommand; import org.apache.doris.nereids.trees.plans.commands.insert.InsertOverwriteTableCommand; +import org.apache.doris.nereids.trees.plans.commands.load.CreateRoutineLoadCommand; +import org.apache.doris.nereids.trees.plans.commands.load.LoadColumnClause; +import org.apache.doris.nereids.trees.plans.commands.load.LoadColumnDesc; +import org.apache.doris.nereids.trees.plans.commands.load.LoadDeleteOnClause; +import org.apache.doris.nereids.trees.plans.commands.load.LoadPartitionNames; +import org.apache.doris.nereids.trees.plans.commands.load.LoadProperty; +import org.apache.doris.nereids.trees.plans.commands.load.LoadSeparator; +import org.apache.doris.nereids.trees.plans.commands.load.LoadSequenceClause; +import org.apache.doris.nereids.trees.plans.commands.load.LoadWhereClause; import org.apache.doris.nereids.trees.plans.commands.refresh.RefreshCatalogCommand; import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; import org.apache.doris.nereids.trees.plans.logical.LogicalCTE; @@ -1451,6 +1471,108 @@ public LogicalSubQueryAlias visitAliasQuery(AliasQueryContext ctx) { }); } + /** + * process LoadProperty in routine load + */ + public LoadProperty visitLoadProperty(LoadPropertyContext ctx) { + LoadProperty loadProperty = null; + if (ctx instanceof SeparatorContext) { + String separator = stripQuotes(((SeparatorContext) ctx).STRING_LITERAL().getText()); + loadProperty = new LoadSeparator(separator); + } else if (ctx instanceof ImportColumnsContext) { + List descList = new ArrayList<>(); + for (DorisParser.ImportColumnDescContext loadColumnDescCtx : ((ImportColumnsContext) ctx) + .importColumnsStatement().importColumnDesc()) { + LoadColumnDesc desc; + if (loadColumnDescCtx.booleanExpression() != null) { + desc = new LoadColumnDesc(loadColumnDescCtx.name.getText(), + getExpression(loadColumnDescCtx.booleanExpression())); + } else { + desc = new LoadColumnDesc(loadColumnDescCtx.name.getText()); + } + descList.add(desc); + } + loadProperty = new LoadColumnClause(descList); + } else if (ctx instanceof ImportDeleteOnContext) { + loadProperty = new LoadDeleteOnClause(getExpression(((ImportDeleteOnContext) ctx) + .importDeleteOnStatement().booleanExpression())); + } else if (ctx instanceof ImportPartitionsContext) { + Pair> partitionSpec = visitPartitionSpec( + ((ImportPartitionsContext) ctx).partitionSpec()); + loadProperty = new LoadPartitionNames(partitionSpec.first, partitionSpec.second); + } else if (ctx instanceof ImportPrecedingFilterContext) { + loadProperty = new LoadWhereClause(getExpression(((ImportPrecedingFilterContext) ctx) + .importPrecedingFilterStatement().booleanExpression()), true); + } else if (ctx instanceof ImportSequenceContext) { + loadProperty = new LoadSequenceClause(((ImportSequenceContext) ctx) + .importSequenceStatement().identifier().getText()); + } else if (ctx instanceof ImportWhereContext) { + loadProperty = new LoadWhereClause(getExpression(((ImportWhereContext) ctx) + .importWhereStatement().booleanExpression()), false); + } + return loadProperty; + } + + @Override + public LogicalPlan visitCreateRoutineLoad(CreateRoutineLoadContext ctx) { + List labelParts = visitMultipartIdentifier(ctx.label); + String labelName = null; + String labelDbName = null; + if (ConnectContext.get().getDatabase().isEmpty() && labelParts.size() == 1) { + throw new AnalysisException("Current database is not set."); + } else if (labelParts.size() == 1) { + labelName = labelParts.get(0); + } else if (labelParts.size() == 2) { + labelDbName = labelParts.get(0); + labelName = labelParts.get(1); + } else if (labelParts.size() == 3) { + labelDbName = labelParts.get(1); + labelName = labelParts.get(2); + } else { + throw new AnalysisException("labelParts in load should be [ctl.][db.]label"); + } + LabelNameInfo jobLabelInfo = new LabelNameInfo(labelDbName, labelName); + String tableName = null; + if (ctx.table != null) { + tableName = ctx.table.getText(); + } + Map properties = ctx.propertyClause() != null + // NOTICE: we should not generate immutable map here, because it will be modified when analyzing. + ? Maps.newHashMap(visitPropertyClause(ctx.propertyClause())) + : Maps.newHashMap(); + String type = ctx.type.getText(); + Map customProperties = ctx.customProperties != null + // NOTICE: we should not generate immutable map here, because it will be modified when analyzing. + ? Maps.newHashMap(visitPropertyItemList(ctx.customProperties)) + : Maps.newHashMap(); + LoadTask.MergeType mergeType = LoadTask.MergeType.APPEND; + if (ctx.WITH() != null) { + if (ctx.DELETE() != null) { + mergeType = LoadTask.MergeType.DELETE; + } else if (ctx.MERGE() != null) { + mergeType = LoadTask.MergeType.MERGE; + } + } + String comment = visitCommentSpec(ctx.commentSpec()); + Map loadPropertyMap = new HashMap<>(); + for (DorisParser.LoadPropertyContext oneLoadPropertyCOntext : ctx.loadProperty()) { + LoadProperty loadProperty = visitLoadProperty(oneLoadPropertyCOntext); + if (loadProperty == null) { + throw new AnalysisException("invalid clause of routine load"); + } + if (loadPropertyMap.get(loadProperty.getClass().getName()) != null) { + throw new AnalysisException("repeat setting of clause load property: " + + loadProperty.getClass().getName()); + } else { + loadPropertyMap.put(loadProperty.getClass().getName(), loadProperty); + } + } + CreateRoutineLoadInfo createRoutineLoadInfo = new CreateRoutineLoadInfo(jobLabelInfo, tableName, + loadPropertyMap, properties, type, customProperties, mergeType, comment); + return new CreateRoutineLoadCommand(createRoutineLoadInfo); + + } + @Override public Command visitCreateRowPolicy(CreateRowPolicyContext ctx) { FilterType filterType = FilterType.of(ctx.type.getText()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index 79a4ff31c1fc20..57eb2bbb6b42c2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -222,5 +222,6 @@ public enum PlanType { RECOVER_DATABASE_COMMAND, RECOVER_TABLE_COMMAND, RECOVER_PARTITION_COMMAND, - REPLAY_COMMAND + REPLAY_COMMAND, + CREATE_ROUTINE_LOAD_COMMAND } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateRoutineLoadInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateRoutineLoadInfo.java new file mode 100644 index 00000000000000..3e0fd0d34fb091 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateRoutineLoadInfo.java @@ -0,0 +1,517 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.commands.info; + +import org.apache.doris.analysis.CreateRoutineLoadStmt; +import org.apache.doris.analysis.Expr; +import org.apache.doris.analysis.ImportColumnDesc; +import org.apache.doris.analysis.ImportColumnsStmt; +import org.apache.doris.analysis.ImportDeleteOnStmt; +import org.apache.doris.analysis.ImportSequenceStmt; +import org.apache.doris.analysis.ImportWhereStmt; +import org.apache.doris.analysis.LoadStmt; +import org.apache.doris.analysis.PartitionNames; +import org.apache.doris.analysis.Separator; +import org.apache.doris.analysis.SlotRef; +import org.apache.doris.analysis.TupleDescriptor; +import org.apache.doris.catalog.Database; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.KeysType; +import org.apache.doris.catalog.OlapTable; +import org.apache.doris.catalog.Table; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.Config; +import org.apache.doris.common.FeNameFormat; +import org.apache.doris.common.UserException; +import org.apache.doris.common.util.TimeUtils; +import org.apache.doris.common.util.Util; +import org.apache.doris.load.RoutineLoadDesc; +import org.apache.doris.load.loadv2.LoadTask; +import org.apache.doris.load.routineload.AbstractDataSourceProperties; +import org.apache.doris.load.routineload.RoutineLoadDataSourcePropertyFactory; +import org.apache.doris.load.routineload.RoutineLoadJob; +import org.apache.doris.nereids.CascadesContext; +import org.apache.doris.nereids.analyzer.Scope; +import org.apache.doris.nereids.analyzer.UnboundRelation; +import org.apache.doris.nereids.glue.translator.ExpressionTranslator; +import org.apache.doris.nereids.glue.translator.PlanTranslatorContext; +import org.apache.doris.nereids.jobs.executor.Rewriter; +import org.apache.doris.nereids.properties.PhysicalProperties; +import org.apache.doris.nereids.rules.analysis.BindRelation; +import org.apache.doris.nereids.rules.analysis.ExpressionAnalyzer; +import org.apache.doris.nereids.rules.expression.ExpressionRewriteContext; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.algebra.OlapScan; +import org.apache.doris.nereids.trees.plans.commands.load.LoadColumnClause; +import org.apache.doris.nereids.trees.plans.commands.load.LoadColumnDesc; +import org.apache.doris.nereids.trees.plans.commands.load.LoadDeleteOnClause; +import org.apache.doris.nereids.trees.plans.commands.load.LoadPartitionNames; +import org.apache.doris.nereids.trees.plans.commands.load.LoadProperty; +import org.apache.doris.nereids.trees.plans.commands.load.LoadSeparator; +import org.apache.doris.nereids.trees.plans.commands.load.LoadSequenceClause; +import org.apache.doris.nereids.trees.plans.commands.load.LoadWhereClause; +import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; +import org.apache.doris.nereids.util.Utils; +import org.apache.doris.qe.ConnectContext; + +import com.google.common.base.Strings; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import org.apache.commons.lang3.StringUtils; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.function.Predicate; + +/** + * info in creating routine load. + */ +public class CreateRoutineLoadInfo { + // routine load properties + public static final String DESIRED_CONCURRENT_NUMBER_PROPERTY = "desired_concurrent_number"; + public static final String CURRENT_CONCURRENT_NUMBER_PROPERTY = "current_concurrent_number"; + // max error number in ten thousand records + public static final String MAX_ERROR_NUMBER_PROPERTY = "max_error_number"; + public static final String MAX_FILTER_RATIO_PROPERTY = "max_filter_ratio"; + // the following 3 properties limit the time and batch size of a single routine load task + public static final String MAX_BATCH_INTERVAL_SEC_PROPERTY = "max_batch_interval"; + public static final String MAX_BATCH_ROWS_PROPERTY = "max_batch_rows"; + public static final String MAX_BATCH_SIZE_PROPERTY = "max_batch_size"; + public static final String EXEC_MEM_LIMIT_PROPERTY = "exec_mem_limit"; + + public static final String FORMAT = "format"; // the value is csv or json, default is csv + public static final String STRIP_OUTER_ARRAY = "strip_outer_array"; + public static final String JSONPATHS = "jsonpaths"; + public static final String JSONROOT = "json_root"; + public static final String NUM_AS_STRING = "num_as_string"; + public static final String FUZZY_PARSE = "fuzzy_parse"; + public static final String PARTIAL_COLUMNS = "partial_columns"; + public static final String WORKLOAD_GROUP = "workload_group"; + public static final String ENDPOINT_REGEX = "[-A-Za-z0-9+&@#/%?=~_|!:,.;]+[-A-Za-z0-9+&@#/%=~_|]"; + public static final String SEND_BATCH_PARALLELISM = "send_batch_parallelism"; + public static final String LOAD_TO_SINGLE_TABLET = "load_to_single_tablet"; + public static final java.util.function.Predicate DESIRED_CONCURRENT_NUMBER_PRED = (v) -> v > 0L; + public static final java.util.function.Predicate MAX_ERROR_NUMBER_PRED = (v) -> v >= 0L; + public static final java.util.function.Predicate MAX_FILTER_RATIO_PRED = (v) -> v >= 0 && v <= 1; + public static final java.util.function.Predicate MAX_BATCH_INTERVAL_PRED = (v) -> v >= 1; + public static final java.util.function.Predicate MAX_BATCH_ROWS_PRED = (v) -> v >= 200000; + public static final java.util.function.Predicate MAX_BATCH_SIZE_PRED = (v) -> v >= 100 * 1024 * 1024 + && v <= (long) (1024 * 1024 * 1024) * 10; + public static final java.util.function.Predicate EXEC_MEM_LIMIT_PRED = (v) -> v >= 0L; + public static final Predicate SEND_BATCH_PARALLELISM_PRED = (v) -> v > 0L; + + private static final String NAME_TYPE = "ROUTINE LOAD NAME"; + + private static final ImmutableSet PROPERTIES_SET = new ImmutableSet.Builder() + .add(DESIRED_CONCURRENT_NUMBER_PROPERTY) + .add(MAX_ERROR_NUMBER_PROPERTY) + .add(MAX_FILTER_RATIO_PROPERTY) + .add(MAX_BATCH_INTERVAL_SEC_PROPERTY) + .add(MAX_BATCH_ROWS_PROPERTY) + .add(MAX_BATCH_SIZE_PROPERTY) + .add(FORMAT) + .add(JSONPATHS) + .add(STRIP_OUTER_ARRAY) + .add(NUM_AS_STRING) + .add(FUZZY_PARSE) + .add(JSONROOT) + .add(LoadStmt.STRICT_MODE) + .add(LoadStmt.TIMEZONE) + .add(EXEC_MEM_LIMIT_PROPERTY) + .add(SEND_BATCH_PARALLELISM) + .add(LOAD_TO_SINGLE_TABLET) + .add(PARTIAL_COLUMNS) + .add(WORKLOAD_GROUP) + .add(LoadStmt.KEY_ENCLOSE) + .add(LoadStmt.KEY_ESCAPE) + .build(); + + private final LabelNameInfo labelNameInfo; + private String tableName; + private final Map loadPropertyMap; + private final Map jobProperties; + private final String typeName; + + // the following variables will be initialized after analyze + // -1 as unset, the default value will set in RoutineLoadJob + private String name; + private String dbName; + private RoutineLoadDesc routineLoadDesc; + private int desiredConcurrentNum = 1; + private long maxErrorNum = -1; + private double maxFilterRatio = -1; + private long maxBatchIntervalS = -1; + private long maxBatchRows = -1; + private long maxBatchSizeBytes = -1; + private boolean strictMode = true; + private long execMemLimit = 2 * 1024 * 1024 * 1024L; + private String timezone = TimeUtils.DEFAULT_TIME_ZONE; + private int sendBatchParallelism = 1; + private boolean loadToSingleTablet = false; + /** + * RoutineLoad support json data. + * Require Params: + * 1) dataFormat = "json" + * 2) jsonPaths = "$.XXX.xxx" + */ + private String format = ""; //default is csv. + private String jsonPaths = ""; + private String jsonRoot = ""; // MUST be a jsonpath string + private boolean stripOuterArray = false; + private boolean numAsString = false; + private boolean fuzzyParse = false; + + private byte enclose; + + private byte escape; + + private long workloadGroupId = -1; + + /** + * support partial columns load(Only Unique Key Columns) + */ + private boolean isPartialUpdate = false; + + private String comment = ""; + + private LoadTask.MergeType mergeType; + + private boolean isMultiTable = false; + + private AbstractDataSourceProperties dataSourceProperties; + + /** + * constructor for create table + */ + public CreateRoutineLoadInfo(LabelNameInfo labelNameInfo, String tableName, + Map loadPropertyMap, + Map jobProperties, String typeName, + Map dataSourceProperties, LoadTask.MergeType mergeType, + String comment) { + this.labelNameInfo = labelNameInfo; + if (StringUtils.isBlank(tableName)) { + this.isMultiTable = true; + } + this.tableName = tableName; + this.loadPropertyMap = loadPropertyMap; + this.jobProperties = jobProperties == null ? Maps.newHashMap() : jobProperties; + this.typeName = typeName.toUpperCase(); + this.dataSourceProperties = RoutineLoadDataSourcePropertyFactory + .createDataSource(typeName, dataSourceProperties, this.isMultiTable); + this.mergeType = mergeType; + this.isPartialUpdate = this.jobProperties.getOrDefault(PARTIAL_COLUMNS, "false").equalsIgnoreCase("true"); + if (comment != null) { + this.comment = comment; + } + } + + /** + * analyze create table info + */ + public void validate(ConnectContext ctx) throws UserException { + // check dbName and tableName + checkDBTable(ctx); + // check name + try { + FeNameFormat.checkCommonName(NAME_TYPE, name); + } catch (org.apache.doris.common.AnalysisException e) { + // 64 is the length of regular expression matching + // (FeNameFormat.COMMON_NAME_REGEX/UNDERSCORE_COMMON_NAME_REGEX) + throw new AnalysisException(e.getMessage() + + " Maybe routine load job name is longer than 64 or contains illegal characters"); + } + // check load properties include column separator etc. + checkLoadProperties(ctx); + // check routine load job properties include desired concurrent number etc. + checkJobProperties(); + // check data source properties + checkDataSourceProperties(); + // analyze merge type + if (routineLoadDesc != null) { + if (mergeType != LoadTask.MergeType.MERGE && routineLoadDesc.getDeleteCondition() != null) { + throw new AnalysisException("not support DELETE ON clause when merge type is not MERGE."); + } + if (mergeType == LoadTask.MergeType.MERGE && routineLoadDesc.getDeleteCondition() == null) { + throw new AnalysisException("Excepted DELETE ON clause when merge type is MERGE."); + } + } else if (mergeType == LoadTask.MergeType.MERGE) { + throw new AnalysisException("Excepted DELETE ON clause when merge type is MERGE."); + } + } + + private void checkDBTable(ConnectContext ctx) throws AnalysisException { + labelNameInfo.validate(ctx); + dbName = labelNameInfo.getDb(); + name = labelNameInfo.getLabel(); + + Database db = Env.getCurrentInternalCatalog().getDbOrAnalysisException(dbName); + if (isPartialUpdate && isMultiTable) { + throw new AnalysisException("Partial update is not supported in multi-table load."); + } + if (isMultiTable) { + return; + } + if (Strings.isNullOrEmpty(tableName)) { + throw new AnalysisException("Table name should not be null"); + } + Table table = db.getTableOrAnalysisException(tableName); + if (mergeType != LoadTask.MergeType.APPEND + && (table.getType() != Table.TableType.OLAP + || ((OlapTable) table).getKeysType() != KeysType.UNIQUE_KEYS)) { + throw new AnalysisException("load by MERGE or DELETE is only supported in unique tables."); + } + if (mergeType != LoadTask.MergeType.APPEND + && !(table.getType() == Table.TableType.OLAP && ((OlapTable) table).hasDeleteSign())) { + throw new AnalysisException("load by MERGE or DELETE need to upgrade table to support batch delete."); + } + if (isPartialUpdate && !((OlapTable) table).getEnableUniqueKeyMergeOnWrite()) { + throw new AnalysisException("load by PARTIAL_COLUMNS is only supported in unique table MoW"); + } + } + + private void checkLoadProperties(ConnectContext ctx) throws UserException { + Separator columnSeparator = null; + // TODO(yangzhengguo01): add line delimiter to properties + Separator lineDelimiter = null; + ImportColumnsStmt importColumnsStmt = null; + ImportWhereStmt precedingImportWhereStmt = null; + ImportWhereStmt importWhereStmt = null; + ImportSequenceStmt importSequenceStmt = null; + PartitionNames partitionNames = null; + ImportDeleteOnStmt importDeleteOnStmt = null; + CascadesContext cascadesContext = null; + ExpressionAnalyzer analyzer = null; + PlanTranslatorContext context = null; + if (!isMultiTable) { + List nameParts = Lists.newArrayList(); + nameParts.add(dbName); + nameParts.add(tableName); + Plan unboundRelation = new UnboundRelation(StatementScopeIdGenerator.newRelationId(), nameParts); + cascadesContext = CascadesContext.initContext(ctx.getStatementContext(), unboundRelation, + PhysicalProperties.ANY); + Rewriter.getWholeTreeRewriterWithCustomJobs(cascadesContext, + ImmutableList.of(Rewriter.bottomUp(new BindRelation()))).execute(); + Plan boundRelation = cascadesContext.getRewritePlan(); + // table could have delete sign in LogicalFilter above + if (cascadesContext.getRewritePlan() instanceof LogicalFilter) { + boundRelation = (Plan) ((LogicalFilter) cascadesContext.getRewritePlan()).child(); + } + context = new PlanTranslatorContext(cascadesContext); + List slots = boundRelation.getOutput(); + Scope scope = new Scope(slots); + analyzer = new ExpressionAnalyzer(null, scope, cascadesContext, false, false); + + Map translateMap = Maps.newHashMap(); + + TupleDescriptor tupleDescriptor = context.generateTupleDesc(); + tupleDescriptor.setTable(((OlapScan) boundRelation).getTable()); + for (int i = 0; i < boundRelation.getOutput().size(); i++) { + SlotReference slotReference = (SlotReference) boundRelation.getOutput().get(i); + SlotRef slotRef = new SlotRef(null, slotReference.getName()); + translateMap.put(slotReference, slotRef); + context.createSlotDesc(tupleDescriptor, slotReference, ((OlapScan) boundRelation).getTable()); + } + } + + if (loadPropertyMap != null) { + for (LoadProperty loadProperty : loadPropertyMap.values()) { + loadProperty.validate(); + if (loadProperty instanceof LoadSeparator) { + String oriSeparator = ((LoadSeparator) loadProperty).getOriSeparator(); + String separator = Separator.convertSeparator(oriSeparator); + columnSeparator = new Separator(separator, oriSeparator); + } else if (loadProperty instanceof LoadColumnClause) { + if (isMultiTable) { + throw new AnalysisException("Multi-table load does not support setting columns info"); + } + List importColumnDescList = new ArrayList<>(); + for (LoadColumnDesc columnDesc : ((LoadColumnClause) loadProperty).getColumns()) { + if (columnDesc.getExpression() != null) { + Expr expr = translateToLegacyExpr(columnDesc.getExpression(), analyzer, + context, cascadesContext); + importColumnDescList.add(new ImportColumnDesc(columnDesc.getColumnName(), expr)); + } else { + importColumnDescList.add(new ImportColumnDesc(columnDesc.getColumnName(), null)); + } + } + importColumnsStmt = new ImportColumnsStmt(importColumnDescList); + } else if (loadProperty instanceof LoadWhereClause) { + if (isMultiTable) { + throw new AnalysisException("Multi-table load does not support setting columns info"); + } + Expr expr = translateToLegacyExpr(((LoadWhereClause) loadProperty).getExpression(), + analyzer, context, cascadesContext); + if (((LoadWhereClause) loadProperty).isPreceding()) { + precedingImportWhereStmt = new ImportWhereStmt(expr, + ((LoadWhereClause) loadProperty).isPreceding()); + } else { + importWhereStmt = new ImportWhereStmt(expr, ((LoadWhereClause) loadProperty).isPreceding()); + } + } else if (loadProperty instanceof LoadPartitionNames) { + partitionNames = new PartitionNames(((LoadPartitionNames) loadProperty).isTemp(), + ((LoadPartitionNames) loadProperty).getPartitionNames()); + } else if (loadProperty instanceof LoadDeleteOnClause) { + Expr expr = translateToLegacyExpr(((LoadDeleteOnClause) loadProperty).getExpression(), + analyzer, context, cascadesContext); + importDeleteOnStmt = new ImportDeleteOnStmt(expr); + } else if (loadProperty instanceof LoadSequenceClause) { + importSequenceStmt = new ImportSequenceStmt( + ((LoadSequenceClause) loadProperty).getSequenceColName()); + } + } + } + routineLoadDesc = new RoutineLoadDesc(columnSeparator, lineDelimiter, importColumnsStmt, + precedingImportWhereStmt, importWhereStmt, + partitionNames, importDeleteOnStmt == null ? null : importDeleteOnStmt.getExpr(), mergeType, + importSequenceStmt == null ? null : importSequenceStmt.getSequenceColName()); + } + + private Expr translateToLegacyExpr(Expression expr, ExpressionAnalyzer analyzer, PlanTranslatorContext context, + CascadesContext cascadesContext) { + Expression expression; + try { + expression = analyzer.analyze(expr, new ExpressionRewriteContext(cascadesContext)); + } catch (org.apache.doris.nereids.exceptions.AnalysisException e) { + throw new org.apache.doris.nereids.exceptions.AnalysisException("In where clause '" + + expr.toSql() + "', " + + Utils.convertFirstChar(e.getMessage())); + } + return ExpressionTranslator.translate(expression, context); + } + + private void checkJobProperties() throws UserException { + Optional optional = jobProperties.keySet().stream().filter( + entity -> !PROPERTIES_SET.contains(entity)).findFirst(); + if (optional.isPresent()) { + throw new AnalysisException(optional.get() + " is invalid property"); + } + + desiredConcurrentNum = ((Long) Util.getLongPropertyOrDefault( + jobProperties.get(DESIRED_CONCURRENT_NUMBER_PROPERTY), + Config.max_routine_load_task_concurrent_num, DESIRED_CONCURRENT_NUMBER_PRED, + DESIRED_CONCURRENT_NUMBER_PROPERTY + " must be greater than 0")).intValue(); + + maxErrorNum = Util.getLongPropertyOrDefault(jobProperties.get(MAX_ERROR_NUMBER_PROPERTY), + RoutineLoadJob.DEFAULT_MAX_ERROR_NUM, MAX_ERROR_NUMBER_PRED, + MAX_ERROR_NUMBER_PROPERTY + " should >= 0"); + + maxFilterRatio = Util.getDoublePropertyOrDefault(jobProperties.get(MAX_FILTER_RATIO_PROPERTY), + RoutineLoadJob.DEFAULT_MAX_FILTER_RATIO, MAX_FILTER_RATIO_PRED, + MAX_FILTER_RATIO_PROPERTY + " should between 0 and 1"); + + maxBatchIntervalS = Util.getLongPropertyOrDefault(jobProperties.get(MAX_BATCH_INTERVAL_SEC_PROPERTY), + RoutineLoadJob.DEFAULT_MAX_INTERVAL_SECOND, MAX_BATCH_INTERVAL_PRED, + MAX_BATCH_INTERVAL_SEC_PROPERTY + " should >= 1"); + + maxBatchRows = Util.getLongPropertyOrDefault(jobProperties.get(MAX_BATCH_ROWS_PROPERTY), + RoutineLoadJob.DEFAULT_MAX_BATCH_ROWS, MAX_BATCH_ROWS_PRED, + MAX_BATCH_ROWS_PROPERTY + " should > 200000"); + + maxBatchSizeBytes = Util.getLongPropertyOrDefault(jobProperties.get(MAX_BATCH_SIZE_PROPERTY), + RoutineLoadJob.DEFAULT_MAX_BATCH_SIZE, MAX_BATCH_SIZE_PRED, + MAX_BATCH_SIZE_PROPERTY + " should between 100MB and 10GB"); + + strictMode = Util.getBooleanPropertyOrDefault(jobProperties.get(LoadStmt.STRICT_MODE), + RoutineLoadJob.DEFAULT_STRICT_MODE, + LoadStmt.STRICT_MODE + " should be a boolean"); + execMemLimit = Util.getLongPropertyOrDefault(jobProperties.get(EXEC_MEM_LIMIT_PROPERTY), + RoutineLoadJob.DEFAULT_EXEC_MEM_LIMIT, EXEC_MEM_LIMIT_PRED, + EXEC_MEM_LIMIT_PROPERTY + " must be greater than 0"); + + sendBatchParallelism = ((Long) Util.getLongPropertyOrDefault(jobProperties.get(SEND_BATCH_PARALLELISM), + ConnectContext.get().getSessionVariable().getSendBatchParallelism(), SEND_BATCH_PARALLELISM_PRED, + SEND_BATCH_PARALLELISM + " must be greater than 0")).intValue(); + loadToSingleTablet = Util.getBooleanPropertyOrDefault(jobProperties.get(LoadStmt.LOAD_TO_SINGLE_TABLET), + RoutineLoadJob.DEFAULT_LOAD_TO_SINGLE_TABLET, + LoadStmt.LOAD_TO_SINGLE_TABLET + " should be a boolean"); + + String encloseStr = jobProperties.get(LoadStmt.KEY_ENCLOSE); + if (encloseStr != null) { + if (encloseStr.length() != 1) { + throw new AnalysisException("enclose must be single-char"); + } else { + enclose = encloseStr.getBytes()[0]; + } + } + String escapeStr = jobProperties.get(LoadStmt.KEY_ESCAPE); + if (escapeStr != null) { + if (escapeStr.length() != 1) { + throw new AnalysisException("enclose must be single-char"); + } else { + escape = escapeStr.getBytes()[0]; + } + } + + String inputWorkloadGroupStr = jobProperties.get(WORKLOAD_GROUP); + if (!StringUtils.isEmpty(inputWorkloadGroupStr)) { + this.workloadGroupId = Env.getCurrentEnv().getWorkloadGroupMgr() + .getWorkloadGroup(ConnectContext.get().getCurrentUserIdentity(), inputWorkloadGroupStr); + } + + if (ConnectContext.get() != null) { + timezone = ConnectContext.get().getSessionVariable().getTimeZone(); + } + timezone = TimeUtils.checkTimeZoneValidAndStandardize(jobProperties.getOrDefault(LoadStmt.TIMEZONE, timezone)); + + format = jobProperties.get(FORMAT); + if (format != null) { + if (format.equalsIgnoreCase("csv")) { + format = ""; // if it's not json, then it's mean csv and set empty + } else if (format.equalsIgnoreCase("json")) { + format = "json"; + jsonPaths = jobProperties.getOrDefault(JSONPATHS, ""); + jsonRoot = jobProperties.getOrDefault(JSONROOT, ""); + stripOuterArray = Boolean.parseBoolean(jobProperties.getOrDefault(STRIP_OUTER_ARRAY, "false")); + numAsString = Boolean.parseBoolean(jobProperties.getOrDefault(NUM_AS_STRING, "false")); + fuzzyParse = Boolean.parseBoolean(jobProperties.getOrDefault(FUZZY_PARSE, "false")); + } else { + throw new UserException("Format type is invalid. format=`" + format + "`"); + } + } else { + format = "csv"; // default csv + } + } + + private void checkDataSourceProperties() throws UserException { + this.dataSourceProperties.setTimezone(this.timezone); + this.dataSourceProperties.analyze(); + } + + /** + * make legacy create routine load statement after validate by nereids + * @return legacy create routine load statement + */ + public CreateRoutineLoadStmt translateToLegacyStmt(ConnectContext ctx) { + return new CreateRoutineLoadStmt(labelNameInfo.transferToLabelName(), dbName, name, tableName, null, + ctx.getStatementContext().getOriginStatement(), ctx.getUserIdentity(), + jobProperties, typeName, routineLoadDesc, + desiredConcurrentNum, maxErrorNum, maxFilterRatio, maxBatchIntervalS, maxBatchRows, maxBatchSizeBytes, + execMemLimit, sendBatchParallelism, timezone, format, jsonPaths, jsonRoot, enclose, escape, workloadGroupId, + loadToSingleTablet, strictMode, isPartialUpdate, stripOuterArray, numAsString, fuzzyParse, + dataSourceProperties + ); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/LabelNameInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/LabelNameInfo.java new file mode 100644 index 00000000000000..c314b5bce0defe --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/LabelNameInfo.java @@ -0,0 +1,108 @@ +// 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. +// This file is copied from +// https://github.com/apache/impala/blob/branch-2.9.0/fe/src/main/java/org/apache/impala/TableName.java +// and modified by Doris + +package org.apache.doris.nereids.trees.plans.commands.info; + +import org.apache.doris.analysis.LabelName; +import org.apache.doris.catalog.Env; +import org.apache.doris.common.FeNameFormat; +import org.apache.doris.nereids.exceptions.AnalysisException; +import org.apache.doris.qe.ConnectContext; + +import com.google.common.base.Strings; + +import java.util.Objects; + +/** + * Label name info + */ +public class LabelNameInfo { + private String label; + private String db; + + public LabelNameInfo() { + + } + + /** + * TableNameInfo + * @param db dbName + * @param label tblName + */ + public LabelNameInfo(String db, String label) { + Objects.requireNonNull(label, "require label object"); + this.label = label; + if (Env.isStoredTableNamesLowerCase()) { + this.label = label.toLowerCase(); + } + this.db = db; + } + + /** + * validate labelNameInfo + * @param ctx ctx + */ + public void validate(ConnectContext ctx) throws org.apache.doris.common.AnalysisException { + if (Strings.isNullOrEmpty(db)) { + db = ctx.getDatabase(); + if (Strings.isNullOrEmpty(db)) { + throw new AnalysisException("No database selected"); + } + } + + if (Strings.isNullOrEmpty(label)) { + throw new AnalysisException("Table name is null"); + } + + FeNameFormat.checkLabel(label); + } + + /** + * get db name + * @return dbName + */ + public String getDb() { + return db; + } + + /** + * set a new database name + * @param db new database name + */ + public void setDb(String db) { + this.db = db; + } + + /** + * get label name + * @return labelName + */ + public String getLabel() { + return label; + } + + /** + * transferToLabelName + * @return LabelName + */ + public LabelName transferToLabelName() { + return new LabelName(db, label); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/load/CreateRoutineLoadCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/load/CreateRoutineLoadCommand.java new file mode 100644 index 00000000000000..da7ab86c17abac --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/load/CreateRoutineLoadCommand.java @@ -0,0 +1,90 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.commands.load; + +import org.apache.doris.analysis.CreateRoutineLoadStmt; +import org.apache.doris.catalog.Env; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.commands.Command; +import org.apache.doris.nereids.trees.plans.commands.NoForward; +import org.apache.doris.nereids.trees.plans.commands.info.CreateRoutineLoadInfo; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.StmtExecutor; + +import java.util.Objects; + +/** + Create routine Load statement, continually load data from a streaming app + + syntax: + CREATE ROUTINE LOAD [database.]name on table + [load properties] + [PROPERTIES + ( + desired_concurrent_number = xxx, + max_error_number = xxx, + k1 = v1, + ... + kn = vn + )] + FROM type of routine load + [( + k1 = v1, + ... + kn = vn + )] + + load properties: + load property [[,] load property] ... + + load property: + column separator | columns_mapping | partitions | where + + column separator: + COLUMNS TERMINATED BY xxx + columns_mapping: + COLUMNS (c1, c2, c3 = c1 + c2) + partitions: + PARTITIONS (p1, p2, p3) + where: + WHERE c1 > 1 + + type of routine load: + KAFKA +*/ +public class CreateRoutineLoadCommand extends Command implements NoForward { + CreateRoutineLoadInfo createRoutineLoadInfo; + + public CreateRoutineLoadCommand(CreateRoutineLoadInfo createRoutineLoadInfo) { + super(PlanType.CREATE_ROUTINE_LOAD_COMMAND); + this.createRoutineLoadInfo = Objects.requireNonNull(createRoutineLoadInfo, "require CreateTableInfo object"); + } + + @Override + public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { + createRoutineLoadInfo.validate(ctx); + CreateRoutineLoadStmt createRoutineLoadStmt = createRoutineLoadInfo.translateToLegacyStmt(ctx); + Env.getCurrentEnv().getRoutineLoadManager().createRoutineLoadJob(createRoutineLoadStmt); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitCreateRoutineLoadCommand(this, context); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/load/LoadColumnClause.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/load/LoadColumnClause.java new file mode 100644 index 00000000000000..a2bc7acbabe076 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/load/LoadColumnClause.java @@ -0,0 +1,35 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.commands.load; + +import java.util.List; + +/** + * load LoadColumnClause for nereids + */ +public class LoadColumnClause implements LoadProperty { + private List columns; + + public LoadColumnClause(List columns) { + this.columns = columns; + } + + public List getColumns() { + return columns; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/load/LoadColumnDesc.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/load/LoadColumnDesc.java new file mode 100644 index 00000000000000..46b8013b159fa4 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/load/LoadColumnDesc.java @@ -0,0 +1,45 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.commands.load; + +import org.apache.doris.nereids.trees.expressions.Expression; + +/** + * load LoadColumnDesc for nereids + */ +public class LoadColumnDesc { + private final String columnName; + private Expression expression; + + public LoadColumnDesc(String column, Expression expression) { + this.columnName = column; + this.expression = expression; + } + + public LoadColumnDesc(String column) { + this.columnName = column; + } + + public String getColumnName() { + return columnName; + } + + public Expression getExpression() { + return expression; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/load/LoadDeleteOnClause.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/load/LoadDeleteOnClause.java new file mode 100644 index 00000000000000..4ab77f8e9bb300 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/load/LoadDeleteOnClause.java @@ -0,0 +1,35 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.commands.load; + +import org.apache.doris.nereids.trees.expressions.Expression; + +/** + * load LoadDeleteOnClause for nereids + */ +public class LoadDeleteOnClause implements LoadProperty { + private final Expression expression; + + public LoadDeleteOnClause(Expression expression) { + this.expression = expression; + } + + public Expression getExpression() { + return expression; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/load/LoadPartitionNames.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/load/LoadPartitionNames.java new file mode 100644 index 00000000000000..77b06bc695c75b --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/load/LoadPartitionNames.java @@ -0,0 +1,70 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.commands.load; + +import org.apache.doris.common.AnalysisException; + +import com.google.common.base.Strings; +import com.google.gson.annotations.SerializedName; + +import java.util.List; + +/** + * load LoadPartitionNames for nereids + */ +public class LoadPartitionNames implements LoadProperty { + @SerializedName(value = "partitionNames") + private final List partitionNames; + // true if these partitions are temp partitions + @SerializedName(value = "isTemp") + private final boolean isTemp; + private final boolean isStar; + private final long count; + + public LoadPartitionNames(boolean isTemp, List partitionNames) { + this.isTemp = isTemp; + this.partitionNames = partitionNames; + this.isStar = false; + this.count = 0; + } + + public List getPartitionNames() { + return partitionNames; + } + + public boolean isTemp() { + return isTemp; + } + + @Override + public void validate() throws AnalysisException { + if (isStar && count > 0) { + throw new AnalysisException("All partition and partition count couldn't be set at the same time."); + } + if (isStar || count > 0) { + return; + } + if (partitionNames == null || partitionNames.isEmpty()) { + throw new AnalysisException("No partition specified in partition lists"); + } + // check if partition name is not empty string + if (partitionNames.stream().anyMatch(Strings::isNullOrEmpty)) { + throw new AnalysisException("there are empty partition name"); + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/load/LoadProperty.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/load/LoadProperty.java new file mode 100644 index 00000000000000..c21ba1c5774bbb --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/load/LoadProperty.java @@ -0,0 +1,36 @@ +// 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. +// This file is copied from +// https://github.com/apache/impala/blob/branch-2.9.0/fe/src/main/java/org/apache/impala/ParseNode.java +// and modified by Doris + +package org.apache.doris.nereids.trees.plans.commands.load; + +import org.apache.doris.common.AnalysisException; + +/** + * load LoadProperty for nereids + */ +public interface LoadProperty { + + /** + * Perform semantic validate of node and all of its children. + * Throws exception if any errors found. + */ + default void validate() throws AnalysisException { + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/load/LoadSeparator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/load/LoadSeparator.java new file mode 100644 index 00000000000000..a1b31cb721638a --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/load/LoadSeparator.java @@ -0,0 +1,33 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.commands.load; + +/** + * load seperator for nereids + */ +public class LoadSeparator implements LoadProperty { + private final String oriSeparator; + + public LoadSeparator(String oriSeparator) { + this.oriSeparator = oriSeparator; + } + + public String getOriSeparator() { + return oriSeparator; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/load/LoadSequenceClause.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/load/LoadSequenceClause.java new file mode 100644 index 00000000000000..e9545382fe7dc3 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/load/LoadSequenceClause.java @@ -0,0 +1,33 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.commands.load; + +/** + * load LoadSequenceClause for nereids + */ +public class LoadSequenceClause implements LoadProperty { + private final String sequenceColName; + + public LoadSequenceClause(String sequenceColName) { + this.sequenceColName = sequenceColName; + } + + public String getSequenceColName() { + return sequenceColName; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/load/LoadWhereClause.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/load/LoadWhereClause.java new file mode 100644 index 00000000000000..627f03e89e5c9b --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/load/LoadWhereClause.java @@ -0,0 +1,42 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.commands.load; + +import org.apache.doris.nereids.trees.expressions.Expression; + +/** + * load LoadWhereClause for nereids + */ +public class LoadWhereClause implements LoadProperty { + private final Expression expression; + + private final boolean isPreceding; + + public LoadWhereClause(Expression expression, boolean isPreceding) { + this.expression = expression; + this.isPreceding = isPreceding; + } + + public Expression getExpression() { + return expression; + } + + public boolean isPreceding() { + return isPreceding; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index c0f86d8dfa45ba..85633cb798b06c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -106,6 +106,7 @@ import org.apache.doris.nereids.trees.plans.commands.insert.BatchInsertIntoTableCommand; import org.apache.doris.nereids.trees.plans.commands.insert.InsertIntoTableCommand; import org.apache.doris.nereids.trees.plans.commands.insert.InsertOverwriteTableCommand; +import org.apache.doris.nereids.trees.plans.commands.load.CreateRoutineLoadCommand; import org.apache.doris.nereids.trees.plans.commands.refresh.RefreshCatalogCommand; /** CommandVisitor. */ @@ -473,4 +474,8 @@ default R visitShowPrivilegesCommand(ShowPrivilegesCommand showPrivilegesCommand default R visitShowTabletsBelongCommand(ShowTabletsBelongCommand showTabletBelongCommand, C context) { return visitCommand(showTabletBelongCommand, context); } + + default R visitCreateRoutineLoadCommand(CreateRoutineLoadCommand createRoutineLoadCommand, C context) { + return visitCommand(createRoutineLoadCommand, context); + } } diff --git a/regression-test/suites/load_p0/routine_load/test_routine_load_condition.groovy b/regression-test/suites/load_p0/routine_load/test_routine_load_condition.groovy index 7735867c749049..d971a298ffb39d 100644 --- a/regression-test/suites/load_p0/routine_load/test_routine_load_condition.groovy +++ b/regression-test/suites/load_p0/routine_load/test_routine_load_condition.groovy @@ -157,7 +157,7 @@ suite("test_routine_load_condition","p0") { } log.info("reason of state changed: ${res[0][11].toString()}".toString()) def json = parseJson(res[0][11]) - assertEquals("(`k12` >= date_sub(curdate(), INTERVAL 2 DAY))", json.whereExpr.toString()) + assertEquals("(k12 >= CAST(days_sub(current_date(), 2) AS datetimev2(0)))", json.whereExpr.toString()) break; } while (true) { @@ -180,4 +180,4 @@ suite("test_routine_load_condition","p0") { sql "DROP TABLE IF EXISTS ${tableName}" } } -} \ No newline at end of file +} diff --git a/regression-test/suites/load_p2/routine_load/test_routine_load.groovy b/regression-test/suites/load_p2/routine_load/test_routine_load.groovy index 2b59c0f8e60d77..24cb5627f7f666 100644 --- a/regression-test/suites/load_p2/routine_load/test_routine_load.groovy +++ b/regression-test/suites/load_p2/routine_load/test_routine_load.groovy @@ -221,7 +221,7 @@ suite("test_routine_load_p2","p2,nonConcurrent") { sql new File("""${context.file.parent}/ddl/${tableName}_create.sql""").text def name = "routine_load_" + tableName - sql """ + checkNereidsExecute(""" CREATE ROUTINE LOAD ${jobs[i]} ON ${name} COLUMNS(${columns[i]}), COLUMNS TERMINATED BY "|" @@ -239,7 +239,7 @@ suite("test_routine_load_p2","p2,nonConcurrent") { "kafka_topic" = "${topics[i]}", "property.kafka_default_offsets" = "OFFSET_BEGINNING" ); - """ + """) sql "sync" i++ } @@ -2380,4 +2380,4 @@ suite("test_routine_load_p2","p2,nonConcurrent") { } } } -} \ No newline at end of file +} From 45dc1c91a0dba0ee5f2663167d5dee24a409bfae Mon Sep 17 00:00:00 2001 From: abmdocrt Date: Wed, 27 Nov 2024 10:57:44 +0800 Subject: [PATCH 031/399] [Fix](group commit) Fix Group Commit Selecting Non-Alive Backend Instances (#44577) --- .../main/java/org/apache/doris/load/GroupCommitManager.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/GroupCommitManager.java b/fe/fe-core/src/main/java/org/apache/doris/load/GroupCommitManager.java index 60394f943654c6..1844ec5bf2b427 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/GroupCommitManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/GroupCommitManager.java @@ -336,7 +336,7 @@ private Long getCachedBackend(String cluster, long tableId) { return null; } Backend backend = Env.getCurrentSystemInfo().getBackend(backendId); - if (backend != null && backend.isActive() && !backend.isDecommissioned()) { + if (backend != null && backend.isAlive() && !backend.isDecommissioned()) { return backend.getId(); } else { tableToBeMap.remove(encode(cluster, tableId)); @@ -353,7 +353,7 @@ private Long getRandomBackend(String cluster, long tableId, List backen OlapTable table = (OlapTable) Env.getCurrentEnv().getInternalCatalog().getTableByTableId(tableId); Collections.shuffle(backends); for (Backend backend : backends) { - if (backend.isActive() && !backend.isDecommissioned()) { + if (backend.isAlive() && !backend.isDecommissioned()) { tableToBeMap.put(encode(cluster, tableId), backend.getId()); tableToPressureMap.put(tableId, new SlidingWindowCounter(table.getGroupCommitIntervalMs() / 1000 + 1)); From 7a6bb51ec4e30019efb1dcd2298bd0dd9772f0cc Mon Sep 17 00:00:00 2001 From: Xinyi Zou Date: Wed, 27 Nov 2024 11:09:17 +0800 Subject: [PATCH 032/399] [fix](arrow-flight-sql) Remove `arrow_flight_sql` group from regression test (#44489) ### What problem does this PR solve? Problem Summary: Add `./run-regression-test.sh --run ... -g arrow_flight_sql` to the pipeline. then add the group `arrow_flight_sql` again, which is safer. --- .../nereids_p0/show/test_show_create_table_nereids.groovy | 2 +- .../suites/query_p0/aggregate/aggregate_count1.groovy | 2 +- .../suites/query_p0/aggregate/select_distinct.groovy | 2 +- .../casesensetive_column/join_with_column_casesensetive.groovy | 2 +- regression-test/suites/query_p0/cast/test_cast.groovy | 2 +- regression-test/suites/query_p0/except/test_query_except.groovy | 2 +- .../suites/query_p0/group_concat/test_group_concat.groovy | 2 +- .../suites/query_p0/grouping_sets/test_grouping_sets1.groovy | 2 +- regression-test/suites/query_p0/having/having.groovy | 2 +- regression-test/suites/query_p0/intersect/test_intersect.groovy | 2 +- regression-test/suites/query_p0/join/test_join2.groovy | 2 +- regression-test/suites/query_p0/join/test_left_join1.groovy | 2 +- .../suites/query_p0/join/test_nestedloop_outer_join.groovy | 2 +- .../suites/query_p0/join/test_partitioned_hash_join.groovy | 2 +- .../suites/query_p0/lateral_view/lateral_view.groovy | 2 +- .../suites/query_p0/limit/OffsetInSubqueryWithJoin.groovy | 2 +- .../suites/query_p0/literal_view/lietral_test.groovy | 2 +- .../suites/query_p0/operator/test_set_operator.groovy | 2 +- .../suites/query_p0/operator/test_sort_operator.groovy | 2 +- .../suites/query_p0/session_variable/test_default_limit.groovy | 2 +- .../query_p0/show/test_show_create_materialized_view.groovy | 2 +- .../suites/query_p0/show/test_show_create_table.groovy | 2 +- .../aggregate_functions/test_aggregate_all_functions.groovy | 2 +- .../sql_functions/case_function/test_case_function_null.groovy | 2 +- .../sql_functions/hash_functions/test_hash_function.groovy | 2 +- .../sql_functions/ip_functions/test_ip_functions.groovy | 2 +- .../sql_functions/json_function/test_query_json_insert.groovy | 2 +- .../sql_functions/json_functions/test_json_function.groovy | 2 +- .../query_p0/sql_functions/math_functions/test_conv.groovy | 2 +- .../search_functions/test_multi_string_search.groovy | 2 +- .../sql_functions/spatial_functions/test_gis_function.groovy | 2 +- .../sql_functions/string_functions/test_string_function.groovy | 2 +- .../query_p0/sql_functions/table_function/explode_split.groovy | 2 +- .../suites/query_p0/sql_functions/test_alias_function.groovy | 2 +- .../suites/query_p0/sql_functions/test_predicate.groovy | 2 +- .../width_bucket_fuctions/test_width_bucket_function.groovy | 2 +- regression-test/suites/query_p0/subquery/test_subquery2.groovy | 2 +- regression-test/suites/query_p0/test_data_type_marks.groovy | 2 +- regression-test/suites/query_p0/test_dict_with_null.groovy | 2 +- regression-test/suites/query_p0/test_orderby_nullliteral.groovy | 2 +- regression-test/suites/query_p0/test_select_constant.groovy | 2 +- .../suites/query_p0/test_select_with_predicate_like.groovy | 2 +- .../suites/query_p0/test_select_with_predicate_prune.groovy | 2 +- .../suites/query_p0/type_inference/test_largeint.groovy | 2 +- .../suites/query_p0/with/test_with_and_two_phase_agg.groovy | 2 +- 45 files changed, 45 insertions(+), 45 deletions(-) diff --git a/regression-test/suites/nereids_p0/show/test_show_create_table_nereids.groovy b/regression-test/suites/nereids_p0/show/test_show_create_table_nereids.groovy index 53f2242d1fa7f6..8c9b42af41a8a6 100644 --- a/regression-test/suites/nereids_p0/show/test_show_create_table_nereids.groovy +++ b/regression-test/suites/nereids_p0/show/test_show_create_table_nereids.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_show_create_table_nereids", "query,arrow_flight_sql") { +suite("test_show_create_table_nereids", "query") { String tb_name = "tb_show_create_table"; try { sql """drop table if exists ${tb_name} """ diff --git a/regression-test/suites/query_p0/aggregate/aggregate_count1.groovy b/regression-test/suites/query_p0/aggregate/aggregate_count1.groovy index 3971f304e38646..cf657cc8ef3e3d 100644 --- a/regression-test/suites/query_p0/aggregate/aggregate_count1.groovy +++ b/regression-test/suites/query_p0/aggregate/aggregate_count1.groovy @@ -17,7 +17,7 @@ * under the License. */ -suite("aggregate_count1", "query,arrow_flight_sql") { +suite("aggregate_count1", "query") { sql """ DROP TABLE IF EXISTS aggregate_count1 """ sql """create table if not exists aggregate_count1 ( name varchar(128), diff --git a/regression-test/suites/query_p0/aggregate/select_distinct.groovy b/regression-test/suites/query_p0/aggregate/select_distinct.groovy index 2d6a8679d87ed8..6456158bdadb0d 100644 --- a/regression-test/suites/query_p0/aggregate/select_distinct.groovy +++ b/regression-test/suites/query_p0/aggregate/select_distinct.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("select_distinct", "arrow_flight_sql") { +suite("select_distinct") { sql """DROP TABLE IF EXISTS decimal_a;""" sql """DROP TABLE IF EXISTS decimal_b;""" sql """DROP TABLE IF EXISTS decimal_c;""" diff --git a/regression-test/suites/query_p0/casesensetive_column/join_with_column_casesensetive.groovy b/regression-test/suites/query_p0/casesensetive_column/join_with_column_casesensetive.groovy index 8bd3b19088a486..45499fc6f248a2 100644 --- a/regression-test/suites/query_p0/casesensetive_column/join_with_column_casesensetive.groovy +++ b/regression-test/suites/query_p0/casesensetive_column/join_with_column_casesensetive.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("join_with_column_casesensetive", "arrow_flight_sql") { +suite("join_with_column_casesensetive") { def tables=["ad_order_data_v1","ad_order_data"] for (String table in tables) { diff --git a/regression-test/suites/query_p0/cast/test_cast.groovy b/regression-test/suites/query_p0/cast/test_cast.groovy index dae669e2965f04..947d61bc828861 100644 --- a/regression-test/suites/query_p0/cast/test_cast.groovy +++ b/regression-test/suites/query_p0/cast/test_cast.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite('test_cast', "arrow_flight_sql") { +suite('test_cast') { def date = "date '2020-01-01'" def datev2 = "datev2 '2020-01-01'" def datetime = "timestamp '2020-01-01 12:34:45'" diff --git a/regression-test/suites/query_p0/except/test_query_except.groovy b/regression-test/suites/query_p0/except/test_query_except.groovy index 410e24f89b92de..1a2aa742d2910d 100644 --- a/regression-test/suites/query_p0/except/test_query_except.groovy +++ b/regression-test/suites/query_p0/except/test_query_except.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_query_except", "arrow_flight_sql") { +suite("test_query_except") { // test query except, depend on query_test_data_load.groovy sql "use test_query_db" qt_select_except1 """ diff --git a/regression-test/suites/query_p0/group_concat/test_group_concat.groovy b/regression-test/suites/query_p0/group_concat/test_group_concat.groovy index 522d66ed64b30b..5054dc2ee3a91d 100644 --- a/regression-test/suites/query_p0/group_concat/test_group_concat.groovy +++ b/regression-test/suites/query_p0/group_concat/test_group_concat.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_group_concat", "query,p0,arrow_flight_sql") { +suite("test_group_concat", "query,p0") { qt_select """ SELECT group_concat(k6) FROM test_query_db.test where k6='false' """ diff --git a/regression-test/suites/query_p0/grouping_sets/test_grouping_sets1.groovy b/regression-test/suites/query_p0/grouping_sets/test_grouping_sets1.groovy index f8180b0ab43846..1f12de6628a5eb 100644 --- a/regression-test/suites/query_p0/grouping_sets/test_grouping_sets1.groovy +++ b/regression-test/suites/query_p0/grouping_sets/test_grouping_sets1.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_grouping_sets1", "arrow_flight_sql") { +suite("test_grouping_sets1") { qt_select """ select col1 diff --git a/regression-test/suites/query_p0/having/having.groovy b/regression-test/suites/query_p0/having/having.groovy index bbad236c973870..fb32b3834af4d6 100644 --- a/regression-test/suites/query_p0/having/having.groovy +++ b/regression-test/suites/query_p0/having/having.groovy @@ -19,7 +19,7 @@ // /testing/trino-product-tests/src/main/resources/sql-tests/testcases/aggregate // and modified by Doris. -suite("having", "query,p0,arrow_flight_sql") { +suite("having", "query,p0") { sql """DROP TABLE IF EXISTS supplier""" sql """CREATE TABLE `supplier` ( `s_suppkey` int(11) NOT NULL, diff --git a/regression-test/suites/query_p0/intersect/test_intersect.groovy b/regression-test/suites/query_p0/intersect/test_intersect.groovy index 7919bec324b876..1c007b95d7d07d 100644 --- a/regression-test/suites/query_p0/intersect/test_intersect.groovy +++ b/regression-test/suites/query_p0/intersect/test_intersect.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_intersect", "arrow_flight_sql") { +suite("test_intersect") { qt_select """ SELECT * FROM (SELECT k1 FROM test_query_db.baseall INTERSECT SELECT k1 FROM test_query_db.test) a ORDER BY k1 diff --git a/regression-test/suites/query_p0/join/test_join2.groovy b/regression-test/suites/query_p0/join/test_join2.groovy index 9158133948f754..6125b9a873f77e 100644 --- a/regression-test/suites/query_p0/join/test_join2.groovy +++ b/regression-test/suites/query_p0/join/test_join2.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_join2", "query,p0,arrow_flight_sql") { +suite("test_join2", "query,p0") { def DBname = "regression_test_join2" def TBname1 = "J1_TBL" def TBname2 = "J2_TBL" diff --git a/regression-test/suites/query_p0/join/test_left_join1.groovy b/regression-test/suites/query_p0/join/test_left_join1.groovy index 104adab4a850d0..d4cbeeee65eda2 100644 --- a/regression-test/suites/query_p0/join/test_left_join1.groovy +++ b/regression-test/suites/query_p0/join/test_left_join1.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_left_join1", "query,p0,arrow_flight_sql") { +suite("test_left_join1", "query,p0") { def tableName = "test_left_join1" sql """drop table if exists ${tableName}""" diff --git a/regression-test/suites/query_p0/join/test_nestedloop_outer_join.groovy b/regression-test/suites/query_p0/join/test_nestedloop_outer_join.groovy index f99dfa042446e9..ad19e554690ee7 100644 --- a/regression-test/suites/query_p0/join/test_nestedloop_outer_join.groovy +++ b/regression-test/suites/query_p0/join/test_nestedloop_outer_join.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_nestedloop_outer_join", "query_p0,arrow_flight_sql") { +suite("test_nestedloop_outer_join", "query_p0") { def tbl1 = "test_nestedloop_outer_join1" def tbl2 = "test_nestedloop_outer_join2" diff --git a/regression-test/suites/query_p0/join/test_partitioned_hash_join.groovy b/regression-test/suites/query_p0/join/test_partitioned_hash_join.groovy index 676cdd06274a68..cbe09ec527ffbc 100644 --- a/regression-test/suites/query_p0/join/test_partitioned_hash_join.groovy +++ b/regression-test/suites/query_p0/join/test_partitioned_hash_join.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_partitioned_hash_join", "query,p0,arrow_flight_sql") { +suite("test_partitioned_hash_join", "query,p0") { sql "drop table if exists test_partitioned_hash_join_l" sql "drop table if exists test_partitioned_hash_join_r" sql """ create table test_partitioned_hash_join_l ( diff --git a/regression-test/suites/query_p0/lateral_view/lateral_view.groovy b/regression-test/suites/query_p0/lateral_view/lateral_view.groovy index bfe6ca76872ea8..a24623590cd0af 100644 --- a/regression-test/suites/query_p0/lateral_view/lateral_view.groovy +++ b/regression-test/suites/query_p0/lateral_view/lateral_view.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("lateral_view", "arrow_flight_sql") { +suite("lateral_view") { sql """ DROP TABLE IF EXISTS `test_explode_bitmap` """ sql """ CREATE TABLE `test_explode_bitmap` ( diff --git a/regression-test/suites/query_p0/limit/OffsetInSubqueryWithJoin.groovy b/regression-test/suites/query_p0/limit/OffsetInSubqueryWithJoin.groovy index caa75ac7be38ed..da0c7231f425d1 100644 --- a/regression-test/suites/query_p0/limit/OffsetInSubqueryWithJoin.groovy +++ b/regression-test/suites/query_p0/limit/OffsetInSubqueryWithJoin.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_offset_in_subquery_with_join", "query,arrow_flight_sql") { +suite("test_offset_in_subquery_with_join", "query") { // define a sql table def testTable = "test_offset_in_subquery_with_join" diff --git a/regression-test/suites/query_p0/literal_view/lietral_test.groovy b/regression-test/suites/query_p0/literal_view/lietral_test.groovy index 27b82c16247a73..6e9d51f0a0d610 100644 --- a/regression-test/suites/query_p0/literal_view/lietral_test.groovy +++ b/regression-test/suites/query_p0/literal_view/lietral_test.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("literal_view_test", "arrow_flight_sql") { +suite("literal_view_test") { sql """DROP TABLE IF EXISTS table1""" diff --git a/regression-test/suites/query_p0/operator/test_set_operator.groovy b/regression-test/suites/query_p0/operator/test_set_operator.groovy index 7d6219585e4c4c..cb05e18b3e870b 100644 --- a/regression-test/suites/query_p0/operator/test_set_operator.groovy +++ b/regression-test/suites/query_p0/operator/test_set_operator.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_set_operators", "query,p0,arrow_flight_sql") { +suite("test_set_operators", "query,p0") { sql """ DROP TABLE IF EXISTS t1; diff --git a/regression-test/suites/query_p0/operator/test_sort_operator.groovy b/regression-test/suites/query_p0/operator/test_sort_operator.groovy index d76daff01f6fcc..24a2b8ef73a424 100644 --- a/regression-test/suites/query_p0/operator/test_sort_operator.groovy +++ b/regression-test/suites/query_p0/operator/test_sort_operator.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_sort_operator", "query,p0,arrow_flight_sql") { +suite("test_sort_operator", "query,p0") { sql """ DROP TABLE IF EXISTS dim_org_ful; diff --git a/regression-test/suites/query_p0/session_variable/test_default_limit.groovy b/regression-test/suites/query_p0/session_variable/test_default_limit.groovy index 2ce3b647142ae4..edda5d51790c56 100644 --- a/regression-test/suites/query_p0/session_variable/test_default_limit.groovy +++ b/regression-test/suites/query_p0/session_variable/test_default_limit.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite('test_default_limit', "arrow_flight_sql") { +suite('test_default_limit') { sql 'drop table if exists baseall' sql 'drop table if exists bigtable' diff --git a/regression-test/suites/query_p0/show/test_show_create_materialized_view.groovy b/regression-test/suites/query_p0/show/test_show_create_materialized_view.groovy index 9550a7fec3dbd2..7c5edcc81a2e8e 100644 --- a/regression-test/suites/query_p0/show/test_show_create_materialized_view.groovy +++ b/regression-test/suites/query_p0/show/test_show_create_materialized_view.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_show_create_materialized_view", "query,arrow_flight_sql") { +suite("test_show_create_materialized_view", "query") { String tableName = "table_for_mv_test"; String mvName = "mv_show_create_materialized_view"; try { diff --git a/regression-test/suites/query_p0/show/test_show_create_table.groovy b/regression-test/suites/query_p0/show/test_show_create_table.groovy index 1e3fc7ff5cb527..6325cbe319fd88 100644 --- a/regression-test/suites/query_p0/show/test_show_create_table.groovy +++ b/regression-test/suites/query_p0/show/test_show_create_table.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_show_create_table", "query,arrow_flight_sql") { +suite("test_show_create_table", "query") { String tb_name = "tb_show_create_table"; try { sql """drop table if exists ${tb_name} """ diff --git a/regression-test/suites/query_p0/sql_functions/aggregate_functions/test_aggregate_all_functions.groovy b/regression-test/suites/query_p0/sql_functions/aggregate_functions/test_aggregate_all_functions.groovy index cdab9472e27dbd..86a951d7ac33c6 100644 --- a/regression-test/suites/query_p0/sql_functions/aggregate_functions/test_aggregate_all_functions.groovy +++ b/regression-test/suites/query_p0/sql_functions/aggregate_functions/test_aggregate_all_functions.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_aggregate_all_functions", "arrow_flight_sql") { +suite("test_aggregate_all_functions") { sql "set batch_size = 4096" diff --git a/regression-test/suites/query_p0/sql_functions/case_function/test_case_function_null.groovy b/regression-test/suites/query_p0/sql_functions/case_function/test_case_function_null.groovy index a91c86b5f48908..41ef1625080806 100644 --- a/regression-test/suites/query_p0/sql_functions/case_function/test_case_function_null.groovy +++ b/regression-test/suites/query_p0/sql_functions/case_function/test_case_function_null.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_case_function_null", "query,p0,arrow_flight_sql") { +suite("test_case_function_null", "query,p0") { sql """ drop table if exists case_null0 """ sql """ create table case_null0 ( `c0` decimalv3(17, 1) NULL, diff --git a/regression-test/suites/query_p0/sql_functions/hash_functions/test_hash_function.groovy b/regression-test/suites/query_p0/sql_functions/hash_functions/test_hash_function.groovy index d547e9fb287d71..590ccd10821f61 100644 --- a/regression-test/suites/query_p0/sql_functions/hash_functions/test_hash_function.groovy +++ b/regression-test/suites/query_p0/sql_functions/hash_functions/test_hash_function.groovy @@ -14,7 +14,7 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations // under the License. -suite("test_hash_function", "arrow_flight_sql") { +suite("test_hash_function") { sql "set batch_size = 4096;" sql "set enable_profile = true;" diff --git a/regression-test/suites/query_p0/sql_functions/ip_functions/test_ip_functions.groovy b/regression-test/suites/query_p0/sql_functions/ip_functions/test_ip_functions.groovy index 5373217503a018..03e9788a58a3b8 100644 --- a/regression-test/suites/query_p0/sql_functions/ip_functions/test_ip_functions.groovy +++ b/regression-test/suites/query_p0/sql_functions/ip_functions/test_ip_functions.groovy @@ -14,7 +14,7 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations // under the License. -suite("test_ip_functions", "arrow_flight_sql") { +suite("test_ip_functions") { sql "set batch_size = 4096;" qt_sql "SELECT ipv4_num_to_string(-1);" diff --git a/regression-test/suites/query_p0/sql_functions/json_function/test_query_json_insert.groovy b/regression-test/suites/query_p0/sql_functions/json_function/test_query_json_insert.groovy index b5865034538a11..c885e3ae3431f3 100644 --- a/regression-test/suites/query_p0/sql_functions/json_function/test_query_json_insert.groovy +++ b/regression-test/suites/query_p0/sql_functions/json_function/test_query_json_insert.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_query_json_insert", "query,arrow_flight_sql") { +suite("test_query_json_insert", "query") { qt_sql "select json_insert('{\"a\": 1, \"b\": [2, 3]}', '\$', null);" qt_sql "select json_insert('{\"k\": [1, 2]}', '\$.k[0]', null, '\$.[1]', null);" def tableName = "test_query_json_insert" diff --git a/regression-test/suites/query_p0/sql_functions/json_functions/test_json_function.groovy b/regression-test/suites/query_p0/sql_functions/json_functions/test_json_function.groovy index 4bd88bf131e727..aa0deec96f46a2 100644 --- a/regression-test/suites/query_p0/sql_functions/json_functions/test_json_function.groovy +++ b/regression-test/suites/query_p0/sql_functions/json_functions/test_json_function.groovy @@ -14,7 +14,7 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations // under the License. -suite("test_json_function", "arrow_flight_sql") { +suite("test_json_function") { sql "set batch_size = 4096;" qt_sql "SELECT get_json_double('{\"k1\":1.3, \"k2\":\"2\"}', \"\$.k1\");" diff --git a/regression-test/suites/query_p0/sql_functions/math_functions/test_conv.groovy b/regression-test/suites/query_p0/sql_functions/math_functions/test_conv.groovy index 3a74abfe9c8b22..6c4867174d11ac 100644 --- a/regression-test/suites/query_p0/sql_functions/math_functions/test_conv.groovy +++ b/regression-test/suites/query_p0/sql_functions/math_functions/test_conv.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_conv", "arrow_flight_sql") { +suite("test_conv") { qt_select "SELECT CONV(15,10,2)" sql """ drop table if exists test_conv; """ diff --git a/regression-test/suites/query_p0/sql_functions/search_functions/test_multi_string_search.groovy b/regression-test/suites/query_p0/sql_functions/search_functions/test_multi_string_search.groovy index f1487d283dfcdf..061665d3b9da6e 100644 --- a/regression-test/suites/query_p0/sql_functions/search_functions/test_multi_string_search.groovy +++ b/regression-test/suites/query_p0/sql_functions/search_functions/test_multi_string_search.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_multi_string_search", "arrow_flight_sql") { +suite("test_multi_string_search") { def table_name = "test_multi_string_search_strings" sql """ DROP TABLE IF EXISTS ${table_name} """ diff --git a/regression-test/suites/query_p0/sql_functions/spatial_functions/test_gis_function.groovy b/regression-test/suites/query_p0/sql_functions/spatial_functions/test_gis_function.groovy index f76cb44cb4ad4b..e98e11ba7e6888 100644 --- a/regression-test/suites/query_p0/sql_functions/spatial_functions/test_gis_function.groovy +++ b/regression-test/suites/query_p0/sql_functions/spatial_functions/test_gis_function.groovy @@ -14,7 +14,7 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations // under the License. -suite("test_gis_function", "arrow_flight_sql") { +suite("test_gis_function") { sql "set batch_size = 4096;" qt_sql "SELECT ST_AsText(ST_Point(24.7, 56.7));" diff --git a/regression-test/suites/query_p0/sql_functions/string_functions/test_string_function.groovy b/regression-test/suites/query_p0/sql_functions/string_functions/test_string_function.groovy index 6e18fb57eeb4cf..f5d32653c818b5 100644 --- a/regression-test/suites/query_p0/sql_functions/string_functions/test_string_function.groovy +++ b/regression-test/suites/query_p0/sql_functions/string_functions/test_string_function.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_string_function", "arrow_flight_sql") { +suite("test_string_function") { sql "set batch_size = 4096;" qt_sql "select elt(0, \"hello\", \"doris\");" diff --git a/regression-test/suites/query_p0/sql_functions/table_function/explode_split.groovy b/regression-test/suites/query_p0/sql_functions/table_function/explode_split.groovy index 53db931c03bb03..b7dd4d640799fb 100644 --- a/regression-test/suites/query_p0/sql_functions/table_function/explode_split.groovy +++ b/regression-test/suites/query_p0/sql_functions/table_function/explode_split.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("explode_split", "arrow_flight_sql") { +suite("explode_split") { def tableName = "test_lv_str" sql """ DROP TABLE IF EXISTS ${tableName} """ diff --git a/regression-test/suites/query_p0/sql_functions/test_alias_function.groovy b/regression-test/suites/query_p0/sql_functions/test_alias_function.groovy index 095ec89e220f1b..8e0e94fa2df805 100644 --- a/regression-test/suites/query_p0/sql_functions/test_alias_function.groovy +++ b/regression-test/suites/query_p0/sql_functions/test_alias_function.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite('test_alias_function', "arrow_flight_sql") { +suite('test_alias_function') { sql ''' CREATE ALIAS FUNCTION IF NOT EXISTS f1(DATETIMEV2(3), INT) with PARAMETER (datetime1, int1) as date_trunc(days_sub(datetime1, int1), 'day')''' diff --git a/regression-test/suites/query_p0/sql_functions/test_predicate.groovy b/regression-test/suites/query_p0/sql_functions/test_predicate.groovy index 6cca6b62c9960b..20b3c179ad5c01 100644 --- a/regression-test/suites/query_p0/sql_functions/test_predicate.groovy +++ b/regression-test/suites/query_p0/sql_functions/test_predicate.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_predicate", "arrow_flight_sql") { +suite("test_predicate") { sql """drop table if exists t1;""" sql """ create table t1 ( diff --git a/regression-test/suites/query_p0/sql_functions/width_bucket_fuctions/test_width_bucket_function.groovy b/regression-test/suites/query_p0/sql_functions/width_bucket_fuctions/test_width_bucket_function.groovy index 1a455da92446f8..d0862a580ca600 100644 --- a/regression-test/suites/query_p0/sql_functions/width_bucket_fuctions/test_width_bucket_function.groovy +++ b/regression-test/suites/query_p0/sql_functions/width_bucket_fuctions/test_width_bucket_function.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_width_bucket_function", "arrow_flight_sql") { +suite("test_width_bucket_function") { qt_sql "select width_bucket(1, 2, 3, 2)" qt_sql "select width_bucket(null, 2, 3, 2)" qt_sql "select width_bucket(6, 2, 6, 4)" diff --git a/regression-test/suites/query_p0/subquery/test_subquery2.groovy b/regression-test/suites/query_p0/subquery/test_subquery2.groovy index a14a44fa152b97..e572459cc72fe3 100644 --- a/regression-test/suites/query_p0/subquery/test_subquery2.groovy +++ b/regression-test/suites/query_p0/subquery/test_subquery2.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_subquery2", "arrow_flight_sql") { +suite("test_subquery2") { sql """DROP TABLE IF EXISTS subquerytest2""" diff --git a/regression-test/suites/query_p0/test_data_type_marks.groovy b/regression-test/suites/query_p0/test_data_type_marks.groovy index 51fb7c9614e488..79803d98723313 100644 --- a/regression-test/suites/query_p0/test_data_type_marks.groovy +++ b/regression-test/suites/query_p0/test_data_type_marks.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_data_type_marks", "arrow_flight_sql") { +suite("test_data_type_marks") { def tbName = "org" sql "DROP TABLE IF EXISTS ${tbName}" sql """ diff --git a/regression-test/suites/query_p0/test_dict_with_null.groovy b/regression-test/suites/query_p0/test_dict_with_null.groovy index 83d253fa4d1b04..b3738bb68aa1ba 100644 --- a/regression-test/suites/query_p0/test_dict_with_null.groovy +++ b/regression-test/suites/query_p0/test_dict_with_null.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("dict_with_null", "query,arrow_flight_sql") { +suite("dict_with_null", "query") { def tableName = "test_dict_with_null" sql "DROP TABLE IF EXISTS ${tableName}" sql """ diff --git a/regression-test/suites/query_p0/test_orderby_nullliteral.groovy b/regression-test/suites/query_p0/test_orderby_nullliteral.groovy index e806060c8bcb1c..fe11c778af0b98 100644 --- a/regression-test/suites/query_p0/test_orderby_nullliteral.groovy +++ b/regression-test/suites/query_p0/test_orderby_nullliteral.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("orderby_nullliteral", "query,arrow_flight_sql") { +suite("orderby_nullliteral", "query") { def tableName = "test_orderby_nullliteral" sql "DROP TABLE IF EXISTS ${tableName}" diff --git a/regression-test/suites/query_p0/test_select_constant.groovy b/regression-test/suites/query_p0/test_select_constant.groovy index 68f0a28a20e853..6015e19576c690 100644 --- a/regression-test/suites/query_p0/test_select_constant.groovy +++ b/regression-test/suites/query_p0/test_select_constant.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_select_constant", "arrow_flight_sql") { +suite("test_select_constant") { qt_select1 'select 100, "test", date("2021-01-02");' qt_select_geo1 'SELECT ST_AsText(ST_Point(123.12345678901234567890,89.1234567890));' } diff --git a/regression-test/suites/query_p0/test_select_with_predicate_like.groovy b/regression-test/suites/query_p0/test_select_with_predicate_like.groovy index 0d01f1b958a11c..9491c4271ca530 100644 --- a/regression-test/suites/query_p0/test_select_with_predicate_like.groovy +++ b/regression-test/suites/query_p0/test_select_with_predicate_like.groovy @@ -14,7 +14,7 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations // under the License. -suite("test_select_with_predicate_like", "arrow_flight_sql") { +suite("test_select_with_predicate_like") { def tables=["test_basic_agg"] for (String table in tables) { diff --git a/regression-test/suites/query_p0/test_select_with_predicate_prune.groovy b/regression-test/suites/query_p0/test_select_with_predicate_prune.groovy index ccd1b9160fb148..768e04b4c327b5 100644 --- a/regression-test/suites/query_p0/test_select_with_predicate_prune.groovy +++ b/regression-test/suites/query_p0/test_select_with_predicate_prune.groovy @@ -14,7 +14,7 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations // under the License. -suite("test_select_with_predicate_prune", "arrow_flight_sql") { +suite("test_select_with_predicate_prune") { sql """ drop table if exists `test_select_with_predicate_prune`; """ diff --git a/regression-test/suites/query_p0/type_inference/test_largeint.groovy b/regression-test/suites/query_p0/type_inference/test_largeint.groovy index 161359cfa97e72..d5cbfa4b479838 100644 --- a/regression-test/suites/query_p0/type_inference/test_largeint.groovy +++ b/regression-test/suites/query_p0/type_inference/test_largeint.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_largeint", "arrow_flight_sql") { +suite("test_largeint") { def tbName = "test_largeint" sql "DROP TABLE IF EXISTS ${tbName}" sql """ diff --git a/regression-test/suites/query_p0/with/test_with_and_two_phase_agg.groovy b/regression-test/suites/query_p0/with/test_with_and_two_phase_agg.groovy index d563ef1630517d..99164a999c557e 100644 --- a/regression-test/suites/query_p0/with/test_with_and_two_phase_agg.groovy +++ b/regression-test/suites/query_p0/with/test_with_and_two_phase_agg.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_with_and_two_phase_agg", "arrow_flight_sql") { +suite("test_with_and_two_phase_agg") { def tableName = "test_with_and_two_phase_agg_table" sql """ DROP TABLE IF EXISTS ${tableName} """ sql """ From 805520bfe4d90965bbd844945bd302eff38dc8f3 Mon Sep 17 00:00:00 2001 From: Pxl Date: Wed, 27 Nov 2024 11:22:57 +0800 Subject: [PATCH 033/399] [Bug](join) fix single null eq get wrong result (#44613) introduced by https://github.com/apache/doris/pull/42398 In order to make the null keys equal when using single null eq, all null keys need to be set to default value. --- be/src/pipeline/exec/hashjoin_build_sink.h | 5 ++++- .../exec/join/process_hash_table_probe_impl.h | 4 ++++ .../data/nereids_p0/join/test_mark_join.out | 7 +++++++ .../data/query_p0/join/test_join.out | 6 ++++++ .../nereids_p0/join/test_mark_join.groovy | 21 +++++++++++++++++++ .../suites/query_p0/join/test_join.groovy | 13 +++++++++++- 6 files changed, 54 insertions(+), 2 deletions(-) diff --git a/be/src/pipeline/exec/hashjoin_build_sink.h b/be/src/pipeline/exec/hashjoin_build_sink.h index d905afa27582fb..cc78e6a769f3c3 100644 --- a/be/src/pipeline/exec/hashjoin_build_sink.h +++ b/be/src/pipeline/exec/hashjoin_build_sink.h @@ -197,7 +197,10 @@ struct ProcessHashTableBuild { SCOPED_TIMER(_parent->_build_table_insert_timer); hash_table_ctx.hash_table->template prepare_build(_rows, _batch_size, *has_null_key); - + // In order to make the null keys equal when using single null eq, all null keys need to be set to default value. + if (_build_raw_ptrs.size() == 1 && null_map) { + _build_raw_ptrs[0]->assume_mutable()->replace_column_null_data(null_map->data()); + } hash_table_ctx.init_serialized_keys(_build_raw_ptrs, _rows, null_map ? null_map->data() : nullptr, true, true, hash_table_ctx.hash_table->get_bucket_size()); diff --git a/be/src/pipeline/exec/join/process_hash_table_probe_impl.h b/be/src/pipeline/exec/join/process_hash_table_probe_impl.h index 05cd3d7d9e0590..fc1153b3419d3e 100644 --- a/be/src/pipeline/exec/join/process_hash_table_probe_impl.h +++ b/be/src/pipeline/exec/join/process_hash_table_probe_impl.h @@ -173,6 +173,10 @@ typename HashTableType::State ProcessHashTableProbe::_init_probe_sid if (!_parent->_ready_probe) { _parent->_ready_probe = true; hash_table_ctx.reset(); + // In order to make the null keys equal when using single null eq, all null keys need to be set to default value. + if (_parent->_probe_columns.size() == 1 && null_map) { + _parent->_probe_columns[0]->assume_mutable()->replace_column_null_data(null_map); + } hash_table_ctx.init_serialized_keys(_parent->_probe_columns, probe_rows, null_map, true, false, hash_table_ctx.hash_table->get_bucket_size()); hash_table_ctx.hash_table->pre_build_idxs(hash_table_ctx.bucket_nums, diff --git a/regression-test/data/nereids_p0/join/test_mark_join.out b/regression-test/data/nereids_p0/join/test_mark_join.out index 59fc7d651ad5e9..a603555667416b 100644 --- a/regression-test/data/nereids_p0/join/test_mark_join.out +++ b/regression-test/data/nereids_p0/join/test_mark_join.out @@ -53,3 +53,10 @@ -- !mark_join_null_conjunct -- \N +-- !mark_join8 -- +1 1 1 false +2 2 2 false +3 \N \N \N +3 \N 3 \N +4 \N 4 true + diff --git a/regression-test/data/query_p0/join/test_join.out b/regression-test/data/query_p0/join/test_join.out index 3814530e0c634d..7cbaf4f77508e8 100644 --- a/regression-test/data/query_p0/join/test_join.out +++ b/regression-test/data/query_p0/join/test_join.out @@ -3243,3 +3243,9 @@ false true true false false -- !sql -- 4 +-- !sql -- +2 \N + +-- !sql -- +2 \N + diff --git a/regression-test/suites/nereids_p0/join/test_mark_join.groovy b/regression-test/suites/nereids_p0/join/test_mark_join.groovy index 3b07c357b5dc04..034900f81acb08 100644 --- a/regression-test/suites/nereids_p0/join/test_mark_join.groovy +++ b/regression-test/suites/nereids_p0/join/test_mark_join.groovy @@ -142,4 +142,25 @@ suite("test_mark_join", "nereids_p0") { """ qt_mark_join_null_conjunct """select null in ( select k1 from test_mark_join_t1);""" + + qt_mark_join8 """ + select + k1, + k2, + k3, + k1 not in ( + select + test_mark_join_t2.k2 + from + test_mark_join_t2 + where + test_mark_join_t2.k3 <=> test_mark_join_t1.k3 + ) vv + from + test_mark_join_t1 + order by + 1, + 2, + 3; + """ } diff --git a/regression-test/suites/query_p0/join/test_join.groovy b/regression-test/suites/query_p0/join/test_join.groovy index e75878cb032aa3..fa83fddbd41482 100644 --- a/regression-test/suites/query_p0/join/test_join.groovy +++ b/regression-test/suites/query_p0/join/test_join.groovy @@ -720,7 +720,7 @@ suite("test_join", "query,p0") { } qt_left_anti_join_with_other_pred "select b.k1 from baseall b left anti join test t on b.k1 = t.k1 and 1 = 2 order by b.k1" - + // null not in (1,2,3,null) = true qt_left_anti_join_null_1 "select b.k1 from baseall b left anti join test t on b.k1 = t.k1 order by b.k1" qt_left_anti_join_null_2 "select b.k1 from baseall b left anti join test_join_empty_view t on b.k1 = t.k1 order by b.k1" @@ -931,6 +931,7 @@ suite("test_join", "query,p0") { // https://github.com/apache/doris/issues/4210 qt_join_bug3"""select * from baseall t1 where k1 = (select min(k1) from test t2 where t2.k1 = t1.k1 and t2.k2=t1.k2) order by k1""" + // null not in (1,2,3) = false qt_join_bug4"""select b.k1 from baseall b where b.k1 not in( select k1 from baseall where k1 is not null )""" @@ -1330,4 +1331,14 @@ suite("test_join", "query,p0") { qt_sql """ select /*+SET_VAR(batch_size=1, disable_join_reorder=true)*/ count(DISTINCT dcqewrt.engineer) as active_person_count from tbl1 dcqewrt left join [broadcast] tbl2 dd on dd.data_dt = dcqewrt.data_dt; """ sql """ DROP TABLE IF EXISTS tbl2; """ sql """ DROP TABLE IF EXISTS tbl1; """ + + + sql "drop table if exists t01;" + sql "drop table if exists t02;" + sql"""create table t01 (id int, a varchar(10)) properties ("replication_num" = "1");""" + sql"""create table t02 (id int, b varchar(10)) properties ("replication_num" = "1");""" + sql"insert into t01 values (1, 'a'), (2, null), (3, 'c');" + sql"insert into t02 values (1, 'b');" + qt_sql"select * from t01 where (not like (a, 'a%')) <=> 'b';" + qt_sql"select * from t01 where (not like (a, 'a%')) <=> (select max(b) from t02); " } From d592e279c1058de37994135a4cc20fd35698a9c1 Mon Sep 17 00:00:00 2001 From: 924060929 Date: Wed, 27 Nov 2024 12:18:56 +0800 Subject: [PATCH 034/399] [fix](sql_cache) fix sql cache result wrong of from_unixtime(col, 'yyyy-MM-dd HH:mm:ss') (#44631) fix sql cache result wrong of from_unixtime(col, 'yyyy-MM-dd HH:mm:ss') which introduced by #33262 the wrong result: is `yyyy-MM-dd HH:mm:ss` --- .../rules/expression/rules/SupportJavaDateFormatter.java | 3 ++- .../executable/DateTimeExtractAndTransform.java | 9 +++++++++ .../main/java/org/apache/doris/qe/SessionVariable.java | 2 +- .../nereids_p0/cache/parse_sql_from_sql_cache.groovy | 7 +++++++ 4 files changed, 19 insertions(+), 2 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SupportJavaDateFormatter.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SupportJavaDateFormatter.java index 27b929a2b9f865..f5b442a3989cd5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SupportJavaDateFormatter.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SupportJavaDateFormatter.java @@ -76,7 +76,8 @@ private static Expression translateJavaFormatter(Expression function, int format return function; } - private static Expression translateJavaFormatter(Expression formatterExpr) { + /** translateJavaFormatter */ + public static Expression translateJavaFormatter(Expression formatterExpr) { if (formatterExpr.isLiteral() && formatterExpr.getDataType().isStringLikeType()) { Literal literal = (Literal) formatterExpr; String originFormatter = literal.getStringValue(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/executable/DateTimeExtractAndTransform.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/executable/DateTimeExtractAndTransform.java index 440f93cac598a2..c908271531884f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/executable/DateTimeExtractAndTransform.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/executable/DateTimeExtractAndTransform.java @@ -20,6 +20,7 @@ import org.apache.doris.catalog.ScalarType; import org.apache.doris.catalog.Type; import org.apache.doris.nereids.exceptions.AnalysisException; +import org.apache.doris.nereids.rules.expression.rules.SupportJavaDateFormatter; import org.apache.doris.nereids.trees.expressions.ExecFunction; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.literal.BigIntLiteral; @@ -293,12 +294,14 @@ private static LocalDateTime firstDayOfWeek(LocalDateTime dateTime) { */ @ExecFunction(name = "date_format") public static Expression dateFormat(DateLiteral date, StringLikeLiteral format) { + format = (StringLikeLiteral) SupportJavaDateFormatter.translateJavaFormatter(format); return new VarcharLiteral(DateUtils.formatBuilder(format.getValue()).toFormatter().format( java.time.LocalDate.of(((int) date.getYear()), ((int) date.getMonth()), ((int) date.getDay())))); } @ExecFunction(name = "date_format") public static Expression dateFormat(DateTimeLiteral date, StringLikeLiteral format) { + format = (StringLikeLiteral) SupportJavaDateFormatter.translateJavaFormatter(format); return new VarcharLiteral(DateUtils.formatBuilder(format.getValue()).toFormatter().format( java.time.LocalDateTime.of(((int) date.getYear()), ((int) date.getMonth()), ((int) date.getDay()), ((int) date.getHour()), ((int) date.getMinute()), ((int) date.getSecond())))); @@ -306,12 +309,14 @@ public static Expression dateFormat(DateTimeLiteral date, StringLikeLiteral form @ExecFunction(name = "date_format") public static Expression dateFormat(DateV2Literal date, StringLikeLiteral format) { + format = (StringLikeLiteral) SupportJavaDateFormatter.translateJavaFormatter(format); return new VarcharLiteral(DateUtils.formatBuilder(format.getValue()).toFormatter().format( java.time.LocalDate.of(((int) date.getYear()), ((int) date.getMonth()), ((int) date.getDay())))); } @ExecFunction(name = "date_format") public static Expression dateFormat(DateTimeV2Literal date, StringLikeLiteral format) { + format = (StringLikeLiteral) SupportJavaDateFormatter.translateJavaFormatter(format); return new VarcharLiteral(DateUtils.formatBuilder(format.getValue()).toFormatter().format( java.time.LocalDateTime.of(((int) date.getYear()), ((int) date.getMonth()), ((int) date.getDay()), ((int) date.getHour()), ((int) date.getMinute()), ((int) date.getSecond())))); @@ -479,6 +484,8 @@ public static Expression fromUnixTime(BigIntLiteral second) { */ @ExecFunction(name = "from_unixtime") public static Expression fromUnixTime(BigIntLiteral second, StringLikeLiteral format) { + format = (StringLikeLiteral) SupportJavaDateFormatter.translateJavaFormatter(format); + // 32536771199L is max valid timestamp of mysql from_unix_time if (second.getValue() < 0 || second.getValue() > 32536771199L) { return new NullLiteral(VarcharType.SYSTEM_DEFAULT); @@ -531,6 +538,7 @@ public static Expression unixTimestamp(DateTimeV2Literal date) { */ @ExecFunction(name = "unix_timestamp") public static Expression unixTimestamp(StringLikeLiteral date, StringLikeLiteral format) { + format = (StringLikeLiteral) SupportJavaDateFormatter.translateJavaFormatter(format); DateTimeFormatter formatter = DateUtils.formatBuilder(format.getValue()).toFormatter(); LocalDateTime dateObj; try { @@ -616,6 +624,7 @@ public static Expression makeDate(IntegerLiteral year, IntegerLiteral dayOfYear) */ @ExecFunction(name = "str_to_date") public static Expression strToDate(StringLikeLiteral str, StringLikeLiteral format) { + format = (StringLikeLiteral) SupportJavaDateFormatter.translateJavaFormatter(format); if (org.apache.doris.analysis.DateLiteral.hasTimePart(format.getStringValue())) { DataType returnType = DataType.fromCatalogType(ScalarType.getDefaultDateType(Type.DATETIME)); if (returnType instanceof DateTimeV2Type) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java index 2c09ee73f57168..71b746c7907262 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java @@ -1011,7 +1011,7 @@ public enum IgnoreSplitType { @VariableMgr.VarAttr(name = ENABLE_ODBC_TRANSCATION) public boolean enableOdbcTransaction = false; - @VariableMgr.VarAttr(name = ENABLE_SQL_CACHE) + @VariableMgr.VarAttr(name = ENABLE_SQL_CACHE, fuzzy = true) public boolean enableSqlCache = false; @VariableMgr.VarAttr(name = ENABLE_QUERY_CACHE) diff --git a/regression-test/suites/nereids_p0/cache/parse_sql_from_sql_cache.groovy b/regression-test/suites/nereids_p0/cache/parse_sql_from_sql_cache.groovy index 6aeeeed0ead602..54ab702888888d 100644 --- a/regression-test/suites/nereids_p0/cache/parse_sql_from_sql_cache.groovy +++ b/regression-test/suites/nereids_p0/cache/parse_sql_from_sql_cache.groovy @@ -820,6 +820,13 @@ suite("parse_sql_from_sql_cache") { assertTrue(profileString.contains("Is Cached: Yes")) } } + }), + extraThread("sql_cache_with_date_format", { + sql "set enable_sql_cache=true" + for (def i in 0..3) { + def result = sql "select FROM_UNIXTIME(UNIX_TIMESTAMP(), 'yyyy-MM-dd HH:mm:ss')" + assertNotEquals("yyyy-MM-dd HH:mm:ss", result[0][0]) + } }) ).get() } From f6ff621a416224b705e35dc456c3d4a08df40fb2 Mon Sep 17 00:00:00 2001 From: zzzxl Date: Wed, 27 Nov 2024 15:59:10 +0800 Subject: [PATCH 035/399] [fix](inverted index) Modify Error Handling for File Open Failure (#44551) Problem Summary: Modify Error Handling for File Open Failure --- be/src/olap/compaction.cpp | 26 +- ...ip_index_compaction_fault_injection.groovy | 240 ++++++++++++++++++ 2 files changed, 261 insertions(+), 5 deletions(-) create mode 100644 regression-test/suites/fault_injection_p0/test_skip_index_compaction_fault_injection.groovy diff --git a/be/src/olap/compaction.cpp b/be/src/olap/compaction.cpp index 68ed0322a9ec92..e71e1862dc8dbb 100644 --- a/be/src/olap/compaction.cpp +++ b/be/src/olap/compaction.cpp @@ -662,15 +662,28 @@ Status Compaction::do_inverted_index_compaction() { try { std::vector> src_idx_dirs(src_segment_num); for (int src_segment_id = 0; src_segment_id < src_segment_num; src_segment_id++) { - src_idx_dirs[src_segment_id] = - DORIS_TRY(inverted_index_file_readers[src_segment_id]->open(index_meta)); + auto res = inverted_index_file_readers[src_segment_id]->open(index_meta); + DBUG_EXECUTE_IF("Compaction::open_inverted_index_file_reader", { + res = ResultError(Status::Error( + "debug point: Compaction::open_index_file_reader error")); + }) + if (!res.has_value()) { + throw Exception(ErrorCode::INVERTED_INDEX_COMPACTION_ERROR, res.error().msg()); + } + src_idx_dirs[src_segment_id] = std::move(res.value()); } for (int dest_segment_id = 0; dest_segment_id < dest_segment_num; dest_segment_id++) { - auto dest_dir = - DORIS_TRY(inverted_index_file_writers[dest_segment_id]->open(index_meta)); + auto res = inverted_index_file_writers[dest_segment_id]->open(index_meta); + DBUG_EXECUTE_IF("Compaction::open_inverted_index_file_writer", { + res = ResultError(Status::Error( + "debug point: Compaction::open_inverted_index_file_writer error")); + }) + if (!res.has_value()) { + throw Exception(ErrorCode::INVERTED_INDEX_COMPACTION_ERROR, res.error().msg()); + } // Destination directories in dest_index_dirs do not need to be deconstructed, // but their lifecycle must be managed by inverted_index_file_writers. - dest_index_dirs[dest_segment_id] = dest_dir.get(); + dest_index_dirs[dest_segment_id] = res.value().get(); } auto st = compact_column(index_meta->index_id(), src_idx_dirs, dest_index_dirs, index_tmp_path.native(), trans_vec, dest_segment_num_rows); @@ -681,6 +694,9 @@ Status Compaction::do_inverted_index_compaction() { } catch (CLuceneError& e) { error_handler(index_meta->index_id(), column_uniq_id); status = Status::Error(e.what()); + } catch (const Exception& e) { + error_handler(index_meta->index_id(), column_uniq_id); + status = Status::Error(e.what()); } } diff --git a/regression-test/suites/fault_injection_p0/test_skip_index_compaction_fault_injection.groovy b/regression-test/suites/fault_injection_p0/test_skip_index_compaction_fault_injection.groovy new file mode 100644 index 00000000000000..cdc2bdda41716d --- /dev/null +++ b/regression-test/suites/fault_injection_p0/test_skip_index_compaction_fault_injection.groovy @@ -0,0 +1,240 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +import org.codehaus.groovy.runtime.IOGroovyMethods + +suite("test_skip_index_compaction_fault_injection", "nonConcurrent") { + def isCloudMode = isCloudMode() + def tableName1 = "test_skip_index_compaction_fault_injection_1" + def tableName2 = "test_skip_index_compaction_fault_injection_2" + def backendId_to_backendIP = [:] + def backendId_to_backendHttpPort = [:] + getBackendIpHttpPort(backendId_to_backendIP, backendId_to_backendHttpPort); + + sql "DROP TABLE IF EXISTS ${tableName1}" + sql """ + CREATE TABLE ${tableName1} ( + `@timestamp` int(11) NULL COMMENT "", + `clientip` varchar(20) NULL COMMENT "", + `request` text NULL COMMENT "", + `status` int(11) NULL COMMENT "", + `size` int(11) NULL COMMENT "", + INDEX clientip_idx (`clientip`) USING INVERTED COMMENT '', + INDEX request_idx (`request`) USING INVERTED PROPERTIES("parser" = "english", "support_phrase" = "true") COMMENT '' + ) ENGINE=OLAP + DUPLICATE KEY(`@timestamp`) + COMMENT "OLAP" + DISTRIBUTED BY RANDOM BUCKETS 1 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1", + "disable_auto_compaction" = "true", + "inverted_index_storage_format" = "V1" + ); + """ + + sql "DROP TABLE IF EXISTS ${tableName2}" + sql """ + CREATE TABLE ${tableName2} ( + `@timestamp` int(11) NULL COMMENT "", + `clientip` varchar(20) NULL COMMENT "", + `request` text NULL COMMENT "", + `status` int(11) NULL COMMENT "", + `size` int(11) NULL COMMENT "", + INDEX clientip_idx (`clientip`) USING INVERTED COMMENT '', + INDEX request_idx (`request`) USING INVERTED PROPERTIES("parser" = "english", "support_phrase" = "true") COMMENT '' + ) ENGINE=OLAP + DUPLICATE KEY(`@timestamp`) + COMMENT "OLAP" + DISTRIBUTED BY RANDOM BUCKETS 1 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1", + "disable_auto_compaction" = "true", + "inverted_index_storage_format" = "V2" + ); + """ + + boolean disableAutoCompaction = false + + def set_be_config = { key, value -> + for (String backend_id: backendId_to_backendIP.keySet()) { + def (code, out, err) = update_be_config(backendId_to_backendIP.get(backend_id), backendId_to_backendHttpPort.get(backend_id), key, value) + logger.info("update config: code=" + code + ", out=" + out + ", err=" + err) + } + } + + def trigger_full_compaction_on_tablets = { tablets -> + for (def tablet : tablets) { + String tablet_id = tablet.TabletId + String backend_id = tablet.BackendId + int times = 1 + + String compactionStatus; + do{ + def (code, out, err) = be_run_full_compaction(backendId_to_backendIP.get(backend_id), backendId_to_backendHttpPort.get(backend_id), tablet_id) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + ++times + sleep(2000) + compactionStatus = parseJson(out.trim()).status.toLowerCase(); + } while (compactionStatus!="success" && times<=10 && compactionStatus!="e-6010") + + + if (compactionStatus == "fail") { + assertEquals(disableAutoCompaction, false) + logger.info("Compaction was done automatically!") + } + if (disableAutoCompaction && compactionStatus!="e-6010") { + assertEquals("success", compactionStatus) + } + } + } + + def wait_full_compaction_done = { tablets -> + for (def tablet in tablets) { + boolean running = true + do { + Thread.sleep(1000) + String tablet_id = tablet.TabletId + String backend_id = tablet.BackendId + def (code, out, err) = be_get_compaction_status(backendId_to_backendIP.get(backend_id), backendId_to_backendHttpPort.get(backend_id), tablet_id) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } while (running) + } + } + + def get_rowset_count = { tablets -> + int rowsetCount = 0 + for (def tablet in tablets) { + def (code, out, err) = curl("GET", tablet.CompactionStatus) + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + rowsetCount +=((List) tabletJson.rowsets).size() + } + return rowsetCount + } + + def check_config = { String key, String value -> + for (String backend_id: backendId_to_backendIP.keySet()) { + def (code, out, err) = show_be_config(backendId_to_backendIP.get(backend_id), backendId_to_backendHttpPort.get(backend_id)) + logger.info("Show config: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def configList = parseJson(out.trim()) + assert configList instanceof List + for (Object ele in (List) configList) { + assert ele instanceof List + if (((List) ele)[0] == key) { + assertEquals(value, ((List) ele)[2]) + } + } + } + } + + def run_test = { tableName -> + sql """ INSERT INTO ${tableName} VALUES (1, "40.135.0.0", "GET /images/hm_bg.jpg HTTP/1.0", 1, 2); """ + sql """ INSERT INTO ${tableName} VALUES (2, "40.135.0.0", "GET /images/hm_bg.jpg HTTP/1.0", 1, 2); """ + sql """ INSERT INTO ${tableName} VALUES (3, "40.135.0.0", "GET /images/hm_bg.jpg HTTP/1.0", 1, 2); """ + sql """ INSERT INTO ${tableName} VALUES (4, "40.135.0.0", "GET /images/hm_bg.jpg HTTP/1.0", 1, 2); """ + sql """ INSERT INTO ${tableName} VALUES (5, "40.135.0.0", "GET /images/hm_bg.jpg HTTP/1.0", 1, 2); """ + sql """ INSERT INTO ${tableName} VALUES (6, "40.135.0.0", "GET /images/hm_bg.jpg HTTP/1.0", 1, 2); """ + sql """ INSERT INTO ${tableName} VALUES (7, "40.135.0.0", "GET /images/hm_bg.jpg HTTP/1.0", 1, 2); """ + sql """ INSERT INTO ${tableName} VALUES (8, "40.135.0.0", "GET /images/hm_bg.jpg HTTP/1.0", 1, 2); """ + sql """ INSERT INTO ${tableName} VALUES (9, "40.135.0.0", "GET /images/hm_bg.jpg HTTP/1.0", 1, 2); """ + sql """ INSERT INTO ${tableName} VALUES (10, "40.135.0.0", "GET /images/hm_bg.jpg HTTP/1.0", 1, 2); """ + + sql "sync" + + def tablets = sql_return_maparray """ show tablets from ${tableName}; """ + logger.info("tablets: {}", tablets) + + int replicaNum = 1 + def dedup_tablets = deduplicate_tablets(tablets) + if (dedup_tablets.size() > 0) { + replicaNum = Math.round(tablets.size() / dedup_tablets.size()) + if (replicaNum != 1 && replicaNum != 3) { + assert(false) + } + } + + int rowsetCount = get_rowset_count.call(tablets); + assert (rowsetCount == 11 * replicaNum) + + // first + trigger_full_compaction_on_tablets.call(tablets) + wait_full_compaction_done.call(tablets) + + rowsetCount = get_rowset_count.call(tablets); + assert (rowsetCount == 11 * replicaNum) + + // second + trigger_full_compaction_on_tablets.call(tablets) + wait_full_compaction_done.call(tablets) + + rowsetCount = get_rowset_count.call(tablets); + if (isCloudMode) { + assert (rowsetCount == (1 + 1) * replicaNum) + } else { + assert (rowsetCount == 1 * replicaNum) + } + } + + boolean invertedIndexCompactionEnable = false + boolean has_update_be_config = false + try { + String backend_id; + backend_id = backendId_to_backendIP.keySet()[0] + def (code, out, err) = show_be_config(backendId_to_backendIP.get(backend_id), backendId_to_backendHttpPort.get(backend_id)) + + logger.info("Show config: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def configList = parseJson(out.trim()) + assert configList instanceof List + + for (Object ele in (List) configList) { + assert ele instanceof List + if (((List) ele)[0] == "inverted_index_compaction_enable") { + invertedIndexCompactionEnable = Boolean.parseBoolean(((List) ele)[2]) + logger.info("inverted_index_compaction_enable: ${((List) ele)[2]}") + } + } + set_be_config.call("inverted_index_compaction_enable", "true") + has_update_be_config = true + check_config.call("inverted_index_compaction_enable", "true"); + + try { + GetDebugPoint().enableDebugPointForAllBEs("Compaction::open_inverted_index_file_reader") + run_test.call(tableName1) + } finally { + GetDebugPoint().disableDebugPointForAllBEs("Compaction::open_inverted_index_file_reader") + } + + try { + GetDebugPoint().enableDebugPointForAllBEs("Compaction::open_inverted_index_file_writer") + run_test.call(tableName2) + } finally { + GetDebugPoint().disableDebugPointForAllBEs("Compaction::open_inverted_index_file_writer") + } + } finally { + if (has_update_be_config) { + set_be_config.call("inverted_index_compaction_enable", invertedIndexCompactionEnable.toString()) + } + } +} \ No newline at end of file From 3ef3c0f428da5c649f04837319d4361999db8aae Mon Sep 17 00:00:00 2001 From: Vallish Pai Date: Wed, 27 Nov 2024 14:35:50 +0530 Subject: [PATCH 036/399] [Enhancement] (nereids)implement showReplicaDistributionCommand in nereids (#44498) Issue Number: close #42782 #42837 --- .../org/apache/doris/nereids/DorisParser.g4 | 8 +- .../apache/doris/analysis/PartitionNames.java | 8 ++ .../apache/doris/catalog/MetadataViewer.java | 2 +- .../nereids/parser/LogicalPlanBuilder.java | 93 +++++++++++++ .../doris/nereids/trees/plans/PlanType.java | 1 + .../nereids/trees/plans/commands/Command.java | 6 +- .../ShowReplicaDistributionCommand.java | 120 ++++++++++++++++ .../commands/info/PartitionNamesInfo.java | 131 ++++++++++++++++++ .../plans/commands/info/TableRefInfo.java | 78 +++++++++++ .../trees/plans/visitor/CommandVisitor.java | 6 + .../org/apache/doris/qe/StmtExecutor.java | 2 +- .../test_database_management_auth.groovy | 5 +- .../test_nereids_show_replica_dist.groovy | 84 +++++++++++ 13 files changed, 535 insertions(+), 9 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowReplicaDistributionCommand.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/PartitionNamesInfo.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/TableRefInfo.java create mode 100644 regression-test/suites/nereids_p0/show/test_nereids_show_replica_dist.groovy diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index d7ed0598d19ddb..f1b17a1b9fb344 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -60,6 +60,7 @@ statementBase | supportedShowStatement #supportedShowStatementAlias | supportedLoadStatement #supportedLoadStatementAlias | supportedRecoverStatement #supportedRecoverStatementAlias + | supportedAdminStatement #supportedAdminStatementAlias | unsupportedStatement #unsupported ; @@ -235,6 +236,7 @@ supportedShowStatement | SHOW CREATE MATERIALIZED VIEW mvName=identifier ON tableName=multipartIdentifier #showCreateMaterializedView | SHOW BACKENDS #showBackends + | SHOW REPLICA DISTRIBUTION FROM baseTableRef #showReplicaDistribution | SHOW FULL? TRIGGERS ((FROM | IN) database=multipartIdentifier)? wildWhere? #showTriggers | SHOW FRONTENDS name=identifier? #showFrontends | SHOW TABLE tableId=INTEGER_VALUE #showTableId @@ -349,7 +351,6 @@ unsupportedShowStatement | SHOW (CLUSTERS | (COMPUTE GROUPS)) #showClusters | SHOW CONVERT_LSC ((FROM | IN) database=multipartIdentifier)? #showConvertLsc | SHOW REPLICA STATUS FROM baseTableRef wildWhere? #showReplicaStatus - | SHOW REPLICA DISTRIBUTION FROM baseTableRef #showREplicaDistribution | SHOW TABLET STORAGE FORMAT VERBOSE? #showTabletStorageFormat | SHOW TABLET DIAGNOSIS tabletId=INTEGER_VALUE #showDiagnoseTablet | SHOW COPY ((FROM | IN) database=multipartIdentifier)? @@ -468,6 +469,10 @@ unsupportedCancelStatement | CANCEL WARM UP JOB wildWhere? #cancelWarmUp ; +supportedAdminStatement + : ADMIN SHOW REPLICA DISTRIBUTION FROM baseTableRef #adminShowReplicaDistribution + ; + supportedRecoverStatement : RECOVER DATABASE name=identifier id=INTEGER_VALUE? (AS alias=identifier)? #recoverDatabase | RECOVER TABLE name=multipartIdentifier @@ -478,7 +483,6 @@ supportedRecoverStatement unsupportedAdminStatement : ADMIN SHOW REPLICA STATUS FROM baseTableRef wildWhere? #adminShowReplicaStatus - | ADMIN SHOW REPLICA DISTRIBUTION FROM baseTableRef #adminShowReplicaDistribution | ADMIN SET REPLICA STATUS PROPERTIES LEFT_PAREN propertyItemList RIGHT_PAREN #adminSetReplicaStatus | ADMIN SET REPLICA VERSION PROPERTIES LEFT_PAREN propertyItemList RIGHT_PAREN #adminSetReplicaVersion | ADMIN REPAIR TABLE baseTableRef #adminRepairTable diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/PartitionNames.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/PartitionNames.java index f82f497176bec2..d49043c4cd5c0b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/PartitionNames.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/PartitionNames.java @@ -81,6 +81,14 @@ public PartitionNames(long partitionCount) { this.count = partitionCount; } + // for nereids + public PartitionNames(boolean isTemp, List partitionNames, boolean isStar, long partitionCount) { + this.partitionNames = partitionNames; + this.isTemp = isTemp; + this.isStar = isStar; + this.count = partitionCount; + } + public List getPartitionNames() { return partitionNames; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/MetadataViewer.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/MetadataViewer.java index fd4a530f4369e1..4da77267857f67 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/MetadataViewer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/MetadataViewer.java @@ -167,7 +167,7 @@ public static List> getTabletDistribution(ShowReplicaDistributionSt return getTabletDistribution(stmt.getDbName(), stmt.getTblName(), stmt.getPartitionNames()); } - private static List> getTabletDistribution( + public static List> getTabletDistribution( String dbName, String tblName, PartitionNames partitionNames) throws DdlException { DecimalFormat df = new DecimalFormat("00.00 %"); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index 6e86b4073f5792..d553d93da9ca0e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -49,6 +49,7 @@ import org.apache.doris.mtmv.MTMVRefreshTriggerInfo; import org.apache.doris.nereids.DorisParser; import org.apache.doris.nereids.DorisParser.AddConstraintContext; +import org.apache.doris.nereids.DorisParser.AdminShowReplicaDistributionContext; import org.apache.doris.nereids.DorisParser.AggClauseContext; import org.apache.doris.nereids.DorisParser.AggStateDataTypeContext; import org.apache.doris.nereids.DorisParser.AliasQueryContext; @@ -64,6 +65,7 @@ import org.apache.doris.nereids.DorisParser.ArrayLiteralContext; import org.apache.doris.nereids.DorisParser.ArrayRangeContext; import org.apache.doris.nereids.DorisParser.ArraySliceContext; +import org.apache.doris.nereids.DorisParser.BaseTableRefContext; import org.apache.doris.nereids.DorisParser.BitOperationContext; import org.apache.doris.nereids.DorisParser.BooleanExpressionContext; import org.apache.doris.nereids.DorisParser.BooleanLiteralContext; @@ -158,6 +160,7 @@ import org.apache.doris.nereids.DorisParser.NamedExpressionContext; import org.apache.doris.nereids.DorisParser.NamedExpressionSeqContext; import org.apache.doris.nereids.DorisParser.NullLiteralContext; +import org.apache.doris.nereids.DorisParser.OptScanParamsContext; import org.apache.doris.nereids.DorisParser.OutFileClauseContext; import org.apache.doris.nereids.DorisParser.ParenthesizedExpressionContext; import org.apache.doris.nereids.DorisParser.PartitionSpecContext; @@ -189,6 +192,7 @@ import org.apache.doris.nereids.DorisParser.RefreshTriggerContext; import org.apache.doris.nereids.DorisParser.RegularQuerySpecificationContext; import org.apache.doris.nereids.DorisParser.RelationContext; +import org.apache.doris.nereids.DorisParser.RelationHintContext; import org.apache.doris.nereids.DorisParser.ReplaceContext; import org.apache.doris.nereids.DorisParser.ResumeMTMVContext; import org.apache.doris.nereids.DorisParser.RollupDefContext; @@ -238,6 +242,7 @@ import org.apache.doris.nereids.DorisParser.ShowPrivilegesContext; import org.apache.doris.nereids.DorisParser.ShowProcContext; import org.apache.doris.nereids.DorisParser.ShowProcedureStatusContext; +import org.apache.doris.nereids.DorisParser.ShowReplicaDistributionContext; import org.apache.doris.nereids.DorisParser.ShowRepositoriesContext; import org.apache.doris.nereids.DorisParser.ShowRolesContext; import org.apache.doris.nereids.DorisParser.ShowSmallFilesContext; @@ -254,6 +259,7 @@ import org.apache.doris.nereids.DorisParser.SingleStatementContext; import org.apache.doris.nereids.DorisParser.SortClauseContext; import org.apache.doris.nereids.DorisParser.SortItemContext; +import org.apache.doris.nereids.DorisParser.SpecifiedPartitionContext; import org.apache.doris.nereids.DorisParser.StarContext; import org.apache.doris.nereids.DorisParser.StatementDefaultContext; import org.apache.doris.nereids.DorisParser.StepPartitionDefContext; @@ -266,7 +272,9 @@ import org.apache.doris.nereids.DorisParser.SystemVariableContext; import org.apache.doris.nereids.DorisParser.TableAliasContext; import org.apache.doris.nereids.DorisParser.TableNameContext; +import org.apache.doris.nereids.DorisParser.TableSnapshotContext; import org.apache.doris.nereids.DorisParser.TableValuedFunctionContext; +import org.apache.doris.nereids.DorisParser.TabletListContext; import org.apache.doris.nereids.DorisParser.TimestampaddContext; import org.apache.doris.nereids.DorisParser.TimestampdiffContext; import org.apache.doris.nereids.DorisParser.TypeConstructorContext; @@ -513,6 +521,7 @@ import org.apache.doris.nereids.trees.plans.commands.ShowPrivilegesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowProcCommand; import org.apache.doris.nereids.trees.plans.commands.ShowProcedureStatusCommand; +import org.apache.doris.nereids.trees.plans.commands.ShowReplicaDistributionCommand; import org.apache.doris.nereids.trees.plans.commands.ShowRepositoriesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowRolesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowSmallFilesCommand; @@ -559,6 +568,7 @@ import org.apache.doris.nereids.trees.plans.commands.info.MTMVPartitionDefinition; import org.apache.doris.nereids.trees.plans.commands.info.PartitionDefinition; import org.apache.doris.nereids.trees.plans.commands.info.PartitionDefinition.MaxValue; +import org.apache.doris.nereids.trees.plans.commands.info.PartitionNamesInfo; import org.apache.doris.nereids.trees.plans.commands.info.PartitionTableInfo; import org.apache.doris.nereids.trees.plans.commands.info.PauseMTMVInfo; import org.apache.doris.nereids.trees.plans.commands.info.RefreshMTMVInfo; @@ -576,6 +586,7 @@ import org.apache.doris.nereids.trees.plans.commands.info.SimpleColumnDefinition; import org.apache.doris.nereids.trees.plans.commands.info.StepPartition; import org.apache.doris.nereids.trees.plans.commands.info.TableNameInfo; +import org.apache.doris.nereids.trees.plans.commands.info.TableRefInfo; import org.apache.doris.nereids.trees.plans.commands.insert.BatchInsertIntoTableCommand; import org.apache.doris.nereids.trees.plans.commands.insert.InsertIntoTableCommand; import org.apache.doris.nereids.trees.plans.commands.insert.InsertOverwriteTableCommand; @@ -4414,6 +4425,88 @@ public LogicalPlan visitShowProc(ShowProcContext ctx) { return new ShowProcCommand(path); } + private TableScanParams visitOptScanParamsContex(OptScanParamsContext ctx) { + if (ctx != null) { + Map map = visitPropertyItemList(ctx.properties); + return new TableScanParams(ctx.funcName.getText(), map); + } + return null; + } + + private TableSnapshot visitTableSnapshotContext(TableSnapshotContext ctx) { + if (ctx != null) { + if (ctx.TIME() != null) { + return new TableSnapshot(stripQuotes(ctx.time.getText())); + } else { + return new TableSnapshot(Long.parseLong(ctx.version.getText())); + } + } + return null; + } + + private List visitRelationHintContext(RelationHintContext ctx) { + final List relationHints; + if (ctx != null) { + relationHints = typedVisit(ctx); + } else { + relationHints = ImmutableList.of(); + } + return relationHints; + } + + private PartitionNamesInfo visitSpecifiedPartitionContext(SpecifiedPartitionContext ctx) { + if (ctx != null) { + List partitions = new ArrayList<>(); + boolean isTempPart = ctx.TEMPORARY() != null; + if (ctx.identifier() != null) { + partitions.add(ctx.identifier().getText()); + } else { + partitions.addAll(visitIdentifierList(ctx.identifierList())); + } + return new PartitionNamesInfo(isTempPart, partitions); + } + return null; + } + + private List visitTabletListContext(TabletListContext ctx) { + List tabletIdList = new ArrayList<>(); + if (ctx != null && ctx.tabletIdList != null) { + ctx.tabletIdList.stream().forEach(tabletToken -> { + tabletIdList.add(Long.parseLong(tabletToken.getText())); + }); + } + return tabletIdList; + } + + private TableRefInfo visitBaseTableRefContext(BaseTableRefContext ctx) { + List nameParts = visitMultipartIdentifier(ctx.multipartIdentifier()); + TableScanParams scanParams = visitOptScanParamsContex(ctx.optScanParams()); + TableSnapshot tableSnapShot = visitTableSnapshotContext(ctx.tableSnapshot()); + PartitionNamesInfo partitionNameInfo = visitSpecifiedPartitionContext(ctx.specifiedPartition()); + List tabletIdList = visitTabletListContext(ctx.tabletList()); + + String tableAlias = null; + if (ctx.tableAlias().strictIdentifier() != null) { + tableAlias = ctx.tableAlias().getText(); + } + TableSample tableSample = ctx.sample() == null ? null : (TableSample) visit(ctx.sample()); + List hints = visitRelationHintContext(ctx.relationHint()); + return new TableRefInfo(new TableNameInfo(nameParts), scanParams, tableSnapShot, partitionNameInfo, + tabletIdList, tableAlias, tableSample, hints); + } + + @Override + public LogicalPlan visitShowReplicaDistribution(ShowReplicaDistributionContext ctx) { + TableRefInfo tableRefInfo = visitBaseTableRefContext(ctx.baseTableRef()); + return new ShowReplicaDistributionCommand(tableRefInfo); + } + + @Override + public LogicalPlan visitAdminShowReplicaDistribution(AdminShowReplicaDistributionContext ctx) { + TableRefInfo tableRefInfo = visitBaseTableRefContext(ctx.baseTableRef()); + return new ShowReplicaDistributionCommand(tableRefInfo); + } + @Override public LogicalPlan visitShowCreateCatalog(ShowCreateCatalogContext ctx) { return new ShowCreateCatalogCommand(ctx.identifier().getText()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index 57eb2bbb6b42c2..b5f511d8fe38f1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -207,6 +207,7 @@ public enum PlanType { SHOW_PROC_COMMAND, SHOW_PLUGINS_COMMAND, SHOW_PRIVILEGES_COMMAND, + SHOW_REPLICA_DISTRIBUTION_COMMAND, SHOW_REPOSITORIES_COMMAND, SHOW_ROLE_COMMAND, SHOW_SMALL_FILES_COMMAND, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/Command.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/Command.java index d1b8e5a263c503..2048dc00f3e976 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/Command.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/Command.java @@ -119,15 +119,15 @@ public Plan withGroupExpression(Optional groupExpression) { throw new RuntimeException("Command do not implement withGroupExpression"); } - public void verifyCommandSupported() throws DdlException { + public void verifyCommandSupported(ConnectContext ctx) throws DdlException { // check command has been supported in cloud mode if (Config.isCloudMode()) { - checkSupportedInCloudMode(); + checkSupportedInCloudMode(ctx); } } // check if the command is supported in cloud mode // see checkStmtSupported() in fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java // override this method if the command is not supported in cloud mode - protected void checkSupportedInCloudMode() throws DdlException {} + protected void checkSupportedInCloudMode(ConnectContext ctx) throws DdlException {} } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowReplicaDistributionCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowReplicaDistributionCommand.java new file mode 100644 index 00000000000000..2a073b59fbccbd --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowReplicaDistributionCommand.java @@ -0,0 +1,120 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.commands; + +import org.apache.doris.analysis.PartitionNames; +import org.apache.doris.analysis.RedirectStatus; +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.MetadataViewer; +import org.apache.doris.catalog.ScalarType; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.DdlException; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.common.util.Util; +import org.apache.doris.mysql.privilege.Auth; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.commands.info.TableRefInfo; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.ShowResultSet; +import org.apache.doris.qe.ShowResultSetMetaData; +import org.apache.doris.qe.StmtExecutor; + +import com.google.common.collect.ImmutableList; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.List; + +/** + * show replica distribution command + */ +public class ShowReplicaDistributionCommand extends ShowCommand { + public static final ImmutableList TITLE_NAMES = new ImmutableList.Builder() + .add("BackendId").add("ReplicaNum").add("ReplicaSize") + .add("NumGraph").add("NumPercent") + .add("SizeGraph").add("SizePercent") + .add("CloudClusterName").add("CloudClusterId") + .build(); + private static final Logger LOG = LogManager.getLogger(ShowReplicaDistributionCommand.class); + private final TableRefInfo tableRefInfo; + + /** + * constructor + */ + + public ShowReplicaDistributionCommand(TableRefInfo tableRefInfo) { + super(PlanType.SHOW_REPLICA_DISTRIBUTION_COMMAND); + this.tableRefInfo = tableRefInfo; + } + + @Override + public ShowResultSet doRun(ConnectContext ctx, StmtExecutor executor) throws Exception { + // check auth + if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, "ADMIN"); + } + tableRefInfo.analyze(ctx); + Util.prohibitExternalCatalog(tableRefInfo.getTableNameInfo().getCtl(), this.getClass().getSimpleName()); + + List> results; + try { + PartitionNames partitionNames = (tableRefInfo.getPartitionNamesInfo() != null) + ? tableRefInfo.getPartitionNamesInfo().translateToLegacyPartitionNames() : null; + results = MetadataViewer.getTabletDistribution(tableRefInfo.getTableNameInfo().getDb(), + tableRefInfo.getTableNameInfo().getTbl(), + partitionNames); + } catch (DdlException e) { + throw new AnalysisException(e.getMessage()); + } + return new ShowResultSet(getMetaData(), results); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitShowReplicaDistributionCommand(this, context); + } + + public ShowResultSetMetaData getMetaData() { + ShowResultSetMetaData.Builder builder = ShowResultSetMetaData.builder(); + for (String title : TITLE_NAMES) { + builder.addColumn(new Column(title, ScalarType.createVarchar(30))); + } + return builder.build(); + } + + @Override + public RedirectStatus toRedirectStatus() { + if (ConnectContext.get().getSessionVariable().getForwardToMaster()) { + return RedirectStatus.FORWARD_NO_SYNC; + } else { + return RedirectStatus.NO_FORWARD; + } + } + + @Override + protected void checkSupportedInCloudMode(ConnectContext ctx) throws DdlException { + if (!ctx.getCurrentUserIdentity().getUser().equals(Auth.ROOT_USER)) { + LOG.info("ShowReplicaDistributionCommand not supported in cloud mode"); + throw new DdlException("Unsupported operation"); + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/PartitionNamesInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/PartitionNamesInfo.java new file mode 100644 index 00000000000000..a31058c8271f0b --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/PartitionNamesInfo.java @@ -0,0 +1,131 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.commands.info; + +import org.apache.doris.analysis.PartitionNames; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.UserException; +import org.apache.doris.qe.ConnectContext; + +import com.google.common.base.Joiner; +import com.google.common.base.Strings; +import com.google.common.collect.Lists; + +import java.util.List; + +/** + * PartitionNamesInfo + */ +public class PartitionNamesInfo { + // Default partition count to collect statistic for external table. + private static final long DEFAULT_PARTITION_COUNT = 100; + private final List partitionNames; + // true if these partitions are temp partitions + private final boolean isTemp; + private final boolean isStar; + private final long count; + + public PartitionNamesInfo(boolean isTemp, List partitionNames) { + this.partitionNames = partitionNames; + this.isTemp = isTemp; + this.isStar = false; + this.count = 0; + } + + public PartitionNamesInfo(PartitionNamesInfo other) { + this.partitionNames = Lists.newArrayList(other.partitionNames); + this.isTemp = other.isTemp; + this.isStar = other.isStar; + this.count = 0; + } + + public PartitionNamesInfo(boolean isStar) { + this.partitionNames = null; + this.isTemp = false; + this.isStar = isStar; + this.count = 0; + } + + public PartitionNamesInfo(long partitionCount) { + this.partitionNames = null; + this.isTemp = false; + this.isStar = false; + this.count = partitionCount; + } + + public List getPartitionNames() { + return partitionNames; + } + + public boolean isTemp() { + return isTemp; + } + + public boolean isStar() { + return isStar; + } + + public long getCount() { + return count; + } + + /** + * validate + */ + public void validate(ConnectContext ctx) throws UserException { + if (isStar && count > 0) { + throw new AnalysisException("All partition and partition count couldn't be set at the same time."); + } + if (isStar || count > 0) { + return; + } + if (partitionNames == null || partitionNames.isEmpty()) { + throw new AnalysisException("No partition specified in partition lists"); + } + // check if partition name is not empty string + if (partitionNames.stream().anyMatch(Strings::isNullOrEmpty)) { + throw new AnalysisException("there are empty partition name"); + } + } + + /** + * toSql + */ + public String toSql() { + if (partitionNames == null || partitionNames.isEmpty()) { + return ""; + } + StringBuilder sb = new StringBuilder(); + if (isTemp) { + sb.append("TEMPORARY "); + } + sb.append("PARTITIONS ("); + sb.append(Joiner.on(", ").join(partitionNames)); + sb.append(")"); + return sb.toString(); + } + + public PartitionNames translateToLegacyPartitionNames() { + return new PartitionNames(isTemp, partitionNames, isStar, count); + } + + @Override + public String toString() { + return toSql(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/TableRefInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/TableRefInfo.java new file mode 100644 index 00000000000000..7171eef841dabe --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/TableRefInfo.java @@ -0,0 +1,78 @@ +// 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. +// This file is modified copy of TableRef + +package org.apache.doris.nereids.trees.plans.commands.info; + +import org.apache.doris.analysis.TableScanParams; +import org.apache.doris.analysis.TableSnapshot; +import org.apache.doris.common.UserException; +import org.apache.doris.nereids.trees.TableSample; +import org.apache.doris.qe.ConnectContext; + +import java.util.List; + +/** + * table ref info + * baseTableRef + : multipartIdentifier optScanParams? tableSnapshot? specifiedPartition? + tabletList? tableAlias sample? relationHint? + currently tableNameInfo(multipartIdentifier) and specifiedPartition API exposed. + if you need write one and use. + */ +public class TableRefInfo { + private final TableNameInfo tableNameInfo; + private final TableScanParams scanParams; + private final TableSnapshot tableSnapShot; + private final PartitionNamesInfo partitionNamesInfo; + private final List tabletIdList; + private final String tableAlias; + private final TableSample tableSample; + private final List relationHints; + + /** + * constructor + */ + public TableRefInfo(TableNameInfo tableNameInfo, TableScanParams scanParams, + TableSnapshot tableSnapShot, PartitionNamesInfo partitionNamesInfo, + List tabletIdList, String tableAlias, + TableSample tableSample, List relationHints) { + this.tableNameInfo = tableNameInfo; + this.scanParams = scanParams; + this.tableSnapShot = tableSnapShot; + this.partitionNamesInfo = partitionNamesInfo; + this.tabletIdList = tabletIdList; + this.tableAlias = tableAlias; + this.tableSample = tableSample; + this.relationHints = relationHints; + } + + public TableNameInfo getTableNameInfo() { + return tableNameInfo; + } + + public PartitionNamesInfo getPartitionNamesInfo() { + return partitionNamesInfo; + } + + public void analyze(ConnectContext ctx) throws UserException { + tableNameInfo.analyze(ctx); + if (partitionNamesInfo != null) { + partitionNamesInfo.validate(ctx); + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index 85633cb798b06c..8cdd9fa3278c70 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -87,6 +87,7 @@ import org.apache.doris.nereids.trees.plans.commands.ShowPrivilegesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowProcCommand; import org.apache.doris.nereids.trees.plans.commands.ShowProcedureStatusCommand; +import org.apache.doris.nereids.trees.plans.commands.ShowReplicaDistributionCommand; import org.apache.doris.nereids.trees.plans.commands.ShowRepositoriesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowRolesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowSmallFilesCommand; @@ -447,6 +448,11 @@ default R visitDropWorkloadGroupCommand(DropWorkloadGroupCommand dropWorkloadGro return visitCommand(dropWorkloadGroupCommand, context); } + default R visitShowReplicaDistributionCommand(ShowReplicaDistributionCommand showReplicaDistributedCommand, + C context) { + return visitCommand(showReplicaDistributedCommand, context); + } + default R visitDropWorkloadPolicyCommand(DropWorkloadPolicyCommand dropWorkloadPolicyCommand, C context) { return visitCommand(dropWorkloadPolicyCommand, context); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java index 05c7853ca469df..05df53ed6796cb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java @@ -756,7 +756,7 @@ private void executeByNereids(TUniqueId queryId) throws Exception { // t3: observer fe receive editlog creating the table from the master fe syncJournalIfNeeded(); try { - ((Command) logicalPlan).verifyCommandSupported(); + ((Command) logicalPlan).verifyCommandSupported(context); ((Command) logicalPlan).run(context, this); } catch (MustFallbackException e) { if (LOG.isDebugEnabled()) { diff --git a/regression-test/suites/auth_call/test_database_management_auth.groovy b/regression-test/suites/auth_call/test_database_management_auth.groovy index 33a26959619e5f..0e1805e20d6840 100644 --- a/regression-test/suites/auth_call/test_database_management_auth.groovy +++ b/regression-test/suites/auth_call/test_database_management_auth.groovy @@ -23,13 +23,14 @@ suite("test_database_management_auth","p0,auth_call") { String user = 'test_database_management_auth_user' String pwd = 'C123_567p' String dbName = 'test_database_management_auth_db' - + def String show_dis_error_msg = "denied" //cloud-mode if (isCloudMode()) { def clusters = sql " SHOW CLUSTERS; " assertTrue(!clusters.isEmpty()) def validCluster = clusters[0][0] sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}"""; + show_dis_error_msg = "Unsupported" } try_sql("DROP USER ${user}") @@ -78,7 +79,7 @@ suite("test_database_management_auth","p0,auth_call") { } test { sql """SHOW REPLICA DISTRIBUTION FROM tbl;""" - exception "denied" + exception "${show_dis_error_msg}" } test { sql """SHOW REPLICA STATUS FROM db1.tbl1;""" diff --git a/regression-test/suites/nereids_p0/show/test_nereids_show_replica_dist.groovy b/regression-test/suites/nereids_p0/show/test_nereids_show_replica_dist.groovy new file mode 100644 index 00000000000000..1cd21d1ea0169c --- /dev/null +++ b/regression-test/suites/nereids_p0/show/test_nereids_show_replica_dist.groovy @@ -0,0 +1,84 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + + +suite("test_nereids_show_replica_dist") { + def tableName = "test_nereids_show_replica_dist" + // create table and insert data + sql """ drop table if exists ${tableName}""" + sql """ + create table ${tableName} ( + `id` int(11), + `name` varchar(128), + `da` date + ) + engine=olap + duplicate key(id) + partition by range(da)( + PARTITION p3 VALUES LESS THAN ('2023-01-01'), + PARTITION p4 VALUES LESS THAN ('2024-01-01'), + PARTITION p5 VALUES LESS THAN ('2025-01-01') + ) + distributed by hash(id) buckets 1 + properties( + "replication_num"="1", + "light_schema_change"="true" + ); + """ + + def queryReplicaCount = { partitionName -> + def result = sql "SHOW REPLICA DISTRIBUTION FROM ${tableName} PARTITION ${partitionName}" + logger.info("${result}") + int sum = 0 + for (row in result) { + sum += row[1].toInteger() + } + sum + } + + def queryAdminReplicaCount = { partitionName -> + def result = sql "ADMIN SHOW REPLICA DISTRIBUTION FROM ${tableName} PARTITION ${partitionName}" + logger.info("${result}") + int sum = 0 + for (row in result) { + sum += row[1].toInteger() + } + sum + } + + def replication_num = 1 + def forceReplicaNum = getFeConfig('force_olap_table_replication_num').toInteger() + if (forceReplicaNum > 0) { + replication_num = forceReplicaNum + } + + assertEquals(replication_num, queryReplicaCount("p3")) + assertEquals(replication_num, queryReplicaCount("p4")) + assertEquals(replication_num, queryReplicaCount("p5")) + + assertEquals(replication_num, queryAdminReplicaCount("p3")) + assertEquals(replication_num, queryAdminReplicaCount("p4")) + assertEquals(replication_num, queryAdminReplicaCount("p5")) + + checkNereidsExecute("SHOW REPLICA DISTRIBUTION FROM ${tableName}") + sql "SHOW REPLICA DISTRIBUTION FROM ${tableName}" + checkNereidsExecute("ADMIN SHOW REPLICA DISTRIBUTION FROM ${tableName}") + sql "ADMIN SHOW REPLICA DISTRIBUTION FROM ${tableName}" + sql """ drop table if exists ${tableName}""" + +} + From f3e9d70ab9558bf8df5145d0432b4d43cceef24b Mon Sep 17 00:00:00 2001 From: Vallish Pai Date: Wed, 27 Nov 2024 14:48:58 +0530 Subject: [PATCH 037/399] [Enhancement] (nereids)implement DropFileCommand in nereids (#44504) Issue Number: close #42619 --- .../org/apache/doris/nereids/DorisParser.g4 | 4 +- .../nereids/parser/LogicalPlanBuilder.java | 13 +++ .../doris/nereids/trees/plans/PlanType.java | 1 + .../trees/plans/commands/DropFileCommand.java | 93 +++++++++++++++++++ .../trees/plans/visitor/CommandVisitor.java | 5 + .../auth_call/test_ddl_file_auth.groovy | 2 +- 6 files changed, 115 insertions(+), 3 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropFileCommand.java diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index f1b17a1b9fb344..cee972a63d210b 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -205,6 +205,8 @@ supportedDropStatement | DROP SQL_BLOCK_RULE (IF EXISTS)? identifierSeq #dropSqlBlockRule | DROP USER (IF EXISTS)? userIdentify #dropUser | DROP WORKLOAD GROUP (IF EXISTS)? name=identifierOrText #dropWorkloadGroup + | DROP FILE name=STRING_LITERAL + ((FROM | IN) database=identifier)? properties=propertyClause #dropFile | DROP WORKLOAD POLICY (IF EXISTS)? name=identifierOrText #dropWorkloadPolicy ; @@ -671,8 +673,6 @@ unsupportedDropStatement | DROP TABLE (IF EXISTS)? name=multipartIdentifier FORCE? #dropTable | DROP VIEW (IF EXISTS)? name=multipartIdentifier #dropView | DROP REPOSITORY name=identifier #dropRepository - | DROP FILE name=STRING_LITERAL - ((FROM | IN) database=identifier)? properties=propertyClause #dropFile | DROP INDEX (IF EXISTS)? name=identifier ON tableName=multipartIdentifier #dropIndex | DROP RESOURCE (IF EXISTS)? name=identifierOrText #dropResource | DROP ROW POLICY (IF EXISTS)? policyName=identifier diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index d553d93da9ca0e..1a49c7ba65fa56 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -106,6 +106,7 @@ import org.apache.doris.nereids.DorisParser.DropCatalogRecycleBinContext; import org.apache.doris.nereids.DorisParser.DropConstraintContext; import org.apache.doris.nereids.DorisParser.DropEncryptkeyContext; +import org.apache.doris.nereids.DorisParser.DropFileContext; import org.apache.doris.nereids.DorisParser.DropMTMVContext; import org.apache.doris.nereids.DorisParser.DropProcedureContext; import org.apache.doris.nereids.DorisParser.DropRoleContext; @@ -474,6 +475,7 @@ import org.apache.doris.nereids.trees.plans.commands.DropCatalogRecycleBinCommand.IdType; import org.apache.doris.nereids.trees.plans.commands.DropConstraintCommand; import org.apache.doris.nereids.trees.plans.commands.DropEncryptkeyCommand; +import org.apache.doris.nereids.trees.plans.commands.DropFileCommand; import org.apache.doris.nereids.trees.plans.commands.DropJobCommand; import org.apache.doris.nereids.trees.plans.commands.DropMTMVCommand; import org.apache.doris.nereids.trees.plans.commands.DropProcedureCommand; @@ -4617,6 +4619,17 @@ public LogicalPlan visitDropEncryptkey(DropEncryptkeyContext ctx) { return new DropEncryptkeyCommand(new EncryptKeyName(nameParts), ctx.EXISTS() != null); } + @Override + public LogicalPlan visitDropFile(DropFileContext ctx) { + String dbName = null; + if (ctx.database != null) { + dbName = ctx.database.getText(); + } + Map properties = ctx.propertyClause() != null + ? Maps.newHashMap(visitPropertyClause(ctx.propertyClause())) : Maps.newHashMap(); + return new DropFileCommand(stripQuotes(ctx.name.getText()), dbName, properties); + } + @Override public LogicalPlan visitDropSqlBlockRule(DropSqlBlockRuleContext ctx) { return new DropSqlBlockRuleCommand(visitIdentifierSeq(ctx.identifierSeq()), ctx.EXISTS() != null); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index b5f511d8fe38f1..20d46d612d02ac 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -174,6 +174,7 @@ public enum PlanType { ALTER_WORKLOAD_POLICY_COMMAND, DROP_CATALOG_RECYCLE_BIN_COMMAND, DROP_ENCRYPTKEY_COMMAND, + DROP_FILE_COMMAND, UNSET_VARIABLE_COMMAND, UNSET_DEFAULT_STORAGE_VAULT_COMMAND, UNSUPPORTED_COMMAND, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropFileCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropFileCommand.java new file mode 100644 index 00000000000000..92e516238a589b --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropFileCommand.java @@ -0,0 +1,93 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.commands; + +import org.apache.doris.catalog.Database; +import org.apache.doris.catalog.Env; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.StmtExecutor; + +import com.google.common.base.Strings; + +import java.util.Map; +import java.util.Optional; + +/** + * drop file command command + */ +public class DropFileCommand extends DropCommand { + public static final String PROP_CATALOG = "catalog"; + private final String fileName; + private String dbName; // update based on current db if not present. + private final Map properties; + private String catalogName = null; + + /** + * constructor + */ + public DropFileCommand(String fileName, String dbName, Map properties) { + super(PlanType.DROP_FILE_COMMAND); + this.fileName = fileName; + this.dbName = dbName; + this.properties = properties; + } + + @Override + public void doRun(ConnectContext ctx, StmtExecutor executor) throws Exception { + // check operation privilege + if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, "ADMIN"); + } + + if (Strings.isNullOrEmpty(dbName)) { + dbName = ctx.getDatabase(); + if (Strings.isNullOrEmpty(dbName)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_NO_DB_ERROR); + } + } + + if (Strings.isNullOrEmpty(fileName)) { + throw new AnalysisException("File name is not specified"); + } + + Optional optional = properties.keySet().stream().filter( + entity -> !PROP_CATALOG.equals(entity)).findFirst(); + if (optional.isPresent()) { + throw new AnalysisException(optional.get() + " is invalid property"); + } + + catalogName = properties.get(PROP_CATALOG); + if (Strings.isNullOrEmpty(catalogName)) { + throw new AnalysisException("catalog name is missing"); + } + + Database db = Env.getCurrentInternalCatalog().getDbOrDdlException(dbName); + Env.getCurrentEnv().getSmallFileMgr().removeFile(db.getId(), catalogName, fileName, false); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitDropFileCommand(this, context); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index 8cdd9fa3278c70..0af9ce4dc9c4cc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -41,6 +41,7 @@ import org.apache.doris.nereids.trees.plans.commands.DropCatalogRecycleBinCommand; import org.apache.doris.nereids.trees.plans.commands.DropConstraintCommand; import org.apache.doris.nereids.trees.plans.commands.DropEncryptkeyCommand; +import org.apache.doris.nereids.trees.plans.commands.DropFileCommand; import org.apache.doris.nereids.trees.plans.commands.DropJobCommand; import org.apache.doris.nereids.trees.plans.commands.DropMTMVCommand; import org.apache.doris.nereids.trees.plans.commands.DropProcedureCommand; @@ -436,6 +437,10 @@ default R visitDropEncryptKeyCommand(DropEncryptkeyCommand dropEncryptkeyCommand return visitCommand(dropEncryptkeyCommand, context); } + default R visitDropFileCommand(DropFileCommand dropFileCommand, C context) { + return visitCommand(dropFileCommand, context); + } + default R visitDropSqlBlockRuleCommand(DropSqlBlockRuleCommand dropSqlBlockRuleCommand, C context) { return visitCommand(dropSqlBlockRuleCommand, context); } diff --git a/regression-test/suites/auth_call/test_ddl_file_auth.groovy b/regression-test/suites/auth_call/test_ddl_file_auth.groovy index 99b71eabddf140..53cab5b3687ba5 100644 --- a/regression-test/suites/auth_call/test_ddl_file_auth.groovy +++ b/regression-test/suites/auth_call/test_ddl_file_auth.groovy @@ -81,7 +81,7 @@ suite("test_ddl_file_auth","p0,auth_call") { def res = sql """SHOW FILE FROM ${dbName};""" assertTrue(res.size() == 1) - sql """DROP FILE "${fileName}" from ${dbName} properties("catalog" = "internal");""" + checkNereidsExecute("""DROP FILE "${fileName}" from ${dbName} properties("catalog" = "internal");""") res = sql """SHOW FILE FROM ${dbName};""" assertTrue(res.size() == 0) } From 5e54970d26eb0db803cfe14b7bac5a831218d8b0 Mon Sep 17 00:00:00 2001 From: deardeng Date: Wed, 27 Nov 2024 18:59:49 +0800 Subject: [PATCH 038/399] (fix)[db] Fix create database and create table data race (#44600) There may be a race condition between CREATE DATABASE and CREATE TABLE. When CREATE DATABASE is being executed, the edit log may get stuck, while fullNameToDb in getDbNullable can still return the database. As a result, the CREATE TABLE process continues, which may ultimately lead to a strange order in the edit log of BDBJE, where CREATE TABLE appears before CREATE DATABASE. --- .../java/org/apache/doris/catalog/Database.java | 2 +- .../apache/doris/datasource/InternalCatalog.java | 13 +++++++++++-- 2 files changed, 12 insertions(+), 3 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java index 09e2ae350c5e98..4f205be0381b37 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java @@ -93,7 +93,7 @@ public class Database extends MetaObject implements Writable, DatabaseIf
, private final Map idToTable; private ConcurrentMap nameToTable; // table name lower case -> table name - private final Map lowerCaseToTableName; + private final ConcurrentMap lowerCaseToTableName; // user define function @SerializedName(value = "name2Function") diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java index 3a9e96bade67f5..18f2e333e9119b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java @@ -441,8 +441,17 @@ public void createDb(CreateDbStmt stmt) throws DdlException { ErrorReport.reportDdlException(ErrorCode.ERR_DB_CREATE_EXISTS, fullDbName); } } else { - unprotectCreateDb(db); - Env.getCurrentEnv().getEditLog().logCreateDb(db); + if (!db.tryWriteLock(100, TimeUnit.SECONDS)) { + LOG.warn("try lock failed, create database failed {}", fullDbName); + ErrorReport.reportDdlException(ErrorCode.ERR_EXECUTE_TIMEOUT, + "create database " + fullDbName + " time out"); + } + try { + unprotectCreateDb(db); + Env.getCurrentEnv().getEditLog().logCreateDb(db); + } finally { + db.writeUnlock(); + } } } finally { unlock(); From c56abeed7b6812b072771a9cb7ef427044d86272 Mon Sep 17 00:00:00 2001 From: zhangdong Date: Wed, 27 Nov 2024 20:07:59 +0800 Subject: [PATCH 039/399] [case](sql_block_rule) fix case return list should order (#44562) --- .../suites/sql_block_rule_p0/test_sql_block_rule.groovy | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/regression-test/suites/sql_block_rule_p0/test_sql_block_rule.groovy b/regression-test/suites/sql_block_rule_p0/test_sql_block_rule.groovy index 13b83d272decdb..001c64fbc0d0ad 100644 --- a/regression-test/suites/sql_block_rule_p0/test_sql_block_rule.groovy +++ b/regression-test/suites/sql_block_rule_p0/test_sql_block_rule.groovy @@ -107,7 +107,7 @@ suite("test_sql_block_rule", "nonConcurrent") { PROPERTIES("sql"="SELECT \\\\* FROM table_2", "global"= "true", "enable"= "true") """ - qt_select4_exist """ + order_qt_select4_exist """ SHOW SQL_BLOCK_RULE """ From da79ceb9bb08c871e92cb346ee67582137e06e72 Mon Sep 17 00:00:00 2001 From: zhangdong Date: Wed, 27 Nov 2024 20:08:43 +0800 Subject: [PATCH 040/399] [fix](auth)Fix the need for low-level table permissions when querying views in certain situations (#44621) ### What problem does this PR solve? fix when `create view v1 as select * from table1 union select * from table2` and user has select_priv of v1,but he can not `select * from v1` ### Release note Fix the need for low-level table permissions when querying views in certain situations --- .../doris/nereids/StatementContext.java | 10 +++ .../rules/rewrite/CheckPrivileges.java | 6 +- .../auth_p0/test_select_view_auth.groovy | 89 +++++++++++++++++++ 3 files changed, 104 insertions(+), 1 deletion(-) create mode 100644 regression-test/suites/auth_p0/test_select_view_auth.groovy diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java index 69848ca8f04da2..6b6e335b74a676 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java @@ -174,6 +174,8 @@ public class StatementContext implements Closeable { private Backend groupCommitMergeBackend; + private boolean privChecked; + public StatementContext() { this(ConnectContext.get(), null, 0); } @@ -580,4 +582,12 @@ public void setGroupCommitMergeBackend( Backend groupCommitMergeBackend) { this.groupCommitMergeBackend = groupCommitMergeBackend; } + + public boolean isPrivChecked() { + return privChecked; + } + + public void setPrivChecked(boolean privChecked) { + this.privChecked = privChecked; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CheckPrivileges.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CheckPrivileges.java index 74609694431e33..ebef2ecea21207 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CheckPrivileges.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CheckPrivileges.java @@ -49,9 +49,13 @@ public class CheckPrivileges extends ColumnPruning { @Override public Plan rewriteRoot(Plan plan, JobContext jobContext) { + // Only enter once, if repeated, the permissions of the table in the view will be checked + if (jobContext.getCascadesContext().getStatementContext().isPrivChecked()) { + return plan; + } this.jobContext = jobContext; super.rewriteRoot(plan, jobContext); - + jobContext.getCascadesContext().getStatementContext().setPrivChecked(true); // don't rewrite plan return plan; } diff --git a/regression-test/suites/auth_p0/test_select_view_auth.groovy b/regression-test/suites/auth_p0/test_select_view_auth.groovy new file mode 100644 index 00000000000000..87ec8cf0aeb758 --- /dev/null +++ b/regression-test/suites/auth_p0/test_select_view_auth.groovy @@ -0,0 +1,89 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("test_select_view_auth","p0,auth") { + String suiteName = "test_select_view_auth" + String user = "${suiteName}_user" + String pwd = 'C123_567p' + String dbName = "${suiteName}_db" + String tableName1 = "${suiteName}_table1" + String tableName2 = "${suiteName}_table2" + String viewName = "${suiteName}_view" + + try_sql("drop user ${user}") + try_sql """drop table if exists ${dbName}.${tableName1}""" + try_sql """drop table if exists ${dbName}.${tableName2}""" + try_sql """drop view if exists ${dbName}.${viewName}""" + sql """drop database if exists ${dbName}""" + + sql """create user '${user}' IDENTIFIED by '${pwd}'""" + + //cloud-mode + if (isCloudMode()) { + def clusters = sql " SHOW CLUSTERS; " + assertTrue(!clusters.isEmpty()) + def validCluster = clusters[0][0] + sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}"""; + } + sql """create database ${dbName}""" + sql("""use ${dbName}""") + sql """ + CREATE TABLE IF NOT EXISTS ${dbName}.`${tableName1}` ( + id BIGINT, + username VARCHAR(20) + ) + DISTRIBUTED BY HASH(id) BUCKETS 2 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + sql """ + CREATE TABLE IF NOT EXISTS ${dbName}.`${tableName2}` ( + id BIGINT, + username VARCHAR(20) + ) + DISTRIBUTED BY HASH(id) BUCKETS 2 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + sql """create view ${dbName}.${viewName} as select * from ${dbName}.${tableName1} union select * from ${dbName}.${tableName2};""" + + sql """grant select_priv on regression_test to ${user}""" + + // table column + connect(user=user, password="${pwd}", url=context.config.jdbcUrl) { + try { + sql "select * from ${dbName}.${viewName}" + } catch (Exception e) { + log.info(e.getMessage()) + assertTrue(e.getMessage().contains("denied")) + } + } + sql """grant select_priv on ${dbName}.${viewName} to ${user}""" + connect(user=user, password="${pwd}", url=context.config.jdbcUrl) { + sql "select * from ${dbName}.${viewName}" + } + + try_sql("drop user ${user}") + try_sql """drop table if exists ${dbName}.${tableName1}""" + try_sql """drop table if exists ${dbName}.${tableName2}""" + try_sql """drop view if exists ${dbName}.${viewName}""" + sql """drop database if exists ${dbName}""" +} From de8cfd163b881237631963499e9bd8d569980217 Mon Sep 17 00:00:00 2001 From: Xinyi Zou Date: Wed, 27 Nov 2024 21:19:17 +0800 Subject: [PATCH 041/399] [opt](memory) Process profile add metadata profile (#44647) ### What problem does this PR solve? Problem Summary: http://127.0.0.1:8040/profile ``` ObjectHeapDump: - AllRowsetsMemBytes: 16.33 MB - AllRowsetsNum: 76.45K - AllSegmentsMemBytes: 4.16 KB - AllSegmentsNum: 1 - BitmapIndexReaderMemBytes: 0 - BitmapIndexReaderNum: 0 - BloomFilterIndexReaderMemBytes: 0 - BloomFilterIndexReaderNum: 0 - ColumnReaderMemBytes: 0 - ColumnReaderNum: 0 - IndexPageReaderMemBytes: 0 - IndexPageReaderNum: 0 - IndexedColumnReaderMemBytes: 0 - IndexedColumnReaderNum: 0 - InvertedIndexReaderMemBytes: 0 - InvertedIndexReaderNum: 0 - OrdinalIndexReaderMemBytes: 0 - OrdinalIndexReaderNum: 0 - RowsetMetaMemBytes: 45.47 MB - RowsetMetaNum: 76.45K - TabletColumnMemBytes: 253.28 MB - TabletColumnNum: 922.18K - TabletIndexMemBytes: 16.78 MB - TabletIndexNum: 104.74K - TabletMetaMemBytes: 17.50 MB - TabletMetaNum: 45.89K - TabletSchemaMemBytes: 60.15 MB - TabletSchemaNum: 49.19K - ZoneMapIndexReaderMemBytes: 0 - ZoneMapIndexReaderNum: 0 ``` --- be/src/olap/metadata_adder.h | 140 +++++++++++++++++++++++++++-- be/src/olap/rowset/rowset.cpp | 7 -- be/src/olap/rowset/rowset.h | 5 +- be/src/runtime/process_profile.cpp | 10 ++- 4 files changed, 146 insertions(+), 16 deletions(-) diff --git a/be/src/olap/metadata_adder.h b/be/src/olap/metadata_adder.h index bdc9e7a398d8a3..559c5db873b2f6 100644 --- a/be/src/olap/metadata_adder.h +++ b/be/src/olap/metadata_adder.h @@ -20,11 +20,16 @@ #include #include +#include "util/runtime_profile.h" + namespace doris { inline bvar::Adder g_rowset_meta_mem_bytes("doris_rowset_meta_mem_bytes"); inline bvar::Adder g_rowset_meta_num("doris_rowset_meta_num"); +inline bvar::Adder g_all_rowsets_mem_bytes("doris_all_rowsets_mem_bytes"); +inline bvar::Adder g_all_rowsets_num("doris_all_rowsets_num"); + inline bvar::Adder g_tablet_meta_mem_bytes("doris_tablet_meta_mem_bytes"); inline bvar::Adder g_tablet_meta_num("doris_tablet_meta_num"); @@ -37,8 +42,8 @@ inline bvar::Adder g_tablet_index_num("doris_tablet_index_num"); inline bvar::Adder g_tablet_schema_mem_bytes("doris_tablet_schema_mem_bytes"); inline bvar::Adder g_tablet_schema_num("doris_tablet_schema_num"); -inline bvar::Adder g_segment_mem_bytes("doris_segment_mem_bytes"); -inline bvar::Adder g_segment_num("doris_segment_num"); +inline bvar::Adder g_all_segments_mem_bytes("doris_all_segments_mem_bytes"); +inline bvar::Adder g_all_segments_num("doris_all_segments_num"); inline bvar::Adder g_column_reader_mem_bytes("doris_column_reader_mem_bytes"); inline bvar::Adder g_column_reader_num("doris_column_reader_num"); @@ -104,6 +109,8 @@ class MetadataAdder { public: MetadataAdder(); + static void dump_metadata_object(RuntimeProfile* object_heap_dump_snapshot); + protected: MetadataAdder(const MetadataAdder& other); @@ -159,6 +166,8 @@ void MetadataAdder::add_mem_size(int64_t val) { } if constexpr (std::is_same_v) { g_rowset_meta_mem_bytes << val; + } else if constexpr (std::is_same_v) { + g_all_rowsets_mem_bytes << val; } else if constexpr (std::is_same_v) { g_tablet_meta_mem_bytes << val; } else if constexpr (std::is_same_v) { @@ -168,7 +177,7 @@ void MetadataAdder::add_mem_size(int64_t val) { } else if constexpr (std::is_same_v) { g_tablet_schema_mem_bytes << val; } else if constexpr (std::is_same_v) { - g_segment_mem_bytes << val; + g_all_segments_mem_bytes << val; } else if constexpr (std::is_same_v) { g_column_reader_mem_bytes << val; } else if constexpr (std::is_same_v) { @@ -185,6 +194,9 @@ void MetadataAdder::add_mem_size(int64_t val) { g_ordinal_index_reader_mem_bytes << val; } else if constexpr (std::is_same_v) { g_zone_map_index_reader_mem_bytes << val; + } else { + LOG(FATAL) << "add_mem_size not match class type: " << typeid(T).name() << ", " << val; + __builtin_unreachable(); } } @@ -195,6 +207,8 @@ void MetadataAdder::add_num(int64_t val) { } if constexpr (std::is_same_v) { g_rowset_meta_num << val; + } else if constexpr (std::is_same_v) { + g_all_rowsets_num << val; } else if constexpr (std::is_same_v) { g_tablet_meta_num << val; } else if constexpr (std::is_same_v) { @@ -204,7 +218,7 @@ void MetadataAdder::add_num(int64_t val) { } else if constexpr (std::is_same_v) { g_tablet_schema_num << val; } else if constexpr (std::is_same_v) { - g_segment_num << val; + g_all_segments_num << val; } else if constexpr (std::is_same_v) { g_column_reader_num << val; } else if constexpr (std::is_same_v) { @@ -221,7 +235,123 @@ void MetadataAdder::add_num(int64_t val) { g_ordinal_index_reader_num << val; } else if constexpr (std::is_same_v) { g_zone_map_index_reader_num << val; + } else { + LOG(FATAL) << "add_num not match class type: " << typeid(T).name() << ", " << val; + __builtin_unreachable(); } } -}; // namespace doris \ No newline at end of file +template +void MetadataAdder::dump_metadata_object(RuntimeProfile* object_heap_dump_snapshot) { + RuntimeProfile::Counter* rowset_meta_mem_bytes_counter = + ADD_COUNTER(object_heap_dump_snapshot, "RowsetMetaMemBytes", TUnit::BYTES); + RuntimeProfile::Counter* rowset_meta_num_counter = + ADD_COUNTER(object_heap_dump_snapshot, "RowsetMetaNum", TUnit::UNIT); + COUNTER_SET(rowset_meta_mem_bytes_counter, g_rowset_meta_mem_bytes.get_value()); + COUNTER_SET(rowset_meta_num_counter, g_rowset_meta_num.get_value()); + + RuntimeProfile::Counter* all_rowsets_mem_bytes_counter = + ADD_COUNTER(object_heap_dump_snapshot, "AllRowsetsMemBytes", TUnit::BYTES); + RuntimeProfile::Counter* all_rowsets_num_counter = + ADD_COUNTER(object_heap_dump_snapshot, "AllRowsetsNum", TUnit::UNIT); + COUNTER_SET(all_rowsets_mem_bytes_counter, g_all_rowsets_mem_bytes.get_value()); + COUNTER_SET(all_rowsets_num_counter, g_all_rowsets_num.get_value()); + + RuntimeProfile::Counter* tablet_meta_mem_bytes_counter = + ADD_COUNTER(object_heap_dump_snapshot, "TabletMetaMemBytes", TUnit::BYTES); + RuntimeProfile::Counter* tablet_meta_num_counter = + ADD_COUNTER(object_heap_dump_snapshot, "TabletMetaNum", TUnit::UNIT); + COUNTER_SET(tablet_meta_mem_bytes_counter, g_tablet_meta_mem_bytes.get_value()); + COUNTER_SET(tablet_meta_num_counter, g_tablet_meta_num.get_value()); + + RuntimeProfile::Counter* tablet_column_mem_bytes_counter = + ADD_COUNTER(object_heap_dump_snapshot, "TabletColumnMemBytes", TUnit::BYTES); + RuntimeProfile::Counter* tablet_column_num_counter = + ADD_COUNTER(object_heap_dump_snapshot, "TabletColumnNum", TUnit::UNIT); + COUNTER_SET(tablet_column_mem_bytes_counter, g_tablet_column_mem_bytes.get_value()); + COUNTER_SET(tablet_column_num_counter, g_tablet_column_num.get_value()); + + RuntimeProfile::Counter* tablet_index_mem_bytes_counter = + ADD_COUNTER(object_heap_dump_snapshot, "TabletIndexMemBytes", TUnit::BYTES); + RuntimeProfile::Counter* tablet_index_num_counter = + ADD_COUNTER(object_heap_dump_snapshot, "TabletIndexNum", TUnit::UNIT); + COUNTER_SET(tablet_index_mem_bytes_counter, g_tablet_index_mem_bytes.get_value()); + COUNTER_SET(tablet_index_num_counter, g_tablet_index_num.get_value()); + + RuntimeProfile::Counter* tablet_schema_mem_bytes_counter = + ADD_COUNTER(object_heap_dump_snapshot, "TabletSchemaMemBytes", TUnit::BYTES); + RuntimeProfile::Counter* tablet_schema_num_counter = + ADD_COUNTER(object_heap_dump_snapshot, "TabletSchemaNum", TUnit::UNIT); + COUNTER_SET(tablet_schema_mem_bytes_counter, g_tablet_schema_mem_bytes.get_value()); + COUNTER_SET(tablet_schema_num_counter, g_tablet_schema_num.get_value()); + + RuntimeProfile::Counter* all_segments_mem_bytes_counter = + ADD_COUNTER(object_heap_dump_snapshot, "AllSegmentsMemBytes", TUnit::BYTES); + RuntimeProfile::Counter* all_segments_num_counter = + ADD_COUNTER(object_heap_dump_snapshot, "AllSegmentsNum", TUnit::UNIT); + COUNTER_SET(all_segments_mem_bytes_counter, g_all_segments_mem_bytes.get_value()); + COUNTER_SET(all_segments_num_counter, g_all_segments_num.get_value()); + + RuntimeProfile::Counter* column_reader_mem_bytes_counter = + ADD_COUNTER(object_heap_dump_snapshot, "ColumnReaderMemBytes", TUnit::BYTES); + RuntimeProfile::Counter* column_reader_num_counter = + ADD_COUNTER(object_heap_dump_snapshot, "ColumnReaderNum", TUnit::UNIT); + COUNTER_SET(column_reader_mem_bytes_counter, g_column_reader_mem_bytes.get_value()); + COUNTER_SET(column_reader_num_counter, g_column_reader_num.get_value()); + + RuntimeProfile::Counter* bitmap_index_reader_mem_bytes_counter = + ADD_COUNTER(object_heap_dump_snapshot, "BitmapIndexReaderMemBytes", TUnit::BYTES); + RuntimeProfile::Counter* bitmap_index_reader_num_counter = + ADD_COUNTER(object_heap_dump_snapshot, "BitmapIndexReaderNum", TUnit::UNIT); + COUNTER_SET(bitmap_index_reader_mem_bytes_counter, g_bitmap_index_reader_mem_bytes.get_value()); + COUNTER_SET(bitmap_index_reader_num_counter, g_bitmap_index_reader_num.get_value()); + + RuntimeProfile::Counter* bloom_filter_index_reader_mem_bytes_counter = + ADD_COUNTER(object_heap_dump_snapshot, "BloomFilterIndexReaderMemBytes", TUnit::BYTES); + RuntimeProfile::Counter* filter_index_reader_num_counter = + ADD_COUNTER(object_heap_dump_snapshot, "BloomFilterIndexReaderNum", TUnit::UNIT); + COUNTER_SET(bloom_filter_index_reader_mem_bytes_counter, + g_bloom_filter_index_reader_mem_bytes.get_value()); + COUNTER_SET(filter_index_reader_num_counter, g_bloom_filter_index_reader_num.get_value()); + + RuntimeProfile::Counter* index_page_reader_mem_bytes_counter = + ADD_COUNTER(object_heap_dump_snapshot, "IndexPageReaderMemBytes", TUnit::BYTES); + RuntimeProfile::Counter* index_page_reader_num_counter = + ADD_COUNTER(object_heap_dump_snapshot, "IndexPageReaderNum", TUnit::UNIT); + COUNTER_SET(index_page_reader_mem_bytes_counter, g_index_page_reader_mem_bytes.get_value()); + COUNTER_SET(index_page_reader_num_counter, g_index_page_reader_num.get_value()); + + RuntimeProfile::Counter* indexed_column_reader_mem_bytes_counter = + ADD_COUNTER(object_heap_dump_snapshot, "IndexedColumnReaderMemBytes", TUnit::BYTES); + RuntimeProfile::Counter* indexed_column_reader_num_counter = + ADD_COUNTER(object_heap_dump_snapshot, "IndexedColumnReaderNum", TUnit::UNIT); + COUNTER_SET(indexed_column_reader_mem_bytes_counter, + g_indexed_column_reader_mem_bytes.get_value()); + COUNTER_SET(indexed_column_reader_num_counter, g_indexed_column_reader_num.get_value()); + + RuntimeProfile::Counter* inverted_index_reader_mem_bytes_counter = + ADD_COUNTER(object_heap_dump_snapshot, "InvertedIndexReaderMemBytes", TUnit::BYTES); + RuntimeProfile::Counter* inverted_index_reader_num_counter = + ADD_COUNTER(object_heap_dump_snapshot, "InvertedIndexReaderNum", TUnit::UNIT); + COUNTER_SET(inverted_index_reader_mem_bytes_counter, + g_inverted_index_reader_mem_bytes.get_value()); + COUNTER_SET(inverted_index_reader_num_counter, g_inverted_index_reader_num.get_value()); + + RuntimeProfile::Counter* ordinal_index_reader_mem_bytes_counter = + ADD_COUNTER(object_heap_dump_snapshot, "OrdinalIndexReaderMemBytes", TUnit::BYTES); + RuntimeProfile::Counter* ordinal_index_reader_num_counter = + ADD_COUNTER(object_heap_dump_snapshot, "OrdinalIndexReaderNum", TUnit::UNIT); + COUNTER_SET(ordinal_index_reader_mem_bytes_counter, + g_ordinal_index_reader_mem_bytes.get_value()); + COUNTER_SET(ordinal_index_reader_num_counter, g_ordinal_index_reader_num.get_value()); + + RuntimeProfile::Counter* zone_map_index_reader_mem_bytes_counter = + ADD_COUNTER(object_heap_dump_snapshot, "ZoneMapIndexReaderMemBytes", TUnit::BYTES); + RuntimeProfile::Counter* zone_map_index_reader_num_counter = + ADD_COUNTER(object_heap_dump_snapshot, "ZoneMapIndexReaderNum", TUnit::UNIT); + COUNTER_SET(zone_map_index_reader_mem_bytes_counter, + g_zone_map_index_reader_mem_bytes.get_value()); + COUNTER_SET(zone_map_index_reader_num_counter, g_zone_map_index_reader_num.get_value()); +} + +}; // namespace doris diff --git a/be/src/olap/rowset/rowset.cpp b/be/src/olap/rowset/rowset.cpp index 256f4d35313d13..ac3a2a7a1dc5c2 100644 --- a/be/src/olap/rowset/rowset.cpp +++ b/be/src/olap/rowset/rowset.cpp @@ -27,8 +27,6 @@ namespace doris { -static bvar::Adder g_total_rowset_num("doris_total_rowset_num"); - Rowset::Rowset(const TabletSchemaSPtr& schema, RowsetMetaSharedPtr rowset_meta, std::string tablet_path) : _rowset_meta(std::move(rowset_meta)), @@ -56,11 +54,6 @@ Rowset::Rowset(const TabletSchemaSPtr& schema, RowsetMetaSharedPtr rowset_meta, } // build schema from RowsetMeta.tablet_schema or Tablet.tablet_schema _schema = _rowset_meta->tablet_schema() ? _rowset_meta->tablet_schema() : schema; - g_total_rowset_num << 1; -} - -Rowset::~Rowset() { - g_total_rowset_num << -1; } Status Rowset::load(bool use_cache) { diff --git a/be/src/olap/rowset/rowset.h b/be/src/olap/rowset/rowset.h index e1a2347f6aeaa8..98d88ba19f2068 100644 --- a/be/src/olap/rowset/rowset.h +++ b/be/src/olap/rowset/rowset.h @@ -33,6 +33,7 @@ #include "common/logging.h" #include "common/status.h" +#include "olap/metadata_adder.h" #include "olap/olap_common.h" #include "olap/rowset/rowset_meta.h" #include "olap/tablet_schema.h" @@ -116,10 +117,8 @@ class RowsetStateMachine { RowsetState _rowset_state; }; -class Rowset : public std::enable_shared_from_this { +class Rowset : public std::enable_shared_from_this, public MetadataAdder { public: - virtual ~Rowset(); - // Open all segment files in this rowset and load necessary metadata. // - `use_cache` : whether to use fd cache, only applicable to alpha rowset now // diff --git a/be/src/runtime/process_profile.cpp b/be/src/runtime/process_profile.cpp index d91aedbeac2025..60b381e9c31578 100644 --- a/be/src/runtime/process_profile.cpp +++ b/be/src/runtime/process_profile.cpp @@ -19,6 +19,7 @@ #include +#include "olap/metadata_adder.h" #include "runtime/memory/memory_profile.h" namespace doris { @@ -37,8 +38,15 @@ void ProcessProfile::refresh_profile() { std::unique_ptr process_profile = std::make_unique("ProcessProfile"); _memory_profile->make_memory_profile(process_profile.get()); - _process_profile.set(std::move(process_profile)); // TODO make other profile + + // 3. dump object heap + RuntimeProfile* object_heap_dump_snapshot = + process_profile->create_child("ObjectHeapDump", true, false); + MetadataAdder::dump_metadata_object(object_heap_dump_snapshot); + // TODO dump other object (block, column, etc.) + + _process_profile.set(std::move(process_profile)); } } // namespace doris From db3a3ec49d20019b1cb871c11aa10d13b974ec22 Mon Sep 17 00:00:00 2001 From: Xinyi Zou Date: Wed, 27 Nov 2024 22:13:44 +0800 Subject: [PATCH 042/399] [env](compile)open compile check in columns class (#44425) ### What problem does this PR solve? Problem Summary: open compile check in columns class --- be/src/util/hash_util.hpp | 16 +++---- be/src/vec/columns/column_const.cpp | 5 ++- be/src/vec/columns/column_const.h | 5 ++- be/src/vec/columns/column_decimal.h | 4 +- be/src/vec/columns/column_dictionary.h | 22 ++++++---- be/src/vec/columns/column_nullable.cpp | 5 ++- be/src/vec/columns/column_nullable.h | 7 ++- be/src/vec/columns/column_object.cpp | 19 +++++--- be/src/vec/columns/column_object.h | 6 +-- be/src/vec/columns/column_string.cpp | 61 +++++++++++++++----------- be/src/vec/columns/column_string.h | 29 ++++++++---- be/src/vec/columns/column_vector.cpp | 3 +- be/src/vec/columns/column_vector.h | 9 ++-- be/src/vec/core/field.h | 11 +++-- 14 files changed, 125 insertions(+), 77 deletions(-) diff --git a/be/src/util/hash_util.hpp b/be/src/util/hash_util.hpp index e9ac72c5ccdcb4..d444daa8c68d11 100644 --- a/be/src/util/hash_util.hpp +++ b/be/src/util/hash_util.hpp @@ -46,7 +46,7 @@ class HashUtil { return std::hash()(value); } - static uint32_t zlib_crc_hash(const void* data, int32_t bytes, uint32_t hash) { + static uint32_t zlib_crc_hash(const void* data, uint32_t bytes, uint32_t hash) { return crc32(hash, (const unsigned char*)data, bytes); } @@ -66,7 +66,7 @@ class HashUtil { // NOTE: Any changes made to this function need to be reflected in Codegen::GetHashFn. // TODO: crc32 hashes with different seeds do not result in different hash functions. // The resulting hashes are correlated. - static uint32_t crc_hash(const void* data, int32_t bytes, uint32_t hash) { + static uint32_t crc_hash(const void* data, uint32_t bytes, uint32_t hash) { if (!CpuInfo::is_supported(CpuInfo::SSE4_2)) { return zlib_crc_hash(data, bytes, hash); } @@ -93,7 +93,7 @@ class HashUtil { return hash; } - static uint64_t crc_hash64(const void* data, int32_t bytes, uint64_t hash) { + static uint64_t crc_hash64(const void* data, uint32_t bytes, uint64_t hash) { uint32_t words = bytes / sizeof(uint32_t); bytes = bytes % sizeof(uint32_t); @@ -125,7 +125,7 @@ class HashUtil { return converter.u64; } #else - static uint32_t crc_hash(const void* data, int32_t bytes, uint32_t hash) { + static uint32_t crc_hash(const void* data, uint32_t bytes, uint32_t hash) { return zlib_crc_hash(data, bytes, hash); } #endif @@ -202,7 +202,7 @@ class HashUtil { // For example, if the data is <1000, 2000, 3000, 4000, ..> and then the mod of 1000 // is taken on the hash, all values will collide to the same bucket. // For string values, Fnv is slightly faster than boost. - static uint32_t fnv_hash(const void* data, int32_t bytes, uint32_t hash) { + static uint32_t fnv_hash(const void* data, uint32_t bytes, uint32_t hash) { const uint8_t* ptr = reinterpret_cast(data); while (bytes--) { @@ -213,7 +213,7 @@ class HashUtil { return hash; } - static uint64_t fnv_hash64(const void* data, int32_t bytes, uint64_t hash) { + static uint64_t fnv_hash64(const void* data, uint32_t bytes, uint64_t hash) { const uint8_t* ptr = reinterpret_cast(data); while (bytes--) { @@ -291,7 +291,7 @@ class HashUtil { // depending on hardware capabilities. // Seed values for different steps of the query execution should use different seeds // to prevent accidental key collisions. (See IMPALA-219 for more details). - static uint32_t hash(const void* data, int32_t bytes, uint32_t seed) { + static uint32_t hash(const void* data, uint32_t bytes, uint32_t seed) { #ifdef __SSE4_2__ if (LIKELY(CpuInfo::is_supported(CpuInfo::SSE4_2))) { @@ -305,7 +305,7 @@ class HashUtil { #endif } - static uint64_t hash64(const void* data, int32_t bytes, uint64_t seed) { + static uint64_t hash64(const void* data, uint32_t bytes, uint64_t seed) { #ifdef _SSE4_2_ if (LIKELY(CpuInfo::is_supported(CpuInfo::SSE4_2))) { return crc_hash64(data, bytes, seed); diff --git a/be/src/vec/columns/column_const.cpp b/be/src/vec/columns/column_const.cpp index f751f1d8d3e1b8..fd05127f6d72aa 100644 --- a/be/src/vec/columns/column_const.cpp +++ b/be/src/vec/columns/column_const.cpp @@ -35,6 +35,7 @@ #include "vec/core/column_with_type_and_name.h" namespace doris::vectorized { +#include "common/compile_check_begin.h" ColumnConst::ColumnConst(const ColumnPtr& data_, size_t s_) : data(data_), s(s_) { /// Squash Const of Const. @@ -66,7 +67,9 @@ ColumnConst::ColumnConst(const ColumnPtr& data_, size_t s_, bool create_with_emp } ColumnPtr ColumnConst::convert_to_full_column() const { - return data->replicate(Offsets(1, s)); + // Assuming the number of replicate rows will not exceed Offset(UInt32), + // currently Column::replicate only supports Uint32 Offsets + return data->replicate(Offsets(1, cast_set(s))); } ColumnPtr ColumnConst::remove_low_cardinality() const { diff --git a/be/src/vec/columns/column_const.h b/be/src/vec/columns/column_const.h index ee3860f0635c93..0fa22ca23bf20b 100644 --- a/be/src/vec/columns/column_const.h +++ b/be/src/vec/columns/column_const.h @@ -48,6 +48,7 @@ class SipHash; namespace doris::vectorized { +#include "common/compile_check_begin.h" class Arena; class Block; @@ -267,7 +268,8 @@ class ColumnConst final : public COWHelper { template T get_value() const { - return get_field().safe_get>(); + // Here the cast is correct, relevant code is rather tricky. + return static_cast(get_field().safe_get>()); } void replace_column_data(const IColumn& rhs, size_t row, size_t self_row = 0) override { @@ -276,3 +278,4 @@ class ColumnConst final : public COWHelper { } }; } // namespace doris::vectorized +#include "common/compile_check_end.h" diff --git a/be/src/vec/columns/column_decimal.h b/be/src/vec/columns/column_decimal.h index 4c2f69d5ef3fb6..946b268436ef6c 100644 --- a/be/src/vec/columns/column_decimal.h +++ b/be/src/vec/columns/column_decimal.h @@ -53,6 +53,7 @@ class ColumnSorter; } // namespace doris namespace doris::vectorized { +#include "common/compile_check_begin.h" /// PaddedPODArray extended by Decimal scale template @@ -261,7 +262,7 @@ class ColumnDecimal final : public COWHelper> { for (U i = 0; i < s; ++i) res[i] = i; auto sort_end = res.end(); - if (limit && limit < s / 8.0) { + if (limit && limit < static_cast(s) / 8.0L) { sort_end = res.begin() + limit; if (reverse) std::partial_sort(res.begin(), sort_end, res.end(), @@ -305,3 +306,4 @@ template using ColumnVectorOrDecimal = typename ColumnVectorOrDecimalT>::Col; } // namespace doris::vectorized +#include "common/compile_check_end.h" diff --git a/be/src/vec/columns/column_dictionary.h b/be/src/vec/columns/column_dictionary.h index ae7d001a31d78c..c4b1f3e27e0cc9 100644 --- a/be/src/vec/columns/column_dictionary.h +++ b/be/src/vec/columns/column_dictionary.h @@ -29,6 +29,7 @@ #include "vec/core/types.h" namespace doris::vectorized { +#include "common/compile_check_begin.h" /** * For low cardinality string columns, using ColumnDictionary can reduce memory @@ -265,9 +266,9 @@ class ColumnDictionary final : public COWHelper> { } } - int32_t find_code(const StringRef& value) const { return _dict.find_code(value); } + T find_code(const StringRef& value) const { return _dict.find_code(value); } - int32_t find_code_by_bound(const StringRef& value, bool greater, bool eq) const { + T find_code_by_bound(const StringRef& value, bool greater, bool eq) const { return _dict.find_code_by_bound(value, greater, eq); } @@ -346,8 +347,9 @@ class ColumnDictionary final : public COWHelper> { _total_str_len += value.size; } - int32_t find_code(const StringRef& value) const { - for (size_t i = 0; i < _dict_data->size(); i++) { + T find_code(const StringRef& value) const { + // _dict_data->size will not exceed the range of T. + for (T i = 0; i < _dict_data->size(); i++) { if ((*_dict_data)[i] == value) { return i; } @@ -384,11 +386,11 @@ class ColumnDictionary final : public COWHelper> { // For dictionary data of char type, sv.size is the schema length, // so use strnlen to remove the 0 at the end to get the actual length. - int32_t len = sv.size; + size_t len = sv.size; if (type == FieldType::OLAP_FIELD_TYPE_CHAR) { len = strnlen(sv.data, sv.size); } - uint32_t hash_val = HashUtil::crc_hash(sv.data, len, 0); + uint32_t hash_val = HashUtil::crc_hash(sv.data, static_cast(len), 0); _hash_values[code] = hash_val; _compute_hash_value_flags[code] = 1; return _hash_values[code]; @@ -412,13 +414,14 @@ class ColumnDictionary final : public COWHelper> { // so upper_bound is the code 0 of b, then evaluate code < 0 and returns empty // If the predicate is col <= 'a' and upper_bound-1 is -1, // then evaluate code <= -1 and returns empty - int32_t find_code_by_bound(const StringRef& value, bool greater, bool eq) const { + T find_code_by_bound(const StringRef& value, bool greater, bool eq) const { auto code = find_code(value); if (code >= 0) { return code; } - auto bound = std::upper_bound(_dict_data->begin(), _dict_data->end(), value) - - _dict_data->begin(); + auto bound = + static_cast(std::upper_bound(_dict_data->begin(), _dict_data->end(), value) - + _dict_data->begin()); return greater ? bound - greater + eq : bound - eq; } @@ -536,3 +539,4 @@ template class ColumnDictionary; using ColumnDictI32 = vectorized::ColumnDictionary; } // namespace doris::vectorized +#include "common/compile_check_end.h" diff --git a/be/src/vec/columns/column_nullable.cpp b/be/src/vec/columns/column_nullable.cpp index 5e34ad4d8d4d1d..c58c78f5611d02 100644 --- a/be/src/vec/columns/column_nullable.cpp +++ b/be/src/vec/columns/column_nullable.cpp @@ -29,6 +29,7 @@ #include "vec/utils/util.hpp" namespace doris::vectorized { +#include "common/compile_check_begin.h" ColumnNullable::ColumnNullable(MutableColumnPtr&& nested_column_, MutableColumnPtr&& null_map_) : NullMapProvider(std::move(null_map_)), nested_column(std::move(nested_column_)) { @@ -62,7 +63,7 @@ void ColumnNullable::update_xxHash_with_value(size_t start, size_t end, uint64_t } else { const auto* __restrict real_null_data = assert_cast(get_null_map_column()).get_data().data(); - for (int i = start; i < end; ++i) { + for (size_t i = start; i < end; ++i) { if (real_null_data[i] != 0) { hash = HashUtil::xxHash64NullWithSeed(hash); } @@ -78,7 +79,7 @@ void ColumnNullable::update_crc_with_value(size_t start, size_t end, uint32_t& h } else { const auto* __restrict real_null_data = assert_cast(get_null_map_column()).get_data().data(); - for (int i = start; i < end; ++i) { + for (size_t i = start; i < end; ++i) { if (real_null_data[i] != 0) { hash = HashUtil::zlib_crc_hash_null(hash); } diff --git a/be/src/vec/columns/column_nullable.h b/be/src/vec/columns/column_nullable.h index 252144fbc5f612..83d5e6af35aeb4 100644 --- a/be/src/vec/columns/column_nullable.h +++ b/be/src/vec/columns/column_nullable.h @@ -43,6 +43,7 @@ class SipHash; namespace doris::vectorized { +#include "common/compile_check_begin.h" class Arena; class ColumnSorter; @@ -404,7 +405,8 @@ class ColumnNullable final : public COWHelper, public N } static constexpr auto MAX_NUMBER_OF_ROWS_FOR_FULL_SEARCH = 1000; size_t num_rows = size(); - size_t num_sampled_rows = std::min(static_cast(num_rows * sample_ratio), num_rows); + size_t num_sampled_rows = std::min( + static_cast(static_cast(num_rows) * sample_ratio), num_rows); size_t num_checked_rows = 0; size_t res = 0; if (num_sampled_rows == num_rows || num_rows <= MAX_NUMBER_OF_ROWS_FOR_FULL_SEARCH) { @@ -423,7 +425,7 @@ class ColumnNullable final : public COWHelper, public N if (num_checked_rows == 0) { return 0.0; } - return static_cast(res) / num_checked_rows; + return static_cast(res) / static_cast(num_checked_rows); } void convert_dict_codes_if_necessary() override { @@ -460,3 +462,4 @@ class ColumnNullable final : public COWHelper, public N ColumnPtr make_nullable(const ColumnPtr& column, bool is_nullable = false); ColumnPtr remove_nullable(const ColumnPtr& column); } // namespace doris::vectorized +#include "common/compile_check_end.h" diff --git a/be/src/vec/columns/column_object.cpp b/be/src/vec/columns/column_object.cpp index d5e52d07bcf788..3e8d3722305e8d 100644 --- a/be/src/vec/columns/column_object.cpp +++ b/be/src/vec/columns/column_object.cpp @@ -82,6 +82,7 @@ #endif namespace doris::vectorized { +#include "common/compile_check_begin.h" namespace { DataTypePtr create_array_of_type(TypeIndex type, size_t num_dimensions, bool is_nullable) { @@ -653,7 +654,7 @@ bool ColumnObject::Subcolumn::check_if_sparse_column(size_t num_rows) { defaults_ratio.push_back(data[i]->get_ratio_of_default_rows()); } double default_ratio = std::accumulate(defaults_ratio.begin(), defaults_ratio.end(), 0.0) / - defaults_ratio.size(); + static_cast(defaults_ratio.size()); return default_ratio >= config::variant_ratio_of_defaults_as_sparse_column; } @@ -1294,7 +1295,11 @@ rapidjson::Value* find_leaf_node_by_path(rapidjson::Value& json, const PathInDat if (!json.IsObject()) { return nullptr; } - rapidjson::Value name(current_key.data(), current_key.size()); + /*! RapidJSON uses 32-bit array/string indices even on 64-bit platforms, + instead of using \c size_t. Users may override the SizeType by defining + \ref RAPIDJSON_NO_SIZETYPEDEFINE. + */ + rapidjson::Value name(current_key.data(), cast_set(current_key.size())); auto it = json.FindMember(name); if (it == json.MemberEnd()) { return nullptr; @@ -1312,7 +1317,7 @@ rapidjson::Value* find_leaf_node_by_path(rapidjson::Value& json, const PathInDat // 3. empty root jsonb value(not null) // 4. type is nothing bool skip_empty_json(const ColumnNullable* nullable, const DataTypePtr& type, - TypeIndex base_type_id, int row, const PathInData& path) { + TypeIndex base_type_id, size_t row, const PathInData& path) { // skip nulls if (nullable && nullable->is_null_at(row)) { return true; @@ -1348,7 +1353,7 @@ Status find_and_set_leave_value(const IColumn* column, const PathInData& path, const DataTypeSerDeSPtr& type_serde, const DataTypePtr& type, TypeIndex base_type_index, rapidjson::Value& root, rapidjson::Document::AllocatorType& allocator, Arena& mem_pool, - int row) { + size_t row) { #ifndef NDEBUG // sanitize type and column if (column->get_name() != type->create_column()->get_name()) { @@ -1416,7 +1421,7 @@ void get_json_by_column_tree(rapidjson::Value& root, rapidjson::Document::Alloca } } -Status ColumnObject::serialize_one_row_to_string(int64_t row, std::string* output) const { +Status ColumnObject::serialize_one_row_to_string(size_t row, std::string* output) const { if (!is_finalized()) { const_cast(this)->finalize(FinalizeMode::READ_MODE); } @@ -1432,7 +1437,7 @@ Status ColumnObject::serialize_one_row_to_string(int64_t row, std::string* outpu return Status::OK(); } -Status ColumnObject::serialize_one_row_to_string(int64_t row, BufferWritable& output) const { +Status ColumnObject::serialize_one_row_to_string(size_t row, BufferWritable& output) const { if (!is_finalized()) { const_cast(this)->finalize(FinalizeMode::READ_MODE); } @@ -1447,7 +1452,7 @@ Status ColumnObject::serialize_one_row_to_string(int64_t row, BufferWritable& ou return Status::OK(); } -Status ColumnObject::serialize_one_row_to_json_format(int64_t row, rapidjson::StringBuffer* output, +Status ColumnObject::serialize_one_row_to_json_format(size_t row, rapidjson::StringBuffer* output, bool* is_null) const { CHECK(is_finalized()); if (subcolumns.empty()) { diff --git a/be/src/vec/columns/column_object.h b/be/src/vec/columns/column_object.h index 21bb44691151b0..e4127197a22b02 100644 --- a/be/src/vec/columns/column_object.h +++ b/be/src/vec/columns/column_object.h @@ -272,12 +272,12 @@ class ColumnObject final : public COWHelper { return subcolumns.get_mutable_root()->data.get_finalized_column_ptr()->assume_mutable(); } - Status serialize_one_row_to_string(int64_t row, std::string* output) const; + Status serialize_one_row_to_string(size_t row, std::string* output) const; - Status serialize_one_row_to_string(int64_t row, BufferWritable& output) const; + Status serialize_one_row_to_string(size_t row, BufferWritable& output) const; // serialize one row to json format - Status serialize_one_row_to_json_format(int64_t row, rapidjson::StringBuffer* output, + Status serialize_one_row_to_json_format(size_t row, rapidjson::StringBuffer* output, bool* is_null) const; // merge multiple sub sparse columns into root diff --git a/be/src/vec/columns/column_string.cpp b/be/src/vec/columns/column_string.cpp index 3caa194551bf79..cb83a29bbada2c 100644 --- a/be/src/vec/columns/column_string.cpp +++ b/be/src/vec/columns/column_string.cpp @@ -34,6 +34,7 @@ #include "vec/core/sort_block.h" namespace doris::vectorized { +#include "common/compile_check_begin.h" template void ColumnStr::sanity_check() const { @@ -74,8 +75,8 @@ MutableColumnPtr ColumnStr::clone_resized(size_t to_size) const { res->offsets.assign(offsets.begin(), offsets.end()); res->chars.assign(chars.begin(), chars.end()); } - - res->offsets.resize_fill(to_size, chars.size()); + // If offset is uint32, size will not exceed, check the size when inserting data into ColumnStr. + res->offsets.resize_fill(to_size, static_cast(chars.size())); } return res; @@ -92,14 +93,14 @@ void ColumnStr::shrink_padding_chars() { // deal the 0-th element. no need to move. auto next_start = offset[0]; - offset[0] = strnlen(data, size_at(0)); + offset[0] = static_cast(strnlen(data, size_at(0))); for (size_t i = 1; i < size; i++) { // get the i-th length and whole move it to cover the last's trailing void auto length = strnlen(data + next_start, offset[i] - next_start); memmove(data + offset[i - 1], data + next_start, length); // offset i will be changed. so save the old value for (i+1)-th to get its length. next_start = offset[i]; - offset[i] = offset[i - 1] + length; + offset[i] = offset[i - 1] + static_cast(length); } chars.resize_fill(offsets.back()); // just call it to shrink memory here. no possible to expand. } @@ -125,8 +126,8 @@ void ColumnStr::insert_range_from_ignore_overflow(const doris::vectorized::IC "Parameter out of bound in IColumnStr::insert_range_from method."); } - size_t nested_offset = src_concrete.offset_at(start); - size_t nested_length = src_concrete.offsets[start + length - 1] - nested_offset; + auto nested_offset = src_concrete.offset_at(start); + auto nested_length = src_concrete.offsets[start + length - 1] - nested_offset; size_t old_chars_size = chars.size(); chars.resize(old_chars_size + nested_length); @@ -136,7 +137,7 @@ void ColumnStr::insert_range_from_ignore_overflow(const doris::vectorized::IC offsets.assign(src_concrete.offsets.begin(), src_concrete.offsets.begin() + length); } else { size_t old_size = offsets.size(); - size_t prev_max_offset = offsets.back(); /// -1th index is Ok, see PaddedPODArray + auto prev_max_offset = offsets.back(); /// -1th index is Ok, see PaddedPODArray offsets.resize(old_size + length); for (size_t i = 0; i < length; ++i) { @@ -161,8 +162,8 @@ void ColumnStr::insert_range_from(const IColumn& src, size_t start, size_t le doris::ErrorCode::INTERNAL_ERROR, "Parameter out of bound in IColumnStr::insert_range_from method."); } - size_t nested_offset = src_offsets[static_cast(start) - 1]; - size_t nested_length = src_offsets[start + length - 1] - nested_offset; + auto nested_offset = src_offsets[static_cast(start) - 1]; + auto nested_length = src_offsets[start + length - 1] - nested_offset; size_t old_chars_size = chars.size(); check_chars_length(old_chars_size + nested_length, offsets.size() + length); @@ -174,11 +175,13 @@ void ColumnStr::insert_range_from(const IColumn& src, size_t start, size_t le offsets.assign(src_offsets.begin(), src_offsets.begin() + length); } else { size_t old_size = offsets.size(); - size_t prev_max_offset = offsets.back(); /// -1th index is Ok, see PaddedPODArray + auto prev_max_offset = offsets.back(); /// -1th index is Ok, see PaddedPODArray offsets.resize(old_size + length); for (size_t i = 0; i < length; ++i) { - offsets[old_size + i] = src_offsets[start + i] - nested_offset + prev_max_offset; + // if Offsets is uint32, size will not exceed range of uint32, cast is OK. + offsets[old_size + i] = + static_cast(src_offsets[start + i] - nested_offset) + prev_max_offset; } } }; @@ -208,7 +211,7 @@ void ColumnStr::insert_many_from(const IColumn& src, size_t position, size_t auto prev_pos = old_chars_size; for (; start_pos < end_pos; ++start_pos) { memcpy(&chars[prev_pos], data_val, data_length); - offsets[start_pos] = prev_pos + data_length; + offsets[start_pos] = static_cast(prev_pos + data_length); prev_pos = prev_pos + data_length; } } @@ -229,7 +232,8 @@ void ColumnStr::insert_indices_from(const IColumn& src, const uint32_t* indic for (const auto* x = indices_begin; x != indices_end; ++x) { int64_t src_offset = *x; total_chars_size += src_offset_data[src_offset] - src_offset_data[src_offset - 1]; - dst_offsets_data[dst_offsets_pos++] = total_chars_size; + // if Offsets is uint32, size will not exceed range of uint32, cast is OK. + dst_offsets_data[dst_offsets_pos++] = static_cast(total_chars_size); } check_chars_length(total_chars_size, offsets.size()); @@ -267,13 +271,16 @@ void ColumnStr::update_crcs_with_value(uint32_t* __restrict hashes, doris::Pr if (null_data == nullptr) { for (size_t i = 0; i < s; i++) { auto data_ref = get_data_at(i); - hashes[i] = HashUtil::zlib_crc_hash(data_ref.data, data_ref.size, hashes[i]); + // If offset is uint32, size will not exceed, check the size when inserting data into ColumnStr. + hashes[i] = HashUtil::zlib_crc_hash(data_ref.data, static_cast(data_ref.size), + hashes[i]); } } else { for (size_t i = 0; i < s; i++) { if (null_data[i] == 0) { auto data_ref = get_data_at(i); - hashes[i] = HashUtil::zlib_crc_hash(data_ref.data, data_ref.size, hashes[i]); + hashes[i] = HashUtil::zlib_crc_hash( + data_ref.data, static_cast(data_ref.size), hashes[i]); } } } @@ -391,8 +398,9 @@ ColumnPtr ColumnStr::permute(const IColumn::Permutation& perm, size_t limit) template StringRef ColumnStr::serialize_value_into_arena(size_t n, Arena& arena, char const*& begin) const { - uint32_t string_size(size_at(n)); - uint32_t offset(offset_at(n)); + // Use uint32 instead of size_t to reduce agg key's length. + auto string_size(static_cast(size_at(n))); + auto offset(static_cast(offset_at(n))); StringRef res; res.size = sizeof(string_size) + string_size; @@ -421,7 +429,7 @@ const char* ColumnStr::deserialize_and_insert_from_arena(const char* pos) { template size_t ColumnStr::get_max_row_byte_size() const { - size_t max_size = 0; + T max_size = 0; size_t num_rows = offsets.size(); for (size_t i = 0; i < num_rows; ++i) { max_size = std::max(max_size, size_at(i)); @@ -434,8 +442,9 @@ template void ColumnStr::serialize_vec(std::vector& keys, size_t num_rows, size_t max_row_byte_size) const { for (size_t i = 0; i < num_rows; ++i) { - uint32_t offset(offset_at(i)); - uint32_t string_size(size_at(i)); + // Use uint32 instead of size_t to reduce agg key's length. + auto offset(static_cast(offset_at(i))); + auto string_size(static_cast(size_at(i))); auto* ptr = const_cast(keys[i].data + keys[i].size); memcpy_fixed(ptr, (char*)&string_size); @@ -458,8 +467,8 @@ void ColumnStr::serialize_vec_with_null_map(std::vector& keys, siz memcpy(dest, null_map + i, sizeof(uint8_t)); if (null_map[i] == 0) { - UInt32 offset(offset_at(i)); - UInt32 string_size(size_at(i)); + auto offset(offset_at(i)); + auto string_size(size_at(i)); memcpy_fixed(dest + 1, (char*)&string_size); memcpy(dest + 1 + sizeof(string_size), &chars[offset], string_size); @@ -475,8 +484,8 @@ void ColumnStr::serialize_vec_with_null_map(std::vector& keys, siz // serialize null first memcpy(dest, null_map + i, sizeof(uint8_t)); - UInt32 offset(offset_at(i)); - UInt32 string_size(size_at(i)); + auto offset(offset_at(i)); + auto string_size(size_at(i)); memcpy_fixed(dest + 1, (char*)&string_size); memcpy(dest + 1 + sizeof(string_size), &chars[offset], string_size); @@ -559,8 +568,8 @@ ColumnPtr ColumnStr::replicate(const IColumn::Offsets& replicate_offsets) con T current_new_offset = 0; for (size_t i = 0; i < col_size; ++i) { - size_t size_to_replicate = replicate_offsets[i] - prev_replicate_offset; - size_t string_size = offsets[i] - prev_string_offset; + T size_to_replicate = replicate_offsets[i] - prev_replicate_offset; + T string_size = offsets[i] - prev_string_offset; for (size_t j = 0; j < size_to_replicate; ++j) { current_new_offset += string_size; diff --git a/be/src/vec/columns/column_string.h b/be/src/vec/columns/column_string.h index f116d4ce1f17cf..1674fd90933dbe 100644 --- a/be/src/vec/columns/column_string.h +++ b/be/src/vec/columns/column_string.h @@ -47,6 +47,7 @@ #include "vec/core/types.h" namespace doris::vectorized { +#include "common/compile_check_begin.h" class Arena; class ColumnSorter; @@ -86,10 +87,10 @@ class ColumnStr final : public COWHelper> { Chars chars; // Start position of i-th element. - size_t ALWAYS_INLINE offset_at(ssize_t i) const { return offsets[i - 1]; } + T ALWAYS_INLINE offset_at(ssize_t i) const { return offsets[i - 1]; } /// Size of i-th element, including terminating zero. - size_t ALWAYS_INLINE size_at(ssize_t i) const { return offsets[i] - offsets[i - 1]; } + T ALWAYS_INLINE size_at(ssize_t i) const { return offsets[i] - offsets[i - 1]; } template struct less; @@ -220,7 +221,7 @@ class ColumnStr final : public COWHelper> { const char* ptr = strings[0].data; for (size_t i = 0; i != num; i++) { - uint32_t len = strings[i].size; + size_t len = strings[i].size; length += len; offset += len; offsets.push_back(offset); @@ -282,7 +283,7 @@ class ColumnStr final : public COWHelper> { Char* data = chars.data(); size_t offset = old_size; for (size_t i = 0; i < num; i++) { - uint32_t len = strings[i].size; + size_t len = strings[i].size; if (len) { memcpy(data + offset, strings[i].data, len); offset += len; @@ -305,7 +306,7 @@ class ColumnStr final : public COWHelper> { Char* data = chars.data(); size_t offset = old_size; for (size_t i = 0; i < num; i++) { - uint32_t len = strings[i].size; + size_t len = strings[i].size; if (len) { memcpy(data + offset, strings[i].data, copy_length); offset += len; @@ -342,9 +343,15 @@ class ColumnStr final : public COWHelper> { for (size_t i = 0; i < num; i++) { int32_t codeword = data_array[i + start_index]; new_size += dict[codeword].size; - offsets[offset_size + i] = new_size; + offsets[offset_size + i] = static_cast(new_size); } + if (new_size > std::numeric_limits::max()) { + throw doris::Exception(ErrorCode::INTERNAL_ERROR, + "ColumnString insert size out of range type {} [{},{}]", + typeid(T).name(), std::numeric_limits::min(), + std::numeric_limits::max()); + } check_chars_length(new_size, offsets.size()); chars.resize(new_size); @@ -406,13 +413,16 @@ class ColumnStr final : public COWHelper> { for (size_t i = start; i < end; ++i) { if (null_data[i] == 0) { auto data_ref = get_data_at(i); - hash = HashUtil::zlib_crc_hash(data_ref.data, data_ref.size, hash); + // If offset is uint32, size will not exceed, check the size when inserting data into ColumnStr. + hash = HashUtil::zlib_crc_hash(data_ref.data, + static_cast(data_ref.size), hash); } } } else { for (size_t i = start; i < end; ++i) { auto data_ref = get_data_at(i); - hash = HashUtil::zlib_crc_hash(data_ref.data, data_ref.size, hash); + hash = HashUtil::zlib_crc_hash(data_ref.data, static_cast(data_ref.size), + hash); } } } @@ -473,7 +483,7 @@ class ColumnStr final : public COWHelper> { void insert_default() override { offsets.push_back(chars.size()); } void insert_many_defaults(size_t length) override { - offsets.resize_fill(offsets.size() + length, chars.size()); + offsets.resize_fill(offsets.size() + length, static_cast(chars.size())); } int compare_at(size_t n, size_t m, const IColumn& rhs_, @@ -525,3 +535,4 @@ class ColumnStr final : public COWHelper> { using ColumnString = ColumnStr; using ColumnString64 = ColumnStr; } // namespace doris::vectorized +#include "common/compile_check_end.h" diff --git a/be/src/vec/columns/column_vector.cpp b/be/src/vec/columns/column_vector.cpp index 0e24446a5cdb92..f0f0bec8b99bb1 100644 --- a/be/src/vec/columns/column_vector.cpp +++ b/be/src/vec/columns/column_vector.cpp @@ -42,6 +42,7 @@ #include "vec/data_types/data_type.h" namespace doris::vectorized { +#include "common/compile_check_begin.h" template StringRef ColumnVector::serialize_value_into_arena(size_t n, Arena& arena, @@ -242,7 +243,7 @@ void ColumnVector::get_permutation(bool reverse, size_t limit, int nan_direct if (s == 0) return; // std::partial_sort need limit << s can get performance benefit - if (limit > (s / 8.0)) limit = 0; + if (limit > (s / 8.0L)) limit = 0; if (limit) { for (size_t i = 0; i < s; ++i) res[i] = i; diff --git a/be/src/vec/columns/column_vector.h b/be/src/vec/columns/column_vector.h index 2cb320b6992095..970997a91864cc 100644 --- a/be/src/vec/columns/column_vector.h +++ b/be/src/vec/columns/column_vector.h @@ -61,6 +61,7 @@ class ColumnSorter; } // namespace doris namespace doris::vectorized { +#include "common/compile_check_begin.h" /** Stuff for comparing numbers. * Integer values are compared as usual. @@ -178,10 +179,9 @@ class ColumnVector final : public COWHelper> { void insert_range_of_integer(T begin, T end) { if constexpr (std::is_integral_v) { auto old_size = data.size(); - data.resize(old_size + (end - begin)); - for (int i = 0; i < end - begin; i++) { - data[old_size + i] = begin + i; - } + auto new_size = old_size + static_cast(end - begin); + data.resize(new_size); + std::iota(data.begin() + old_size, data.begin() + new_size, begin); } else { throw doris::Exception(ErrorCode::INTERNAL_ERROR, "double column not support insert_range_of_integer"); @@ -409,3 +409,4 @@ class ColumnVector final : public COWHelper> { }; } // namespace doris::vectorized +#include "common/compile_check_end.h" diff --git a/be/src/vec/core/field.h b/be/src/vec/core/field.h index 8113dc602fbd4e..341f65e075ed11 100644 --- a/be/src/vec/core/field.h +++ b/be/src/vec/core/field.h @@ -165,7 +165,7 @@ class JsonbField { public: JsonbField() = default; - JsonbField(const char* ptr, uint32_t len) : size(len) { + JsonbField(const char* ptr, size_t len) : size(len) { data = new char[size]; if (!data) { LOG(FATAL) << "new data buffer failed, size: " << size; @@ -213,7 +213,7 @@ class JsonbField { } const char* get_value() const { return data; } - uint32_t get_size() const { return size; } + size_t get_size() const { return size; } bool operator<(const JsonbField& r) const { LOG(FATAL) << "comparing between JsonbField is not supported"; @@ -252,7 +252,7 @@ class JsonbField { private: char* data = nullptr; - uint32_t size = 0; + size_t size = 0; }; template @@ -498,6 +498,9 @@ class Field { bool is_null() const { return which == Types::Null; } + // The template parameter T needs to be consistent with `which`. + // If not, use NearestFieldType<> externally. + // Maybe modify this in the future, reference: https://github.com/ClickHouse/ClickHouse/pull/22003 template T& get() { using TWithoutRef = std::remove_reference_t; @@ -520,6 +523,8 @@ class Field { return true; } + // The template parameter T needs to be consistent with `which`. + // If not, use NearestFieldType<> externally. template bool try_get(T& result) const { const Types::Which requested = TypeToEnum>::value; From 9b9bd9ee0e152aaa84d4933d2973dfc2ce11319d Mon Sep 17 00:00:00 2001 From: wangbo Date: Thu, 28 Nov 2024 10:09:07 +0800 Subject: [PATCH 043/399] [Improment]log more detail when query hits policy (#44685) --- .../workload_management/workload_action.cpp | 2 +- .../workload_management/workload_condition.h | 20 +++++++++++++++++++ .../workload_management/workload_query_info.h | 3 ++- .../workload_sched_policy.cpp | 5 +++++ 4 files changed, 28 insertions(+), 2 deletions(-) diff --git a/be/src/runtime/workload_management/workload_action.cpp b/be/src/runtime/workload_management/workload_action.cpp index 8e6e3b19e2c385..77042b074fd624 100644 --- a/be/src/runtime/workload_management/workload_action.cpp +++ b/be/src/runtime/workload_management/workload_action.cpp @@ -25,7 +25,7 @@ void WorkloadActionCancelQuery::exec(WorkloadQueryInfo* query_info) { std::stringstream msg; msg << "query " << query_info->query_id << " cancelled by workload policy: " << query_info->policy_name - << ", id:" << query_info->policy_id; + << ", id:" << query_info->policy_id << ", " << query_info->cond_eval_msg; std::string msg_str = msg.str(); LOG(INFO) << "[workload_schedule]" << msg_str; ExecEnv::GetInstance()->fragment_mgr()->cancel_query(query_info->tquery_id, diff --git a/be/src/runtime/workload_management/workload_condition.h b/be/src/runtime/workload_management/workload_condition.h index a85268a8dc3a6a..cf53a5f07ddf9d 100644 --- a/be/src/runtime/workload_management/workload_condition.h +++ b/be/src/runtime/workload_management/workload_condition.h @@ -33,6 +33,10 @@ class WorkloadCondition { virtual bool eval(std::string str_val) = 0; virtual WorkloadMetricType get_workload_metric_type() = 0; + + virtual std::string get_metric_string() = 0; + + virtual std::string get_metric_value_string() = 0; }; class WorkloadConditionQueryTime : public WorkloadCondition { @@ -45,6 +49,10 @@ class WorkloadConditionQueryTime : public WorkloadCondition { return WorkloadMetricType::QUERY_TIME; } + std::string get_metric_string() override { return "query_time"; } + + std::string get_metric_value_string() override { return std::to_string(_query_time); } + private: int64_t _query_time; WorkloadCompareOperator _op; @@ -56,6 +64,10 @@ class WorkloadConditionScanRows : public WorkloadCondition { bool eval(std::string str_val) override; WorkloadMetricType get_workload_metric_type() override { return WorkloadMetricType::SCAN_ROWS; } + std::string get_metric_string() override { return "scan_rows"; } + + std::string get_metric_value_string() override { return std::to_string(_scan_rows); } + private: int64_t _scan_rows; WorkloadCompareOperator _op; @@ -69,6 +81,10 @@ class WorkloadConditionScanBytes : public WorkloadCondition { return WorkloadMetricType::SCAN_BYTES; } + std::string get_metric_string() override { return "scan_bytes"; } + + std::string get_metric_value_string() override { return std::to_string(_scan_bytes); } + private: int64_t _scan_bytes; WorkloadCompareOperator _op; @@ -82,6 +98,10 @@ class WorkloadConditionQueryMemory : public WorkloadCondition { return WorkloadMetricType::QUERY_MEMORY_BYTES; } + std::string get_metric_string() override { return "query_memory"; } + + std::string get_metric_value_string() override { return std::to_string(_query_memory_bytes); } + private: int64_t _query_memory_bytes; WorkloadCompareOperator _op; diff --git a/be/src/runtime/workload_management/workload_query_info.h b/be/src/runtime/workload_management/workload_query_info.h index e544668e1039ed..16151eec390746 100644 --- a/be/src/runtime/workload_management/workload_query_info.h +++ b/be/src/runtime/workload_management/workload_query_info.h @@ -30,7 +30,8 @@ class WorkloadQueryInfo { std::string query_id; int64_t wg_id; int64_t policy_id; - std::string policy_name; + std::string policy_name {""}; + std::string cond_eval_msg {""}; }; } // namespace doris \ No newline at end of file diff --git a/be/src/runtime/workload_management/workload_sched_policy.cpp b/be/src/runtime/workload_management/workload_sched_policy.cpp index efa8965dd77121..63b9362bc217be 100644 --- a/be/src/runtime/workload_management/workload_sched_policy.cpp +++ b/be/src/runtime/workload_management/workload_sched_policy.cpp @@ -60,6 +60,7 @@ bool WorkloadSchedPolicy::is_match(WorkloadQueryInfo* query_info_ptr) { } auto& metric_val_map = query_info_ptr->metric_map; + std::string cond_eval_msg = ""; for (auto& cond : _condition_list) { if (metric_val_map.find(cond->get_workload_metric_type()) == metric_val_map.end()) { return false; @@ -69,7 +70,11 @@ bool WorkloadSchedPolicy::is_match(WorkloadQueryInfo* query_info_ptr) { if (!cond->eval(val)) { return false; } + cond_eval_msg += cond->get_metric_string() + ":" + val + "(" + + cond->get_metric_value_string() + "), "; } + cond_eval_msg = cond_eval_msg.substr(0, cond_eval_msg.size() - 2); + query_info_ptr->cond_eval_msg = cond_eval_msg; return true; } From f1d37cb951ee4b83bc9e3ff1d6dca7fb5bf37ac5 Mon Sep 17 00:00:00 2001 From: walter Date: Thu, 28 Nov 2024 10:13:12 +0800 Subject: [PATCH 044/399] [improve](backup) Limit the output size of show backup/restore job progress (#44681) A backup/restore job might involve millions of tablets, and the output of the show backup/restore command might be huge without limitation. --- .../org/apache/doris/alter/RollupJobV2.java | 4 ++-- .../org/apache/doris/backup/BackupJob.java | 22 ++++++++++++++----- .../org/apache/doris/backup/RestoreJob.java | 22 ++++++++++++++----- 3 files changed, 34 insertions(+), 14 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java b/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java index 8eb87050bf6d77..1d09224e6fa38f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java @@ -570,7 +570,7 @@ protected void runRunningJob() throws AlterCancelException { continue; } - long visiableVersion = partition.getVisibleVersion(); + long visibleVersion = partition.getVisibleVersion(); short expectReplicationNum = tbl.getPartitionInfo().getReplicaAllocation( partitionId).getTotalReplicaNum(); MaterializedIndex rollupIndex = entry.getValue(); @@ -579,7 +579,7 @@ protected void runRunningJob() throws AlterCancelException { int healthyReplicaNum = 0; for (Replica replica : replicas) { if (!replica.isBad() && replica.getLastFailedVersion() < 0 - && replica.checkVersionCatchUp(visiableVersion, false)) { + && replica.checkVersionCatchUp(visibleVersion, false)) { healthyReplicaNum++; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/backup/BackupJob.java b/fe/fe-core/src/main/java/org/apache/doris/backup/BackupJob.java index 621a2b1d9f7d29..d0a957dcd9ae0d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/backup/BackupJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/backup/BackupJob.java @@ -1059,6 +1059,19 @@ public synchronized Snapshot getSnapshot() { } public synchronized List getInfo() { + String unfinishedTaskIdsStr = unfinishedTaskIds.entrySet().stream() + .map(e -> "[" + e.getKey() + "=" + e.getValue() + "]") + .limit(100) + .collect(Collectors.joining(", ")); + String taskProgressStr = taskProgress.entrySet().stream() + .map(e -> "[" + e.getKey() + ": " + e.getValue().first + "/" + e.getValue().second + "]") + .limit(100) + .collect(Collectors.joining(", ")); + String taskErrMsgStr = taskErrMsg.entrySet().stream() + .map(e -> "[" + e.getKey() + ": " + e.getValue() + "]") + .limit(100) + .collect(Collectors.joining(", ")); + List info = Lists.newArrayList(); info.add(String.valueOf(jobId)); info.add(label); @@ -1069,12 +1082,9 @@ public synchronized List getInfo() { info.add(TimeUtils.longToTimeString(snapshotFinishedTime)); info.add(TimeUtils.longToTimeString(snapshotUploadFinishedTime)); info.add(TimeUtils.longToTimeString(finishedTime)); - info.add(Joiner.on(", ").join(unfinishedTaskIds.entrySet())); - info.add(Joiner.on(", ").join(taskProgress.entrySet().stream().map( - e -> "[" + e.getKey() + ": " + e.getValue().first + "/" + e.getValue().second + "]").collect( - Collectors.toList()))); - info.add(Joiner.on(", ").join(taskErrMsg.entrySet().stream().map(n -> "[" + n.getKey() + ": " + n.getValue() - + "]").collect(Collectors.toList()))); + info.add(unfinishedTaskIdsStr); + info.add(taskProgressStr); + info.add(taskErrMsgStr); info.add(status.toString()); info.add(String.valueOf(timeoutMs / 1000)); return info; diff --git a/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java b/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java index 0b5b0cb03390f3..f1abb0c9e632bc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java @@ -2208,6 +2208,11 @@ public List getFullInfo() { } public synchronized List getInfo(boolean isBrief) { + String unfinishedTaskIdsStr = unfinishedSignatureToId.entrySet().stream() + .map(e -> "[" + e.getKey() + "=" + e.getValue() + "]") + .limit(100) + .collect(Collectors.joining(", ")); + List info = Lists.newArrayList(); info.add(String.valueOf(jobId)); info.add(label); @@ -2227,13 +2232,18 @@ public synchronized List getInfo(boolean isBrief) { info.add(TimeUtils.longToTimeString(snapshotFinishedTime)); info.add(TimeUtils.longToTimeString(downloadFinishedTime)); info.add(TimeUtils.longToTimeString(finishedTime)); - info.add(Joiner.on(", ").join(unfinishedSignatureToId.entrySet())); + info.add(unfinishedTaskIdsStr); if (!isBrief) { - info.add(Joiner.on(", ").join(taskProgress.entrySet().stream().map( - e -> "[" + e.getKey() + ": " + e.getValue().first + "/" + e.getValue().second + "]").collect( - Collectors.toList()))); - info.add(Joiner.on(", ").join(taskErrMsg.entrySet().stream().map(n -> "[" + n.getKey() + ": " - + n.getValue() + "]").collect(Collectors.toList()))); + String taskProgressStr = taskProgress.entrySet().stream() + .map(e -> "[" + e.getKey() + ": " + e.getValue().first + "/" + e.getValue().second + "]") + .limit(100) + .collect(Collectors.joining(", ")); + String taskErrMsgStr = taskErrMsg.entrySet().stream() + .map(n -> "[" + n.getKey() + ": " + n.getValue() + "]") + .limit(100) + .collect(Collectors.joining(", ")); + info.add(taskProgressStr); + info.add(taskErrMsgStr); } info.add(status.toString()); info.add(String.valueOf(timeoutMs / 1000)); From 913cda6a96351ac58e22e628d16100574c49b0b8 Mon Sep 17 00:00:00 2001 From: zhangdong Date: Thu, 28 Nov 2024 10:22:20 +0800 Subject: [PATCH 045/399] [feat](mtmv)Unified external table interface supporting partition refresh and partition pruning (#44673) ### What problem does this PR solve? - Add `MvccTable` to represent a table that supports querying specified version data - Add the `MvccSnapshot` interface to store snapshot information of mvcc at a certain moment in time - Add the `MvccSnapshot` parameter to the method of the `MTMVRelatedTableIf `interface to retrieve data of a specified version - Partition pruning related methods combined with the `MvccSnapshot` parameter are used to obtain partition information for a specified version - Load the snapshot information of mvccTable at the beginning of the query plan and store it in StatementContext ### Release note Unified external table interface supporting partition refresh and partition pruning --- .../java/org/apache/doris/catalog/MTMV.java | 7 +- .../org/apache/doris/catalog/OlapTable.java | 25 +++++- .../doris/datasource/ExternalTable.java | 18 ++-- .../datasource/hive/HMSExternalTable.java | 33 ++++---- .../doris/datasource/mvcc/MvccSnapshot.java | 25 ++++++ .../doris/datasource/mvcc/MvccTable.java | 33 ++++++++ .../doris/datasource/mvcc/MvccTableInfo.java | 84 +++++++++++++++++++ .../paimon/PaimonExternalTable.java | 14 ++-- .../mtmv/MTMVPartitionExprDateTrunc.java | 2 +- .../apache/doris/mtmv/MTMVPartitionInfo.java | 3 +- .../apache/doris/mtmv/MTMVPartitionUtil.java | 12 +-- ...MTMVRelatedPartitionDescInitGenerator.java | 4 +- ...MVRelatedPartitionDescRollUpGenerator.java | 3 +- .../apache/doris/mtmv/MTMVRelatedTableIf.java | 27 +++--- .../apache/doris/nereids/CascadesContext.java | 8 ++ .../apache/doris/nereids/NereidsPlanner.java | 2 +- .../doris/nereids/StatementContext.java | 31 +++++++ .../exploration/mv/MaterializedViewUtils.java | 4 +- .../rules/rewrite/PruneFileScanPartition.java | 6 +- .../plans/commands/info/CreateMTMVInfo.java | 2 +- .../info/MTMVPartitionDefinition.java | 3 +- .../trees/plans/logical/LogicalFileScan.java | 3 +- .../doris/mtmv/MTMVPartitionUtilTest.java | 6 +- 23 files changed, 278 insertions(+), 77 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/mvcc/MvccSnapshot.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/mvcc/MvccTable.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/mvcc/MvccTableInfo.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java index 825bdef9f09819..955bfd4279fd5c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java @@ -59,7 +59,6 @@ import java.util.Map; import java.util.Map.Entry; import java.util.Optional; -import java.util.OptionalLong; import java.util.Set; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.locks.ReentrantReadWriteLock; @@ -364,7 +363,7 @@ public MTMVRefreshSnapshot getRefreshSnapshot() { * @return mvPartitionName ==> mvPartitionKeyDesc */ public Map generateMvPartitionDescs() throws AnalysisException { - Map mtmvItems = getAndCopyPartitionItems(OptionalLong.empty()); + Map mtmvItems = getAndCopyPartitionItems(); Map result = Maps.newHashMap(); for (Entry entry : mtmvItems.entrySet()) { result.put(entry.getKey(), entry.getValue().toPartitionKeyDesc()); @@ -393,7 +392,7 @@ public Pair>, Map> calculateDoublyPartit Map baseToMv = Maps.newHashMap(); Map> relatedPartitionDescs = MTMVPartitionUtil .generateRelatedPartitionDescs(mvPartitionInfo, mvProperties); - Map mvPartitionItems = getAndCopyPartitionItems(OptionalLong.empty()); + Map mvPartitionItems = getAndCopyPartitionItems(); for (Entry entry : mvPartitionItems.entrySet()) { Set basePartitionNames = relatedPartitionDescs.getOrDefault(entry.getValue().toPartitionKeyDesc(), Sets.newHashSet()); @@ -426,7 +425,7 @@ public Map> calculatePartitionMappings() throws AnalysisExce Map> res = Maps.newHashMap(); Map> relatedPartitionDescs = MTMVPartitionUtil .generateRelatedPartitionDescs(mvPartitionInfo, mvProperties); - Map mvPartitionItems = getAndCopyPartitionItems(OptionalLong.empty()); + Map mvPartitionItems = getAndCopyPartitionItems(); for (Entry entry : mvPartitionItems.entrySet()) { res.put(entry.getKey(), relatedPartitionDescs.getOrDefault(entry.getValue().toPartitionKeyDesc(), Sets.newHashSet())); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java index ec3bd2acbc57d6..5d57540017f067 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java @@ -50,6 +50,7 @@ import org.apache.doris.common.util.PropertyAnalyzer; import org.apache.doris.common.util.Util; import org.apache.doris.datasource.InternalCatalog; +import org.apache.doris.datasource.mvcc.MvccSnapshot; import org.apache.doris.mtmv.MTMVRefreshContext; import org.apache.doris.mtmv.MTMVRelatedTableIf; import org.apache.doris.mtmv.MTMVSnapshotIf; @@ -110,7 +111,6 @@ import java.util.Map.Entry; import java.util.Objects; import java.util.Optional; -import java.util.OptionalLong; import java.util.Set; import java.util.TreeMap; import java.util.concurrent.ConcurrentHashMap; @@ -1049,6 +1049,10 @@ public PartitionInfo getPartitionInfo() { } @Override + public Set getPartitionColumnNames(Optional snapshot) throws DdlException { + return getPartitionColumnNames(); + } + public Set getPartitionColumnNames() throws DdlException { Set partitionColumnNames = Sets.newHashSet(); if (partitionInfo instanceof SinglePartitionInfo) { @@ -3251,12 +3255,21 @@ public long getVisibleVersionTime() { } @Override + public PartitionType getPartitionType(Optional snapshot) { + return getPartitionType(); + } + public PartitionType getPartitionType() { return partitionInfo.getType(); } @Override - public Map getAndCopyPartitionItems(OptionalLong snapshotId) throws AnalysisException { + public Map getAndCopyPartitionItems(Optional snapshot) + throws AnalysisException { + return getAndCopyPartitionItems(); + } + + public Map getAndCopyPartitionItems() throws AnalysisException { if (!tryReadLock(1, TimeUnit.MINUTES)) { throw new AnalysisException("get table read lock timeout, database=" + getDBName() + ",table=" + getName()); } @@ -3275,13 +3288,17 @@ public Map getAndCopyPartitionItems(OptionalLong snapshot } @Override + public List getPartitionColumns(Optional snapshot) { + return getPartitionColumns(); + } + public List getPartitionColumns() { return getPartitionInfo().getPartitionColumns(); } @Override public MTMVSnapshotIf getPartitionSnapshot(String partitionName, MTMVRefreshContext context, - OptionalLong snapshotId) + Optional snapshot) throws AnalysisException { Map partitionVersions = context.getBaseVersions().getPartitionVersions(); long partitionId = getPartitionOrAnalysisException(partitionName).getId(); @@ -3291,7 +3308,7 @@ public MTMVSnapshotIf getPartitionSnapshot(String partitionName, MTMVRefreshCont } @Override - public MTMVSnapshotIf getTableSnapshot(MTMVRefreshContext context, OptionalLong snapshotId) { + public MTMVSnapshotIf getTableSnapshot(MTMVRefreshContext context, Optional snapshot) { Map tableVersions = context.getBaseVersions().getTableVersions(); long visibleVersion = tableVersions.containsKey(id) ? tableVersions.get(id) : getVisibleVersion(); return new MTMVVersionSnapshot(visibleVersion, id); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalTable.java index 3aee5550acf646..d82959954f2607 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalTable.java @@ -31,6 +31,7 @@ import org.apache.doris.common.io.Writable; import org.apache.doris.common.util.PropertyAnalyzer; import org.apache.doris.common.util.Util; +import org.apache.doris.datasource.mvcc.MvccSnapshot; import org.apache.doris.nereids.trees.plans.logical.LogicalFileScan.SelectedPartitions; import org.apache.doris.persist.gson.GsonPostProcessable; import org.apache.doris.persist.gson.GsonUtils; @@ -55,7 +56,6 @@ import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.OptionalLong; import java.util.Set; /** @@ -373,17 +373,17 @@ public TableIndexes getTableIndexes() { /** * Retrieve all partitions and initialize SelectedPartitions * - * @param snapshotId if not support mvcc, ignore this + * @param snapshot if not support mvcc, ignore this * @return */ - public SelectedPartitions initSelectedPartitions(OptionalLong snapshotId) { + public SelectedPartitions initSelectedPartitions(Optional snapshot) { if (!supportPartitionPruned()) { return SelectedPartitions.NOT_PRUNED; } - if (CollectionUtils.isEmpty(this.getPartitionColumns(snapshotId))) { + if (CollectionUtils.isEmpty(this.getPartitionColumns(snapshot))) { return SelectedPartitions.NOT_PRUNED; } - Map nameToPartitionItems = getNameToPartitionItems(snapshotId); + Map nameToPartitionItems = getNameToPartitionItems(snapshot); return new SelectedPartitions(nameToPartitionItems.size(), nameToPartitionItems, false); } @@ -391,10 +391,10 @@ public SelectedPartitions initSelectedPartitions(OptionalLong snapshotId) { * get partition map * If partition related operations are supported, this method needs to be implemented in the subclass * - * @param snapshotId if not support mvcc, ignore this + * @param snapshot if not support mvcc, ignore this * @return partitionName ==> PartitionItem */ - public Map getNameToPartitionItems(OptionalLong snapshotId) { + public Map getNameToPartitionItems(Optional snapshot) { return Collections.emptyMap(); } @@ -402,10 +402,10 @@ public Map getNameToPartitionItems(OptionalLong snapshotI * get partition column list * If partition related operations are supported, this method needs to be implemented in the subclass * - * @param snapshotId if not support mvcc, ignore this + * @param snapshot if not support mvcc, ignore this * @return */ - public List getPartitionColumns(OptionalLong snapshotId) { + public List getPartitionColumns(Optional snapshot) { return Collections.emptyList(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalTable.java index 1c30fa24cfb51e..6d65f8bcdbccb7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalTable.java @@ -33,6 +33,7 @@ import org.apache.doris.datasource.SchemaCacheValue; import org.apache.doris.datasource.hudi.HudiUtils; import org.apache.doris.datasource.iceberg.IcebergUtils; +import org.apache.doris.datasource.mvcc.MvccSnapshot; import org.apache.doris.mtmv.MTMVBaseTableIf; import org.apache.doris.mtmv.MTMVMaxTimestampSnapshot; import org.apache.doris.mtmv.MTMVRefreshContext; @@ -83,7 +84,6 @@ import java.util.Map; import java.util.Map.Entry; import java.util.Optional; -import java.util.OptionalLong; import java.util.Set; import java.util.stream.Collectors; @@ -288,7 +288,6 @@ public List getPartitionColumnTypes() { .orElse(Collections.emptyList()); } - @Override public List getPartitionColumns() { makeSureInitialized(); Optional schemaCacheValue = getSchemaCacheValue(); @@ -297,7 +296,7 @@ public List getPartitionColumns() { } @Override - public List getPartitionColumns(OptionalLong snapshotId) { + public List getPartitionColumns(Optional snapshot) { return getPartitionColumns(); } @@ -307,7 +306,7 @@ public boolean supportPartitionPruned() { } @Override - public Map getNameToPartitionItems(OptionalLong snapshotId) { + public Map getNameToPartitionItems(Optional snapshot) { return getNameToPartitionItems(); } @@ -756,34 +755,32 @@ public Set getDistributionColumnNames() { } @Override + public PartitionType getPartitionType(Optional snapshot) { + return getPartitionType(); + } + public PartitionType getPartitionType() { return getPartitionColumns().size() > 0 ? PartitionType.LIST : PartitionType.UNPARTITIONED; } @Override + public Set getPartitionColumnNames(Optional snapshot) { + return getPartitionColumnNames(); + } + public Set getPartitionColumnNames() { return getPartitionColumns().stream() .map(c -> c.getName().toLowerCase()).collect(Collectors.toSet()); } @Override - public Map getAndCopyPartitionItems(OptionalLong snapshotId) { - HiveMetaStoreCache cache = Env.getCurrentEnv().getExtMetaCacheMgr() - .getMetaStoreCache((HMSExternalCatalog) getCatalog()); - HiveMetaStoreCache.HivePartitionValues hivePartitionValues = cache.getPartitionValues( - getDbName(), getName(), getPartitionColumnTypes()); - Map res = Maps.newHashMap(); - Map idToPartitionItem = hivePartitionValues.getIdToPartitionItem(); - BiMap idToName = hivePartitionValues.getPartitionNameToIdMap().inverse(); - for (Entry entry : idToPartitionItem.entrySet()) { - res.put(idToName.get(entry.getKey()), entry.getValue()); - } - return res; + public Map getAndCopyPartitionItems(Optional snapshot) { + return getNameToPartitionItems(); } @Override public MTMVSnapshotIf getPartitionSnapshot(String partitionName, MTMVRefreshContext context, - OptionalLong snapshotId) throws AnalysisException { + Optional snapshot) throws AnalysisException { HiveMetaStoreCache cache = Env.getCurrentEnv().getExtMetaCacheMgr() .getMetaStoreCache((HMSExternalCatalog) getCatalog()); HiveMetaStoreCache.HivePartitionValues hivePartitionValues = cache.getPartitionValues( @@ -795,7 +792,7 @@ public MTMVSnapshotIf getPartitionSnapshot(String partitionName, MTMVRefreshCont } @Override - public MTMVSnapshotIf getTableSnapshot(MTMVRefreshContext context, OptionalLong snapshotId) + public MTMVSnapshotIf getTableSnapshot(MTMVRefreshContext context, Optional snapshot) throws AnalysisException { if (getPartitionType() == PartitionType.UNPARTITIONED) { return new MTMVMaxTimestampSnapshot(getName(), getLastDdlTime()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/mvcc/MvccSnapshot.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/mvcc/MvccSnapshot.java new file mode 100644 index 00000000000000..d7826b0a5de19e --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/mvcc/MvccSnapshot.java @@ -0,0 +1,25 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.datasource.mvcc; + +/** + * The snapshot information of mvcc is defined by each table, + * but it should be ensured that the table information queried through this snapshot remains unchanged + */ +public interface MvccSnapshot { +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/mvcc/MvccTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/mvcc/MvccTable.java new file mode 100644 index 00000000000000..d69e0f3114df0c --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/mvcc/MvccTable.java @@ -0,0 +1,33 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.datasource.mvcc; + +import org.apache.doris.catalog.TableIf; + +/** + * The table that needs to query data based on the version needs to implement this interface. + */ +public interface MvccTable extends TableIf { + /** + * Retrieve the current snapshot information of the table, + * and the returned result will be used for the entire process of this query + * + * @return MvccSnapshot + */ + MvccSnapshot loadSnapshot(); +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/mvcc/MvccTableInfo.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/mvcc/MvccTableInfo.java new file mode 100644 index 00000000000000..0d865f837c8c4e --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/mvcc/MvccTableInfo.java @@ -0,0 +1,84 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.datasource.mvcc; + +import org.apache.doris.catalog.DatabaseIf; +import org.apache.doris.catalog.TableIf; +import org.apache.doris.datasource.CatalogIf; + +import com.google.common.base.Objects; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +public class MvccTableInfo { + private static final Logger LOG = LogManager.getLogger(MvccTableInfo.class); + + private String tableName; + private String dbName; + private String ctlName; + + public MvccTableInfo(TableIf table) { + java.util.Objects.requireNonNull(table, "table is null"); + DatabaseIf database = table.getDatabase(); + java.util.Objects.requireNonNull(database, "database is null"); + CatalogIf catalog = database.getCatalog(); + java.util.Objects.requireNonNull(database, "catalog is null"); + this.tableName = table.getName(); + this.dbName = database.getFullName(); + this.ctlName = catalog.getName(); + } + + public String getTableName() { + return tableName; + } + + public String getDbName() { + return dbName; + } + + public String getCtlName() { + return ctlName; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + MvccTableInfo that = (MvccTableInfo) o; + return Objects.equal(tableName, that.tableName) && Objects.equal( + dbName, that.dbName) && Objects.equal(ctlName, that.ctlName); + } + + @Override + public int hashCode() { + return Objects.hashCode(tableName, dbName, ctlName); + } + + @Override + public String toString() { + return "MvccTableInfo{" + + "tableName='" + tableName + '\'' + + ", dbName='" + dbName + '\'' + + ", ctlName='" + ctlName + '\'' + + '}'; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonExternalTable.java index 632a0da0ebd316..7fe3c858448e3f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonExternalTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonExternalTable.java @@ -28,6 +28,7 @@ import org.apache.doris.common.DdlException; import org.apache.doris.datasource.ExternalTable; import org.apache.doris.datasource.SchemaCacheValue; +import org.apache.doris.datasource.mvcc.MvccSnapshot; import org.apache.doris.mtmv.MTMVBaseTableIf; import org.apache.doris.mtmv.MTMVRefreshContext; import org.apache.doris.mtmv.MTMVRelatedTableIf; @@ -67,7 +68,6 @@ import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.OptionalLong; import java.util.Set; import java.util.stream.Collectors; @@ -313,29 +313,29 @@ public void beforeMTMVRefresh(MTMV mtmv) throws DdlException { } @Override - public Map getAndCopyPartitionItems(OptionalLong snapshotId) { + public Map getAndCopyPartitionItems(Optional snapshot) { return Maps.newHashMap(getPartitionInfoFromCache().getNameToPartitionItem()); } @Override - public PartitionType getPartitionType() { + public PartitionType getPartitionType(Optional snapshot) { return getPartitionColumnsFromCache().size() > 0 ? PartitionType.LIST : PartitionType.UNPARTITIONED; } @Override - public Set getPartitionColumnNames() { + public Set getPartitionColumnNames(Optional snapshot) { return getPartitionColumnsFromCache().stream() .map(c -> c.getName().toLowerCase()).collect(Collectors.toSet()); } @Override - public List getPartitionColumns() { + public List getPartitionColumns(Optional snapshot) { return getPartitionColumnsFromCache(); } @Override public MTMVSnapshotIf getPartitionSnapshot(String partitionName, MTMVRefreshContext context, - OptionalLong snapshotId) + Optional snapshot) throws AnalysisException { PaimonPartition paimonPartition = getPartitionInfoFromCache().getNameToPartition().get(partitionName); if (paimonPartition == null) { @@ -345,7 +345,7 @@ public MTMVSnapshotIf getPartitionSnapshot(String partitionName, MTMVRefreshCont } @Override - public MTMVSnapshotIf getTableSnapshot(MTMVRefreshContext context, OptionalLong snapshotId) + public MTMVSnapshotIf getTableSnapshot(MTMVRefreshContext context, Optional snapshot) throws AnalysisException { return new MTMVVersionSnapshot(getLatestSnapshotIdFromCache()); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionExprDateTrunc.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionExprDateTrunc.java index ea15c84d1b925d..95a8717e01c4c7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionExprDateTrunc.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionExprDateTrunc.java @@ -69,7 +69,7 @@ public void analyze(MTMVPartitionInfo mvPartitionInfo) throws AnalysisException String.format("timeUnit not support: %s, only support: %s", this.timeUnit, timeUnits)); } MTMVRelatedTableIf relatedTable = mvPartitionInfo.getRelatedTable(); - PartitionType partitionType = relatedTable.getPartitionType(); + PartitionType partitionType = relatedTable.getPartitionType(Optional.empty()); if (partitionType == PartitionType.RANGE) { Type partitionColumnType = MTMVPartitionUtil .getPartitionColumnType(mvPartitionInfo.getRelatedTable(), mvPartitionInfo.getRelatedCol()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionInfo.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionInfo.java index b3cd239269abc7..7eae44db0af4cc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionInfo.java @@ -25,6 +25,7 @@ import com.google.gson.annotations.SerializedName; import java.util.List; +import java.util.Optional; /** * MTMVPartitionInfo @@ -115,7 +116,7 @@ public int getRelatedColPos() throws AnalysisException { if (partitionType == MTMVPartitionType.SELF_MANAGE) { throw new AnalysisException("partitionType is: " + partitionType); } - List partitionColumns = getRelatedTable().getPartitionColumns(); + List partitionColumns = getRelatedTable().getPartitionColumns(Optional.empty()); for (int i = 0; i < partitionColumns.size(); i++) { if (partitionColumns.get(i).getName().equalsIgnoreCase(relatedCol)) { return i; diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionUtil.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionUtil.java index 1bbc51fb004c57..8ba022de415006 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionUtil.java @@ -50,7 +50,7 @@ import java.util.Map; import java.util.Map.Entry; import java.util.Objects; -import java.util.OptionalLong; +import java.util.Optional; import java.util.Set; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -330,7 +330,7 @@ public static boolean isSyncWithPartitions(MTMVRefreshContext context, String mt } for (String relatedPartitionName : relatedPartitionNames) { MTMVSnapshotIf relatedPartitionCurrentSnapshot = relatedTable - .getPartitionSnapshot(relatedPartitionName, context, OptionalLong.empty()); + .getPartitionSnapshot(relatedPartitionName, context, Optional.empty()); if (!mtmv.getRefreshSnapshot() .equalsWithRelatedPartition(mtmvPartitionName, relatedPartitionName, relatedPartitionCurrentSnapshot)) { @@ -447,7 +447,7 @@ private static boolean isSyncWithBaseTable(MTMVRefreshContext context, String mt if (!baseTable.needAutoRefresh()) { return true; } - MTMVSnapshotIf baseTableCurrentSnapshot = baseTable.getTableSnapshot(context, OptionalLong.empty()); + MTMVSnapshotIf baseTableCurrentSnapshot = baseTable.getTableSnapshot(context, Optional.empty()); return mtmv.getRefreshSnapshot() .equalsWithBaseTable(mtmvPartitionName, new BaseTableInfo(baseTable), baseTableCurrentSnapshot); } @@ -483,7 +483,7 @@ private static MTMVRefreshPartitionSnapshot generatePartitionSnapshot(MTMVRefres MTMVRelatedTableIf relatedTable = mtmv.getMvPartitionInfo().getRelatedTable(); for (String relatedPartitionName : relatedPartitionNames) { MTMVSnapshotIf partitionSnapshot = relatedTable - .getPartitionSnapshot(relatedPartitionName, context, OptionalLong.empty()); + .getPartitionSnapshot(relatedPartitionName, context, Optional.empty()); refreshPartitionSnapshot.getPartitions() .put(relatedPartitionName, partitionSnapshot); } @@ -498,13 +498,13 @@ private static MTMVRefreshPartitionSnapshot generatePartitionSnapshot(MTMVRefres continue; } refreshPartitionSnapshot.addTableSnapshot(baseTableInfo, - ((MTMVRelatedTableIf) table).getTableSnapshot(context, OptionalLong.empty())); + ((MTMVRelatedTableIf) table).getTableSnapshot(context, Optional.empty())); } return refreshPartitionSnapshot; } public static Type getPartitionColumnType(MTMVRelatedTableIf relatedTable, String col) throws AnalysisException { - List partitionColumns = relatedTable.getPartitionColumns(); + List partitionColumns = relatedTable.getPartitionColumns(Optional.empty()); for (Column column : partitionColumns) { if (column.getName().equals(col)) { return column.getType(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelatedPartitionDescInitGenerator.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelatedPartitionDescInitGenerator.java index ef3100dec4c732..c6b4e331184e2a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelatedPartitionDescInitGenerator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelatedPartitionDescInitGenerator.java @@ -20,7 +20,7 @@ import org.apache.doris.common.AnalysisException; import java.util.Map; -import java.util.OptionalLong; +import java.util.Optional; /** * get all related partition descs @@ -30,6 +30,6 @@ public class MTMVRelatedPartitionDescInitGenerator implements MTMVRelatedPartiti @Override public void apply(MTMVPartitionInfo mvPartitionInfo, Map mvProperties, RelatedPartitionDescResult lastResult) throws AnalysisException { - lastResult.setItems(mvPartitionInfo.getRelatedTable().getAndCopyPartitionItems(OptionalLong.empty())); + lastResult.setItems(mvPartitionInfo.getRelatedTable().getAndCopyPartitionItems(Optional.empty())); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelatedPartitionDescRollUpGenerator.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelatedPartitionDescRollUpGenerator.java index 76e20ef70f5d92..325fab819d9a09 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelatedPartitionDescRollUpGenerator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelatedPartitionDescRollUpGenerator.java @@ -31,6 +31,7 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; +import java.util.Optional; import java.util.Set; /** @@ -45,7 +46,7 @@ public void apply(MTMVPartitionInfo mvPartitionInfo, Map mvPrope return; } MTMVRelatedTableIf relatedTable = mvPartitionInfo.getRelatedTable(); - PartitionType partitionType = relatedTable.getPartitionType(); + PartitionType partitionType = relatedTable.getPartitionType(Optional.empty()); if (partitionType == PartitionType.RANGE) { lastResult.setDescs(rollUpRange(lastResult.getDescs(), mvPartitionInfo)); } else if (partitionType == PartitionType.LIST) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelatedTableIf.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelatedTableIf.java index e18784ae253a0f..c4261aa78f10be 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelatedTableIf.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelatedTableIf.java @@ -23,10 +23,11 @@ import org.apache.doris.catalog.TableIf; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.DdlException; +import org.apache.doris.datasource.mvcc.MvccSnapshot; import java.util.List; import java.util.Map; -import java.util.OptionalLong; +import java.util.Optional; import java.util.Set; /** @@ -39,32 +40,35 @@ public interface MTMVRelatedTableIf extends TableIf { * Note: This method is called every time there is a refresh and transparent rewrite, * so if this method is slow, it will significantly reduce query performance * - * @param snapshotId + * @param snapshot * @return partitionName->PartitionItem */ - Map getAndCopyPartitionItems(OptionalLong snapshotId) throws AnalysisException; + Map getAndCopyPartitionItems(Optional snapshot) throws AnalysisException; /** * getPartitionType LIST/RANGE/UNPARTITIONED * + * @param snapshot * @return */ - PartitionType getPartitionType(); + PartitionType getPartitionType(Optional snapshot); /** * getPartitionColumnNames * + * @param snapshot * @return * @throws DdlException */ - Set getPartitionColumnNames() throws DdlException; + Set getPartitionColumnNames(Optional snapshot) throws DdlException; /** * getPartitionColumns * + * @param snapshot * @return */ - List getPartitionColumns(); + List getPartitionColumns(Optional snapshot); /** * getPartitionSnapshot @@ -72,14 +76,14 @@ public interface MTMVRelatedTableIf extends TableIf { * If snapshots have already been obtained in bulk in the context, * the results should be obtained directly from the context * - * @param snapshotId + * @param snapshot * @param partitionName * @param context * @return partition snapshot at current time * @throws AnalysisException */ - MTMVSnapshotIf getPartitionSnapshot(String partitionName, MTMVRefreshContext context, OptionalLong snapshotId) - throws AnalysisException; + MTMVSnapshotIf getPartitionSnapshot(String partitionName, MTMVRefreshContext context, + Optional snapshot) throws AnalysisException; /** * getTableSnapshot @@ -87,12 +91,13 @@ MTMVSnapshotIf getPartitionSnapshot(String partitionName, MTMVRefreshContext con * If snapshots have already been obtained in bulk in the context, * the results should be obtained directly from the context * - * @param snapshotId + * @param snapshot * @param context * @return table snapshot at current time * @throws AnalysisException */ - MTMVSnapshotIf getTableSnapshot(MTMVRefreshContext context, OptionalLong snapshotId) throws AnalysisException; + MTMVSnapshotIf getTableSnapshot(MTMVRefreshContext context, Optional snapshot) + throws AnalysisException; /** * Does the current type of table allow timed triggering diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java index a5c966370f030d..17ae5883063fb7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java @@ -70,6 +70,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Maps; import com.google.common.collect.Multimap; +import org.apache.commons.collections.MapUtils; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -462,6 +463,13 @@ private Set> getTables(LogicalPlan logicalPlan) { return tableNames; } + public Map, TableIf> getOrExtractTables(LogicalPlan logicalPlan) { + if (MapUtils.isEmpty(tables)) { + extractTables(logicalPlan); + } + return tables; + } + private Set> extractTableNamesFromHaving(LogicalHaving having) { Set subqueryExprs = having.getPredicate() .collect(SubqueryExpr.class::isInstance); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java index 8fb0f6e77976da..c7478411a5de11 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java @@ -216,7 +216,7 @@ public Plan planWithLock(LogicalPlan plan, PhysicalProperties requireProperties, plan = preprocess(plan); initCascadesContext(plan, requireProperties); - + statementContext.loadSnapshots(cascadesContext.getOrExtractTables(plan)); try (Lock lock = new Lock(plan, cascadesContext)) { Plan resultPlan = planWithoutLock(plan, explainLevel, showPlanProcess, requireProperties); lockCallback.accept(resultPlan); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java index 6b6e335b74a676..b172f9dc591bd9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java @@ -24,6 +24,9 @@ import org.apache.doris.common.Id; import org.apache.doris.common.IdGenerator; import org.apache.doris.common.Pair; +import org.apache.doris.datasource.mvcc.MvccSnapshot; +import org.apache.doris.datasource.mvcc.MvccTable; +import org.apache.doris.datasource.mvcc.MvccTableInfo; import org.apache.doris.nereids.hint.Hint; import org.apache.doris.nereids.memo.Group; import org.apache.doris.nereids.rules.analysis.ColumnAliasGenerator; @@ -174,6 +177,8 @@ public class StatementContext implements Closeable { private Backend groupCommitMergeBackend; + private final Map snapshots = Maps.newHashMap(); + private boolean privChecked; public StatementContext() { @@ -510,6 +515,32 @@ public void addPlannerHook(PlannerHook plannerHook) { this.plannerHooks.add(plannerHook); } + /** + * Load snapshot information of mvcc + * + * @param tables Tables used in queries + */ + public void loadSnapshots(Map, TableIf> tables) { + if (tables == null) { + return; + } + for (TableIf tableIf : tables.values()) { + if (tableIf instanceof MvccTable) { + snapshots.put(new MvccTableInfo(tableIf), ((MvccTable) tableIf).loadSnapshot()); + } + } + } + + /** + * Obtain snapshot information of mvcc + * + * @param mvccTable mvccTable + * @return MvccSnapshot + */ + public MvccSnapshot getSnapshot(MvccTable mvccTable) { + return snapshots.get(new MvccTableInfo(mvccTable)); + } + private static class CloseableResource implements Closeable { public final String resourceName; public final String threadName; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtils.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtils.java index a659c2f9990a3f..484abd11f01e72 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtils.java @@ -481,13 +481,13 @@ public Void visitLogicalRelation(LogicalRelation relation, IncrementCheckerConte return null; } MTMVRelatedTableIf relatedTable = (MTMVRelatedTableIf) table; - PartitionType type = relatedTable.getPartitionType(); + PartitionType type = relatedTable.getPartitionType(Optional.empty()); if (PartitionType.UNPARTITIONED.equals(type)) { context.addFailReason(String.format("related base table is not partition table, the table is %s", table.getName())); return null; } - Set partitionColumnSet = new HashSet<>(relatedTable.getPartitionColumns()); + Set partitionColumnSet = new HashSet<>(relatedTable.getPartitionColumns(Optional.empty())); Column mvReferenceColumn = contextPartitionColumn.getColumn().get(); Expr definExpr = mvReferenceColumn.getDefineExpr(); if (definExpr instanceof SlotRef) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneFileScanPartition.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneFileScanPartition.java index d50219383072df..4bbb0a8aa76270 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneFileScanPartition.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneFileScanPartition.java @@ -36,7 +36,7 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; -import java.util.OptionalLong; +import java.util.Optional; import java.util.function.Function; import java.util.stream.Collectors; @@ -75,7 +75,7 @@ private SelectedPartitions pruneExternalPartitions(ExternalTable externalTable, LogicalFilter filter, LogicalFileScan scan, CascadesContext ctx) { Map selectedPartitionItems = Maps.newHashMap(); // todo: real snapshotId - if (CollectionUtils.isEmpty(externalTable.getPartitionColumns(OptionalLong.empty()))) { + if (CollectionUtils.isEmpty(externalTable.getPartitionColumns(Optional.empty()))) { // non partitioned table, return NOT_PRUNED. // non partition table will be handled in HiveScanNode. return SelectedPartitions.NOT_PRUNED; @@ -84,7 +84,7 @@ private SelectedPartitions pruneExternalPartitions(ExternalTable externalTable, .stream() .collect(Collectors.toMap(slot -> slot.getName().toLowerCase(), Function.identity())); // todo: real snapshotId - List partitionSlots = externalTable.getPartitionColumns(OptionalLong.empty()) + List partitionSlots = externalTable.getPartitionColumns(Optional.empty()) .stream() .map(column -> scanOutput.get(column.getName().toLowerCase())) .collect(Collectors.toList()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java index de5e188d5a65bf..8c44b42a5ccfbd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java @@ -351,7 +351,7 @@ private PartitionDesc generatePartitionDesc(ConnectContext ctx) { allPartitionDescs.size(), ctx.getSessionVariable().getCreateTablePartitionMaxNum())); } try { - PartitionType type = relatedTable.getPartitionType(); + PartitionType type = relatedTable.getPartitionType(Optional.empty()); if (type == PartitionType.RANGE) { return new RangePartitionDesc(Lists.newArrayList(mvPartitionInfo.getPartitionCol()), allPartitionDescs); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/MTMVPartitionDefinition.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/MTMVPartitionDefinition.java index 427e2368e7ab2b..c4117e8608e29d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/MTMVPartitionDefinition.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/MTMVPartitionDefinition.java @@ -54,6 +54,7 @@ import com.google.common.collect.Sets; import java.util.List; +import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; @@ -147,7 +148,7 @@ private RelatedTableInfo getRelatedTableInfo(NereidsPlanner planner, ConnectCont MTMVRelatedTableIf mtmvBaseRealtedTable = MTMVUtil.getRelatedTable(relatedTableInfo.getTableInfo()); Set partitionColumnNames = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER); try { - partitionColumnNames.addAll(mtmvBaseRealtedTable.getPartitionColumnNames()); + partitionColumnNames.addAll(mtmvBaseRealtedTable.getPartitionColumnNames(Optional.empty())); } catch (DdlException e) { throw new AnalysisException(e.getMessage(), e); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalFileScan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalFileScan.java index 010c30d915d529..96b8e032d11274 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalFileScan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalFileScan.java @@ -36,7 +36,6 @@ import java.util.Map; import java.util.Objects; import java.util.Optional; -import java.util.OptionalLong; /** * Logical file scan for external catalog. @@ -64,7 +63,7 @@ public LogicalFileScan(RelationId id, ExternalTable table, List qualifie Optional tableSample, Optional tableSnapshot) { // todo: real snapshotId this(id, table, qualifier, Optional.empty(), Optional.empty(), - table.initSelectedPartitions(OptionalLong.empty()), tableSample, tableSnapshot); + table.initSelectedPartitions(Optional.empty()), tableSample, tableSnapshot); } public SelectedPartitions getSelectedPartitions() { diff --git a/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVPartitionUtilTest.java b/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVPartitionUtilTest.java index 96ac59b81216bc..e5d2e21a8db626 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVPartitionUtilTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVPartitionUtilTest.java @@ -35,7 +35,7 @@ import org.junit.Test; import java.util.List; -import java.util.OptionalLong; +import java.util.Optional; import java.util.Set; public class MTMVPartitionUtilTest { @@ -113,7 +113,7 @@ public void setUp() throws NoSuchMethodException, SecurityException, AnalysisExc minTimes = 0; result = true; - baseOlapTable.getTableSnapshot((MTMVRefreshContext) any, (OptionalLong) any); + baseOlapTable.getTableSnapshot((MTMVRefreshContext) any, (Optional) any); minTimes = 0; result = baseSnapshotIf; @@ -133,7 +133,7 @@ public void setUp() throws NoSuchMethodException, SecurityException, AnalysisExc minTimes = 0; result = true; - baseOlapTable.getPartitionSnapshot(anyString, (MTMVRefreshContext) any, (OptionalLong) any); + baseOlapTable.getPartitionSnapshot(anyString, (MTMVRefreshContext) any, (Optional) any); minTimes = 0; result = baseSnapshotIf; From 016d389e14c5d98b2c689373fa68cb5434bb09bb Mon Sep 17 00:00:00 2001 From: amory Date: Thu, 28 Nov 2024 10:25:41 +0800 Subject: [PATCH 046/399] [fix](ip-search)fix ip search with inverted index (#44580) this pr is to solve is_ip_address_in_range with invalid param --- be/src/vec/functions/function_ip.h | 5 +++++ .../test_ip_cidr_search_with_inverted_index.out | 4 ---- .../test_ip_cidr_search_with_inverted_index.groovy | 9 +++++++-- 3 files changed, 12 insertions(+), 6 deletions(-) diff --git a/be/src/vec/functions/function_ip.h b/be/src/vec/functions/function_ip.h index b90d1b2bcf9434..724121ce57c109 100644 --- a/be/src/vec/functions/function_ip.h +++ b/be/src/vec/functions/function_ip.h @@ -694,6 +694,11 @@ class FunctionIsIPAddressInRange : public IFunction { reinterpret_cast(&cidr_range_ipv6_data[1]), cidr._prefix); min_ip = cidr_range_ipv6_data[0]; max_ip = cidr_range_ipv6_data[1]; + } else { + // if here param is invalid for current column to calcute min_ip|max_ip we just return + return Status::Error( + "Inverted index evaluate skipped, data type " + arg_type->get_name() + + " can not support this cidr " + arg_column->get_data_at(0).to_string()); } // apply for inverted index std::shared_ptr res_roaring = std::make_shared(); diff --git a/regression-test/data/inverted_index_p0/test_ip_cidr_search_with_inverted_index.out b/regression-test/data/inverted_index_p0/test_ip_cidr_search_with_inverted_index.out index f365449b9e5244..2211a11b95a697 100644 --- a/regression-test/data/inverted_index_p0/test_ip_cidr_search_with_inverted_index.out +++ b/regression-test/data/inverted_index_p0/test_ip_cidr_search_with_inverted_index.out @@ -50,13 +50,9 @@ -- !sql -- --- !sql -- - -- !sql -- 2 42.117.228.166 2001:16a0:2:200a::2 --- !sql -- - -- !sql -- 5 255.255.255.255 \N diff --git a/regression-test/suites/inverted_index_p0/test_ip_cidr_search_with_inverted_index.groovy b/regression-test/suites/inverted_index_p0/test_ip_cidr_search_with_inverted_index.groovy index 61751f2404f2d2..191c818f351fde 100644 --- a/regression-test/suites/inverted_index_p0/test_ip_cidr_search_with_inverted_index.groovy +++ b/regression-test/suites/inverted_index_p0/test_ip_cidr_search_with_inverted_index.groovy @@ -118,8 +118,13 @@ suite("test_ip_cidr_search_with_inverted_index", "nonConcurrent"){ def execute_sql = { sqlList -> def i = 0 for (sqlStr in sqlList) { + if (i == 0 || i == 4 ) { + // inverted index will skip and do eval expr + ++i; + continue; + } try { - log.info("execute sql: i") + log.info("execute sql: ${i} : ${sqlStr}") GetDebugPoint().enableDebugPointForAllBEs(checkpoints_name, [req_id: i]) order_qt_sql """ ${sqlStr} """ } finally { @@ -131,4 +136,4 @@ suite("test_ip_cidr_search_with_inverted_index", "nonConcurrent"){ execute_sql.call(create_sql.call()) -} \ No newline at end of file +} From 0f90d46ec8b064df54be986c24860fdc2e912671 Mon Sep 17 00:00:00 2001 From: qiye Date: Thu, 28 Nov 2024 10:32:50 +0800 Subject: [PATCH 047/399] [fix](compaction)Fix cluster key column duplicated in value columns group (#44610) Issue Number: close #44609 Related pr: #40372 Problem Summary: `key_columns` will be used after move. This will cause skip in construction of `value_columns` useless. cluster key column will be duplicated added in value column group. --- be/src/olap/merger.cpp | 2 +- .../test_index_compaction_unique_keys_arr.out | 68 +++++++ ...st_index_compaction_unique_keys_arr.groovy | 189 ++++++++++-------- 3 files changed, 178 insertions(+), 81 deletions(-) diff --git a/be/src/olap/merger.cpp b/be/src/olap/merger.cpp index 694b7d6db84d8e..b207cc4c5ad22b 100644 --- a/be/src/olap/merger.cpp +++ b/be/src/olap/merger.cpp @@ -201,7 +201,7 @@ void Merger::vertical_split_columns(const TabletSchema& tablet_schema, << ", delete_sign_idx=" << delete_sign_idx; // for duplicate no keys if (!key_columns.empty()) { - column_groups->emplace_back(std::move(key_columns)); + column_groups->emplace_back(key_columns); } std::vector value_columns; diff --git a/regression-test/data/inverted_index_p0/array_contains/test_index_compaction_unique_keys_arr.out b/regression-test/data/inverted_index_p0/array_contains/test_index_compaction_unique_keys_arr.out index 3edad07916c9b5..a401117495f32d 100644 --- a/regression-test/data/inverted_index_p0/array_contains/test_index_compaction_unique_keys_arr.out +++ b/regression-test/data/inverted_index_p0/array_contains/test_index_compaction_unique_keys_arr.out @@ -67,3 +67,71 @@ 2 bason bason hate pear 99 3 bason bason hate pear 99 +-- !sql -- +1 bason bason hate pear 99 +2 bason bason hate pear 99 +3 bason bason hate pear 99 + +-- !sql -- + +-- !sql -- +1 bason bason hate pear 99 +2 bason bason hate pear 99 +3 bason bason hate pear 99 + +-- !sql -- +1 bason bason hate pear 99 +2 bason bason hate pear 99 +3 bason bason hate pear 99 + +-- !sql -- +1 bason bason hate pear 99 +2 bason bason hate pear 99 +3 bason bason hate pear 99 + +-- !sql -- + +-- !sql -- +1 bason bason hate pear 99 +2 bason bason hate pear 99 +3 bason bason hate pear 99 + +-- !sql -- +1 bason bason hate pear 99 +2 bason bason hate pear 99 +3 bason bason hate pear 99 + +-- !sql -- +1 bason bason hate pear 99 +2 bason bason hate pear 99 +3 bason bason hate pear 99 + +-- !sql -- + +-- !sql -- +1 bason bason hate pear 99 +2 bason bason hate pear 99 +3 bason bason hate pear 99 + +-- !sql -- +1 bason bason hate pear 99 +2 bason bason hate pear 99 +3 bason bason hate pear 99 + +-- !sql -- +1 bason bason hate pear 99 +2 bason bason hate pear 99 +3 bason bason hate pear 99 + +-- !sql -- + +-- !sql -- +1 bason bason hate pear 99 +2 bason bason hate pear 99 +3 bason bason hate pear 99 + +-- !sql -- +1 bason bason hate pear 99 +2 bason bason hate pear 99 +3 bason bason hate pear 99 + diff --git a/regression-test/suites/inverted_index_p0/array_contains/test_index_compaction_unique_keys_arr.groovy b/regression-test/suites/inverted_index_p0/array_contains/test_index_compaction_unique_keys_arr.groovy index 013d5a558111e7..e07618be6c1370 100644 --- a/regression-test/suites/inverted_index_p0/array_contains/test_index_compaction_unique_keys_arr.groovy +++ b/regression-test/suites/inverted_index_p0/array_contains/test_index_compaction_unique_keys_arr.groovy @@ -112,69 +112,22 @@ suite("test_index_compaction_unique_keys_arr", "array_contains_inverted_index") } } - boolean invertedIndexCompactionEnable = false - boolean has_update_be_config = false - try { - String backend_id; - backend_id = backendId_to_backendIP.keySet()[0] - def (code, out, err) = show_be_config(backendId_to_backendIP.get(backend_id), backendId_to_backendHttpPort.get(backend_id)) - - logger.info("Show config: code=" + code + ", out=" + out + ", err=" + err) - assertEquals(code, 0) - def configList = parseJson(out.trim()) - assert configList instanceof List + def run_test = { table_name -> - for (Object ele in (List) configList) { - assert ele instanceof List - if (((List) ele)[0] == "inverted_index_compaction_enable") { - invertedIndexCompactionEnable = Boolean.parseBoolean(((List) ele)[2]) - logger.info("inverted_index_compaction_enable: ${((List) ele)[2]}") - } - if (((List) ele)[0] == "disable_auto_compaction") { - disableAutoCompaction = Boolean.parseBoolean(((List) ele)[2]) - logger.info("disable_auto_compaction: ${((List) ele)[2]}") - } - } - set_be_config.call("inverted_index_compaction_enable", "true") - has_update_be_config = true - // check updated config - check_config.call("inverted_index_compaction_enable", "true"); + sql """ INSERT INTO ${table_name} VALUES (1, "andy", "andy love apple", 100); """ + sql """ INSERT INTO ${table_name} VALUES (1, "bason", "bason hate pear", 99); """ + sql """ INSERT INTO ${table_name} VALUES (2, "andy", "andy love apple", 100); """ + sql """ INSERT INTO ${table_name} VALUES (2, "bason", "bason hate pear", 99); """ + sql """ INSERT INTO ${table_name} VALUES (3, "andy", "andy love apple", 100); """ + sql """ INSERT INTO ${table_name} VALUES (3, "bason", "bason hate pear", 99); """ - sql """ DROP TABLE IF EXISTS ${tableName}; """ - sql """ - CREATE TABLE ${tableName} ( - `id` int(11) NULL, - `name` varchar(255) NULL, - `hobbies` text NULL, - `score` int(11) NULL, - index index_name (name) using inverted, - index index_hobbies (hobbies) using inverted properties("parser"="english"), - index index_score (score) using inverted - ) ENGINE=OLAP - UNIQUE KEY(`id`) - COMMENT 'OLAP' - DISTRIBUTED BY HASH(`id`) BUCKETS 1 - PROPERTIES ( - "replication_num" = "1", - "disable_auto_compaction" = "true", - "enable_unique_key_merge_on_write" = "true" - ); - """ - - sql """ INSERT INTO ${tableName} VALUES (1, "andy", "andy love apple", 100); """ - sql """ INSERT INTO ${tableName} VALUES (1, "bason", "bason hate pear", 99); """ - sql """ INSERT INTO ${tableName} VALUES (2, "andy", "andy love apple", 100); """ - sql """ INSERT INTO ${tableName} VALUES (2, "bason", "bason hate pear", 99); """ - sql """ INSERT INTO ${tableName} VALUES (3, "andy", "andy love apple", 100); """ - sql """ INSERT INTO ${tableName} VALUES (3, "bason", "bason hate pear", 99); """ - - qt_sql """ select * from ${tableName} order by id, name, hobbies, score """ - qt_sql """ select * from ${tableName} where name match "andy" order by id, name, hobbies, score """ - qt_sql """ select * from ${tableName} where hobbies match "pear" order by id, name, hobbies, score """ - qt_sql """ select * from ${tableName} where score < 100 order by id, name, hobbies, score """ + qt_sql """ select * from ${table_name} order by id, name, hobbies, score """ + qt_sql """ select * from ${table_name} where name match "andy" order by id, name, hobbies, score """ + qt_sql """ select * from ${table_name} where hobbies match "pear" order by id, name, hobbies, score """ + qt_sql """ select * from ${table_name} where score < 100 order by id, name, hobbies, score """ //TabletId,ReplicaId,BackendId,SchemaHash,Version,LstSuccessVersion,LstFailedVersion,LstFailedTime,LocalDataSize,RemoteDataSize,RowCount,State,LstConsistencyCheckTime,CheckVersion,VersionCount,PathHash,MetaUrl,CompactionStatus - def tablets = sql_return_maparray """ show tablets from ${tableName}; """ + def tablets = sql_return_maparray """ show tablets from ${table_name}; """ def dedup_tablets = deduplicate_tablets(tablets) // In the p0 testing environment, there are no expected operations such as scaling down BE (backend) services @@ -189,7 +142,7 @@ suite("test_index_compaction_unique_keys_arr", "array_contains_inverted_index") int rowsetCount = get_rowset_count.call(tablets); assert (rowsetCount == 7 * replicaNum) - // trigger full compactions for all tablets in ${tableName} + // trigger full compactions for all tablets in ${table_name} trigger_full_compaction_on_tablets.call(tablets) // wait for full compaction done @@ -203,23 +156,23 @@ suite("test_index_compaction_unique_keys_arr", "array_contains_inverted_index") assert (rowsetCount == 1 * replicaNum) } - qt_sql """ select * from ${tableName} order by id, name, hobbies, score """ - qt_sql """ select * from ${tableName} where name match "andy" order by id, name, hobbies, score """ - qt_sql """ select * from ${tableName} where hobbies match "pear" order by id, name, hobbies, score """ - qt_sql """ select * from ${tableName} where score < 100 order by id, name, hobbies, score """ + qt_sql """ select * from ${table_name} order by id, name, hobbies, score """ + qt_sql """ select * from ${table_name} where name match "andy" order by id, name, hobbies, score """ + qt_sql """ select * from ${table_name} where hobbies match "pear" order by id, name, hobbies, score """ + qt_sql """ select * from ${table_name} where score < 100 order by id, name, hobbies, score """ // insert more data and trigger full compaction again - sql """ INSERT INTO ${tableName} VALUES (1, "andy", "andy love apple", 100); """ - sql """ INSERT INTO ${tableName} VALUES (1, "bason", "bason hate pear", 99); """ - sql """ INSERT INTO ${tableName} VALUES (2, "andy", "andy love apple", 100); """ - sql """ INSERT INTO ${tableName} VALUES (2, "bason", "bason hate pear", 99); """ - sql """ INSERT INTO ${tableName} VALUES (3, "andy", "andy love apple", 100); """ - sql """ INSERT INTO ${tableName} VALUES (3, "bason", "bason hate pear", 99); """ - - qt_sql """ select * from ${tableName} order by id, name, hobbies, score """ - qt_sql """ select * from ${tableName} where name match "andy" order by id, name, hobbies, score """ - qt_sql """ select * from ${tableName} where hobbies match "pear" order by id, name, hobbies, score """ - qt_sql """ select * from ${tableName} where score < 100 order by id, name, hobbies, score """ + sql """ INSERT INTO ${table_name} VALUES (1, "andy", "andy love apple", 100); """ + sql """ INSERT INTO ${table_name} VALUES (1, "bason", "bason hate pear", 99); """ + sql """ INSERT INTO ${table_name} VALUES (2, "andy", "andy love apple", 100); """ + sql """ INSERT INTO ${table_name} VALUES (2, "bason", "bason hate pear", 99); """ + sql """ INSERT INTO ${table_name} VALUES (3, "andy", "andy love apple", 100); """ + sql """ INSERT INTO ${table_name} VALUES (3, "bason", "bason hate pear", 99); """ + + qt_sql """ select * from ${table_name} order by id, name, hobbies, score """ + qt_sql """ select * from ${table_name} where name match "andy" order by id, name, hobbies, score """ + qt_sql """ select * from ${table_name} where hobbies match "pear" order by id, name, hobbies, score """ + qt_sql """ select * from ${table_name} where score < 100 order by id, name, hobbies, score """ rowsetCount = get_rowset_count.call(tablets); if (isCloudMode) { @@ -228,7 +181,7 @@ suite("test_index_compaction_unique_keys_arr", "array_contains_inverted_index") assert (rowsetCount == 7 * replicaNum) } - // trigger full compactions for all tablets in ${tableName} + // trigger full compactions for all tablets in ${table_name} trigger_full_compaction_on_tablets.call(tablets) // wait for full compaction done @@ -242,10 +195,86 @@ suite("test_index_compaction_unique_keys_arr", "array_contains_inverted_index") assert (rowsetCount == 1 * replicaNum) } - qt_sql """ select * from ${tableName} order by id, name, hobbies, score """ - qt_sql """ select * from ${tableName} where name match "andy" order by id, name, hobbies, score """ - qt_sql """ select * from ${tableName} where hobbies match "pear" order by id, name, hobbies, score """ - qt_sql """ select * from ${tableName} where score < 100 order by id, name, hobbies, score """ + qt_sql """ select * from ${table_name} order by id, name, hobbies, score """ + qt_sql """ select * from ${table_name} where name match "andy" order by id, name, hobbies, score """ + qt_sql """ select * from ${table_name} where hobbies match "pear" order by id, name, hobbies, score """ + qt_sql """ select * from ${table_name} where score < 100 order by id, name, hobbies, score """ + } + + boolean invertedIndexCompactionEnable = false + boolean has_update_be_config = false + try { + String backend_id; + backend_id = backendId_to_backendIP.keySet()[0] + def (code, out, err) = show_be_config(backendId_to_backendIP.get(backend_id), backendId_to_backendHttpPort.get(backend_id)) + + logger.info("Show config: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def configList = parseJson(out.trim()) + assert configList instanceof List + + for (Object ele in (List) configList) { + assert ele instanceof List + if (((List) ele)[0] == "inverted_index_compaction_enable") { + invertedIndexCompactionEnable = Boolean.parseBoolean(((List) ele)[2]) + logger.info("inverted_index_compaction_enable: ${((List) ele)[2]}") + } + if (((List) ele)[0] == "disable_auto_compaction") { + disableAutoCompaction = Boolean.parseBoolean(((List) ele)[2]) + logger.info("disable_auto_compaction: ${((List) ele)[2]}") + } + } + set_be_config.call("inverted_index_compaction_enable", "true") + has_update_be_config = true + // check updated config + check_config.call("inverted_index_compaction_enable", "true"); + + sql """ DROP TABLE IF EXISTS ${tableName}; """ + sql """ + CREATE TABLE ${tableName} ( + `id` int(11) NULL, + `name` varchar(255) NULL, + `hobbies` text NULL, + `score` int(11) NULL, + index index_name (name) using inverted, + index index_hobbies (hobbies) using inverted properties("parser"="english"), + index index_score (score) using inverted + ) ENGINE=OLAP + UNIQUE KEY(`id`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`id`) BUCKETS 1 + PROPERTIES ( + "replication_num" = "1", + "disable_auto_compaction" = "true", + "enable_unique_key_merge_on_write" = "true" + ); + """ + + run_test.call(tableName) + + tableName = "test_index_compaction_unique_keys_arr_cluster_key" + sql """ DROP TABLE IF EXISTS ${tableName}; """ + sql """ + CREATE TABLE ${tableName} ( + `id` int(11) NULL, + `name` varchar(255) NULL, + `hobbies` text NULL, + `score` int(11) NULL, + index index_name (name) using inverted, + index index_hobbies (hobbies) using inverted properties("parser"="english"), + index index_score (score) using inverted + ) ENGINE=OLAP + UNIQUE KEY(`id`) + CLUSTER BY (`score`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`id`) BUCKETS 1 + PROPERTIES ( + "replication_num" = "1", + "disable_auto_compaction" = "true", + "enable_unique_key_merge_on_write" = "true" + ); + """ + run_test.call(tableName) } finally { if (has_update_be_config) { From cf0b4c95af89bd4ffadc53c09aff2e9e78a3e542 Mon Sep 17 00:00:00 2001 From: morrySnow Date: Thu, 28 Nov 2024 10:42:55 +0800 Subject: [PATCH 048/399] [fix](planner) FunctionCallExpr clone should be deep copy (#44678) --- .../java/org/apache/doris/analysis/FunctionCallExpr.java | 2 +- .../main/java/org/apache/doris/analysis/FunctionName.java | 5 +++++ 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/FunctionCallExpr.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/FunctionCallExpr.java index 53bb2ba95acb25..51e18d750f263f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/FunctionCallExpr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/FunctionCallExpr.java @@ -432,7 +432,7 @@ public FunctionCallExpr(FunctionCallExpr e, FunctionParams params) { protected FunctionCallExpr(FunctionCallExpr other) { super(other); - fnName = other.fnName; + fnName = other.fnName != null ? other.fnName.clone() : null; orderByElements = other.orderByElements; isAnalyticFnCall = other.isAnalyticFnCall; // aggOp = other.aggOp; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/FunctionName.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/FunctionName.java index b6feddb7d256ed..d0e695f6660a43 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/FunctionName.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/FunctionName.java @@ -183,4 +183,9 @@ public static FunctionName read(DataInput in) throws IOException { public int hashCode() { return 31 * Objects.hashCode(db) + Objects.hashCode(fn); } + + @Override + public FunctionName clone() { + return new FunctionName(db, fn); + } } From ee5691b9dd4c7a1ed4ee9364c2261ebd3be52881 Mon Sep 17 00:00:00 2001 From: minghong Date: Thu, 28 Nov 2024 11:18:28 +0800 Subject: [PATCH 049/399] [test](nereids)add regression case: disable join reorder if there are invalid stats (#43501) ### What problem does this PR solve? add test case for pr #41790. this case is blocked by #43279 --- .../data/nereids_p0/stats/invalid_stats.out | 31 +++++ .../stats/invalid_stats/invalid_stats.out | 31 +++++ .../stats/{ => col_stats}/column_stats.groovy | 0 .../nereids_p0/stats/invalid_stats.groovy | 111 ------------------ .../stats/invalid_stats/invalid_stats.groovy | 77 ++++++++++++ 5 files changed, 139 insertions(+), 111 deletions(-) create mode 100644 regression-test/data/nereids_p0/stats/invalid_stats.out create mode 100644 regression-test/data/nereids_p0/stats/invalid_stats/invalid_stats.out rename regression-test/suites/nereids_p0/stats/{ => col_stats}/column_stats.groovy (100%) delete mode 100644 regression-test/suites/nereids_p0/stats/invalid_stats.groovy create mode 100644 regression-test/suites/nereids_p0/stats/invalid_stats/invalid_stats.groovy diff --git a/regression-test/data/nereids_p0/stats/invalid_stats.out b/regression-test/data/nereids_p0/stats/invalid_stats.out new file mode 100644 index 00000000000000..9b1b2e2aa97528 --- /dev/null +++ b/regression-test/data/nereids_p0/stats/invalid_stats.out @@ -0,0 +1,31 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !reorder_1 -- +PhysicalResultSink +--hashJoin[INNER_JOIN broadcast] hashCondition=((region.r_regionkey = nation.n_regionkey)) otherCondition=() +----PhysicalOlapScan[nation] +----PhysicalOlapScan[region] + +-- !ndv_min_max_invalid -- +PhysicalResultSink +--hashJoin[INNER_JOIN broadcast] hashCondition=((region.r_regionkey = nation.n_regionkey)) otherCondition=() +----PhysicalOlapScan[region] +----PhysicalOlapScan[nation] + +-- !reorder_2 -- +PhysicalResultSink +--hashJoin[INNER_JOIN broadcast] hashCondition=((region.r_regionkey = nation.n_regionkey)) otherCondition=() +----PhysicalOlapScan[nation] +----PhysicalOlapScan[region] + +-- !order_3 -- +PhysicalResultSink +--hashJoin[INNER_JOIN broadcast] hashCondition=((region.r_regionkey = nation.n_regionkey)) otherCondition=() +----PhysicalOlapScan[nation] +----PhysicalOlapScan[region] + +-- !ndv_row_invalid -- +PhysicalResultSink +--hashJoin[INNER_JOIN broadcast] hashCondition=((region.r_regionkey = nation.n_regionkey)) otherCondition=() +----PhysicalOlapScan[region] +----PhysicalOlapScan[nation] + diff --git a/regression-test/data/nereids_p0/stats/invalid_stats/invalid_stats.out b/regression-test/data/nereids_p0/stats/invalid_stats/invalid_stats.out new file mode 100644 index 00000000000000..9b1b2e2aa97528 --- /dev/null +++ b/regression-test/data/nereids_p0/stats/invalid_stats/invalid_stats.out @@ -0,0 +1,31 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !reorder_1 -- +PhysicalResultSink +--hashJoin[INNER_JOIN broadcast] hashCondition=((region.r_regionkey = nation.n_regionkey)) otherCondition=() +----PhysicalOlapScan[nation] +----PhysicalOlapScan[region] + +-- !ndv_min_max_invalid -- +PhysicalResultSink +--hashJoin[INNER_JOIN broadcast] hashCondition=((region.r_regionkey = nation.n_regionkey)) otherCondition=() +----PhysicalOlapScan[region] +----PhysicalOlapScan[nation] + +-- !reorder_2 -- +PhysicalResultSink +--hashJoin[INNER_JOIN broadcast] hashCondition=((region.r_regionkey = nation.n_regionkey)) otherCondition=() +----PhysicalOlapScan[nation] +----PhysicalOlapScan[region] + +-- !order_3 -- +PhysicalResultSink +--hashJoin[INNER_JOIN broadcast] hashCondition=((region.r_regionkey = nation.n_regionkey)) otherCondition=() +----PhysicalOlapScan[nation] +----PhysicalOlapScan[region] + +-- !ndv_row_invalid -- +PhysicalResultSink +--hashJoin[INNER_JOIN broadcast] hashCondition=((region.r_regionkey = nation.n_regionkey)) otherCondition=() +----PhysicalOlapScan[region] +----PhysicalOlapScan[nation] + diff --git a/regression-test/suites/nereids_p0/stats/column_stats.groovy b/regression-test/suites/nereids_p0/stats/col_stats/column_stats.groovy similarity index 100% rename from regression-test/suites/nereids_p0/stats/column_stats.groovy rename to regression-test/suites/nereids_p0/stats/col_stats/column_stats.groovy diff --git a/regression-test/suites/nereids_p0/stats/invalid_stats.groovy b/regression-test/suites/nereids_p0/stats/invalid_stats.groovy deleted file mode 100644 index 5304cd8c2c1fa9..00000000000000 --- a/regression-test/suites/nereids_p0/stats/invalid_stats.groovy +++ /dev/null @@ -1,111 +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. - -suite("invalid_stats") { - // multi_sql """ - // set global enable_auto_analyze=false; - // SET enable_nereids_planner=true; - // SET enable_fallback_to_original_planner=false; - // set disable_nereids_rules=PRUNE_EMPTY_PARTITION; - - - // drop table if exists region; - // CREATE TABLE region ( - // r_regionkey int NOT NULL, - // r_name VARCHAR(25) NOT NULL, - // r_comment VARCHAR(152) - // )ENGINE=OLAP - // DUPLICATE KEY(`r_regionkey`) - // COMMENT "OLAP" - // DISTRIBUTED BY HASH(`r_regionkey`) BUCKETS 1 - // PROPERTIES ( - // "replication_num" = "1" - // ); - - // drop table if exists nation; - // CREATE TABLE `nation` ( - // `n_nationkey` int(11) NOT NULL, - // `n_name` varchar(25) NOT NULL, - // `n_regionkey` int(11) NOT NULL, - // `n_comment` varchar(152) NULL - // ) ENGINE=OLAP - // DUPLICATE KEY(`N_NATIONKEY`) - // COMMENT "OLAP" - // DISTRIBUTED BY HASH(`N_NATIONKEY`) BUCKETS 1 - // PROPERTIES ( - // "replication_num" = "1" - // ); - // alter table nation modify column n_nationkey set stats ('ndv'='25', 'num_nulls'='0', 'min_value'='0', 'max_value'='24', 'row_count'='25'); - - // alter table nation modify column n_regionkey set stats ('ndv'='5', 'num_nulls'='0', 'min_value'='0', 'max_value'='4', 'row_count'='25'); - - // """ - - // explain { - // sql "select * from region" - // notContains("join reorder with unknown column statistics") - // } - - // explain { - // sql "select * from region where r_regionkey=1" - // contains("join reorder with unknown column statistics") - // } - - // explain { - // sql "select r_regionkey from region group by r_regionkey" - // contains("join reorder with unknown column statistics") - // } - - // explain { - // sql "select r_regionkey from region join nation on r_regionkey=n_regionkey" - // contains("join reorder with unknown column statistics") - // } - - // sql "alter table region modify column r_regionkey set stats ('ndv'='5', 'num_nulls'='0', 'min_value'='0', 'max_value'='4', 'row_count'='5');" - - // explain { - // sql "select * from region where r_regionkey=1" - // notContains("join reorder with unknown column statistics") - // } - - // explain { - // sql "select r_regionkey from region group by r_regionkey" - // notContains("join reorder with unknown column statistics") - // } - - // explain { - // sql "select r_regionkey from region join nation on r_regionkey=n_regionkey" - // notContains("join reorder with unknown column statistics") - // } - - // explain { - // sql "select r_name from region join nation on r_regionkey=n_regionkey" - // notContains("join reorder with unknown column statistics") - // } - - // explain { - // sql """ - // select r_name - // from (select r_name, r_regionkey + 1 x from region) T join nation on T.x=n_regionkey - // """ - // notContains("join reorder with unknown column statistics") - // } -} -// disable jo: alter table region modify column r_regionkey set stats ('ndv'='0', 'num_nulls'='0', 'min_value'='0', 'max_value'='4', 'row_count'='0'); -// disable jo: alter table region modify column r_regionkey set stats ('ndv'='11', 'num_nulls'='0', 'min_value'='0', 'max_value'='4', 'row_count'='1'); - -// alter table region modify column r_regionkey set stats ('ndv'='10', 'num_nulls'='0', 'min_value'='0', 'max_value'='4', 'row_count'='1'); diff --git a/regression-test/suites/nereids_p0/stats/invalid_stats/invalid_stats.groovy b/regression-test/suites/nereids_p0/stats/invalid_stats/invalid_stats.groovy new file mode 100644 index 00000000000000..51f1a37f118a42 --- /dev/null +++ b/regression-test/suites/nereids_p0/stats/invalid_stats/invalid_stats.groovy @@ -0,0 +1,77 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("invalid_stats") { + multi_sql """ + set global enable_auto_analyze=false; + SET enable_nereids_planner=true; + SET enable_fallback_to_original_planner=false; + set disable_nereids_rules=PRUNE_EMPTY_PARTITION; + set ignore_shape_nodes=PhysicalProject; + + drop table if exists region; + CREATE TABLE region ( + r_regionkey int NOT NULL, + r_name VARCHAR(25) NOT NULL, + r_comment VARCHAR(152) + )ENGINE=OLAP + DUPLICATE KEY(`r_regionkey`) + COMMENT "OLAP" + DISTRIBUTED BY HASH(`r_regionkey`) BUCKETS 1 + PROPERTIES ( + "replication_num" = "1" + ); + + drop table if exists nation; + CREATE TABLE `nation` ( + `n_nationkey` int(11) NOT NULL, + `n_name` varchar(25) NOT NULL, + `n_regionkey` int(11) NOT NULL, + `n_comment` varchar(152) NULL + ) ENGINE=OLAP + DUPLICATE KEY(`N_NATIONKEY`) + COMMENT "OLAP" + DISTRIBUTED BY HASH(`N_NATIONKEY`) BUCKETS 1 + PROPERTIES ( + "replication_num" = "1" + ); + alter table nation modify column n_nationkey set stats ('ndv'='25', 'num_nulls'='0', 'min_value'='0', 'max_value'='24', 'row_count'='25'); + + alter table region modify column r_regionkey set stats ('ndv'='5', 'num_nulls'='0', 'min_value'='0', 'max_value'='4', 'row_count'='5'); + + """ + + qt_reorder_1 "explain shape plan select r_regionkey from region join nation on r_regionkey=n_regionkey" + + sql "alter table region modify column r_regionkey set stats ('ndv'='0', 'num_nulls'='0', 'min_value'='0', 'max_value'='4', 'row_count'='0');" + + // r_regionkey stats invalid: ndv=0, but min or max is not null + qt_ndv_min_max_invalid "explain shape plan select r_regionkey from region join nation on r_regionkey=n_regionkey" + + // inject normal stats and check join order is nation-region + sql "alter table region modify column r_regionkey set stats ('ndv'='5', 'num_nulls'='0', 'min_value'='0', 'max_value'='4', 'row_count'='5');" + + qt_reorder_2 "explain shape plan select r_regionkey from region join nation on r_regionkey=n_regionkey" + + // r_regionkey stats invalid: ndv > 10*row + sql "alter table region modify column r_regionkey set stats ('ndv'='10', 'num_nulls'='0', 'min_value'='0', 'max_value'='4', 'row_count'='1');" + qt_order_3 "explain shape plan select r_regionkey from region join nation on r_regionkey=n_regionkey" + + sql "alter table region modify column r_regionkey set stats ('ndv'='11', 'num_nulls'='0', 'min_value'='0', 'max_value'='4', 'row_count'='1');" + qt_ndv_row_invalid "explain shape plan select r_regionkey from region join nation on r_regionkey=n_regionkey" + +} From fa3bdbce966512da6986046df8558c1d04e93f61 Mon Sep 17 00:00:00 2001 From: minghong Date: Thu, 28 Nov 2024 11:19:32 +0800 Subject: [PATCH 050/399] [opt](nereids) enhance PUSH_DOWN_AGG_THROUGH_JOIN_ONE_SIDE (#43856) ### What problem does this PR solve? PUSH_DOWN_AGG_THROUGH_JOIN_ONE_SIDE has some restrictions do not support count(*) do not support join with other join conditions do not support the project between agg and join that contains non-slot expressions this pr removes above restrictions for pattern: agg-project-join --- .../PushDownAggThroughJoinOneSide.java | 123 +++++++++++++----- .../PushDownMinMaxSumThroughJoinTest.java | 16 +-- .../push_down_count_through_join_one_side.out | 22 ++++ ...sh_down_count_through_join_one_side.groovy | 95 ++++++++++++++ 4 files changed, 212 insertions(+), 44 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownAggThroughJoinOneSide.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownAggThroughJoinOneSide.java index f32bf8ea91b355..c5d3d0fb49a0a5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownAggThroughJoinOneSide.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownAggThroughJoinOneSide.java @@ -36,6 +36,7 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList.Builder; +import com.google.common.collect.Lists; import java.util.ArrayList; import java.util.HashMap; @@ -74,8 +75,8 @@ public List buildRules() { Set funcs = agg.getAggregateFunctions(); return !funcs.isEmpty() && funcs.stream() .allMatch(f -> (f instanceof Min || f instanceof Max || f instanceof Sum - || (f instanceof Count && !((Count) f).isCountStar())) && !f.isDistinct() - && f.child(0) instanceof Slot); + || f instanceof Count && !f.isDistinct() + && (f.children().isEmpty() || f.child(0) instanceof Slot))); }) .thenApply(ctx -> { Set enableNereidsRules = ctx.cascadesContext.getConnectContext() @@ -88,15 +89,16 @@ public List buildRules() { }) .toRule(RuleType.PUSH_DOWN_AGG_THROUGH_JOIN_ONE_SIDE), logicalAggregate(logicalProject(innerLogicalJoin())) - .when(agg -> agg.child().isAllSlots()) - .when(agg -> agg.child().child().getOtherJoinConjuncts().isEmpty()) - .whenNot(agg -> agg.child().children().stream().anyMatch(p -> p instanceof LogicalAggregate)) + // .when(agg -> agg.child().isAllSlots()) + // .when(agg -> agg.child().child().getOtherJoinConjuncts().isEmpty()) + .whenNot(agg -> agg.child() + .child(0).children().stream().anyMatch(p -> p instanceof LogicalAggregate)) .when(agg -> { Set funcs = agg.getAggregateFunctions(); return !funcs.isEmpty() && funcs.stream() .allMatch(f -> (f instanceof Min || f instanceof Max || f instanceof Sum - || (f instanceof Count && (!((Count) f).isCountStar()))) && !f.isDistinct() - && f.child(0) instanceof Slot); + || f instanceof Count) && !f.isDistinct() + && (f.children().isEmpty() || f.child(0) instanceof Slot)); }) .thenApply(ctx -> { Set enableNereidsRules = ctx.cascadesContext.getConnectContext() @@ -118,23 +120,6 @@ public static LogicalAggregate pushMinMaxSumCount(LogicalAggregate join, List projects) { List leftOutput = join.left().getOutput(); List rightOutput = join.right().getOutput(); - - List leftFuncs = new ArrayList<>(); - List rightFuncs = new ArrayList<>(); - for (AggregateFunction func : agg.getAggregateFunctions()) { - Slot slot = (Slot) func.child(0); - if (leftOutput.contains(slot)) { - leftFuncs.add(func); - } else if (rightOutput.contains(slot)) { - rightFuncs.add(func); - } else { - throw new IllegalStateException("Slot " + slot + " not found in join output"); - } - } - if (leftFuncs.isEmpty() && rightFuncs.isEmpty()) { - return null; - } - Set leftGroupBy = new HashSet<>(); Set rightGroupBy = new HashSet<>(); for (Expression e : agg.getGroupByExpressions()) { @@ -144,18 +129,71 @@ public static LogicalAggregate pushMinMaxSumCount(LogicalAggregate inputForAliasSet = proj.getInputSlots(); + for (Slot aliasInputSlot : inputForAliasSet) { + if (leftOutput.contains(aliasInputSlot)) { + leftGroupBy.add(aliasInputSlot); + } else if (rightOutput.contains(aliasInputSlot)) { + rightGroupBy.add(aliasInputSlot); + } else { + return null; + } + } + break; + } + } + } } } - join.getHashJoinConjuncts().forEach(e -> e.getInputSlots().forEach(slot -> { - if (leftOutput.contains(slot)) { - leftGroupBy.add(slot); - } else if (rightOutput.contains(slot)) { - rightGroupBy.add(slot); + + List leftFuncs = new ArrayList<>(); + List rightFuncs = new ArrayList<>(); + Count countStar = null; + Count rewrittenCountStar = null; + for (AggregateFunction func : agg.getAggregateFunctions()) { + if (func instanceof Count && ((Count) func).isCountStar()) { + countStar = (Count) func; + } else { + Slot slot = (Slot) func.child(0); + if (leftOutput.contains(slot)) { + leftFuncs.add(func); + } else if (rightOutput.contains(slot)) { + rightFuncs.add(func); + } else { + throw new IllegalStateException("Slot " + slot + " not found in join output"); + } + } + } + // rewrite count(*) to count(A), where A is slot from left/right group by key + if (countStar != null) { + if (!leftGroupBy.isEmpty()) { + rewrittenCountStar = (Count) countStar.withChildren(leftGroupBy.iterator().next()); + leftFuncs.add(rewrittenCountStar); + } else if (!rightGroupBy.isEmpty()) { + rewrittenCountStar = (Count) countStar.withChildren(rightGroupBy.iterator().next()); + rightFuncs.add(rewrittenCountStar); } else { - throw new IllegalStateException("Slot " + slot + " not found in join output"); + return null; + } + } + for (Expression condition : join.getHashJoinConjuncts()) { + for (Slot joinConditionSlot : condition.getInputSlots()) { + if (leftOutput.contains(joinConditionSlot)) { + leftGroupBy.add(joinConditionSlot); + } else if (rightOutput.contains(joinConditionSlot)) { + rightGroupBy.add(joinConditionSlot); + } else { + // apply failed + return null; + } } - })); + } Plan left = join.left(); Plan right = join.right(); @@ -196,6 +234,10 @@ public static LogicalAggregate pushMinMaxSumCount(LogicalAggregate pushMinMaxSumCount(LogicalAggregate newProjections = Lists.newArrayList(); + newProjections.addAll(project.getProjects()); + Set leftDifference = new HashSet(left.getOutput()); + leftDifference.removeAll(project.getProjects()); + newProjections.addAll(leftDifference); + Set rightDifference = new HashSet(right.getOutput()); + rightDifference.removeAll(project.getProjects()); + newProjections.addAll(rightDifference); + newAggChild = ((LogicalProject) agg.child()).withProjectsAndChild(newProjections, newJoin); + } + return agg.withAggOutputChild(newOutputExprs, newAggChild); } private static Expression replaceAggFunc(AggregateFunction func, Slot inputSlot) { diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PushDownMinMaxSumThroughJoinTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PushDownMinMaxSumThroughJoinTest.java index 58ab7fbe9e925f..cffe91045d0ab2 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PushDownMinMaxSumThroughJoinTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PushDownMinMaxSumThroughJoinTest.java @@ -323,11 +323,11 @@ void testSingleCountStar() { .applyTopDown(new PushDownAggThroughJoinOneSide()) .printlnTree() .matches( - logicalAggregate( - logicalJoin( - logicalOlapScan(), + logicalJoin( + logicalAggregate( logicalOlapScan() - ) + ), + logicalOlapScan() ) ); } @@ -346,11 +346,9 @@ void testBothSideCountAndCountStar() { PlanChecker.from(MemoTestUtils.createConnectContext(), plan) .applyTopDown(new PushDownAggThroughJoinOneSide()) .matches( - logicalAggregate( - logicalJoin( - logicalOlapScan(), - logicalOlapScan() - ) + logicalJoin( + logicalAggregate(logicalOlapScan()), + logicalAggregate(logicalOlapScan()) ) ); } diff --git a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_count_through_join_one_side.out b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_count_through_join_one_side.out index da69919becd7f2..8267eb3e38ff91 100644 --- a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_count_through_join_one_side.out +++ b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_count_through_join_one_side.out @@ -1034,3 +1034,25 @@ Used: UnUsed: use_push_down_agg_through_join_one_side SyntaxError: +-- !shape -- +PhysicalResultSink +--PhysicalTopN[MERGE_SORT] +----PhysicalTopN[LOCAL_SORT] +------hashAgg[GLOBAL] +--------hashAgg[LOCAL] +----------hashJoin[INNER_JOIN] hashCondition=((dwd_tracking_sensor_init_tmp_ymd.dt = dw_user_b2c_tracking_info_tmp_ymd.dt) and (dwd_tracking_sensor_init_tmp_ymd.guid = dw_user_b2c_tracking_info_tmp_ymd.guid)) otherCondition=((dwd_tracking_sensor_init_tmp_ymd.dt >= substring(first_visit_time, 1, 10))) +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------filter((dwd_tracking_sensor_init_tmp_ymd.dt = '2024-08-19') and (dwd_tracking_sensor_init_tmp_ymd.tracking_type = 'click')) +------------------PhysicalOlapScan[dwd_tracking_sensor_init_tmp_ymd] +------------filter((dw_user_b2c_tracking_info_tmp_ymd.dt = '2024-08-19')) +--------------PhysicalOlapScan[dw_user_b2c_tracking_info_tmp_ymd] + +Hint log: +Used: use_PUSH_DOWN_AGG_THROUGH_JOIN_ONE_SIDE +UnUsed: +SyntaxError: + +-- !agg_pushed -- +2 是 2024-08-19 + diff --git a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_through_join_one_side.groovy b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_through_join_one_side.groovy index 02e06710296333..e551fa04c9110a 100644 --- a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_through_join_one_side.groovy +++ b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_through_join_one_side.groovy @@ -426,4 +426,99 @@ suite("push_down_count_through_join_one_side") { qt_with_hint_groupby_pushdown_nested_queries """ explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ count(*) from (select * from count_t_one_side where score > 20) t1 join (select * from count_t_one_side where id < 100) t2 on t1.id = t2.id group by t1.name; """ + + sql """ + drop table if exists dw_user_b2c_tracking_info_tmp_ymd; + create table dw_user_b2c_tracking_info_tmp_ymd ( + guid int, + dt varchar, + first_visit_time varchar + )Engine=Olap + DUPLICATE KEY(guid) + distributed by hash(dt) buckets 3 + properties('replication_num' = '1'); + + insert into dw_user_b2c_tracking_info_tmp_ymd values (1, '2024-08-19', '2024-08-19'); + + drop table if exists dwd_tracking_sensor_init_tmp_ymd; + create table dwd_tracking_sensor_init_tmp_ymd ( + guid int, + dt varchar, + tracking_type varchar + )Engine=Olap + DUPLICATE KEY(guid) + distributed by hash(dt) buckets 3 + properties('replication_num' = '1'); + + insert into dwd_tracking_sensor_init_tmp_ymd values(1, '2024-08-19', 'click'), (1, '2024-08-19', 'click'); + """ + sql """ + set ENABLE_NEREIDS_RULES = "PUSH_DOWN_AGG_THROUGH_JOIN_ONE_SIDE"; + set disable_join_reorder=true; + """ + + qt_shape """ + explain shape plan + SELECT /*+use_cbo_rule(PUSH_DOWN_AGG_THROUGH_JOIN_ONE_SIDE)*/ + Count(*) AS accee593, + CASE + WHEN dwd_tracking_sensor_init_tmp_ymd.dt = + Substring(dw_user_b2c_tracking_info_tmp_ymd.first_visit_time, 1, + 10) THEN + '是' + WHEN dwd_tracking_sensor_init_tmp_ymd.dt > + Substring(dw_user_b2c_tracking_info_tmp_ymd.first_visit_time, 1, + 10) THEN + '否' + ELSE '-1' + end AS a1302fb2, + dwd_tracking_sensor_init_tmp_ymd.dt AS ad466123 + FROM dwd_tracking_sensor_init_tmp_ymd + LEFT JOIN dw_user_b2c_tracking_info_tmp_ymd + ON dwd_tracking_sensor_init_tmp_ymd.guid = + dw_user_b2c_tracking_info_tmp_ymd.guid + AND dwd_tracking_sensor_init_tmp_ymd.dt = + dw_user_b2c_tracking_info_tmp_ymd.dt + WHERE dwd_tracking_sensor_init_tmp_ymd.dt = '2024-08-19' + AND dw_user_b2c_tracking_info_tmp_ymd.dt = '2024-08-19' + AND dwd_tracking_sensor_init_tmp_ymd.dt >= + Substring(dw_user_b2c_tracking_info_tmp_ymd.first_visit_time, 1, 10) + AND dwd_tracking_sensor_init_tmp_ymd.tracking_type = 'click' + GROUP BY 2, + 3 + ORDER BY 3 ASC + LIMIT 10000; + """ + + qt_agg_pushed """ + SELECT /*+use_cbo_rule(PUSH_DOWN_AGG_THROUGH_JOIN_ONE_SIDE)*/ + Count(*) AS accee593, + CASE + WHEN dwd_tracking_sensor_init_tmp_ymd.dt = + Substring(dw_user_b2c_tracking_info_tmp_ymd.first_visit_time, 1, + 10) THEN + '是' + WHEN dwd_tracking_sensor_init_tmp_ymd.dt > + Substring(dw_user_b2c_tracking_info_tmp_ymd.first_visit_time, 1, + 10) THEN + '否' + ELSE '-1' + end AS a1302fb2, + dwd_tracking_sensor_init_tmp_ymd.dt AS ad466123 + FROM dwd_tracking_sensor_init_tmp_ymd + LEFT JOIN dw_user_b2c_tracking_info_tmp_ymd + ON dwd_tracking_sensor_init_tmp_ymd.guid = + dw_user_b2c_tracking_info_tmp_ymd.guid + AND dwd_tracking_sensor_init_tmp_ymd.dt = + dw_user_b2c_tracking_info_tmp_ymd.dt + WHERE dwd_tracking_sensor_init_tmp_ymd.dt = '2024-08-19' + AND dw_user_b2c_tracking_info_tmp_ymd.dt = '2024-08-19' + AND dwd_tracking_sensor_init_tmp_ymd.dt >= + Substring(dw_user_b2c_tracking_info_tmp_ymd.first_visit_time, 1, 10) + AND dwd_tracking_sensor_init_tmp_ymd.tracking_type = 'click' + GROUP BY 2, + 3 + ORDER BY 3 ASC + LIMIT 10000; + """ } From b7d985d8ff85504b20e6fd59597b143797cee020 Mon Sep 17 00:00:00 2001 From: LiBinfeng <1204975323@qq.com> Date: Thu, 28 Nov 2024 11:36:11 +0800 Subject: [PATCH 051/399] [feat](Nereids) support cancel commands (#44422) Support cancelLoadCommand cancelExportCommand cancelWarmUpJobCommand --- .../org/apache/doris/nereids/DorisParser.g4 | 10 +- .../doris/analysis/CancelExportStmt.java | 9 + .../apache/doris/analysis/CancelLoadStmt.java | 9 + .../apache/doris/job/manager/JobManager.java | 89 ++++++++++ .../java/org/apache/doris/load/ExportMgr.java | 74 ++++++++ .../apache/doris/load/loadv2/LoadManager.java | 94 +++++++++- .../nereids/parser/LogicalPlanBuilder.java | 38 ++++ .../doris/nereids/trees/plans/PlanType.java | 3 + .../trees/plans/commands/CancelCommand.java | 162 ++++++++++++++++++ .../plans/commands/CancelExportCommand.java | 89 ++++++++++ .../plans/commands/CancelJobTaskCommand.java | 2 +- .../plans/commands/CancelLoadCommand.java | 94 ++++++++++ .../commands/CancelWarmUpJobCommand.java | 116 +++++++++++++ .../trees/plans/visitor/CommandVisitor.java | 15 ++ 14 files changed, 792 insertions(+), 12 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CancelCommand.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CancelExportCommand.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CancelLoadCommand.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CancelWarmUpJobCommand.java diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index cee972a63d210b..279329089a4ade 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -59,6 +59,7 @@ statementBase | supportedRefreshStatement #supportedRefreshStatementAlias | supportedShowStatement #supportedShowStatementAlias | supportedLoadStatement #supportedLoadStatementAlias + | supportedCancelStatement #supportedCancelStatementAlias | supportedRecoverStatement #supportedRecoverStatementAlias | supportedAdminStatement #supportedAdminStatementAlias | unsupportedStatement #unsupported @@ -455,10 +456,14 @@ unsupportedCleanStatement | CLEAN ALL QUERY STATS #cleanAllQueryStats ; -unsupportedCancelStatement +supportedCancelStatement : CANCEL LOAD ((FROM | IN) database=identifier)? wildWhere? #cancelLoad | CANCEL EXPORT ((FROM | IN) database=identifier)? wildWhere? #cancelExport - | CANCEL ALTER TABLE (ROLLUP | (MATERIALIZED VIEW) | COLUMN) + | CANCEL WARM UP JOB wildWhere? #cancelWarmUpJob + ; + +unsupportedCancelStatement + : CANCEL ALTER TABLE (ROLLUP | (MATERIALIZED VIEW) | COLUMN) FROM tableName=multipartIdentifier (LEFT_PAREN jobIds+=INTEGER_VALUE (COMMA jobIds+=INTEGER_VALUE)* RIGHT_PAREN)? #cancelAlterTable | CANCEL BUILD INDEX ON tableName=multipartIdentifier @@ -468,7 +473,6 @@ unsupportedCancelStatement (COMMA hostPorts+=STRING_LITERAL)* #cancelDecommisionBackend | CANCEL BACKUP ((FROM | IN) database=identifier)? #cancelBackup | CANCEL RESTORE ((FROM | IN) database=identifier)? #cancelRestore - | CANCEL WARM UP JOB wildWhere? #cancelWarmUp ; supportedAdminStatement diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/CancelExportStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/CancelExportStmt.java index 63448cf35a027a..fa6d7640781e17 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/CancelExportStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/CancelExportStmt.java @@ -58,6 +58,15 @@ public CancelExportStmt(String dbName, Expr whereClause) { this.whereClause = whereClause; } + public CancelExportStmt(String dbName, Expr whereClause, String label, CompoundPredicate.Operator operator, + String state) { + this.dbName = dbName; + this.whereClause = whereClause; + this.label = label; + this.operator = operator; + this.state = state; + } + private void checkColumn(Expr expr, boolean like) throws AnalysisException { String inputCol = ((SlotRef) expr.getChild(0)).getColumnName(); if (!SUPPORT_COLUMNS.contains(inputCol.toLowerCase())) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/CancelLoadStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/CancelLoadStmt.java index 1371e308675708..bbffb2649738fb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/CancelLoadStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/CancelLoadStmt.java @@ -59,6 +59,15 @@ public CancelLoadStmt(String dbName, Expr whereClause) { this.SUPPORT_COLUMNS.add("state"); } + public CancelLoadStmt(String dbName, Expr whereClause, String label, CompoundPredicate.Operator operator, + String state) { + this.dbName = dbName; + this.whereClause = whereClause; + this.label = label; + this.operator = operator; + this.state = state; + } + private void checkColumn(Expr expr, boolean like) throws AnalysisException { String inputCol = ((SlotRef) expr.getChild(0)).getColumnName(); if (!SUPPORT_COLUMNS.contains(inputCol)) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/manager/JobManager.java b/fe/fe-core/src/main/java/org/apache/doris/job/manager/JobManager.java index 47a3a0c5c19d5c..db2ca6eb81d156 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/manager/JobManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/manager/JobManager.java @@ -41,6 +41,8 @@ import org.apache.doris.job.scheduler.JobScheduler; import org.apache.doris.load.loadv2.JobState; import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.nereids.trees.expressions.And; +import org.apache.doris.nereids.trees.expressions.BinaryOperator; import org.apache.doris.qe.ConnectContext; import com.google.common.collect.Lists; @@ -522,4 +524,91 @@ private static void addNeedCancelLoadJob(String label, String state, // job.updateLoadingStatus(beId, loadId, fragmentId, scannedRows, scannedBytes, isDone); // } // } + + /** + * used for nereids planner + */ + public void cancelLoadJob(String dbName, String label, String state, BinaryOperator operator) + throws JobException, AnalysisException, DdlException { + Database db = Env.getCurrentInternalCatalog().getDbOrDdlException(dbName); + // List of load jobs waiting to be cancelled + List unfinishedLoadJob; + readLock(); + try { + List loadJobs = Env.getCurrentEnv().getLabelProcessor().getJobs(db); + List matchLoadJobs = Lists.newArrayList(); + addNeedCancelLoadJob(label, state, operator, loadJobs, matchLoadJobs); + if (matchLoadJobs.isEmpty()) { + throw new JobException("Load job does not exist"); + } + // check state here + unfinishedLoadJob = + matchLoadJobs.stream().filter(InsertJob::isRunning) + .collect(Collectors.toList()); + if (unfinishedLoadJob.isEmpty()) { + throw new JobException("There is no uncompleted job"); + } + } finally { + readUnlock(); + } + // check auth + if (unfinishedLoadJob.size() > 1 || unfinishedLoadJob.get(0).getTableNames().isEmpty()) { + if (Env.getCurrentEnv().getAccessManager() + .checkDbPriv(ConnectContext.get(), InternalCatalog.INTERNAL_CATALOG_NAME, dbName, + PrivPredicate.LOAD)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_DBACCESS_DENIED_ERROR, "LOAD", + ConnectContext.get().getQualifiedUser(), + ConnectContext.get().getRemoteIP(), dbName); + } + } else { + for (String tableName : unfinishedLoadJob.get(0).getTableNames()) { + if (Env.getCurrentEnv().getAccessManager() + .checkTblPriv(ConnectContext.get(), InternalCatalog.INTERNAL_CATALOG_NAME, dbName, + tableName, + PrivPredicate.LOAD)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_TABLEACCESS_DENIED_ERROR, "LOAD", + ConnectContext.get().getQualifiedUser(), + ConnectContext.get().getRemoteIP(), dbName + ":" + tableName); + } + } + } + for (InsertJob loadJob : unfinishedLoadJob) { + try { + alterJobStatus(loadJob.getJobId(), JobStatus.STOPPED); + } catch (JobException e) { + log.warn("Fail to cancel job, its label: {}", loadJob.getLabelName()); + } + } + } + + private static void addNeedCancelLoadJob(String label, String state, + BinaryOperator operator, List loadJobs, + List matchLoadJobs) + throws AnalysisException { + PatternMatcher matcher = PatternMatcherWrapper.createMysqlPattern(label, + CaseSensibility.LABEL.getCaseSensibility()); + matchLoadJobs.addAll( + loadJobs.stream() + .filter(job -> !job.isCancelled()) + .filter(job -> { + if (operator != null) { + // compound + boolean labelFilter = + label.contains("%") ? matcher.match(job.getLabelName()) + : job.getLabelName().equalsIgnoreCase(label); + boolean stateFilter = job.getJobStatus().name().equalsIgnoreCase(state); + return operator instanceof And ? labelFilter && stateFilter : + labelFilter || stateFilter; + } + if (StringUtils.isNotEmpty(label)) { + return label.contains("%") ? matcher.match(job.getLabelName()) + : job.getLabelName().equalsIgnoreCase(label); + } + if (StringUtils.isNotEmpty(state)) { + return job.getJobStatus().name().equalsIgnoreCase(state); + } + return false; + }).collect(Collectors.toList()) + ); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/ExportMgr.java b/fe/fe-core/src/main/java/org/apache/doris/load/ExportMgr.java index 49ebbfe7dcddb0..398cc8f4ac2518 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/ExportMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/ExportMgr.java @@ -38,6 +38,8 @@ import org.apache.doris.common.util.TimeUtils; import org.apache.doris.datasource.InternalCatalog; import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.nereids.trees.expressions.BinaryOperator; +import org.apache.doris.nereids.trees.expressions.Or; import org.apache.doris.qe.ConnectContext; import org.apache.doris.scheduler.exception.JobException; @@ -160,6 +162,78 @@ public void cancelExportJob(CancelExportStmt stmt) throws DdlException, Analysis } } + private List getWaitingCancelJobs(String label, String state, BinaryOperator operator) + throws AnalysisException { + Predicate jobFilter = buildCancelJobFilter(label, state, operator); + readLock(); + try { + return getJobs().stream().filter(jobFilter).collect(Collectors.toList()); + } finally { + readUnlock(); + } + } + + @VisibleForTesting + public static Predicate buildCancelJobFilter(String label, String state, BinaryOperator operator) + throws AnalysisException { + PatternMatcher matcher = PatternMatcherWrapper.createMysqlPattern(label, + CaseSensibility.LABEL.getCaseSensibility()); + + return job -> { + boolean labelFilter = true; + boolean stateFilter = true; + if (StringUtils.isNotEmpty(label)) { + labelFilter = label.contains("%") ? matcher.match(job.getLabel()) : + job.getLabel().equalsIgnoreCase(label); + } + if (StringUtils.isNotEmpty(state)) { + stateFilter = job.getState().name().equalsIgnoreCase(state); + } + + if (operator != null && operator instanceof Or) { + return labelFilter || stateFilter; + } + + return labelFilter && stateFilter; + }; + } + + /** + * used for Nereids planner + */ + public void cancelExportJob(String label, String state, BinaryOperator operator, String dbName) + throws DdlException, AnalysisException { + // List of export jobs waiting to be cancelled + List matchExportJobs = getWaitingCancelJobs(label, state, operator); + if (matchExportJobs.isEmpty()) { + throw new DdlException("Export job(s) do not exist"); + } + matchExportJobs = matchExportJobs.stream() + .filter(job -> !job.isFinalState()).collect(Collectors.toList()); + if (matchExportJobs.isEmpty()) { + throw new DdlException("All export job(s) are at final state (CANCELLED/FINISHED)"); + } + + // check auth + checkCancelExportJobAuth(InternalCatalog.INTERNAL_CATALOG_NAME, dbName, matchExportJobs); + // Must add lock to protect export job. + // Because job may be cancelled when generating task executors, + // the cancel process may clear the task executor list at same time, + // which will cause ConcurrentModificationException + writeLock(); + try { + for (ExportJob exportJob : matchExportJobs) { + // exportJob.cancel(ExportFailMsg.CancelType.USER_CANCEL, "user cancel"); + exportJob.updateExportJobState(ExportJobState.CANCELLED, 0L, null, + ExportFailMsg.CancelType.USER_CANCEL, "user cancel"); + } + } catch (JobException e) { + throw new AnalysisException(e.getMessage()); + } finally { + writeUnlock(); + } + } + public void checkCancelExportJobAuth(String ctlName, String dbName, List jobs) throws AnalysisException { if (jobs.size() > 1) { if (!Env.getCurrentEnv().getAccessManager() diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadManager.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadManager.java index 07c459d61cf056..5c39a27e24f828 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadManager.java @@ -46,6 +46,8 @@ import org.apache.doris.load.FailMsg.CancelType; import org.apache.doris.load.Load; import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.nereids.trees.expressions.And; +import org.apache.doris.nereids.trees.expressions.BinaryOperator; import org.apache.doris.persist.CleanLabelOperationLog; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.OriginStatement; @@ -246,23 +248,22 @@ public void recordFinishedLoadJob(String label, long transactionId, String dbNam * Match need cancel loadJob by stmt. **/ @VisibleForTesting - public static void addNeedCancelLoadJob(CancelLoadStmt stmt, List loadJobs, List matchLoadJobs) + public static void addNeedCancelLoadJob(String label, String state, BinaryOperator operator, + List loadJobs, List matchLoadJobs) throws AnalysisException { - String label = stmt.getLabel(); - String state = stmt.getState(); PatternMatcher matcher = PatternMatcherWrapper.createMysqlPattern(label, CaseSensibility.LABEL.getCaseSensibility()); matchLoadJobs.addAll( loadJobs.stream() .filter(job -> job.getState() != JobState.CANCELLED) .filter(job -> { - if (stmt.getOperator() != null) { + if (operator != null) { // compound boolean labelFilter = label.contains("%") ? matcher.match(job.getLabel()) : job.getLabel().equalsIgnoreCase(label); boolean stateFilter = job.getState().name().equalsIgnoreCase(state); - return Operator.AND.equals(stmt.getOperator()) ? labelFilter && stateFilter : + return operator instanceof And ? labelFilter && stateFilter : labelFilter || stateFilter; } if (StringUtils.isNotEmpty(label)) { @@ -280,8 +281,9 @@ public static void addNeedCancelLoadJob(CancelLoadStmt stmt, List loadJ /** * Cancel load job by stmt. **/ - public void cancelLoadJob(CancelLoadStmt stmt) throws DdlException, AnalysisException { - Database db = Env.getCurrentInternalCatalog().getDbOrDdlException(stmt.getDbName()); + public void cancelLoadJob(String dbName, String label, String state, BinaryOperator operator) + throws DdlException, AnalysisException { + Database db = Env.getCurrentInternalCatalog().getDbOrDdlException(dbName); // List of load jobs waiting to be cancelled List unfinishedLoadJob; readLock(); @@ -291,7 +293,7 @@ public void cancelLoadJob(CancelLoadStmt stmt) throws DdlException, AnalysisExce throw new DdlException("Load job does not exist"); } List matchLoadJobs = Lists.newArrayList(); - addNeedCancelLoadJob(stmt, + addNeedCancelLoadJob(label, state, operator, labelToLoadJobs.values().stream().flatMap(Collection::stream).collect(Collectors.toList()), matchLoadJobs); if (matchLoadJobs.isEmpty()) { @@ -318,6 +320,82 @@ public void cancelLoadJob(CancelLoadStmt stmt) throws DdlException, AnalysisExce } } + /** + * Match need cancel loadJob by stmt. + **/ + @VisibleForTesting + public static void addNeedCancelLoadJob(CancelLoadStmt stmt, List loadJobs, List matchLoadJobs) + throws AnalysisException { + String label = stmt.getLabel(); + String state = stmt.getState(); + PatternMatcher matcher = PatternMatcherWrapper.createMysqlPattern(label, + CaseSensibility.LABEL.getCaseSensibility()); + matchLoadJobs.addAll( + loadJobs.stream() + .filter(job -> job.getState() != JobState.CANCELLED) + .filter(job -> { + if (stmt.getOperator() != null) { + // compound + boolean labelFilter = + label.contains("%") ? matcher.match(job.getLabel()) + : job.getLabel().equalsIgnoreCase(label); + boolean stateFilter = job.getState().name().equalsIgnoreCase(state); + return Operator.AND.equals(stmt.getOperator()) ? labelFilter && stateFilter : + labelFilter || stateFilter; + } + if (StringUtils.isNotEmpty(label)) { + return label.contains("%") ? matcher.match(job.getLabel()) + : job.getLabel().equalsIgnoreCase(label); + } + if (StringUtils.isNotEmpty(state)) { + return job.getState().name().equalsIgnoreCase(state); + } + return false; + }).collect(Collectors.toList()) + ); + } + + /** + * Cancel load job by stmt. + **/ + public void cancelLoadJob(CancelLoadStmt stmt) throws DdlException, AnalysisException { + Database db = Env.getCurrentInternalCatalog().getDbOrDdlException(stmt.getDbName()); + // List of load jobs waiting to be cancelled + List unfinishedLoadJob; + readLock(); + try { + Map> labelToLoadJobs = dbIdToLabelToLoadJobs.get(db.getId()); + if (labelToLoadJobs == null) { + throw new DdlException("Load job does not exist"); + } + List matchLoadJobs = Lists.newArrayList(); + addNeedCancelLoadJob(stmt, + labelToLoadJobs.values().stream().flatMap(Collection::stream).collect(Collectors.toList()), + matchLoadJobs); + if (matchLoadJobs.isEmpty()) { + throw new DdlException("Load job does not exist"); + } + // check state here + unfinishedLoadJob = + matchLoadJobs.stream().filter(entity -> !entity.isTxnDone()).collect(Collectors.toList()); + if (unfinishedLoadJob.isEmpty()) { + throw new DdlException("There is no uncompleted job"); + } + } finally { + readUnlock(); + } + for (LoadJob loadJob : unfinishedLoadJob) { + try { + loadJob.cancelJob(new FailMsg(FailMsg.CancelType.USER_CANCEL, "user cancel")); + } catch (DdlException e) { + throw new DdlException( + "Cancel load job [" + loadJob.getId() + "] fail, " + "label=[" + loadJob.getLabel() + + + "] failed msg=" + e.getMessage()); + } + } + } + /** * Replay end load job. **/ diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index 1a49c7ba65fa56..abd6afb84bc7f2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -457,8 +457,11 @@ import org.apache.doris.nereids.trees.plans.commands.AlterWorkloadGroupCommand; import org.apache.doris.nereids.trees.plans.commands.AlterWorkloadPolicyCommand; import org.apache.doris.nereids.trees.plans.commands.CallCommand; +import org.apache.doris.nereids.trees.plans.commands.CancelExportCommand; import org.apache.doris.nereids.trees.plans.commands.CancelJobTaskCommand; +import org.apache.doris.nereids.trees.plans.commands.CancelLoadCommand; import org.apache.doris.nereids.trees.plans.commands.CancelMTMVTaskCommand; +import org.apache.doris.nereids.trees.plans.commands.CancelWarmUpJobCommand; import org.apache.doris.nereids.trees.plans.commands.CleanAllProfileCommand; import org.apache.doris.nereids.trees.plans.commands.Command; import org.apache.doris.nereids.trees.plans.commands.Constraint; @@ -1086,6 +1089,41 @@ public ShowCreateMTMVCommand visitShowCreateMTMV(ShowCreateMTMVContext ctx) { return new ShowCreateMTMVCommand(new ShowCreateMTMVInfo(new TableNameInfo(nameParts))); } + @Override + public CancelExportCommand visitCancelExport(DorisParser.CancelExportContext ctx) { + String databaseName = null; + if (ctx.database != null) { + databaseName = stripQuotes(ctx.database.getText()); + } + Expression wildWhere = null; + if (ctx.wildWhere() != null) { + wildWhere = getWildWhere(ctx.wildWhere()); + } + return new CancelExportCommand(databaseName, wildWhere); + } + + @Override + public CancelLoadCommand visitCancelLoad(DorisParser.CancelLoadContext ctx) { + String databaseName = null; + if (ctx.database != null) { + databaseName = stripQuotes(ctx.database.getText()); + } + Expression wildWhere = null; + if (ctx.wildWhere() != null) { + wildWhere = getWildWhere(ctx.wildWhere()); + } + return new CancelLoadCommand(databaseName, wildWhere); + } + + @Override + public CancelWarmUpJobCommand visitCancelWarmUpJob(DorisParser.CancelWarmUpJobContext ctx) { + Expression wildWhere = null; + if (ctx.wildWhere() != null) { + wildWhere = getWildWhere(ctx.wildWhere()); + } + return new CancelWarmUpJobCommand(wildWhere); + } + @Override public CancelMTMVTaskCommand visitCancelMTMVTask(CancelMTMVTaskContext ctx) { List nameParts = visitMultipartIdentifier(ctx.mvName); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index 20d46d612d02ac..09fa71f5614e87 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -158,6 +158,9 @@ public enum PlanType { PAUSE_MTMV_COMMAND, RESUME_MTMV_COMMAND, SHOW_CREATE_MTMV_COMMAND, + CANCEL_EXPORT_COMMAND, + CANCEL_LOAD_COMMAND, + CANCEL_WARM_UP_JOB_COMMAND, CANCEL_MTMV_TASK_COMMAND, CALL_COMMAND, CREATE_PROCEDURE_COMMAND, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CancelCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CancelCommand.java new file mode 100644 index 00000000000000..88ecf3cba1e6d8 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CancelCommand.java @@ -0,0 +1,162 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.commands; + +import org.apache.doris.analysis.Expr; +import org.apache.doris.analysis.SlotRef; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.nereids.CascadesContext; +import org.apache.doris.nereids.analyzer.UnboundSlot; +import org.apache.doris.nereids.glue.translator.ExpressionTranslator; +import org.apache.doris.nereids.glue.translator.PlanTranslatorContext; +import org.apache.doris.nereids.properties.PhysicalProperties; +import org.apache.doris.nereids.trees.expressions.BinaryOperator; +import org.apache.doris.nereids.trees.expressions.CompoundPredicate; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.Like; +import org.apache.doris.nereids.trees.expressions.Not; +import org.apache.doris.nereids.trees.expressions.literal.StringLikeLiteral; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.logical.LogicalEmptyRelation; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.StmtExecutor; + +import com.google.common.base.Strings; + +import java.util.ArrayList; +import java.util.Map; + +/** + * cancel load command + */ +public abstract class CancelCommand extends Command implements ForwardWithSync { + public CancelCommand(PlanType type) { + super(type); + } + + @Override + public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { + + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return null; + } + + /** + * translate to legacy expr, which do not need complex expression and table columns + */ + public Expr translateToLegacyExpr(ConnectContext ctx, Expression expression) { + LogicalEmptyRelation plan = new LogicalEmptyRelation( + ConnectContext.get().getStatementContext().getNextRelationId(), + new ArrayList<>()); + CascadesContext cascadesContext = CascadesContext.initContext(ctx.getStatementContext(), plan, + PhysicalProperties.ANY); + PlanTranslatorContext planTranslatorContext = new PlanTranslatorContext(cascadesContext); + ExpressionToExpr translator = new ExpressionToExpr(); + return expression.accept(translator, planTranslatorContext); + } + + private static class ExpressionToExpr extends ExpressionTranslator { + @Override + public Expr visitUnboundSlot(UnboundSlot unboundSlot, PlanTranslatorContext context) { + String inputCol = unboundSlot.getName(); + return new SlotRef(null, inputCol); + } + } + + /** + * check where filter for cancel load/export commands + * @param expression where clause + * @param supportedColumns only these kind of columns is supported + * @throws AnalysisException analyze exceptions + */ + public void checkWhereFilter(Expression expression, Map supportedColumns) throws AnalysisException { + if (null == expression) { + throw new AnalysisException("Where clause can't be null"); + } else if (expression instanceof Like) { + likeCheck(expression, supportedColumns); + } else if (expression instanceof BinaryOperator) { + binaryCheck(expression, supportedColumns); + } else if (expression instanceof CompoundPredicate) { + compoundCheck(expression, supportedColumns); + } else { + throw new AnalysisException("Only support like/binary/compound predicate"); + } + } + + private void checkColumn(Expression expr, boolean like, Map supportedColumns) + throws AnalysisException { + if (!(expr.child(0) instanceof UnboundSlot)) { + throw new AnalysisException("Current only support label and state, invalid column: " + + expr.child(0).toSql()); + } + String inputCol = ((UnboundSlot) expr.child(0)).getName(); + if (!supportedColumns.keySet().contains(inputCol.toLowerCase())) { + throw new AnalysisException("Current only support label and state, invalid column: " + inputCol); + } + if (!(expr.child(1) instanceof StringLikeLiteral)) { + throw new AnalysisException("Value must be a string"); + } + + String inputValue = ((StringLikeLiteral) expr.child(1)).getStringValue(); + if (Strings.isNullOrEmpty(inputValue)) { + throw new AnalysisException("Value can't be null"); + } + + if (inputCol.equalsIgnoreCase("label")) { + supportedColumns.put("label", inputValue); + } + + if (inputCol.equalsIgnoreCase("state")) { + if (like) { + throw new AnalysisException("Only label can use like"); + } + supportedColumns.put("state", inputValue); + } + } + + private void likeCheck(Expression expr, Map supportedColumns) throws AnalysisException { + checkColumn(expr, true, supportedColumns); + } + + private void binaryCheck(Expression expr, Map supportedColumns) throws AnalysisException { + checkColumn(expr, false, supportedColumns); + } + + private void compoundCheck(Expression expr, Map supportedColumns) throws AnalysisException { + // current only support label and state + if (expr instanceof Not) { + throw new AnalysisException("Current not support NOT operator"); + } + for (int i = 0; i < 2; i++) { + Expression child = expr.child(i); + if (child instanceof CompoundPredicate) { + throw new AnalysisException("Current not support nested clause"); + } else if (child instanceof Like) { + likeCheck(child, supportedColumns); + } else if (child instanceof BinaryOperator) { + binaryCheck(child, supportedColumns); + } else { + throw new AnalysisException("Only support like/binary predicate"); + } + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CancelExportCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CancelExportCommand.java new file mode 100644 index 00000000000000..f7defefb422476 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CancelExportCommand.java @@ -0,0 +1,89 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.commands; + +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.UserException; +import org.apache.doris.load.ExportJobState; +import org.apache.doris.nereids.trees.expressions.BinaryOperator; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.StmtExecutor; + +import com.google.common.base.Strings; + +import java.util.HashMap; +import java.util.Map; + +/** + * cancel export command + */ +public class CancelExportCommand extends CancelCommand implements ForwardWithSync { + private Map supportedColumns = new HashMap<>(); + + private String dbName; + + private String label; + + private String state; + + private Expression whereClause; + + public CancelExportCommand(String dbName, Expression whereClause) { + super(PlanType.CANCEL_EXPORT_COMMAND); + this.dbName = dbName; + this.whereClause = whereClause; + } + + @Override + public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { + validate(ctx); + ctx.getEnv().getExportMgr().cancelExportJob(label, state, (BinaryOperator) whereClause, dbName); + } + + private void validate(ConnectContext ctx) throws UserException { + if (Strings.isNullOrEmpty(dbName)) { + dbName = ctx.getDatabase(); + if (Strings.isNullOrEmpty(dbName)) { + throw new AnalysisException("No database selected"); + } + } + + supportedColumns.put("label", ""); + supportedColumns.put("state", ""); + checkWhereFilter(whereClause, supportedColumns); + if (!Strings.isNullOrEmpty(supportedColumns.get("label"))) { + label = supportedColumns.get("label"); + } + if (!Strings.isNullOrEmpty(supportedColumns.get("state"))) { + state = supportedColumns.get("state"); + ExportJobState jobState = ExportJobState.valueOf(state); + if (jobState != ExportJobState.PENDING + && jobState != ExportJobState.EXPORTING) { + throw new AnalysisException("Only support PENDING/EXPORTING, invalid state: " + state); + } + } + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitCancelExportCommand(this, context); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CancelJobTaskCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CancelJobTaskCommand.java index a9ea241e3b63ec..e02c731911a18e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CancelJobTaskCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CancelJobTaskCommand.java @@ -36,7 +36,7 @@ /** * base class for all drop commands */ -public class CancelJobTaskCommand extends Command implements ForwardWithSync { +public class CancelJobTaskCommand extends CancelCommand implements ForwardWithSync { private static final String jobNameKey = "jobName"; private static final String taskIdKey = "taskId"; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CancelLoadCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CancelLoadCommand.java new file mode 100644 index 00000000000000..4d3068ed08f559 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CancelLoadCommand.java @@ -0,0 +1,94 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.commands; + +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.UserException; +import org.apache.doris.job.exception.JobException; +import org.apache.doris.load.ExportJobState; +import org.apache.doris.nereids.trees.expressions.BinaryOperator; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.StmtExecutor; + +import com.google.common.base.Strings; + +import java.util.HashMap; +import java.util.Map; + +/** + * cancel load command + */ +public class CancelLoadCommand extends CancelCommand implements ForwardWithSync { + private Map supportedColumns = new HashMap<>(); + + private String dbName; + + private String label; + + private String state; + + private Expression whereClause; + + public CancelLoadCommand(String dbName, Expression whereClause) { + super(PlanType.CANCEL_EXPORT_COMMAND); + this.dbName = dbName; + this.whereClause = whereClause; + } + + @Override + public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { + validate(ctx); + try { + ctx.getEnv().getJobManager().cancelLoadJob(dbName, label, state, (BinaryOperator) whereClause); + } catch (JobException e) { + ctx.getEnv().getLoadManager().cancelLoadJob(dbName, label, state, (BinaryOperator) whereClause); + } + } + + private void validate(ConnectContext ctx) throws UserException { + if (Strings.isNullOrEmpty(dbName)) { + dbName = ctx.getDatabase(); + if (Strings.isNullOrEmpty(dbName)) { + throw new AnalysisException("No database selected"); + } + } + + supportedColumns.put("label", ""); + supportedColumns.put("state", ""); + checkWhereFilter(whereClause, supportedColumns); + if (!Strings.isNullOrEmpty(supportedColumns.get("label"))) { + label = supportedColumns.get("label"); + } + if (!Strings.isNullOrEmpty(supportedColumns.get("state"))) { + state = supportedColumns.get("state"); + ExportJobState jobState = ExportJobState.valueOf(state); + if (jobState != ExportJobState.PENDING + && jobState != ExportJobState.EXPORTING) { + throw new AnalysisException("Only support PENDING/EXPORTING, invalid state: " + state); + } + } + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitCancelLoadCommand(this, context); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CancelWarmUpJobCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CancelWarmUpJobCommand.java new file mode 100644 index 00000000000000..b5d26b4c4213ff --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CancelWarmUpJobCommand.java @@ -0,0 +1,116 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.commands; + +import org.apache.doris.analysis.CancelCloudWarmUpStmt; +import org.apache.doris.analysis.Expr; +import org.apache.doris.cloud.catalog.CloudEnv; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.Config; +import org.apache.doris.nereids.CascadesContext; +import org.apache.doris.nereids.analyzer.UnboundSlot; +import org.apache.doris.nereids.glue.translator.ExpressionTranslator; +import org.apache.doris.nereids.glue.translator.PlanTranslatorContext; +import org.apache.doris.nereids.properties.PhysicalProperties; +import org.apache.doris.nereids.trees.expressions.EqualTo; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.literal.IntegerLiteral; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.logical.LogicalEmptyRelation; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.StmtExecutor; + +import java.util.ArrayList; + +/** + * cancel warm up job command + */ +public class CancelWarmUpJobCommand extends Command implements ForwardWithSync { + private Expression whereClause; + private long jobId; + private Expr legacyWhereClause; + + public CancelWarmUpJobCommand(Expression whereClause) { + super(PlanType.CANCEL_WARM_UP_JOB_COMMAND); + this.whereClause = whereClause; + } + + @Override + public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { + validate(ctx); + if (Config.isCloudMode()) { + CancelCloudWarmUpStmt stmt = new CancelCloudWarmUpStmt(legacyWhereClause); + ((CloudEnv) ctx.getEnv()).cancelCloudWarmUp(stmt); + } + } + + public long getJobId() { + return jobId; + } + + /** + * validate cloud warm up job + * @param ctx connect context + * @throws AnalysisException check whether this sql is legal + */ + public void validate(ConnectContext ctx) throws AnalysisException { + if (!Config.isCloudMode()) { + throw new AnalysisException("The sql is illegal in disk mode "); + } + if (whereClause == null) { + throw new AnalysisException("Missing job id"); + } + boolean valid = true; + CHECK: { + if (!(whereClause instanceof EqualTo)) { + valid = false; + break CHECK; + } + + // left child + if (!(whereClause.child(0) instanceof UnboundSlot)) { + valid = false; + break CHECK; + } + String leftKey = ((UnboundSlot) whereClause.child(0)).getName(); + if (leftKey.equalsIgnoreCase("id") && (whereClause.child(1) instanceof IntegerLiteral)) { + jobId = ((IntegerLiteral) whereClause.child(1)).getLongValue(); + } else { + valid = false; + } + } + + if (!valid) { + throw new AnalysisException("Where clause should looks like one of them: id = 123"); + } + + LogicalEmptyRelation plan = new LogicalEmptyRelation( + ConnectContext.get().getStatementContext().getNextRelationId(), + new ArrayList<>()); + CascadesContext cascadesContext = CascadesContext.initContext(ctx.getStatementContext(), plan, + PhysicalProperties.ANY); + PlanTranslatorContext planTranslatorContext = new PlanTranslatorContext(cascadesContext); + legacyWhereClause = ExpressionTranslator.translate(whereClause, planTranslatorContext); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitCancelWarmUpJobCommand(this, context); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index 0af9ce4dc9c4cc..ef1688c3398b82 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -25,8 +25,11 @@ import org.apache.doris.nereids.trees.plans.commands.AlterWorkloadGroupCommand; import org.apache.doris.nereids.trees.plans.commands.AlterWorkloadPolicyCommand; import org.apache.doris.nereids.trees.plans.commands.CallCommand; +import org.apache.doris.nereids.trees.plans.commands.CancelExportCommand; import org.apache.doris.nereids.trees.plans.commands.CancelJobTaskCommand; +import org.apache.doris.nereids.trees.plans.commands.CancelLoadCommand; import org.apache.doris.nereids.trees.plans.commands.CancelMTMVTaskCommand; +import org.apache.doris.nereids.trees.plans.commands.CancelWarmUpJobCommand; import org.apache.doris.nereids.trees.plans.commands.CleanAllProfileCommand; import org.apache.doris.nereids.trees.plans.commands.Command; import org.apache.doris.nereids.trees.plans.commands.CreateJobCommand; @@ -223,6 +226,18 @@ default R visitShowCreateMTMVCommand(ShowCreateMTMVCommand showCreateMTMVCommand return visitCommand(showCreateMTMVCommand, context); } + default R visitCancelLoadCommand(CancelLoadCommand cancelLoadCommand, C context) { + return visitCommand(cancelLoadCommand, context); + } + + default R visitCancelExportCommand(CancelExportCommand cancelExportCommand, C context) { + return visitCommand(cancelExportCommand, context); + } + + default R visitCancelWarmUpJobCommand(CancelWarmUpJobCommand cancelWarmUpJobCommand, C context) { + return visitCommand(cancelWarmUpJobCommand, context); + } + default R visitCancelMTMVTaskCommand(CancelMTMVTaskCommand cancelMTMVTaskCommand, C context) { return visitCommand(cancelMTMVTaskCommand, context); } From c4288a0c14914e02077cd94ecdb3e8efb59d4706 Mon Sep 17 00:00:00 2001 From: walter Date: Thu, 28 Nov 2024 12:01:14 +0800 Subject: [PATCH 052/399] [feat](binlog) Support drop rollup binlog (#44677) ref #43435 Related PR: https://github.com/selectdb/ccr-syncer/pull/269 --- .../doris/alter/MaterializedViewHandler.java | 16 +++++------ .../apache/doris/binlog/BinlogManager.java | 17 ++++++++++++ .../org/apache/doris/binlog/DBBinlog.java | 9 +++++++ .../doris/datasource/InternalCatalog.java | 4 +-- .../apache/doris/persist/BatchDropInfo.java | 12 +++++++-- .../org/apache/doris/persist/DropInfo.java | 20 ++++++++++++-- .../org/apache/doris/persist/EditLog.java | 27 ++++++++++++++----- .../doris/persist/DropAndRecoverInfoTest.java | 10 +++---- gensrc/thrift/FrontendService.thrift | 4 +-- 9 files changed, 90 insertions(+), 29 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java b/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java index d02e91a379f560..a6f1cae9987678 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java @@ -944,14 +944,12 @@ public void processBatchDropRollup(List dropRollupClauses, Database } // drop data in memory - Set indexIdSet = new HashSet<>(); - Set rollupNameSet = new HashSet<>(); + Map rollupNameMap = new HashMap<>(); for (AlterClause alterClause : dropRollupClauses) { DropRollupClause dropRollupClause = (DropRollupClause) alterClause; String rollupIndexName = dropRollupClause.getRollupName(); long rollupIndexId = dropMaterializedView(rollupIndexName, olapTable); - indexIdSet.add(rollupIndexId); - rollupNameSet.add(rollupIndexName); + rollupNameMap.put(rollupIndexId, rollupIndexName); } // batch log drop rollup operation @@ -959,10 +957,10 @@ public void processBatchDropRollup(List dropRollupClauses, Database long dbId = db.getId(); long tableId = olapTable.getId(); String tableName = olapTable.getName(); - editLog.logBatchDropRollup(new BatchDropInfo(dbId, tableId, tableName, indexIdSet)); - deleteIndexList = indexIdSet.stream().collect(Collectors.toList()); + editLog.logBatchDropRollup(new BatchDropInfo(dbId, tableId, tableName, rollupNameMap)); + deleteIndexList = rollupNameMap.keySet().stream().collect(Collectors.toList()); LOG.info("finished drop rollup index[{}] in table[{}]", - String.join("", rollupNameSet), olapTable.getName()); + String.join("", rollupNameMap.values()), olapTable.getName()); } finally { olapTable.writeUnlock(); } @@ -982,8 +980,8 @@ public void processDropMaterializedView(DropMaterializedViewStmt dropMaterialize long mvIndexId = dropMaterializedView(mvName, olapTable); // Step3: log drop mv operation EditLog editLog = Env.getCurrentEnv().getEditLog(); - editLog.logDropRollup( - new DropInfo(db.getId(), olapTable.getId(), olapTable.getName(), mvIndexId, false, false, 0)); + editLog.logDropRollup(new DropInfo(db.getId(), olapTable.getId(), olapTable.getName(), + mvIndexId, mvName, false, false, 0)); deleteIndexList.add(mvIndexId); LOG.info("finished drop materialized view [{}] in table [{}]", mvName, olapTable.getName()); } catch (MetaNotFoundException e) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/binlog/BinlogManager.java b/fe/fe-core/src/main/java/org/apache/doris/binlog/BinlogManager.java index 3a033c981038c2..67bb99a8bcdc18 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/binlog/BinlogManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/binlog/BinlogManager.java @@ -30,6 +30,7 @@ import org.apache.doris.persist.BarrierLog; import org.apache.doris.persist.BatchModifyPartitionsInfo; import org.apache.doris.persist.BinlogGcInfo; +import org.apache.doris.persist.DropInfo; import org.apache.doris.persist.DropPartitionInfo; import org.apache.doris.persist.ModifyCommentOperationLog; import org.apache.doris.persist.ModifyTablePropertyOperationLog; @@ -429,6 +430,22 @@ public void addIndexChangeJob(IndexChangeJob indexChangeJob, long commitSeq) { addBinlog(dbId, tableIds, commitSeq, timestamp, type, data, false, indexChangeJob); } + public void addDropRollup(DropInfo info, long commitSeq) { + if (StringUtils.isEmpty(info.getIndexName())) { + LOG.warn("skip drop rollup binlog, because indexName is empty. info: {}", info); + return; + } + + long dbId = info.getDbId(); + List tableIds = Lists.newArrayList(); + tableIds.add(info.getTableId()); + long timestamp = -1; + TBinlogType type = TBinlogType.DROP_ROLLUP; + String data = info.toJson(); + + addBinlog(dbId, tableIds, commitSeq, timestamp, type, data, false, info); + } + // get binlog by dbId, return first binlog.version > version public Pair getBinlog(long dbId, long tableId, long prevCommitSeq) { TStatus status = new TStatus(TStatusCode.OK); diff --git a/fe/fe-core/src/main/java/org/apache/doris/binlog/DBBinlog.java b/fe/fe-core/src/main/java/org/apache/doris/binlog/DBBinlog.java index c96e994be91c3a..b78ed389a0fe86 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/binlog/DBBinlog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/binlog/DBBinlog.java @@ -23,6 +23,7 @@ import org.apache.doris.common.Pair; import org.apache.doris.common.proc.BaseProcResult; import org.apache.doris.persist.BarrierLog; +import org.apache.doris.persist.DropInfo; import org.apache.doris.persist.DropPartitionInfo; import org.apache.doris.persist.ReplaceTableOperationLog; import org.apache.doris.thrift.TBinlog; @@ -649,6 +650,9 @@ private void recordDroppedResources(TBinlogType binlogType, long commitSeq, Stri case REPLACE_TABLE: raw = ReplaceTableOperationLog.fromJson(data); break; + case DROP_ROLLUP: + raw = DropInfo.fromJson(data); + break; case BARRIER: raw = BarrierLog.fromJson(data); break; @@ -693,6 +697,11 @@ private void recordDroppedResources(TBinlogType binlogType, long commitSeq, Obje if (!record.isSwapTable()) { droppedTables.add(Pair.of(record.getOrigTblId(), commitSeq)); } + } else if (binlogType == TBinlogType.DROP_ROLLUP && raw instanceof DropInfo) { + long indexId = ((DropInfo) raw).getIndexId(); + if (indexId > 0) { + droppedIndexes.add(Pair.of(indexId, commitSeq)); + } } else if (binlogType == TBinlogType.BARRIER && raw instanceof BarrierLog) { BarrierLog log = (BarrierLog) raw; // keep compatible with doris 2.0/2.1 diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java index 18f2e333e9119b..bdc6626f63df81 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java @@ -1015,7 +1015,7 @@ private void dropTableInternal(Database db, Table table, boolean isView, boolean Env.getCurrentEnv().getQueryStats().clear(Env.getCurrentEnv().getCurrentCatalog().getId(), db.getId(), table.getId()); - DropInfo info = new DropInfo(db.getId(), table.getId(), tableName, -1L, isView, forceDrop, recycleTime); + DropInfo info = new DropInfo(db.getId(), table.getId(), tableName, isView, forceDrop, recycleTime); Env.getCurrentEnv().getEditLog().logDropTable(info); Env.getCurrentEnv().getMtmvService().dropTable(table); } @@ -3259,7 +3259,7 @@ private boolean createOlapTable(Database db, CreateTableStmt stmt) throws UserEx try { dropTable(db, tableId, true, false, 0L); if (hadLogEditCreateTable) { - DropInfo info = new DropInfo(db.getId(), tableId, olapTable.getName(), -1L, false, true, 0L); + DropInfo info = new DropInfo(db.getId(), tableId, olapTable.getName(), false, true, 0L); Env.getCurrentEnv().getEditLog().logDropTable(info); } } catch (Exception ex) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/BatchDropInfo.java b/fe/fe-core/src/main/java/org/apache/doris/persist/BatchDropInfo.java index fdfc44e27bbebc..260ad316d3cc24 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/BatchDropInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/BatchDropInfo.java @@ -26,6 +26,7 @@ import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; +import java.util.Map; import java.util.Objects; import java.util.Set; @@ -43,12 +44,15 @@ public class BatchDropInfo implements Writable { private String tableName; // not used in equals and hashCode @SerializedName(value = "indexIdSet") private Set indexIdSet; + @SerializedName(value = "indexNameMap") + private Map indexNameMap; // not used in equals and hashCode - public BatchDropInfo(long dbId, long tableId, String tableName, Set indexIdSet) { + public BatchDropInfo(long dbId, long tableId, String tableName, Map indexNameMap) { this.dbId = dbId; this.tableId = tableId; this.tableName = tableName; - this.indexIdSet = indexIdSet; + this.indexIdSet = indexNameMap.keySet(); + this.indexNameMap = indexNameMap; } @Override @@ -82,6 +86,10 @@ public Set getIndexIdSet() { return indexIdSet; } + public Map getIndexNameMap() { + return indexNameMap; + } + public long getDbId() { return dbId; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/DropInfo.java b/fe/fe-core/src/main/java/org/apache/doris/persist/DropInfo.java index 461f3ddd67d5a7..69994caf23d5dc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/DropInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/DropInfo.java @@ -38,6 +38,8 @@ public class DropInfo implements Writable { private String tableName; // not used in equals and hashCode @SerializedName(value = "indexId") private long indexId; + @SerializedName(value = "indexName") + private String indexName; // not used in equals and hashCode @SerializedName(value = "isView") private boolean isView = false; @SerializedName(value = "forceDrop") @@ -48,12 +50,18 @@ public class DropInfo implements Writable { public DropInfo() { } - public DropInfo(long dbId, long tableId, String tableName, long indexId, boolean isView, boolean forceDrop, - long recycleTime) { + public DropInfo(long dbId, long tableId, String tableName, boolean isView, boolean forceDrop, + long recycleTime) { + this(dbId, tableId, tableName, -1, "", isView, forceDrop, recycleTime); + } + + public DropInfo(long dbId, long tableId, String tableName, long indexId, String indexName, boolean isView, + boolean forceDrop, long recycleTime) { this.dbId = dbId; this.tableId = tableId; this.tableName = tableName; this.indexId = indexId; + this.indexName = indexName; this.isView = isView; this.forceDrop = forceDrop; this.recycleTime = recycleTime; @@ -75,6 +83,10 @@ public long getIndexId() { return this.indexId; } + public String getIndexName() { + return this.indexName; + } + public boolean isView() { return this.isView; } @@ -133,4 +145,8 @@ public boolean equals(Object obj) { public String toJson() { return GsonUtils.GSON.toJson(this); } + + public static DropInfo fromJson(String json) { + return GsonUtils.GSON.fromJson(json, DropInfo.class); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java b/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java index 1e70eb634b2073..7d1f2127eecaaf 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java @@ -105,6 +105,7 @@ import java.io.IOException; import java.util.List; +import java.util.Map; /** * EditLog maintains a log of the memory modifications. @@ -341,15 +342,18 @@ public static void loadJournal(Env env, Long logId, JournalEntity journal) { case OperationType.OP_DROP_ROLLUP: { DropInfo info = (DropInfo) journal.getData(); env.getMaterializedViewHandler().replayDropRollup(info, env); + env.getBinlogManager().addDropRollup(info, logId); break; } case OperationType.OP_BATCH_DROP_ROLLUP: { BatchDropInfo batchDropInfo = (BatchDropInfo) journal.getData(); - for (long indexId : batchDropInfo.getIndexIdSet()) { - env.getMaterializedViewHandler().replayDropRollup( - new DropInfo(batchDropInfo.getDbId(), batchDropInfo.getTableId(), - batchDropInfo.getTableName(), indexId, false, false, 0), - env); + for (Map.Entry entry : batchDropInfo.getIndexNameMap().entrySet()) { + long indexId = entry.getKey(); + String indexName = entry.getValue(); + DropInfo info = new DropInfo(batchDropInfo.getDbId(), batchDropInfo.getTableId(), + batchDropInfo.getTableName(), indexId, indexName, false, false, 0); + env.getMaterializedViewHandler().replayDropRollup(info, env); + env.getBinlogManager().addDropRollup(info, logId); } break; } @@ -1463,11 +1467,20 @@ public void logRecoverTable(RecoverInfo info) { } public void logDropRollup(DropInfo info) { - logEdit(OperationType.OP_DROP_ROLLUP, info); + long logId = logEdit(OperationType.OP_DROP_ROLLUP, info); + Env.getCurrentEnv().getBinlogManager().addDropRollup(info, logId); } public void logBatchDropRollup(BatchDropInfo batchDropInfo) { - logEdit(OperationType.OP_BATCH_DROP_ROLLUP, batchDropInfo); + long logId = logEdit(OperationType.OP_BATCH_DROP_ROLLUP, batchDropInfo); + for (Map.Entry entry : batchDropInfo.getIndexNameMap().entrySet()) { + DropInfo info = new DropInfo(batchDropInfo.getDbId(), + batchDropInfo.getTableId(), + batchDropInfo.getTableName(), + entry.getKey(), entry.getValue(), + false, true, 0); + Env.getCurrentEnv().getBinlogManager().addDropRollup(info, logId); + } } public void logFinishConsistencyCheck(ConsistencyCheckInfo info) { diff --git a/fe/fe-core/src/test/java/org/apache/doris/persist/DropAndRecoverInfoTest.java b/fe/fe-core/src/test/java/org/apache/doris/persist/DropAndRecoverInfoTest.java index 88aa22ded22e5e..8c74fba275352d 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/persist/DropAndRecoverInfoTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/persist/DropAndRecoverInfoTest.java @@ -44,7 +44,7 @@ public void testDropInfoSerialization() throws Exception { DropInfo info1 = new DropInfo(); info1.write(dos); - DropInfo info2 = new DropInfo(1, 2, "t2", -1, false, true, 0); + DropInfo info2 = new DropInfo(1, 2, "t2", -1, "", false, true, 0); info2.write(dos); dos.flush(); @@ -65,10 +65,10 @@ public void testDropInfoSerialization() throws Exception { Assert.assertEquals(rInfo2, rInfo2); Assert.assertNotEquals(rInfo2, this); - Assert.assertNotEquals(info2, new DropInfo(0, 2, "t2", -1L, false, true, 0)); - Assert.assertNotEquals(info2, new DropInfo(1, 0, "t0", -1L, false, true, 0)); - Assert.assertNotEquals(info2, new DropInfo(1, 2, "t2", -1L, false, false, 0)); - Assert.assertEquals(info2, new DropInfo(1, 2, "t2", -1L, false, true, 0)); + Assert.assertNotEquals(info2, new DropInfo(0, 2, "t2", -1L, "", false, true, 0)); + Assert.assertNotEquals(info2, new DropInfo(1, 0, "t0", -1L, "", false, true, 0)); + Assert.assertNotEquals(info2, new DropInfo(1, 2, "t2", -1L, "", false, false, 0)); + Assert.assertEquals(info2, new DropInfo(1, 2, "t2", -1L, "", false, true, 0)); // 3. delete files dis.close(); diff --git a/gensrc/thrift/FrontendService.thrift b/gensrc/thrift/FrontendService.thrift index fc7f98e16e3f74..e2af8937425d0c 100644 --- a/gensrc/thrift/FrontendService.thrift +++ b/gensrc/thrift/FrontendService.thrift @@ -1196,6 +1196,7 @@ enum TBinlogType { INDEX_CHANGE_JOB = 20, RENAME_ROLLUP = 21, RENAME_PARTITION = 22, + DROP_ROLLUP = 23, // Keep some IDs for allocation so that when new binlog types are added in the // future, the changes can be picked back to the old versions without breaking @@ -1212,8 +1213,7 @@ enum TBinlogType { // MODIFY_XXX = 17, // MIN_UNKNOWN = 18, // UNKNOWN_3 = 19, - MIN_UNKNOWN = 23, - UNKNOWN_8 = 24, + MIN_UNKNOWN = 24, UNKNOWN_9 = 25, UNKNOWN_10 = 26, UNKNOWN_11 = 27, From 6ba91dc39c763018f9030b877b435aa9ea704c28 Mon Sep 17 00:00:00 2001 From: Gabriel Date: Thu, 28 Nov 2024 13:38:10 +0800 Subject: [PATCH 053/399] [fix](runtime filter) Avoid to use ignored runtime filter (#44708) ### What problem does this PR solve? Filter size should be set by a global size instead of a local size. This behaviour is introduced by #43885 . --- be/src/exprs/runtime_filter.cpp | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/be/src/exprs/runtime_filter.cpp b/be/src/exprs/runtime_filter.cpp index 6eed49e8567e1c..efe96ae3395432 100644 --- a/be/src/exprs/runtime_filter.cpp +++ b/be/src/exprs/runtime_filter.cpp @@ -1114,9 +1114,6 @@ Status IRuntimeFilter::send_filter_size(RuntimeState* state, uint64_t local_filt std::lock_guard l(*local_merge_filters->lock); local_merge_filters->merge_size_times--; local_merge_filters->local_merged_size += local_filter_size; - if (_has_local_target) { - set_synced_size(local_filter_size); - } if (local_merge_filters->merge_size_times) { return Status::OK(); } else { @@ -1539,9 +1536,10 @@ void IRuntimeFilter::update_runtime_filter_type_to_profile(uint64_t local_merge_ std::string IRuntimeFilter::debug_string() const { return fmt::format( "RuntimeFilter: (id = {}, type = {}, is_broadcast: {}, " - "build_bf_cardinality: {}, error_msg: {}", + "build_bf_cardinality: {}, ignored: {}, error_msg: {}", _filter_id, to_string(_runtime_filter_type), _is_broadcast_join, - _wrapper->get_build_bf_cardinality(), _wrapper->_context->err_msg); + _wrapper->get_build_bf_cardinality(), _wrapper->is_ignored(), + _wrapper->_context->err_msg); } Status IRuntimeFilter::merge_from(const RuntimePredicateWrapper* wrapper) { From 47a74146bb57a657190d377a137a02c1d894f210 Mon Sep 17 00:00:00 2001 From: Vallish Pai Date: Thu, 28 Nov 2024 11:31:31 +0530 Subject: [PATCH 054/399] [Enhancement] Support move truncated old data to recycle bin (#43107) --- .../org/apache/doris/nereids/DorisParser.g4 | 2 +- fe/fe-core/src/main/cup/sql_parser.cup | 4 +- .../doris/analysis/TruncateTableStmt.java | 14 +- .../org/apache/doris/catalog/OlapTable.java | 154 +++++++++++------- .../doris/catalog/RecyclePartitionParam.java | 31 ++++ .../doris/datasource/InternalCatalog.java | 54 +++--- .../doris/persist/TruncateTableInfo.java | 10 +- .../doris/catalog/TruncateTableTest.java | 2 +- .../test_insert_overwrite_multiple.out | 15 ++ .../test_truncate_multiple.out | 16 ++ .../test_truncate_recover.out | 22 +++ .../test_truncate_recover_list.out | 18 ++ .../test_truncate_recover_no_partition.out | 13 ++ .../test_truncate_recover_restart_fe.out | 27 +++ .../data/ddl_p0/test_truncate_table_force.out | 12 ++ .../test_insert_overwrite_multiple.groovy | 68 ++++++++ .../test_insert_overwrite_recover.groovy | 2 +- ...sert_overwrite_recover_no_partition.groovy | 4 +- .../test_truncate_multiple.groovy | 75 +++++++++ .../test_truncate_recover.groovy | 76 +++++++++ .../test_truncate_recover_list.groovy | 56 +++++++ .../test_truncate_recover_no_partition.groovy | 76 +++++++++ .../test_truncate_recover_restart_fe.groovy | 90 ++++++++++ .../suites/ddl_p0/test_truncate_table.groovy | 5 +- .../ddl_p0/test_truncate_table_force.groovy | 79 +++++++++ 25 files changed, 833 insertions(+), 92 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/catalog/RecyclePartitionParam.java create mode 100644 regression-test/data/catalog_recycle_bin_p0/test_insert_overwrite_multiple.out create mode 100644 regression-test/data/catalog_recycle_bin_p0/test_truncate_multiple.out create mode 100644 regression-test/data/catalog_recycle_bin_p0/test_truncate_recover.out create mode 100644 regression-test/data/catalog_recycle_bin_p0/test_truncate_recover_list.out create mode 100644 regression-test/data/catalog_recycle_bin_p0/test_truncate_recover_no_partition.out create mode 100644 regression-test/data/catalog_recycle_bin_p0/test_truncate_recover_restart_fe.out create mode 100644 regression-test/data/ddl_p0/test_truncate_table_force.out create mode 100644 regression-test/suites/catalog_recycle_bin_p0/test_insert_overwrite_multiple.groovy create mode 100644 regression-test/suites/catalog_recycle_bin_p0/test_truncate_multiple.groovy create mode 100644 regression-test/suites/catalog_recycle_bin_p0/test_truncate_recover.groovy create mode 100644 regression-test/suites/catalog_recycle_bin_p0/test_truncate_recover_list.groovy create mode 100644 regression-test/suites/catalog_recycle_bin_p0/test_truncate_recover_no_partition.groovy create mode 100644 regression-test/suites/catalog_recycle_bin_p0/test_truncate_recover_restart_fe.groovy create mode 100644 regression-test/suites/ddl_p0/test_truncate_table_force.groovy diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index 279329089a4ade..08f9058bbb14d6 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -869,7 +869,7 @@ unsupportedUseStatement ; unsupportedDmlStatement - : TRUNCATE TABLE multipartIdentifier specifiedPartition? #truncateTable + : TRUNCATE TABLE multipartIdentifier specifiedPartition? FORCE? #truncateTable | COPY INTO name=multipartIdentifier columns=identifierList? FROM (stageAndPattern | (LEFT_PAREN SELECT selectColumnClause FROM stageAndPattern whereClause? RIGHT_PAREN)) diff --git a/fe/fe-core/src/main/cup/sql_parser.cup b/fe/fe-core/src/main/cup/sql_parser.cup index 1f4083d6ea0edc..03f1f02f158f84 100644 --- a/fe/fe-core/src/main/cup/sql_parser.cup +++ b/fe/fe-core/src/main/cup/sql_parser.cup @@ -7976,9 +7976,9 @@ admin_stmt ::= ; truncate_stmt ::= - KW_TRUNCATE KW_TABLE base_table_ref:tblRef + KW_TRUNCATE KW_TABLE base_table_ref:tblRef opt_force:force {: - RESULT = new TruncateTableStmt(tblRef); + RESULT = new TruncateTableStmt(tblRef, force); :} ; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/TruncateTableStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/TruncateTableStmt.java index 9543ff1853b742..f751934316d722 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/TruncateTableStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/TruncateTableStmt.java @@ -30,15 +30,21 @@ public class TruncateTableStmt extends DdlStmt implements NotFallbackInParser { private TableRef tblRef; + private boolean forceDrop; - public TruncateTableStmt(TableRef tblRef) { + public TruncateTableStmt(TableRef tblRef, boolean forceDrop) { this.tblRef = tblRef; + this.forceDrop = forceDrop; } public TableRef getTblRef() { return tblRef; } + public boolean isForceDrop() { + return forceDrop; + } + @Override public void analyze(Analyzer analyzer) throws AnalysisException, UserException { super.analyze(analyzer); @@ -75,6 +81,9 @@ public String toSql() { if (tblRef.getPartitionNames() != null) { sb.append(tblRef.getPartitionNames().toSql()); } + if (isForceDrop()) { + sb.append(" FORCE"); + } return sb.toString(); } @@ -83,6 +92,9 @@ public String toSqlWithoutTable() { if (tblRef.getPartitionNames() != null) { sb.append(tblRef.getPartitionNames().toSql()); } + if (isForceDrop()) { + sb.append(" FORCE"); + } return sb.toString(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java index 5d57540017f067..8925e483c29df7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java @@ -1131,63 +1131,9 @@ private Partition dropPartition(long dbId, String partitionName, boolean isForce if (partition != null) { idToPartition.remove(partition.getId()); nameToPartition.remove(partitionName); - - if (!isForceDrop) { - // recycle partition - if (partitionInfo.getType() == PartitionType.RANGE) { - Env.getCurrentRecycleBin().recyclePartition(dbId, id, name, partition, - partitionInfo.getItem(partition.getId()).getItems(), - new ListPartitionItem(Lists.newArrayList(new PartitionKey())), - partitionInfo.getDataProperty(partition.getId()), - partitionInfo.getReplicaAllocation(partition.getId()), - partitionInfo.getIsInMemory(partition.getId()), - partitionInfo.getIsMutable(partition.getId())); - - } else if (partitionInfo.getType() == PartitionType.LIST) { - // construct a dummy range - List dummyColumns = new ArrayList<>(); - dummyColumns.add(new Column("dummy", PrimitiveType.INT)); - PartitionKey dummyKey = null; - try { - dummyKey = PartitionKey.createInfinityPartitionKey(dummyColumns, false); - } catch (AnalysisException e) { - LOG.warn("should not happen", e); - } - Range dummyRange = Range.open(new PartitionKey(), dummyKey); - - Env.getCurrentRecycleBin().recyclePartition(dbId, id, name, partition, - dummyRange, - partitionInfo.getItem(partition.getId()), - partitionInfo.getDataProperty(partition.getId()), - partitionInfo.getReplicaAllocation(partition.getId()), - partitionInfo.getIsInMemory(partition.getId()), - partitionInfo.getIsMutable(partition.getId())); - } else { - // unpartition - // construct a dummy range and dummy list. - List dummyColumns = new ArrayList<>(); - dummyColumns.add(new Column("dummy", PrimitiveType.INT)); - PartitionKey dummyKey = null; - try { - dummyKey = PartitionKey.createInfinityPartitionKey(dummyColumns, false); - } catch (AnalysisException e) { - LOG.warn("should not happen", e); - } - Range dummyRange = Range.open(new PartitionKey(), dummyKey); - Env.getCurrentRecycleBin().recyclePartition(dbId, id, name, partition, - dummyRange, - new ListPartitionItem(Lists.newArrayList(new PartitionKey())), - partitionInfo.getDataProperty(partition.getId()), - partitionInfo.getReplicaAllocation(partition.getId()), - partitionInfo.getIsInMemory(partition.getId()), - partitionInfo.getIsMutable(partition.getId())); - } - } else if (!reserveTablets) { - Env.getCurrentEnv().onErasePartition(partition); - } - - // drop partition info - partitionInfo.dropPartition(partition.getId()); + RecyclePartitionParam recyclePartitionParam = new RecyclePartitionParam(); + fillInfo(partition, recyclePartitionParam); + dropPartitionCommon(dbId, isForceDrop, recyclePartitionParam, partition, reserveTablets); } return partition; } @@ -1200,6 +1146,81 @@ public Partition dropPartition(long dbId, String partitionName, boolean isForceD return dropPartition(dbId, partitionName, isForceDrop, !isForceDrop); } + private void dropPartitionCommon(long dbId, boolean isForceDrop, + RecyclePartitionParam recyclePartitionParam, + Partition partition, + boolean reserveTablets) { + if (!isForceDrop) { + // recycle partition + if (partitionInfo.getType() == PartitionType.RANGE) { + Env.getCurrentRecycleBin().recyclePartition(dbId, id, name, partition, + recyclePartitionParam.partitionItem.getItems(), + new ListPartitionItem(Lists.newArrayList(new PartitionKey())), + recyclePartitionParam.dataProperty, + recyclePartitionParam.replicaAlloc, + recyclePartitionParam.isInMemory, + recyclePartitionParam.isMutable); + + } else if (partitionInfo.getType() == PartitionType.LIST) { + // construct a dummy range + List dummyColumns = new ArrayList<>(); + dummyColumns.add(new Column("dummy", PrimitiveType.INT)); + PartitionKey dummyKey = null; + try { + dummyKey = PartitionKey.createInfinityPartitionKey(dummyColumns, false); + } catch (AnalysisException e) { + LOG.warn("should not happen", e); + } + Range dummyRange = Range.open(new PartitionKey(), dummyKey); + + Env.getCurrentRecycleBin().recyclePartition(dbId, id, name, partition, + dummyRange, + recyclePartitionParam.partitionItem, + recyclePartitionParam.dataProperty, + recyclePartitionParam.replicaAlloc, + recyclePartitionParam.isInMemory, + recyclePartitionParam.isMutable); + } else { + // unpartition + // construct a dummy range and dummy list. + List dummyColumns = new ArrayList<>(); + dummyColumns.add(new Column("dummy", PrimitiveType.INT)); + PartitionKey dummyKey = null; + try { + dummyKey = PartitionKey.createInfinityPartitionKey(dummyColumns, false); + } catch (AnalysisException e) { + LOG.warn("should not happen", e); + } + Range dummyRange = Range.open(new PartitionKey(), dummyKey); + Env.getCurrentRecycleBin().recyclePartition(dbId, id, name, partition, + dummyRange, + new ListPartitionItem(Lists.newArrayList(new PartitionKey())), + recyclePartitionParam.dataProperty, + recyclePartitionParam.replicaAlloc, + recyclePartitionParam.isInMemory, + recyclePartitionParam.isMutable); + } + } else if (!reserveTablets) { + Env.getCurrentEnv().onErasePartition(partition); + } + + // drop partition info + partitionInfo.dropPartition(partition.getId()); + } + + public Partition dropPartitionForTruncate(long dbId, boolean isForceDrop, + RecyclePartitionParam recyclePartitionParam) { + // 1. If "isForceDrop" is false, the partition will be added to the Catalog Recyle bin, and all tablets of this + // partition will not be deleted. + // 2. If "ifForceDrop" is true, the partition will be dropped immediately + Partition partition = recyclePartitionParam.partition; + if (partition != null) { + idToPartition.remove(partition.getId()); + dropPartitionCommon(dbId, isForceDrop, recyclePartitionParam, partition, false); + } + return partition; + } + /* * A table may contain both formal and temporary partitions. * There are several methods to get the partition of a table. @@ -2031,13 +2052,24 @@ public static OlapTable read(DataInput in) throws IOException { return GsonUtils.GSON.fromJson(Text.readString(in), OlapTable.class); } + + public void fillInfo(Partition partition, RecyclePartitionParam recyclePartitionParam) { + recyclePartitionParam.dataProperty = partitionInfo.getDataProperty(partition.getId()); + recyclePartitionParam.replicaAlloc = partitionInfo.getReplicaAllocation(partition.getId()); + recyclePartitionParam.isInMemory = partitionInfo.getIsInMemory(partition.getId()); + recyclePartitionParam.isMutable = partitionInfo.getIsMutable(partition.getId()); + recyclePartitionParam.partitionItem = partitionInfo.getItem(partition.getId()); + recyclePartitionParam.partition = partition; + } + /* * this method is currently used for truncating table(partitions). * the new partition has new id, so we need to change all 'id-related' members * * return the old partition. */ - public Partition replacePartition(Partition newPartition) { + public Partition replacePartition(Partition newPartition, + RecyclePartitionParam recyclePartitionParam) { Partition oldPartition = nameToPartition.remove(newPartition.getName()); idToPartition.remove(oldPartition.getId()); @@ -2048,6 +2080,12 @@ public Partition replacePartition(Partition newPartition) { ReplicaAllocation replicaAlloc = partitionInfo.getReplicaAllocation(oldPartition.getId()); boolean isInMemory = partitionInfo.getIsInMemory(oldPartition.getId()); boolean isMutable = partitionInfo.getIsMutable(oldPartition.getId()); + recyclePartitionParam.dataProperty = dataProperty; + recyclePartitionParam.replicaAlloc = replicaAlloc; + recyclePartitionParam.isInMemory = isInMemory; + recyclePartitionParam.isMutable = isMutable; + recyclePartitionParam.partitionItem = partitionInfo.getItem(oldPartition.getId()); + recyclePartitionParam.partition = oldPartition; if (partitionInfo.getType() == PartitionType.RANGE || partitionInfo.getType() == PartitionType.LIST) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/RecyclePartitionParam.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/RecyclePartitionParam.java new file mode 100644 index 00000000000000..51f38638938db5 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/RecyclePartitionParam.java @@ -0,0 +1,31 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.catalog; + +public class RecyclePartitionParam { + public Partition partition; + public PartitionItem partitionItem; + public DataProperty dataProperty; + public ReplicaAllocation replicaAlloc; + public boolean isInMemory; + public boolean isMutable = true; + + public RecyclePartitionParam() { + // do nothing. + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java index bdc6626f63df81..1430d4a32b1789 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java @@ -103,6 +103,7 @@ import org.apache.doris.catalog.PrimitiveType; import org.apache.doris.catalog.RandomDistributionInfo; import org.apache.doris.catalog.RangePartitionItem; +import org.apache.doris.catalog.RecyclePartitionParam; import org.apache.doris.catalog.Replica; import org.apache.doris.catalog.Replica.ReplicaState; import org.apache.doris.catalog.ReplicaAllocation; @@ -3545,7 +3546,7 @@ private void checkAutoIncColumns(List columns, KeysType type) throws Ddl public void truncateTable(TruncateTableStmt truncateTableStmt) throws DdlException { TableRef tblRef = truncateTableStmt.getTblRef(); TableName dbTbl = tblRef.getName(); - + boolean isForceDrop = truncateTableStmt.isForceDrop(); // check, and save some info which need to be checked again later Map origPartitions = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER); Map partitionsDistributionInfo = Maps.newHashMap(); @@ -3574,7 +3575,9 @@ public void truncateTable(TruncateTableStmt truncateTableStmt) throws DdlExcepti } // If need absolutely correct, should check running txn here. // But if the txn is in prepare state, cann't known which partitions had load data. - if (!partition.hasData()) { + if ((isForceDrop) && (!partition.hasData())) { + // if not force drop, then need to add partition to + // recycle bin, so behavior for recover would be clear continue; } origPartitions.put(partName, partition.getId()); @@ -3585,7 +3588,9 @@ public void truncateTable(TruncateTableStmt truncateTableStmt) throws DdlExcepti for (Partition partition : olapTable.getPartitions()) { // If need absolutely correct, should check running txn here. // But if the txn is in prepare state, cann't known which partitions had load data. - if (!partition.hasData()) { + if ((isForceDrop) && (!partition.hasData())) { + // if not force drop, then need to add partition to + // recycle bin, so behavior for recover would be clear continue; } origPartitions.put(partition.getName(), partition.getId()); @@ -3732,13 +3737,16 @@ public void truncateTable(TruncateTableStmt truncateTableStmt) throws DdlExcepti throw new DdlException("Table[" + copiedTbl.getName() + "]'s meta has been changed. try again."); } - // replace - oldPartitions = truncateTableInternal(olapTable, newPartitions, truncateEntireTable); + //replace + Map recyclePartitionParamMap = new HashMap<>(); + oldPartitions = truncateTableInternal(olapTable, newPartitions, + truncateEntireTable, recyclePartitionParamMap, isForceDrop); // write edit log TruncateTableInfo info = new TruncateTableInfo(db.getId(), db.getFullName(), olapTable.getId(), olapTable.getName(), - newPartitions, truncateEntireTable, truncateTableStmt.toSqlWithoutTable(), oldPartitions); + newPartitions, truncateEntireTable, + truncateTableStmt.toSqlWithoutTable(), oldPartitions, isForceDrop); Env.getCurrentEnv().getEditLog().logTruncateTable(info); } catch (DdlException e) { failedCleanCallback.run(); @@ -3749,8 +3757,6 @@ public void truncateTable(TruncateTableStmt truncateTableStmt) throws DdlExcepti } } - erasePartitionDropBackendReplicas(oldPartitions); - PartitionNames partitionNames = truncateEntireTable ? null : new PartitionNames(false, tblRef.getPartitionNames().getPartitionNames()); Env.getCurrentEnv().getAnalysisManager().dropStats(olapTable, partitionNames); @@ -3759,47 +3765,49 @@ public void truncateTable(TruncateTableStmt truncateTableStmt) throws DdlExcepti } private List truncateTableInternal(OlapTable olapTable, List newPartitions, - boolean isEntireTable) { + boolean isEntireTable, Map recyclePartitionParamMap, boolean isforceDrop) { // use new partitions to replace the old ones. List oldPartitions = Lists.newArrayList(); - Set oldTabletIds = Sets.newHashSet(); for (Partition newPartition : newPartitions) { - Partition oldPartition = olapTable.replacePartition(newPartition); + RecyclePartitionParam recyclePartitionParam = new RecyclePartitionParam(); + Partition oldPartition = olapTable.replacePartition(newPartition, recyclePartitionParam); oldPartitions.add(oldPartition); - // save old tablets to be removed - for (MaterializedIndex index : oldPartition.getMaterializedIndices(IndexExtState.ALL)) { - index.getTablets().forEach(t -> { - oldTabletIds.add(t.getId()); - }); - } + recyclePartitionParamMap.put(oldPartition.getId(), recyclePartitionParam); } if (isEntireTable) { Set oldPartitionsIds = oldPartitions.stream().map(Partition::getId).collect(Collectors.toSet()); for (Partition partition : olapTable.getAllTempPartitions()) { if (!oldPartitionsIds.contains(partition.getId())) { + RecyclePartitionParam recyclePartitionParam = new RecyclePartitionParam(); + olapTable.fillInfo(partition, recyclePartitionParam); oldPartitions.add(partition); + recyclePartitionParamMap.put(partition.getId(), recyclePartitionParam); + // clear temp partition from memory. + // tablet may be moved to recycle bin or deleted inside + // dropPartitionForTruncate function. + olapTable.dropTempPartition(partition.getName(), false); } } - // drop all temp partitions - olapTable.dropAllTempPartitions(); } - // remove the tablets in old partitions - for (Long tabletId : oldTabletIds) { - Env.getCurrentInvertedIndex().deleteTablet(tabletId); + for (Map.Entry pair : recyclePartitionParamMap.entrySet()) { + olapTable.dropPartitionForTruncate(olapTable.getDatabase().getId(), isforceDrop, pair.getValue()); } return oldPartitions; } public void replayTruncateTable(TruncateTableInfo info) throws MetaNotFoundException { + boolean isForceDrop = info.getForce(); List oldPartitions = Lists.newArrayList(); Database db = (Database) getDbOrMetaException(info.getDbId()); OlapTable olapTable = (OlapTable) db.getTableOrMetaException(info.getTblId(), TableType.OLAP); olapTable.writeLock(); try { - truncateTableInternal(olapTable, info.getPartitions(), info.isEntireTable()); + Map recyclePartitionParamMap = new HashMap<>(); + truncateTableInternal(olapTable, info.getPartitions(), info.isEntireTable(), + recyclePartitionParamMap, isForceDrop); // add tablet to inverted index TabletInvertedIndex invertedIndex = Env.getCurrentInvertedIndex(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/TruncateTableInfo.java b/fe/fe-core/src/main/java/org/apache/doris/persist/TruncateTableInfo.java index b252b2a38233f7..ffb59d2afdffd9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/TruncateTableInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/TruncateTableInfo.java @@ -49,13 +49,15 @@ public class TruncateTableInfo implements Writable { private String rawSql = ""; @SerializedName(value = "op") private Map oldPartitions = new HashMap<>(); + @SerializedName(value = "force") + private boolean force = true; // older version it was forced always. public TruncateTableInfo() { } public TruncateTableInfo(long dbId, String db, long tblId, String table, List partitions, - boolean isEntireTable, String rawSql, List oldPartitions) { + boolean isEntireTable, String rawSql, List oldPartitions, boolean force) { this.dbId = dbId; this.db = db; this.tblId = tblId; @@ -66,6 +68,7 @@ public TruncateTableInfo(long dbId, String db, long tblId, String table, List + def result = sql_return_maparray("show partitions from ${testTable}") + return result.collectEntries { [it.PartitionName, it.PartitionId as long] } + } + + def partitionIds1 = getPartitionIds() + assertEquals(["p1", "p2", "p3"].toSet(), partitionIds1.keySet()) + + sql "insert into ${testTable} values ('2020-01-01', 1.0, 'a', 1)" + sql "insert into ${testTable} values ('2020-03-10', 1.0, 'a', 1)" + order_qt_select_1 "SELECT * FROM ${testTable}" + + sql """truncate table ${testTable} force;""" + // if we use force, the empty partitions skiped, + // so that partition ID doesnt change. + def partitionIds2 = getPartitionIds() + assertEquals(["p1", "p2", "p3"].toSet(), partitionIds2.keySet()) + assertNotEquals(partitionIds1.get("p1"), partitionIds2.get("p1")) + assertEquals(partitionIds1.get("p2"), partitionIds2.get("p2")) + assertNotEquals(partitionIds1.get("p3"), partitionIds2.get("p3")) + order_qt_select_2 "SELECT * FROM ${testTable}" + + sql "insert into ${testTable} values ('2020-02-10', 1.0, 'a', 1)" + order_qt_select_3 "SELECT * FROM ${testTable}" + sql """truncate table ${testTable} partitions (p1, p2) force;""" + order_qt_select_4 "SELECT * FROM ${testTable}" + + def partitionIds3 = getPartitionIds() + assertEquals(["p1", "p2", "p3"].toSet(), partitionIds3.keySet()) + assertEquals(partitionIds2.get("p1"), partitionIds3.get("p1")) + assertNotEquals(partitionIds2.get("p2"), partitionIds3.get("p2")) + assertEquals(partitionIds2.get("p3"), partitionIds3.get("p3")) + + sql "DROP TABLE IF EXISTS ${testTable}" +} + From 325a67c736ee202ac03d231899135819df2d7743 Mon Sep 17 00:00:00 2001 From: Vallish Pai Date: Thu, 28 Nov 2024 11:36:21 +0530 Subject: [PATCH 055/399] [chore](test) Add info about buildKit required with docker-compose (#44095) got error when trying to create docker cluster using private image. adding more info so that can help others. --- docker/runtime/doris-compose/Readme.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docker/runtime/doris-compose/Readme.md b/docker/runtime/doris-compose/Readme.md index c4c4dc0990f0fc..fea13ab1821882 100644 --- a/docker/runtime/doris-compose/Readme.md +++ b/docker/runtime/doris-compose/Readme.md @@ -32,6 +32,8 @@ docker run hello-world if have problem with permission denied, then [add-docker-permission](https://docs.docker.com/engine/install/linux-postinstall/). +Make sure BuildKit configured in the machine. if not follow [docker-with-BuildKit](https://docs.docker.com/build/buildkit/). + ##### 2. The doris image should contains ``` From 96079bcc127f83af91b53ab0b4f8f70306d0a5a2 Mon Sep 17 00:00:00 2001 From: Vallish Pai Date: Thu, 28 Nov 2024 11:37:08 +0530 Subject: [PATCH 056/399] [Enhancement](recyclebin) drop table keep temp partitions in recycle-bin (#44097) --- .../doris/datasource/InternalCatalog.java | 4 -- .../test_temp_partitions_table.out | 28 +++++++++ .../test_temp_partitions_table.groovy | 63 +++++++++++++++++++ 3 files changed, 91 insertions(+), 4 deletions(-) create mode 100644 regression-test/data/catalog_recycle_bin_p0/test_temp_partitions_table.out create mode 100644 regression-test/suites/catalog_recycle_bin_p0/test_temp_partitions_table.groovy diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java index 1430d4a32b1789..64a759b9747044 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java @@ -1037,10 +1037,6 @@ public boolean unprotectDropTable(Database db, Table table, boolean isForceDrop, long recycleTime) { if (table.getType() == TableType.ELASTICSEARCH) { esRepository.deRegisterTable(table.getId()); - } else if (table.isManagedTable()) { - // drop all temp partitions of this table, so that there is no temp partitions in recycle bin, - // which make things easier. - ((OlapTable) table).dropAllTempPartitions(); } if (table.getType() == TableType.MATERIALIZED_VIEW) { Env.getCurrentEnv().getMtmvService().deregisterMTMV((MTMV) table); diff --git a/regression-test/data/catalog_recycle_bin_p0/test_temp_partitions_table.out b/regression-test/data/catalog_recycle_bin_p0/test_temp_partitions_table.out new file mode 100644 index 00000000000000..c1715c55c89247 --- /dev/null +++ b/regression-test/data/catalog_recycle_bin_p0/test_temp_partitions_table.out @@ -0,0 +1,28 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !sql_1 -- +1 1 1 + +-- !sql_2 -- +7 1 3 + +-- !sql_3 -- +16 1234 t + +-- !sql_recover1 -- +1 1 1 + +-- !sql_recover2 -- +7 1 3 + +-- !sql_recover3 -- +16 1234 t + +-- !sql_newrecover1 -- +1 1 1 + +-- !sql_newrecover2 -- +7 1 3 + +-- !sql_newrecover3 -- +16 1234 t + diff --git a/regression-test/suites/catalog_recycle_bin_p0/test_temp_partitions_table.groovy b/regression-test/suites/catalog_recycle_bin_p0/test_temp_partitions_table.groovy new file mode 100644 index 00000000000000..e1e68846c52bf6 --- /dev/null +++ b/regression-test/suites/catalog_recycle_bin_p0/test_temp_partitions_table.groovy @@ -0,0 +1,63 @@ + + // Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("test_temp_partitions_table") { + def table = "test_temp_partitions_table" + def table2 = "test_temp_partitions_table_2" + + sql """ + CREATE TABLE ${table} ( + id BIGINT, + val BIGINT, + str VARCHAR(114) + ) DUPLICATE KEY(`id`) + PARTITION BY RANGE(`id`) + ( + PARTITION `p1` VALUES LESS THAN ('5'), + PARTITION `p2` VALUES LESS THAN ('10') + ) + DISTRIBUTED BY HASH(`id`) BUCKETS 3 + PROPERTIES ( + "replication_num"="1" + ); + """ + + sql """ALTER TABLE ${table} ADD TEMPORARY PARTITION tp1 VALUES [("15"), ("20"));""" + + sql "INSERT INTO ${table} VALUES(1, 1,'1')" + sql "INSERT INTO ${table} VALUES(7, 1,'3')" + sql "INSERT INTO ${table} TEMPORARY PARTITION(tp1) values(16,1234, 't');" + sql "SYNC" + qt_sql_1 "SELECT * FROM ${table} PARTITION p1" + qt_sql_2 "SELECT * FROM ${table} PARTITION p2" + qt_sql_3 """select * from ${table} temporary partition(tp1);""" + sql "DROP table ${table} " + sql "recover table ${table} " + + qt_sql_recover1 "SELECT * FROM ${table} PARTITION p1" + qt_sql_recover2 "SELECT * FROM ${table} PARTITION p2" + qt_sql_recover3 """select * from ${table} temporary partition(tp1);""" + + sql "DROP table ${table} " + sql "recover table ${table} as ${table2}" + + qt_sql_newrecover1 "SELECT * FROM ${table2} PARTITION p1" + qt_sql_newrecover2 "SELECT * FROM ${table2} PARTITION p2" + qt_sql_newrecover3 """select * from ${table2} temporary partition(tp1);""" + +} \ No newline at end of file From 5c36328b1008dd306e6caa80a67e7071ab8427c4 Mon Sep 17 00:00:00 2001 From: meiyi Date: Thu, 28 Nov 2024 14:15:39 +0800 Subject: [PATCH 057/399] [fix](case) fix some cluster key case (#44674) --- .../org/apache/doris/regression/Config.groovy | 29 ++++++- .../doris/regression/suite/Suite.groovy | 4 + .../insert_p0/transaction/txn_insert.groovy | 2 +- .../suites/nereids_function_p0/load.groovy | 5 +- .../test_col_data_type_boundary.groovy | 79 +++++++++++++++---- .../point_query_p0/test_point_query_ck.groovy | 5 +- ...l_update_insert_light_schema_change.groovy | 3 + ...partial_update_insert_schema_change.groovy | 3 + .../test_partial_update_schema_change.groovy | 3 + ...tial_update_schema_change_row_store.groovy | 3 + .../test_partial_update_seq_col.groovy | 3 + .../test_partial_update_seq_col_delete.groovy | 3 + .../test_partial_update_seq_type.groovy | 3 + ...test_partial_update_seq_type_delete.groovy | 3 + .../test_partial_update_strict_mode.groovy | 3 + .../test_partial_update_upsert.groovy | 3 + ..._partial_update_with_inverted_index.groovy | 3 + ...test_partial_update_with_row_column.groovy | 4 + .../test_update_on_current_timestamp.groovy | 4 + .../test_create_table.groovy | 4 + .../test_schema_change_ck.groovy | 6 +- 21 files changed, 152 insertions(+), 23 deletions(-) diff --git a/regression-test/framework/src/main/groovy/org/apache/doris/regression/Config.groovy b/regression-test/framework/src/main/groovy/org/apache/doris/regression/Config.groovy index 7b0d24482a29a1..5aa6d7ce7e7218 100644 --- a/regression-test/framework/src/main/groovy/org/apache/doris/regression/Config.groovy +++ b/regression-test/framework/src/main/groovy/org/apache/doris/regression/Config.groovy @@ -524,7 +524,7 @@ class Config { // docker suite no need external cluster. // so can ignore error here. } - + config.excludeUnsupportedCase() return config } @@ -1008,6 +1008,33 @@ class Config { } } + boolean isClusterKeyEnabled() { + try { + def result = JdbcUtils.executeToMapArray(getRootConnection(), "SHOW FRONTEND CONFIG LIKE 'random_add_cluster_keys_for_mow'") + log.info("show random_add_cluster_keys_for_mow config: ${result}".toString()) + return result[0].Value.toString().equalsIgnoreCase("true") + } catch (Throwable t) { + log.warn("Fetch server config 'random_add_cluster_keys_for_mow' failed, jdbcUrl: ${jdbcUrl}".toString(), t) + return false + } + } + + void excludeUnsupportedCase() { + boolean isCKEnabled = isClusterKeyEnabled() + log.info("random_add_cluster_keys_for_mow in fe.conf: ${isCKEnabled}".toString()) + if (isCKEnabled) { + excludeDirectorySet.add("unique_with_mow_p0/partial_update") + excludeDirectorySet.add("unique_with_mow_p0/flexible") + excludeDirectorySet.add("doc") + List excludeCases = ["test_table_properties", "test_default_hll", "test_default_pi", "test_full_compaction", "test_full_compaction_by_table_id", "test_create_table", "txn_insert", "test_update_mow", "test_new_update", "test_update_unique", "test_partial_update_generated_column", "nereids_partial_update_native_insert_stmt", "partial_update", "nereids_update_on_current_timestamp", "update_on_current_timestamp", "test_default_bitmap_empty", "nereids_delete_mow_partial_update", "delete_mow_partial_update", "partial_update_seq_col", "nereids_partial_update_native_insert_stmt_complex", "regression_test_variant_delete_and_update", "test_unique_table_auto_inc_partial_update_correct_stream_load", "test_unique_table_auto_inc", "test_unique_table_auto_inc_partial_update_correct_insert", "test_update_schema_change"] + for (def excludeCase in excludeCases) { + excludeSuiteWildcard.add(excludeCase) + } + log.info("excludeDirectorySet: ${excludeDirectorySet}".toString()) + log.info("excludeSuiteWildcard: ${excludeSuiteWildcard}".toString()) + } + } + Connection getConnection() { return DriverManager.getConnection(jdbcUrl, jdbcUser, jdbcPassword) } diff --git a/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy b/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy index bd16342268f1e3..2a5ba5acc414b7 100644 --- a/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy +++ b/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy @@ -1641,6 +1641,10 @@ class Suite implements GroovyInterceptable { } } + boolean isClusterKeyEnabled() { + return getFeConfig("random_add_cluster_keys_for_mow").equals("true") + } + boolean enableStoragevault() { if (Strings.isNullOrEmpty(context.config.metaServiceHttpAddress) || Strings.isNullOrEmpty(context.config.instanceId) diff --git a/regression-test/suites/insert_p0/transaction/txn_insert.groovy b/regression-test/suites/insert_p0/transaction/txn_insert.groovy index a4868ca6b11ffe..b54fc66e714fbd 100644 --- a/regression-test/suites/insert_p0/transaction/txn_insert.groovy +++ b/regression-test/suites/insert_p0/transaction/txn_insert.groovy @@ -717,7 +717,7 @@ suite("txn_insert") { } // 18. column update(mow table) - if (use_nereids_planner) { + if (use_nereids_planner && !isClusterKeyEnabled()) { def unique_table = "txn_insert_cu" for (def i in 0..3) { sql """ drop table if exists ${unique_table}_${i} """ diff --git a/regression-test/suites/nereids_function_p0/load.groovy b/regression-test/suites/nereids_function_p0/load.groovy index d4aeea530412cb..2777774449e3e4 100644 --- a/regression-test/suites/nereids_function_p0/load.groovy +++ b/regression-test/suites/nereids_function_p0/load.groovy @@ -155,6 +155,7 @@ suite("load") { } } + if (!isClusterKeyEnabled()) { // test fn_test_ip_nullable_rowstore table with update action sql "update fn_test_ip_nullable_rowstore set ip4 = '' where id = 1;" sql_res = sql "select * from fn_test_ip_nullable_rowstore where id = 1;" @@ -169,6 +170,7 @@ suite("load") { sql "update fn_test_ip_nullable_rowstore set ip6 = '::1' where id = 1;" sql_res = sql "select * from fn_test_ip_nullable_rowstore where id = 1;" assertEquals(sql_res[0].toString(), '[1, 127.0.0.1, ::1, "127.0.0.1", "::1"]') + } streamLoad { table "fn_test_ip_not_nullable" @@ -218,6 +220,7 @@ suite("load") { } } + if (!isClusterKeyEnabled()) { // test fn_test_ip_not_nullable_rowstore table with update action // not null will throw exception if we has data in table test { @@ -237,7 +240,7 @@ suite("load") { sql "update fn_test_ip_not_nullable_rowstore set ip6 = '::2' where id = 1;" sql_res1 = sql "select * from fn_test_ip_not_nullable_rowstore where id = 1;" assertEquals(sql_res1[0].toString(), '[1, 192.10.10.1, ::2, "127.0.0.1", "::1"]') - + } // make some normal ipv4/ipv6 data for sql function , which is increased one by one // 29-50 A 类地址 ; 51-68 B 类地址 ; 69-87 C 类地址 ; 88-100 D 类地址 diff --git a/regression-test/suites/partition_p0/auto_partition/test_col_data_type_boundary.groovy b/regression-test/suites/partition_p0/auto_partition/test_col_data_type_boundary.groovy index 47fe118860e71a..65098e0fd23f20 100644 --- a/regression-test/suites/partition_p0/auto_partition/test_col_data_type_boundary.groovy +++ b/regression-test/suites/partition_p0/auto_partition/test_col_data_type_boundary.groovy @@ -68,7 +68,11 @@ suite("test_col_data_type_boundary") { sql """alter table table_tinyint modify column c_tinyint largeint key;""" } catch (Exception e) { log.info(e.getMessage()) - assertTrue(e.getMessage().contains("Can not modify partition column")) + if (isClusterKeyEnabled()) { + assertTrue(e.getMessage().contains("Can not modify ")) + } else { + assertTrue(e.getMessage().contains("Can not modify partition column")) + } } sql """drop table if exists table_smallint""" @@ -97,7 +101,11 @@ suite("test_col_data_type_boundary") { sql """alter table table_smallint modify column c_smallint largeint key;""" } catch (Exception e) { log.info(e.getMessage()) - assertTrue(e.getMessage().contains("Can not modify partition column")) + if (isClusterKeyEnabled()) { + assertTrue(e.getMessage().contains("Can not modify ")) + } else { + assertTrue(e.getMessage().contains("Can not modify partition column")) + } } sql """drop table if exists table_int""" @@ -126,7 +134,11 @@ suite("test_col_data_type_boundary") { sql """alter table table_int modify column c_int largeint key;""" } catch (Exception e) { log.info(e.getMessage()) - assertTrue(e.getMessage().contains("Can not modify partition column")) + if (isClusterKeyEnabled()) { + assertTrue(e.getMessage().contains("Can not modify ")) + } else { + assertTrue(e.getMessage().contains("Can not modify partition column")) + } } sql """drop table if exists table_bigint""" @@ -155,7 +167,11 @@ suite("test_col_data_type_boundary") { sql """alter table table_bigint modify column c_bigint largeint key;""" } catch (Exception e) { log.info(e.getMessage()) - assertTrue(e.getMessage().contains("Can not modify partition column")) + if (isClusterKeyEnabled()) { + assertTrue(e.getMessage().contains("Can not modify ")) + } else { + assertTrue(e.getMessage().contains("Can not modify partition column")) + } } sql """drop table if exists table_largeint""" @@ -225,7 +241,11 @@ suite("test_col_data_type_boundary") { sql """alter table table_date_range modify column c_date datetime key;""" } catch (Exception e) { log.info(e.getMessage()) - assertTrue(e.getMessage().contains("Can not modify partition column")) + if (isClusterKeyEnabled()) { + assertTrue(e.getMessage().contains("Can not modify ")) + } else { + assertTrue(e.getMessage().contains("Can not modify partition column")) + } } @@ -249,7 +269,11 @@ suite("test_col_data_type_boundary") { sql """alter table table_date_list modify column c_date datetime key;""" } catch (Exception e) { log.info(e.getMessage()) - assertTrue(e.getMessage().contains("Can not modify partition column")) + if (isClusterKeyEnabled()) { + assertTrue(e.getMessage().contains("Can not modify ")) + } else { + assertTrue(e.getMessage().contains("Can not modify partition column")) + } } sql """drop table if exists table_datetime_range""" @@ -274,7 +298,11 @@ suite("test_col_data_type_boundary") { sql """alter table table_datetime_range modify column c_datetime date key;""" } catch (Exception e) { log.info(e.getMessage()) - assertTrue(e.getMessage().contains("Can not modify partition column")) + if (isClusterKeyEnabled()) { + assertTrue(e.getMessage().contains("Can not modify ")) + } else { + assertTrue(e.getMessage().contains("Can not modify partition column")) + } } sql """drop table if exists table_datetime_list""" @@ -299,7 +327,11 @@ suite("test_col_data_type_boundary") { sql """alter table table_datetime_list modify column c_datetime date key;""" } catch (Exception e) { log.info(e.getMessage()) - assertTrue(e.getMessage().contains("Can not modify partition column")) + if (isClusterKeyEnabled()) { + assertTrue(e.getMessage().contains("Can not modify ")) + } else { + assertTrue(e.getMessage().contains("Can not modify partition column")) + } } @@ -323,7 +355,11 @@ suite("test_col_data_type_boundary") { sql """alter table table_char modify column c_char varchar key;""" } catch (Exception e) { log.info(e.getMessage()) - assertTrue(e.getMessage().contains("Can not modify partition column")) + if (isClusterKeyEnabled()) { + assertTrue(e.getMessage().contains("Can not modify ")) + } else { + assertTrue(e.getMessage().contains("Can not modify partition column")) + } } @@ -347,7 +383,11 @@ suite("test_col_data_type_boundary") { sql """alter table table_varchar modify column c_varchar date key;""" } catch (Exception e) { log.info(e.getMessage()) - assertTrue(e.getMessage().contains("Can not modify partition column")) + if (isClusterKeyEnabled()) { + assertTrue(e.getMessage().contains("Can not modify ")) + } else { + assertTrue(e.getMessage().contains("Can not modify partition column")) + } } @@ -371,10 +411,19 @@ suite("test_col_data_type_boundary") { assertEquals(select_rows[0][0], 3) assertEquals(partitions_res1.size(), 2) - sql """alter table table_datetime_range modify column c_int largeint;""" - partitions_res1 = sql """show partitions from table_datetime_range order by PartitionId;""" - select_rows = sql """select count() from table_datetime_range;""" - assertEquals(select_rows[0][0], 3) - assertEquals(partitions_res1.size(), 2) + try { + sql """alter table table_datetime_range modify column c_int largeint;""" + partitions_res1 = sql """show partitions from table_datetime_range order by PartitionId;""" + select_rows = sql """select count() from table_datetime_range;""" + assertEquals(select_rows[0][0], 3) + assertEquals(partitions_res1.size(), 2) + } catch (Exception e) { + log.info(e.getMessage()) + if (isClusterKeyEnabled()) { + assertTrue(e.getMessage().contains("Can not modify ")) + } else { + throw e + } + } } diff --git a/regression-test/suites/point_query_p0/test_point_query_ck.groovy b/regression-test/suites/point_query_p0/test_point_query_ck.groovy index 09cf644cb9a48a..33c31dc0970929 100644 --- a/regression-test/suites/point_query_p0/test_point_query_ck.groovy +++ b/regression-test/suites/point_query_p0/test_point_query_ck.groovy @@ -202,7 +202,7 @@ suite("test_point_query_ck") { qe_point_select stmt qe_point_select stmt // invalidate cache - sql "sync" + // sql "sync" nprep_sql """ INSERT INTO ${tableName} VALUES(1235, 120939.11130, "a ddd", "xxxxxx", "2030-01-02", "2020-01-01 12:36:38", 22.822, "7022-01-01 11:30:38", 0, 1929111.1111,[119291.19291], ["111", "222", "333"], 2) """ qe_point_select stmt qe_point_select stmt @@ -218,9 +218,10 @@ suite("test_point_query_ck") { qe_point_select stmt qe_point_select stmt - sql """ + nprep_sql """ ALTER table ${tableName} ADD COLUMN new_column1 INT default "0"; """ + sql "select 1" qe_point_select stmt } // disable useServerPrepStmts diff --git a/regression-test/suites/unique_with_mow_c_p0/partial_update/test_partial_update_insert_light_schema_change.groovy b/regression-test/suites/unique_with_mow_c_p0/partial_update/test_partial_update_insert_light_schema_change.groovy index 33f71ab307661b..cc6faa03dc0d79 100644 --- a/regression-test/suites/unique_with_mow_c_p0/partial_update/test_partial_update_insert_light_schema_change.groovy +++ b/regression-test/suites/unique_with_mow_c_p0/partial_update/test_partial_update_insert_light_schema_change.groovy @@ -19,6 +19,9 @@ import java.util.concurrent.TimeUnit import org.awaitility.Awaitility suite("test_partial_update_insert_light_schema_change", "p0") { + if (isClusterKeyEnabled()) { + return + } String db = context.config.getDbNameByFile(context.file) sql "select 1;" // to create database diff --git a/regression-test/suites/unique_with_mow_c_p0/partial_update/test_partial_update_insert_schema_change.groovy b/regression-test/suites/unique_with_mow_c_p0/partial_update/test_partial_update_insert_schema_change.groovy index 07740b9a09de20..62140ac58bf1cf 100644 --- a/regression-test/suites/unique_with_mow_c_p0/partial_update/test_partial_update_insert_schema_change.groovy +++ b/regression-test/suites/unique_with_mow_c_p0/partial_update/test_partial_update_insert_schema_change.groovy @@ -19,6 +19,9 @@ import java.util.concurrent.TimeUnit import org.awaitility.Awaitility suite("test_partial_update_insert_schema_change", "p0") { + if (isClusterKeyEnabled()) { + return + } // ===== light schema change ===== // test add value column diff --git a/regression-test/suites/unique_with_mow_c_p0/partial_update/test_partial_update_schema_change.groovy b/regression-test/suites/unique_with_mow_c_p0/partial_update/test_partial_update_schema_change.groovy index c40e98f6badcbc..d2e27699524635 100644 --- a/regression-test/suites/unique_with_mow_c_p0/partial_update/test_partial_update_schema_change.groovy +++ b/regression-test/suites/unique_with_mow_c_p0/partial_update/test_partial_update_schema_change.groovy @@ -19,6 +19,9 @@ import java.util.concurrent.TimeUnit import org.awaitility.Awaitility suite("test_partial_update_schema_change", "p0") { + if (isClusterKeyEnabled()) { + return + } /* ============================================== light schema change cases: ============================================== */ diff --git a/regression-test/suites/unique_with_mow_c_p0/partial_update/test_partial_update_schema_change_row_store.groovy b/regression-test/suites/unique_with_mow_c_p0/partial_update/test_partial_update_schema_change_row_store.groovy index 72698530f0a7b8..cffb682488a2d3 100644 --- a/regression-test/suites/unique_with_mow_c_p0/partial_update/test_partial_update_schema_change_row_store.groovy +++ b/regression-test/suites/unique_with_mow_c_p0/partial_update/test_partial_update_schema_change_row_store.groovy @@ -19,6 +19,9 @@ import java.util.concurrent.TimeUnit import org.awaitility.Awaitility suite("test_partial_update_row_store_schema_change", "p0") { + if (isClusterKeyEnabled()) { + return + } /* ============================================== light schema change cases: ============================================== */ diff --git a/regression-test/suites/unique_with_mow_c_p0/partial_update/test_partial_update_seq_col.groovy b/regression-test/suites/unique_with_mow_c_p0/partial_update/test_partial_update_seq_col.groovy index 111037ae18f6b8..884882b7c93934 100644 --- a/regression-test/suites/unique_with_mow_c_p0/partial_update/test_partial_update_seq_col.groovy +++ b/regression-test/suites/unique_with_mow_c_p0/partial_update/test_partial_update_seq_col.groovy @@ -17,6 +17,9 @@ // under the License. suite("test_primary_key_partial_update_seq_col", "p0") { + if (isClusterKeyEnabled()) { + return + } String db = context.config.getDbNameByFile(context.file) sql "select 1;" // to create database diff --git a/regression-test/suites/unique_with_mow_c_p0/partial_update/test_partial_update_seq_col_delete.groovy b/regression-test/suites/unique_with_mow_c_p0/partial_update/test_partial_update_seq_col_delete.groovy index dca646948b4951..b37cdc68525d77 100644 --- a/regression-test/suites/unique_with_mow_c_p0/partial_update/test_partial_update_seq_col_delete.groovy +++ b/regression-test/suites/unique_with_mow_c_p0/partial_update/test_partial_update_seq_col_delete.groovy @@ -17,6 +17,9 @@ // under the License. suite("test_primary_key_partial_update_seq_col_delete", "p0") { + if (isClusterKeyEnabled()) { + return + } String db = context.config.getDbNameByFile(context.file) sql "select 1;" // to create database diff --git a/regression-test/suites/unique_with_mow_c_p0/partial_update/test_partial_update_seq_type.groovy b/regression-test/suites/unique_with_mow_c_p0/partial_update/test_partial_update_seq_type.groovy index d7d55725df6a62..3075faa022240f 100644 --- a/regression-test/suites/unique_with_mow_c_p0/partial_update/test_partial_update_seq_type.groovy +++ b/regression-test/suites/unique_with_mow_c_p0/partial_update/test_partial_update_seq_type.groovy @@ -17,6 +17,9 @@ // under the License. suite("test_primary_key_partial_update_seq_type", "p0") { + if (isClusterKeyEnabled()) { + return + } String db = context.config.getDbNameByFile(context.file) sql "select 1;" // to create database diff --git a/regression-test/suites/unique_with_mow_c_p0/partial_update/test_partial_update_seq_type_delete.groovy b/regression-test/suites/unique_with_mow_c_p0/partial_update/test_partial_update_seq_type_delete.groovy index 6ad60e0cd75f31..142feebef559c1 100644 --- a/regression-test/suites/unique_with_mow_c_p0/partial_update/test_partial_update_seq_type_delete.groovy +++ b/regression-test/suites/unique_with_mow_c_p0/partial_update/test_partial_update_seq_type_delete.groovy @@ -17,6 +17,9 @@ // under the License. suite("test_primary_key_partial_update_seq_type_delete", "p0") { + if (isClusterKeyEnabled()) { + return + } String db = context.config.getDbNameByFile(context.file) sql "select 1;" // to create database diff --git a/regression-test/suites/unique_with_mow_c_p0/partial_update/test_partial_update_strict_mode.groovy b/regression-test/suites/unique_with_mow_c_p0/partial_update/test_partial_update_strict_mode.groovy index fe47a5ebc89450..313fc97d06ba55 100644 --- a/regression-test/suites/unique_with_mow_c_p0/partial_update/test_partial_update_strict_mode.groovy +++ b/regression-test/suites/unique_with_mow_c_p0/partial_update/test_partial_update_strict_mode.groovy @@ -17,6 +17,9 @@ // under the License. suite("test_partial_update_strict_mode", "p0") { + if (isClusterKeyEnabled()) { + return + } String db = context.config.getDbNameByFile(context.file) sql "select 1;" // to create database diff --git a/regression-test/suites/unique_with_mow_c_p0/partial_update/test_partial_update_upsert.groovy b/regression-test/suites/unique_with_mow_c_p0/partial_update/test_partial_update_upsert.groovy index 5e4a02f5407904..c8203cb0f9da53 100644 --- a/regression-test/suites/unique_with_mow_c_p0/partial_update/test_partial_update_upsert.groovy +++ b/regression-test/suites/unique_with_mow_c_p0/partial_update/test_partial_update_upsert.groovy @@ -17,6 +17,9 @@ // under the License. suite("test_partial_update_upsert", "p0") { + if (isClusterKeyEnabled()) { + return + } String db = context.config.getDbNameByFile(context.file) sql "select 1;" // to create database diff --git a/regression-test/suites/unique_with_mow_c_p0/partial_update/test_partial_update_with_inverted_index.groovy b/regression-test/suites/unique_with_mow_c_p0/partial_update/test_partial_update_with_inverted_index.groovy index f72160bb594ee9..3395757d876516 100644 --- a/regression-test/suites/unique_with_mow_c_p0/partial_update/test_partial_update_with_inverted_index.groovy +++ b/regression-test/suites/unique_with_mow_c_p0/partial_update/test_partial_update_with_inverted_index.groovy @@ -17,6 +17,9 @@ // under the License. suite("test_partial_update_with_inverted_index", "p0") { + if (isClusterKeyEnabled()) { + return + } String db = context.config.getDbNameByFile(context.file) sql "select 1;" // to create database diff --git a/regression-test/suites/unique_with_mow_c_p0/partial_update/test_partial_update_with_row_column.groovy b/regression-test/suites/unique_with_mow_c_p0/partial_update/test_partial_update_with_row_column.groovy index 4b4de014b6772d..ea80d898ff18e8 100644 --- a/regression-test/suites/unique_with_mow_c_p0/partial_update/test_partial_update_with_row_column.groovy +++ b/regression-test/suites/unique_with_mow_c_p0/partial_update/test_partial_update_with_row_column.groovy @@ -17,6 +17,10 @@ // under the License. suite("test_primary_key_partial_update_with_row_column", "p0") { + if (isClusterKeyEnabled()) { + return + } + def tableName = "test_primary_key_partial_update_with_row_column" // create table diff --git a/regression-test/suites/unique_with_mow_c_p0/partial_update/test_update_on_current_timestamp.groovy b/regression-test/suites/unique_with_mow_c_p0/partial_update/test_update_on_current_timestamp.groovy index 85f4f721273587..794097ffa14f4a 100644 --- a/regression-test/suites/unique_with_mow_c_p0/partial_update/test_update_on_current_timestamp.groovy +++ b/regression-test/suites/unique_with_mow_c_p0/partial_update/test_update_on_current_timestamp.groovy @@ -17,6 +17,10 @@ // under the License. suite("test_mow_update_on_current_timestamp", "p0") { + if (isClusterKeyEnabled()) { + return + } + sql 'set experimental_enable_nereids_planner=true' sql 'set enable_fallback_to_original_planner=false' sql 'set enable_nereids_dml=true' diff --git a/regression-test/suites/unique_with_mow_c_p0/test_create_table.groovy b/regression-test/suites/unique_with_mow_c_p0/test_create_table.groovy index a7810c2a6c3b12..c8cb94904c328e 100644 --- a/regression-test/suites/unique_with_mow_c_p0/test_create_table.groovy +++ b/regression-test/suites/unique_with_mow_c_p0/test_create_table.groovy @@ -16,6 +16,10 @@ // under the License. suite("test_create_table") { + if (isClusterKeyEnabled()) { + return + } + def tableName = "cluster_key_test_create_table" sql """ DROP TABLE IF EXISTS ${tableName} """ onFinish { diff --git a/regression-test/suites/unique_with_mow_c_p0/test_schema_change_ck.groovy b/regression-test/suites/unique_with_mow_c_p0/test_schema_change_ck.groovy index 09849d9fce22db..fb59aabf05dcd1 100644 --- a/regression-test/suites/unique_with_mow_c_p0/test_schema_change_ck.groovy +++ b/regression-test/suites/unique_with_mow_c_p0/test_schema_change_ck.groovy @@ -242,8 +242,8 @@ suite("test_schema_change_ck") { order_qt_select_restore_roll2 """select k2, k1, c4, c3 from ${tableName};""" // restore - logger.info(""" RESTORE SNAPSHOT ${context.dbName}.${backup} FROM `${repoName}` ON (`${tableName}`) PROPERTIES ("backup_timestamp" = "${snapshot}","replication_num" = "1" ) """) - sql """ RESTORE SNAPSHOT ${context.dbName}.${backup} FROM `${repoName}` ON (`${tableName}`) PROPERTIES ("backup_timestamp" = "${snapshot}","replication_num" = "1" ) """ + logger.info(""" RESTORE SNAPSHOT ${context.dbName}.${backup} FROM `${repoName}` ON (`${tableName}`) PROPERTIES ("backup_timestamp" = "${snapshot}","reserve_replica" = "true" ) """) + sql """ RESTORE SNAPSHOT ${context.dbName}.${backup} FROM `${repoName}` ON (`${tableName}`) PROPERTIES ("backup_timestamp" = "${snapshot}","reserve_replica" = "true" ) """ syncer.waitAllRestoreFinish(context.dbName) result = sql """ show tablets from ${tableName}; """ logger.info("tablets 1: ${result}") @@ -255,7 +255,7 @@ suite("test_schema_change_ck") { // restore sql """ drop table ${tableName}; """ - sql """ RESTORE SNAPSHOT ${context.dbName}.${backup} FROM `${repoName}` ON (`${tableName}`) PROPERTIES ("backup_timestamp" = "${snapshot}","replication_num" = "1" ) """ + sql """ RESTORE SNAPSHOT ${context.dbName}.${backup} FROM `${repoName}` ON (`${tableName}`) PROPERTIES ("backup_timestamp" = "${snapshot}","reserve_replica" = "true" ) """ syncer.waitAllRestoreFinish(context.dbName) result = sql """ show tablets from ${tableName}; """ logger.info("tablets 2: ${result}") From 42f9c217f935ac621aa0d28256b0ece622655c02 Mon Sep 17 00:00:00 2001 From: shee <13843187+qzsee@users.noreply.github.com> Date: Thu, 28 Nov 2024 14:45:07 +0800 Subject: [PATCH 058/399] [Metric](FE) add slow query metric (#41800) --- .../java/org/apache/doris/metric/MetricCalculator.java | 8 ++++++++ .../src/main/java/org/apache/doris/metric/MetricRepo.java | 6 ++++++ .../src/main/java/org/apache/doris/qe/AuditLogHelper.java | 1 + 3 files changed, 15 insertions(+) diff --git a/fe/fe-core/src/main/java/org/apache/doris/metric/MetricCalculator.java b/fe/fe-core/src/main/java/org/apache/doris/metric/MetricCalculator.java index f4beed94b0c636..06d206377d7811 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/metric/MetricCalculator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/metric/MetricCalculator.java @@ -33,6 +33,7 @@ public class MetricCalculator extends TimerTask { private long lastQueryCounter = -1; private long lastRequestCounter = -1; private long lastQueryErrCounter = -1; + private long lastQuerySlowCounter = -1; private Map clusterLastRequestCounter = new HashMap<>(); private Map clusterLastQueryCounter = new HashMap<>(); @@ -50,6 +51,7 @@ private void update() { lastQueryCounter = MetricRepo.COUNTER_QUERY_ALL.getValue(); lastRequestCounter = MetricRepo.COUNTER_REQUEST_ALL.getValue(); lastQueryErrCounter = MetricRepo.COUNTER_QUERY_ERR.getValue(); + lastQuerySlowCounter = MetricRepo.COUNTER_QUERY_SLOW.getValue(); initCloudMetrics(); return; } @@ -74,6 +76,12 @@ private void update() { MetricRepo.GAUGE_QUERY_ERR_RATE.setValue(errRate < 0 ? 0.0 : errRate); lastQueryErrCounter = currentErrCounter; + // slow rate + long currentSlowCounter = MetricRepo.COUNTER_QUERY_SLOW.getValue(); + double slowRate = (double) (currentSlowCounter - lastQuerySlowCounter) / interval; + MetricRepo.GAUGE_QUERY_SLOW_RATE.setValue(slowRate < 0 ? 0.0 : slowRate); + lastQuerySlowCounter = currentSlowCounter; + updateCloudMetrics(interval); lastTs = currentTs; diff --git a/fe/fe-core/src/main/java/org/apache/doris/metric/MetricRepo.java b/fe/fe-core/src/main/java/org/apache/doris/metric/MetricRepo.java index 86741758250e84..ca731eacf4b0bb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/metric/MetricRepo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/metric/MetricRepo.java @@ -74,6 +74,7 @@ public final class MetricRepo { public static LongCounterMetric COUNTER_REQUEST_ALL; public static LongCounterMetric COUNTER_QUERY_ALL; public static LongCounterMetric COUNTER_QUERY_ERR; + public static LongCounterMetric COUNTER_QUERY_SLOW; public static LongCounterMetric COUNTER_QUERY_TABLE; public static LongCounterMetric COUNTER_QUERY_OLAP_TABLE; public static LongCounterMetric COUNTER_QUERY_HIVE_TABLE; @@ -142,6 +143,7 @@ public final class MetricRepo { public static GaugeMetricImpl GAUGE_QUERY_PER_SECOND; public static GaugeMetricImpl GAUGE_REQUEST_PER_SECOND; public static GaugeMetricImpl GAUGE_QUERY_ERR_RATE; + public static GaugeMetricImpl GAUGE_QUERY_SLOW_RATE; public static GaugeMetricImpl GAUGE_MAX_TABLET_COMPACTION_SCORE; private static Map, Long> loadJobNum = Maps.newHashMap(); @@ -292,6 +294,8 @@ public Long getValue() { DORIS_METRIC_REGISTER.addMetrics(GAUGE_REQUEST_PER_SECOND); GAUGE_QUERY_ERR_RATE = new GaugeMetricImpl<>("query_err_rate", MetricUnit.NOUNIT, "query error rate", 0.0); DORIS_METRIC_REGISTER.addMetrics(GAUGE_QUERY_ERR_RATE); + GAUGE_QUERY_SLOW_RATE = new GaugeMetricImpl<>("query_slow_rate", MetricUnit.NOUNIT, "query slow rate", 0.0); + DORIS_METRIC_REGISTER.addMetrics(GAUGE_QUERY_SLOW_RATE); GAUGE_MAX_TABLET_COMPACTION_SCORE = new GaugeMetricImpl<>("max_tablet_compaction_score", MetricUnit.NOUNIT, "max tablet compaction score of all backends", 0L); DORIS_METRIC_REGISTER.addMetrics(GAUGE_MAX_TABLET_COMPACTION_SCORE); @@ -303,6 +307,8 @@ public Long getValue() { DORIS_METRIC_REGISTER.addMetrics(COUNTER_QUERY_ALL); COUNTER_QUERY_ERR = new LongCounterMetric("query_err", MetricUnit.REQUESTS, "total error query"); DORIS_METRIC_REGISTER.addMetrics(COUNTER_QUERY_ERR); + COUNTER_QUERY_SLOW = new LongCounterMetric("query_slow", MetricUnit.REQUESTS, "total slow query"); + DORIS_METRIC_REGISTER.addMetrics(COUNTER_QUERY_SLOW); COUNTER_QUERY_TABLE = new LongCounterMetric("query_table", MetricUnit.REQUESTS, "total query from table"); DORIS_METRIC_REGISTER.addMetrics(COUNTER_QUERY_TABLE); COUNTER_QUERY_OLAP_TABLE = new LongCounterMetric("query_olap_table", MetricUnit.REQUESTS, diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/AuditLogHelper.java b/fe/fe-core/src/main/java/org/apache/doris/qe/AuditLogHelper.java index 12958d53390c89..903b72898c9512 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/AuditLogHelper.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/AuditLogHelper.java @@ -249,6 +249,7 @@ private static void logAuditLogImpl(ConnectContext ctx, String origStmt, Stateme if (elapseMs > Config.qe_slow_log_ms) { String sqlDigest = DigestUtils.md5Hex(((Queriable) parsedStmt).toDigest()); auditEventBuilder.setSqlDigest(sqlDigest); + MetricRepo.COUNTER_QUERY_SLOW.increase(1L); } } auditEventBuilder.setIsQuery(true) From 10c74f3e829b6005cb3254e663940031ad26fe93 Mon Sep 17 00:00:00 2001 From: Sridhar R Manikarnike Date: Thu, 28 Nov 2024 12:18:46 +0530 Subject: [PATCH 059/399] [Enhancement] (nereids)implement showCreateViewCommand in nereids (#43145) Issue Number: close #42737 implement showCreateViewCommand in nereids --- .../org/apache/doris/nereids/DorisParser.g4 | 2 +- .../nereids/parser/LogicalPlanBuilder.java | 9 ++ .../doris/nereids/trees/plans/PlanType.java | 1 + .../plans/commands/ShowCreateViewCommand.java | 126 ++++++++++++++++++ .../trees/plans/visitor/CommandVisitor.java | 5 + .../nereids_p0/show/test_show_create_view.out | 7 + .../show/test_show_create_view.groovy | 76 +++++++++++ 7 files changed, 225 insertions(+), 1 deletion(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowCreateViewCommand.java create mode 100644 regression-test/data/nereids_p0/show/test_show_create_view.out create mode 100644 regression-test/suites/nereids_p0/show/test_show_create_view.groovy diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index 08f9058bbb14d6..6444f533ac0458 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -236,6 +236,7 @@ supportedShowStatement | SHOW STORAGE? ENGINES #showStorageEngines | SHOW CREATE CATALOG name=identifier #showCreateCatalog | SHOW SQL_BLOCK_RULE (FOR ruleName=identifier)? #showSqlBlockRule + | SHOW CREATE VIEW name=multipartIdentifier #showCreateView | SHOW CREATE MATERIALIZED VIEW mvName=identifier ON tableName=multipartIdentifier #showCreateMaterializedView | SHOW BACKENDS #showBackends @@ -292,7 +293,6 @@ unsupportedShowStatement | SHOW FULL? VIEWS ((FROM | IN) database=multipartIdentifier)? wildWhere? #showViews | SHOW FULL? PROCESSLIST #showProcessList | SHOW (GLOBAL | SESSION | LOCAL)? STATUS wildWhere? #showStatus - | SHOW CREATE VIEW name=multipartIdentifier #showCreateView | SHOW CREATE MATERIALIZED VIEW name=multipartIdentifier #showMaterializedView | SHOW CREATE (DATABASE | SCHEMA) name=multipartIdentifier #showCreateDatabase | SHOW CREATE (GLOBAL | SESSION | LOCAL)? FUNCTION functionIdentifier diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index abd6afb84bc7f2..f2119ddd61d3bd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -230,6 +230,7 @@ import org.apache.doris.nereids.DorisParser.ShowCreateMaterializedViewContext; import org.apache.doris.nereids.DorisParser.ShowCreateProcedureContext; import org.apache.doris.nereids.DorisParser.ShowCreateTableContext; +import org.apache.doris.nereids.DorisParser.ShowCreateViewContext; import org.apache.doris.nereids.DorisParser.ShowDeleteContext; import org.apache.doris.nereids.DorisParser.ShowDynamicPartitionContext; import org.apache.doris.nereids.DorisParser.ShowEventsContext; @@ -514,6 +515,7 @@ import org.apache.doris.nereids.trees.plans.commands.ShowCreateMaterializedViewCommand; import org.apache.doris.nereids.trees.plans.commands.ShowCreateProcedureCommand; import org.apache.doris.nereids.trees.plans.commands.ShowCreateTableCommand; +import org.apache.doris.nereids.trees.plans.commands.ShowCreateViewCommand; import org.apache.doris.nereids.trees.plans.commands.ShowDeleteCommand; import org.apache.doris.nereids.trees.plans.commands.ShowDynamicPartitionCommand; import org.apache.doris.nereids.trees.plans.commands.ShowEventsCommand; @@ -4563,6 +4565,12 @@ public LogicalPlan visitShowCreateTable(ShowCreateTableContext ctx) { return new ShowCreateTableCommand(new TableNameInfo(nameParts), ctx.BRIEF() != null); } + @Override + public LogicalPlan visitShowCreateView(ShowCreateViewContext ctx) { + List nameParts = visitMultipartIdentifier(ctx.name); + return new ShowCreateViewCommand(new TableNameInfo(nameParts)); + } + @Override public LogicalPlan visitShowCreateMaterializedView(ShowCreateMaterializedViewContext ctx) { List nameParts = visitMultipartIdentifier(ctx.tableName); @@ -4642,6 +4650,7 @@ public RecoverPartitionCommand visitRecoverPartition(RecoverPartitionContext ctx } @Override + public LogicalPlan visitShowBroker(ShowBrokerContext ctx) { return new ShowBrokerCommand(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index 09fa71f5614e87..dced9a0a579827 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -200,6 +200,7 @@ public enum PlanType { SHOW_CREATE_CATALOG_COMMAND, SHOW_CREATE_MATERIALIZED_VIEW_COMMAND, SHOW_CREATE_TABLE_COMMAND, + SHOW_CREATE_VIEW_COMMAND, SHOW_DELETE_COMMAND, SHOW_DYNAMIC_PARTITION_COMMAND, SHOW_EVENTS_COMMAND, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowCreateViewCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowCreateViewCommand.java new file mode 100644 index 00000000000000..96fb047c542f85 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowCreateViewCommand.java @@ -0,0 +1,126 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.commands; + +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.DatabaseIf; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.MTMV; +import org.apache.doris.catalog.ScalarType; +import org.apache.doris.catalog.TableIf; +import org.apache.doris.catalog.View; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.commands.info.TableNameInfo; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.ShowResultSet; +import org.apache.doris.qe.ShowResultSetMetaData; +import org.apache.doris.qe.StmtExecutor; + +import com.google.common.collect.Lists; + +import java.util.List; + +/** + * Represents the command for SHOW CREATE VIEW. + */ +public class ShowCreateViewCommand extends ShowCommand { + private static final ShowResultSetMetaData VIEW_META_DATA = + ShowResultSetMetaData.builder() + .addColumn(new Column("View", ScalarType.createVarchar(20))) + .addColumn(new Column("Create View", ScalarType.createVarchar(30))) + .addColumn(new Column("character_set_client", ScalarType.createVarchar(30))) + .addColumn(new Column("collation_connection", ScalarType.createVarchar(30))) + .build(); + + private final TableNameInfo tblNameInfo; + + public ShowCreateViewCommand(TableNameInfo tableNameInfo) { + super(PlanType.SHOW_CREATE_VIEW_COMMAND); + this.tblNameInfo = tableNameInfo; + } + + private void validate(ConnectContext ctx) throws AnalysisException { + tblNameInfo.analyze(ctx); + + TableIf tableIf = Env.getCurrentEnv().getCatalogMgr() + .getCatalogOrAnalysisException(tblNameInfo.getCtl()) + .getDbOrAnalysisException(tblNameInfo.getDb()).getTableOrAnalysisException(tblNameInfo.getTbl()); + + if (tableIf instanceof MTMV) { + ErrorReport.reportAnalysisException("not support async materialized view, " + + "please use `show create materialized view`"); + } + + PrivPredicate wanted; + if (tableIf instanceof View) { + wanted = PrivPredicate.SHOW_VIEW; + } else { + wanted = PrivPredicate.SHOW; + } + + if (!Env.getCurrentEnv().getAccessManager().checkTblPriv(ConnectContext.get(), + tblNameInfo.getCtl(), tblNameInfo.getDb(), tblNameInfo.getTbl(), wanted)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_TABLEACCESS_DENIED_ERROR, "SHOW CREATE TABLE", + ConnectContext.get().getQualifiedUser(), + ConnectContext.get().getRemoteIP(), + tblNameInfo.getDb() + ": " + tblNameInfo.getTbl()); + } + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitShowCreateViewCommand(this, context); + } + + @Override + public ShowResultSet doRun(ConnectContext ctx, StmtExecutor executor) throws Exception { + validate(ctx); + // Fetch the catalog, database, and view metadata + DatabaseIf db = ctx.getEnv().getCatalogMgr().getCatalogOrAnalysisException(tblNameInfo.getCtl()) + .getDbOrMetaException(tblNameInfo.getDb()); + TableIf view = db.getTableOrMetaException(tblNameInfo.getTbl()); + + if (!(view instanceof View)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_WRONG_OBJECT, tblNameInfo.getDb(), tblNameInfo.getTbl(), + "VIEW", "Use 'SHOW CREATE TABLE '" + tblNameInfo.getTbl()); + } + + List> rows = Lists.newArrayList(); + // Lock the view to ensure consistent metadata access + view.readLock(); + try { + List createViewStmt = Lists.newArrayList(); + ctx.getEnv().getDdlStmt(null, null, view, createViewStmt, null, null, false, true, + false, -1L, false, false); + + if (!createViewStmt.isEmpty()) { + rows.add(Lists.newArrayList(view.getName(), createViewStmt.get(0), "utf8mb4", "utf8mb4_0900_bin")); + } + } finally { + view.readUnlock(); + } + + // Set the result set and send it using the executor + return new ShowResultSet(VIEW_META_DATA, rows); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index ef1688c3398b82..b4a56f71d4166e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -79,6 +79,7 @@ import org.apache.doris.nereids.trees.plans.commands.ShowCreateMaterializedViewCommand; import org.apache.doris.nereids.trees.plans.commands.ShowCreateProcedureCommand; import org.apache.doris.nereids.trees.plans.commands.ShowCreateTableCommand; +import org.apache.doris.nereids.trees.plans.commands.ShowCreateViewCommand; import org.apache.doris.nereids.trees.plans.commands.ShowDeleteCommand; import org.apache.doris.nereids.trees.plans.commands.ShowDynamicPartitionCommand; import org.apache.doris.nereids.trees.plans.commands.ShowEventsCommand; @@ -396,6 +397,10 @@ default R visitShowCreateMaterializedViewCommand(ShowCreateMaterializedViewComma return visitCommand(showCreateMtlzViewCommand, context); } + default R visitShowCreateViewCommand(ShowCreateViewCommand showCreateViewCommand, C context) { + return visitCommand(showCreateViewCommand, context); + } + default R visitAlterRoleCommand(AlterRoleCommand alterRoleCommand, C context) { return visitCommand(alterRoleCommand, context); } diff --git a/regression-test/data/nereids_p0/show/test_show_create_view.out b/regression-test/data/nereids_p0/show/test_show_create_view.out new file mode 100644 index 00000000000000..8ac669337b482c --- /dev/null +++ b/regression-test/data/nereids_p0/show/test_show_create_view.out @@ -0,0 +1,7 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !cmd -- +view_show_create_view CREATE VIEW `view_show_create_view` AS SELECT `internal`.`regression_test_nereids_p0_show`.`table_for_view_test`.`id`, `internal`.`regression_test_nereids_p0_show`.`table_for_view_test`.`name` FROM `internal`.`regression_test_nereids_p0_show`.`table_for_view_test`; utf8mb4 utf8mb4_0900_bin + +-- !cmd -- +view_show_create_view_2 CREATE VIEW `view_show_create_view_2` AS SELECT `internal`.`regression_test_nereids_p0_show`.`table_for_view_test_2`.`key_field`, `internal`.`regression_test_nereids_p0_show`.`table_for_view_test_2`.`value` FROM `internal`.`regression_test_nereids_p0_show`.`table_for_view_test_2`; utf8mb4 utf8mb4_0900_bin + diff --git a/regression-test/suites/nereids_p0/show/test_show_create_view.groovy b/regression-test/suites/nereids_p0/show/test_show_create_view.groovy new file mode 100644 index 00000000000000..65ed4758a9ae66 --- /dev/null +++ b/regression-test/suites/nereids_p0/show/test_show_create_view.groovy @@ -0,0 +1,76 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("test_show_create_view", "query,arrow_flight_sql") { + String view_name = "view_show_create_view"; + String table_name = "table_for_view_test"; + try { + // Create a table for testing + sql """ + CREATE TABLE IF NOT EXISTS ${table_name} ( + id INT COMMENT "Primary key", + name STRING COMMENT "Name field" + ) + DISTRIBUTED BY HASH(id) BUCKETS 5 + PROPERTIES ("replication_num" = "1"); + """ + + // Create a view based on the created table + sql """ + CREATE VIEW IF NOT EXISTS ${view_name} AS + SELECT id, name FROM ${table_name} + """ + + // Execute the SHOW CREATE VIEW command + checkNereidsExecute("""show create view `${view_name}`;""") + qt_cmd("""show create view `${view_name}`;""") + } finally { + // Drop the view and table after testing + try_sql("DROP VIEW IF EXISTS `${view_name}`") + try_sql("DROP TABLE IF EXISTS `${table_name}`") + } + + // Additional case: Create another view based on a different table + String view_name_2 = "view_show_create_view_2"; + String table_name_2 = "table_for_view_test_2"; + try { + // Create another table for testing + sql """ + CREATE TABLE IF NOT EXISTS ${table_name_2} ( + `key_field` INT COMMENT "Key field", + `value` STRING COMMENT "Value field" + ) + DISTRIBUTED BY HASH(key_field) BUCKETS 3 + PROPERTIES ("replication_num" = "1"); + """ + + // Create a view based on the new table + sql """ + CREATE VIEW IF NOT EXISTS ${view_name_2} AS + SELECT key_field, value FROM ${table_name_2} + """ + + // Execute the SHOW CREATE VIEW command for the new view + checkNereidsExecute("""show create view `${view_name_2}`;""") + qt_cmd("""show create view `${view_name_2}`;""") + + } finally { + // Drop the view and table after testing + try_sql("DROP VIEW IF EXISTS `${view_name_2}`") + try_sql("DROP TABLE IF EXISTS `${table_name_2}`") + } +} From b761e4ec34be7a0bba2fb805e604e7c8283032cf Mon Sep 17 00:00:00 2001 From: Sridhar R Manikarnike Date: Thu, 28 Nov 2024 12:19:48 +0530 Subject: [PATCH 060/399] [Enhancement] (nereids)implement showDiagnoseTabletCommand in nereids (#43159) Issue Number: close #42836 implement showDiagnoseTabletCommand in nereids --- .../org/apache/doris/nereids/DorisParser.g4 | 2 +- .../nereids/parser/LogicalPlanBuilder.java | 8 ++ .../doris/nereids/trees/plans/PlanType.java | 1 + .../commands/ShowDiagnoseTabletCommand.java | 84 +++++++++++++++++++ .../trees/plans/visitor/CommandVisitor.java | 5 ++ .../show/test_show_diagnose_tablet.groovy | 39 +++++++++ 6 files changed, 138 insertions(+), 1 deletion(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowDiagnoseTabletCommand.java create mode 100644 regression-test/suites/nereids_p0/show/test_show_diagnose_tablet.groovy diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index 6444f533ac0458..1b1e218a958146 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -242,6 +242,7 @@ supportedShowStatement | SHOW BACKENDS #showBackends | SHOW REPLICA DISTRIBUTION FROM baseTableRef #showReplicaDistribution | SHOW FULL? TRIGGERS ((FROM | IN) database=multipartIdentifier)? wildWhere? #showTriggers + | SHOW TABLET DIAGNOSIS tabletId=INTEGER_VALUE #showDiagnoseTablet | SHOW FRONTENDS name=identifier? #showFrontends | SHOW TABLE tableId=INTEGER_VALUE #showTableId | SHOW WHITELIST #showWhitelist @@ -355,7 +356,6 @@ unsupportedShowStatement | SHOW CONVERT_LSC ((FROM | IN) database=multipartIdentifier)? #showConvertLsc | SHOW REPLICA STATUS FROM baseTableRef wildWhere? #showReplicaStatus | SHOW TABLET STORAGE FORMAT VERBOSE? #showTabletStorageFormat - | SHOW TABLET DIAGNOSIS tabletId=INTEGER_VALUE #showDiagnoseTablet | SHOW COPY ((FROM | IN) database=multipartIdentifier)? whereClause? sortClause? limitClause? #showCopy | SHOW WARM UP JOB wildWhere? #showWarmUpJob diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index f2119ddd61d3bd..2db26dc15011f0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -232,6 +232,7 @@ import org.apache.doris.nereids.DorisParser.ShowCreateTableContext; import org.apache.doris.nereids.DorisParser.ShowCreateViewContext; import org.apache.doris.nereids.DorisParser.ShowDeleteContext; +import org.apache.doris.nereids.DorisParser.ShowDiagnoseTabletContext; import org.apache.doris.nereids.DorisParser.ShowDynamicPartitionContext; import org.apache.doris.nereids.DorisParser.ShowEventsContext; import org.apache.doris.nereids.DorisParser.ShowFrontendsContext; @@ -517,6 +518,7 @@ import org.apache.doris.nereids.trees.plans.commands.ShowCreateTableCommand; import org.apache.doris.nereids.trees.plans.commands.ShowCreateViewCommand; import org.apache.doris.nereids.trees.plans.commands.ShowDeleteCommand; +import org.apache.doris.nereids.trees.plans.commands.ShowDiagnoseTabletCommand; import org.apache.doris.nereids.trees.plans.commands.ShowDynamicPartitionCommand; import org.apache.doris.nereids.trees.plans.commands.ShowEventsCommand; import org.apache.doris.nereids.trees.plans.commands.ShowFrontendsCommand; @@ -4559,6 +4561,12 @@ public LogicalPlan visitShowStorageEngines(ShowStorageEnginesContext ctx) { return new ShowStorageEnginesCommand(); } + @Override + public LogicalPlan visitShowDiagnoseTablet(ShowDiagnoseTabletContext ctx) { + long tabletId = Long.parseLong(ctx.INTEGER_VALUE().getText()); + return new ShowDiagnoseTabletCommand(tabletId); + } + @Override public LogicalPlan visitShowCreateTable(ShowCreateTableContext ctx) { List nameParts = visitMultipartIdentifier(ctx.name); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index dced9a0a579827..8f474ea395f889 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -202,6 +202,7 @@ public enum PlanType { SHOW_CREATE_TABLE_COMMAND, SHOW_CREATE_VIEW_COMMAND, SHOW_DELETE_COMMAND, + SHOW_DIAGNOSE_TABLET_COMMAND, SHOW_DYNAMIC_PARTITION_COMMAND, SHOW_EVENTS_COMMAND, SHOW_FRONTENDS_COMMAND, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowDiagnoseTabletCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowDiagnoseTabletCommand.java new file mode 100644 index 00000000000000..6dac6ca214ede5 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowDiagnoseTabletCommand.java @@ -0,0 +1,84 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.commands; + +import org.apache.doris.analysis.RedirectStatus; +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.ScalarType; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.ShowResultSet; +import org.apache.doris.qe.ShowResultSetMetaData; +import org.apache.doris.qe.StmtExecutor; +import org.apache.doris.system.Diagnoser; + +import com.google.common.collect.ImmutableList; + +import java.util.List; + +/** + * Represents the command for SHOW TABLET DIAGNOSIS + */ +public class ShowDiagnoseTabletCommand extends ShowCommand { + public static final ImmutableList TITLE_NAMES = new ImmutableList.Builder() + .add("Item").add("Info").add("Suggestion") + .build(); + + private final long tabletId; + + public ShowDiagnoseTabletCommand(long tabletId) { + super(PlanType.SHOW_DIAGNOSE_TABLET_COMMAND); + this.tabletId = tabletId; + } + + public ShowResultSetMetaData getMetaData() { + ShowResultSetMetaData.Builder builder = ShowResultSetMetaData.builder(); + for (String title : TITLE_NAMES) { + builder.addColumn(new Column(title, ScalarType.createVarchar(1024))); + } + return builder.build(); + } + + @Override + public ShowResultSet doRun(ConnectContext ctx, StmtExecutor executor) throws Exception { + // check auth + if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, "ADMIN"); + } + + List> resultRowSet = Diagnoser.diagnoseTablet(tabletId); + // Set the result set and send it using the executor + return new ShowResultSet(getMetaData(), resultRowSet); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitShowDiagnoseTabletCommand(this, context); + } + + @Override + public RedirectStatus toRedirectStatus() { + return RedirectStatus.FORWARD_NO_SYNC; + } + +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index b4a56f71d4166e..c00c0ef90ca018 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -81,6 +81,7 @@ import org.apache.doris.nereids.trees.plans.commands.ShowCreateTableCommand; import org.apache.doris.nereids.trees.plans.commands.ShowCreateViewCommand; import org.apache.doris.nereids.trees.plans.commands.ShowDeleteCommand; +import org.apache.doris.nereids.trees.plans.commands.ShowDiagnoseTabletCommand; import org.apache.doris.nereids.trees.plans.commands.ShowDynamicPartitionCommand; import org.apache.doris.nereids.trees.plans.commands.ShowEventsCommand; import org.apache.doris.nereids.trees.plans.commands.ShowFrontendsCommand; @@ -433,6 +434,10 @@ default R visitRecoverDatabaseCommand(RecoverDatabaseCommand recoverDatabaseComm return visitCommand(recoverDatabaseCommand, context); } + default R visitShowDiagnoseTabletCommand(ShowDiagnoseTabletCommand showDiagnoseTabletCommand, C context) { + return visitCommand(showDiagnoseTabletCommand, context); + } + default R visitRecoverTableCommand(RecoverTableCommand recoverTableCommand, C context) { return visitCommand(recoverTableCommand, context); } diff --git a/regression-test/suites/nereids_p0/show/test_show_diagnose_tablet.groovy b/regression-test/suites/nereids_p0/show/test_show_diagnose_tablet.groovy new file mode 100644 index 00000000000000..029fac7428e0bb --- /dev/null +++ b/regression-test/suites/nereids_p0/show/test_show_diagnose_tablet.groovy @@ -0,0 +1,39 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("test_show_diagnose_tablet_nereids", "query,diagnose") { + String tableName = "test_table_for_diagnosis"; + String tabletId = ""; + try { + // Create a new table to test the SHOW TABLET DIAGNOSIS command + sql "CREATE TABLE IF NOT EXISTS ${tableName} (id INT, name STRING) DUPLICATE KEY(id) DISTRIBUTED BY HASH(id) BUCKETS 3 PROPERTIES('replication_num'='1');" + + // Extract tablet ID from the created table + def showTabletsResult = sql "SHOW TABLETS FROM ${tableName}" + assert showTabletsResult.size() > 0 + tabletId = showTabletsResult[0][0] // Assuming the first tablet ID is used + + // Execute the SHOW TABLET DIAGNOSIS command and verify the output + checkNereidsExecute("SHOW TABLET DIAGNOSIS ${tabletId}") + } catch (Exception e) { + log.error("Failed to execute SHOW TABLET DIAGNOSIS command", e) + throw e + } finally { + try_sql("DROP TABLE IF EXISTS ${tableName}") + } +} + From a4d9aaa75cf1c059d36bbddc456effffb50396f0 Mon Sep 17 00:00:00 2001 From: feiniaofeiafei Date: Thu, 28 Nov 2024 14:56:39 +0800 Subject: [PATCH 061/399] [opt](nereids) Use a unified approach to deal with monotonic function in partition prune (#42934) This pr use a unified approach to deal with monotonic function in partition prune. When adding a monotonic function in partition prune, we don't need to add visitXXX in OneRangePartitionEvaluator, only need to implement the Monotonic interface. --- .../rules/OneRangePartitionEvaluator.java | 76 +++++++------------ .../expressions/functions/Monotonic.java | 3 +- .../functions/scalar/ConvertTz.java | 3 +- .../expressions/functions/scalar/Date.java | 3 +- .../functions/scalar/DateTrunc.java | 3 +- 5 files changed, 31 insertions(+), 57 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/OneRangePartitionEvaluator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/OneRangePartitionEvaluator.java index 1fb8954ab16547..7e91d5502f7fad 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/OneRangePartitionEvaluator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/OneRangePartitionEvaluator.java @@ -43,9 +43,8 @@ import org.apache.doris.nereids.trees.expressions.Or; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.functions.Monotonic; -import org.apache.doris.nereids.trees.expressions.functions.scalar.ConvertTz; -import org.apache.doris.nereids.trees.expressions.functions.scalar.Date; -import org.apache.doris.nereids.trees.expressions.functions.scalar.DateTrunc; +import org.apache.doris.nereids.trees.expressions.functions.scalar.NonNullable; +import org.apache.doris.nereids.trees.expressions.functions.scalar.Nullable; import org.apache.doris.nereids.trees.expressions.literal.BooleanLiteral; import org.apache.doris.nereids.trees.expressions.literal.Literal; import org.apache.doris.nereids.trees.expressions.literal.MaxLiteral; @@ -91,6 +90,7 @@ public class OneRangePartitionEvaluator private final List lowers; private final List uppers; private final List> inputs; + // whether the Expression in partition range may be null. private final Map partitionSlotContainsNull; private final Map slotToType; @@ -456,7 +456,12 @@ private EvaluateRangeResult evaluateChildrenThenThis(Expression expr, EvaluateRa for (int i = 0; i < children.size(); i++) { Expression child = children.get(i); - EvaluateRangeResult childResult = child.accept(this, context); + EvaluateRangeResult childResult; + if (child instanceof Monotonic) { + childResult = visitMonotonic(child, context); + } else { + childResult = child.accept(this, context); + } if (!childResult.result.equals(child)) { hasNewChildren = true; } @@ -615,45 +620,6 @@ private List toMultiNereidsLiterals(PartitionKey partitionKey) { return literals; } - @Override - public EvaluateRangeResult visitDateTrunc(DateTrunc dateTrunc, EvaluateRangeInput context) { - EvaluateRangeResult result = super.visitDateTrunc(dateTrunc, context); - if (!(result.result instanceof DateTrunc)) { - return result; - } - Expression dateTruncChild = dateTrunc.child(0); - if (partitionSlotContainsNull.containsKey(dateTruncChild)) { - partitionSlotContainsNull.put(dateTrunc, true); - } - return computeMonotonicFunctionRange(result, context.rangeMap); - } - - @Override - public EvaluateRangeResult visitDate(Date date, EvaluateRangeInput context) { - EvaluateRangeResult result = super.visitDate(date, context); - if (!(result.result instanceof Date)) { - return result; - } - Expression dateChild = date.child(0); - if (partitionSlotContainsNull.containsKey(dateChild)) { - partitionSlotContainsNull.put(date, true); - } - return computeMonotonicFunctionRange(result, context.rangeMap); - } - - @Override - public EvaluateRangeResult visitConvertTz(ConvertTz convertTz, EvaluateRangeInput context) { - EvaluateRangeResult result = super.visitConvertTz(convertTz, context); - if (!(result.result instanceof ConvertTz)) { - return result; - } - Expression converTzChild = convertTz.child(0); - if (partitionSlotContainsNull.containsKey(converTzChild)) { - partitionSlotContainsNull.put(convertTz, true); - } - return computeMonotonicFunctionRange(result, context.rangeMap); - } - private boolean isPartitionSlot(Slot slot) { return slotToType.containsKey(slot); } @@ -820,15 +786,27 @@ private List> commonComputeOnePartitionInputs() { return onePartitionInputs; } - private EvaluateRangeResult computeMonotonicFunctionRange(EvaluateRangeResult result, - Map rangeMap) { + public EvaluateRangeResult visitMonotonic(Expression monotonic, EvaluateRangeInput context) { + EvaluateRangeResult rangeResult = evaluateChildrenThenThis(monotonic, context); + if (!rangeResult.result.getClass().equals(monotonic.getClass())) { + return rangeResult; + } + return computeMonotonicFunctionRange(rangeResult, context); + } + + private EvaluateRangeResult computeMonotonicFunctionRange(EvaluateRangeResult result, EvaluateRangeInput context) { Monotonic func = (Monotonic) result.result; - if (rangeMap.containsKey(func)) { + if (context.rangeMap.containsKey(func)) { return new EvaluateRangeResult((Expression) func, ImmutableMap.of((Expression) func, - rangeMap.get(func)), result.childrenResult); + context.rangeMap.get(func)), result.childrenResult); } int childIndex = func.getMonotonicFunctionChildIndex(); Expression funcChild = func.child(childIndex); + boolean isNullable = partitionSlotContainsNull.getOrDefault(funcChild, true); + Expression withNullable = func.withConstantArgs(isNullable ? new Nullable(funcChild) + : new NonNullable(funcChild)); + partitionSlotContainsNull.put((Expression) func, withNullable.nullable()); + if (!result.childrenResult.get(0).columnRanges.containsKey(funcChild)) { return result; } @@ -854,7 +832,7 @@ private EvaluateRangeResult computeMonotonicFunctionRange(EvaluateRangeResult re ColumnRange newRange = ColumnRange.all(); if (lowerValue instanceof Literal && upperValue instanceof Literal && lowerValue.equals(upperValue)) { newRange = ColumnRange.singleton((Literal) lowerValue); - rangeMap.put((Expression) func, newRange); + context.rangeMap.put((Expression) func, newRange); newRanges.put((Expression) func, newRange); return new EvaluateRangeResult(lowerValue, newRanges, result.childrenResult); } else { @@ -864,7 +842,7 @@ private EvaluateRangeResult computeMonotonicFunctionRange(EvaluateRangeResult re if (upperValue instanceof Literal) { newRange = newRange.withUpperBound((Literal) upperValue); } - rangeMap.put((Expression) func, newRange); + context.rangeMap.put((Expression) func, newRange); newRanges.put((Expression) func, newRange); return new EvaluateRangeResult((Expression) func, newRanges, result.childrenResult); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/Monotonic.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/Monotonic.java index 2fdde0e7415613..bcaa040cb2a650 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/Monotonic.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/Monotonic.java @@ -18,7 +18,6 @@ package org.apache.doris.nereids.trees.expressions.functions; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.literal.Literal; /** monotonicity of expressions */ public interface Monotonic extends ExpressionTrait { @@ -32,5 +31,5 @@ public interface Monotonic extends ExpressionTrait { // return the function with the arguments replaced by literal // e.g. date_trunc(dt, 'day'), dt in range ['2020-01-01 10:00:00', '2020-01-03 10:00:00'] // return date_trunc('2020-01-01 10:00:00', 'day') - Expression withConstantArgs(Literal literal); + Expression withConstantArgs(Expression literal); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ConvertTz.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ConvertTz.java index faee04cacd82b8..e0d717cd04f66a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ConvertTz.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ConvertTz.java @@ -24,7 +24,6 @@ import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.functions.Monotonic; import org.apache.doris.nereids.trees.expressions.functions.PropagateNullLiteral; -import org.apache.doris.nereids.trees.expressions.literal.Literal; import org.apache.doris.nereids.trees.expressions.literal.NullLiteral; import org.apache.doris.nereids.trees.expressions.literal.StringLikeLiteral; import org.apache.doris.nereids.trees.expressions.shape.TernaryExpression; @@ -99,7 +98,7 @@ public int getMonotonicFunctionChildIndex() { } @Override - public Expression withConstantArgs(Literal literal) { + public Expression withConstantArgs(Expression literal) { return new ConvertTz(literal, child(1), child(2)); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Date.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Date.java index 3f7ae84975031a..80c6178cd3ffb8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Date.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Date.java @@ -23,7 +23,6 @@ import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.functions.Monotonic; import org.apache.doris.nereids.trees.expressions.functions.PropagateNullLiteral; -import org.apache.doris.nereids.trees.expressions.literal.Literal; import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; import org.apache.doris.nereids.types.DateTimeType; @@ -84,7 +83,7 @@ public int getMonotonicFunctionChildIndex() { } @Override - public Expression withConstantArgs(Literal literal) { + public Expression withConstantArgs(Expression literal) { return new Date(literal); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DateTrunc.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DateTrunc.java index cbd2da5627b375..2a4d838c4fd5ba 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DateTrunc.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DateTrunc.java @@ -23,7 +23,6 @@ import org.apache.doris.nereids.trees.expressions.functions.AlwaysNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.functions.Monotonic; -import org.apache.doris.nereids.trees.expressions.literal.Literal; import org.apache.doris.nereids.trees.expressions.literal.VarcharLiteral; import org.apache.doris.nereids.trees.expressions.shape.BinaryExpression; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; @@ -105,7 +104,7 @@ public int getMonotonicFunctionChildIndex() { } @Override - public Expression withConstantArgs(Literal literal) { + public Expression withConstantArgs(Expression literal) { return new DateTrunc(literal, child(1)); } } From 12192739e052ad6c03aeab7126bf2e7cfb35fa90 Mon Sep 17 00:00:00 2001 From: yiguolei Date: Thu, 28 Nov 2024 15:59:52 +0800 Subject: [PATCH 062/399] [chore](compile) fix some compile warning in BE (#44454) --- .../serde/data_type_date64_serde.cpp | 8 ++++---- .../array/function_array_contains_all.cpp | 17 ++++++++-------- .../functions/array/function_array_element.h | 8 ++++++-- .../array/function_array_intersect.cpp | 3 ++- .../vec/functions/array/function_array_join.h | 2 ++ .../vec/functions/array/function_array_map.h | 18 +++++++++-------- .../functions/array/function_array_mapped.h | 3 ++- .../vec/functions/array/function_array_nary.h | 2 ++ .../functions/array/function_array_range.cpp | 7 ++++--- .../array/function_array_shuffle.cpp | 3 ++- .../functions/array/function_array_sortby.cpp | 2 +- .../functions/array/function_array_split.cpp | 2 +- .../functions/array/function_array_union.cpp | 1 + .../functions/array/function_array_utils.h | 4 +++- .../array/function_array_with_constant.cpp | 6 ++++-- be/src/vec/functions/date_format_type.h | 15 +++++++------- be/src/vec/functions/date_time_transforms.h | 20 ++++++++++++++----- .../functions/function_binary_arithmetic.h | 1 - be/src/vec/functions/function_bit_count.cpp | 6 ++++-- be/src/vec/functions/function_bitmap.cpp | 15 ++++++++------ be/src/vec/functions/function_case.h | 10 ++++++---- be/src/vec/functions/function_cast.h | 1 - be/src/vec/runtime/vdatetime_value.cpp | 10 +++++----- be/src/vec/runtime/vdatetime_value.h | 14 ++++++------- 24 files changed, 107 insertions(+), 71 deletions(-) diff --git a/be/src/vec/data_types/serde/data_type_date64_serde.cpp b/be/src/vec/data_types/serde/data_type_date64_serde.cpp index 6cb5b31ae387a2..8e102ec6e3a729 100644 --- a/be/src/vec/data_types/serde/data_type_date64_serde.cpp +++ b/be/src/vec/data_types/serde/data_type_date64_serde.cpp @@ -167,12 +167,12 @@ void DataTypeDate64SerDe::write_column_to_arrow(const IColumn& column, const Nul for (size_t i = start; i < end; ++i) { char buf[64]; const VecDateTimeValue* time_val = (const VecDateTimeValue*)(&col_data[i]); - int len = time_val->to_buffer(buf); + size_t len = time_val->to_buffer(buf); if (null_map && (*null_map)[i]) { checkArrowStatus(string_builder.AppendNull(), column.get_name(), array_builder->type()->name()); } else { - checkArrowStatus(string_builder.Append(buf, len), column.get_name(), + checkArrowStatus(string_builder.Append(buf, cast_set(len)), column.get_name(), array_builder->type()->name()); } } @@ -305,8 +305,8 @@ Status DataTypeDate64SerDe::write_column_to_orc(const std::string& timezone, con continue; } - int len = binary_cast(col_data[row_id]) - .to_buffer(const_cast(bufferRef.data) + offset); + size_t len = binary_cast(col_data[row_id]) + .to_buffer(const_cast(bufferRef.data) + offset); REALLOC_MEMORY_FOR_ORC_WRITER() diff --git a/be/src/vec/functions/array/function_array_contains_all.cpp b/be/src/vec/functions/array/function_array_contains_all.cpp index 298289278bfa5f..67279babd5f49b 100644 --- a/be/src/vec/functions/array/function_array_contains_all.cpp +++ b/be/src/vec/functions/array/function_array_contains_all.cpp @@ -29,6 +29,7 @@ #include "vec/functions/simple_function_factory.h" namespace doris::vectorized { +#include "common/compile_check_begin.h" class FunctionArrayContainsAll : public IFunction { public: @@ -189,12 +190,12 @@ class FunctionArrayContainsAll : public IFunction { for (ssize_t row = 0; row < input_rows_count; ++row) { auto left_index = index_check_const(row, left_is_const); auto right_index = index_check_const(row, right_is_const); - ssize_t left_start = (*left_data.offsets_ptr)[left_index - 1]; - ssize_t left_end = (*left_data.offsets_ptr)[left_index]; - ssize_t left_size = left_end - left_start; - ssize_t right_start = (*right_data.offsets_ptr)[right_index - 1]; - ssize_t right_end = (*right_data.offsets_ptr)[right_index]; - ssize_t right_size = right_end - right_start; + size_t left_start = (*left_data.offsets_ptr)[left_index - 1]; + size_t left_end = (*left_data.offsets_ptr)[left_index]; + size_t left_size = left_end - left_start; + size_t right_start = (*right_data.offsets_ptr)[right_index - 1]; + size_t right_end = (*right_data.offsets_ptr)[right_index]; + size_t right_size = right_end - right_start; // case: [1,2,3] : [] if (right_size == 0) { dst_data[row] = 1; @@ -216,7 +217,7 @@ class FunctionArrayContainsAll : public IFunction { is_equal_value = false; break; } - int left_nested_loop_pos = left_pos; + size_t left_nested_loop_pos = left_pos; right_pos = right_start; while (right_pos < right_end) { bool left_nested_data_is_null = @@ -261,5 +262,5 @@ void register_function_array_contains_all(SimpleFunctionFactory& factory) { factory.register_function(); factory.register_alias("array_contains_all", "hasSubstr"); } - +#include "common/compile_check_end.h" } // namespace doris::vectorized diff --git a/be/src/vec/functions/array/function_array_element.h b/be/src/vec/functions/array/function_array_element.h index 7a4807608c3ef9..e4b57348c883ae 100644 --- a/be/src/vec/functions/array/function_array_element.h +++ b/be/src/vec/functions/array/function_array_element.h @@ -58,6 +58,7 @@ class FunctionContext; } // namespace doris namespace doris::vectorized { +#include "common/compile_check_begin.h" class FunctionArrayElement : public IFunction { public: @@ -147,14 +148,16 @@ class FunctionArrayElement : public IFunction { size_t end = offsets[i]; for (size_t j = begin; j < end; j++) { if (nested_ptr->compare_at(j, i, *right_column, -1) == 0) { - matched_indices->insert_value(j - begin + 1); + matched_indices->insert_value( + cast_set(j - begin + 1)); matched = true; break; } } if (!matched) { - matched_indices->insert_value(end - begin + 1); // make indices for null + matched_indices->insert_value(cast_set( + end - begin + 1)); // make indices for null } } @@ -415,4 +418,5 @@ class FunctionArrayElement : public IFunction { } }; +#include "common/compile_check_end.h" } // namespace doris::vectorized diff --git a/be/src/vec/functions/array/function_array_intersect.cpp b/be/src/vec/functions/array/function_array_intersect.cpp index e68ddf30542e6a..369b85631f38a7 100644 --- a/be/src/vec/functions/array/function_array_intersect.cpp +++ b/be/src/vec/functions/array/function_array_intersect.cpp @@ -24,6 +24,7 @@ namespace doris::vectorized { +#include "common/compile_check_begin.h" struct NameArrayIntersect { static constexpr auto name = "array_intersect"; }; @@ -41,7 +42,7 @@ struct IntersectAction { bool apply_null() { return result_null_flag; } // Handle Non-Null element. - void apply(Map& map, const int arg_idx, const int row_idx, + void apply(Map& map, const size_t arg_idx, const size_t row_idx, const ColumnArrayExecutionData& param) { current_null_flag = false; size_t start_off = (*param.offsets_ptr)[row_idx - 1]; diff --git a/be/src/vec/functions/array/function_array_join.h b/be/src/vec/functions/array/function_array_join.h index f7ffd4dfca3449..957b2288fb746a 100644 --- a/be/src/vec/functions/array/function_array_join.h +++ b/be/src/vec/functions/array/function_array_join.h @@ -25,6 +25,7 @@ namespace doris::vectorized { +#include "common/compile_check_begin.h" struct NameArrayJoin { static constexpr auto name = "array_join"; }; @@ -266,4 +267,5 @@ struct ArrayJoinImpl { } }; +#include "common/compile_check_end.h" } // namespace doris::vectorized diff --git a/be/src/vec/functions/array/function_array_map.h b/be/src/vec/functions/array/function_array_map.h index bcb71a4a79f264..fd4a2fc59f3548 100644 --- a/be/src/vec/functions/array/function_array_map.h +++ b/be/src/vec/functions/array/function_array_map.h @@ -26,6 +26,7 @@ #include "vec/functions/function_helpers.h" namespace doris::vectorized { +#include "common/compile_check_begin.h" enum class MapOperation { INTERSECT, UNION }; @@ -64,9 +65,9 @@ struct OpenMapImpl { // this method calculate rows to get a rest dst data void apply(ColumnArrayMutableData& dst, const ColumnArrayExecutionDatas params, - std::vector& col_const, int start_row, int end_row) { + std::vector& col_const, size_t start_row, size_t end_row) { size_t dst_off = 0; - for (int row = start_row; row < end_row; ++row) { + for (size_t row = start_row; row < end_row; ++row) { reset(); for (int i = 0; i < params.size(); ++i) { action.apply(map, i, index_check_const(row, col_const[i]), params[i]); @@ -110,11 +111,11 @@ struct OpenMapImpl { } void apply(ColumnArrayMutableData& dst, const ColumnArrayExecutionDatas params, - std::vector& col_const, int start_row, int end_row) { + std::vector& col_const, size_t start_row, size_t end_row) { size_t dst_off = 0; - for (int row = start_row; row < end_row; ++row) { + for (size_t row = start_row; row < end_row; ++row) { reset(); - for (int i = 0; i < params.size(); ++i) { + for (size_t i = 0; i < params.size(); ++i) { action.apply(map, i, index_check_const(row, col_const[i]), params[i]); } // nullmap @@ -162,7 +163,7 @@ struct ArrayMapImpl { } static Status execute(ColumnPtr& res_ptr, ColumnArrayExecutionDatas datas, - std::vector& col_const, int start_row, int end_row) { + std::vector& col_const, size_t start_row, size_t end_row) { ColumnArrayMutableData dst = create_mutable_data(datas[0].nested_col, datas[0].nested_nullmap_data); if (_execute_internal(dst, datas, col_const, start_row, end_row)) { @@ -175,7 +176,7 @@ struct ArrayMapImpl { private: template static bool _execute_internal(ColumnArrayMutableData& dst, ColumnArrayExecutionDatas datas, - std::vector& col_const, int start_row, int end_row) { + std::vector& col_const, size_t start_row, size_t end_row) { for (auto data : datas) { if (!check_column(*data.nested_col)) { return false; @@ -192,10 +193,11 @@ struct ArrayMapImpl { template requires(sizeof...(Ts) > 0) static bool _execute_internal(ColumnArrayMutableData& dst, ColumnArrayExecutionDatas datas, - std::vector& col_const, int start_row, int end_row) { + std::vector& col_const, size_t start_row, size_t end_row) { return _execute_internal(dst, datas, col_const, start_row, end_row) || _execute_internal(dst, datas, col_const, start_row, end_row); } }; +#include "common/compile_check_end.h" } // namespace doris::vectorized diff --git a/be/src/vec/functions/array/function_array_mapped.h b/be/src/vec/functions/array/function_array_mapped.h index 93f7d068e4b01d..5c84baf9dfadb8 100644 --- a/be/src/vec/functions/array/function_array_mapped.h +++ b/be/src/vec/functions/array/function_array_mapped.h @@ -26,7 +26,7 @@ namespace doris { namespace vectorized { - +#include "common/compile_check_begin.h" /** Higher-order functions for arrays. * These functions optionally apply a map (transform) to array (or multiple arrays of identical size) by lambda function, * and return some result based on that transformation. @@ -72,5 +72,6 @@ class FunctionArrayMapped : public IFunction { } }; +#include "common/compile_check_end.h" } // namespace vectorized } // namespace doris diff --git a/be/src/vec/functions/array/function_array_nary.h b/be/src/vec/functions/array/function_array_nary.h index 923b64473a8d1d..e9b2009f68484c 100644 --- a/be/src/vec/functions/array/function_array_nary.h +++ b/be/src/vec/functions/array/function_array_nary.h @@ -25,6 +25,7 @@ #include "vec/functions/function_helpers.h" namespace doris::vectorized { +#include "common/compile_check_begin.h" // Functions with more than two arrays of the same element type. template @@ -78,4 +79,5 @@ class FunctionArrayNary : public IFunction { } }; +#include "common/compile_check_end.h" } // namespace doris::vectorized diff --git a/be/src/vec/functions/array/function_array_range.cpp b/be/src/vec/functions/array/function_array_range.cpp index b6b814a26e339a..8a3de3754503ae 100644 --- a/be/src/vec/functions/array/function_array_range.cpp +++ b/be/src/vec/functions/array/function_array_range.cpp @@ -52,6 +52,7 @@ class FunctionContext; } // namespace doris namespace doris::vectorized { +#include "common/compile_check_begin.h" template class FunctionArrayRange : public IFunction { @@ -174,7 +175,7 @@ struct RangeImplUtil { PaddedPODArray& nested_column, PaddedPODArray& dest_nested_null_map, ColumnArray::Offsets64& dest_offsets) { - int rows = start.size(); + size_t rows = start.size(); for (auto row = 0; row < rows; ++row) { auto idx = start[row]; auto end_row = end[row]; @@ -193,7 +194,7 @@ struct RangeImplUtil { return Status::InvalidArgument("Array size exceeds the limit {}", max_array_size_as_field); } - int offset = dest_offsets.back(); + size_t offset = dest_offsets.back(); while (idx < end[row]) { nested_column.push_back(idx); dest_nested_null_map.push_back(0); @@ -213,7 +214,7 @@ struct RangeImplUtil { dest_offsets.push_back(dest_offsets.back()); continue; } else { - int offset = dest_offsets.back(); + size_t offset = dest_offsets.back(); using UNIT = std::conditional_t, std::integral_constant, TimeUnitOrVoid>; diff --git a/be/src/vec/functions/array/function_array_shuffle.cpp b/be/src/vec/functions/array/function_array_shuffle.cpp index 648b06318df3dc..fbf5bcfcdf1686 100644 --- a/be/src/vec/functions/array/function_array_shuffle.cpp +++ b/be/src/vec/functions/array/function_array_shuffle.cpp @@ -45,6 +45,7 @@ class FunctionContext; } // namespace doris namespace doris::vectorized { +#include "common/compile_check_begin.h" class FunctionArrayShuffle : public IFunction { public: @@ -71,7 +72,7 @@ class FunctionArrayShuffle : public IFunction { block.get_by_position(arguments[0]).column->convert_to_full_column_if_const(); const auto& src_column_array = assert_cast(*src_column); - uint32_t seed = time(nullptr); + size_t seed = time(nullptr); if (arguments.size() == 2) { ColumnPtr seed_column = block.get_by_position(arguments[1]).column->convert_to_full_column_if_const(); diff --git a/be/src/vec/functions/array/function_array_sortby.cpp b/be/src/vec/functions/array/function_array_sortby.cpp index 499a5e37ab9380..899bb40fba1423 100644 --- a/be/src/vec/functions/array/function_array_sortby.cpp +++ b/be/src/vec/functions/array/function_array_sortby.cpp @@ -46,7 +46,7 @@ class FunctionContext; } // namespace doris namespace doris::vectorized { - +#include "common/compile_check_begin.h" class FunctionArraySortBy : public IFunction { public: static constexpr auto name = "array_sortby"; diff --git a/be/src/vec/functions/array/function_array_split.cpp b/be/src/vec/functions/array/function_array_split.cpp index 7f7a847abc71f4..5e58a9f189a657 100644 --- a/be/src/vec/functions/array/function_array_split.cpp +++ b/be/src/vec/functions/array/function_array_split.cpp @@ -46,7 +46,7 @@ class FunctionContext; } // namespace doris namespace doris::vectorized { - +#include "common/compile_check_begin.h" template class FunctionArraySplit : public IFunction { public: diff --git a/be/src/vec/functions/array/function_array_union.cpp b/be/src/vec/functions/array/function_array_union.cpp index 75f7ec211b00c1..8d40682193584d 100644 --- a/be/src/vec/functions/array/function_array_union.cpp +++ b/be/src/vec/functions/array/function_array_union.cpp @@ -28,6 +28,7 @@ namespace doris::vectorized { +#include "common/compile_check_begin.h" struct NameArrayUnion { static constexpr auto name = "array_union"; }; diff --git a/be/src/vec/functions/array/function_array_utils.h b/be/src/vec/functions/array/function_array_utils.h index 36bf811b770899..dc522ed813760a 100644 --- a/be/src/vec/functions/array/function_array_utils.h +++ b/be/src/vec/functions/array/function_array_utils.h @@ -29,6 +29,7 @@ class IColumn; } // namespace doris namespace doris::vectorized { +#include "common/compile_check_begin.h" struct ColumnArrayMutableData { public: @@ -70,7 +71,7 @@ struct ColumnArrayExecutionData { dst.offsets_ptr->push_back((*offsets_ptr)[row]); size_t off = (*offsets_ptr)[row - 1]; size_t len = (*offsets_ptr)[row] - off; - for (int start = off; start < off + len; ++start) { + for (size_t start = off; start < off + len; ++start) { if (nested_nullmap_data && nested_nullmap_data[start]) { dst.nested_col->insert_default(); dst.nested_nullmap_data->push_back(1); @@ -94,4 +95,5 @@ void slice_array(ColumnArrayMutableData& dst, ColumnArrayExecutionData& src, const IColumn& offset_column, const IColumn* length_column); using ColumnArrayExecutionDatas = std::vector; +#include "common/compile_check_end.h" } // namespace doris::vectorized diff --git a/be/src/vec/functions/array/function_array_with_constant.cpp b/be/src/vec/functions/array/function_array_with_constant.cpp index ba385535ebb3a6..7b9f3b43271f05 100644 --- a/be/src/vec/functions/array/function_array_with_constant.cpp +++ b/be/src/vec/functions/array/function_array_with_constant.cpp @@ -48,6 +48,7 @@ class Block; } // namespace doris namespace doris::vectorized { +#include "common/compile_check_begin.h" /* array_with_constant(num, T) / array_repeat(T, num) - return array of constants with length num. * array_with_constant(2, 'xxx') = ['xxx', 'xxx'] @@ -89,7 +90,8 @@ class FunctionArrayWithConstant : public IFunction { ColumnArray::Offset64 offset = 0; std::vector array_sizes; array_sizes.reserve(input_rows_count); - for (size_t i = 0; i < input_rows_count; ++i) { + // The array size will never gt int max value. + for (int i = 0; i < input_rows_count; ++i) { auto array_size = num->get_int(i); if (UNLIKELY(array_size < 0) || UNLIKELY(array_size > max_array_size_as_field)) { return Status::InvalidArgument("Array size should in range(0, {}) in function: {}", @@ -132,5 +134,5 @@ void register_function_array_with_constant(SimpleFunctionFactory& factory) { factory.register_function>(); factory.register_function>(); } - +#include "common/compile_check_end.h" } // namespace doris::vectorized diff --git a/be/src/vec/functions/date_format_type.h b/be/src/vec/functions/date_format_type.h index 071ecf44853e1d..d7a0dcea2b028f 100644 --- a/be/src/vec/functions/date_format_type.h +++ b/be/src/vec/functions/date_format_type.h @@ -22,6 +22,7 @@ #include "vec/common/string_ref.h" namespace doris::vectorized::time_format_type { +#include "common/compile_check_begin.h" // Used to optimize commonly used date formats. inline StringRef rewrite_specific_format(const char* raw_str, size_t str_size) { @@ -41,18 +42,18 @@ inline StringRef rewrite_specific_format(const char* raw_str, size_t str_size) { template void put_year(T y, char* buf, int& i) { int t = y / 100; - buf[i++] = t / 10 + '0'; - buf[i++] = t % 10 + '0'; + buf[i++] = cast_set(t / 10 + '0'); + buf[i++] = cast_set(t % 10 + '0'); t = y % 100; - buf[i++] = t / 10 + '0'; - buf[i++] = t % 10 + '0'; + buf[i++] = cast_set(t / 10 + '0'); + buf[i++] = cast_set(t % 10 + '0'); } template void put_other(T m, char* buf, int& i) { - buf[i++] = m / 10 + '0'; - buf[i++] = m % 10 + '0'; + buf[i++] = cast_set(m / 10 + '0'); + buf[i++] = cast_set(m % 10 + '0'); } // NoneImpl indicates that no specific optimization has been applied, and the general logic is used for processing. @@ -152,5 +153,5 @@ inline FormatImplVariant string_to_impl(const std::string& format) { return NoneImpl {}; } } - +#include "common/compile_check_end.h" } // namespace doris::vectorized::time_format_type diff --git a/be/src/vec/functions/date_time_transforms.h b/be/src/vec/functions/date_time_transforms.h index d2c30d9d706332..c23fe147400412 100644 --- a/be/src/vec/functions/date_time_transforms.h +++ b/be/src/vec/functions/date_time_transforms.h @@ -38,6 +38,7 @@ #include "vec/utils/util.hpp" namespace doris::vectorized { +#include "common/compile_check_begin.h" #define TIME_FUNCTION_IMPL(CLASS, UNIT, FUNCTION) \ template \ @@ -313,7 +314,8 @@ struct TransformerToStringOneArgument { const auto& date_time_value = reinterpret_cast::T&>( t); - res_offsets[i] = Transform::execute(date_time_value, res_data, offset); + res_offsets[i] = + cast_set(Transform::execute(date_time_value, res_data, offset)); null_map[i] = !date_time_value.is_valid_date(); } } @@ -331,7 +333,8 @@ struct TransformerToStringOneArgument { const auto& date_time_value = reinterpret_cast::T&>( t); - res_offsets[i] = Transform::execute(date_time_value, res_data, offset); + res_offsets[i] = + cast_set(Transform::execute(date_time_value, res_data, offset)); DCHECK(date_time_value.is_valid_date()); } } @@ -360,7 +363,7 @@ struct TransformerToStringTwoArgument { } else { std::tie(new_offset, is_null) = Transform::execute(t, format, res_data, offset); } - res_offsets[i] = new_offset; + res_offsets[i] = cast_set(new_offset); null_map[i] = is_null; } } @@ -375,7 +378,11 @@ struct Transformer { null_map.resize(size); for (size_t i = 0; i < size; ++i) { - vec_to[i] = Transform::execute(vec_from[i]); + // The transform result maybe an int, int32, but the result maybe short + // for example, year function. It is only a short. + auto res = Transform::execute(vec_from[i]); + using RESULT_TYPE = std::decay_t; + vec_to[i] = cast_set(res); null_map[i] = !((typename DateTraits::T&)(vec_from[i])) .is_valid_date(); } @@ -386,7 +393,9 @@ struct Transformer { vec_to.resize(size); for (size_t i = 0; i < size; ++i) { - vec_to[i] = Transform::execute(vec_from[i]); + auto res = Transform::execute(vec_from[i]); + using RESULT_TYPE = std::decay_t; + vec_to[i] = cast_set(res); DCHECK(((typename DateTraits::T&)(vec_from[i])) .is_valid_date()); } @@ -465,4 +474,5 @@ struct DateTimeTransformImpl { } }; +#include "common/compile_check_end.h" } // namespace doris::vectorized diff --git a/be/src/vec/functions/function_binary_arithmetic.h b/be/src/vec/functions/function_binary_arithmetic.h index 31e1d50f45b03c..4c0b8e7a0890dc 100644 --- a/be/src/vec/functions/function_binary_arithmetic.h +++ b/be/src/vec/functions/function_binary_arithmetic.h @@ -42,7 +42,6 @@ #include "vec/utils/template_helpers.hpp" namespace doris::vectorized { - // Arithmetic operations: +, -, *, |, &, ^, ~ // need implement apply(a, b) diff --git a/be/src/vec/functions/function_bit_count.cpp b/be/src/vec/functions/function_bit_count.cpp index 4c3e45d81c8a8c..d4d9306a7c58bb 100644 --- a/be/src/vec/functions/function_bit_count.cpp +++ b/be/src/vec/functions/function_bit_count.cpp @@ -28,7 +28,7 @@ #include "vec/functions/simple_function_factory.h" namespace doris::vectorized { - +#include "common/compile_check_begin.h" struct NameBitCount { static constexpr auto name = "bit_count"; }; @@ -43,7 +43,9 @@ struct BitCountImpl { if constexpr (std::is_same_v || std::is_same_v || std::is_same_v || std::is_same_v || std::is_same_v) { - return std::popcount(static_cast>(a)); + // ResultType already check the length + return cast_set( + std::popcount(static_cast>(a))); } else { throw Exception(ErrorCode::INVALID_ARGUMENT, "bit_count only support using INTEGER as operator"); diff --git a/be/src/vec/functions/function_bitmap.cpp b/be/src/vec/functions/function_bitmap.cpp index 64059472efcee9..d5f2075dffb6d2 100644 --- a/be/src/vec/functions/function_bitmap.cpp +++ b/be/src/vec/functions/function_bitmap.cpp @@ -76,6 +76,7 @@ class FunctionContext; } // namespace doris namespace doris::vectorized { +#include "common/compile_check_begin.h" struct BitmapEmpty { static constexpr auto name = "bitmap_empty"; @@ -112,7 +113,7 @@ struct ToBitmap { continue; } else { const char* raw_str = reinterpret_cast(&data[offsets[i - 1]]); - size_t str_size = offsets[i] - offsets[i - 1]; + int str_size = cast_set(offsets[i] - offsets[i - 1]); StringParser::ParseResult parse_result = StringParser::PARSE_SUCCESS; uint64_t int_value = StringParser::string_to_unsigned_int( raw_str, str_size, &parse_result); @@ -168,7 +169,8 @@ struct ToBitmapWithCheck { continue; } else { const char* raw_str = reinterpret_cast(&data[offsets[i - 1]]); - size_t str_size = offsets[i] - offsets[i - 1]; + // The string lenght is less than 2G, so that cast the str size to int, not use size_t + int str_size = cast_set(offsets[i] - offsets[i - 1]); StringParser::ParseResult parse_result = StringParser::PARSE_SUCCESS; uint64_t int_value = StringParser::string_to_unsigned_int( raw_str, str_size, &parse_result); @@ -265,11 +267,11 @@ struct BitmapFromBase64 { return Status::OK(); } std::string decode_buff; - int last_decode_buff_len = 0; - int curr_decode_buff_len = 0; + size_t last_decode_buff_len = 0; + size_t curr_decode_buff_len = 0; for (size_t i = 0; i < input_rows_count; ++i) { const char* src_str = reinterpret_cast(&data[offsets[i - 1]]); - int64_t src_size = offsets[i] - offsets[i - 1]; + size_t src_size = offsets[i] - offsets[i - 1]; if (0 != src_size % 4) { // return Status::InvalidArgument( // fmt::format("invalid base64: {}", std::string(src_str, src_size))); @@ -984,6 +986,7 @@ struct BitmapToBase64 { using Chars = ColumnString::Chars; using Offsets = ColumnString::Offsets; + // ColumnString not support 64bit, only 32bit, so that the max size is 4G static Status vector(const std::vector& data, Chars& chars, Offsets& offsets) { size_t size = data.size(); offsets.resize(size); @@ -1015,7 +1018,7 @@ struct BitmapToBase64 { DCHECK(outlen > 0); encoded_offset += (int)(4.0 * ceil((double)cur_ser_size / 3.0)); - offsets[i] = encoded_offset; + offsets[i] = cast_set(encoded_offset); } return Status::OK(); } diff --git a/be/src/vec/functions/function_case.h b/be/src/vec/functions/function_case.h index fdfecc211caed8..af44ea0d9b1ace 100644 --- a/be/src/vec/functions/function_case.h +++ b/be/src/vec/functions/function_case.h @@ -46,6 +46,7 @@ #include "vec/utils/template_helpers.hpp" namespace doris::vectorized { +#include "common/compile_check_begin.h" template struct FunctionCaseName; @@ -89,7 +90,7 @@ struct CaseWhenColumnHolder { : std::nullopt); int begin = 0 + has_case; - int end = arguments.size() - has_else; + int end = cast_set(arguments.size() - has_else); pair_count = (end - begin) / 2 + 1; // when/then at [1: pair_count) for (int i = begin; i < end; i += 2) { @@ -131,7 +132,7 @@ class FunctionCase : public IFunction { DataTypePtr get_return_type_impl(const DataTypes& arguments) const override { int loop_start = has_case ? 2 : 1; - int loop_end = has_else ? arguments.size() - 1 : arguments.size(); + int loop_end = cast_set(has_else ? arguments.size() - 1 : arguments.size()); bool is_nullable = false; if (!has_else || arguments[loop_end].get()->is_nullable()) { @@ -156,7 +157,7 @@ class FunctionCase : public IFunction { Status execute_short_circuit(const DataTypePtr& data_type, Block& block, uint32_t result, CaseWhenColumnHolder column_holder) const { auto case_column_ptr = column_holder.when_ptrs[0].value_or(nullptr); - int rows_count = column_holder.rows_count; + size_t rows_count = column_holder.rows_count; // `then` data index corresponding to each row of results, 0 represents `else`. auto then_idx_uptr = std::unique_ptr(new int[rows_count]); @@ -203,7 +204,7 @@ class FunctionCase : public IFunction { column_holder); } - int rows_count = column_holder.rows_count; + size_t rows_count = column_holder.rows_count; // `then` data index corresponding to each row of results, 0 represents `else`. auto then_idx_uptr = std::unique_ptr(new uint8_t[rows_count]); @@ -426,4 +427,5 @@ class FunctionCase : public IFunction { } }; +#include "common/compile_check_end.h" } // namespace doris::vectorized diff --git a/be/src/vec/functions/function_cast.h b/be/src/vec/functions/function_cast.h index 7027a4e9c622bc..3ad456d60d201f 100644 --- a/be/src/vec/functions/function_cast.h +++ b/be/src/vec/functions/function_cast.h @@ -2401,5 +2401,4 @@ class FunctionBuilderCast : public FunctionBuilderImpl { bool use_default_implementation_for_nulls() const override { return false; } bool use_default_implementation_for_low_cardinality_columns() const override { return false; } }; - } // namespace doris::vectorized diff --git a/be/src/vec/runtime/vdatetime_value.cpp b/be/src/vec/runtime/vdatetime_value.cpp index f7e72efa0899ea..86c50f0936f30d 100644 --- a/be/src/vec/runtime/vdatetime_value.cpp +++ b/be/src/vec/runtime/vdatetime_value.cpp @@ -591,7 +591,7 @@ static char* append_with_prefix(const char* str, int str_len, char prefix, int t return to + str_len; } -int VecDateTimeValue::compute_format_len(const char* format, int len) { +int VecDateTimeValue::compute_format_len(const char* format, size_t len) { int size = 0; const char* ptr = format; const char* end = format + len; @@ -684,8 +684,8 @@ char* write_four_digits_to_string(int number, char* dst) { return dst + 4; } -bool VecDateTimeValue::to_format_string_conservative(const char* format, int len, char* to, - int max_valid_length) const { +bool VecDateTimeValue::to_format_string_conservative(const char* format, size_t len, char* to, + size_t max_valid_length) const { if (check_range(_year, _month, _day, _hour, _minute, _second, _type)) { return false; } @@ -3450,8 +3450,8 @@ void DateV2Value::set_microsecond(uint64_t microsecond) { } template -bool DateV2Value::to_format_string_conservative(const char* format, int len, char* to, - int max_valid_length) const { +bool DateV2Value::to_format_string_conservative(const char* format, size_t len, char* to, + size_t max_valid_length) const { if (is_invalid(year(), month(), day(), hour(), minute(), second(), microsecond())) { return false; } diff --git a/be/src/vec/runtime/vdatetime_value.h b/be/src/vec/runtime/vdatetime_value.h index 2cf7636347851a..dc7ba70b0b04ac 100644 --- a/be/src/vec/runtime/vdatetime_value.h +++ b/be/src/vec/runtime/vdatetime_value.h @@ -398,11 +398,11 @@ class VecDateTimeValue { // Now this type is a temp solution with little changes // for performance of checking, may return false when just APPROACH BUT NOT REACH max_valid_length. // so need a little big buffer and its length as max_valid_length to make sure store valid data. // to make sure of this. make the buffer size = + SAFE_FORMAT_STRING_MARGIN. and pass this size as max_valid_length - bool to_format_string_conservative(const char* format, int len, char* to, - int max_valid_length) const; + bool to_format_string_conservative(const char* format, size_t len, char* to, + size_t max_valid_length) const; // compute the length of data format pattern - static int compute_format_len(const char* format, int len); + static int compute_format_len(const char* format, size_t len); // Return true if range or date is invalid static bool check_range(uint32_t year, uint32_t month, uint32_t day, uint32_t hour, @@ -830,11 +830,11 @@ class DateV2Value { // for performance of checking, may return false when just APPROACH BUT NOT REACH max_valid_length. // so need a little big buffer and its length as max_valid_length to make sure store valid data. // to make sure of this. make the buffer size = + SAFE_FORMAT_STRING_MARGIN. and pass this size as max_valid_length - bool to_format_string_conservative(const char* format, int len, char* to, - int max_valid_length) const; + bool to_format_string_conservative(const char* format, size_t len, char* to, + size_t max_valid_length) const; - bool from_date_format_str(const char* format, int format_len, const char* value, - int64_t value_len) { + bool from_date_format_str(const char* format, size_t format_len, const char* value, + size_t value_len) { return from_date_format_str(format, format_len, value, value_len, nullptr); } From d77bfa09d85d9759965c109cc50ee89e8fbde499 Mon Sep 17 00:00:00 2001 From: Gabriel Date: Thu, 28 Nov 2024 16:33:16 +0800 Subject: [PATCH 063/399] =?UTF-8?q?[Improvement](shuffle)=20Use=20a=20knob?= =?UTF-8?q?=20to=20decide=20whether=20a=20serial=20exchange=E2=80=A6=20(#4?= =?UTF-8?q?4676)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit … should be used This improvement was completed in #43199 and reverted by #44075 due to performance fallback. After fixing it, this improvement is re-submited. A new knob to control a exchange node should be serial or not. For example, a partitioned hash join should be executed like below: ``` ┌────────────────────────────┐ ┌────────────────────────────┐ │ │ │ │ │Exchange(HASH PARTITIONED N)│ │Exchange(HASH PARTITIONED N)│ │ │ │ │ └────────────────────────────┴─────────┬────────┴────────────────────────────┘ │ │ │ │ │ │ ┌──────▼──────┐ │ │ │ HASH JOIN │ │ │ └─────────────┘ ``` After turning on this knob, the real plan should be: ``` ┌──────────────────────────────┐ ┌──────────────────────────────┐ │ │ │ │ │ Exchange (HASH PARTITIONED 1)│ │ Exchange (HASH PARTITIONED 1)│ │ │ │ │ └────────────┬─────────────────┘ └────────────┬─────────────────┘ │ │ │ │ │ │ │ │ │ │ ┌──────────────▼─────────────────────┐ ┌──────────────▼─────────────────────┐ │ │ │ │ │ Local Exchange(HASH PARTITIONED N)│ │ Local Exchange(HASH PARTITIONED N)│ │ 1 -> N │ │ 1 -> N │ └────────────────────────────────────┴─────────┬────────┴────────────────────────────────────┴ │ │ │ │ │ │ ┌──────▼──────┐ │ │ │ HASH JOIN │ │ │ └─────────────┘ ``` For large cluster, X (mappers) * Y (reducers) rpc channels can be reduced to X (mappers) * Z (BEs). --- .../org/apache/doris/planner/ExchangeNode.java | 13 ++++++++++++- .../org/apache/doris/planner/PlanFragment.java | 14 ++++++-------- .../java/org/apache/doris/planner/PlanNode.java | 7 +++++++ .../java/org/apache/doris/planner/ScanNode.java | 5 +++++ .../main/java/org/apache/doris/qe/Coordinator.java | 10 ++-------- .../java/org/apache/doris/qe/SessionVariable.java | 11 +++++++++++ 6 files changed, 43 insertions(+), 17 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/ExchangeNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/ExchangeNode.java index 1ca1db56bfc7c6..cb6628b01c556b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/ExchangeNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/ExchangeNode.java @@ -25,6 +25,7 @@ import org.apache.doris.analysis.TupleDescriptor; import org.apache.doris.analysis.TupleId; import org.apache.doris.common.UserException; +import org.apache.doris.qe.ConnectContext; import org.apache.doris.statistics.StatisticalType; import org.apache.doris.statistics.StatsRecursiveDerive; import org.apache.doris.thrift.TExchangeNode; @@ -169,6 +170,10 @@ public void setMergeInfo(SortInfo info) { @Override protected void toThrift(TPlanNode msg) { + // If this fragment has another scan node, this exchange node is serial or not should be decided by the scan + // node. + msg.setIsSerialOperator((isSerialOperator() || fragment.hasSerialScanNode()) + && fragment.useSerialSource(ConnectContext.get())); msg.node_type = TPlanNodeType.EXCHANGE_NODE; msg.exchange_node = new TExchangeNode(); for (TupleId tid : tupleIds) { @@ -228,11 +233,17 @@ public void setRightChildOfBroadcastHashJoin(boolean value) { */ @Override public boolean isSerialOperator() { - return partitionType == TPartitionType.UNPARTITIONED && mergeInfo != null; + return (ConnectContext.get() != null && ConnectContext.get().getSessionVariable().isUseSerialExchange() + || partitionType == TPartitionType.UNPARTITIONED) && mergeInfo != null; } @Override public boolean hasSerialChildren() { return isSerialOperator(); } + + @Override + public boolean hasSerialScanChildren() { + return false; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/PlanFragment.java b/fe/fe-core/src/main/java/org/apache/doris/planner/PlanFragment.java index fef3de9b696e9e..ab5307c07e904a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/PlanFragment.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/PlanFragment.java @@ -515,15 +515,13 @@ public boolean useSerialSource(ConnectContext context) { && !hasNullAwareLeftAntiJoin() // If planRoot is not a serial operator and has serial children, we can use serial source and improve // parallelism of non-serial operators. - && sink instanceof DataStreamSink && !planRoot.isSerialOperator() - && planRoot.hasSerialChildren(); + // For bucket shuffle / colocate join fragment, always use serial source if the bucket scan nodes are + // serial. + && (hasSerialScanNode() || (sink instanceof DataStreamSink && !planRoot.isSerialOperator() + && planRoot.hasSerialChildren())); } - public int getNumBackends() { - return numBackends; - } - - public void setNumBackends(int numBackends) { - this.numBackends = numBackends; + public boolean hasSerialScanNode() { + return planRoot.hasSerialScanChildren(); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/PlanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/PlanNode.java index 14bd34e93e1f43..73768435154b76 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/PlanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/PlanNode.java @@ -1388,4 +1388,11 @@ public boolean hasSerialChildren() { } return children.stream().allMatch(PlanNode::hasSerialChildren); } + + public boolean hasSerialScanChildren() { + if (children.isEmpty()) { + return false; + } + return children.stream().anyMatch(PlanNode::hasSerialScanChildren); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/ScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/ScanNode.java index a2583868346704..b4033a0535ef3d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/ScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/ScanNode.java @@ -861,4 +861,9 @@ public boolean isSerialOperator() { < ConnectContext.get().getSessionVariable().getParallelExecInstanceNum() * numScanBackends() || (ConnectContext.get() != null && ConnectContext.get().getSessionVariable().isForceToLocalShuffle()); } + + @Override + public boolean hasSerialScanChildren() { + return isSerialOperator(); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java index 3a6f6e4f84069d..262b5836689ab0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java @@ -1886,17 +1886,11 @@ protected void computeFragmentHosts() throws Exception { return scanNode.getId().asInt() == planNodeId; }).findFirst(); - /** - * Ignore storage data distribution iff: - * 1. `parallelExecInstanceNum * numBackends` is larger than scan ranges. - * 2. Use Nereids planner. - */ boolean sharedScan = true; int expectedInstanceNum = Math.min(parallelExecInstanceNum, leftMostNode.getNumInstances()); - boolean ignoreStorageDataDistribution = node.isPresent() - && fragment.useSerialSource(context); - if (node.isPresent() && ignoreStorageDataDistribution) { + boolean ignoreStorageDataDistribution = fragment.useSerialSource(context); + if (ignoreStorageDataDistribution) { expectedInstanceNum = Math.max(expectedInstanceNum, 1); // if have limit and no conjuncts, only need 1 instance to save cpu and // mem resource diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java index 71b746c7907262..115614a41873a6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java @@ -268,6 +268,8 @@ public class SessionVariable implements Serializable, Writable { public static final String IGNORE_STORAGE_DATA_DISTRIBUTION = "ignore_storage_data_distribution"; + public static final String USE_SERIAL_EXCHANGE = "use_serial_exchange"; + public static final String ENABLE_PARALLEL_SCAN = "enable_parallel_scan"; // Limit the max count of scanners to prevent generate too many scanners. @@ -1112,6 +1114,10 @@ public enum IgnoreSplitType { varType = VariableAnnotation.EXPERIMENTAL, needForward = true) private boolean ignoreStorageDataDistribution = true; + @VariableMgr.VarAttr(name = USE_SERIAL_EXCHANGE, fuzzy = true, + varType = VariableAnnotation.EXPERIMENTAL, needForward = true) + private boolean useSerialExchange = false; + @VariableMgr.VarAttr( name = ENABLE_LOCAL_SHUFFLE, fuzzy = false, varType = VariableAnnotation.EXPERIMENTAL, description = {"是否在pipelineX引擎上开启local shuffle优化", @@ -2353,6 +2359,7 @@ public void initFuzzyModeVariables() { this.parallelPrepareThreshold = random.nextInt(32) + 1; this.enableCommonExprPushdown = random.nextBoolean(); this.enableLocalExchange = random.nextBoolean(); + this.useSerialExchange = random.nextBoolean(); // This will cause be dead loop, disable it first // this.disableJoinReorder = random.nextBoolean(); this.enableCommonExpPushDownForInvertedIndex = random.nextBoolean(); @@ -4563,6 +4570,10 @@ public boolean isEnableCooldownReplicaAffinity() { return enableCooldownReplicaAffinity; } + public boolean isUseSerialExchange() { + return useSerialExchange && getEnableLocalExchange(); + } + public void setDisableInvertedIndexV1ForVaraint(boolean disableInvertedIndexV1ForVaraint) { this.disableInvertedIndexV1ForVaraint = disableInvertedIndexV1ForVaraint; } From 1ae91b89936e35d4ea329169cbf0dae3be75668a Mon Sep 17 00:00:00 2001 From: 924060929 Date: Thu, 28 Nov 2024 17:38:40 +0800 Subject: [PATCH 064/399] [fix](nereids) fix months_add/ months_sub/ years_add/years_sub compute wrong result because SimplifyArithmeticComparisonRule (#44725) fix months_add/ months_sub/ years_add/years_sub compute wrong result because SimplifyArithmeticComparisonRule introduced by #25180 --- .../rules/SimplifyArithmeticComparisonRule.java | 11 +++-------- .../rules/expression/SimplifyArithmeticRuleTest.java | 8 ++++---- .../datetime_functions/test_date_function.groovy | 11 +++++++++++ .../datetime_functions/test_date_function.groovy | 11 +++++++++++ 4 files changed, 29 insertions(+), 12 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifyArithmeticComparisonRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifyArithmeticComparisonRule.java index 6d18bc7b3807a6..863bfd18f458fd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifyArithmeticComparisonRule.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifyArithmeticComparisonRule.java @@ -32,14 +32,10 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.HoursSub; import org.apache.doris.nereids.trees.expressions.functions.scalar.MinutesAdd; import org.apache.doris.nereids.trees.expressions.functions.scalar.MinutesSub; -import org.apache.doris.nereids.trees.expressions.functions.scalar.MonthsAdd; -import org.apache.doris.nereids.trees.expressions.functions.scalar.MonthsSub; import org.apache.doris.nereids.trees.expressions.functions.scalar.SecondsAdd; import org.apache.doris.nereids.trees.expressions.functions.scalar.SecondsSub; import org.apache.doris.nereids.trees.expressions.functions.scalar.WeeksAdd; import org.apache.doris.nereids.trees.expressions.functions.scalar.WeeksSub; -import org.apache.doris.nereids.trees.expressions.functions.scalar.YearsAdd; -import org.apache.doris.nereids.trees.expressions.functions.scalar.YearsSub; import org.apache.doris.nereids.trees.expressions.literal.IntegerLiteral; import org.apache.doris.nereids.trees.expressions.literal.Literal; import org.apache.doris.nereids.util.TypeCoercionUtils; @@ -66,10 +62,9 @@ public class SimplifyArithmeticComparisonRule implements ExpressionPatternRuleFa .put(Add.class, Subtract.class) .put(Subtract.class, Add.class) .put(Divide.class, Multiply.class) - .put(YearsSub.class, YearsAdd.class) - .put(YearsAdd.class, YearsSub.class) - .put(MonthsSub.class, MonthsAdd.class) - .put(MonthsAdd.class, MonthsSub.class) + // ATTN: YearsAdd, MonthsAdd can not reverse + // for example, months_add(date '2024-01-31', 1) = date '2024-02-29' can not reverse to + // date '2024-01-31' = months_sub(date '2024-02-29', 1) .put(WeeksSub.class, WeeksAdd.class) .put(WeeksAdd.class, WeeksSub.class) .put(DaysSub.class, DaysAdd.class) diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/SimplifyArithmeticRuleTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/SimplifyArithmeticRuleTest.java index 55ce93da63d5aa..f23aefe5267c1a 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/SimplifyArithmeticRuleTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/SimplifyArithmeticRuleTest.java @@ -151,10 +151,10 @@ void testSimplifyDateTimeComparison() { FoldConstantRule.INSTANCE ) )); - assertRewriteAfterTypeCoercion("years_add(IA, 1) > '2021-01-01 00:00:00'", "(cast(IA as DATETIMEV2(0)) > '2020-01-01 00:00:00')"); - assertRewriteAfterTypeCoercion("years_sub(IA, 1) > '2021-01-01 00:00:00'", "(cast(IA as DATETIMEV2(0)) > '2022-01-01 00:00:00')"); - assertRewriteAfterTypeCoercion("months_add(IA, 1) > '2021-01-01 00:00:00'", "(cast(IA as DATETIMEV2(0)) > '2020-12-01 00:00:00')"); - assertRewriteAfterTypeCoercion("months_sub(IA, 1) > '2021-01-01 00:00:00'", "(cast(IA as DATETIMEV2(0)) > '2021-02-01 00:00:00')"); + assertRewriteAfterTypeCoercion("years_add(IA, 1) > '2021-01-01 00:00:00'", "(years_add(cast(IA as DATETIMEV2(0)), 1) > '2021-01-01 00:00:00')"); + assertRewriteAfterTypeCoercion("years_sub(IA, 1) > '2021-01-01 00:00:00'", "(years_sub(cast(IA as DATETIMEV2(0)), 1) > '2021-01-01 00:00:00')"); + assertRewriteAfterTypeCoercion("months_add(IA, 1) > '2021-01-01 00:00:00'", "(months_add(cast(IA as DATETIMEV2(0)), 1) > '2021-01-01 00:00:00')"); + assertRewriteAfterTypeCoercion("months_sub(IA, 1) > '2021-01-01 00:00:00'", "(months_sub(cast(IA as DATETIMEV2(0)), 1) > '2021-01-01 00:00:00')"); assertRewriteAfterTypeCoercion("weeks_add(IA, 1) > '2021-01-01 00:00:00'", "(cast(IA as DATETIMEV2(0)) > '2020-12-25 00:00:00')"); assertRewriteAfterTypeCoercion("weeks_sub(IA, 1) > '2021-01-01 00:00:00'", "(cast(IA as DATETIMEV2(0)) > '2021-01-08 00:00:00')"); assertRewriteAfterTypeCoercion("days_add(IA, 1) > '2021-01-01 00:00:00'", "(cast(IA as DATETIMEV2(0)) > '2020-12-31 00:00:00')"); diff --git a/regression-test/suites/nereids_p0/sql_functions/datetime_functions/test_date_function.groovy b/regression-test/suites/nereids_p0/sql_functions/datetime_functions/test_date_function.groovy index 5bde4423253744..bdd92166f7e686 100644 --- a/regression-test/suites/nereids_p0/sql_functions/datetime_functions/test_date_function.groovy +++ b/regression-test/suites/nereids_p0/sql_functions/datetime_functions/test_date_function.groovy @@ -720,4 +720,15 @@ suite("test_date_function") { qt_sql_time_value """ select cast(4562632 as time), hour(cast(4562632 as time)) , minute(cast(4562632 as time)) , second(cast(4562632 as time)); """ + + def test_simplify = { + test { + sql "select months_add(dt, 1) = date '2024-02-29' from (select date '2024-01-31' as dt)a" + result([[true]]) + } + test { + sql "select years_add(dt, 1) = date '2025-02-28' from (select date '2024-02-29' as dt)a" + result([[true]]) + } + }() } diff --git a/regression-test/suites/query_p0/sql_functions/datetime_functions/test_date_function.groovy b/regression-test/suites/query_p0/sql_functions/datetime_functions/test_date_function.groovy index 511f2bc35a67a7..8bd43dae685afc 100644 --- a/regression-test/suites/query_p0/sql_functions/datetime_functions/test_date_function.groovy +++ b/regression-test/suites/query_p0/sql_functions/datetime_functions/test_date_function.groovy @@ -818,4 +818,15 @@ suite("test_date_function") { qt_sql_varchar1 """ select dt, fmt, unix_timestamp(dt, fmt) as k1 from date_varchar order by k1,dt,fmt; """ qt_sql_varchar1 """ select dt, unix_timestamp(dt, "%Y-%m-%d") as k1 from date_varchar order by k1,dt,fmt; """ qt_sql_varchar1 """ select fmt, unix_timestamp("1990-12-12", fmt) as k1 from date_varchar order by k1,dt,fmt; """ + + def test_simplify = { + test { + sql "select months_add(dt, 1) = date '2024-02-29' from (select date '2024-01-31' as dt)a" + result([[true]]) + } + test { + sql "select years_add(dt, 1) = date '2025-02-28' from (select date '2024-02-29' as dt)a" + result([[true]]) + } + }() } From 51ad321b84cf211efd63415a1052e2f72a3415c5 Mon Sep 17 00:00:00 2001 From: zclllhhjj Date: Thu, 28 Nov 2024 18:24:16 +0800 Subject: [PATCH 065/399] [Enhancement](function) Print exact column name when execute non_nullable failed (#44679) ### What problem does this PR solve? Before it print the column type. now more precisely print the column name --- be/src/vec/functions/function_nullables.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/be/src/vec/functions/function_nullables.cpp b/be/src/vec/functions/function_nullables.cpp index 88bb2b38fab65c..91bce24f48fc8b 100644 --- a/be/src/vec/functions/function_nullables.cpp +++ b/be/src/vec/functions/function_nullables.cpp @@ -96,7 +96,7 @@ class FunctionNonNullable : public IFunction { if (col_null->has_null()) [[unlikely]] { return Status::InvalidArgument( "There's NULL value in column {} which is illegal for non_nullable", - data.column->get_name()); + data.name); } const ColumnPtr& nest_col = col_null->get_nested_column_ptr(); block.replace_by_position(result, nest_col->clone_resized(nest_col->size())); From 9a3622409966af5d5ab220ffc881df4ed0d64cf2 Mon Sep 17 00:00:00 2001 From: Pxl Date: Thu, 28 Nov 2024 22:12:25 +0800 Subject: [PATCH 066/399] [Chore](log) set some variable's fuzzy to true and display inited for bf func check (#44752) ### What problem does this PR solve? set some variable's fuzzy to true and display inited for bf func check --- be/src/exprs/bloom_filter_func.h | 14 +++++++------- .../java/org/apache/doris/qe/SessionVariable.java | 6 +++--- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/be/src/exprs/bloom_filter_func.h b/be/src/exprs/bloom_filter_func.h index 54ad7502810484..674a429581643b 100644 --- a/be/src/exprs/bloom_filter_func.h +++ b/be/src/exprs/bloom_filter_func.h @@ -157,19 +157,18 @@ class BloomFilterFuncBase : public RuntimeFilterFuncBase { return Status::InternalError("bloomfilter_func is nullptr"); } if (bloomfilter_func->_bloom_filter == nullptr) { - return Status::InternalError("bloomfilter_func->_bloom_filter is nullptr"); + return Status::InternalError( + "bloomfilter_func->_bloom_filter is nullptr, bloomfilter_func->inited: {}", + bloomfilter_func->_inited); } // If `_inited` is false, there is no memory allocated in bloom filter and this is the first // call for `merge` function. So we just reuse this bloom filter, and we don't need to // allocate memory again. if (!_inited) { - auto* other_func = static_cast(bloomfilter_func); if (_bloom_filter != nullptr) { - return Status::InternalError("_bloom_filter must is nullptr"); + return Status::InternalError("_bloom_filter must is nullptr, inited: {}", _inited); } - _bloom_filter = bloomfilter_func->_bloom_filter; - _bloom_filter_alloced = other_func->_bloom_filter_alloced; - _inited = true; + light_copy(bloomfilter_func); return Status::OK(); } auto* other_func = static_cast(bloomfilter_func); @@ -207,7 +206,8 @@ class BloomFilterFuncBase : public RuntimeFilterFuncBase { bool contain_null() const { if (!_bloom_filter) { - throw Exception(ErrorCode::INTERNAL_ERROR, "_bloom_filter is nullptr"); + throw Exception(ErrorCode::INTERNAL_ERROR, "_bloom_filter is nullptr, inited: {}", + _inited); } return _bloom_filter->contain_null(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java index 115614a41873a6..f977c3ccb05f17 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java @@ -1202,10 +1202,10 @@ public enum IgnoreSplitType { @VariableMgr.VarAttr(name = RUNTIME_FILTER_TYPE, fuzzy = true, needForward = true) private int runtimeFilterType = 12; - @VariableMgr.VarAttr(name = RUNTIME_FILTER_MAX_IN_NUM, needForward = true) + @VariableMgr.VarAttr(name = RUNTIME_FILTER_MAX_IN_NUM, needForward = true, fuzzy = true) private int runtimeFilterMaxInNum = 1024; - @VariableMgr.VarAttr(name = ENABLE_SYNC_RUNTIME_FILTER_SIZE, needForward = true) + @VariableMgr.VarAttr(name = ENABLE_SYNC_RUNTIME_FILTER_SIZE, needForward = true, fuzzy = true) private boolean enableSyncRuntimeFilterSize = true; @VariableMgr.VarAttr(name = ENABLE_PARALLEL_RESULT_SINK, needForward = true, fuzzy = true) @@ -1456,7 +1456,7 @@ public void setEnableLeftZigZag(boolean enableLeftZigZag) { @VariableMgr.VarAttr(name = BROADCAST_HASHTABLE_MEM_LIMIT_PERCENTAGE, needForward = true) private double broadcastHashtableMemLimitPercentage = 0.2; - @VariableMgr.VarAttr(name = ENABLE_RUNTIME_FILTER_PRUNE, needForward = true) + @VariableMgr.VarAttr(name = ENABLE_RUNTIME_FILTER_PRUNE, needForward = true, fuzzy = true) public boolean enableRuntimeFilterPrune = true; /** From 358ed8caac231f88f0ce40d484ac5c2fdc626f8b Mon Sep 17 00:00:00 2001 From: daidai Date: Thu, 28 Nov 2024 22:13:27 +0800 Subject: [PATCH 067/399] [feature](hive)support hive catalog read json table. (#43469) ### What problem does this PR solve? Problem Summary: Support reading json format hive table like: ```mysql mysql> show create table basic_json_table; CREATE TABLE `basic_json_table`( `id` int, `name` string, `age` tinyint, `salary` float, `is_active` boolean, `join_date` date, `last_login` timestamp, `height` double, `profile` binary, `rating` decimal(10,2)) ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe' ``` Behavior changed: To implement this feature, this pr modifies `new_json_reader`. Previously, `new_json_reader` could only insert data into columnString. In order to support inserting data into columns of other types, `DataTypeSerDe` is introduced to insert data into columns. To maintain compatibility with previous versions, changes to this pr are triggered only when reading hive json tables. Limitation of Use: 1. Currently, only query is supported, and writing is not supported. 2. Currently, only the `ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe';` scenario is supported. For some properties specified in `with serdeproperties`, Doris does not take effect. 3. Since Hive does not allow columns with the same name but different case when creating a table in Json format (including inside a Struct), we convert the field names in the Json data to lowercase when reading the Json data file, and then match according to the lowercase field names. For field names that are duplicated after being converted to lowercase in the data, the value of the last field is used (consistent with Hive behavior). example: ``` create table json_table( column int )ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe'; a.json: {"column":1,"COLumn",2,"COLUMN":3} {"column":10,"COLumn",20} {"column":100} in Hive : load a.json to table json_table in Doris query: --- 3 20 100 --- ``` Todo(in next pr): Merge `serde` and `json_reader` ,because they have logical conflicts. ### Release note Hive catalog support read json format table. --- .../data_types/serde/data_type_array_serde.h | 2 + .../data_types/serde/data_type_map_serde.h | 4 + .../serde/data_type_nullable_serde.h | 2 + be/src/vec/data_types/serde/data_type_serde.h | 12 +- .../data_types/serde/data_type_struct_serde.h | 2 + .../vec/exec/format/json/new_json_reader.cpp | 559 +++++++++++++----- be/src/vec/exec/format/json/new_json_reader.h | 32 +- be/src/vec/exec/scan/vfile_scanner.cpp | 4 +- .../create_preinstalled_scripts/run69.hql | 35 ++ .../create_preinstalled_scripts/run70.hql | 73 +++ .../create_preinstalled_scripts/run71.hql | 13 + .../json_all_complex_types/dt=dt1/000000_0 | 3 + .../json_all_complex_types/dt=dt2/000000_0 | 1 + .../json_all_complex_types/dt=dt3/000000_0 | 2 + .../json/json_load_data_table/1 | 13 + .../json/json_nested_complex_table/1 | 2 + .../json/json_nested_complex_table/2 | 1 + .../json/json_nested_complex_table/modify_2 | 2 + .../hive/HiveMetaStoreClientHelper.java | 3 + .../datasource/hive/source/HiveScanNode.java | 41 +- .../hive/hive_json_basic_test.out | 115 ++++ .../hive/hive_json_basic_test.groovy | 71 +++ 22 files changed, 832 insertions(+), 160 deletions(-) create mode 100644 docker/thirdparties/docker-compose/hive/scripts/create_preinstalled_scripts/run69.hql create mode 100644 docker/thirdparties/docker-compose/hive/scripts/create_preinstalled_scripts/run70.hql create mode 100644 docker/thirdparties/docker-compose/hive/scripts/create_preinstalled_scripts/run71.hql create mode 100644 docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/json/json_all_complex_types/dt=dt1/000000_0 create mode 100644 docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/json/json_all_complex_types/dt=dt2/000000_0 create mode 100644 docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/json/json_all_complex_types/dt=dt3/000000_0 create mode 100644 docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/json/json_load_data_table/1 create mode 100644 docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/json/json_nested_complex_table/1 create mode 100644 docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/json/json_nested_complex_table/2 create mode 100644 docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/json/json_nested_complex_table/modify_2 create mode 100644 regression-test/data/external_table_p0/hive/hive_json_basic_test.out create mode 100644 regression-test/suites/external_table_p0/hive/hive_json_basic_test.groovy diff --git a/be/src/vec/data_types/serde/data_type_array_serde.h b/be/src/vec/data_types/serde/data_type_array_serde.h index 5b15f48f502b81..cdd2115576030e 100644 --- a/be/src/vec/data_types/serde/data_type_array_serde.h +++ b/be/src/vec/data_types/serde/data_type_array_serde.h @@ -101,6 +101,8 @@ class DataTypeArraySerDe : public DataTypeSerDe { nested_serde->set_return_object_as_string(value); } + virtual DataTypeSerDeSPtrs get_nested_serdes() const override { return {nested_serde}; } + private: template Status _write_column_to_mysql(const IColumn& column, MysqlRowBuffer& result, diff --git a/be/src/vec/data_types/serde/data_type_map_serde.h b/be/src/vec/data_types/serde/data_type_map_serde.h index 599533eb4d0951..51c11300d44c01 100644 --- a/be/src/vec/data_types/serde/data_type_map_serde.h +++ b/be/src/vec/data_types/serde/data_type_map_serde.h @@ -96,6 +96,10 @@ class DataTypeMapSerDe : public DataTypeSerDe { value_serde->set_return_object_as_string(value); } + virtual DataTypeSerDeSPtrs get_nested_serdes() const override { + return {key_serde, value_serde}; + } + private: template Status _write_column_to_mysql(const IColumn& column, MysqlRowBuffer& result, diff --git a/be/src/vec/data_types/serde/data_type_nullable_serde.h b/be/src/vec/data_types/serde/data_type_nullable_serde.h index 6051c7f722d62a..159db890540dbc 100644 --- a/be/src/vec/data_types/serde/data_type_nullable_serde.h +++ b/be/src/vec/data_types/serde/data_type_nullable_serde.h @@ -99,6 +99,8 @@ class DataTypeNullableSerDe : public DataTypeSerDe { int64_t row_num) const override; Status read_one_cell_from_json(IColumn& column, const rapidjson::Value& result) const override; + virtual DataTypeSerDeSPtrs get_nested_serdes() const override { return {nested_serde}; } + private: template Status _write_column_to_mysql(const IColumn& column, MysqlRowBuffer& result, diff --git a/be/src/vec/data_types/serde/data_type_serde.h b/be/src/vec/data_types/serde/data_type_serde.h index f0e9eb27961439..bec0fabfcca9c3 100644 --- a/be/src/vec/data_types/serde/data_type_serde.h +++ b/be/src/vec/data_types/serde/data_type_serde.h @@ -100,6 +100,10 @@ class IColumn; class Arena; class IDataType; +class DataTypeSerDe; +using DataTypeSerDeSPtr = std::shared_ptr; +using DataTypeSerDeSPtrs = std::vector; + // Deserialize means read from different file format or memory format, // for example read from arrow, read from parquet. // Serialize means write the column cell or the total column into another @@ -337,6 +341,11 @@ class DataTypeSerDe { Arena& mem_pool, int64_t row_num) const; virtual Status read_one_cell_from_json(IColumn& column, const rapidjson::Value& result) const; + virtual DataTypeSerDeSPtrs get_nested_serdes() const { + throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, + "Method get_nested_serdes is not supported for this serde"); + } + protected: bool _return_object_as_string = false; // This parameter indicates what level the serde belongs to and is mainly used for complex types @@ -379,9 +388,6 @@ inline void checkArrowStatus(const arrow::Status& status, const std::string& col } } -using DataTypeSerDeSPtr = std::shared_ptr; -using DataTypeSerDeSPtrs = std::vector; - DataTypeSerDeSPtrs create_data_type_serdes( const std::vector>& types); DataTypeSerDeSPtrs create_data_type_serdes(const std::vector& slots); diff --git a/be/src/vec/data_types/serde/data_type_struct_serde.h b/be/src/vec/data_types/serde/data_type_struct_serde.h index 2ec871838d7ed7..fc77e26b71f882 100644 --- a/be/src/vec/data_types/serde/data_type_struct_serde.h +++ b/be/src/vec/data_types/serde/data_type_struct_serde.h @@ -172,6 +172,8 @@ class DataTypeStructSerDe : public DataTypeSerDe { } } + virtual DataTypeSerDeSPtrs get_nested_serdes() const override { return elem_serdes_ptrs; } + private: std::optional try_get_position_by_name(const String& name) const; diff --git a/be/src/vec/exec/format/json/new_json_reader.cpp b/be/src/vec/exec/format/json/new_json_reader.cpp index d726de3d99817e..d79e86520741cd 100644 --- a/be/src/vec/exec/format/json/new_json_reader.cpp +++ b/be/src/vec/exec/format/json/new_json_reader.cpp @@ -54,8 +54,11 @@ #include "util/slice.h" #include "util/uid_util.h" #include "vec/columns/column.h" +#include "vec/columns/column_array.h" +#include "vec/columns/column_map.h" #include "vec/columns/column_nullable.h" #include "vec/columns/column_string.h" +#include "vec/columns/column_struct.h" #include "vec/common/assert_cast.h" #include "vec/common/typeid_cast.h" #include "vec/core/block.h" @@ -164,10 +167,18 @@ void NewJsonReader::_init_file_description() { } Status NewJsonReader::init_reader( - const std::unordered_map& col_default_value_ctx) { + const std::unordered_map& col_default_value_ctx, + bool is_load) { + _is_load = is_load; + // generate _col_default_value_map RETURN_IF_ERROR(_get_column_default_value(_file_slot_descs, col_default_value_ctx)); + //use serde insert data to column. + for (auto* slot_desc : _file_slot_descs) { + _serdes.emplace_back(slot_desc->get_data_type_ptr()->get_serde()); + } + // create decompressor. // _decompressor may be nullptr if this is not a compressed file RETURN_IF_ERROR(Decompressor::create_decompressor(_file_compress_type, &_decompressor)); @@ -390,6 +401,9 @@ Status NewJsonReader::_get_range_params() { if (_params.file_attributes.__isset.fuzzy_parse) { _fuzzy_parse = _params.file_attributes.fuzzy_parse; } + if (_range.table_format_params.table_format_type == "hive") { + _is_hive_table = true; + } return Status::OK(); } @@ -477,8 +491,8 @@ Status NewJsonReader::_vhandle_simple_json(RuntimeState* /*state*/, Block& block bool valid = false; if (_next_row >= _total_rows) { // parse json and generic document Status st = _parse_json(is_empty_row, eof); - if (st.is()) { - continue; // continue to read next + if (_is_load && st.is()) { + continue; // continue to read next (for load, after this , already append error to file.) } RETURN_IF_ERROR(st); if (*is_empty_row) { @@ -761,7 +775,20 @@ Status NewJsonReader::_set_column_value(rapidjson::Value& objectValue, Block& bl _append_empty_skip_bitmap_value(block, cur_row_count); } - for (auto* slot_desc : slot_descs) { + if (_is_hive_table) { + //don't like _fuzzy_parse,each line read in must modify name_map once. + + for (int i = 0; i < objectValue.MemberCount(); ++i) { + auto it = objectValue.MemberBegin() + i; + std::string field_name(it->name.GetString(), it->name.GetStringLength()); + std::transform(field_name.begin(), field_name.end(), field_name.begin(), ::tolower); + + //Use the last value with the same name. + _name_map.emplace(field_name, i); + } + } + for (size_t slot_idx = 0; slot_idx < slot_descs.size(); ++slot_idx) { + auto* slot_desc = slot_descs[slot_idx]; if (!slot_desc->is_materialized()) { continue; } @@ -773,7 +800,7 @@ Status NewJsonReader::_set_column_value(rapidjson::Value& objectValue, Block& bl auto* column_ptr = block.get_by_position(dest_index).column->assume_mutable().get(); rapidjson::Value::ConstMemberIterator it = objectValue.MemberEnd(); - if (_fuzzy_parse) { + if (_fuzzy_parse || _is_hive_table) { auto idx_it = _name_map.find(slot_desc->col_name()); if (idx_it != _name_map.end() && idx_it->second < objectValue.MemberCount()) { it = objectValue.MemberBegin() + idx_it->second; @@ -785,7 +812,8 @@ Status NewJsonReader::_set_column_value(rapidjson::Value& objectValue, Block& bl if (it != objectValue.MemberEnd()) { const rapidjson::Value& value = it->value; - RETURN_IF_ERROR(_write_data_to_column(&value, slot_desc, column_ptr, valid)); + RETURN_IF_ERROR(_write_data_to_column(&value, slot_desc->type(), column_ptr, + slot_desc->col_name(), _serdes[slot_idx], valid)); if (!(*valid)) { return Status::OK(); } @@ -814,14 +842,15 @@ Status NewJsonReader::_set_column_value(rapidjson::Value& objectValue, Block& bl column_ptr->insert_default(); } else { // not found, filling with default value - RETURN_IF_ERROR(_fill_missing_column(slot_desc, column_ptr, valid)); + RETURN_IF_ERROR( + _fill_missing_column(slot_desc, _serdes[slot_idx], column_ptr, valid)); if (!(*valid)) { return Status::OK(); } } } } - if (!has_valid_value) { + if (!has_valid_value && _is_load) { // there is no valid value in json line but has filled with default value before // so remove this line in block string col_names; @@ -845,79 +874,188 @@ Status NewJsonReader::_set_column_value(rapidjson::Value& objectValue, Block& bl } Status NewJsonReader::_write_data_to_column(rapidjson::Value::ConstValueIterator value, - SlotDescriptor* slot_desc, IColumn* column_ptr, + const TypeDescriptor& type_desc, + vectorized::IColumn* column_ptr, + const std::string& column_name, DataTypeSerDeSPtr serde, bool* valid) { - const char* str_value = nullptr; - char tmp_buf[128] = {0}; - int32_t wbytes = 0; - std::string json_str; - ColumnNullable* nullable_column = nullptr; - if (slot_desc->is_nullable()) { + vectorized::IColumn* data_column_ptr = column_ptr; + DataTypeSerDeSPtr data_serde = serde; + + bool value_is_null = (value == nullptr) || (value->GetType() == rapidjson::Type::kNullType); + + if (column_ptr->is_nullable()) { nullable_column = reinterpret_cast(column_ptr); - // kNullType will put 1 into the Null map, so there is no need to push 0 for kNullType. - if (value->GetType() != rapidjson::Type::kNullType) { + data_column_ptr = nullable_column->get_nested_column().get_ptr(); + data_serde = serde->get_nested_serdes()[0]; + + if (value_is_null) { + nullable_column->insert_default(); + *valid = true; + return Status::OK(); + } else { nullable_column->get_null_map_data().push_back(0); + } + + } else if (value_is_null) [[unlikely]] { + if (_is_load) { + RETURN_IF_ERROR(_append_error_msg( + *value, "Json value is null, but the column `{}` is not nullable.", column_name, + valid)); + return Status::OK(); + } else { - nullable_column->insert_default(); + return Status::DataQualityError( + "Json value is null, but the column `{}` is not nullable.", column_name); } - column_ptr = &nullable_column->get_nested_column(); } - switch (value->GetType()) { - case rapidjson::Type::kStringType: - str_value = value->GetString(); - wbytes = value->GetStringLength(); - break; - case rapidjson::Type::kNumberType: - if (value->IsUint()) { - wbytes = snprintf(tmp_buf, sizeof(tmp_buf), "%u", value->GetUint()); - } else if (value->IsInt()) { - wbytes = snprintf(tmp_buf, sizeof(tmp_buf), "%d", value->GetInt()); - } else if (value->IsUint64()) { - wbytes = snprintf(tmp_buf, sizeof(tmp_buf), "%" PRIu64, value->GetUint64()); - } else if (value->IsInt64()) { - wbytes = snprintf(tmp_buf, sizeof(tmp_buf), "%" PRId64, value->GetInt64()); - } else if (value->IsFloat() || value->IsDouble()) { - auto* end = fmt::format_to(tmp_buf, "{}", value->GetDouble()); - wbytes = end - tmp_buf; + if (_is_load || !type_desc.is_complex_type()) { + if (value->IsString()) { + Slice slice {value->GetString(), value->GetStringLength()}; + RETURN_IF_ERROR(data_serde->deserialize_one_cell_from_json(*data_column_ptr, slice, + _serde_options)); + } else { - return Status::InternalError("It should not here."); + // We can `switch (value->GetType()) case: kNumberType`. + // Note that `if (value->IsInt())`, but column is FloatColumn. + // Or for any type, use `NewJsonReader::_print_json_value(*value)`. + + const char* str_value = nullptr; + char tmp_buf[128] = {0}; + size_t wbytes = 0; + std::string json_str; + + switch (value->GetType()) { + case rapidjson::Type::kStringType: + str_value = value->GetString(); + wbytes = value->GetStringLength(); + break; + case rapidjson::Type::kNumberType: + if (value->IsUint()) { + wbytes = snprintf(tmp_buf, sizeof(tmp_buf), "%u", value->GetUint()); + } else if (value->IsInt()) { + wbytes = snprintf(tmp_buf, sizeof(tmp_buf), "%d", value->GetInt()); + } else if (value->IsUint64()) { + wbytes = snprintf(tmp_buf, sizeof(tmp_buf), "%" PRIu64, value->GetUint64()); + } else if (value->IsInt64()) { + wbytes = snprintf(tmp_buf, sizeof(tmp_buf), "%" PRId64, value->GetInt64()); + } else if (value->IsFloat() || value->IsDouble()) { + auto* end = fmt::format_to(tmp_buf, "{}", value->GetDouble()); + wbytes = end - tmp_buf; + } else { + return Status::InternalError("It should not here."); + } + str_value = tmp_buf; + break; + case rapidjson::Type::kFalseType: + wbytes = 1; + str_value = (char*)"0"; + break; + case rapidjson::Type::kTrueType: + wbytes = 1; + str_value = (char*)"1"; + break; + default: + // for other type, we convert it to string to save + json_str = NewJsonReader::_print_json_value(*value); + wbytes = json_str.size(); + str_value = json_str.c_str(); + break; + } + Slice slice {str_value, wbytes}; + RETURN_IF_ERROR(data_serde->deserialize_one_cell_from_json(*data_column_ptr, slice, + _serde_options)); } - str_value = tmp_buf; - break; - case rapidjson::Type::kFalseType: - wbytes = 1; - str_value = (char*)"0"; - break; - case rapidjson::Type::kTrueType: - wbytes = 1; - str_value = (char*)"1"; - break; - case rapidjson::Type::kNullType: - if (!slot_desc->is_nullable()) { - RETURN_IF_ERROR(_append_error_msg( - *value, "Json value is null, but the column `{}` is not nullable.", - slot_desc->col_name(), valid)); - return Status::OK(); + } else if (type_desc.type == TYPE_STRUCT) { + if (!value->IsObject()) [[unlikely]] { + return Status::DataQualityError( + "Json value isn't object, but the column `{}` is struct.", column_name); } - // return immediately to prevent from repeatedly insert_data - *valid = true; - return Status::OK(); - default: - // for other type like array or object. we convert it to string to save - json_str = NewJsonReader::_print_json_value(*value); - wbytes = json_str.size(); - str_value = json_str.c_str(); - break; - } + auto sub_col_size = type_desc.children.size(); + const auto& struct_value = value->GetObject(); - // TODO: if the vexpr can support another 'slot_desc type' than 'TYPE_VARCHAR', - // we need use a function to support these types to insert data in columns. - DCHECK(slot_desc->type().type == TYPE_VARCHAR || slot_desc->type().type == TYPE_STRING) - << slot_desc->type().type << ", query id: " << print_id(_state->query_id()); - assert_cast(column_ptr)->insert_data(str_value, wbytes); + auto sub_serdes = data_serde->get_nested_serdes(); + auto struct_column_ptr = assert_cast(data_column_ptr); + + std::map sub_col_name_to_idx; + for (size_t sub_col_idx = 0; sub_col_idx < sub_col_size; sub_col_idx++) { + sub_col_name_to_idx.emplace(type_desc.field_names[sub_col_idx], sub_col_idx); + } + + std::vector sub_values(sub_col_size, nullptr); + for (const auto& sub : struct_value) { + if (!sub.name.IsString()) [[unlikely]] { + return Status::DataQualityError( + "Json file struct column `{}` subfield name isn't a String", column_name); + } + + auto sub_key_char = sub.name.GetString(); + auto sub_key_length = sub.name.GetStringLength(); + + std::string sub_key(sub_key_char, sub_key_length); + std::transform(sub_key.begin(), sub_key.end(), sub_key.begin(), ::tolower); + + if (sub_col_name_to_idx.find(sub_key) == sub_col_name_to_idx.end()) [[unlikely]] { + continue; + } + size_t sub_column_idx = sub_col_name_to_idx[sub_key]; + sub_values[sub_column_idx] = &sub.value; + } + + for (size_t sub_col_idx = 0; sub_col_idx < sub_col_size; sub_col_idx++) { + auto sub_value = sub_values[sub_col_idx]; + + const auto& sub_col_type = type_desc.children[sub_col_idx]; + + RETURN_IF_ERROR(_write_data_to_column( + sub_value, sub_col_type, struct_column_ptr->get_column(sub_col_idx).get_ptr(), + column_name + "." + type_desc.field_names[sub_col_idx], sub_serdes[sub_col_idx], + valid)); + } + } else if (type_desc.type == TYPE_MAP) { + if (!value->IsObject()) [[unlikely]] { + return Status::DataQualityError("Json value isn't object, but the column `{}` is map.", + column_name); + } + const auto& object_value = value->GetObject(); + auto sub_serdes = data_serde->get_nested_serdes(); + auto map_column_ptr = assert_cast(data_column_ptr); + + for (const auto& member_value : object_value) { + RETURN_IF_ERROR(_write_data_to_column( + &member_value.name, type_desc.children[0], + map_column_ptr->get_keys_ptr()->assume_mutable()->get_ptr(), + column_name + ".key", sub_serdes[0], valid)); + + RETURN_IF_ERROR(_write_data_to_column( + &member_value.value, type_desc.children[1], + map_column_ptr->get_values_ptr()->assume_mutable()->get_ptr(), + column_name + ".value", sub_serdes[1], valid)); + } + + auto& offsets = map_column_ptr->get_offsets(); + offsets.emplace_back(offsets.back() + object_value.MemberCount()); + } else if (type_desc.type == TYPE_ARRAY) { + if (!value->IsArray()) [[unlikely]] { + return Status::DataQualityError("Json value isn't array, but the column `{}` is array.", + column_name); + } + const auto& array_value = value->GetArray(); + auto sub_serdes = data_serde->get_nested_serdes(); + auto array_column_ptr = assert_cast(data_column_ptr); + + for (const auto& sub_value : array_value) { + RETURN_IF_ERROR(_write_data_to_column(&sub_value, type_desc.children[0], + array_column_ptr->get_data().get_ptr(), + column_name + ".element", sub_serdes[0], valid)); + } + auto& offsets = array_column_ptr->get_offsets(); + offsets.emplace_back(offsets.back() + array_value.Size()); + } else { + return Status::InternalError("Not support load to complex column."); + } *valid = true; return Status::OK(); @@ -949,20 +1087,21 @@ Status NewJsonReader::_write_columns_by_jsonpath(rapidjson::Value& objectValue, // if json_values' size > 1, it means we just match an array, not a wrapped one, so no need to unwrap. json_values = &((*json_values)[0]); } - RETURN_IF_ERROR(_write_data_to_column(json_values, slot_descs[i], column_ptr, valid)); + RETURN_IF_ERROR(_write_data_to_column(json_values, slot_descs[i]->type(), column_ptr, + slot_descs[i]->col_name(), _serdes[i], valid)); if (!(*valid)) { return Status::OK(); } has_valid_value = true; } else { // not found, filling with default value - RETURN_IF_ERROR(_fill_missing_column(slot_desc, column_ptr, valid)); + RETURN_IF_ERROR(_fill_missing_column(slot_desc, _serdes[i], column_ptr, valid)); if (!(*valid)) { return Status::OK(); } } } - if (!has_valid_value) { + if (!has_valid_value && _is_load) { // there is no valid value in json line but has filled with default value before // so remove this line in block for (int i = 0; i < block.columns(); ++i) { @@ -1112,7 +1251,7 @@ Status NewJsonReader::_simdjson_handle_simple_json(RuntimeState* /*state*/, Bloc // step2: get json value by json doc Status st = _get_json_value(&size, eof, &error, is_empty_row); - if (st.is()) { + if (_is_load && st.is()) { return Status::OK(); } RETURN_IF_ERROR(st); @@ -1386,28 +1525,42 @@ Status NewJsonReader::_simdjson_set_column_value(simdjson::ondemand::object* val for (auto field : *value) { std::string_view key = field.unescaped_key(); StringRef name_ref(key.data(), key.size()); + std::string key_string; + if (_is_hive_table) { + key_string = name_ref.to_string(); + std::transform(key_string.begin(), key_string.end(), key_string.begin(), ::tolower); + name_ref = StringRef(key_string); + } const size_t column_index = _column_index(name_ref, key_index++); if (UNLIKELY(ssize_t(column_index) < 0)) { // This key is not exist in slot desc, just ignore continue; } - if (_seen_columns[column_index]) { - continue; - } if (column_index == skip_bitmap_col_idx) { continue; } + if (_seen_columns[column_index]) { + if (_is_hive_table) { + //Since value can only be traversed once, + // we can only insert the original value first, then delete it, and then reinsert the new value + block.get_by_position(column_index).column->assume_mutable()->pop_back(1); + } else { + continue; + } + } simdjson::ondemand::value val = field.value(); auto* column_ptr = block.get_by_position(column_index).column->assume_mutable().get(); - RETURN_IF_ERROR( - _simdjson_write_data_to_column(val, slot_descs[column_index], column_ptr, valid)); + RETURN_IF_ERROR(_simdjson_write_data_to_column( + val, slot_descs[column_index]->type(), column_ptr, + slot_descs[column_index]->col_name(), _serdes[column_index], valid)); if (!(*valid)) { return Status::OK(); } _seen_columns[column_index] = true; has_valid_value = true; } - if (!has_valid_value) { + + if (!has_valid_value && _is_load) { string col_names; for (auto* slot_desc : slot_descs) { col_names.append(slot_desc->col_name() + ", "); @@ -1472,7 +1625,7 @@ Status NewJsonReader::_simdjson_set_column_value(simdjson::ondemand::object* val _process_skip_bitmap_mark(slot_desc, column_ptr, block, cur_row_count, valid); column_ptr->insert_default(); } else { - RETURN_IF_ERROR(_fill_missing_column(slot_desc, column_ptr, valid)); + RETURN_IF_ERROR(_fill_missing_column(slot_desc, _serdes[i], column_ptr, valid)); if (!(*valid)) { return Status::OK(); } @@ -1482,12 +1635,6 @@ Status NewJsonReader::_simdjson_set_column_value(simdjson::ondemand::object* val DCHECK(column_ptr->size() == cur_row_count + 1); } -#ifndef NDEBUG - // Check all columns rows matched - for (size_t i = 0; i < block.columns(); ++i) { - DCHECK_EQ(block.get_by_position(i).column->size(), cur_row_count + 1); - } -#endif // There is at least one valid value here DCHECK(nullcount < block.columns()); *valid = true; @@ -1495,54 +1642,180 @@ Status NewJsonReader::_simdjson_set_column_value(simdjson::ondemand::object* val } Status NewJsonReader::_simdjson_write_data_to_column(simdjson::ondemand::value& value, - SlotDescriptor* slot_desc, IColumn* column, - bool* valid) { - // write + const TypeDescriptor& type_desc, + vectorized::IColumn* column_ptr, + const std::string& column_name, + DataTypeSerDeSPtr serde, bool* valid) { ColumnNullable* nullable_column = nullptr; - IColumn* column_ptr = nullptr; - if (slot_desc->is_nullable()) { - nullable_column = assert_cast(column); - column_ptr = &nullable_column->get_nested_column(); - } - // TODO: if the vexpr can support another 'slot_desc type' than 'TYPE_VARCHAR', - // we need use a function to support these types to insert data in columns. - auto* column_string = assert_cast(column_ptr); - switch (value.type()) { - case simdjson::ondemand::json_type::null: { - if (column->is_nullable()) { - // insert_default already push 1 to null_map - nullable_column->insert_default(); + vectorized::IColumn* data_column_ptr = column_ptr; + DataTypeSerDeSPtr data_serde = serde; + + if (column_ptr->is_nullable()) { + nullable_column = reinterpret_cast(column_ptr); + + data_column_ptr = nullable_column->get_nested_column().get_ptr(); + data_serde = serde->get_nested_serdes()[0]; + + // kNullType will put 1 into the Null map, so there is no need to push 0 for kNullType. + if (value.type() != simdjson::ondemand::json_type::null) { + nullable_column->get_null_map_data().push_back(0); } else { + nullable_column->insert_default(); + *valid = true; + return Status::OK(); + } + } else if (value.type() == simdjson::ondemand::json_type::null) [[unlikely]] { + if (_is_load) { RETURN_IF_ERROR(_append_error_msg( nullptr, "Json value is null, but the column `{}` is not nullable.", - slot_desc->col_name(), valid)); + column_name, valid)); return Status::OK(); - } - break; - } - case simdjson::ondemand::json_type::boolean: { - nullable_column->get_null_map_data().push_back(0); - if (value.get_bool()) { - column_string->insert_data("1", 1); } else { - column_string->insert_data("0", 1); + return Status::DataQualityError( + "Json value is null, but the column `{}` is not nullable.", column_name); } - break; } - default: { + + if (_is_load || !type_desc.is_complex_type()) { if (value.type() == simdjson::ondemand::json_type::string) { - auto* unescape_buffer = - reinterpret_cast(_simdjson_ondemand_unscape_padding_buffer.data()); - std::string_view unescaped_value = - _ondemand_json_parser->unescape(value.get_raw_json_string(), unescape_buffer); - nullable_column->get_null_map_data().push_back(0); - column_string->insert_data(unescaped_value.data(), unescaped_value.length()); - break; + std::string_view value_string = value.get_string(); + Slice slice {value_string.data(), value_string.size()}; + RETURN_IF_ERROR(data_serde->deserialize_one_cell_from_json(*data_column_ptr, slice, + _serde_options)); + + } else { + // Maybe we can `switch (value->GetType()) case: kNumberType`. + // Note that `if (value->IsInt())`, but column is FloatColumn. + std::string_view json_str = simdjson::to_json_string(value); + Slice slice {json_str.data(), json_str.size()}; + RETURN_IF_ERROR(data_serde->deserialize_one_cell_from_json(*data_column_ptr, slice, + _serde_options)); + } + } else if (type_desc.type == TYPE_STRUCT) { + if (value.type() != simdjson::ondemand::json_type::object) [[unlikely]] { + return Status::DataQualityError( + "Json value isn't object, but the column `{}` is struct.", column_name); + } + + auto sub_col_size = type_desc.children.size(); + simdjson::ondemand::object struct_value = value.get_object(); + auto sub_serdes = data_serde->get_nested_serdes(); + auto struct_column_ptr = assert_cast(data_column_ptr); + + std::map sub_col_name_to_idx; + for (size_t sub_col_idx = 0; sub_col_idx < sub_col_size; sub_col_idx++) { + sub_col_name_to_idx.emplace(type_desc.field_names[sub_col_idx], sub_col_idx); + } + vector has_value(sub_col_size, false); + for (simdjson::ondemand::field sub : struct_value) { + std::string_view sub_key_view = sub.unescaped_key(); + std::string sub_key(sub_key_view.data(), sub_key_view.length()); + std::transform(sub_key.begin(), sub_key.end(), sub_key.begin(), ::tolower); + + if (sub_col_name_to_idx.find(sub_key) == sub_col_name_to_idx.end()) [[unlikely]] { + continue; + } + size_t sub_column_idx = sub_col_name_to_idx[sub_key]; + auto sub_column_ptr = struct_column_ptr->get_column(sub_column_idx).get_ptr(); + + if (has_value[sub_column_idx]) [[unlikely]] { + // Since struct_value can only be traversed once, we can only insert + // the original value first, then delete it, and then reinsert the new value. + sub_column_ptr->pop_back(1); + } + has_value[sub_column_idx] = true; + + const auto& sub_col_type = type_desc.children[sub_column_idx]; + RETURN_IF_ERROR(_simdjson_write_data_to_column( + sub.value(), sub_col_type, sub_column_ptr, column_name + "." + sub_key, + sub_serdes[sub_column_idx], valid)); } - auto value_str = simdjson::to_json_string(value).value(); - nullable_column->get_null_map_data().push_back(0); - column_string->insert_data(value_str.data(), value_str.length()); - } + + //fill missing subcolumn + for (size_t sub_col_idx = 0; sub_col_idx < sub_col_size; sub_col_idx++) { + if (has_value[sub_col_idx] == true) { + continue; + } + + auto sub_column_ptr = struct_column_ptr->get_column(sub_col_idx).get_ptr(); + if (sub_column_ptr->is_nullable()) { + sub_column_ptr->insert_default(); + continue; + } else [[unlikely]] { + return Status::DataQualityError( + "Json file structColumn miss field {} and this column isn't nullable.", + column_name + "." + type_desc.field_names[sub_col_idx]); + } + } + } else if (type_desc.type == TYPE_MAP) { + if (value.type() != simdjson::ondemand::json_type::object) [[unlikely]] { + return Status::DataQualityError("Json value isn't object, but the column `{}` is map.", + column_name); + } + simdjson::ondemand::object object_value = value.get_object(); + + auto sub_serdes = data_serde->get_nested_serdes(); + auto map_column_ptr = assert_cast(data_column_ptr); + + size_t field_count = 0; + for (simdjson::ondemand::field member_value : object_value) { + auto f = [](std::string_view key_view, const TypeDescriptor& type_desc, + vectorized::IColumn* column_ptr, DataTypeSerDeSPtr serde, + vectorized::DataTypeSerDe::FormatOptions serde_options, bool* valid) { + auto data_column_ptr = column_ptr; + auto data_serde = serde; + if (column_ptr->is_nullable()) { + auto nullable_column = static_cast(column_ptr); + + nullable_column->get_null_map_data().push_back(0); + data_column_ptr = nullable_column->get_nested_column().get_ptr(); + data_serde = serde->get_nested_serdes()[0]; + } + Slice slice(key_view.data(), key_view.length()); + + RETURN_IF_ERROR(data_serde->deserialize_one_cell_from_json(*data_column_ptr, slice, + serde_options)); + return Status::OK(); + }; + + RETURN_IF_ERROR(f(member_value.unescaped_key(), type_desc.children[0], + map_column_ptr->get_keys_ptr()->assume_mutable()->get_ptr(), + sub_serdes[0], _serde_options, valid)); + + simdjson::ondemand::value field_value = member_value.value(); + RETURN_IF_ERROR(_simdjson_write_data_to_column( + field_value, type_desc.children[1], + map_column_ptr->get_values_ptr()->assume_mutable()->get_ptr(), + column_name + ".value", sub_serdes[1], valid)); + field_count++; + } + + auto& offsets = map_column_ptr->get_offsets(); + offsets.emplace_back(offsets.back() + field_count); + + } else if (type_desc.type == TYPE_ARRAY) { + if (value.type() != simdjson::ondemand::json_type::array) [[unlikely]] { + return Status::DataQualityError("Json value isn't array, but the column `{}` is array.", + column_name); + } + + simdjson::ondemand::array array_value = value.get_array(); + + auto sub_serdes = data_serde->get_nested_serdes(); + auto array_column_ptr = assert_cast(data_column_ptr); + + int field_count = 0; + for (simdjson::ondemand::value sub_value : array_value) { + RETURN_IF_ERROR(_simdjson_write_data_to_column( + sub_value, type_desc.children[0], array_column_ptr->get_data().get_ptr(), + column_name + ".element", sub_serdes[0], valid)); + field_count++; + } + auto& offsets = array_column_ptr->get_offsets(); + offsets.emplace_back(offsets.back() + field_count); + + } else { + return Status::InternalError("Not support load to complex column."); } *valid = true; return Status::OK(); @@ -1750,13 +2023,14 @@ Status NewJsonReader::_simdjson_write_columns_by_jsonpath( has_valid_value = true; } else if (i >= _parsed_jsonpaths.size() || st.is()) { // not match in jsondata, filling with default value - RETURN_IF_ERROR(_fill_missing_column(slot_desc, column_ptr, valid)); + RETURN_IF_ERROR(_fill_missing_column(slot_desc, _serdes[i], column_ptr, valid)); if (!(*valid)) { return Status::OK(); } } else { - RETURN_IF_ERROR( - _simdjson_write_data_to_column(json_value, slot_desc, column_ptr, valid)); + RETURN_IF_ERROR(_simdjson_write_data_to_column(json_value, slot_desc->type(), + column_ptr, slot_desc->col_name(), + _serdes[i], valid)); if (!(*valid)) { return Status::OK(); } @@ -1814,25 +2088,30 @@ Status NewJsonReader::_get_column_default_value( return Status::OK(); } -Status NewJsonReader::_fill_missing_column(SlotDescriptor* slot_desc, IColumn* column_ptr, - bool* valid) { - if (slot_desc->is_nullable()) { - auto* nullable_column = reinterpret_cast(column_ptr); - column_ptr = &nullable_column->get_nested_column(); - auto col_value = _col_default_value_map.find(slot_desc->col_name()); - if (col_value == _col_default_value_map.end()) { +Status NewJsonReader::_fill_missing_column(SlotDescriptor* slot_desc, DataTypeSerDeSPtr serde, + IColumn* column_ptr, bool* valid) { + auto col_value = _col_default_value_map.find(slot_desc->col_name()); + if (col_value == _col_default_value_map.end()) { + if (slot_desc->is_nullable()) { + auto* nullable_column = static_cast(column_ptr); nullable_column->insert_default(); } else { - const std::string& v_str = col_value->second; - nullable_column->get_null_map_data().push_back(0); - assert_cast(column_ptr)->insert_data(v_str.c_str(), v_str.size()); + if (_is_load) { + RETURN_IF_ERROR(_append_error_msg( + nullptr, "The column `{}` is not nullable, but it's not found in jsondata.", + slot_desc->col_name(), valid)); + } else { + return Status::DataQualityError( + "The column `{}` is not nullable, but it's not found in jsondata.", + slot_desc->col_name()); + } } } else { - RETURN_IF_ERROR(_append_error_msg( - nullptr, "The column `{}` is not nullable, but it's not found in jsondata.", - slot_desc->col_name(), valid)); + const std::string& v_str = col_value->second; + Slice column_default_value {v_str}; + RETURN_IF_ERROR(serde->deserialize_one_cell_from_json(*column_ptr, column_default_value, + _serde_options)); } - *valid = true; return Status::OK(); } diff --git a/be/src/vec/exec/format/json/new_json_reader.h b/be/src/vec/exec/format/json/new_json_reader.h index 985bf999676970..482a1ced747cef 100644 --- a/be/src/vec/exec/format/json/new_json_reader.h +++ b/be/src/vec/exec/format/json/new_json_reader.h @@ -87,7 +87,8 @@ class NewJsonReader : public GenericReader { ~NewJsonReader() override = default; Status init_reader(const std::unordered_map& - col_default_value_ctx); + col_default_value_ctx, + bool is_load); Status get_next_block(Block* block, size_t* read_rows, bool* eof) override; Status get_columns(std::unordered_map* name_to_type, std::unordered_set* missing_cols) override; @@ -128,7 +129,8 @@ class NewJsonReader : public GenericReader { const std::vector& slot_descs, bool* valid); Status _write_data_to_column(rapidjson::Value::ConstValueIterator value, - SlotDescriptor* slot_desc, vectorized::IColumn* column_ptr, + const TypeDescriptor& type_desc, vectorized::IColumn* column_ptr, + const std::string& column_name, DataTypeSerDeSPtr serde, bool* valid); Status _write_columns_by_jsonpath(rapidjson::Value& objectValue, @@ -177,8 +179,10 @@ class NewJsonReader : public GenericReader { const std::vector& slot_descs, bool* valid); Status _simdjson_write_data_to_column(simdjson::ondemand::value& value, - SlotDescriptor* slot_desc, - vectorized::IColumn* column_ptr, bool* valid); + const TypeDescriptor& type_desc, + vectorized::IColumn* column_ptr, + const std::string& column_name, DataTypeSerDeSPtr serde, + bool* valid); Status _simdjson_write_columns_by_jsonpath(simdjson::ondemand::object* value, const std::vector& slot_descs, @@ -196,8 +200,8 @@ class NewJsonReader : public GenericReader { const std::unordered_map& col_default_value_ctx); - Status _fill_missing_column(SlotDescriptor* slot_desc, vectorized::IColumn* column_ptr, - bool* valid); + Status _fill_missing_column(SlotDescriptor* slot_desc, DataTypeSerDeSPtr serde, + vectorized::IColumn* column_ptr, bool* valid); // fe will add skip_bitmap_col to _file_slot_descs iff the target olap table has skip_bitmap_col // and the current load is a flexible partial update @@ -292,6 +296,22 @@ class NewJsonReader : public GenericReader { std::unordered_map _col_default_value_map; int32_t skip_bitmap_col_idx {-1}; + + bool _is_load = true; + //Used to indicate whether it is a stream load. When loading, only data will be inserted into columnString. + //If an illegal value is encountered during the load process, `_append_error_msg` should be called + //instead of directly returning `Status::DataQualityError` + + bool _is_hive_table = false; + // In hive : create table xxx ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe'; + // Hive will not allow you to create columns with the same name but different case, including field names inside + // structs, and will automatically convert uppercase names in create sql to lowercase.However, when Hive loads data + // to table, the column names in the data may be uppercase,and there may be multiple columns with + // the same name but different capitalization.We refer to the behavior of hive, convert all column names + // in the data to lowercase,and use the last one as the insertion value + + DataTypeSerDeSPtrs _serdes; + vectorized::DataTypeSerDe::FormatOptions _serde_options; }; } // namespace vectorized diff --git a/be/src/vec/exec/scan/vfile_scanner.cpp b/be/src/vec/exec/scan/vfile_scanner.cpp index 982ac6072dc018..3053adebbb521e 100644 --- a/be/src/vec/exec/scan/vfile_scanner.cpp +++ b/be/src/vec/exec/scan/vfile_scanner.cpp @@ -965,8 +965,8 @@ Status VFileScanner::_get_next_reader() { _cur_reader = NewJsonReader::create_unique(_state, _profile, &_counter, *_params, range, _file_slot_descs, &_scanner_eof, _io_ctx.get()); - init_status = - ((NewJsonReader*)(_cur_reader.get()))->init_reader(_col_default_value_ctx); + init_status = ((NewJsonReader*)(_cur_reader.get())) + ->init_reader(_col_default_value_ctx, _is_load); break; } case TFileFormatType::FORMAT_AVRO: { diff --git a/docker/thirdparties/docker-compose/hive/scripts/create_preinstalled_scripts/run69.hql b/docker/thirdparties/docker-compose/hive/scripts/create_preinstalled_scripts/run69.hql new file mode 100644 index 00000000000000..adf0f7d56b27d9 --- /dev/null +++ b/docker/thirdparties/docker-compose/hive/scripts/create_preinstalled_scripts/run69.hql @@ -0,0 +1,35 @@ +use `default`; + + +CREATE TABLE json_nested_complex_table ( + user_ID STRING, + user_PROFILE STRUCT< + name: STRING, + AGE: INT, + preferences: MAP< + STRING, + STRUCT< + preference_ID: INT, + preference_VALUES: ARRAY + > + > + >, + activity_LOG ARRAY< + STRUCT< + activity_DATE: STRING, + activities: MAP< + STRING, + STRUCT< + `DETAILS`: STRING, + metrics: MAP + > + > + > + > +) ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe' + +LOCATION + '/user/doris/preinstalled_data/json/json_nested_complex_table'; + + +msck repair table json_nested_complex_table; diff --git a/docker/thirdparties/docker-compose/hive/scripts/create_preinstalled_scripts/run70.hql b/docker/thirdparties/docker-compose/hive/scripts/create_preinstalled_scripts/run70.hql new file mode 100644 index 00000000000000..73df8cba557bcb --- /dev/null +++ b/docker/thirdparties/docker-compose/hive/scripts/create_preinstalled_scripts/run70.hql @@ -0,0 +1,73 @@ +use `default`; + + +CREATE TABLE json_all_complex_types ( + `id` int, + `boolean_col` boolean, + `tinyint_col` tinyint, + `smallint_col` smallint, + `int_col` int, + `bigint_col` bigint, + `float_col` float, + `double_col` double, + `decimal_col1` decimal(9,0), + `decimal_col2` decimal(8,4), + `decimal_col3` decimal(18,6), + `decimal_col4` decimal(38,12), + `string_col` string, + `binary_col` binary, + `date_col` date, + `timestamp_col1` timestamp, + `timestamp_col2` timestamp, + `timestamp_col3` timestamp, + `char_col1` char(50), + `char_col2` char(100), + `char_col3` char(255), + `varchar_col1` varchar(50), + `varchar_col2` varchar(100), + `varchar_col3` varchar(255), + `t_map_string` map, + `t_map_varchar` map, + `t_map_char` map, + `t_map_int` map, + `t_map_bigint` map, + `t_map_float` map, + `t_map_double` map, + `t_map_boolean` map, + `t_map_decimal_precision_2` map, + `t_map_decimal_precision_4` map, + `t_map_decimal_precision_8` map, + `t_map_decimal_precision_17` map, + `t_map_decimal_precision_18` map, + `t_map_decimal_precision_38` map, + `t_array_string` array, + `t_array_int` array, + `t_array_bigint` array, + `t_array_float` array, + `t_array_double` array, + `t_array_boolean` array, + `t_array_varchar` array, + `t_array_char` array, + `t_array_decimal_precision_2` array, + `t_array_decimal_precision_4` array, + `t_array_decimal_precision_8` array, + `t_array_decimal_precision_17` array, + `t_array_decimal_precision_18` array, + `t_array_decimal_precision_38` array, + `t_struct_bigint` struct, + `t_complex` map>>, + `t_struct_nested` struct>, + `t_struct_null` struct, + `t_struct_non_nulls_after_nulls` struct, + `t_nested_struct_non_nulls_after_nulls` struct>, + `t_map_null_value` map, + `t_array_string_starting_with_nulls` array, + `t_array_string_with_nulls_in_between` array, + `t_array_string_ending_with_nulls` array, + `t_array_string_all_nulls` array + ) PARTITIONED BY (`dt` string) +ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe' +LOCATION + '/user/doris/preinstalled_data/json/json_all_complex_types'; + +msck repair table json_all_complex_types; diff --git a/docker/thirdparties/docker-compose/hive/scripts/create_preinstalled_scripts/run71.hql b/docker/thirdparties/docker-compose/hive/scripts/create_preinstalled_scripts/run71.hql new file mode 100644 index 00000000000000..ec99e72d2f5780 --- /dev/null +++ b/docker/thirdparties/docker-compose/hive/scripts/create_preinstalled_scripts/run71.hql @@ -0,0 +1,13 @@ +use `default`; + + +CREATE TABLE json_load_data_table ( + `id` int, + `col1` int, + `col2` struct< col2a:int, col2b:string>, + `col3` map +) ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe' +LOCATION + '/user/doris/preinstalled_data/json/json_load_data_table'; + +msck repair table json_load_data_table; diff --git a/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/json/json_all_complex_types/dt=dt1/000000_0 b/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/json/json_all_complex_types/dt=dt1/000000_0 new file mode 100644 index 00000000000000..5fe37cbc6f098e --- /dev/null +++ b/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/json/json_all_complex_types/dt=dt1/000000_0 @@ -0,0 +1,3 @@ +{"id":1,"boolean_col":true,"tinyint_col":127,"smallint_col":32767,"int_col":2147483647,"bigint_col":9223372036854775807,"float_col":123.45,"double_col":123456.789,"decimal_col1":123456789,"decimal_col2":1234.5678,"decimal_col3":123456.789012,"decimal_col4":123456789.012345678901,"string_col":"string_value","binary_col":"binary_value","date_col":"2024-03-20","timestamp_col1":"2024-03-20 12:00:00","timestamp_col2":"2024-03-20 12:00:00.123456789","timestamp_col3":"2024-03-20 12:00:00.123456789","char_col1":"char_value1 ","char_col2":"char_value2 ","char_col3":"char_value3 ","varchar_col1":"varchar_value1","varchar_col2":"varchar_value2","varchar_col3":"varchar_value3","t_map_string":{"key1":"value1"},"t_map_varchar":{"key1":"value1"},"t_map_char":{"a ":"b "},"t_map_int":{"1":10},"t_map_bigint":{"1":100000000000},"t_map_float":{"1.1":10.1},"t_map_double":{"1.1":10.1},"t_map_boolean":{"true":false},"t_map_decimal_precision_2":{"1.1":1.1},"t_map_decimal_precision_4":{"1.23":1.23},"t_map_decimal_precision_8":{"1.2345":1.2345},"t_map_decimal_precision_17":{"1.23456789":1.23456789},"t_map_decimal_precision_18":{"1.23456789":1.23456789},"t_map_decimal_precision_38":{"1.2345678901234568":1.2345678901234568},"t_array_string":["string1","string2"],"t_array_int":[1,2,3],"t_array_bigint":[100000000000,200000000000],"t_array_float":[1.1,2.2],"t_array_double":[1.123456789,2.123456789],"t_array_boolean":[true,false],"t_array_varchar":["varchar1","varchar2"],"t_array_char":["char1 ","char2 "],"t_array_decimal_precision_2":[1.1,2.2],"t_array_decimal_precision_4":[1.23,2.34],"t_array_decimal_precision_8":[1.2345,2.3456],"t_array_decimal_precision_17":[1.23456789,2.34567891],"t_array_decimal_precision_18":[1.23456789,2.34567891],"t_array_decimal_precision_38":[1.2345678901234568,2.3456789012345679],"t_struct_bigint":{"s_bigint":1234567890},"t_complex":{"key":[{"s_int":123}]},"t_struct_nested":{"struct_field":["value1","value2"]},"t_struct_null":{"struct_field_null":null,"struct_field_null2":null},"t_struct_non_nulls_after_nulls":{"struct_non_nulls_after_nulls1":123,"struct_non_nulls_after_nulls2":"value"},"t_nested_struct_non_nulls_after_nulls":{"struct_field1":123,"struct_field2":"value","strict_field3":{"nested_struct_field1":123,"nested_struct_field2":"nested_value"}},"t_map_null_value":{"null_key":null},"t_array_string_starting_with_nulls":[null,"value1","value2"],"t_array_string_with_nulls_in_between":["value1",null,"value2"],"t_array_string_ending_with_nulls":["value1","value2",null],"t_array_string_all_nulls":[null,null,null]} +{"id":2,"boolean_col":false,"tinyint_col":58,"smallint_col":12345,"int_col":2147483000,"bigint_col":null,"float_col":789.56,"double_col":654321.123,"decimal_col1":987654321,"decimal_col2":5678.1234,"decimal_col3":987654.321098,"decimal_col4":987654321.098765432109,"string_col":"changed_string","binary_col":"new_binary_value","date_col":"2025-05-25","timestamp_col1":"2025-05-25 15:30:00","timestamp_col2":"2025-05-25 15:30:00.654321987","timestamp_col3":"2025-05-25 15:30:00.654321987","char_col1":"char_new_value1 ","char_col2":"char_new_value2 ","char_col3":"char_new_value3 ","varchar_col1":"varchar_new_value1","varchar_col2":"varchar_new_value2","varchar_col3":"varchar_new_value3","t_map_string":{"key2":"value2"},"t_map_varchar":{"key2":"value2"},"t_map_char":{"x ":"y "},"t_map_int":{"2":20},"t_map_bigint":{"2":200000000000},"t_map_float":{"2.2":20.2},"t_map_double":{"2.2":20.2},"t_map_boolean":{"false":true},"t_map_decimal_precision_2":{"2.2":2.2},"t_map_decimal_precision_4":{"2.34":2.34},"t_map_decimal_precision_8":{"2.3456":2.3456},"t_map_decimal_precision_17":{"2.34567891":2.34567891},"t_map_decimal_precision_18":{"2.34567891":2.34567891},"t_map_decimal_precision_38":{"2.3456789012345679":2.3456789012345679},"t_array_string":["string3","string4"],"t_array_int":[4,5,6],"t_array_bigint":[300000000000,400000000000],"t_array_float":[2.2,3.3],"t_array_double":[2.123456789,3.123456789],"t_array_boolean":[false,true],"t_array_varchar":["varchar3","varchar4"],"t_array_char":["char3 ","char4 "],"t_array_decimal_precision_2":[2.2,3.3],"t_array_decimal_precision_4":[2.34,3.45],"t_array_decimal_precision_8":[2.3456,3.4567],"t_array_decimal_precision_17":[2.34567891,3.45678901],"t_array_decimal_precision_18":[2.34567891,3.45678901],"t_array_decimal_precision_38":[2.3456789012345679,3.4567890123456789],"t_struct_bigint":{"s_bigint":9876543210},"t_complex":{"key2":[{"s_int":456}]},"t_struct_nested":{"struct_field":["new_value1","new_value2"]},"t_struct_null":{"struct_field_null":null,"struct_field_null2":null},"t_struct_non_nulls_after_nulls":{"struct_non_nulls_after_nulls1":456,"struct_non_nulls_after_nulls2":"new_value"},"t_nested_struct_non_nulls_after_nulls":{"struct_field1":456,"struct_field2":"new_value","strict_field3":{"nested_struct_field1":456,"nested_struct_field2":"nested_value2"}},"t_map_null_value":{"null_key":null},"t_array_string_starting_with_nulls":[null,"new_value1","new_value2"],"t_array_string_with_nulls_in_between":["new_value1",null,"new_value2"],"t_array_string_ending_with_nulls":["new_value1","new_value2",null],"t_array_string_all_nulls":[null,null,null]} +{"id":3,"boolean_col":false,"tinyint_col":-128,"smallint_col":-32768,"int_col":-2147483648,"bigint_col":-9223372036854775808,"float_col":-3.4028235E38,"double_col":-1.7976931348623157E308,"decimal_col1":-999999999,"decimal_col2":-9999.9999,"decimal_col3":-999999999.999999,"decimal_col4":null,"string_col":"min_string_value","binary_col":"xxxx","date_col":"2001-01-01","timestamp_col1":"2001-01-01 00:00:00","timestamp_col2":"2001-01-01 00:00:00","timestamp_col3":"2001-01-01 00:00:00","char_col1":"char_min_value1 ","char_col2":"char_min_value2 ","char_col3":"char_min_value3 ","varchar_col1":"varchar_min_value1","varchar_col2":"varchar_min_value2","varchar_col3":"varchar_min_value3","t_map_string":{"min_key":"min_value"},"t_map_varchar":{"min_key":"min_value"},"t_map_char":{"a ":"z "},"t_map_int":{"-1":-100},"t_map_bigint":{"-1":-100000000000},"t_map_float":{"-1.1":-10.1},"t_map_double":{"-1.1":-10.1},"t_map_boolean":{"false":true},"t_map_decimal_precision_2":{"-1.1":-1.1},"t_map_decimal_precision_4":{"-1.23":-1.23},"t_map_decimal_precision_8":{"-1.2345":-1.2345},"t_map_decimal_precision_17":{"-1.23456789":-1.23456789},"t_map_decimal_precision_18":{"-1.23456789":-1.23456789},"t_map_decimal_precision_38":{"-1.2345678901234568":-1.2345678901234568},"t_array_string":["min_string1","min_string2"],"t_array_int":[-10,-5,-3],"t_array_bigint":[-100000000000,-200000000000],"t_array_float":[-1.1,-2.2],"t_array_double":[-1.123456789,-2.123456789],"t_array_boolean":[false,true],"t_array_varchar":["min_varchar1","min_varchar2"],"t_array_char":["min_char1 ","min_char2 "],"t_array_decimal_precision_2":[-1.1,-2.2],"t_array_decimal_precision_4":[-1.23,-2.34],"t_array_decimal_precision_8":[-1.2345,-2.3456],"t_array_decimal_precision_17":[-1.23456789,-2.34567891],"t_array_decimal_precision_18":[-1.23456789,-2.34567891],"t_array_decimal_precision_38":[-1.2345678901234568,-2.3456789012345679],"t_struct_bigint":{"s_bigint":-1234567890},"t_complex":{"min_key":[{"s_int":-123}]},"t_struct_nested":{"struct_field":["min_value1","min_value2"]},"t_struct_null":{"struct_field_null":null,"struct_field_null2":null},"t_struct_non_nulls_after_nulls":{"struct_non_nulls_after_nulls1":-123,"struct_non_nulls_after_nulls2":"min_value"},"t_nested_struct_non_nulls_after_nulls":{"struct_field1":-123,"struct_field2":"min_value","strict_field3":{"nested_struct_field1":-123,"nested_struct_field2":"nested_value"}},"t_map_null_value":{"null_key":null},"t_array_string_starting_with_nulls":[null,"min_value1","min_value2"],"t_array_string_with_nulls_in_between":["min_value1",null,"min_value2"],"t_array_string_ending_with_nulls":["min_value1","min_value2",null],"t_array_string_all_nulls":[null,null,null]} diff --git a/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/json/json_all_complex_types/dt=dt2/000000_0 b/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/json/json_all_complex_types/dt=dt2/000000_0 new file mode 100644 index 00000000000000..0a823bee693d76 --- /dev/null +++ b/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/json/json_all_complex_types/dt=dt2/000000_0 @@ -0,0 +1 @@ +{"id":4,"boolean_col":null,"tinyint_col":null,"smallint_col":null,"int_col":null,"bigint_col":null,"float_col":123.45,"double_col":null,"decimal_col1":null,"decimal_col2":null,"decimal_col3":null,"decimal_col4":null,"string_col":null,"binary_col":null,"date_col":null,"timestamp_col1":null,"timestamp_col2":null,"timestamp_col3":null,"char_col1":null,"char_col2":null,"char_col3":null,"varchar_col1":null,"varchar_col2":null,"varchar_col3":null,"t_map_string":null,"t_map_varchar":null,"t_map_char":null,"t_map_int":{"1":10},"t_map_bigint":null,"t_map_float":null,"t_map_double":null,"t_map_boolean":null,"t_map_decimal_precision_2":null,"t_map_decimal_precision_4":null,"t_map_decimal_precision_8":null,"t_map_decimal_precision_17":null,"t_map_decimal_precision_18":null,"t_map_decimal_precision_38":null,"t_array_string":null,"t_array_int":null,"t_array_bigint":null,"t_array_float":null,"t_array_double":null,"t_array_boolean":null,"t_array_varchar":null,"t_array_char":null,"t_array_decimal_precision_2":null,"t_array_decimal_precision_4":null,"t_array_decimal_precision_8":[1.2345,2.3456],"t_array_decimal_precision_17":null,"t_array_decimal_precision_18":null,"t_array_decimal_precision_38":null,"t_struct_bigint":null,"t_complex":null,"t_struct_nested":null,"t_struct_null":null,"t_struct_non_nulls_after_nulls":null,"t_nested_struct_non_nulls_after_nulls":null,"t_map_null_value":null,"t_array_string_starting_with_nulls":[null,"value1","value2"],"t_array_string_with_nulls_in_between":null,"t_array_string_ending_with_nulls":null,"t_array_string_all_nulls":null} diff --git a/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/json/json_all_complex_types/dt=dt3/000000_0 b/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/json/json_all_complex_types/dt=dt3/000000_0 new file mode 100644 index 00000000000000..a5e46399fdd553 --- /dev/null +++ b/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/json/json_all_complex_types/dt=dt3/000000_0 @@ -0,0 +1,2 @@ +{"id":5,"boolean_col":null,"tinyint_col":null,"smallint_col":null,"int_col":null,"bigint_col":null,"float_col":null,"double_col":null,"decimal_col1":null,"decimal_col2":null,"decimal_col3":null,"decimal_col4":null,"string_col":null,"binary_col":null,"date_col":null,"timestamp_col1":null,"timestamp_col2":null,"timestamp_col3":null,"char_col1":null,"char_col2":null,"char_col3":null,"varchar_col1":null,"varchar_col2":null,"varchar_col3":null,"t_map_string":null,"t_map_varchar":null,"t_map_char":null,"t_map_int":null,"t_map_bigint":null,"t_map_float":null,"t_map_double":null,"t_map_boolean":null,"t_map_decimal_precision_2":null,"t_map_decimal_precision_4":null,"t_map_decimal_precision_8":null,"t_map_decimal_precision_17":null,"t_map_decimal_precision_18":null,"t_map_decimal_precision_38":null,"t_array_string":null,"t_array_int":null,"t_array_bigint":null,"t_array_float":null,"t_array_double":null,"t_array_boolean":null,"t_array_varchar":null,"t_array_char":null,"t_array_decimal_precision_2":null,"t_array_decimal_precision_4":null,"t_array_decimal_precision_8":null,"t_array_decimal_precision_17":null,"t_array_decimal_precision_18":null,"t_array_decimal_precision_38":null,"t_struct_bigint":null,"t_complex":null,"t_struct_nested":null,"t_struct_null":null,"t_struct_non_nulls_after_nulls":null,"t_nested_struct_non_nulls_after_nulls":null,"t_map_null_value":null,"t_array_string_starting_with_nulls":null,"t_array_string_with_nulls_in_between":null,"t_array_string_ending_with_nulls":null,"t_array_string_all_nulls":null} +{"id":6,"boolean_col":null,"tinyint_col":null,"smallint_col":null,"int_col":null,"bigint_col":null,"float_col":null,"double_col":null,"decimal_col1":null,"decimal_col2":null,"decimal_col3":null,"decimal_col4":null,"string_col":null,"binary_col":null,"date_col":null,"timestamp_col1":null,"timestamp_col2":null,"timestamp_col3":null,"char_col1":null,"char_col2":null,"char_col3":null,"varchar_col1":null,"varchar_col2":null,"varchar_col3":null,"t_map_string":null,"t_map_varchar":null,"t_map_char":null,"t_map_int":null,"t_map_bigint":null,"t_map_float":null,"t_map_double":null,"t_map_boolean":null,"t_map_decimal_precision_2":null,"t_map_decimal_precision_4":null,"t_map_decimal_precision_8":null,"t_map_decimal_precision_17":null,"t_map_decimal_precision_18":null,"t_map_decimal_precision_38":null,"t_array_string":null,"t_array_int":null,"t_array_bigint":null,"t_array_float":null,"t_array_double":null,"t_array_boolean":null,"t_array_varchar":null,"t_array_char":null,"t_array_decimal_precision_2":null,"t_array_decimal_precision_4":null,"t_array_decimal_precision_8":null,"t_array_decimal_precision_17":null,"t_array_decimal_precision_18":null,"t_array_decimal_precision_38":null,"t_struct_bigint":null,"t_complex":null,"t_struct_nested":null,"t_struct_null":null,"t_struct_non_nulls_after_nulls":null,"t_nested_struct_non_nulls_after_nulls":null,"t_map_null_value":null,"t_array_string_starting_with_nulls":null,"t_array_string_with_nulls_in_between":null,"t_array_string_ending_with_nulls":null,"t_array_string_all_nulls":null} diff --git a/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/json/json_load_data_table/1 b/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/json/json_load_data_table/1 new file mode 100644 index 00000000000000..70d1265f98d826 --- /dev/null +++ b/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/json/json_load_data_table/1 @@ -0,0 +1,13 @@ +{"id":1,"col1":10,"col2":{"col2a":10,"col2b":"string1"},"col3":{"1":"string10"}} +{"id":2,"col1":10,"col1":20,"col2":{"col2b":"string2","col2a":0,"Col2A":20},"col3":{"2":"string2"}} +{"id":3,"col1":10,"col1":20,"COL1":30,"COL2":{"col2a":30,"col2b":"string3"}} +{"id":4,"COL1":40,"col2":{"col2a":10,"col2b":"string4","new_col":"new_val","col2a":40},"col3":{"4":"string4"}} +{"id":5} +{"id":6,"col1":60,"col2":{"COL2a":60,"col2b":600},"col3":{"6":600}} +{"id":7,"col1":70,"col3":{"7":"string7"},"col2":{"col2b":"string7","col2a":70}} + + + + +{} +{"a":5} diff --git a/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/json/json_nested_complex_table/1 b/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/json/json_nested_complex_table/1 new file mode 100644 index 00000000000000..11342c441bce00 --- /dev/null +++ b/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/json/json_nested_complex_table/1 @@ -0,0 +1,2 @@ +{"user_id":"user1","user_profile":{"name":"Alice","age":28,"preferences":{"sports":{"preference_id":101,"preference_values":["soccer","tennis"]},"music":{"preference_id":102,"preference_values":["rock","classical"]}}},"activity_log":[{"activity_date":"2024-08-01","activities":{"workout":{"details":"Morning run","metrics":{"duration":30.5,"calories":200.0}},"reading":{"details":"Read book on Hive","metrics":{"pages":50.0,"time":2.0}}}},{"activity_date":"2024-08-02","activities":{"travel":{"details":"Flight to NY","metrics":{"distance":500.0,"time":3.0}},"meeting":{"details":"Project meeting","metrics":{"duration":1.5,"participants":5.0}}}}]} +{"user_id":"user2","user_profile":{"name":"Bob","age":32,"preferences":{"books":{"preference_id":201,"preference_values":["fiction","non-fiction"]},"travel":{"preference_id":202,"preference_values":["beaches","mountains"]}}},"activity_log":[{"activity_date":"2024-08-01","activities":{"hiking":{"details":"Mountain trail","metrics":{"distance":10.0,"elevation":500.0}},"photography":{"details":"Wildlife photoshoot","metrics":{"photos_taken":100.0,"time":4.0}}}},{"activity_date":"2024-08-02","activities":{"workshop":{"details":"Photography workshop","metrics":{"duration":3.0,"participants":15.0}},"shopping":{"details":"Bought camera gear","metrics":{"items":5.0,"cost":1500.0}}}}]} diff --git a/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/json/json_nested_complex_table/2 b/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/json/json_nested_complex_table/2 new file mode 100644 index 00000000000000..e1b0befc7bca31 --- /dev/null +++ b/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/json/json_nested_complex_table/2 @@ -0,0 +1 @@ +{"user_id":"user3","user_profile":{"name":"Carol","age":24,"preferences":{"food":{"preference_id":301,"preference_values":["vegan","desserts"]},"movies":{"preference_id":302,"preference_values":["action","comedy"]}}},"activity_log":[{"activity_date":"2024-08-01","activities":{"cooking":{"details":"Made vegan meal","metrics":{"time_spent":1.5,"calories":500.0}},"movie":{"details":"Watched action movie","metrics":{"duration":2.0,"rating":8.5}}}},{"activity_date":"2024-08-02","activities":{"gym":{"details":"Strength training","metrics":{"duration":1.0,"calories":300.0}},"shopping":{"details":"Bought groceries","metrics":{"items":10.0,"cost":100.0}}}}]} diff --git a/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/json/json_nested_complex_table/modify_2 b/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/json/json_nested_complex_table/modify_2 new file mode 100644 index 00000000000000..08f1586f3aa91c --- /dev/null +++ b/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/json/json_nested_complex_table/modify_2 @@ -0,0 +1,2 @@ +{"user_ID":"user4","user_PROFILE":{"name":"Carol","age":24,"preferences":{"food":{"preference_ID":301,"preference_VALUES":["vegan","desserts"]},"movies":{"preference_ID":302,"preference_VALUES":["action","comedy"]}}},"activity_LOG":[{"activity_DATE":"2024-08-01","activities":{"cooking":{"DETAILS":"Made vegan meal","metrics":{"time_spent":1.5,"calories":500.0}},"movie":{"DETAILS":"Watched action movie","metrics":{"duration":2.0,"rating":8.5}}}},{"activity_DATE":"2024-08-02","activities":{"gym":{"DETAILS":"Strength training","metrics":{"duration":1.0,"calories":300.0}},"shopping":{"DETAILS":"Bought groceries","metrics":{"items":10.0,"cost":100.0}}}}]} + diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetaStoreClientHelper.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetaStoreClientHelper.java index 97032467cec765..0f839d238b2b1e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetaStoreClientHelper.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetaStoreClientHelper.java @@ -94,6 +94,9 @@ public class HiveMetaStoreClientHelper { private static final Pattern digitPattern = Pattern.compile("(\\d+)"); + public static final String HIVE_JSON_SERDE = "org.apache.hive.hcatalog.data.JsonSerDe"; + public static final String LEGACY_HIVE_JSON_SERDE = "org.apache.hadoop.hive.serde2.JsonSerDe"; + public enum HiveFileFormat { TEXT_FILE(0, "text"), PARQUET(1, "parquet"), diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/source/HiveScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/source/HiveScanNode.java index e710bdb935d7bc..3a2a4d3eb5c6ae 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/source/HiveScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/source/HiveScanNode.java @@ -364,14 +364,21 @@ public TableIf getTargetTable() { @Override public TFileFormatType getFileFormatType() throws UserException { TFileFormatType type = null; - String inputFormatName = hmsTable.getRemoteTable().getSd().getInputFormat(); + Table table = hmsTable.getRemoteTable(); + String inputFormatName = table.getSd().getInputFormat(); String hiveFormat = HiveMetaStoreClientHelper.HiveFileFormat.getFormat(inputFormatName); if (hiveFormat.equals(HiveMetaStoreClientHelper.HiveFileFormat.PARQUET.getDesc())) { type = TFileFormatType.FORMAT_PARQUET; } else if (hiveFormat.equals(HiveMetaStoreClientHelper.HiveFileFormat.ORC.getDesc())) { type = TFileFormatType.FORMAT_ORC; } else if (hiveFormat.equals(HiveMetaStoreClientHelper.HiveFileFormat.TEXT_FILE.getDesc())) { - type = TFileFormatType.FORMAT_CSV_PLAIN; + String serDeLib = table.getSd().getSerdeInfo().getSerializationLib(); + if (serDeLib.equals(HiveMetaStoreClientHelper.HIVE_JSON_SERDE) + || serDeLib.equals(HiveMetaStoreClientHelper.LEGACY_HIVE_JSON_SERDE)) { + type = TFileFormatType.FORMAT_JSON; + } else { + type = TFileFormatType.FORMAT_CSV_PLAIN; + } } return type; } @@ -383,11 +390,12 @@ protected Map getLocationProperties() throws UserException { @Override protected TFileAttributes getFileAttributes() throws UserException { - TFileTextScanRangeParams textParams = new TFileTextScanRangeParams(); + TFileAttributes fileAttributes = new TFileAttributes(); Table table = hmsTable.getRemoteTable(); // TODO: separate hive text table and OpenCsv table String serDeLib = table.getSd().getSerdeInfo().getSerializationLib(); if (serDeLib.equals("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe")) { + TFileTextScanRangeParams textParams = new TFileTextScanRangeParams(); // set properties of LazySimpleSerDe // 1. set column separator textParams.setColumnSeparator(HiveProperties.getFieldDelimiter(table)); @@ -401,7 +409,10 @@ protected TFileAttributes getFileAttributes() throws UserException { HiveProperties.getEscapeDelimiter(table).ifPresent(d -> textParams.setEscape(d.getBytes()[0])); // 6. set null format textParams.setNullFormat(HiveProperties.getNullFormat(table)); + fileAttributes.setTextParams(textParams); + fileAttributes.setHeaderType(""); } else if (serDeLib.equals("org.apache.hadoop.hive.serde2.OpenCSVSerde")) { + TFileTextScanRangeParams textParams = new TFileTextScanRangeParams(); // set set properties of OpenCSVSerde // 1. set column separator textParams.setColumnSeparator(HiveProperties.getSeparatorChar(table)); @@ -411,17 +422,29 @@ protected TFileAttributes getFileAttributes() throws UserException { textParams.setEnclose(HiveProperties.getQuoteChar(table).getBytes()[0]); // 4. set escape char textParams.setEscape(HiveProperties.getEscapeChar(table).getBytes()[0]); + fileAttributes.setTextParams(textParams); + fileAttributes.setHeaderType(""); + if (textParams.isSetEnclose()) { + fileAttributes.setTrimDoubleQuotes(true); + } + } else if (serDeLib.equals("org.apache.hive.hcatalog.data.JsonSerDe")) { + TFileTextScanRangeParams textParams = new TFileTextScanRangeParams(); + textParams.setColumnSeparator("\t"); + textParams.setLineDelimiter("\n"); + fileAttributes.setTextParams(textParams); + + fileAttributes.setJsonpaths(""); + fileAttributes.setJsonRoot(""); + fileAttributes.setNumAsString(true); + fileAttributes.setFuzzyParse(false); + fileAttributes.setReadJsonByLine(true); + fileAttributes.setStripOuterArray(false); + fileAttributes.setHeaderType(""); } else { throw new UserException( "unsupported hive table serde: " + serDeLib); } - TFileAttributes fileAttributes = new TFileAttributes(); - fileAttributes.setTextParams(textParams); - fileAttributes.setHeaderType(""); - if (textParams.isSet(TFileTextScanRangeParams._Fields.ENCLOSE)) { - fileAttributes.setTrimDoubleQuotes(true); - } return fileAttributes; } diff --git a/regression-test/data/external_table_p0/hive/hive_json_basic_test.out b/regression-test/data/external_table_p0/hive/hive_json_basic_test.out new file mode 100644 index 00000000000000..9023f5d72b1ac3 --- /dev/null +++ b/regression-test/data/external_table_p0/hive/hive_json_basic_test.out @@ -0,0 +1,115 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !q1 -- +1 true 127 32767 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value binary_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123457 2024-03-20T12:00:00.123457 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"a ":"b "} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] ["varchar1", "varchar2"] ["char1 ", "char2 "] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint":1234567890} {"key":[{"s_int":123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"struct_non_nulls_after_nulls1":123, "struct_non_nulls_after_nulls2":"value"} {"struct_field1":123, "struct_field2":"value", "strict_field3":{"nested_struct_field1":123, "nested_struct_field2":"nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] dt1 +2 false 58 12345 2147483000 \N 789.56 654321.123 987654321 5678.1234 987654.321098 987654321.098765432109 changed_string new_binary_value 2025-05-25 2025-05-25T15:30 2025-05-25T15:30:00.654322 2025-05-25T15:30:00.654322 char_new_value1 char_new_value2 char_new_value3 varchar_new_value1 varchar_new_value2 varchar_new_value3 {"key2":"value2"} {"key2":"value2"} {"x ":"y "} {2:20} {2:200000000000} {2.2:20.2} {2.2:20.2} {0:1} {2.2:2.2} {2.34:2.34} {2.3456:2.3456} {2.34567891:2.34567891} {2.34567891:2.34567891} {2.3456789012345679:2.3456789012345679} ["string3", "string4"] [4, 5, 6] [300000000000, 400000000000] [2.2, 3.3] [2.123456789, 3.123456789] [0, 1] ["varchar3", "varchar4"] ["char3 ", "char4 "] [2.2, 3.3] [2.34, 3.45] [2.3456, 3.4567] [2.34567891, 3.45678901] [2.34567891, 3.45678901] [2.3456789012345679, 3.4567890123456789] {"s_bigint":9876543210} {"key2":[{"s_int":456}]} {"struct_field":["new_value1", "new_value2"]} {"struct_field_null":null, "struct_field_null2":null} {"struct_non_nulls_after_nulls1":456, "struct_non_nulls_after_nulls2":"new_value"} {"struct_field1":456, "struct_field2":"new_value", "strict_field3":{"nested_struct_field1":456, "nested_struct_field2":"nested_value2"}} {"null_key":null} [null, "new_value1", "new_value2"] ["new_value1", null, "new_value2"] ["new_value1", "new_value2", null] [null, null, null] dt1 +3 false -128 -32768 -2147483648 -9223372036854775808 -3.4028235e+38 -1.7976931348623157E308 -999999999 -9999.9999 -999999999.999999 \N min_string_value xxxx 2001-01-01 2001-01-01T00:00 2001-01-01T00:00 2001-01-01T00:00 char_min_value1 char_min_value2 char_min_value3 varchar_min_value1 varchar_min_value2 varchar_min_value3 {"min_key":"min_value"} {"min_key":"min_value"} {"a ":"z "} {-1:-100} {-1:-100000000000} {-1.1:-10.1} {-1.1:-10.1} {0:1} {-1.1:-1.1} {-1.23:-1.23} {-1.2345:-1.2345} {-1.23456789:-1.23456789} {-1.23456789:-1.23456789} {-1.2345678901234568:-1.2345678901234568} ["min_string1", "min_string2"] [-10, -5, -3] [-100000000000, -200000000000] [-1.1, -2.2] [-1.123456789, -2.123456789] [0, 1] ["min_varchar1", "min_varchar2"] ["min_char1 ", "min_char2 "] [-1.1, -2.2] [-1.23, -2.34] [-1.2345, -2.3456] [-1.23456789, -2.34567891] [-1.23456789, -2.34567891] [-1.2345678901234568, -2.3456789012345679] {"s_bigint":-1234567890} {"min_key":[{"s_int":-123}]} {"struct_field":["min_value1", "min_value2"]} {"struct_field_null":null, "struct_field_null2":null} {"struct_non_nulls_after_nulls1":-123, "struct_non_nulls_after_nulls2":"min_value"} {"struct_field1":-123, "struct_field2":"min_value", "strict_field3":{"nested_struct_field1":-123, "nested_struct_field2":"nested_value"}} {"null_key":null} [null, "min_value1", "min_value2"] ["min_value1", null, "min_value2"] ["min_value1", "min_value2", null] [null, null, null] dt1 +4 \N \N \N \N \N 123.45 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N {1:10} \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N [1.2345, 2.3456] \N \N \N \N \N \N \N \N \N \N [null, "value1", "value2"] \N \N \N dt2 +5 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N dt3 +6 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N dt3 + +-- !q2 -- +3 false -128 -32768 -2147483648 -9223372036854775808 -3.4028235e+38 -1.7976931348623157E308 -999999999 -9999.9999 -999999999.999999 \N min_string_value xxxx 2001-01-01 2001-01-01T00:00 2001-01-01T00:00 2001-01-01T00:00 char_min_value1 char_min_value2 char_min_value3 varchar_min_value1 varchar_min_value2 varchar_min_value3 {"min_key":"min_value"} {"min_key":"min_value"} {"a ":"z "} {-1:-100} {-1:-100000000000} {-1.1:-10.1} {-1.1:-10.1} {0:1} {-1.1:-1.1} {-1.23:-1.23} {-1.2345:-1.2345} {-1.23456789:-1.23456789} {-1.23456789:-1.23456789} {-1.2345678901234568:-1.2345678901234568} ["min_string1", "min_string2"] [-10, -5, -3] [-100000000000, -200000000000] [-1.1, -2.2] [-1.123456789, -2.123456789] [0, 1] ["min_varchar1", "min_varchar2"] ["min_char1 ", "min_char2 "] [-1.1, -2.2] [-1.23, -2.34] [-1.2345, -2.3456] [-1.23456789, -2.34567891] [-1.23456789, -2.34567891] [-1.2345678901234568, -2.3456789012345679] {"s_bigint":-1234567890} {"min_key":[{"s_int":-123}]} {"struct_field":["min_value1", "min_value2"]} {"struct_field_null":null, "struct_field_null2":null} {"struct_non_nulls_after_nulls1":-123, "struct_non_nulls_after_nulls2":"min_value"} {"struct_field1":-123, "struct_field2":"min_value", "strict_field3":{"nested_struct_field1":-123, "nested_struct_field2":"nested_value"}} {"null_key":null} [null, "min_value1", "min_value2"] ["min_value1", null, "min_value2"] ["min_value1", "min_value2", null] [null, null, null] dt1 + +-- !q3 -- +1 true 127 32767 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value binary_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123457 2024-03-20T12:00:00.123457 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"a ":"b "} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] ["varchar1", "varchar2"] ["char1 ", "char2 "] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint":1234567890} {"key":[{"s_int":123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"struct_non_nulls_after_nulls1":123, "struct_non_nulls_after_nulls2":"value"} {"struct_field1":123, "struct_field2":"value", "strict_field3":{"nested_struct_field1":123, "nested_struct_field2":"nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] dt1 + +-- !q4 -- +123.45 +789.56 +-3.4028235e+38 +123.45 + +-- !q5 -- +2 false 58 12345 2147483000 \N 789.56 654321.123 987654321 5678.1234 987654.321098 987654321.098765432109 changed_string new_binary_value 2025-05-25 2025-05-25T15:30 2025-05-25T15:30:00.654322 2025-05-25T15:30:00.654322 char_new_value1 char_new_value2 char_new_value3 varchar_new_value1 varchar_new_value2 varchar_new_value3 {"key2":"value2"} {"key2":"value2"} {"x ":"y "} {2:20} {2:200000000000} {2.2:20.2} {2.2:20.2} {0:1} {2.2:2.2} {2.34:2.34} {2.3456:2.3456} {2.34567891:2.34567891} {2.34567891:2.34567891} {2.3456789012345679:2.3456789012345679} ["string3", "string4"] [4, 5, 6] [300000000000, 400000000000] [2.2, 3.3] [2.123456789, 3.123456789] [0, 1] ["varchar3", "varchar4"] ["char3 ", "char4 "] [2.2, 3.3] [2.34, 3.45] [2.3456, 3.4567] [2.34567891, 3.45678901] [2.34567891, 3.45678901] [2.3456789012345679, 3.4567890123456789] {"s_bigint":9876543210} {"key2":[{"s_int":456}]} {"struct_field":["new_value1", "new_value2"]} {"struct_field_null":null, "struct_field_null2":null} {"struct_non_nulls_after_nulls1":456, "struct_non_nulls_after_nulls2":"new_value"} {"struct_field1":456, "struct_field2":"new_value", "strict_field3":{"nested_struct_field1":456, "nested_struct_field2":"nested_value2"}} {"null_key":null} [null, "new_value1", "new_value2"] ["new_value1", null, "new_value2"] ["new_value1", "new_value2", null] [null, null, null] dt1 + +-- !q6 -- +user1 {"name":"Alice", "age":28, "preferences":{"sports":{"preference_id":101, "preference_values":["soccer", "tennis"]}, "music":{"preference_id":102, "preference_values":["rock", "classical"]}}} [{"activity_date":"2024-08-01", "activities":{"workout":{"details":"Morning run", "metrics":{"duration":30.5, "calories":200}}, "reading":{"details":"Read book on Hive", "metrics":{"pages":50, "time":2}}}}, {"activity_date":"2024-08-02", "activities":{"travel":{"details":"Flight to NY", "metrics":{"distance":500, "time":3}}, "meeting":{"details":"Project meeting", "metrics":{"duration":1.5, "participants":5}}}}] +user2 {"name":"Bob", "age":32, "preferences":{"books":{"preference_id":201, "preference_values":["fiction", "non-fiction"]}, "travel":{"preference_id":202, "preference_values":["beaches", "mountains"]}}} [{"activity_date":"2024-08-01", "activities":{"hiking":{"details":"Mountain trail", "metrics":{"distance":10, "elevation":500}}, "photography":{"details":"Wildlife photoshoot", "metrics":{"photos_taken":100, "time":4}}}}, {"activity_date":"2024-08-02", "activities":{"workshop":{"details":"Photography workshop", "metrics":{"duration":3, "participants":15}}, "shopping":{"details":"Bought camera gear", "metrics":{"items":5, "cost":1500}}}}] +user3 {"name":"Carol", "age":24, "preferences":{"food":{"preference_id":301, "preference_values":["vegan", "desserts"]}, "movies":{"preference_id":302, "preference_values":["action", "comedy"]}}} [{"activity_date":"2024-08-01", "activities":{"cooking":{"details":"Made vegan meal", "metrics":{"time_spent":1.5, "calories":500}}, "movie":{"details":"Watched action movie", "metrics":{"duration":2, "rating":8.5}}}}, {"activity_date":"2024-08-02", "activities":{"gym":{"details":"Strength training", "metrics":{"duration":1, "calories":300}}, "shopping":{"details":"Bought groceries", "metrics":{"items":10, "cost":100}}}}] +user4 {"name":"Carol", "age":24, "preferences":{"food":{"preference_id":301, "preference_values":["vegan", "desserts"]}, "movies":{"preference_id":302, "preference_values":["action", "comedy"]}}} [{"activity_date":"2024-08-01", "activities":{"cooking":{"details":"Made vegan meal", "metrics":{"time_spent":1.5, "calories":500}}, "movie":{"details":"Watched action movie", "metrics":{"duration":2, "rating":8.5}}}}, {"activity_date":"2024-08-02", "activities":{"gym":{"details":"Strength training", "metrics":{"duration":1, "calories":300}}, "shopping":{"details":"Bought groceries", "metrics":{"items":10, "cost":100}}}}] + +-- !q7 -- +user1 [{"activity_date":"2024-08-01", "activities":{"workout":{"details":"Morning run", "metrics":{"duration":30.5, "calories":200}}, "reading":{"details":"Read book on Hive", "metrics":{"pages":50, "time":2}}}}, {"activity_date":"2024-08-02", "activities":{"travel":{"details":"Flight to NY", "metrics":{"distance":500, "time":3}}, "meeting":{"details":"Project meeting", "metrics":{"duration":1.5, "participants":5}}}}] +user2 [{"activity_date":"2024-08-01", "activities":{"hiking":{"details":"Mountain trail", "metrics":{"distance":10, "elevation":500}}, "photography":{"details":"Wildlife photoshoot", "metrics":{"photos_taken":100, "time":4}}}}, {"activity_date":"2024-08-02", "activities":{"workshop":{"details":"Photography workshop", "metrics":{"duration":3, "participants":15}}, "shopping":{"details":"Bought camera gear", "metrics":{"items":5, "cost":1500}}}}] +user3 [{"activity_date":"2024-08-01", "activities":{"cooking":{"details":"Made vegan meal", "metrics":{"time_spent":1.5, "calories":500}}, "movie":{"details":"Watched action movie", "metrics":{"duration":2, "rating":8.5}}}}, {"activity_date":"2024-08-02", "activities":{"gym":{"details":"Strength training", "metrics":{"duration":1, "calories":300}}, "shopping":{"details":"Bought groceries", "metrics":{"items":10, "cost":100}}}}] +user4 [{"activity_date":"2024-08-01", "activities":{"cooking":{"details":"Made vegan meal", "metrics":{"time_spent":1.5, "calories":500}}, "movie":{"details":"Watched action movie", "metrics":{"duration":2, "rating":8.5}}}}, {"activity_date":"2024-08-02", "activities":{"gym":{"details":"Strength training", "metrics":{"duration":1, "calories":300}}, "shopping":{"details":"Bought groceries", "metrics":{"items":10, "cost":100}}}}] + +-- !q8 -- +\N \N \N \N +\N \N \N \N +1 10 {"col2a":10, "col2b":"string1"} {1:"string10"} +2 20 {"col2a":20, "col2b":"string2"} {2:"string2"} +3 30 {"col2a":30, "col2b":"string3"} \N +4 40 {"col2a":40, "col2b":"string4"} {4:"string4"} +5 \N \N \N +6 60 {"col2a":60, "col2b":"600"} {6:"600"} +7 70 {"col2a":70, "col2b":"string7"} {7:"string7"} + +-- !q9 -- +\N \N +\N \N +\N 5 +10 1 +20 2 +30 3 +40 4 +60 6 +70 7 + +-- !q1 -- +1 true 127 32767 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value binary_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123457 2024-03-20T12:00:00.123457 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"a ":"b "} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] ["varchar1", "varchar2"] ["char1 ", "char2 "] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint":1234567890} {"key":[{"s_int":123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"struct_non_nulls_after_nulls1":123, "struct_non_nulls_after_nulls2":"value"} {"struct_field1":123, "struct_field2":"value", "strict_field3":{"nested_struct_field1":123, "nested_struct_field2":"nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] dt1 +2 false 58 12345 2147483000 \N 789.56 654321.123 987654321 5678.1234 987654.321098 987654321.098765432109 changed_string new_binary_value 2025-05-25 2025-05-25T15:30 2025-05-25T15:30:00.654322 2025-05-25T15:30:00.654322 char_new_value1 char_new_value2 char_new_value3 varchar_new_value1 varchar_new_value2 varchar_new_value3 {"key2":"value2"} {"key2":"value2"} {"x ":"y "} {2:20} {2:200000000000} {2.2:20.2} {2.2:20.2} {0:1} {2.2:2.2} {2.34:2.34} {2.3456:2.3456} {2.34567891:2.34567891} {2.34567891:2.34567891} {2.3456789012345679:2.3456789012345679} ["string3", "string4"] [4, 5, 6] [300000000000, 400000000000] [2.2, 3.3] [2.123456789, 3.123456789] [0, 1] ["varchar3", "varchar4"] ["char3 ", "char4 "] [2.2, 3.3] [2.34, 3.45] [2.3456, 3.4567] [2.34567891, 3.45678901] [2.34567891, 3.45678901] [2.3456789012345679, 3.4567890123456789] {"s_bigint":9876543210} {"key2":[{"s_int":456}]} {"struct_field":["new_value1", "new_value2"]} {"struct_field_null":null, "struct_field_null2":null} {"struct_non_nulls_after_nulls1":456, "struct_non_nulls_after_nulls2":"new_value"} {"struct_field1":456, "struct_field2":"new_value", "strict_field3":{"nested_struct_field1":456, "nested_struct_field2":"nested_value2"}} {"null_key":null} [null, "new_value1", "new_value2"] ["new_value1", null, "new_value2"] ["new_value1", "new_value2", null] [null, null, null] dt1 +3 false -128 -32768 -2147483648 -9223372036854775808 -3.4028235e+38 -1.7976931348623157E308 -999999999 -9999.9999 -999999999.999999 \N min_string_value xxxx 2001-01-01 2001-01-01T00:00 2001-01-01T00:00 2001-01-01T00:00 char_min_value1 char_min_value2 char_min_value3 varchar_min_value1 varchar_min_value2 varchar_min_value3 {"min_key":"min_value"} {"min_key":"min_value"} {"a ":"z "} {-1:-100} {-1:-100000000000} {-1.1:-10.1} {-1.1:-10.1} {0:1} {-1.1:-1.1} {-1.23:-1.23} {-1.2345:-1.2345} {-1.23456789:-1.23456789} {-1.23456789:-1.23456789} {-1.2345678901234568:-1.2345678901234568} ["min_string1", "min_string2"] [-10, -5, -3] [-100000000000, -200000000000] [-1.1, -2.2] [-1.123456789, -2.123456789] [0, 1] ["min_varchar1", "min_varchar2"] ["min_char1 ", "min_char2 "] [-1.1, -2.2] [-1.23, -2.34] [-1.2345, -2.3456] [-1.23456789, -2.34567891] [-1.23456789, -2.34567891] [-1.2345678901234568, -2.3456789012345679] {"s_bigint":-1234567890} {"min_key":[{"s_int":-123}]} {"struct_field":["min_value1", "min_value2"]} {"struct_field_null":null, "struct_field_null2":null} {"struct_non_nulls_after_nulls1":-123, "struct_non_nulls_after_nulls2":"min_value"} {"struct_field1":-123, "struct_field2":"min_value", "strict_field3":{"nested_struct_field1":-123, "nested_struct_field2":"nested_value"}} {"null_key":null} [null, "min_value1", "min_value2"] ["min_value1", null, "min_value2"] ["min_value1", "min_value2", null] [null, null, null] dt1 +4 \N \N \N \N \N 123.45 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N {1:10} \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N [1.2345, 2.3456] \N \N \N \N \N \N \N \N \N \N [null, "value1", "value2"] \N \N \N dt2 +5 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N dt3 +6 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N dt3 + +-- !q2 -- +3 false -128 -32768 -2147483648 -9223372036854775808 -3.4028235e+38 -1.7976931348623157E308 -999999999 -9999.9999 -999999999.999999 \N min_string_value xxxx 2001-01-01 2001-01-01T00:00 2001-01-01T00:00 2001-01-01T00:00 char_min_value1 char_min_value2 char_min_value3 varchar_min_value1 varchar_min_value2 varchar_min_value3 {"min_key":"min_value"} {"min_key":"min_value"} {"a ":"z "} {-1:-100} {-1:-100000000000} {-1.1:-10.1} {-1.1:-10.1} {0:1} {-1.1:-1.1} {-1.23:-1.23} {-1.2345:-1.2345} {-1.23456789:-1.23456789} {-1.23456789:-1.23456789} {-1.2345678901234568:-1.2345678901234568} ["min_string1", "min_string2"] [-10, -5, -3] [-100000000000, -200000000000] [-1.1, -2.2] [-1.123456789, -2.123456789] [0, 1] ["min_varchar1", "min_varchar2"] ["min_char1 ", "min_char2 "] [-1.1, -2.2] [-1.23, -2.34] [-1.2345, -2.3456] [-1.23456789, -2.34567891] [-1.23456789, -2.34567891] [-1.2345678901234568, -2.3456789012345679] {"s_bigint":-1234567890} {"min_key":[{"s_int":-123}]} {"struct_field":["min_value1", "min_value2"]} {"struct_field_null":null, "struct_field_null2":null} {"struct_non_nulls_after_nulls1":-123, "struct_non_nulls_after_nulls2":"min_value"} {"struct_field1":-123, "struct_field2":"min_value", "strict_field3":{"nested_struct_field1":-123, "nested_struct_field2":"nested_value"}} {"null_key":null} [null, "min_value1", "min_value2"] ["min_value1", null, "min_value2"] ["min_value1", "min_value2", null] [null, null, null] dt1 + +-- !q3 -- +1 true 127 32767 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value binary_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123457 2024-03-20T12:00:00.123457 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"a ":"b "} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] ["varchar1", "varchar2"] ["char1 ", "char2 "] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint":1234567890} {"key":[{"s_int":123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"struct_non_nulls_after_nulls1":123, "struct_non_nulls_after_nulls2":"value"} {"struct_field1":123, "struct_field2":"value", "strict_field3":{"nested_struct_field1":123, "nested_struct_field2":"nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] dt1 + +-- !q4 -- +123.45 +789.56 +-3.4028235e+38 +123.45 + +-- !q5 -- +2 false 58 12345 2147483000 \N 789.56 654321.123 987654321 5678.1234 987654.321098 987654321.098765432109 changed_string new_binary_value 2025-05-25 2025-05-25T15:30 2025-05-25T15:30:00.654322 2025-05-25T15:30:00.654322 char_new_value1 char_new_value2 char_new_value3 varchar_new_value1 varchar_new_value2 varchar_new_value3 {"key2":"value2"} {"key2":"value2"} {"x ":"y "} {2:20} {2:200000000000} {2.2:20.2} {2.2:20.2} {0:1} {2.2:2.2} {2.34:2.34} {2.3456:2.3456} {2.34567891:2.34567891} {2.34567891:2.34567891} {2.3456789012345679:2.3456789012345679} ["string3", "string4"] [4, 5, 6] [300000000000, 400000000000] [2.2, 3.3] [2.123456789, 3.123456789] [0, 1] ["varchar3", "varchar4"] ["char3 ", "char4 "] [2.2, 3.3] [2.34, 3.45] [2.3456, 3.4567] [2.34567891, 3.45678901] [2.34567891, 3.45678901] [2.3456789012345679, 3.4567890123456789] {"s_bigint":9876543210} {"key2":[{"s_int":456}]} {"struct_field":["new_value1", "new_value2"]} {"struct_field_null":null, "struct_field_null2":null} {"struct_non_nulls_after_nulls1":456, "struct_non_nulls_after_nulls2":"new_value"} {"struct_field1":456, "struct_field2":"new_value", "strict_field3":{"nested_struct_field1":456, "nested_struct_field2":"nested_value2"}} {"null_key":null} [null, "new_value1", "new_value2"] ["new_value1", null, "new_value2"] ["new_value1", "new_value2", null] [null, null, null] dt1 + +-- !q6 -- +user1 {"name":"Alice", "age":28, "preferences":{"sports":{"preference_id":101, "preference_values":["soccer", "tennis"]}, "music":{"preference_id":102, "preference_values":["rock", "classical"]}}} [{"activity_date":"2024-08-01", "activities":{"workout":{"details":"Morning run", "metrics":{"duration":30.5, "calories":200}}, "reading":{"details":"Read book on Hive", "metrics":{"pages":50, "time":2}}}}, {"activity_date":"2024-08-02", "activities":{"travel":{"details":"Flight to NY", "metrics":{"distance":500, "time":3}}, "meeting":{"details":"Project meeting", "metrics":{"duration":1.5, "participants":5}}}}] +user2 {"name":"Bob", "age":32, "preferences":{"books":{"preference_id":201, "preference_values":["fiction", "non-fiction"]}, "travel":{"preference_id":202, "preference_values":["beaches", "mountains"]}}} [{"activity_date":"2024-08-01", "activities":{"hiking":{"details":"Mountain trail", "metrics":{"distance":10, "elevation":500}}, "photography":{"details":"Wildlife photoshoot", "metrics":{"photos_taken":100, "time":4}}}}, {"activity_date":"2024-08-02", "activities":{"workshop":{"details":"Photography workshop", "metrics":{"duration":3, "participants":15}}, "shopping":{"details":"Bought camera gear", "metrics":{"items":5, "cost":1500}}}}] +user3 {"name":"Carol", "age":24, "preferences":{"food":{"preference_id":301, "preference_values":["vegan", "desserts"]}, "movies":{"preference_id":302, "preference_values":["action", "comedy"]}}} [{"activity_date":"2024-08-01", "activities":{"cooking":{"details":"Made vegan meal", "metrics":{"time_spent":1.5, "calories":500}}, "movie":{"details":"Watched action movie", "metrics":{"duration":2, "rating":8.5}}}}, {"activity_date":"2024-08-02", "activities":{"gym":{"details":"Strength training", "metrics":{"duration":1, "calories":300}}, "shopping":{"details":"Bought groceries", "metrics":{"items":10, "cost":100}}}}] +user4 {"name":"Carol", "age":24, "preferences":{"food":{"preference_id":301, "preference_values":["vegan", "desserts"]}, "movies":{"preference_id":302, "preference_values":["action", "comedy"]}}} [{"activity_date":"2024-08-01", "activities":{"cooking":{"details":"Made vegan meal", "metrics":{"time_spent":1.5, "calories":500}}, "movie":{"details":"Watched action movie", "metrics":{"duration":2, "rating":8.5}}}}, {"activity_date":"2024-08-02", "activities":{"gym":{"details":"Strength training", "metrics":{"duration":1, "calories":300}}, "shopping":{"details":"Bought groceries", "metrics":{"items":10, "cost":100}}}}] + +-- !q7 -- +user1 [{"activity_date":"2024-08-01", "activities":{"workout":{"details":"Morning run", "metrics":{"duration":30.5, "calories":200}}, "reading":{"details":"Read book on Hive", "metrics":{"pages":50, "time":2}}}}, {"activity_date":"2024-08-02", "activities":{"travel":{"details":"Flight to NY", "metrics":{"distance":500, "time":3}}, "meeting":{"details":"Project meeting", "metrics":{"duration":1.5, "participants":5}}}}] +user2 [{"activity_date":"2024-08-01", "activities":{"hiking":{"details":"Mountain trail", "metrics":{"distance":10, "elevation":500}}, "photography":{"details":"Wildlife photoshoot", "metrics":{"photos_taken":100, "time":4}}}}, {"activity_date":"2024-08-02", "activities":{"workshop":{"details":"Photography workshop", "metrics":{"duration":3, "participants":15}}, "shopping":{"details":"Bought camera gear", "metrics":{"items":5, "cost":1500}}}}] +user3 [{"activity_date":"2024-08-01", "activities":{"cooking":{"details":"Made vegan meal", "metrics":{"time_spent":1.5, "calories":500}}, "movie":{"details":"Watched action movie", "metrics":{"duration":2, "rating":8.5}}}}, {"activity_date":"2024-08-02", "activities":{"gym":{"details":"Strength training", "metrics":{"duration":1, "calories":300}}, "shopping":{"details":"Bought groceries", "metrics":{"items":10, "cost":100}}}}] +user4 [{"activity_date":"2024-08-01", "activities":{"cooking":{"details":"Made vegan meal", "metrics":{"time_spent":1.5, "calories":500}}, "movie":{"details":"Watched action movie", "metrics":{"duration":2, "rating":8.5}}}}, {"activity_date":"2024-08-02", "activities":{"gym":{"details":"Strength training", "metrics":{"duration":1, "calories":300}}, "shopping":{"details":"Bought groceries", "metrics":{"items":10, "cost":100}}}}] + +-- !q8 -- +\N \N \N \N +\N \N \N \N +1 10 {"col2a":10, "col2b":"string1"} {1:"string10"} +2 20 {"col2a":20, "col2b":"string2"} {2:"string2"} +3 30 {"col2a":30, "col2b":"string3"} \N +4 40 {"col2a":40, "col2b":"string4"} {4:"string4"} +5 \N \N \N +6 60 {"col2a":60, "col2b":"600"} {6:"600"} +7 70 {"col2a":70, "col2b":"string7"} {7:"string7"} + +-- !q9 -- +\N \N +\N \N +\N 5 +10 1 +20 2 +30 3 +40 4 +60 6 +70 7 + diff --git a/regression-test/suites/external_table_p0/hive/hive_json_basic_test.groovy b/regression-test/suites/external_table_p0/hive/hive_json_basic_test.groovy new file mode 100644 index 00000000000000..9d05e1a4c7403d --- /dev/null +++ b/regression-test/suites/external_table_p0/hive/hive_json_basic_test.groovy @@ -0,0 +1,71 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("hive_json_basic_test", "p0,external,hive,external_docker,external_docker_hive") { + + + String enabled = context.config.otherConfigs.get("enableHiveTest") + if (enabled == null || !enabled.equalsIgnoreCase("true")) { + logger.info("diable Hive test.") + return; + } + + for (String hivePrefix : ["hive2", "hive3"]) { + try { + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + String hms_port = context.config.otherConfigs.get(hivePrefix + "HmsPort") + String catalog_name = "${hivePrefix}_hive_json_basic_test" + String broker_name = "hdfs" + + sql """drop catalog if exists ${catalog_name}""" + sql """create catalog if not exists ${catalog_name} properties ( + 'type'='hms', + 'hive.metastore.uris'='thrift://${externalEnvIp}:${hms_port}' + );""" + sql """use `${catalog_name}`.`default`""" + + String tb1 = """json_all_complex_types""" + String tb2 = """json_nested_complex_table""" + String tb3 = """json_load_data_table""" + + def tables = sql """ show tables """ + logger.info("tables = ${tables}") + + qt_q1 """ select * from ${tb1} order by id """ + qt_q2 """ select * from ${tb1} where tinyint_col < 0 order by id """ + qt_q3 """ select * from ${tb1} where bigint_col > 0 order by id """ + qt_q4 """ select float_col from ${tb1} where float_col is not null order by id """ + qt_q5 """ select * from ${tb1} where id = 2 order by id """ + + + + qt_q6 """ select * from ${tb2} order by user_id""" + qt_q7 """ select user_id,activity_log from ${tb2} order by user_id""" + + + order_qt_q8 """ select * from ${tb3} order by id """ + + order_qt_q9 """ select col1,id from ${tb3} order by id """ + + + + + sql """drop catalog if exists ${catalog_name}""" + } finally { + } + } +} From 839208cbdda0b2531acb866da2ada98f0598e04e Mon Sep 17 00:00:00 2001 From: yujun Date: Thu, 28 Nov 2024 22:43:33 +0800 Subject: [PATCH 068/399] [fix](nereids) fix UnknownValue's reference in simplify range rule (#44637) for sql 'a > 10 and b > 20' will generate an UnknowValue with sourceValues = [ Range(a > 10), Range(b > 20) ]. in UnknowValue's constructor, it set reference = sourceValues[0].reference, then its reference = a. This is an error. when A union/interset B generating an UnknowValue, this UnknownValue's reference should be: 1. A.reference if A.reference = B.reference; 2. expr(A union/interset B) if A.reference != B.reference; --- .../rules/expression/rules/SimplifyRange.java | 235 ++++++++++-------- .../rules/expression/SimplifyRangeTest.java | 4 + .../nereids/rules/rewrite/OrToInTest.java | 2 +- .../nereids_hint_tpcds_p0/shape/query41.out | 2 +- .../nereids_hint_tpcds_p0/shape/query47.out | 2 +- .../nereids_hint_tpcds_p0/shape/query88.out | 16 +- .../shape/query41.out | 2 +- .../shape/query47.out | 2 +- .../shape/query88.out | 16 +- .../noStatsRfPrune/query41.out | 2 +- .../noStatsRfPrune/query57.out | 2 +- .../no_stats_shape/query41.out | 2 +- .../no_stats_shape/query57.out | 2 +- .../rf_prune/query41.out | 2 +- .../rf_prune/query57.out | 2 +- .../shape/query41.out | 2 +- .../shape/query57.out | 2 +- .../shape/query41.out | 2 +- .../shape/query47.out | 2 +- .../shape/query57.out | 2 +- .../tpcds_sf100/noStatsRfPrune/query41.out | 2 +- .../tpcds_sf100/noStatsRfPrune/query57.out | 2 +- .../tpcds_sf100/no_stats_shape/query41.out | 2 +- .../tpcds_sf100/no_stats_shape/query57.out | 2 +- .../tpcds_sf100/rf_prune/query41.out | 2 +- .../tpcds_sf100/rf_prune/query57.out | 2 +- .../tpcds_sf100/shape/query41.out | 2 +- .../tpcds_sf100/shape/query57.out | 2 +- .../tpcds_sf1000/shape/query41.out | 2 +- .../tpcds_sf1000/shape/query47.out | 2 +- .../tpcds_sf1000/shape/query88.out | 16 +- .../expression/test_simplify_range.groovy | 28 +++ 32 files changed, 211 insertions(+), 156 deletions(-) create mode 100644 regression-test/suites/nereids_rules_p0/expression/test_simplify_range.groovy diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifyRange.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifyRange.java index 297f11abd0e00e..ca50f7c4d49225 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifyRange.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifyRange.java @@ -92,12 +92,12 @@ public List> buildRules() { /** rewrite */ public static Expression rewrite(CompoundPredicate expr, ExpressionRewriteContext context) { ValueDesc valueDesc = expr.accept(new RangeInference(), context); - Expression exprForNonNull = valueDesc.toExpressionForNonNull(); - if (exprForNonNull == null) { + Expression toExpr = valueDesc.toExpression(); + if (toExpr == null) { // this mean cannot simplify - return valueDesc.exprForNonNull; + return valueDesc.toExpr; } - return exprForNonNull; + return toExpr; } private static class RangeInference extends ExpressionVisitor { @@ -197,18 +197,18 @@ private ValueDesc simplify(ExpressionRewriteContext context, } // use UnknownValue to wrap different references - return new UnknownValue(context, valuePerRefs, originExpr, exprOp); + return new UnknownValue(context, originExpr, valuePerRefs, exprOp); } } private abstract static class ValueDesc { ExpressionRewriteContext context; - Expression exprForNonNull; + Expression toExpr; Expression reference; - public ValueDesc(ExpressionRewriteContext context, Expression reference, Expression exprForNonNull) { + public ValueDesc(ExpressionRewriteContext context, Expression reference, Expression toExpr) { this.context = context; - this.exprForNonNull = exprForNonNull; + this.toExpr = toExpr; this.reference = reference; } @@ -220,28 +220,28 @@ public static ValueDesc union(ExpressionRewriteContext context, if (count == discrete.values.size()) { return range; } - Expression exprForNonNull = FoldConstantRuleOnFE.evaluate( - ExpressionUtils.or(range.exprForNonNull, discrete.exprForNonNull), context); + Expression toExpr = FoldConstantRuleOnFE.evaluate( + ExpressionUtils.or(range.toExpr, discrete.toExpr), context); List sourceValues = reverseOrder ? ImmutableList.of(discrete, range) : ImmutableList.of(range, discrete); - return new UnknownValue(context, sourceValues, exprForNonNull, ExpressionUtils::or); + return new UnknownValue(context, toExpr, sourceValues, ExpressionUtils::or); } public abstract ValueDesc intersect(ValueDesc other); public static ValueDesc intersect(ExpressionRewriteContext context, RangeValue range, DiscreteValue discrete) { - DiscreteValue result = new DiscreteValue(context, discrete.reference, discrete.exprForNonNull); + DiscreteValue result = new DiscreteValue(context, discrete.reference, discrete.toExpr); discrete.values.stream().filter(x -> range.range.contains(x)).forEach(result.values::add); if (!result.values.isEmpty()) { return result; } - Expression originExprForNonNull = FoldConstantRuleOnFE.evaluate( - ExpressionUtils.and(range.exprForNonNull, discrete.exprForNonNull), context); - return new EmptyValue(context, range.reference, originExprForNonNull); + Expression originExpr = FoldConstantRuleOnFE.evaluate( + ExpressionUtils.and(range.toExpr, discrete.toExpr), context); + return new EmptyValue(context, range.reference, originExpr); } - public abstract Expression toExpressionForNonNull(); + public abstract Expression toExpression(); public static ValueDesc range(ExpressionRewriteContext context, ComparisonPredicate predicate) { Literal value = (Literal) predicate.right(); @@ -271,8 +271,8 @@ public static ValueDesc discrete(ExpressionRewriteContext context, InPredicate i private static class EmptyValue extends ValueDesc { - public EmptyValue(ExpressionRewriteContext context, Expression reference, Expression exprForNonNull) { - super(context, reference, exprForNonNull); + public EmptyValue(ExpressionRewriteContext context, Expression reference, Expression toExpr) { + super(context, reference, toExpr); } @Override @@ -286,7 +286,7 @@ public ValueDesc intersect(ValueDesc other) { } @Override - public Expression toExpressionForNonNull() { + public Expression toExpression() { if (reference.nullable()) { return new And(new IsNull(reference), new NullLiteral(BooleanType.INSTANCE)); } else { @@ -303,8 +303,8 @@ public Expression toExpressionForNonNull() { private static class RangeValue extends ValueDesc { Range range; - public RangeValue(ExpressionRewriteContext context, Expression reference, Expression exprForNonNull) { - super(context, reference, exprForNonNull); + public RangeValue(ExpressionRewriteContext context, Expression reference, Expression toExpr) { + super(context, reference, toExpr); } @Override @@ -312,26 +312,25 @@ public ValueDesc union(ValueDesc other) { if (other instanceof EmptyValue) { return other.union(this); } - try { - if (other instanceof RangeValue) { - Expression originExprForNonNull = FoldConstantRuleOnFE.evaluate( - ExpressionUtils.or(exprForNonNull, other.exprForNonNull), context); - RangeValue o = (RangeValue) other; - if (range.isConnected(o.range)) { - RangeValue rangeValue = new RangeValue(context, reference, originExprForNonNull); - rangeValue.range = range.span(o.range); - return rangeValue; - } - return new UnknownValue(context, ImmutableList.of(this, other), - originExprForNonNull, ExpressionUtils::or); + if (other instanceof RangeValue) { + Expression originExpr = FoldConstantRuleOnFE.evaluate( + ExpressionUtils.or(toExpr, other.toExpr), context); + RangeValue o = (RangeValue) other; + if (range.isConnected(o.range)) { + RangeValue rangeValue = new RangeValue(context, reference, originExpr); + rangeValue.range = range.span(o.range); + return rangeValue; } + return new UnknownValue(context, originExpr, + ImmutableList.of(this, other), ExpressionUtils::or); + } + if (other instanceof DiscreteValue) { return union(context, this, (DiscreteValue) other, false); - } catch (Exception e) { - Expression originExprForNonNull = FoldConstantRuleOnFE.evaluate( - ExpressionUtils.or(exprForNonNull, other.exprForNonNull), context); - return new UnknownValue(context, ImmutableList.of(this, other), - originExprForNonNull, ExpressionUtils::or); } + Expression originExpr = FoldConstantRuleOnFE.evaluate( + ExpressionUtils.or(toExpr, other.toExpr), context); + return new UnknownValue(context, originExpr, + ImmutableList.of(this, other), ExpressionUtils::or); } @Override @@ -339,29 +338,28 @@ public ValueDesc intersect(ValueDesc other) { if (other instanceof EmptyValue) { return other.intersect(this); } - try { - if (other instanceof RangeValue) { - Expression originExprForNonNull = FoldConstantRuleOnFE.evaluate( - ExpressionUtils.and(exprForNonNull, other.exprForNonNull), context); - RangeValue o = (RangeValue) other; - if (range.isConnected(o.range)) { - RangeValue rangeValue = new RangeValue(context, reference, originExprForNonNull); - rangeValue.range = range.intersection(o.range); - return rangeValue; - } - return new EmptyValue(context, reference, originExprForNonNull); + if (other instanceof RangeValue) { + Expression originExpr = FoldConstantRuleOnFE.evaluate( + ExpressionUtils.and(toExpr, other.toExpr), context); + RangeValue o = (RangeValue) other; + if (range.isConnected(o.range)) { + RangeValue rangeValue = new RangeValue(context, reference, originExpr); + rangeValue.range = range.intersection(o.range); + return rangeValue; } + return new EmptyValue(context, reference, originExpr); + } + if (other instanceof DiscreteValue) { return intersect(context, this, (DiscreteValue) other); - } catch (Exception e) { - Expression originExprForNonNull = FoldConstantRuleOnFE.evaluate( - ExpressionUtils.and(exprForNonNull, other.exprForNonNull), context); - return new UnknownValue(context, ImmutableList.of(this, other), - originExprForNonNull, ExpressionUtils::and); } + Expression originExpr = FoldConstantRuleOnFE.evaluate( + ExpressionUtils.and(toExpr, other.toExpr), context); + return new UnknownValue(context, originExpr, + ImmutableList.of(this, other), ExpressionUtils::and); } @Override - public Expression toExpressionForNonNull() { + public Expression toExpression() { List result = Lists.newArrayList(); if (range.hasLowerBound()) { if (range.lowerBoundType() == BoundType.CLOSED) { @@ -403,13 +401,13 @@ private static class DiscreteValue extends ValueDesc { Set values; public DiscreteValue(ExpressionRewriteContext context, - Expression reference, Expression exprForNonNull, Literal... values) { - this(context, reference, exprForNonNull, Arrays.asList(values)); + Expression reference, Expression toExpr, Literal... values) { + this(context, reference, toExpr, Arrays.asList(values)); } public DiscreteValue(ExpressionRewriteContext context, - Expression reference, Expression exprForNonNull, Collection values) { - super(context, reference, exprForNonNull); + Expression reference, Expression toExpr, Collection values) { + super(context, reference, toExpr); this.values = Sets.newTreeSet(values); } @@ -418,22 +416,21 @@ public ValueDesc union(ValueDesc other) { if (other instanceof EmptyValue) { return other.union(this); } - try { - if (other instanceof DiscreteValue) { - Expression originExprForNonNull = FoldConstantRuleOnFE.evaluate( - ExpressionUtils.or(exprForNonNull, other.exprForNonNull), context); - DiscreteValue discreteValue = new DiscreteValue(context, reference, originExprForNonNull); - discreteValue.values.addAll(((DiscreteValue) other).values); - discreteValue.values.addAll(this.values); - return discreteValue; - } + if (other instanceof DiscreteValue) { + Expression originExpr = FoldConstantRuleOnFE.evaluate( + ExpressionUtils.or(toExpr, other.toExpr), context); + DiscreteValue discreteValue = new DiscreteValue(context, reference, originExpr); + discreteValue.values.addAll(((DiscreteValue) other).values); + discreteValue.values.addAll(this.values); + return discreteValue; + } + if (other instanceof RangeValue) { return union(context, (RangeValue) other, this, true); - } catch (Exception e) { - Expression originExprForNonNull = FoldConstantRuleOnFE.evaluate( - ExpressionUtils.or(exprForNonNull, other.exprForNonNull), context); - return new UnknownValue(context, ImmutableList.of(this, other), - originExprForNonNull, ExpressionUtils::or); } + Expression originExpr = FoldConstantRuleOnFE.evaluate( + ExpressionUtils.or(toExpr, other.toExpr), context); + return new UnknownValue(context, originExpr, + ImmutableList.of(this, other), ExpressionUtils::or); } @Override @@ -441,30 +438,29 @@ public ValueDesc intersect(ValueDesc other) { if (other instanceof EmptyValue) { return other.intersect(this); } - try { - if (other instanceof DiscreteValue) { - Expression originExprForNonNull = FoldConstantRuleOnFE.evaluate( - ExpressionUtils.and(exprForNonNull, other.exprForNonNull), context); - DiscreteValue discreteValue = new DiscreteValue(context, reference, originExprForNonNull); - discreteValue.values.addAll(((DiscreteValue) other).values); - discreteValue.values.retainAll(this.values); - if (discreteValue.values.isEmpty()) { - return new EmptyValue(context, reference, originExprForNonNull); - } else { - return discreteValue; - } + if (other instanceof DiscreteValue) { + Expression originExpr = FoldConstantRuleOnFE.evaluate( + ExpressionUtils.and(toExpr, other.toExpr), context); + DiscreteValue discreteValue = new DiscreteValue(context, reference, originExpr); + discreteValue.values.addAll(((DiscreteValue) other).values); + discreteValue.values.retainAll(this.values); + if (discreteValue.values.isEmpty()) { + return new EmptyValue(context, reference, originExpr); + } else { + return discreteValue; } + } + if (other instanceof RangeValue) { return intersect(context, (RangeValue) other, this); - } catch (Exception e) { - Expression originExprForNonNull = FoldConstantRuleOnFE.evaluate( - ExpressionUtils.and(exprForNonNull, other.exprForNonNull), context); - return new UnknownValue(context, ImmutableList.of(this, other), - originExprForNonNull, ExpressionUtils::and); } + Expression originExpr = FoldConstantRuleOnFE.evaluate( + ExpressionUtils.and(toExpr, other.toExpr), context); + return new UnknownValue(context, originExpr, + ImmutableList.of(this, other), ExpressionUtils::and); } @Override - public Expression toExpressionForNonNull() { + public Expression toExpression() { // NOTICE: it's related with `InPredicateToEqualToRule` // They are same processes, so must change synchronously. if (values.size() == 1) { @@ -498,41 +494,68 @@ private UnknownValue(ExpressionRewriteContext context, Expression expr) { mergeExprOp = null; } - public UnknownValue(ExpressionRewriteContext context, - List sourceValues, Expression exprForNonNull, BinaryOperator mergeExprOp) { - super(context, sourceValues.get(0).reference, exprForNonNull); + public UnknownValue(ExpressionRewriteContext context, Expression toExpr, + List sourceValues, BinaryOperator mergeExprOp) { + super(context, getReference(sourceValues, toExpr), toExpr); this.sourceValues = ImmutableList.copyOf(sourceValues); this.mergeExprOp = mergeExprOp; } + // reference is used to simplify multiple ValueDescs. + // when ValueDesc A op ValueDesc B, only A and B's references equals, + // can reduce them, like A op B = A. + // If A and B's reference not equal, A op B will always get UnknownValue(A op B). + // + // for example: + // 1. RangeValue(a < 10, reference=a) union RangeValue(a > 20, reference=a) + // = UnknownValue1(a < 10 or a > 20, reference=a) + // 2. RangeValue(a < 10, reference=a) union RangeValue(b > 20, reference=b) + // = UnknownValue2(a < 10 or b > 20, reference=(a < 10 or b > 20)) + // then given EmptyValue(, reference=a) E, + // 1. since E and UnknownValue1's reference equals, then + // E union UnknownValue1 = E.union(UnknownValue1) = UnknownValue1, + // 2. since E and UnknownValue2's reference not equals, then + // E union UnknownValue2 = UnknownValue3(E union UnknownValue2, reference=E union UnknownValue2) + private static Expression getReference(List sourceValues, Expression toExpr) { + Expression reference = sourceValues.get(0).reference; + for (int i = 1; i < sourceValues.size(); i++) { + if (!reference.equals(sourceValues.get(i).reference)) { + return toExpr; + } + } + return reference; + } + @Override public ValueDesc union(ValueDesc other) { - Expression originExprForNonNull = FoldConstantRuleOnFE.evaluate( - ExpressionUtils.or(exprForNonNull, other.exprForNonNull), context); - return new UnknownValue(context, ImmutableList.of(this, other), originExprForNonNull, ExpressionUtils::or); + Expression originExpr = FoldConstantRuleOnFE.evaluate( + ExpressionUtils.or(toExpr, other.toExpr), context); + return new UnknownValue(context, originExpr, + ImmutableList.of(this, other), ExpressionUtils::or); } @Override public ValueDesc intersect(ValueDesc other) { - Expression originExprForNonNull = FoldConstantRuleOnFE.evaluate( - ExpressionUtils.and(exprForNonNull, other.exprForNonNull), context); - return new UnknownValue(context, ImmutableList.of(this, other), originExprForNonNull, ExpressionUtils::and); + Expression originExpr = FoldConstantRuleOnFE.evaluate( + ExpressionUtils.and(toExpr, other.toExpr), context); + return new UnknownValue(context, originExpr, + ImmutableList.of(this, other), ExpressionUtils::and); } @Override - public Expression toExpressionForNonNull() { + public Expression toExpression() { if (sourceValues.isEmpty()) { - return exprForNonNull; + return toExpr; } - Expression result = sourceValues.get(0).toExpressionForNonNull(); + Expression result = sourceValues.get(0).toExpression(); for (int i = 1; i < sourceValues.size(); i++) { - result = mergeExprOp.apply(result, sourceValues.get(i).toExpressionForNonNull()); + result = mergeExprOp.apply(result, sourceValues.get(i).toExpression()); } result = FoldConstantRuleOnFE.evaluate(result, context); // ATTN: we must return original expr, because OrToIn is implemented with MutableState, // newExpr will lose these states leading to dead loop by OrToIn -> SimplifyRange -> FoldConstantByFE - if (result.equals(exprForNonNull)) { - return exprForNonNull; + if (result.equals(toExpr)) { + return toExpr; } return result; } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/SimplifyRangeTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/SimplifyRangeTest.java index 329efee77a8f57..d6a7b9e5585d14 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/SimplifyRangeTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/SimplifyRangeTest.java @@ -101,6 +101,7 @@ public void testSimplify() { assertRewrite("TA > 5 + 1 and TA > 10", "cast(TA as smallint) > 6 and TA > 10"); assertRewrite("(TA > 1 and TA > 10) or TA > 20", "TA > 10"); assertRewrite("(TA > 1 or TA > 10) and TA > 20", "TA > 20"); + assertRewrite("(TA < 1 and TA > 10) or TA = 20 and TB > 10", "(TA is null and null) or TA = 20 and TB > 10"); assertRewrite("(TA + TB > 1 or TA + TB > 10) and TA + TB > 20", "TA + TB > 20"); assertRewrite("TA > 10 or TA > 10", "TA > 10"); assertRewrite("(TA > 10 or TA > 20) and (TB > 10 and TB < 20)", "TA > 10 and (TB > 10 and TB < 20) "); @@ -131,7 +132,10 @@ public void testSimplify() { assertRewrite("TA in (1) and TA in (3)", "TA is null and null"); assertRewrite("TA in (1) and TA in (1)", "TA = 1"); assertRewriteNotNull("(TA > 3 and TA < 1) and TB < 5", "FALSE"); + assertRewrite("(TA > 3 and TA < 1) and (TA > 5 and TA = 4)", "TA is null and null"); + assertRewrite("(TA > 3 and TA < 1) or (TA > 5 and TA = 4)", "TA is null and null"); assertRewrite("(TA > 3 and TA < 1) and TB < 5", "TA is null and null and TB < 5"); + assertRewrite("(TA > 3 and TA < 1) and (TB < 5 and TB = 6)", "TA is null and null and TB is null"); assertRewrite("TA > 3 and TB < 5 and TA < 1", "TA is null and null and TB < 5"); assertRewrite("(TA > 3 and TA < 1) or TB < 5", "(TA is null and null) or TB < 5"); assertRewrite("((IA = 1 AND SC ='1') OR SC = '1212') AND IA =1", "((IA = 1 AND SC ='1') OR SC = '1212') AND IA =1"); diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/OrToInTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/OrToInTest.java index aaca3edf101949..5ce7871db144bc 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/OrToInTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/OrToInTest.java @@ -193,7 +193,7 @@ void test15() { String expr = "x=1 or (a=1 and b=2) or (a=2 and c=3)"; Expression expression = PARSER.parseExpression(expr); Expression rewritten = OrToIn.INSTANCE.rewriteTree(expression, context); - Assertions.assertEquals("((x = 1) OR (((a = 1) AND (b = 2)) OR ((a = 2) AND (c = 3))))", + Assertions.assertEquals("(((x = 1) OR ((a = 1) AND (b = 2))) OR ((a = 2) AND (c = 3)))", rewritten.toSql()); } diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query41.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query41.out index c870e728a39819..ea5acccd883550 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query41.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query41.out @@ -18,6 +18,6 @@ PhysicalResultSink ------------------------PhysicalDistribute[DistributionSpecHash] --------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------filter((((i_color IN ('forest', 'lime', 'maroon', 'navy', 'powder', 'sky', 'slate', 'smoke') AND i_units IN ('Bunch', 'Case', 'Dozen', 'Gross', 'Lb', 'Ounce', 'Pallet', 'Pound')) AND ((((((item.i_category = 'Women') AND i_color IN ('forest', 'lime')) AND i_units IN ('Pallet', 'Pound')) AND i_size IN ('economy', 'small')) OR ((((item.i_category = 'Women') AND i_color IN ('navy', 'slate')) AND i_units IN ('Bunch', 'Gross')) AND i_size IN ('extra large', 'petite'))) OR (((((item.i_category = 'Men') AND i_color IN ('powder', 'sky')) AND i_units IN ('Dozen', 'Lb')) AND i_size IN ('N/A', 'large')) OR ((((item.i_category = 'Men') AND i_color IN ('maroon', 'smoke')) AND i_units IN ('Case', 'Ounce')) AND i_size IN ('economy', 'small'))))) OR ((i_color IN ('aquamarine', 'dark', 'firebrick', 'frosted', 'papaya', 'peach', 'plum', 'sienna') AND i_units IN ('Box', 'Bundle', 'Carton', 'Cup', 'Dram', 'Each', 'Tbl', 'Ton')) AND ((((((item.i_category = 'Women') AND i_color IN ('aquamarine', 'dark')) AND i_units IN ('Tbl', 'Ton')) AND i_size IN ('economy', 'small')) OR ((((item.i_category = 'Women') AND i_color IN ('frosted', 'plum')) AND i_units IN ('Box', 'Dram')) AND i_size IN ('extra large', 'petite'))) OR (((((item.i_category = 'Men') AND i_color IN ('papaya', 'peach')) AND i_units IN ('Bundle', 'Carton')) AND i_size IN ('N/A', 'large')) OR ((((item.i_category = 'Men') AND i_color IN ('firebrick', 'sienna')) AND i_units IN ('Cup', 'Each')) AND i_size IN ('economy', 'small')))))) and i_category IN ('Men', 'Women') and i_size IN ('N/A', 'economy', 'extra large', 'large', 'petite', 'small')) +------------------------------filter((((i_color IN ('forest', 'lime', 'maroon', 'navy', 'powder', 'sky', 'slate', 'smoke') AND i_units IN ('Bunch', 'Case', 'Dozen', 'Gross', 'Lb', 'Ounce', 'Pallet', 'Pound')) AND (((((((item.i_category = 'Women') AND i_color IN ('forest', 'lime')) AND i_units IN ('Pallet', 'Pound')) AND i_size IN ('economy', 'small')) OR ((((item.i_category = 'Women') AND i_color IN ('navy', 'slate')) AND i_units IN ('Bunch', 'Gross')) AND i_size IN ('extra large', 'petite'))) OR ((((item.i_category = 'Men') AND i_color IN ('powder', 'sky')) AND i_units IN ('Dozen', 'Lb')) AND i_size IN ('N/A', 'large'))) OR ((((item.i_category = 'Men') AND i_color IN ('maroon', 'smoke')) AND i_units IN ('Case', 'Ounce')) AND i_size IN ('economy', 'small')))) OR ((i_color IN ('aquamarine', 'dark', 'firebrick', 'frosted', 'papaya', 'peach', 'plum', 'sienna') AND i_units IN ('Box', 'Bundle', 'Carton', 'Cup', 'Dram', 'Each', 'Tbl', 'Ton')) AND (((((((item.i_category = 'Women') AND i_color IN ('aquamarine', 'dark')) AND i_units IN ('Tbl', 'Ton')) AND i_size IN ('economy', 'small')) OR ((((item.i_category = 'Women') AND i_color IN ('frosted', 'plum')) AND i_units IN ('Box', 'Dram')) AND i_size IN ('extra large', 'petite'))) OR ((((item.i_category = 'Men') AND i_color IN ('papaya', 'peach')) AND i_units IN ('Bundle', 'Carton')) AND i_size IN ('N/A', 'large'))) OR ((((item.i_category = 'Men') AND i_color IN ('firebrick', 'sienna')) AND i_units IN ('Cup', 'Each')) AND i_size IN ('economy', 'small'))))) and i_category IN ('Men', 'Women') and i_size IN ('N/A', 'economy', 'extra large', 'large', 'petite', 'small')) --------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query47.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query47.out index 2bc34ca1076a83..5554797a7b9dc3 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query47.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query47.out @@ -21,7 +21,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------------------------PhysicalProject --------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 ------------------------------------PhysicalProject ---------------------------------------filter((((date_dim.d_year = 2000) OR ((date_dim.d_year = 1999) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2001) AND (date_dim.d_moy = 1))) and d_year IN (1999, 2000, 2001) and d_year IN (1999, 2000, 2001)) +--------------------------------------filter((((date_dim.d_year = 2000) OR ((date_dim.d_year = 1999) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2001) AND (date_dim.d_moy = 1))) and d_year IN (1999, 2000, 2001)) ----------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query88.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query88.out index 01644900894142..49e27f92b64a61 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query88.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query88.out @@ -24,7 +24,7 @@ PhysicalResultSink --------------------------------------filter((time_dim.t_hour = 8) and (time_dim.t_minute >= 30)) ----------------------------------------PhysicalOlapScan[time_dim] --------------------------------PhysicalProject -----------------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3) and hd_dep_count IN (-1, 0, 3)) +----------------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3)) ------------------------------------PhysicalOlapScan[household_demographics] ----------------------------PhysicalProject ------------------------------filter((store.s_store_name = 'ese')) @@ -45,7 +45,7 @@ PhysicalResultSink --------------------------------------filter((time_dim.t_hour = 9) and (time_dim.t_minute < 30)) ----------------------------------------PhysicalOlapScan[time_dim] --------------------------------PhysicalProject -----------------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3) and hd_dep_count IN (-1, 0, 3)) +----------------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3)) ------------------------------------PhysicalOlapScan[household_demographics] ----------------------------PhysicalProject ------------------------------filter((store.s_store_name = 'ese')) @@ -66,7 +66,7 @@ PhysicalResultSink ------------------------------------filter((time_dim.t_hour = 9) and (time_dim.t_minute >= 30)) --------------------------------------PhysicalOlapScan[time_dim] ------------------------------PhysicalProject ---------------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3) and hd_dep_count IN (-1, 0, 3)) +--------------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3)) ----------------------------------PhysicalOlapScan[household_demographics] --------------------------PhysicalProject ----------------------------filter((store.s_store_name = 'ese')) @@ -87,7 +87,7 @@ PhysicalResultSink ----------------------------------filter((time_dim.t_hour = 10) and (time_dim.t_minute < 30)) ------------------------------------PhysicalOlapScan[time_dim] ----------------------------PhysicalProject -------------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3) and hd_dep_count IN (-1, 0, 3)) +------------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3)) --------------------------------PhysicalOlapScan[household_demographics] ------------------------PhysicalProject --------------------------filter((store.s_store_name = 'ese')) @@ -108,7 +108,7 @@ PhysicalResultSink --------------------------------filter((time_dim.t_hour = 10) and (time_dim.t_minute >= 30)) ----------------------------------PhysicalOlapScan[time_dim] --------------------------PhysicalProject -----------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3) and hd_dep_count IN (-1, 0, 3)) +----------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3)) ------------------------------PhysicalOlapScan[household_demographics] ----------------------PhysicalProject ------------------------filter((store.s_store_name = 'ese')) @@ -129,7 +129,7 @@ PhysicalResultSink ------------------------------filter((time_dim.t_hour = 11) and (time_dim.t_minute < 30)) --------------------------------PhysicalOlapScan[time_dim] ------------------------PhysicalProject ---------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3) and hd_dep_count IN (-1, 0, 3)) +--------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3)) ----------------------------PhysicalOlapScan[household_demographics] --------------------PhysicalProject ----------------------filter((store.s_store_name = 'ese')) @@ -150,7 +150,7 @@ PhysicalResultSink ----------------------------filter((time_dim.t_hour = 11) and (time_dim.t_minute >= 30)) ------------------------------PhysicalOlapScan[time_dim] ----------------------PhysicalProject -------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3) and hd_dep_count IN (-1, 0, 3)) +------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3)) --------------------------PhysicalOlapScan[household_demographics] ------------------PhysicalProject --------------------filter((store.s_store_name = 'ese')) @@ -171,7 +171,7 @@ PhysicalResultSink --------------------------filter((time_dim.t_hour = 12) and (time_dim.t_minute < 30)) ----------------------------PhysicalOlapScan[time_dim] --------------------PhysicalProject -----------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3) and hd_dep_count IN (-1, 0, 3)) +----------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3)) ------------------------PhysicalOlapScan[household_demographics] ----------------PhysicalProject ------------------filter((store.s_store_name = 'ese')) diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query41.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query41.out index c870e728a39819..ea5acccd883550 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query41.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query41.out @@ -18,6 +18,6 @@ PhysicalResultSink ------------------------PhysicalDistribute[DistributionSpecHash] --------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------filter((((i_color IN ('forest', 'lime', 'maroon', 'navy', 'powder', 'sky', 'slate', 'smoke') AND i_units IN ('Bunch', 'Case', 'Dozen', 'Gross', 'Lb', 'Ounce', 'Pallet', 'Pound')) AND ((((((item.i_category = 'Women') AND i_color IN ('forest', 'lime')) AND i_units IN ('Pallet', 'Pound')) AND i_size IN ('economy', 'small')) OR ((((item.i_category = 'Women') AND i_color IN ('navy', 'slate')) AND i_units IN ('Bunch', 'Gross')) AND i_size IN ('extra large', 'petite'))) OR (((((item.i_category = 'Men') AND i_color IN ('powder', 'sky')) AND i_units IN ('Dozen', 'Lb')) AND i_size IN ('N/A', 'large')) OR ((((item.i_category = 'Men') AND i_color IN ('maroon', 'smoke')) AND i_units IN ('Case', 'Ounce')) AND i_size IN ('economy', 'small'))))) OR ((i_color IN ('aquamarine', 'dark', 'firebrick', 'frosted', 'papaya', 'peach', 'plum', 'sienna') AND i_units IN ('Box', 'Bundle', 'Carton', 'Cup', 'Dram', 'Each', 'Tbl', 'Ton')) AND ((((((item.i_category = 'Women') AND i_color IN ('aquamarine', 'dark')) AND i_units IN ('Tbl', 'Ton')) AND i_size IN ('economy', 'small')) OR ((((item.i_category = 'Women') AND i_color IN ('frosted', 'plum')) AND i_units IN ('Box', 'Dram')) AND i_size IN ('extra large', 'petite'))) OR (((((item.i_category = 'Men') AND i_color IN ('papaya', 'peach')) AND i_units IN ('Bundle', 'Carton')) AND i_size IN ('N/A', 'large')) OR ((((item.i_category = 'Men') AND i_color IN ('firebrick', 'sienna')) AND i_units IN ('Cup', 'Each')) AND i_size IN ('economy', 'small')))))) and i_category IN ('Men', 'Women') and i_size IN ('N/A', 'economy', 'extra large', 'large', 'petite', 'small')) +------------------------------filter((((i_color IN ('forest', 'lime', 'maroon', 'navy', 'powder', 'sky', 'slate', 'smoke') AND i_units IN ('Bunch', 'Case', 'Dozen', 'Gross', 'Lb', 'Ounce', 'Pallet', 'Pound')) AND (((((((item.i_category = 'Women') AND i_color IN ('forest', 'lime')) AND i_units IN ('Pallet', 'Pound')) AND i_size IN ('economy', 'small')) OR ((((item.i_category = 'Women') AND i_color IN ('navy', 'slate')) AND i_units IN ('Bunch', 'Gross')) AND i_size IN ('extra large', 'petite'))) OR ((((item.i_category = 'Men') AND i_color IN ('powder', 'sky')) AND i_units IN ('Dozen', 'Lb')) AND i_size IN ('N/A', 'large'))) OR ((((item.i_category = 'Men') AND i_color IN ('maroon', 'smoke')) AND i_units IN ('Case', 'Ounce')) AND i_size IN ('economy', 'small')))) OR ((i_color IN ('aquamarine', 'dark', 'firebrick', 'frosted', 'papaya', 'peach', 'plum', 'sienna') AND i_units IN ('Box', 'Bundle', 'Carton', 'Cup', 'Dram', 'Each', 'Tbl', 'Ton')) AND (((((((item.i_category = 'Women') AND i_color IN ('aquamarine', 'dark')) AND i_units IN ('Tbl', 'Ton')) AND i_size IN ('economy', 'small')) OR ((((item.i_category = 'Women') AND i_color IN ('frosted', 'plum')) AND i_units IN ('Box', 'Dram')) AND i_size IN ('extra large', 'petite'))) OR ((((item.i_category = 'Men') AND i_color IN ('papaya', 'peach')) AND i_units IN ('Bundle', 'Carton')) AND i_size IN ('N/A', 'large'))) OR ((((item.i_category = 'Men') AND i_color IN ('firebrick', 'sienna')) AND i_units IN ('Cup', 'Each')) AND i_size IN ('economy', 'small'))))) and i_category IN ('Men', 'Women') and i_size IN ('N/A', 'economy', 'extra large', 'large', 'petite', 'small')) --------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query47.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query47.out index caec3b20581ed5..d51d48c5ab5677 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query47.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query47.out @@ -21,7 +21,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------------------------PhysicalProject --------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 ------------------------------------PhysicalProject ---------------------------------------filter((((date_dim.d_year = 2000) OR ((date_dim.d_year = 1999) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2001) AND (date_dim.d_moy = 1))) and d_year IN (1999, 2000, 2001) and d_year IN (1999, 2000, 2001)) +--------------------------------------filter((((date_dim.d_year = 2000) OR ((date_dim.d_year = 1999) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2001) AND (date_dim.d_moy = 1))) and d_year IN (1999, 2000, 2001)) ----------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query88.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query88.out index dc3f1ece14f64d..3c8f0335f1f415 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query88.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query88.out @@ -23,7 +23,7 @@ PhysicalResultSink ------------------------------------filter((time_dim.t_hour = 8) and (time_dim.t_minute >= 30)) --------------------------------------PhysicalOlapScan[time_dim] ------------------------------PhysicalProject ---------------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3) and hd_dep_count IN (-1, 0, 3)) +--------------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3)) ----------------------------------PhysicalOlapScan[household_demographics] --------------------------PhysicalProject ----------------------------filter((store.s_store_name = 'ese')) @@ -43,7 +43,7 @@ PhysicalResultSink ------------------------------------filter((time_dim.t_hour = 9) and (time_dim.t_minute < 30)) --------------------------------------PhysicalOlapScan[time_dim] ------------------------------PhysicalProject ---------------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3) and hd_dep_count IN (-1, 0, 3)) +--------------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3)) ----------------------------------PhysicalOlapScan[household_demographics] --------------------------PhysicalProject ----------------------------filter((store.s_store_name = 'ese')) @@ -63,7 +63,7 @@ PhysicalResultSink ----------------------------------filter((time_dim.t_hour = 9) and (time_dim.t_minute >= 30)) ------------------------------------PhysicalOlapScan[time_dim] ----------------------------PhysicalProject -------------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3) and hd_dep_count IN (-1, 0, 3)) +------------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3)) --------------------------------PhysicalOlapScan[household_demographics] ------------------------PhysicalProject --------------------------filter((store.s_store_name = 'ese')) @@ -83,7 +83,7 @@ PhysicalResultSink --------------------------------filter((time_dim.t_hour = 10) and (time_dim.t_minute < 30)) ----------------------------------PhysicalOlapScan[time_dim] --------------------------PhysicalProject -----------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3) and hd_dep_count IN (-1, 0, 3)) +----------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3)) ------------------------------PhysicalOlapScan[household_demographics] ----------------------PhysicalProject ------------------------filter((store.s_store_name = 'ese')) @@ -103,7 +103,7 @@ PhysicalResultSink ------------------------------filter((time_dim.t_hour = 10) and (time_dim.t_minute >= 30)) --------------------------------PhysicalOlapScan[time_dim] ------------------------PhysicalProject ---------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3) and hd_dep_count IN (-1, 0, 3)) +--------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3)) ----------------------------PhysicalOlapScan[household_demographics] --------------------PhysicalProject ----------------------filter((store.s_store_name = 'ese')) @@ -123,7 +123,7 @@ PhysicalResultSink ----------------------------filter((time_dim.t_hour = 11) and (time_dim.t_minute < 30)) ------------------------------PhysicalOlapScan[time_dim] ----------------------PhysicalProject -------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3) and hd_dep_count IN (-1, 0, 3)) +------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3)) --------------------------PhysicalOlapScan[household_demographics] ------------------PhysicalProject --------------------filter((store.s_store_name = 'ese')) @@ -143,7 +143,7 @@ PhysicalResultSink --------------------------filter((time_dim.t_hour = 11) and (time_dim.t_minute >= 30)) ----------------------------PhysicalOlapScan[time_dim] --------------------PhysicalProject -----------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3) and hd_dep_count IN (-1, 0, 3)) +----------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3)) ------------------------PhysicalOlapScan[household_demographics] ----------------PhysicalProject ------------------filter((store.s_store_name = 'ese')) @@ -163,7 +163,7 @@ PhysicalResultSink ------------------------filter((time_dim.t_hour = 12) and (time_dim.t_minute < 30)) --------------------------PhysicalOlapScan[time_dim] ------------------PhysicalProject ---------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3) and hd_dep_count IN (-1, 0, 3)) +--------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3)) ----------------------PhysicalOlapScan[household_demographics] --------------PhysicalProject ----------------filter((store.s_store_name = 'ese')) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query41.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query41.out index ff811ef082cc23..c9c1ffff499741 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query41.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query41.out @@ -18,6 +18,6 @@ PhysicalResultSink ------------------------PhysicalDistribute[DistributionSpecHash] --------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------filter((((i_color IN ('aquamarine', 'blue', 'chartreuse', 'chiffon', 'dodger', 'gainsboro', 'tan', 'violet') AND i_units IN ('Bunch', 'Dozen', 'Each', 'Ounce', 'Oz', 'Pound', 'Ton', 'Tsp')) AND ((((((item.i_category = 'Women') AND i_color IN ('aquamarine', 'gainsboro')) AND i_units IN ('Dozen', 'Ounce')) AND i_size IN ('economy', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('chiffon', 'violet')) AND i_units IN ('Pound', 'Ton')) AND i_size IN ('extra large', 'small'))) OR (((((item.i_category = 'Men') AND i_color IN ('blue', 'chartreuse')) AND i_units IN ('Each', 'Oz')) AND i_size IN ('N/A', 'large')) OR ((((item.i_category = 'Men') AND i_color IN ('dodger', 'tan')) AND i_units IN ('Bunch', 'Tsp')) AND i_size IN ('economy', 'medium'))))) OR ((i_color IN ('almond', 'blanched', 'indian', 'lime', 'peru', 'saddle', 'spring', 'tomato') AND i_units IN ('Box', 'Carton', 'Case', 'Dram', 'Gram', 'Pallet', 'Tbl', 'Unknown')) AND ((((((item.i_category = 'Women') AND i_color IN ('blanched', 'tomato')) AND i_units IN ('Case', 'Tbl')) AND i_size IN ('economy', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('almond', 'lime')) AND i_units IN ('Box', 'Dram')) AND i_size IN ('extra large', 'small'))) OR (((((item.i_category = 'Men') AND i_color IN ('peru', 'saddle')) AND i_units IN ('Gram', 'Pallet')) AND i_size IN ('N/A', 'large')) OR ((((item.i_category = 'Men') AND i_color IN ('indian', 'spring')) AND i_units IN ('Carton', 'Unknown')) AND i_size IN ('economy', 'medium')))))) and i_category IN ('Men', 'Women') and i_size IN ('N/A', 'economy', 'extra large', 'large', 'medium', 'small')) +------------------------------filter((((i_color IN ('aquamarine', 'blue', 'chartreuse', 'chiffon', 'dodger', 'gainsboro', 'tan', 'violet') AND i_units IN ('Bunch', 'Dozen', 'Each', 'Ounce', 'Oz', 'Pound', 'Ton', 'Tsp')) AND (((((((item.i_category = 'Women') AND i_color IN ('aquamarine', 'gainsboro')) AND i_units IN ('Dozen', 'Ounce')) AND i_size IN ('economy', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('chiffon', 'violet')) AND i_units IN ('Pound', 'Ton')) AND i_size IN ('extra large', 'small'))) OR ((((item.i_category = 'Men') AND i_color IN ('blue', 'chartreuse')) AND i_units IN ('Each', 'Oz')) AND i_size IN ('N/A', 'large'))) OR ((((item.i_category = 'Men') AND i_color IN ('dodger', 'tan')) AND i_units IN ('Bunch', 'Tsp')) AND i_size IN ('economy', 'medium')))) OR ((i_color IN ('almond', 'blanched', 'indian', 'lime', 'peru', 'saddle', 'spring', 'tomato') AND i_units IN ('Box', 'Carton', 'Case', 'Dram', 'Gram', 'Pallet', 'Tbl', 'Unknown')) AND (((((((item.i_category = 'Women') AND i_color IN ('blanched', 'tomato')) AND i_units IN ('Case', 'Tbl')) AND i_size IN ('economy', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('almond', 'lime')) AND i_units IN ('Box', 'Dram')) AND i_size IN ('extra large', 'small'))) OR ((((item.i_category = 'Men') AND i_color IN ('peru', 'saddle')) AND i_units IN ('Gram', 'Pallet')) AND i_size IN ('N/A', 'large'))) OR ((((item.i_category = 'Men') AND i_color IN ('indian', 'spring')) AND i_units IN ('Carton', 'Unknown')) AND i_size IN ('economy', 'medium'))))) and i_category IN ('Men', 'Women') and i_size IN ('N/A', 'economy', 'extra large', 'large', 'medium', 'small')) --------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query57.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query57.out index a05f8e84e68830..842b18a4eac9e6 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query57.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query57.out @@ -23,7 +23,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------------------------PhysicalProject --------------------------------------PhysicalOlapScan[item] --------------------------------PhysicalProject -----------------------------------filter((((date_dim.d_year = 1999) OR ((date_dim.d_year = 1998) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2000) AND (date_dim.d_moy = 1))) and d_year IN (1998, 1999, 2000) and d_year IN (1998, 1999, 2000)) +----------------------------------filter((((date_dim.d_year = 1999) OR ((date_dim.d_year = 1998) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2000) AND (date_dim.d_moy = 1))) and d_year IN (1998, 1999, 2000)) ------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[call_center] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query41.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query41.out index ff811ef082cc23..c9c1ffff499741 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query41.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query41.out @@ -18,6 +18,6 @@ PhysicalResultSink ------------------------PhysicalDistribute[DistributionSpecHash] --------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------filter((((i_color IN ('aquamarine', 'blue', 'chartreuse', 'chiffon', 'dodger', 'gainsboro', 'tan', 'violet') AND i_units IN ('Bunch', 'Dozen', 'Each', 'Ounce', 'Oz', 'Pound', 'Ton', 'Tsp')) AND ((((((item.i_category = 'Women') AND i_color IN ('aquamarine', 'gainsboro')) AND i_units IN ('Dozen', 'Ounce')) AND i_size IN ('economy', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('chiffon', 'violet')) AND i_units IN ('Pound', 'Ton')) AND i_size IN ('extra large', 'small'))) OR (((((item.i_category = 'Men') AND i_color IN ('blue', 'chartreuse')) AND i_units IN ('Each', 'Oz')) AND i_size IN ('N/A', 'large')) OR ((((item.i_category = 'Men') AND i_color IN ('dodger', 'tan')) AND i_units IN ('Bunch', 'Tsp')) AND i_size IN ('economy', 'medium'))))) OR ((i_color IN ('almond', 'blanched', 'indian', 'lime', 'peru', 'saddle', 'spring', 'tomato') AND i_units IN ('Box', 'Carton', 'Case', 'Dram', 'Gram', 'Pallet', 'Tbl', 'Unknown')) AND ((((((item.i_category = 'Women') AND i_color IN ('blanched', 'tomato')) AND i_units IN ('Case', 'Tbl')) AND i_size IN ('economy', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('almond', 'lime')) AND i_units IN ('Box', 'Dram')) AND i_size IN ('extra large', 'small'))) OR (((((item.i_category = 'Men') AND i_color IN ('peru', 'saddle')) AND i_units IN ('Gram', 'Pallet')) AND i_size IN ('N/A', 'large')) OR ((((item.i_category = 'Men') AND i_color IN ('indian', 'spring')) AND i_units IN ('Carton', 'Unknown')) AND i_size IN ('economy', 'medium')))))) and i_category IN ('Men', 'Women') and i_size IN ('N/A', 'economy', 'extra large', 'large', 'medium', 'small')) +------------------------------filter((((i_color IN ('aquamarine', 'blue', 'chartreuse', 'chiffon', 'dodger', 'gainsboro', 'tan', 'violet') AND i_units IN ('Bunch', 'Dozen', 'Each', 'Ounce', 'Oz', 'Pound', 'Ton', 'Tsp')) AND (((((((item.i_category = 'Women') AND i_color IN ('aquamarine', 'gainsboro')) AND i_units IN ('Dozen', 'Ounce')) AND i_size IN ('economy', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('chiffon', 'violet')) AND i_units IN ('Pound', 'Ton')) AND i_size IN ('extra large', 'small'))) OR ((((item.i_category = 'Men') AND i_color IN ('blue', 'chartreuse')) AND i_units IN ('Each', 'Oz')) AND i_size IN ('N/A', 'large'))) OR ((((item.i_category = 'Men') AND i_color IN ('dodger', 'tan')) AND i_units IN ('Bunch', 'Tsp')) AND i_size IN ('economy', 'medium')))) OR ((i_color IN ('almond', 'blanched', 'indian', 'lime', 'peru', 'saddle', 'spring', 'tomato') AND i_units IN ('Box', 'Carton', 'Case', 'Dram', 'Gram', 'Pallet', 'Tbl', 'Unknown')) AND (((((((item.i_category = 'Women') AND i_color IN ('blanched', 'tomato')) AND i_units IN ('Case', 'Tbl')) AND i_size IN ('economy', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('almond', 'lime')) AND i_units IN ('Box', 'Dram')) AND i_size IN ('extra large', 'small'))) OR ((((item.i_category = 'Men') AND i_color IN ('peru', 'saddle')) AND i_units IN ('Gram', 'Pallet')) AND i_size IN ('N/A', 'large'))) OR ((((item.i_category = 'Men') AND i_color IN ('indian', 'spring')) AND i_units IN ('Carton', 'Unknown')) AND i_size IN ('economy', 'medium'))))) and i_category IN ('Men', 'Women') and i_size IN ('N/A', 'economy', 'extra large', 'large', 'medium', 'small')) --------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query57.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query57.out index 9b61dc54e2a91e..e640ec24d7b3e3 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query57.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query57.out @@ -23,7 +23,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------------------------PhysicalProject --------------------------------------PhysicalOlapScan[item] --------------------------------PhysicalProject -----------------------------------filter((((date_dim.d_year = 1999) OR ((date_dim.d_year = 1998) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2000) AND (date_dim.d_moy = 1))) and d_year IN (1998, 1999, 2000) and d_year IN (1998, 1999, 2000)) +----------------------------------filter((((date_dim.d_year = 1999) OR ((date_dim.d_year = 1998) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2000) AND (date_dim.d_moy = 1))) and d_year IN (1998, 1999, 2000)) ------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[call_center] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query41.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query41.out index ff811ef082cc23..c9c1ffff499741 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query41.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query41.out @@ -18,6 +18,6 @@ PhysicalResultSink ------------------------PhysicalDistribute[DistributionSpecHash] --------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------filter((((i_color IN ('aquamarine', 'blue', 'chartreuse', 'chiffon', 'dodger', 'gainsboro', 'tan', 'violet') AND i_units IN ('Bunch', 'Dozen', 'Each', 'Ounce', 'Oz', 'Pound', 'Ton', 'Tsp')) AND ((((((item.i_category = 'Women') AND i_color IN ('aquamarine', 'gainsboro')) AND i_units IN ('Dozen', 'Ounce')) AND i_size IN ('economy', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('chiffon', 'violet')) AND i_units IN ('Pound', 'Ton')) AND i_size IN ('extra large', 'small'))) OR (((((item.i_category = 'Men') AND i_color IN ('blue', 'chartreuse')) AND i_units IN ('Each', 'Oz')) AND i_size IN ('N/A', 'large')) OR ((((item.i_category = 'Men') AND i_color IN ('dodger', 'tan')) AND i_units IN ('Bunch', 'Tsp')) AND i_size IN ('economy', 'medium'))))) OR ((i_color IN ('almond', 'blanched', 'indian', 'lime', 'peru', 'saddle', 'spring', 'tomato') AND i_units IN ('Box', 'Carton', 'Case', 'Dram', 'Gram', 'Pallet', 'Tbl', 'Unknown')) AND ((((((item.i_category = 'Women') AND i_color IN ('blanched', 'tomato')) AND i_units IN ('Case', 'Tbl')) AND i_size IN ('economy', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('almond', 'lime')) AND i_units IN ('Box', 'Dram')) AND i_size IN ('extra large', 'small'))) OR (((((item.i_category = 'Men') AND i_color IN ('peru', 'saddle')) AND i_units IN ('Gram', 'Pallet')) AND i_size IN ('N/A', 'large')) OR ((((item.i_category = 'Men') AND i_color IN ('indian', 'spring')) AND i_units IN ('Carton', 'Unknown')) AND i_size IN ('economy', 'medium')))))) and i_category IN ('Men', 'Women') and i_size IN ('N/A', 'economy', 'extra large', 'large', 'medium', 'small')) +------------------------------filter((((i_color IN ('aquamarine', 'blue', 'chartreuse', 'chiffon', 'dodger', 'gainsboro', 'tan', 'violet') AND i_units IN ('Bunch', 'Dozen', 'Each', 'Ounce', 'Oz', 'Pound', 'Ton', 'Tsp')) AND (((((((item.i_category = 'Women') AND i_color IN ('aquamarine', 'gainsboro')) AND i_units IN ('Dozen', 'Ounce')) AND i_size IN ('economy', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('chiffon', 'violet')) AND i_units IN ('Pound', 'Ton')) AND i_size IN ('extra large', 'small'))) OR ((((item.i_category = 'Men') AND i_color IN ('blue', 'chartreuse')) AND i_units IN ('Each', 'Oz')) AND i_size IN ('N/A', 'large'))) OR ((((item.i_category = 'Men') AND i_color IN ('dodger', 'tan')) AND i_units IN ('Bunch', 'Tsp')) AND i_size IN ('economy', 'medium')))) OR ((i_color IN ('almond', 'blanched', 'indian', 'lime', 'peru', 'saddle', 'spring', 'tomato') AND i_units IN ('Box', 'Carton', 'Case', 'Dram', 'Gram', 'Pallet', 'Tbl', 'Unknown')) AND (((((((item.i_category = 'Women') AND i_color IN ('blanched', 'tomato')) AND i_units IN ('Case', 'Tbl')) AND i_size IN ('economy', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('almond', 'lime')) AND i_units IN ('Box', 'Dram')) AND i_size IN ('extra large', 'small'))) OR ((((item.i_category = 'Men') AND i_color IN ('peru', 'saddle')) AND i_units IN ('Gram', 'Pallet')) AND i_size IN ('N/A', 'large'))) OR ((((item.i_category = 'Men') AND i_color IN ('indian', 'spring')) AND i_units IN ('Carton', 'Unknown')) AND i_size IN ('economy', 'medium'))))) and i_category IN ('Men', 'Women') and i_size IN ('N/A', 'economy', 'extra large', 'large', 'medium', 'small')) --------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query57.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query57.out index 053e8bfc4c7f54..4a7de98eb56b50 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query57.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query57.out @@ -21,7 +21,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------------------------PhysicalProject --------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 ------------------------------------PhysicalProject ---------------------------------------filter((((date_dim.d_year = 1999) OR ((date_dim.d_year = 1998) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2000) AND (date_dim.d_moy = 1))) and d_year IN (1998, 1999, 2000) and d_year IN (1998, 1999, 2000)) +--------------------------------------filter((((date_dim.d_year = 1999) OR ((date_dim.d_year = 1998) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2000) AND (date_dim.d_moy = 1))) and d_year IN (1998, 1999, 2000)) ----------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query41.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query41.out index ff811ef082cc23..c9c1ffff499741 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query41.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query41.out @@ -18,6 +18,6 @@ PhysicalResultSink ------------------------PhysicalDistribute[DistributionSpecHash] --------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------filter((((i_color IN ('aquamarine', 'blue', 'chartreuse', 'chiffon', 'dodger', 'gainsboro', 'tan', 'violet') AND i_units IN ('Bunch', 'Dozen', 'Each', 'Ounce', 'Oz', 'Pound', 'Ton', 'Tsp')) AND ((((((item.i_category = 'Women') AND i_color IN ('aquamarine', 'gainsboro')) AND i_units IN ('Dozen', 'Ounce')) AND i_size IN ('economy', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('chiffon', 'violet')) AND i_units IN ('Pound', 'Ton')) AND i_size IN ('extra large', 'small'))) OR (((((item.i_category = 'Men') AND i_color IN ('blue', 'chartreuse')) AND i_units IN ('Each', 'Oz')) AND i_size IN ('N/A', 'large')) OR ((((item.i_category = 'Men') AND i_color IN ('dodger', 'tan')) AND i_units IN ('Bunch', 'Tsp')) AND i_size IN ('economy', 'medium'))))) OR ((i_color IN ('almond', 'blanched', 'indian', 'lime', 'peru', 'saddle', 'spring', 'tomato') AND i_units IN ('Box', 'Carton', 'Case', 'Dram', 'Gram', 'Pallet', 'Tbl', 'Unknown')) AND ((((((item.i_category = 'Women') AND i_color IN ('blanched', 'tomato')) AND i_units IN ('Case', 'Tbl')) AND i_size IN ('economy', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('almond', 'lime')) AND i_units IN ('Box', 'Dram')) AND i_size IN ('extra large', 'small'))) OR (((((item.i_category = 'Men') AND i_color IN ('peru', 'saddle')) AND i_units IN ('Gram', 'Pallet')) AND i_size IN ('N/A', 'large')) OR ((((item.i_category = 'Men') AND i_color IN ('indian', 'spring')) AND i_units IN ('Carton', 'Unknown')) AND i_size IN ('economy', 'medium')))))) and i_category IN ('Men', 'Women') and i_size IN ('N/A', 'economy', 'extra large', 'large', 'medium', 'small')) +------------------------------filter((((i_color IN ('aquamarine', 'blue', 'chartreuse', 'chiffon', 'dodger', 'gainsboro', 'tan', 'violet') AND i_units IN ('Bunch', 'Dozen', 'Each', 'Ounce', 'Oz', 'Pound', 'Ton', 'Tsp')) AND (((((((item.i_category = 'Women') AND i_color IN ('aquamarine', 'gainsboro')) AND i_units IN ('Dozen', 'Ounce')) AND i_size IN ('economy', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('chiffon', 'violet')) AND i_units IN ('Pound', 'Ton')) AND i_size IN ('extra large', 'small'))) OR ((((item.i_category = 'Men') AND i_color IN ('blue', 'chartreuse')) AND i_units IN ('Each', 'Oz')) AND i_size IN ('N/A', 'large'))) OR ((((item.i_category = 'Men') AND i_color IN ('dodger', 'tan')) AND i_units IN ('Bunch', 'Tsp')) AND i_size IN ('economy', 'medium')))) OR ((i_color IN ('almond', 'blanched', 'indian', 'lime', 'peru', 'saddle', 'spring', 'tomato') AND i_units IN ('Box', 'Carton', 'Case', 'Dram', 'Gram', 'Pallet', 'Tbl', 'Unknown')) AND (((((((item.i_category = 'Women') AND i_color IN ('blanched', 'tomato')) AND i_units IN ('Case', 'Tbl')) AND i_size IN ('economy', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('almond', 'lime')) AND i_units IN ('Box', 'Dram')) AND i_size IN ('extra large', 'small'))) OR ((((item.i_category = 'Men') AND i_color IN ('peru', 'saddle')) AND i_units IN ('Gram', 'Pallet')) AND i_size IN ('N/A', 'large'))) OR ((((item.i_category = 'Men') AND i_color IN ('indian', 'spring')) AND i_units IN ('Carton', 'Unknown')) AND i_size IN ('economy', 'medium'))))) and i_category IN ('Men', 'Women') and i_size IN ('N/A', 'economy', 'extra large', 'large', 'medium', 'small')) --------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query57.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query57.out index 7bbcf019701d4d..a0157318139d88 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query57.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query57.out @@ -21,7 +21,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------------------------PhysicalProject --------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 ------------------------------------PhysicalProject ---------------------------------------filter((((date_dim.d_year = 1999) OR ((date_dim.d_year = 1998) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2000) AND (date_dim.d_moy = 1))) and d_year IN (1998, 1999, 2000) and d_year IN (1998, 1999, 2000)) +--------------------------------------filter((((date_dim.d_year = 1999) OR ((date_dim.d_year = 1998) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2000) AND (date_dim.d_moy = 1))) and d_year IN (1998, 1999, 2000)) ----------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query41.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query41.out index a470709da6831c..16a3a620b77c8e 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query41.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query41.out @@ -18,6 +18,6 @@ PhysicalResultSink ------------------------PhysicalDistribute[DistributionSpecHash] --------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------filter((((i_color IN ('black', 'chocolate', 'cornflower', 'firebrick', 'frosted', 'magenta', 'rose', 'slate') AND i_units IN ('Box', 'Bundle', 'Carton', 'Dram', 'Gross', 'Lb', 'Oz', 'Pound')) AND ((((((item.i_category = 'Women') AND i_color IN ('frosted', 'rose')) AND i_units IN ('Gross', 'Lb')) AND i_size IN ('large', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('black', 'chocolate')) AND i_units IN ('Box', 'Dram')) AND i_size IN ('economy', 'petite'))) OR (((((item.i_category = 'Men') AND i_color IN ('magenta', 'slate')) AND i_units IN ('Bundle', 'Carton')) AND i_size IN ('N/A', 'small')) OR ((((item.i_category = 'Men') AND i_color IN ('cornflower', 'firebrick')) AND i_units IN ('Oz', 'Pound')) AND i_size IN ('large', 'medium'))))) OR ((i_color IN ('almond', 'aquamarine', 'cyan', 'lavender', 'maroon', 'papaya', 'purple', 'steel') AND i_units IN ('Bunch', 'Case', 'Cup', 'Each', 'Gram', 'N/A', 'Pallet', 'Tsp')) AND ((((((item.i_category = 'Women') AND i_color IN ('almond', 'steel')) AND i_units IN ('Case', 'Tsp')) AND i_size IN ('large', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('aquamarine', 'purple')) AND i_units IN ('Bunch', 'Gram')) AND i_size IN ('economy', 'petite'))) OR (((((item.i_category = 'Men') AND i_color IN ('lavender', 'papaya')) AND i_units IN ('Cup', 'Pallet')) AND i_size IN ('N/A', 'small')) OR ((((item.i_category = 'Men') AND i_color IN ('cyan', 'maroon')) AND i_units IN ('Each', 'N/A')) AND i_size IN ('large', 'medium')))))) and i_category IN ('Men', 'Women') and i_size IN ('N/A', 'economy', 'large', 'medium', 'petite', 'small')) +------------------------------filter((((i_color IN ('black', 'chocolate', 'cornflower', 'firebrick', 'frosted', 'magenta', 'rose', 'slate') AND i_units IN ('Box', 'Bundle', 'Carton', 'Dram', 'Gross', 'Lb', 'Oz', 'Pound')) AND (((((((item.i_category = 'Women') AND i_color IN ('frosted', 'rose')) AND i_units IN ('Gross', 'Lb')) AND i_size IN ('large', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('black', 'chocolate')) AND i_units IN ('Box', 'Dram')) AND i_size IN ('economy', 'petite'))) OR ((((item.i_category = 'Men') AND i_color IN ('magenta', 'slate')) AND i_units IN ('Bundle', 'Carton')) AND i_size IN ('N/A', 'small'))) OR ((((item.i_category = 'Men') AND i_color IN ('cornflower', 'firebrick')) AND i_units IN ('Oz', 'Pound')) AND i_size IN ('large', 'medium')))) OR ((i_color IN ('almond', 'aquamarine', 'cyan', 'lavender', 'maroon', 'papaya', 'purple', 'steel') AND i_units IN ('Bunch', 'Case', 'Cup', 'Each', 'Gram', 'N/A', 'Pallet', 'Tsp')) AND (((((((item.i_category = 'Women') AND i_color IN ('almond', 'steel')) AND i_units IN ('Case', 'Tsp')) AND i_size IN ('large', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('aquamarine', 'purple')) AND i_units IN ('Bunch', 'Gram')) AND i_size IN ('economy', 'petite'))) OR ((((item.i_category = 'Men') AND i_color IN ('lavender', 'papaya')) AND i_units IN ('Cup', 'Pallet')) AND i_size IN ('N/A', 'small'))) OR ((((item.i_category = 'Men') AND i_color IN ('cyan', 'maroon')) AND i_units IN ('Each', 'N/A')) AND i_size IN ('large', 'medium'))))) and i_category IN ('Men', 'Women') and i_size IN ('N/A', 'economy', 'large', 'medium', 'petite', 'small')) --------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query47.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query47.out index 591df70c5abce8..a3205932929569 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query47.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query47.out @@ -23,7 +23,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------------------------PhysicalProject --------------------------------------PhysicalOlapScan[item] --------------------------------PhysicalProject -----------------------------------filter((((date_dim.d_year = 1999) OR ((date_dim.d_year = 1998) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2000) AND (date_dim.d_moy = 1))) and d_year IN (1998, 1999, 2000) and d_year IN (1998, 1999, 2000)) +----------------------------------filter((((date_dim.d_year = 1999) OR ((date_dim.d_year = 1998) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2000) AND (date_dim.d_moy = 1))) and d_year IN (1998, 1999, 2000)) ------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query57.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query57.out index 9b61dc54e2a91e..e640ec24d7b3e3 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query57.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query57.out @@ -23,7 +23,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------------------------PhysicalProject --------------------------------------PhysicalOlapScan[item] --------------------------------PhysicalProject -----------------------------------filter((((date_dim.d_year = 1999) OR ((date_dim.d_year = 1998) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2000) AND (date_dim.d_moy = 1))) and d_year IN (1998, 1999, 2000) and d_year IN (1998, 1999, 2000)) +----------------------------------filter((((date_dim.d_year = 1999) OR ((date_dim.d_year = 1998) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2000) AND (date_dim.d_moy = 1))) and d_year IN (1998, 1999, 2000)) ------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[call_center] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query41.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query41.out index ff811ef082cc23..c9c1ffff499741 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query41.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query41.out @@ -18,6 +18,6 @@ PhysicalResultSink ------------------------PhysicalDistribute[DistributionSpecHash] --------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------filter((((i_color IN ('aquamarine', 'blue', 'chartreuse', 'chiffon', 'dodger', 'gainsboro', 'tan', 'violet') AND i_units IN ('Bunch', 'Dozen', 'Each', 'Ounce', 'Oz', 'Pound', 'Ton', 'Tsp')) AND ((((((item.i_category = 'Women') AND i_color IN ('aquamarine', 'gainsboro')) AND i_units IN ('Dozen', 'Ounce')) AND i_size IN ('economy', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('chiffon', 'violet')) AND i_units IN ('Pound', 'Ton')) AND i_size IN ('extra large', 'small'))) OR (((((item.i_category = 'Men') AND i_color IN ('blue', 'chartreuse')) AND i_units IN ('Each', 'Oz')) AND i_size IN ('N/A', 'large')) OR ((((item.i_category = 'Men') AND i_color IN ('dodger', 'tan')) AND i_units IN ('Bunch', 'Tsp')) AND i_size IN ('economy', 'medium'))))) OR ((i_color IN ('almond', 'blanched', 'indian', 'lime', 'peru', 'saddle', 'spring', 'tomato') AND i_units IN ('Box', 'Carton', 'Case', 'Dram', 'Gram', 'Pallet', 'Tbl', 'Unknown')) AND ((((((item.i_category = 'Women') AND i_color IN ('blanched', 'tomato')) AND i_units IN ('Case', 'Tbl')) AND i_size IN ('economy', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('almond', 'lime')) AND i_units IN ('Box', 'Dram')) AND i_size IN ('extra large', 'small'))) OR (((((item.i_category = 'Men') AND i_color IN ('peru', 'saddle')) AND i_units IN ('Gram', 'Pallet')) AND i_size IN ('N/A', 'large')) OR ((((item.i_category = 'Men') AND i_color IN ('indian', 'spring')) AND i_units IN ('Carton', 'Unknown')) AND i_size IN ('economy', 'medium')))))) and i_category IN ('Men', 'Women') and i_size IN ('N/A', 'economy', 'extra large', 'large', 'medium', 'small')) +------------------------------filter((((i_color IN ('aquamarine', 'blue', 'chartreuse', 'chiffon', 'dodger', 'gainsboro', 'tan', 'violet') AND i_units IN ('Bunch', 'Dozen', 'Each', 'Ounce', 'Oz', 'Pound', 'Ton', 'Tsp')) AND (((((((item.i_category = 'Women') AND i_color IN ('aquamarine', 'gainsboro')) AND i_units IN ('Dozen', 'Ounce')) AND i_size IN ('economy', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('chiffon', 'violet')) AND i_units IN ('Pound', 'Ton')) AND i_size IN ('extra large', 'small'))) OR ((((item.i_category = 'Men') AND i_color IN ('blue', 'chartreuse')) AND i_units IN ('Each', 'Oz')) AND i_size IN ('N/A', 'large'))) OR ((((item.i_category = 'Men') AND i_color IN ('dodger', 'tan')) AND i_units IN ('Bunch', 'Tsp')) AND i_size IN ('economy', 'medium')))) OR ((i_color IN ('almond', 'blanched', 'indian', 'lime', 'peru', 'saddle', 'spring', 'tomato') AND i_units IN ('Box', 'Carton', 'Case', 'Dram', 'Gram', 'Pallet', 'Tbl', 'Unknown')) AND (((((((item.i_category = 'Women') AND i_color IN ('blanched', 'tomato')) AND i_units IN ('Case', 'Tbl')) AND i_size IN ('economy', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('almond', 'lime')) AND i_units IN ('Box', 'Dram')) AND i_size IN ('extra large', 'small'))) OR ((((item.i_category = 'Men') AND i_color IN ('peru', 'saddle')) AND i_units IN ('Gram', 'Pallet')) AND i_size IN ('N/A', 'large'))) OR ((((item.i_category = 'Men') AND i_color IN ('indian', 'spring')) AND i_units IN ('Carton', 'Unknown')) AND i_size IN ('economy', 'medium'))))) and i_category IN ('Men', 'Women') and i_size IN ('N/A', 'economy', 'extra large', 'large', 'medium', 'small')) --------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query57.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query57.out index a05f8e84e68830..842b18a4eac9e6 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query57.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query57.out @@ -23,7 +23,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------------------------PhysicalProject --------------------------------------PhysicalOlapScan[item] --------------------------------PhysicalProject -----------------------------------filter((((date_dim.d_year = 1999) OR ((date_dim.d_year = 1998) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2000) AND (date_dim.d_moy = 1))) and d_year IN (1998, 1999, 2000) and d_year IN (1998, 1999, 2000)) +----------------------------------filter((((date_dim.d_year = 1999) OR ((date_dim.d_year = 1998) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2000) AND (date_dim.d_moy = 1))) and d_year IN (1998, 1999, 2000)) ------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[call_center] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query41.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query41.out index ff811ef082cc23..c9c1ffff499741 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query41.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query41.out @@ -18,6 +18,6 @@ PhysicalResultSink ------------------------PhysicalDistribute[DistributionSpecHash] --------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------filter((((i_color IN ('aquamarine', 'blue', 'chartreuse', 'chiffon', 'dodger', 'gainsboro', 'tan', 'violet') AND i_units IN ('Bunch', 'Dozen', 'Each', 'Ounce', 'Oz', 'Pound', 'Ton', 'Tsp')) AND ((((((item.i_category = 'Women') AND i_color IN ('aquamarine', 'gainsboro')) AND i_units IN ('Dozen', 'Ounce')) AND i_size IN ('economy', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('chiffon', 'violet')) AND i_units IN ('Pound', 'Ton')) AND i_size IN ('extra large', 'small'))) OR (((((item.i_category = 'Men') AND i_color IN ('blue', 'chartreuse')) AND i_units IN ('Each', 'Oz')) AND i_size IN ('N/A', 'large')) OR ((((item.i_category = 'Men') AND i_color IN ('dodger', 'tan')) AND i_units IN ('Bunch', 'Tsp')) AND i_size IN ('economy', 'medium'))))) OR ((i_color IN ('almond', 'blanched', 'indian', 'lime', 'peru', 'saddle', 'spring', 'tomato') AND i_units IN ('Box', 'Carton', 'Case', 'Dram', 'Gram', 'Pallet', 'Tbl', 'Unknown')) AND ((((((item.i_category = 'Women') AND i_color IN ('blanched', 'tomato')) AND i_units IN ('Case', 'Tbl')) AND i_size IN ('economy', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('almond', 'lime')) AND i_units IN ('Box', 'Dram')) AND i_size IN ('extra large', 'small'))) OR (((((item.i_category = 'Men') AND i_color IN ('peru', 'saddle')) AND i_units IN ('Gram', 'Pallet')) AND i_size IN ('N/A', 'large')) OR ((((item.i_category = 'Men') AND i_color IN ('indian', 'spring')) AND i_units IN ('Carton', 'Unknown')) AND i_size IN ('economy', 'medium')))))) and i_category IN ('Men', 'Women') and i_size IN ('N/A', 'economy', 'extra large', 'large', 'medium', 'small')) +------------------------------filter((((i_color IN ('aquamarine', 'blue', 'chartreuse', 'chiffon', 'dodger', 'gainsboro', 'tan', 'violet') AND i_units IN ('Bunch', 'Dozen', 'Each', 'Ounce', 'Oz', 'Pound', 'Ton', 'Tsp')) AND (((((((item.i_category = 'Women') AND i_color IN ('aquamarine', 'gainsboro')) AND i_units IN ('Dozen', 'Ounce')) AND i_size IN ('economy', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('chiffon', 'violet')) AND i_units IN ('Pound', 'Ton')) AND i_size IN ('extra large', 'small'))) OR ((((item.i_category = 'Men') AND i_color IN ('blue', 'chartreuse')) AND i_units IN ('Each', 'Oz')) AND i_size IN ('N/A', 'large'))) OR ((((item.i_category = 'Men') AND i_color IN ('dodger', 'tan')) AND i_units IN ('Bunch', 'Tsp')) AND i_size IN ('economy', 'medium')))) OR ((i_color IN ('almond', 'blanched', 'indian', 'lime', 'peru', 'saddle', 'spring', 'tomato') AND i_units IN ('Box', 'Carton', 'Case', 'Dram', 'Gram', 'Pallet', 'Tbl', 'Unknown')) AND (((((((item.i_category = 'Women') AND i_color IN ('blanched', 'tomato')) AND i_units IN ('Case', 'Tbl')) AND i_size IN ('economy', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('almond', 'lime')) AND i_units IN ('Box', 'Dram')) AND i_size IN ('extra large', 'small'))) OR ((((item.i_category = 'Men') AND i_color IN ('peru', 'saddle')) AND i_units IN ('Gram', 'Pallet')) AND i_size IN ('N/A', 'large'))) OR ((((item.i_category = 'Men') AND i_color IN ('indian', 'spring')) AND i_units IN ('Carton', 'Unknown')) AND i_size IN ('economy', 'medium'))))) and i_category IN ('Men', 'Women') and i_size IN ('N/A', 'economy', 'extra large', 'large', 'medium', 'small')) --------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query57.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query57.out index 9b61dc54e2a91e..e640ec24d7b3e3 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query57.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query57.out @@ -23,7 +23,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------------------------PhysicalProject --------------------------------------PhysicalOlapScan[item] --------------------------------PhysicalProject -----------------------------------filter((((date_dim.d_year = 1999) OR ((date_dim.d_year = 1998) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2000) AND (date_dim.d_moy = 1))) and d_year IN (1998, 1999, 2000) and d_year IN (1998, 1999, 2000)) +----------------------------------filter((((date_dim.d_year = 1999) OR ((date_dim.d_year = 1998) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2000) AND (date_dim.d_moy = 1))) and d_year IN (1998, 1999, 2000)) ------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[call_center] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query41.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query41.out index ff811ef082cc23..c9c1ffff499741 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query41.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query41.out @@ -18,6 +18,6 @@ PhysicalResultSink ------------------------PhysicalDistribute[DistributionSpecHash] --------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------filter((((i_color IN ('aquamarine', 'blue', 'chartreuse', 'chiffon', 'dodger', 'gainsboro', 'tan', 'violet') AND i_units IN ('Bunch', 'Dozen', 'Each', 'Ounce', 'Oz', 'Pound', 'Ton', 'Tsp')) AND ((((((item.i_category = 'Women') AND i_color IN ('aquamarine', 'gainsboro')) AND i_units IN ('Dozen', 'Ounce')) AND i_size IN ('economy', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('chiffon', 'violet')) AND i_units IN ('Pound', 'Ton')) AND i_size IN ('extra large', 'small'))) OR (((((item.i_category = 'Men') AND i_color IN ('blue', 'chartreuse')) AND i_units IN ('Each', 'Oz')) AND i_size IN ('N/A', 'large')) OR ((((item.i_category = 'Men') AND i_color IN ('dodger', 'tan')) AND i_units IN ('Bunch', 'Tsp')) AND i_size IN ('economy', 'medium'))))) OR ((i_color IN ('almond', 'blanched', 'indian', 'lime', 'peru', 'saddle', 'spring', 'tomato') AND i_units IN ('Box', 'Carton', 'Case', 'Dram', 'Gram', 'Pallet', 'Tbl', 'Unknown')) AND ((((((item.i_category = 'Women') AND i_color IN ('blanched', 'tomato')) AND i_units IN ('Case', 'Tbl')) AND i_size IN ('economy', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('almond', 'lime')) AND i_units IN ('Box', 'Dram')) AND i_size IN ('extra large', 'small'))) OR (((((item.i_category = 'Men') AND i_color IN ('peru', 'saddle')) AND i_units IN ('Gram', 'Pallet')) AND i_size IN ('N/A', 'large')) OR ((((item.i_category = 'Men') AND i_color IN ('indian', 'spring')) AND i_units IN ('Carton', 'Unknown')) AND i_size IN ('economy', 'medium')))))) and i_category IN ('Men', 'Women') and i_size IN ('N/A', 'economy', 'extra large', 'large', 'medium', 'small')) +------------------------------filter((((i_color IN ('aquamarine', 'blue', 'chartreuse', 'chiffon', 'dodger', 'gainsboro', 'tan', 'violet') AND i_units IN ('Bunch', 'Dozen', 'Each', 'Ounce', 'Oz', 'Pound', 'Ton', 'Tsp')) AND (((((((item.i_category = 'Women') AND i_color IN ('aquamarine', 'gainsboro')) AND i_units IN ('Dozen', 'Ounce')) AND i_size IN ('economy', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('chiffon', 'violet')) AND i_units IN ('Pound', 'Ton')) AND i_size IN ('extra large', 'small'))) OR ((((item.i_category = 'Men') AND i_color IN ('blue', 'chartreuse')) AND i_units IN ('Each', 'Oz')) AND i_size IN ('N/A', 'large'))) OR ((((item.i_category = 'Men') AND i_color IN ('dodger', 'tan')) AND i_units IN ('Bunch', 'Tsp')) AND i_size IN ('economy', 'medium')))) OR ((i_color IN ('almond', 'blanched', 'indian', 'lime', 'peru', 'saddle', 'spring', 'tomato') AND i_units IN ('Box', 'Carton', 'Case', 'Dram', 'Gram', 'Pallet', 'Tbl', 'Unknown')) AND (((((((item.i_category = 'Women') AND i_color IN ('blanched', 'tomato')) AND i_units IN ('Case', 'Tbl')) AND i_size IN ('economy', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('almond', 'lime')) AND i_units IN ('Box', 'Dram')) AND i_size IN ('extra large', 'small'))) OR ((((item.i_category = 'Men') AND i_color IN ('peru', 'saddle')) AND i_units IN ('Gram', 'Pallet')) AND i_size IN ('N/A', 'large'))) OR ((((item.i_category = 'Men') AND i_color IN ('indian', 'spring')) AND i_units IN ('Carton', 'Unknown')) AND i_size IN ('economy', 'medium'))))) and i_category IN ('Men', 'Women') and i_size IN ('N/A', 'economy', 'extra large', 'large', 'medium', 'small')) --------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query57.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query57.out index 053e8bfc4c7f54..4a7de98eb56b50 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query57.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query57.out @@ -21,7 +21,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------------------------PhysicalProject --------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 ------------------------------------PhysicalProject ---------------------------------------filter((((date_dim.d_year = 1999) OR ((date_dim.d_year = 1998) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2000) AND (date_dim.d_moy = 1))) and d_year IN (1998, 1999, 2000) and d_year IN (1998, 1999, 2000)) +--------------------------------------filter((((date_dim.d_year = 1999) OR ((date_dim.d_year = 1998) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2000) AND (date_dim.d_moy = 1))) and d_year IN (1998, 1999, 2000)) ----------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query41.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query41.out index ff811ef082cc23..c9c1ffff499741 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query41.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query41.out @@ -18,6 +18,6 @@ PhysicalResultSink ------------------------PhysicalDistribute[DistributionSpecHash] --------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------filter((((i_color IN ('aquamarine', 'blue', 'chartreuse', 'chiffon', 'dodger', 'gainsboro', 'tan', 'violet') AND i_units IN ('Bunch', 'Dozen', 'Each', 'Ounce', 'Oz', 'Pound', 'Ton', 'Tsp')) AND ((((((item.i_category = 'Women') AND i_color IN ('aquamarine', 'gainsboro')) AND i_units IN ('Dozen', 'Ounce')) AND i_size IN ('economy', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('chiffon', 'violet')) AND i_units IN ('Pound', 'Ton')) AND i_size IN ('extra large', 'small'))) OR (((((item.i_category = 'Men') AND i_color IN ('blue', 'chartreuse')) AND i_units IN ('Each', 'Oz')) AND i_size IN ('N/A', 'large')) OR ((((item.i_category = 'Men') AND i_color IN ('dodger', 'tan')) AND i_units IN ('Bunch', 'Tsp')) AND i_size IN ('economy', 'medium'))))) OR ((i_color IN ('almond', 'blanched', 'indian', 'lime', 'peru', 'saddle', 'spring', 'tomato') AND i_units IN ('Box', 'Carton', 'Case', 'Dram', 'Gram', 'Pallet', 'Tbl', 'Unknown')) AND ((((((item.i_category = 'Women') AND i_color IN ('blanched', 'tomato')) AND i_units IN ('Case', 'Tbl')) AND i_size IN ('economy', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('almond', 'lime')) AND i_units IN ('Box', 'Dram')) AND i_size IN ('extra large', 'small'))) OR (((((item.i_category = 'Men') AND i_color IN ('peru', 'saddle')) AND i_units IN ('Gram', 'Pallet')) AND i_size IN ('N/A', 'large')) OR ((((item.i_category = 'Men') AND i_color IN ('indian', 'spring')) AND i_units IN ('Carton', 'Unknown')) AND i_size IN ('economy', 'medium')))))) and i_category IN ('Men', 'Women') and i_size IN ('N/A', 'economy', 'extra large', 'large', 'medium', 'small')) +------------------------------filter((((i_color IN ('aquamarine', 'blue', 'chartreuse', 'chiffon', 'dodger', 'gainsboro', 'tan', 'violet') AND i_units IN ('Bunch', 'Dozen', 'Each', 'Ounce', 'Oz', 'Pound', 'Ton', 'Tsp')) AND (((((((item.i_category = 'Women') AND i_color IN ('aquamarine', 'gainsboro')) AND i_units IN ('Dozen', 'Ounce')) AND i_size IN ('economy', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('chiffon', 'violet')) AND i_units IN ('Pound', 'Ton')) AND i_size IN ('extra large', 'small'))) OR ((((item.i_category = 'Men') AND i_color IN ('blue', 'chartreuse')) AND i_units IN ('Each', 'Oz')) AND i_size IN ('N/A', 'large'))) OR ((((item.i_category = 'Men') AND i_color IN ('dodger', 'tan')) AND i_units IN ('Bunch', 'Tsp')) AND i_size IN ('economy', 'medium')))) OR ((i_color IN ('almond', 'blanched', 'indian', 'lime', 'peru', 'saddle', 'spring', 'tomato') AND i_units IN ('Box', 'Carton', 'Case', 'Dram', 'Gram', 'Pallet', 'Tbl', 'Unknown')) AND (((((((item.i_category = 'Women') AND i_color IN ('blanched', 'tomato')) AND i_units IN ('Case', 'Tbl')) AND i_size IN ('economy', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('almond', 'lime')) AND i_units IN ('Box', 'Dram')) AND i_size IN ('extra large', 'small'))) OR ((((item.i_category = 'Men') AND i_color IN ('peru', 'saddle')) AND i_units IN ('Gram', 'Pallet')) AND i_size IN ('N/A', 'large'))) OR ((((item.i_category = 'Men') AND i_color IN ('indian', 'spring')) AND i_units IN ('Carton', 'Unknown')) AND i_size IN ('economy', 'medium'))))) and i_category IN ('Men', 'Women') and i_size IN ('N/A', 'economy', 'extra large', 'large', 'medium', 'small')) --------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query57.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query57.out index 7bbcf019701d4d..a0157318139d88 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query57.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query57.out @@ -21,7 +21,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------------------------PhysicalProject --------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 ------------------------------------PhysicalProject ---------------------------------------filter((((date_dim.d_year = 1999) OR ((date_dim.d_year = 1998) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2000) AND (date_dim.d_moy = 1))) and d_year IN (1998, 1999, 2000) and d_year IN (1998, 1999, 2000)) +--------------------------------------filter((((date_dim.d_year = 1999) OR ((date_dim.d_year = 1998) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2000) AND (date_dim.d_moy = 1))) and d_year IN (1998, 1999, 2000)) ----------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query41.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query41.out index c870e728a39819..ea5acccd883550 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query41.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query41.out @@ -18,6 +18,6 @@ PhysicalResultSink ------------------------PhysicalDistribute[DistributionSpecHash] --------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------filter((((i_color IN ('forest', 'lime', 'maroon', 'navy', 'powder', 'sky', 'slate', 'smoke') AND i_units IN ('Bunch', 'Case', 'Dozen', 'Gross', 'Lb', 'Ounce', 'Pallet', 'Pound')) AND ((((((item.i_category = 'Women') AND i_color IN ('forest', 'lime')) AND i_units IN ('Pallet', 'Pound')) AND i_size IN ('economy', 'small')) OR ((((item.i_category = 'Women') AND i_color IN ('navy', 'slate')) AND i_units IN ('Bunch', 'Gross')) AND i_size IN ('extra large', 'petite'))) OR (((((item.i_category = 'Men') AND i_color IN ('powder', 'sky')) AND i_units IN ('Dozen', 'Lb')) AND i_size IN ('N/A', 'large')) OR ((((item.i_category = 'Men') AND i_color IN ('maroon', 'smoke')) AND i_units IN ('Case', 'Ounce')) AND i_size IN ('economy', 'small'))))) OR ((i_color IN ('aquamarine', 'dark', 'firebrick', 'frosted', 'papaya', 'peach', 'plum', 'sienna') AND i_units IN ('Box', 'Bundle', 'Carton', 'Cup', 'Dram', 'Each', 'Tbl', 'Ton')) AND ((((((item.i_category = 'Women') AND i_color IN ('aquamarine', 'dark')) AND i_units IN ('Tbl', 'Ton')) AND i_size IN ('economy', 'small')) OR ((((item.i_category = 'Women') AND i_color IN ('frosted', 'plum')) AND i_units IN ('Box', 'Dram')) AND i_size IN ('extra large', 'petite'))) OR (((((item.i_category = 'Men') AND i_color IN ('papaya', 'peach')) AND i_units IN ('Bundle', 'Carton')) AND i_size IN ('N/A', 'large')) OR ((((item.i_category = 'Men') AND i_color IN ('firebrick', 'sienna')) AND i_units IN ('Cup', 'Each')) AND i_size IN ('economy', 'small')))))) and i_category IN ('Men', 'Women') and i_size IN ('N/A', 'economy', 'extra large', 'large', 'petite', 'small')) +------------------------------filter((((i_color IN ('forest', 'lime', 'maroon', 'navy', 'powder', 'sky', 'slate', 'smoke') AND i_units IN ('Bunch', 'Case', 'Dozen', 'Gross', 'Lb', 'Ounce', 'Pallet', 'Pound')) AND (((((((item.i_category = 'Women') AND i_color IN ('forest', 'lime')) AND i_units IN ('Pallet', 'Pound')) AND i_size IN ('economy', 'small')) OR ((((item.i_category = 'Women') AND i_color IN ('navy', 'slate')) AND i_units IN ('Bunch', 'Gross')) AND i_size IN ('extra large', 'petite'))) OR ((((item.i_category = 'Men') AND i_color IN ('powder', 'sky')) AND i_units IN ('Dozen', 'Lb')) AND i_size IN ('N/A', 'large'))) OR ((((item.i_category = 'Men') AND i_color IN ('maroon', 'smoke')) AND i_units IN ('Case', 'Ounce')) AND i_size IN ('economy', 'small')))) OR ((i_color IN ('aquamarine', 'dark', 'firebrick', 'frosted', 'papaya', 'peach', 'plum', 'sienna') AND i_units IN ('Box', 'Bundle', 'Carton', 'Cup', 'Dram', 'Each', 'Tbl', 'Ton')) AND (((((((item.i_category = 'Women') AND i_color IN ('aquamarine', 'dark')) AND i_units IN ('Tbl', 'Ton')) AND i_size IN ('economy', 'small')) OR ((((item.i_category = 'Women') AND i_color IN ('frosted', 'plum')) AND i_units IN ('Box', 'Dram')) AND i_size IN ('extra large', 'petite'))) OR ((((item.i_category = 'Men') AND i_color IN ('papaya', 'peach')) AND i_units IN ('Bundle', 'Carton')) AND i_size IN ('N/A', 'large'))) OR ((((item.i_category = 'Men') AND i_color IN ('firebrick', 'sienna')) AND i_units IN ('Cup', 'Each')) AND i_size IN ('economy', 'small'))))) and i_category IN ('Men', 'Women') and i_size IN ('N/A', 'economy', 'extra large', 'large', 'petite', 'small')) --------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query47.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query47.out index caec3b20581ed5..d51d48c5ab5677 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query47.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query47.out @@ -21,7 +21,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------------------------PhysicalProject --------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 ------------------------------------PhysicalProject ---------------------------------------filter((((date_dim.d_year = 2000) OR ((date_dim.d_year = 1999) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2001) AND (date_dim.d_moy = 1))) and d_year IN (1999, 2000, 2001) and d_year IN (1999, 2000, 2001)) +--------------------------------------filter((((date_dim.d_year = 2000) OR ((date_dim.d_year = 1999) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2001) AND (date_dim.d_moy = 1))) and d_year IN (1999, 2000, 2001)) ----------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query88.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query88.out index dc3f1ece14f64d..3c8f0335f1f415 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query88.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query88.out @@ -23,7 +23,7 @@ PhysicalResultSink ------------------------------------filter((time_dim.t_hour = 8) and (time_dim.t_minute >= 30)) --------------------------------------PhysicalOlapScan[time_dim] ------------------------------PhysicalProject ---------------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3) and hd_dep_count IN (-1, 0, 3)) +--------------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3)) ----------------------------------PhysicalOlapScan[household_demographics] --------------------------PhysicalProject ----------------------------filter((store.s_store_name = 'ese')) @@ -43,7 +43,7 @@ PhysicalResultSink ------------------------------------filter((time_dim.t_hour = 9) and (time_dim.t_minute < 30)) --------------------------------------PhysicalOlapScan[time_dim] ------------------------------PhysicalProject ---------------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3) and hd_dep_count IN (-1, 0, 3)) +--------------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3)) ----------------------------------PhysicalOlapScan[household_demographics] --------------------------PhysicalProject ----------------------------filter((store.s_store_name = 'ese')) @@ -63,7 +63,7 @@ PhysicalResultSink ----------------------------------filter((time_dim.t_hour = 9) and (time_dim.t_minute >= 30)) ------------------------------------PhysicalOlapScan[time_dim] ----------------------------PhysicalProject -------------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3) and hd_dep_count IN (-1, 0, 3)) +------------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3)) --------------------------------PhysicalOlapScan[household_demographics] ------------------------PhysicalProject --------------------------filter((store.s_store_name = 'ese')) @@ -83,7 +83,7 @@ PhysicalResultSink --------------------------------filter((time_dim.t_hour = 10) and (time_dim.t_minute < 30)) ----------------------------------PhysicalOlapScan[time_dim] --------------------------PhysicalProject -----------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3) and hd_dep_count IN (-1, 0, 3)) +----------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3)) ------------------------------PhysicalOlapScan[household_demographics] ----------------------PhysicalProject ------------------------filter((store.s_store_name = 'ese')) @@ -103,7 +103,7 @@ PhysicalResultSink ------------------------------filter((time_dim.t_hour = 10) and (time_dim.t_minute >= 30)) --------------------------------PhysicalOlapScan[time_dim] ------------------------PhysicalProject ---------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3) and hd_dep_count IN (-1, 0, 3)) +--------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3)) ----------------------------PhysicalOlapScan[household_demographics] --------------------PhysicalProject ----------------------filter((store.s_store_name = 'ese')) @@ -123,7 +123,7 @@ PhysicalResultSink ----------------------------filter((time_dim.t_hour = 11) and (time_dim.t_minute < 30)) ------------------------------PhysicalOlapScan[time_dim] ----------------------PhysicalProject -------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3) and hd_dep_count IN (-1, 0, 3)) +------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3)) --------------------------PhysicalOlapScan[household_demographics] ------------------PhysicalProject --------------------filter((store.s_store_name = 'ese')) @@ -143,7 +143,7 @@ PhysicalResultSink --------------------------filter((time_dim.t_hour = 11) and (time_dim.t_minute >= 30)) ----------------------------PhysicalOlapScan[time_dim] --------------------PhysicalProject -----------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3) and hd_dep_count IN (-1, 0, 3)) +----------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3)) ------------------------PhysicalOlapScan[household_demographics] ----------------PhysicalProject ------------------filter((store.s_store_name = 'ese')) @@ -163,7 +163,7 @@ PhysicalResultSink ------------------------filter((time_dim.t_hour = 12) and (time_dim.t_minute < 30)) --------------------------PhysicalOlapScan[time_dim] ------------------PhysicalProject ---------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3) and hd_dep_count IN (-1, 0, 3)) +--------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3)) ----------------------PhysicalOlapScan[household_demographics] --------------PhysicalProject ----------------filter((store.s_store_name = 'ese')) diff --git a/regression-test/suites/nereids_rules_p0/expression/test_simplify_range.groovy b/regression-test/suites/nereids_rules_p0/expression/test_simplify_range.groovy new file mode 100644 index 00000000000000..c6d12f3e6bc0be --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/expression/test_simplify_range.groovy @@ -0,0 +1,28 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite('test_simplify_range') { + def tbl_1 = 'test_simplify_range_tbl_1' + sql "DROP TABLE IF EXISTS ${tbl_1} FORCE" + sql "CREATE TABLE ${tbl_1}(a DECIMAL(16,8), b INT) PROPERTIES ('replication_num' = '1')" + sql "INSERT INTO ${tbl_1} VALUES(null, 10)" + test { + sql "SELECT a BETWEEN 100.02 and 40.123 OR a IN (54.0402) AND b < 10 FROM ${tbl_1}" + result([[null]]) + } + sql "DROP TABLE IF EXISTS ${tbl_1} FORCE" +} From 5641c2dcf7e08bda64b5118fca7a1905af89f56b Mon Sep 17 00:00:00 2001 From: Tiewei Fang Date: Thu, 28 Nov 2024 23:27:10 +0800 Subject: [PATCH 069/399] [fix](Outfile) Fix the data type mapping for complex types in Doris to the ORC and Parquet file formats. (#44041) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What problem does this PR solve? Problem Summary: As before, the behavior of exporting of complex data types in Doris is as follows:   | orc type | parquet type | csv -- | -- | -- | -- bitmap | string | Not Supported | Not Supported quantile_state | Not Supported | Not Supported | Not Supported hll | string | string | invisible string jsonb | Not Supported | string | string variant | Not Supported | string | string What's more, there are some issues when exporting complex data types to the ORC file format. This PR does two things: 1. Fix the problem with exporting complex data types from Doris. 2. Support exporting these three complex types to both the ORC and the Parquet file format.   | orc type | parquet type | csv -- | -- | -- | -- bitmap | binary | binary | "NULL" quantile_state | binary | binary | "NULL" hll | binary | binary | "NULL" jsonb | string | string | string variant | string | string | string ### Release note [fix](Outfile) Fix the data type mapping for complex types in Doris to the ORC and Parquet file formats. --- be/src/util/arrow/row_batch.cpp | 9 +- .../serde/data_type_bitmap_serde.cpp | 59 +++++++++- .../data_types/serde/data_type_bitmap_serde.h | 13 +-- .../serde/data_type_date64_serde.cpp | 20 +--- .../data_types/serde/data_type_hll_serde.cpp | 48 ++++---- .../data_types/serde/data_type_ipv6_serde.cpp | 37 ++---- .../serde/data_type_jsonb_serde.cpp | 25 ++++- .../serde/data_type_number_serde.cpp | 36 ++---- .../serde/data_type_object_serde.cpp | 33 ++++++ .../data_types/serde/data_type_object_serde.h | 4 +- .../serde/data_type_quantilestate_serde.h | 56 ++++++++- be/src/vec/data_types/serde/data_type_serde.h | 12 ++ .../apache/doris/analysis/OutFileClause.java | 21 ++-- .../outfile/test_outfile_complex_type.out | 25 +++++ .../test_outfile_jsonb_and_variant.out | 25 +++++ .../outfile/test_outfile_complex_type.groovy | 106 ++++++++++++++++++ .../test_outfile_jsonb_and_variant.groovy | 104 +++++++++++++++++ 17 files changed, 501 insertions(+), 132 deletions(-) create mode 100644 regression-test/data/export_p0/outfile/test_outfile_complex_type.out create mode 100644 regression-test/data/export_p0/outfile/test_outfile_jsonb_and_variant.out create mode 100644 regression-test/suites/export_p0/outfile/test_outfile_complex_type.groovy create mode 100644 regression-test/suites/export_p0/outfile/test_outfile_jsonb_and_variant.groovy diff --git a/be/src/util/arrow/row_batch.cpp b/be/src/util/arrow/row_batch.cpp index dd11d5ae46f740..a0cd77aee41931 100644 --- a/be/src/util/arrow/row_batch.cpp +++ b/be/src/util/arrow/row_batch.cpp @@ -24,6 +24,7 @@ #include #include #include +#include #include #include @@ -84,12 +85,10 @@ Status convert_to_arrow_type(const TypeDescriptor& type, std::shared_ptr #include #include @@ -27,6 +28,7 @@ #include "vec/columns/column_const.h" #include "vec/common/arena.h" #include "vec/common/assert_cast.h" +#include "vec/data_types/serde/data_type_nullable_serde.h" namespace doris { @@ -34,6 +36,29 @@ namespace vectorized { class IColumn; #include "common/compile_check_begin.h" +Status DataTypeBitMapSerDe::serialize_column_to_json(const IColumn& column, int64_t start_idx, + int64_t end_idx, BufferWritable& bw, + FormatOptions& options) const { + SERIALIZE_COLUMN_TO_JSON(); +} + +Status DataTypeBitMapSerDe::serialize_one_cell_to_json(const IColumn& column, int64_t row_num, + BufferWritable& bw, + FormatOptions& options) const { + /** + * For null values in ordinary types, we use \N to represent them; + * for null values in nested types, we use null to represent them, just like the json format. + */ + if (_nesting_level >= 2) { + bw.write(DataTypeNullableSerDe::NULL_IN_COMPLEX_TYPE.c_str(), + strlen(NULL_IN_COMPLEX_TYPE.c_str())); + } else { + bw.write(DataTypeNullableSerDe::NULL_IN_CSV_FOR_ORDINARY_TYPE.c_str(), + strlen(NULL_IN_CSV_FOR_ORDINARY_TYPE.c_str())); + } + return Status::OK(); +} + Status DataTypeBitMapSerDe::deserialize_column_from_json_vector( IColumn& column, std::vector& slices, int* num_deserialized, const FormatOptions& options) const { @@ -96,6 +121,26 @@ void DataTypeBitMapSerDe::write_one_cell_to_jsonb(const IColumn& column, JsonbWr result.writeEndBinary(); } +void DataTypeBitMapSerDe::write_column_to_arrow(const IColumn& column, const NullMap* null_map, + arrow::ArrayBuilder* array_builder, int64_t start, + int64_t end, const cctz::time_zone& ctz) const { + const auto& col = assert_cast(column); + auto& builder = assert_cast(*array_builder); + for (size_t string_i = start; string_i < end; ++string_i) { + if (null_map && (*null_map)[string_i]) { + checkArrowStatus(builder.AppendNull(), column.get_name(), + array_builder->type()->name()); + } else { + auto& bitmap_value = const_cast(col.get_element(string_i)); + std::string memory_buffer(bitmap_value.getSizeInBytes(), '0'); + bitmap_value.write_to(memory_buffer.data()); + checkArrowStatus( + builder.Append(memory_buffer.data(), static_cast(memory_buffer.size())), + column.get_name(), array_builder->type()->name()); + } + } +} + void DataTypeBitMapSerDe::read_one_cell_from_jsonb(IColumn& column, const JsonbValue* arg) const { auto& col = reinterpret_cast(column); auto blob = static_cast(arg); @@ -148,11 +193,19 @@ Status DataTypeBitMapSerDe::write_column_to_orc(const std::string& timezone, con auto& col_data = assert_cast(column); orc::StringVectorBatch* cur_batch = dynamic_cast(orc_col_batch); + INIT_MEMORY_FOR_ORC_WRITER() + for (size_t row_id = start; row_id < end; row_id++) { if (cur_batch->notNull[row_id] == 1) { - const auto& ele = col_data.get_data_at(row_id); - cur_batch->data[row_id] = const_cast(ele.data); - cur_batch->length[row_id] = ele.size; + auto bitmap_value = const_cast(col_data.get_element(row_id)); + size_t len = bitmap_value.getSizeInBytes(); + + REALLOC_MEMORY_FOR_ORC_WRITER() + + bitmap_value.write_to(const_cast(bufferRef.data) + offset); + cur_batch->data[row_id] = const_cast(bufferRef.data) + offset; + cur_batch->length[row_id] = len; + offset += len; } } diff --git a/be/src/vec/data_types/serde/data_type_bitmap_serde.h b/be/src/vec/data_types/serde/data_type_bitmap_serde.h index a4be5b8ec204f1..24c2e6f930d203 100644 --- a/be/src/vec/data_types/serde/data_type_bitmap_serde.h +++ b/be/src/vec/data_types/serde/data_type_bitmap_serde.h @@ -36,14 +36,10 @@ class DataTypeBitMapSerDe : public DataTypeSerDe { DataTypeBitMapSerDe(int nesting_level = 1) : DataTypeSerDe(nesting_level) {}; Status serialize_one_cell_to_json(const IColumn& column, int64_t row_num, BufferWritable& bw, - FormatOptions& options) const override { - return Status::NotSupported("serialize_one_cell_to_json with type [{}]", column.get_name()); - } + FormatOptions& options) const override; Status serialize_column_to_json(const IColumn& column, int64_t start_idx, int64_t end_idx, - BufferWritable& bw, FormatOptions& options) const override { - return Status::NotSupported("serialize_column_to_json with type [{}]", column.get_name()); - } + BufferWritable& bw, FormatOptions& options) const override; Status deserialize_one_cell_from_json(IColumn& column, Slice& slice, const FormatOptions& options) const override; @@ -63,10 +59,7 @@ class DataTypeBitMapSerDe : public DataTypeSerDe { void write_column_to_arrow(const IColumn& column, const NullMap* null_map, arrow::ArrayBuilder* array_builder, int64_t start, int64_t end, - const cctz::time_zone& ctz) const override { - throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, - "write_column_to_arrow with type " + column.get_name()); - } + const cctz::time_zone& ctz) const override; void read_column_from_arrow(IColumn& column, const arrow::Array* arrow_array, int start, int end, const cctz::time_zone& ctz) const override { diff --git a/be/src/vec/data_types/serde/data_type_date64_serde.cpp b/be/src/vec/data_types/serde/data_type_date64_serde.cpp index 8e102ec6e3a729..c91db85be5bcda 100644 --- a/be/src/vec/data_types/serde/data_type_date64_serde.cpp +++ b/be/src/vec/data_types/serde/data_type_date64_serde.cpp @@ -289,16 +289,7 @@ Status DataTypeDate64SerDe::write_column_to_orc(const std::string& timezone, con auto& col_data = static_cast&>(column).get_data(); orc::StringVectorBatch* cur_batch = dynamic_cast(orc_col_batch); - char* ptr = (char*)malloc(BUFFER_UNIT_SIZE); - if (!ptr) { - return Status::InternalError( - "malloc memory error when write largeint column data to orc file."); - } - StringRef bufferRef; - bufferRef.data = ptr; - bufferRef.size = BUFFER_UNIT_SIZE; - size_t offset = 0; - const size_t begin_off = offset; + INIT_MEMORY_FOR_ORC_WRITER() for (size_t row_id = start; row_id < end; row_id++) { if (cur_batch->notNull[row_id] == 0) { @@ -310,18 +301,11 @@ Status DataTypeDate64SerDe::write_column_to_orc(const std::string& timezone, con REALLOC_MEMORY_FOR_ORC_WRITER() + cur_batch->data[row_id] = const_cast(bufferRef.data) + offset; cur_batch->length[row_id] = len; offset += len; } - size_t data_off = 0; - for (size_t row_id = start; row_id < end; row_id++) { - if (cur_batch->notNull[row_id] == 1) { - cur_batch->data[row_id] = const_cast(bufferRef.data) + begin_off + data_off; - data_off += cur_batch->length[row_id]; - } - } - buffer_list.emplace_back(bufferRef); cur_batch->numElements = end - start; return Status::OK(); } diff --git a/be/src/vec/data_types/serde/data_type_hll_serde.cpp b/be/src/vec/data_types/serde/data_type_hll_serde.cpp index c22bb31862e41c..42260b092605e1 100644 --- a/be/src/vec/data_types/serde/data_type_hll_serde.cpp +++ b/be/src/vec/data_types/serde/data_type_hll_serde.cpp @@ -21,6 +21,7 @@ #include #include +#include #include #include "arrow/array/builder_binary.h" @@ -48,28 +49,17 @@ Status DataTypeHLLSerDe::serialize_column_to_json(const IColumn& column, int64_t Status DataTypeHLLSerDe::serialize_one_cell_to_json(const IColumn& column, int64_t row_num, BufferWritable& bw, FormatOptions& options) const { - if (!options._output_object_data) { - /** - * For null values in ordinary types, we use \N to represent them; - * for null values in nested types, we use null to represent them, just like the json format. - */ - if (_nesting_level >= 2) { - bw.write(DataTypeNullableSerDe::NULL_IN_COMPLEX_TYPE.c_str(), - strlen(NULL_IN_COMPLEX_TYPE.c_str())); - } else { - bw.write(DataTypeNullableSerDe::NULL_IN_CSV_FOR_ORDINARY_TYPE.c_str(), - strlen(NULL_IN_CSV_FOR_ORDINARY_TYPE.c_str())); - } - return Status::OK(); + /** + * For null values in ordinary types, we use \N to represent them; + * for null values in nested types, we use null to represent them, just like the json format. + */ + if (_nesting_level >= 2) { + bw.write(DataTypeNullableSerDe::NULL_IN_COMPLEX_TYPE.c_str(), + strlen(NULL_IN_COMPLEX_TYPE.c_str())); + } else { + bw.write(DataTypeNullableSerDe::NULL_IN_CSV_FOR_ORDINARY_TYPE.c_str(), + strlen(NULL_IN_CSV_FOR_ORDINARY_TYPE.c_str())); } - auto col_row = check_column_const_set_readability(column, row_num); - ColumnPtr ptr = col_row.first; - row_num = col_row.second; - auto& data = const_cast(assert_cast(*ptr).get_element(row_num)); - std::unique_ptr buf = - std::make_unique_for_overwrite(data.max_serialized_size()); - size_t size = data.serialize((uint8*)buf.get()); - bw.write(buf.get(), size); return Status::OK(); } @@ -139,7 +129,7 @@ void DataTypeHLLSerDe::write_column_to_arrow(const IColumn& column, const NullMa arrow::ArrayBuilder* array_builder, int64_t start, int64_t end, const cctz::time_zone& ctz) const { const auto& col = assert_cast(column); - auto& builder = assert_cast(*array_builder); + auto& builder = assert_cast(*array_builder); for (size_t string_i = start; string_i < end; ++string_i) { if (null_map && (*null_map)[string_i]) { checkArrowStatus(builder.AppendNull(), column.get_name(), @@ -198,11 +188,19 @@ Status DataTypeHLLSerDe::write_column_to_orc(const std::string& timezone, const auto& col_data = assert_cast(column); orc::StringVectorBatch* cur_batch = dynamic_cast(orc_col_batch); + INIT_MEMORY_FOR_ORC_WRITER() + for (size_t row_id = start; row_id < end; row_id++) { if (cur_batch->notNull[row_id] == 1) { - const auto& ele = col_data.get_data_at(row_id); - cur_batch->data[row_id] = const_cast(ele.data); - cur_batch->length[row_id] = ele.size; + auto hll_value = const_cast(col_data.get_element(row_id)); + size_t len = hll_value.max_serialized_size(); + + REALLOC_MEMORY_FOR_ORC_WRITER() + + hll_value.serialize((uint8_t*)(bufferRef.data) + offset); + cur_batch->data[row_id] = const_cast(bufferRef.data) + offset; + cur_batch->length[row_id] = len; + offset += len; } } diff --git a/be/src/vec/data_types/serde/data_type_ipv6_serde.cpp b/be/src/vec/data_types/serde/data_type_ipv6_serde.cpp index ac4dbc030432b8..e899de93c90ce0 100644 --- a/be/src/vec/data_types/serde/data_type_ipv6_serde.cpp +++ b/be/src/vec/data_types/serde/data_type_ipv6_serde.cpp @@ -187,38 +187,23 @@ Status DataTypeIPv6SerDe::write_column_to_orc(const std::string& timezone, const std::vector& buffer_list) const { const auto& col_data = assert_cast(column).get_data(); orc::StringVectorBatch* cur_batch = assert_cast(orc_col_batch); - char* ptr = (char*)malloc(BUFFER_UNIT_SIZE); - if (!ptr) { - return Status::InternalError( - "malloc memory error when write largeint column data to orc file."); - } - StringRef bufferRef; - bufferRef.data = ptr; - bufferRef.size = BUFFER_UNIT_SIZE; - size_t offset = 0; - const size_t begin_off = offset; - - for (size_t row_id = start; row_id < end; row_id++) { - if (cur_batch->notNull[row_id] == 0) { - continue; - } - std::string ipv6_str = IPv6Value::to_string(col_data[row_id]); - size_t len = ipv6_str.size(); - REALLOC_MEMORY_FOR_ORC_WRITER() + INIT_MEMORY_FOR_ORC_WRITER() - strcpy(const_cast(bufferRef.data) + offset, ipv6_str.c_str()); - offset += len; - cur_batch->length[row_id] = len; - } - size_t data_off = 0; for (size_t row_id = start; row_id < end; row_id++) { if (cur_batch->notNull[row_id] == 1) { - cur_batch->data[row_id] = const_cast(bufferRef.data) + begin_off + data_off; - data_off += cur_batch->length[row_id]; + std::string ipv6_str = IPv6Value::to_string(col_data[row_id]); + size_t len = ipv6_str.size(); + + REALLOC_MEMORY_FOR_ORC_WRITER() + + strcpy(const_cast(bufferRef.data) + offset, ipv6_str.c_str()); + cur_batch->data[row_id] = const_cast(bufferRef.data) + offset; + cur_batch->length[row_id] = len; + offset += len; } } - buffer_list.emplace_back(bufferRef); + cur_batch->numElements = end - start; return Status::OK(); } diff --git a/be/src/vec/data_types/serde/data_type_jsonb_serde.cpp b/be/src/vec/data_types/serde/data_type_jsonb_serde.cpp index 10218e4164d5c1..adc041f511198e 100644 --- a/be/src/vec/data_types/serde/data_type_jsonb_serde.cpp +++ b/be/src/vec/data_types/serde/data_type_jsonb_serde.cpp @@ -23,6 +23,7 @@ #include #include +#include #include "arrow/array/builder_binary.h" #include "common/exception.h" @@ -142,7 +143,29 @@ Status DataTypeJsonbSerDe::write_column_to_orc(const std::string& timezone, cons orc::ColumnVectorBatch* orc_col_batch, int64_t start, int64_t end, std::vector& buffer_list) const { - return Status::NotSupported("write_column_to_orc with type [{}]", column.get_name()); + auto* cur_batch = dynamic_cast(orc_col_batch); + const auto& string_column = assert_cast(column); + + INIT_MEMORY_FOR_ORC_WRITER() + + for (size_t row_id = start; row_id < end; row_id++) { + if (cur_batch->notNull[row_id] == 1) { + std::string_view string_ref = string_column.get_data_at(row_id).to_string_view(); + auto serialized_value = std::make_unique( + JsonbToJson::jsonb_to_json_string(string_ref.data(), string_ref.size())); + auto len = serialized_value->size(); + + REALLOC_MEMORY_FOR_ORC_WRITER() + + memcpy(const_cast(bufferRef.data) + offset, serialized_value->data(), len); + cur_batch->data[row_id] = const_cast(bufferRef.data) + offset; + cur_batch->length[row_id] = len; + offset += len; + } + } + + cur_batch->numElements = end - start; + return Status::OK(); } void convert_jsonb_to_rapidjson(const JsonbValue& val, rapidjson::Value& target, diff --git a/be/src/vec/data_types/serde/data_type_number_serde.cpp b/be/src/vec/data_types/serde/data_type_number_serde.cpp index 9416fc9a8b3020..55c7b2c9505dae 100644 --- a/be/src/vec/data_types/serde/data_type_number_serde.cpp +++ b/be/src/vec/data_types/serde/data_type_number_serde.cpp @@ -342,38 +342,22 @@ Status DataTypeNumberSerDe::write_column_to_orc(const std::string& timezone, if constexpr (std::is_same_v) { // largeint orc::StringVectorBatch* cur_batch = dynamic_cast(orc_col_batch); - char* ptr = (char*)malloc(BUFFER_UNIT_SIZE); - if (!ptr) { - return Status::InternalError( - "malloc memory error when write largeint column data to orc file."); - } - StringRef bufferRef; - bufferRef.data = ptr; - bufferRef.size = BUFFER_UNIT_SIZE; - size_t offset = 0; - const size_t begin_off = offset; + INIT_MEMORY_FOR_ORC_WRITER() for (size_t row_id = start; row_id < end; row_id++) { - if (cur_batch->notNull[row_id] == 0) { - continue; - } - std::string value_str = fmt::format("{}", col_data[row_id]); - size_t len = value_str.size(); + if (cur_batch->notNull[row_id] == 1) { + std::string value_str = fmt::format("{}", col_data[row_id]); + size_t len = value_str.size(); - REALLOC_MEMORY_FOR_ORC_WRITER() + REALLOC_MEMORY_FOR_ORC_WRITER() - strcpy(const_cast(bufferRef.data) + offset, value_str.c_str()); - offset += len; - cur_batch->length[row_id] = len; - } - size_t data_off = 0; - for (size_t row_id = start; row_id < end; row_id++) { - if (cur_batch->notNull[row_id] == 1) { - cur_batch->data[row_id] = const_cast(bufferRef.data) + begin_off + data_off; - data_off += cur_batch->length[row_id]; + strcpy(const_cast(bufferRef.data) + offset, value_str.c_str()); + cur_batch->data[row_id] = const_cast(bufferRef.data) + offset; + cur_batch->length[row_id] = len; + offset += len; } } - buffer_list.emplace_back(bufferRef); + cur_batch->numElements = end - start; } else if constexpr (std::is_same_v || std::is_same_v) { // tinyint/boolean WRITE_INTEGRAL_COLUMN_TO_ORC(orc::ByteVectorBatch) diff --git a/be/src/vec/data_types/serde/data_type_object_serde.cpp b/be/src/vec/data_types/serde/data_type_object_serde.cpp index f67194372857cf..fc536d9ef0df7b 100644 --- a/be/src/vec/data_types/serde/data_type_object_serde.cpp +++ b/be/src/vec/data_types/serde/data_type_object_serde.cpp @@ -20,6 +20,7 @@ #include #include +#include #include "common/exception.h" #include "common/status.h" @@ -164,6 +165,38 @@ void DataTypeObjectSerDe::write_column_to_arrow(const IColumn& column, const Nul } } +Status DataTypeObjectSerDe::write_column_to_orc(const std::string& timezone, const IColumn& column, + const NullMap* null_map, + orc::ColumnVectorBatch* orc_col_batch, + int64_t start, int64_t end, + std::vector& buffer_list) const { + const auto* var = check_and_get_column(column); + orc::StringVectorBatch* cur_batch = dynamic_cast(orc_col_batch); + + INIT_MEMORY_FOR_ORC_WRITER() + + for (size_t row_id = start; row_id < end; row_id++) { + if (cur_batch->notNull[row_id] == 1) { + auto serialized_value = std::make_unique(); + if (!var->serialize_one_row_to_string(row_id, serialized_value.get())) { + throw doris::Exception(ErrorCode::INTERNAL_ERROR, "Failed to serialize variant {}", + var->dump_structure()); + } + auto len = serialized_value->length(); + + REALLOC_MEMORY_FOR_ORC_WRITER() + + memcpy(const_cast(bufferRef.data) + offset, serialized_value->data(), len); + cur_batch->data[row_id] = const_cast(bufferRef.data) + offset; + cur_batch->length[row_id] = len; + offset += len; + } + } + + cur_batch->numElements = end - start; + return Status::OK(); +} + } // namespace vectorized } // namespace doris diff --git a/be/src/vec/data_types/serde/data_type_object_serde.h b/be/src/vec/data_types/serde/data_type_object_serde.h index 414755ef0f8d04..c08d4d0af0d2c3 100644 --- a/be/src/vec/data_types/serde/data_type_object_serde.h +++ b/be/src/vec/data_types/serde/data_type_object_serde.h @@ -89,9 +89,7 @@ class DataTypeObjectSerDe : public DataTypeSerDe { Status write_column_to_orc(const std::string& timezone, const IColumn& column, const NullMap* null_map, orc::ColumnVectorBatch* orc_col_batch, int64_t start, int64_t end, - std::vector& buffer_list) const override { - return Status::NotSupported("write_column_to_orc with type " + column.get_name()); - } + std::vector& buffer_list) const override; private: template diff --git a/be/src/vec/data_types/serde/data_type_quantilestate_serde.h b/be/src/vec/data_types/serde/data_type_quantilestate_serde.h index 9608bf1cb7899a..d3526ba389925f 100644 --- a/be/src/vec/data_types/serde/data_type_quantilestate_serde.h +++ b/be/src/vec/data_types/serde/data_type_quantilestate_serde.h @@ -17,6 +17,7 @@ #pragma once +#include #include #include #include @@ -32,6 +33,7 @@ #include "vec/columns/column_const.h" #include "vec/common/arena.h" #include "vec/common/string_ref.h" +#include "vec/data_types/serde/data_type_nullable_serde.h" namespace doris { @@ -43,12 +45,23 @@ class DataTypeQuantileStateSerDe : public DataTypeSerDe { Status serialize_one_cell_to_json(const IColumn& column, int64_t row_num, BufferWritable& bw, FormatOptions& options) const override { - return Status::NotSupported("serialize_one_cell_to_json with type [{}]", column.get_name()); + /** + * For null values in ordinary types, we use \N to represent them; + * for null values in nested types, we use null to represent them, just like the json format. + */ + if (_nesting_level >= 2) { + bw.write(DataTypeNullableSerDe::NULL_IN_COMPLEX_TYPE.c_str(), + strlen(NULL_IN_COMPLEX_TYPE.c_str())); + } else { + bw.write(DataTypeNullableSerDe::NULL_IN_CSV_FOR_ORDINARY_TYPE.c_str(), + strlen(NULL_IN_CSV_FOR_ORDINARY_TYPE.c_str())); + } + return Status::OK(); } Status serialize_column_to_json(const IColumn& column, int64_t start_idx, int64_t end_idx, BufferWritable& bw, FormatOptions& options) const override { - return Status::NotSupported("serialize_column_to_json with type [{}]", column.get_name()); + SERIALIZE_COLUMN_TO_JSON(); } Status deserialize_one_cell_from_json(IColumn& column, Slice& slice, const FormatOptions& options) const override { @@ -102,8 +115,21 @@ class DataTypeQuantileStateSerDe : public DataTypeSerDe { void write_column_to_arrow(const IColumn& column, const NullMap* null_map, arrow::ArrayBuilder* array_builder, int64_t start, int64_t end, const cctz::time_zone& ctz) const override { - throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, - "write_column_to_arrow with type " + column.get_name()); + const auto& col = assert_cast(column); + auto& builder = assert_cast(*array_builder); + for (size_t string_i = start; string_i < end; ++string_i) { + if (null_map && (*null_map)[string_i]) { + checkArrowStatus(builder.AppendNull(), column.get_name(), + array_builder->type()->name()); + } else { + auto& quantile_state_value = const_cast(col.get_element(string_i)); + std::string memory_buffer(quantile_state_value.get_serialized_size(), '0'); + quantile_state_value.serialize((uint8_t*)memory_buffer.data()); + checkArrowStatus(builder.Append(memory_buffer.data(), + static_cast(memory_buffer.size())), + column.get_name(), array_builder->type()->name()); + } + } } void read_column_from_arrow(IColumn& column, const arrow::Array* arrow_array, int start, int end, const cctz::time_zone& ctz) const override { @@ -126,7 +152,27 @@ class DataTypeQuantileStateSerDe : public DataTypeSerDe { const NullMap* null_map, orc::ColumnVectorBatch* orc_col_batch, int64_t start, int64_t end, std::vector& buffer_list) const override { - return Status::NotSupported("write_column_to_orc with type [{}]", column.get_name()); + auto& col_data = assert_cast(column); + orc::StringVectorBatch* cur_batch = dynamic_cast(orc_col_batch); + + INIT_MEMORY_FOR_ORC_WRITER() + + for (size_t row_id = start; row_id < end; row_id++) { + if (cur_batch->notNull[row_id] == 1) { + auto quantilestate_value = const_cast(col_data.get_element(row_id)); + size_t len = quantilestate_value.get_serialized_size(); + + REALLOC_MEMORY_FOR_ORC_WRITER() + + quantilestate_value.serialize((uint8_t*)(bufferRef.data) + offset); + cur_batch->data[row_id] = const_cast(bufferRef.data) + offset; + cur_batch->length[row_id] = len; + offset += len; + } + } + + cur_batch->numElements = end - start; + return Status::OK(); } private: diff --git a/be/src/vec/data_types/serde/data_type_serde.h b/be/src/vec/data_types/serde/data_type_serde.h index bec0fabfcca9c3..1a089bb73fe99c 100644 --- a/be/src/vec/data_types/serde/data_type_serde.h +++ b/be/src/vec/data_types/serde/data_type_serde.h @@ -77,6 +77,18 @@ struct ColumnVectorBatch; ++*num_deserialized; \ } +#define INIT_MEMORY_FOR_ORC_WRITER() \ + char* ptr = (char*)malloc(BUFFER_UNIT_SIZE); \ + if (!ptr) { \ + return Status::InternalError( \ + "malloc memory error when write largeint column data to orc file."); \ + } \ + StringRef bufferRef; \ + bufferRef.data = ptr; \ + bufferRef.size = BUFFER_UNIT_SIZE; \ + size_t offset = 0; \ + buffer_list.emplace_back(bufferRef); + #define REALLOC_MEMORY_FOR_ORC_WRITER() \ while (bufferRef.size - BUFFER_RESERVED_SIZE < offset + len) { \ char* new_ptr = (char*)malloc(bufferRef.size + BUFFER_UNIT_SIZE); \ diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/OutFileClause.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/OutFileClause.java index ef65b405853765..026e4da29b59da 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/OutFileClause.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/OutFileClause.java @@ -36,7 +36,6 @@ import org.apache.doris.common.util.Util; import org.apache.doris.datasource.property.PropertyConverter; import org.apache.doris.datasource.property.constants.S3Properties; -import org.apache.doris.qe.ConnectContext; import org.apache.doris.thrift.TFileCompressType; import org.apache.doris.thrift.TFileFormatType; import org.apache.doris.thrift.TParquetCompressionType; @@ -302,11 +301,8 @@ private String dorisTypeToOrcTypeMap(Type dorisType) throws AnalysisException { break; case HLL: case BITMAP: - if (!(ConnectContext.get() != null && ConnectContext.get() - .getSessionVariable().isReturnObjectDataAsBinary())) { - break; - } - orcType = "string"; + case QUANTILE_STATE: + orcType = "binary"; break; case DATEV2: orcType = "date"; @@ -327,6 +323,8 @@ private String dorisTypeToOrcTypeMap(Type dorisType) throws AnalysisException { case DATE: case DATETIME: case IPV6: + case VARIANT: + case JSONB: orcType = "string"; break; case DECIMALV2: @@ -445,6 +443,8 @@ private void analyzeForOrcFormat(List resultExprs, List colLabels) case DATE: case DATETIME: case IPV6: + case VARIANT: + case JSONB: checkOrcType(schema.second, "string", true, resultType.getPrimitiveType().toString()); break; case DECIMAL32: @@ -455,13 +455,8 @@ private void analyzeForOrcFormat(List resultExprs, List colLabels) break; case HLL: case BITMAP: - if (ConnectContext.get() != null && ConnectContext.get() - .getSessionVariable().isReturnObjectDataAsBinary()) { - checkOrcType(schema.second, "string", true, resultType.getPrimitiveType().toString()); - } else { - throw new AnalysisException("Orc format does not support column type: " - + resultType.getPrimitiveType()); - } + case QUANTILE_STATE: + checkOrcType(schema.second, "binary", true, resultType.getPrimitiveType().toString()); break; case STRUCT: checkOrcType(schema.second, "struct", false, resultType.getPrimitiveType().toString()); diff --git a/regression-test/data/export_p0/outfile/test_outfile_complex_type.out b/regression-test/data/export_p0/outfile/test_outfile_complex_type.out new file mode 100644 index 00000000000000..cd6f000b6c5052 --- /dev/null +++ b/regression-test/data/export_p0/outfile/test_outfile_complex_type.out @@ -0,0 +1,25 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !select_load_parquet -- +20220201 0 0000004501000000000000F03F 0101675D86AC33FA8CD6 +20220201 1 0000004501000000000000F0BF 01010B3C52B765A11A2F +20220201 2 00000045010000000000000000 0101DDEA60F9C89AA329 +20220201 3 0000004501000000000000F03F 0101EF81F59130F8B748 +20220201 4 00000045010000000000000040 010114CAA737BD54146E +20220201 5 00000045010000000000000840 0101DCBC5BA258F9602C + +-- !select_load_orc -- +20220201 0 0000004501000000000000F03F 0101675D86AC33FA8CD6 +20220201 1 0000004501000000000000F0BF 01010B3C52B765A11A2F +20220201 2 00000045010000000000000000 0101DDEA60F9C89AA329 +20220201 3 0000004501000000000000F03F 0101EF81F59130F8B748 +20220201 4 00000045010000000000000040 010114CAA737BD54146E +20220201 5 00000045010000000000000840 0101DCBC5BA258F9602C + +-- !select_load_csv -- +20220201 0 \N \N \N +20220201 1 \N \N \N +20220201 2 \N \N \N +20220201 3 \N \N \N +20220201 4 \N \N \N +20220201 5 \N \N \N + diff --git a/regression-test/data/export_p0/outfile/test_outfile_jsonb_and_variant.out b/regression-test/data/export_p0/outfile/test_outfile_jsonb_and_variant.out new file mode 100644 index 00000000000000..d25830939645f6 --- /dev/null +++ b/regression-test/data/export_p0/outfile/test_outfile_jsonb_and_variant.out @@ -0,0 +1,25 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !select_load_parquet -- +20220201 0 {"k1":"100"} {"k1":"100"} +20220201 1 {"k1":"100","k2":"123"} {"k1":"100","k2":"123"} +20220201 2 {"k1":"100","abc":"567"} {"abc":"567","k1":"100"} +20220201 3 {"k1":"100","k3":123} {"k1":"100","k3":123} +20220201 4 {"k1":"100","doris":"nereids"} {"doris":"nereids","k1":"100"} +20220201 5 {"k1":"100","doris":"pipeline"} {"doris":"pipeline","k1":"100"} + +-- !select_load_orc -- +20220201 0 {"k1":"100"} {"k1":"100"} +20220201 1 {"k1":"100","k2":"123"} {"k1":"100","k2":"123"} +20220201 2 {"k1":"100","abc":"567"} {"abc":"567","k1":"100"} +20220201 3 {"k1":"100","k3":123} {"k1":"100","k3":123} +20220201 4 {"k1":"100","doris":"nereids"} {"doris":"nereids","k1":"100"} +20220201 5 {"k1":"100","doris":"pipeline"} {"doris":"pipeline","k1":"100"} + +-- !select_load_orc -- +20220201 0 {"k1":"100"} {"k1":"100"} +20220201 1 {"k1":"100","k2":"123"} {"k1":"100","k2":"123"} +20220201 2 {"k1":"100","abc":"567"} {"abc":"567","k1":"100"} +20220201 3 {"k1":"100","k3":123} {"k1":"100","k3":123} +20220201 4 {"k1":"100","doris":"nereids"} {"doris":"nereids","k1":"100"} +20220201 5 {"k1":"100","doris":"pipeline"} {"doris":"pipeline","k1":"100"} + diff --git a/regression-test/suites/export_p0/outfile/test_outfile_complex_type.groovy b/regression-test/suites/export_p0/outfile/test_outfile_complex_type.groovy new file mode 100644 index 00000000000000..49f81732791126 --- /dev/null +++ b/regression-test/suites/export_p0/outfile/test_outfile_complex_type.groovy @@ -0,0 +1,106 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("test_outfile_complex_type", "p0") { + String ak = getS3AK() + String sk = getS3SK() + String s3_endpoint = getS3Endpoint() + String region = getS3Region() + String bucket = context.config.otherConfigs.get("s3BucketName"); + + def export_table_name = "test_outfile_complex_type_table" + def outFilePath = "${bucket}/outfile/complex_type/exp_" + + def outfile_to_S3 = { format -> + // select ... into outfile ... + def res = sql """ + SELECT * FROM ${export_table_name} t + INTO OUTFILE "s3://${outFilePath}" + FORMAT AS ${format} + PROPERTIES ( + "s3.endpoint" = "${s3_endpoint}", + "s3.region" = "${region}", + "s3.secret_key"="${sk}", + "s3.access_key" = "${ak}" + ); + """ + + return res[0][3] + } + + sql """ DROP TABLE IF EXISTS ${export_table_name} """ + sql """ + CREATE TABLE `${export_table_name}` ( + `dt` int(11) NULL COMMENT "", + `id` int(11) NULL COMMENT "", + `price` quantile_state QUANTILE_UNION NOT NULL COMMENT "", + `hll_t` hll hll_union, + `device_id` bitmap BITMAP_UNION + ) ENGINE=OLAP + AGGREGATE KEY(`dt`, `id`) + DISTRIBUTED BY HASH(`dt`) + PROPERTIES ( + "replication_num" = "1" + ); + """ + + sql """ + INSERT INTO `${export_table_name}` values + (20220201,0, to_quantile_state(1, 2048), hll_hash(1), to_bitmap(243)), + (20220201,1, to_quantile_state(-1, 2048), hll_hash(2), bitmap_from_array([1,2,3,4,5,434543])), + (20220201,2, to_quantile_state(0, 2048), hll_hash(3), to_bitmap(1234566)), + (20220201,3, to_quantile_state(1, 2048), hll_hash(4), to_bitmap(8888888888888)), + (20220201,4, to_quantile_state(2, 2048), hll_hash(5), to_bitmap(98392819412234)), + (20220201,5, to_quantile_state(3, 2048), hll_hash(6), to_bitmap(253234234)); + """ + + // parquet file format + def format = "parquet" + def outfile_url = outfile_to_S3("${format}") + qt_select_load_parquet """ SELECT dt, id, hex(price), hex(hll_t) FROM S3 ( + "uri" = "http://${bucket}.${s3_endpoint}${outfile_url.substring(5 + bucket.length(), outfile_url.length() - 1)}0.${format}", + "ACCESS_KEY"= "${ak}", + "SECRET_KEY" = "${sk}", + "format" = "${format}", + "region" = "${region}" + ); + """ + + // orc file foramt + format = "orc" + outfile_url = outfile_to_S3("${format}") + qt_select_load_orc """ SELECT dt, id, hex(price), hex(hll_t) FROM S3 ( + "uri" = "http://${bucket}.${s3_endpoint}${outfile_url.substring(5 + bucket.length(), outfile_url.length() - 1)}0.${format}", + "ACCESS_KEY"= "${ak}", + "SECRET_KEY" = "${sk}", + "format" = "${format}", + "region" = "${region}" + ); + """ + + // csv file foramt + format = "csv" + outfile_url = outfile_to_S3("${format}") + qt_select_load_csv """ SELECT * FROM S3 ( + "uri" = "http://${bucket}.${s3_endpoint}${outfile_url.substring(5 + bucket.length(), outfile_url.length() - 1)}0.${format}", + "ACCESS_KEY"= "${ak}", + "SECRET_KEY" = "${sk}", + "format" = "${format}", + "region" = "${region}" + ); + """ +} \ No newline at end of file diff --git a/regression-test/suites/export_p0/outfile/test_outfile_jsonb_and_variant.groovy b/regression-test/suites/export_p0/outfile/test_outfile_jsonb_and_variant.groovy new file mode 100644 index 00000000000000..ed3019436aef29 --- /dev/null +++ b/regression-test/suites/export_p0/outfile/test_outfile_jsonb_and_variant.groovy @@ -0,0 +1,104 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("test_outfile_jsonb_and_variant", "p0") { + String ak = getS3AK() + String sk = getS3SK() + String s3_endpoint = getS3Endpoint() + String region = getS3Region() + String bucket = context.config.otherConfigs.get("s3BucketName"); + + def export_table_name = "test_outfile_jsonb_and_variant_table" + def outFilePath = "${bucket}/outfile/jsonb_and_variant/exp_" + + def outfile_to_S3 = { format -> + // select ... into outfile ... + def res = sql """ + SELECT * FROM ${export_table_name} t + INTO OUTFILE "s3://${outFilePath}" + FORMAT AS ${format} + PROPERTIES ( + "s3.endpoint" = "${s3_endpoint}", + "s3.region" = "${region}", + "s3.secret_key"="${sk}", + "s3.access_key" = "${ak}" + ); + """ + + return res[0][3] + } + + sql """ DROP TABLE IF EXISTS ${export_table_name} """ + sql """ + CREATE TABLE `${export_table_name}` ( + `dt` int(11) NULL COMMENT "", + `id` int(11) NULL COMMENT "", + `json_col` JSON NULL COMMENT "", + `variant_col` variant NULL COMMENT "" + ) ENGINE=OLAP + DISTRIBUTED BY HASH(`dt`) + PROPERTIES ( + "replication_num" = "1" + ); + """ + + sql """ + INSERT INTO `${export_table_name}` values + (20220201,0, '{"k1": "100"}', '{"k1": "100"}'), + (20220201,1, '{"k1": "100", "k2": "123"}', '{"k1": "100", "k2": "123"}'), + (20220201,2, '{"k1": "100", "abc": "567"}', '{"k1": "100", "abc": "567"}'), + (20220201,3, '{"k1": "100", "k3": 123}', '{"k1": "100", "k3": 123}'), + (20220201,4, '{"k1": "100", "doris": "nereids"}', '{"k1": "100", "doris": "nereids"}'), + (20220201,5, '{"k1": "100", "doris": "pipeline"}', '{"k1": "100", "doris": "pipeline"}'); + """ + + // parquet file format + def format = "parquet" + def outfile_url = outfile_to_S3("${format}") + qt_select_load_parquet """ SELECT * FROM S3 ( + "uri" = "http://${bucket}.${s3_endpoint}${outfile_url.substring(5 + bucket.length(), outfile_url.length() - 1)}0.${format}", + "ACCESS_KEY"= "${ak}", + "SECRET_KEY" = "${sk}", + "format" = "${format}", + "region" = "${region}" + ); + """ + + // orc file foramt + format = "orc" + outfile_url = outfile_to_S3("${format}") + qt_select_load_orc """ SELECT * FROM S3 ( + "uri" = "http://${bucket}.${s3_endpoint}${outfile_url.substring(5 + bucket.length(), outfile_url.length() - 1)}0.${format}", + "ACCESS_KEY"= "${ak}", + "SECRET_KEY" = "${sk}", + "format" = "${format}", + "region" = "${region}" + ); + """ + + // orc file foramt + format = "csv" + outfile_url = outfile_to_S3("${format}") + qt_select_load_orc """ SELECT * FROM S3 ( + "uri" = "http://${bucket}.${s3_endpoint}${outfile_url.substring(5 + bucket.length(), outfile_url.length() - 1)}0.${format}", + "ACCESS_KEY"= "${ak}", + "SECRET_KEY" = "${sk}", + "format" = "${format}", + "region" = "${region}" + ); + """ +} \ No newline at end of file From 24f1eeffa3cb5533d28095a6b4d9434a7e3e5d10 Mon Sep 17 00:00:00 2001 From: zy-kkk Date: Thu, 28 Nov 2024 23:27:41 +0800 Subject: [PATCH 070/399] [Enhancement](jdbc catalog) Support reading some array types of PostgreSQL (#44160) ### What problem does this PR solve? Issue Number: close #xxx Related PR: #xxx Problem Summary: This PR supports reading some PostgreSQL array types, including: ``` { "int2", "int4", "int8", "smallserial", "serial", "bigserial", "float4", "float8", "timestamp", "timestamptz", "date", "bool", "bpchar", "varchar", "text" } ``` ### Release note [feat](jdbc catalog) Support reading some array types of PostgreSQL --- .../postgresql/init/02-create-table.sql | 88 ++++++++ .../postgresql/init/04-insert.sql | 210 ++++++++++++++++++ .../doris/jdbc/PostgreSQLJdbcExecutor.java | 85 ++++++- .../java/org/apache/doris/common/Config.java | 2 +- .../jdbc/client/JdbcPostgreSQLClient.java | 92 +++++++- .../datasource/jdbc/util/JdbcFieldSchema.java | 15 ++ .../jdbc/test_pg_jdbc_catalog.out | 11 + .../jdbc/test_pg_jdbc_catalog.groovy | 5 + 8 files changed, 504 insertions(+), 4 deletions(-) diff --git a/docker/thirdparties/docker-compose/postgresql/init/02-create-table.sql b/docker/thirdparties/docker-compose/postgresql/init/02-create-table.sql index 7983d55a018917..bafdd615839e14 100644 --- a/docker/thirdparties/docker-compose/postgresql/init/02-create-table.sql +++ b/docker/thirdparties/docker-compose/postgresql/init/02-create-table.sql @@ -263,3 +263,91 @@ CREATE TABLE catalog_pg_test.num_zero ( id varchar(20) NULL, num numeric NULL ); + +CREATE TABLE catalog_pg_test.test_all_types_array ( + id int4 NOT NULL, + char_value _bpchar NULL, + varchar_value _varchar NULL, + date_value _date NULL, + smallint_value _int2 NULL, + int_value _int4 NULL, + bigint_value _int8 NULL, + timestamp_value _timestamp NULL, + decimal_value _numeric NULL, + bit_value _bit NULL, + real_value _float4 NULL, + cidr_value _cidr NULL, + inet_value _inet NULL, + macaddr_value _macaddr NULL, + bitn_value _bit NULL, + bitnv_value _varbit NULL, + jsonb_value _jsonb NULL, + point_value _point NULL, + line_value _line NULL, + lseg_value _lseg NULL, + box_value _box NULL, + path_value _path NULL, + polygon_value _polygon NULL, + circle_value _circle NULL +); + + +CREATE TABLE catalog_pg_test.test_all_types_2d_array ( + id int4 NOT NULL, + char_value _bpchar NULL, + varchar_value _varchar NULL, + date_value _date NULL, + smallint_value _int2 NULL, + int_value _int4 NULL, + bigint_value _int8 NULL, + timestamp_value _timestamp NULL, + decimal_value _numeric NULL, + bit_value _bit NULL, + real_value _float4 NULL, + cidr_value _cidr NULL, + inet_value _inet NULL, + macaddr_value _macaddr NULL, + bitn_value _bit NULL, + bitnv_value _varbit NULL, + jsonb_value _jsonb NULL, + point_value _point NULL, + line_value _line NULL, + lseg_value _lseg NULL, + box_value _box NULL, + path_value _path NULL, + polygon_value _polygon NULL, + circle_value _circle NULL +); + +CREATE TABLE catalog_pg_test.test_all_support_types_array ( + id int4 NOT NULL, + char_value bpchar[] NULL, + varchar_value varchar[] NULL, + text_value text[] null, + date_value date[] NULL, + smallint_value int2[] NULL, + int_value int4[] NULL, + bigint_value int8[] NULL, + timestamp_value timestamp[] NULL, + timestamptz_value timestamptz[] null, + bool_value bool[] NULL, + float4_value float4[] null, + float8_value float8[] null +); + +CREATE TABLE catalog_pg_test.test_all_support_types_array_2d ( + id int4 NOT NULL, + char_value bpchar[][] NULL, + varchar_value varchar[][] NULL, + text_value text[][] NULL, + date_value date[][] NULL, + smallint_value int2[][] NULL, + int_value int4[][] NULL, + bigint_value int8[][] NULL, + timestamp_value timestamp[][] NULL, + timestamptz_value timestamptz[][] NULL, + bool_value bool[][] NULL, + float4_value float4[][] NULL, + float8_value float8[][] NULL +); + diff --git a/docker/thirdparties/docker-compose/postgresql/init/04-insert.sql b/docker/thirdparties/docker-compose/postgresql/init/04-insert.sql index ae3570dfc186ce..b2a14148ead53e 100644 --- a/docker/thirdparties/docker-compose/postgresql/init/04-insert.sql +++ b/docker/thirdparties/docker-compose/postgresql/init/04-insert.sql @@ -2730,3 +2730,213 @@ values('1','A','2020-09-02 00:00:00') INSERT INTO catalog_pg_test.num_zero (id, num) VALUES ('001', 123.45); INSERT INTO catalog_pg_test.num_zero (id, num) VALUES ('002', 678.90); INSERT INTO catalog_pg_test.num_zero (id, num) VALUES ('003', 1011.12); + +INSERT INTO catalog_pg_test.test_all_types_2d_array VALUES +(1, + ARRAY[ARRAY['abc', 'def']::text[], ARRAY['ghi', 'jkl']::text[]], + ARRAY[ARRAY['abc', 'def']::text[], ARRAY['ghi', 'jkl']::text[]], + ARRAY[ARRAY['2022-10-11'::date, '2022-11-12'::date], ARRAY['2023-01-01'::date, '2023-02-02'::date]], + ARRAY[ARRAY[1, 2], ARRAY[3, 4]], + ARRAY[ARRAY[1, 2], ARRAY[2, 3]], + ARRAY[ARRAY[3, 4], ARRAY[4, 5]], + ARRAY[ARRAY['2022-10-22 10:59:59'::timestamp, '2023-01-01 12:00:00'::timestamp], + ARRAY['2023-02-01 14:30:00'::timestamp, '2023-02-10 16:45:00'::timestamp]], + ARRAY[ARRAY[34.123, 45.678], ARRAY[56.789, 67.890]], + ARRAY[ARRAY[0::bit, 1::bit], ARRAY[1::bit, 0::bit]], + ARRAY[ARRAY[12.123456, 34.567890], ARRAY[56.789123, 45.678345]], + ARRAY[ARRAY['10.16.10.14'::cidr, '192.168.1.1'::cidr], + ARRAY['172.16.0.1'::cidr, '192.168.2.2'::cidr]], + ARRAY[ARRAY['10.16.10.14'::inet, '192.168.1.1'::inet], + ARRAY['172.16.0.1'::inet, '192.168.2.2'::inet]], + ARRAY[ARRAY['ff:ff:ff:ff:ff:AA'::macaddr, '00:14:22:01:23:45'::macaddr], + ARRAY['11:22:33:44:55:66'::macaddr, '77:88:99:00:11:22'::macaddr]], + ARRAY[ARRAY['1010101010'::bit(10), '1100101010'::bit(10)], + ARRAY['1110001111'::bit(10), '0001110000'::bit(10)]], + ARRAY[ARRAY[cast(10 as bit(5)), cast(20 as bit(5))], + ARRAY[cast(30 as bit(5)), cast(40 as bit(5))]], + ARRAY[ARRAY['{"id":1}'::jsonb, '{"id":2}'::jsonb], + ARRAY['{"id":3}'::jsonb, '{"id":4}'::jsonb]], + ARRAY[ARRAY['(1,1)'::point, '(2,2)'::point], + ARRAY['(3,3)'::point, '(4,4)'::point]], + ARRAY[ARRAY['{1,1,1}'::line, '{2,2,2}'::line], + ARRAY['{3,3,3}'::line, '{4,4,4}'::line]], + ARRAY[ARRAY['(1,1),(2,2)'::lseg, '(3,3),(4,4)'::lseg], + ARRAY['(5,5),(6,6)'::lseg, '(7,7),(8,8)'::lseg]], + ARRAY[ARRAY['(1,1),(2,2)'::box, '(5,5),(6,6)'::box], + ARRAY['(7,7),(8,8)'::box, '(9,9),(10,10)'::box]], + ARRAY[ARRAY['(1,1),(2,2),(2,1)'::path, '(3,3),(4,4),(5,5)'::path], + ARRAY['(6,6),(7,7),(8,8)'::path, '(9,9),(10,10),(11,11)'::path]], + ARRAY[ARRAY['((1,1),(2,2),(2,1))'::polygon, '((3,3),(4,4),(5,5))'::polygon], + ARRAY['((6,6),(7,7),(8,8))'::polygon, '((9,9),(10,10),(11,11))'::polygon]], + ARRAY[ARRAY['<(0,0),1>'::circle, '<(5,5),10>'::circle], + ARRAY['<(2,2),3>'::circle, '<(7,7),14>'::circle]] +); + +INSERT INTO catalog_pg_test.test_all_types_array VALUES +(1, + ARRAY['abc', 'def'], + ARRAY['ghi', 'jkl'], + ARRAY['2022-10-11'::date, '2022-11-12'::date], + ARRAY[1, 2], + ARRAY[2, 3], + ARRAY[3, 4], + ARRAY['2022-10-22 10:59:59'::timestamp, '2023-01-01 12:00:00'::timestamp], + ARRAY[34.123, 45.678], + ARRAY[0::bit, 1::bit], + ARRAY[12.123456, 34.567890], + ARRAY['10.16.10.14'::cidr, '192.168.1.1'::cidr], + ARRAY['10.16.10.14'::inet, '192.168.0.1'::inet], + ARRAY['ff:ff:ff:ff:ff:AA'::macaddr, '00:14:22:01:23:45'::macaddr], + ARRAY['1010101010'::bit(10), '1100101010'::bit(10)], + ARRAY[cast('00010' as bit(5)), cast('10100' as bit(5))], -- bit(5) values need 5 bits + ARRAY['{"id":1}'::jsonb, '{"id":2}'::jsonb], + ARRAY['(1,1)'::point, '(2,2)'::point], + ARRAY['{1,1,1}'::line, '{2,2,2}'::line], + ARRAY['(1,1),(2,2)'::lseg, '(3,3),(4,4)'::lseg], + ARRAY['(1,1),(2,2)'::box, '(5,5),(6,6)'::box], + ARRAY['(1,1),(2,2),(2,1)'::path, '(3,3),(4,4),(5,5)'::path], + ARRAY['((1,1),(2,2),(2,1))'::polygon, '((3,3),(4,4),(5,5))'::polygon], + ARRAY['<(0,0),1>'::circle, '<(5,5),10>'::circle] +); + +INSERT INTO catalog_pg_test.test_all_support_types_array ( + id, char_value, varchar_value, text_value, date_value, + smallint_value, int_value, bigint_value, timestamp_value, + timestamptz_value, bool_value, float4_value, float8_value +) VALUES ( + 1, + ARRAY['A', 'B', 'C'], + ARRAY['Hello', 'World'], + ARRAY['Text1', 'Text2'], + ARRAY['2024-01-01'::date, '2024-01-02'::date], + ARRAY[10, 20], + ARRAY[100, 200], + ARRAY[1000, 2000], + ARRAY['2024-01-01 10:00:00'::timestamp, '2024-01-02 12:00:00'::timestamp], + ARRAY['2024-01-01 10:00:00+00'::timestamptz, '2024-01-02 12:00:00+00'::timestamptz], + ARRAY[true, false], + ARRAY[1.1, 2.2], + ARRAY[3.3, 4.4] +); + +INSERT INTO catalog_pg_test.test_all_support_types_array ( + id, char_value, varchar_value, text_value, date_value, + smallint_value, int_value, bigint_value, timestamp_value, + timestamptz_value, bool_value, float4_value, float8_value +) VALUES ( + 2, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null +); + +INSERT INTO catalog_pg_test.test_all_support_types_array ( + id, char_value, varchar_value, text_value, date_value, + smallint_value, int_value, bigint_value, timestamp_value, + timestamptz_value, bool_value, float4_value, float8_value +) VALUES ( + 3, + ARRAY['A', 'B', 'C', null], + ARRAY['Hello', 'World', null], + ARRAY['Text1', 'Text2', null], + ARRAY['2024-01-01'::date, '2024-01-02'::date, null], + ARRAY[10, 20, null], + ARRAY[100, 200, null], + ARRAY[1000, 2000, null], + ARRAY['2024-01-01 10:00:00'::timestamp, '2024-01-02 12:00:00'::timestamp, null], + ARRAY['2024-01-01 10:00:00+00'::timestamptz, '2024-01-02 12:00:00+00'::timestamptz, null], + ARRAY[true, false, null], + ARRAY[1.1, 2.2, null], + ARRAY[3.3, 4.4, null] +); + +INSERT INTO catalog_pg_test.test_all_support_types_array_2d ( + id, char_value, varchar_value, text_value, date_value, + smallint_value, int_value, bigint_value, timestamp_value, + timestamptz_value, bool_value, float4_value, float8_value +) VALUES ( + 1, + ARRAY[ARRAY['A', 'B'], ARRAY['C', 'D']], + ARRAY[ARRAY['Hello', 'World'], ARRAY['Foo', 'Bar']], + ARRAY[ARRAY['Text1', 'Text2'], ARRAY['Text3', 'Text4']], + ARRAY[ARRAY['2024-01-01'::date, '2024-01-02'::date], ARRAY['2024-02-01'::date, '2024-02-02'::date]], + ARRAY[ARRAY[10, 20], ARRAY[30, 40]], + ARRAY[ARRAY[100, 200], ARRAY[300, 400]], + ARRAY[ARRAY[1000, 2000], ARRAY[3000, 4000]], + ARRAY[ARRAY['2024-01-01 10:00:00'::timestamp, '2024-01-02 12:00:00'::timestamp], ARRAY['2024-03-01 10:00:00'::timestamp, '2024-03-02 12:00:00'::timestamp]], + ARRAY[ARRAY['2024-01-01 10:00:00+00'::timestamptz, '2024-01-02 12:00:00+00'::timestamptz], ARRAY['2024-03-01 10:00:00+00'::timestamptz, '2024-03-02 12:00:00+00'::timestamptz]], + ARRAY[ARRAY[true, false], ARRAY[false, true]], + ARRAY[ARRAY[1.1, 2.2], ARRAY[3.3, 4.4]], + ARRAY[ARRAY[5.5, 6.6], ARRAY[7.7, 8.8]] +); + +INSERT INTO catalog_pg_test.test_all_support_types_array_2d ( + id, char_value, varchar_value, text_value, date_value, + smallint_value, int_value, bigint_value, timestamp_value, + timestamptz_value, bool_value, float4_value, float8_value +) VALUES ( + 2, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null +); + +INSERT INTO catalog_pg_test.test_all_support_types_array_2d ( + id, char_value, varchar_value, text_value, date_value, + smallint_value, int_value, bigint_value, timestamp_value, + timestamptz_value, bool_value, float4_value, float8_value +) VALUES ( + 3, + ARRAY[ARRAY['A', null], ARRAY['C', 'D']], + ARRAY[ARRAY['Hello', null], ARRAY['Foo', 'Bar']], + ARRAY[ARRAY['Text1', null], ARRAY['Text3', 'Text4']], + ARRAY[ARRAY['2024-01-01'::date, null], ARRAY['2024-02-01'::date, '2024-02-02'::date]], + ARRAY[ARRAY[10, null], ARRAY[30, 40]], + ARRAY[ARRAY[100, null], ARRAY[300, 400]], + ARRAY[ARRAY[1000, null], ARRAY[3000, 4000]], + ARRAY[ARRAY['2024-01-01 10:00:00'::timestamp, null], ARRAY['2024-03-01 10:00:00'::timestamp, '2024-03-02 12:00:00'::timestamp]], + ARRAY[ARRAY['2024-01-01 10:00:00+00'::timestamptz, null], ARRAY['2024-03-01 10:00:00+00'::timestamptz, '2024-03-02 12:00:00+00'::timestamptz]], + ARRAY[ARRAY[true,null], ARRAY[false, true]], + ARRAY[ARRAY[1.1, null], ARRAY[3.3, 4.4]], + ARRAY[ARRAY[5.5, null], ARRAY[7.7, 8.8]] +); + + +INSERT INTO catalog_pg_test.test_all_support_types_array_2d ( + id, char_value, varchar_value, text_value, date_value, + smallint_value, int_value, bigint_value, timestamp_value, + timestamptz_value, bool_value, float4_value, float8_value +) VALUES ( + 4, + ARRAY[ARRAY['A', 'B'], ARRAY[NULL, NULL]], + ARRAY[ARRAY['Hello', 'World'], ARRAY[NULL, NULL]], + ARRAY[ARRAY['Text1', 'Text2'], ARRAY[NULL, NULL]], + ARRAY[ARRAY['2024-01-01'::date, '2024-01-02'::date], ARRAY[NULL::date, NULL::date]], + ARRAY[ARRAY[10, 20], ARRAY[NULL::smallint, NULL::smallint]], + ARRAY[ARRAY[100, 200], ARRAY[NULL::int, NULL::int]], + ARRAY[ARRAY[1000, 2000], ARRAY[NULL::bigint, NULL::bigint]], + ARRAY[ARRAY['2024-01-01 10:00:00'::timestamp, '2024-01-02 12:00:00'::timestamp], ARRAY[NULL::timestamp, NULL::timestamp]], + ARRAY[ARRAY['2024-01-01 10:00:00+00'::timestamptz, '2024-01-02 12:00:00+00'::timestamptz], ARRAY[NULL::timestamptz, NULL::timestamptz]], + ARRAY[ARRAY[true, false], ARRAY[NULL::boolean, NULL::boolean]], + ARRAY[ARRAY[1.1, 2.2], ARRAY[NULL::real, NULL::real]], + ARRAY[ARRAY[5.5, 6.6], ARRAY[NULL::double precision, NULL::double precision]] +); + diff --git a/fe/be-java-extensions/jdbc-scanner/src/main/java/org/apache/doris/jdbc/PostgreSQLJdbcExecutor.java b/fe/be-java-extensions/jdbc-scanner/src/main/java/org/apache/doris/jdbc/PostgreSQLJdbcExecutor.java index 89fdb0ba997773..c25bcec9a497f1 100644 --- a/fe/be-java-extensions/jdbc-scanner/src/main/java/org/apache/doris/jdbc/PostgreSQLJdbcExecutor.java +++ b/fe/be-java-extensions/jdbc-scanner/src/main/java/org/apache/doris/jdbc/PostgreSQLJdbcExecutor.java @@ -22,14 +22,22 @@ import org.apache.doris.common.jni.vec.ColumnValueConverter; import org.apache.doris.common.jni.vec.VectorTable; +import com.google.common.collect.Lists; +import org.apache.log4j.Logger; + import java.math.BigDecimal; +import java.sql.Date; import java.sql.SQLException; import java.sql.Timestamp; import java.time.LocalDate; import java.time.LocalDateTime; import java.time.OffsetDateTime; +import java.util.ArrayList; +import java.util.List; public class PostgreSQLJdbcExecutor extends BaseJdbcExecutor { + private static final Logger LOG = Logger.getLogger(PostgreSQLJdbcExecutor.class); + public PostgreSQLJdbcExecutor(byte[] thriftParams) throws Exception { super(thriftParams); } @@ -41,7 +49,8 @@ protected void initializeBlock(int columnCount, String[] replaceStringList, int if (outputTable.getColumnType(i).getType() == Type.DATETIME || outputTable.getColumnType(i).getType() == Type.DATETIMEV2) { block.add(new Object[batchSizeNum]); - } else if (outputTable.getColumnType(i).getType() == Type.STRING) { + } else if (outputTable.getColumnType(i).getType() == Type.STRING + || outputTable.getColumnType(i).getType() == Type.ARRAY) { block.add(new Object[batchSizeNum]); } else { block.add(outputTable.getColumn(i).newObjectContainerArray(batchSizeNum)); @@ -78,6 +87,9 @@ protected Object getColumnValue(int columnIndex, ColumnType type, String[] repla case VARCHAR: case STRING: return resultSet.getObject(columnIndex + 1); + case ARRAY: + java.sql.Array array = resultSet.getArray(columnIndex + 1); + return array == null ? null : convertArrayToList(array.getArray()); default: throw new IllegalArgumentException("Unsupported column type: " + type.getType()); } @@ -110,6 +122,10 @@ protected ColumnValueConverter getOutputConverter(ColumnType columnType, String return input.toString(); } }, String.class); + case ARRAY: + return createConverter( + (Object input) -> convertArray((List) input, columnType.getChildTypes().get(0)), + List.class); default: return null; } @@ -122,4 +138,71 @@ private static String pgByteArrayToHexString(byte[] bytes) { } return hexString.toString(); } + + private List convertArrayToList(Object array) { + if (array == null) { + return null; + } + + int length = java.lang.reflect.Array.getLength(array); + List list = new ArrayList<>(length); + + for (int i = 0; i < length; i++) { + Object element = java.lang.reflect.Array.get(array, i); + list.add(element); + } + + return list; + } + + private List convertArray(List array, ColumnType type) { + if (array == null) { + return null; + } + switch (type.getType()) { + case DATE: + case DATEV2: { + List result = new ArrayList<>(); + for (Object element : array) { + if (element == null) { + result.add(null); + } else { + result.add(((Date) element).toLocalDate()); + } + } + return result; + } + case DATETIME: + case DATETIMEV2: { + List result = new ArrayList<>(); + for (Object element : array) { + if (element == null) { + result.add(null); + } else { + if (element instanceof Timestamp) { + result.add(((Timestamp) element).toLocalDateTime()); + } else if (element instanceof OffsetDateTime) { + result.add(((OffsetDateTime) element).toLocalDateTime()); + } else { + result.add((LocalDateTime) element); + } + } + } + return result; + } + case ARRAY: + List> resultArray = Lists.newArrayList(); + for (Object element : array) { + if (element == null) { + resultArray.add(null); + } else { + List nestedList = convertArrayToList(element); + resultArray.add(convertArray(nestedList, type.getChildTypes().get(0))); + } + } + return resultArray; + default: + return array; + } + } } diff --git a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java index 66dbff5abed8e6..3caa137c858440 100644 --- a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java +++ b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java @@ -164,7 +164,7 @@ public class Config extends ConfigBase { "MySQL Jdbc Catalog mysql does not support pushdown functions"}) public static String[] jdbc_mysql_unsupported_pushdown_functions = {"date_trunc", "money_format", "negative"}; - @ConfField(description = {"强制 SQLServer Jdbc Catalog 加密为 false", + @ConfField(mutable = true, masterOnly = true, description = {"强制 SQLServer Jdbc Catalog 加密为 false", "Force SQLServer Jdbc Catalog encrypt to false"}) public static boolean force_sqlserver_jdbc_encrypt_false = false; diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcPostgreSQLClient.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcPostgreSQLClient.java index e6694ffdc674bb..481e5ea5e490c6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcPostgreSQLClient.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcPostgreSQLClient.java @@ -17,17 +17,75 @@ package org.apache.doris.datasource.jdbc.client; +import org.apache.doris.catalog.ArrayType; import org.apache.doris.catalog.PrimitiveType; import org.apache.doris.catalog.ScalarType; import org.apache.doris.catalog.Type; +import org.apache.doris.common.util.Util; import org.apache.doris.datasource.jdbc.util.JdbcFieldSchema; +import com.google.common.collect.Lists; + +import java.sql.Connection; +import java.sql.DatabaseMetaData; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Types; +import java.util.Arrays; +import java.util.List; +import java.util.Optional; + public class JdbcPostgreSQLClient extends JdbcClient { + private static final String[] supportedInnerType = new String[] { + "int2", "int4", "int8", "smallserial", "serial", + "bigserial", "float4", "float8", "timestamp", "timestamptz", + "date", "bool", "bpchar", "varchar", "text" + }; + protected JdbcPostgreSQLClient(JdbcClientConfig jdbcClientConfig) { super(jdbcClientConfig); } + @Override + public List getJdbcColumnsInfo(String localDbName, String localTableName) { + Connection conn = null; + ResultSet rs = null; + List tableSchema = Lists.newArrayList(); + String remoteDbName = getRemoteDatabaseName(localDbName); + String remoteTableName = getRemoteTableName(localDbName, localTableName); + try { + conn = getConnection(); + DatabaseMetaData databaseMetaData = conn.getMetaData(); + String catalogName = getCatalogName(conn); + rs = getRemoteColumns(databaseMetaData, catalogName, remoteDbName, remoteTableName); + while (rs.next()) { + int dataType = rs.getInt("DATA_TYPE"); + int arrayDimensions = 0; + if (dataType == Types.ARRAY) { + String columnName = rs.getString("COLUMN_NAME"); + try (PreparedStatement pstmt = conn.prepareStatement( + String.format("SELECT array_ndims(%s) FROM %s.%s LIMIT 1", + columnName, remoteDbName, remoteTableName))) { + try (ResultSet arrayRs = pstmt.executeQuery()) { + if (arrayRs.next()) { + arrayDimensions = arrayRs.getInt(1); + } + } + } + } + tableSchema.add(new JdbcFieldSchema(rs, arrayDimensions)); + } + } catch (SQLException e) { + throw new JdbcClientException("failed to get jdbc columns info for remote table `%s.%s`: %s", + remoteDbName, remoteTableName, Util.getRootCauseMessage(e)); + } finally { + close(rs, conn); + } + return tableSchema; + } + @Override protected String[] getTableTypes() { return new String[] {"TABLE", "PARTITIONED TABLE", "VIEW", "MATERIALIZED VIEW", "FOREIGN TABLE"}; @@ -99,8 +157,38 @@ protected Type jdbcTypeToDoris(JdbcFieldSchema fieldSchema) { case "json": case "jsonb": return ScalarType.createStringType(); - default: - return Type.UNSUPPORTED; + default: { + if (fieldSchema.getDataType() == Types.ARRAY && pgType.startsWith("_")) { + return convertArrayType(fieldSchema); + } else { + return Type.UNSUPPORTED; + } + } + } + } + + private Type convertArrayType(JdbcFieldSchema fieldSchema) { + int arrayDimensions = fieldSchema.getArrayDimensions().orElse(0); + if (arrayDimensions == 0) { + return Type.UNSUPPORTED; + } + + String innerType = fieldSchema.getDataTypeName().orElse("unknown").substring(1); + + boolean isSupported = Arrays.asList(supportedInnerType).contains(innerType); + if (!isSupported) { + return Type.UNSUPPORTED; + } + if (innerType.equals("bpchar")) { + innerType = "text"; + } + JdbcFieldSchema innerFieldSchema = new JdbcFieldSchema(fieldSchema); + innerFieldSchema.setDataTypeName(Optional.of(innerType)); + Type arrayInnerType = jdbcTypeToDoris(innerFieldSchema); + Type arrayType = ArrayType.create(arrayInnerType, true); + for (int i = 1; i < arrayDimensions; i++) { + arrayType = ArrayType.create(arrayType, true); } + return arrayType; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/util/JdbcFieldSchema.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/util/JdbcFieldSchema.java index 7d643fac25c7e4..8cbd27ed684072 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/util/JdbcFieldSchema.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/util/JdbcFieldSchema.java @@ -37,6 +37,7 @@ public class JdbcFieldSchema { // For NUMERIC/DECIMAL, columnSize means precision. protected Optional columnSize; protected Optional decimalDigits; + protected Optional arrayDimensions; // Base number (usually 10 or 2) protected int numPrecRadix; // column description @@ -53,6 +54,7 @@ public JdbcFieldSchema(JdbcFieldSchema other) { this.dataTypeName = other.dataTypeName; this.columnSize = other.columnSize; this.decimalDigits = other.decimalDigits; + this.arrayDimensions = other.arrayDimensions; this.numPrecRadix = other.numPrecRadix; this.remarks = other.remarks; this.charOctetLength = other.charOctetLength; @@ -71,6 +73,19 @@ public JdbcFieldSchema(ResultSet rs) throws SQLException { this.charOctetLength = rs.getInt("CHAR_OCTET_LENGTH"); } + public JdbcFieldSchema(ResultSet rs, int arrayDimensions) throws SQLException { + this.columnName = rs.getString("COLUMN_NAME"); + this.dataType = getInteger(rs, "DATA_TYPE").orElseThrow(() -> new IllegalStateException("DATA_TYPE is null")); + this.dataTypeName = Optional.ofNullable(rs.getString("TYPE_NAME")); + this.columnSize = getInteger(rs, "COLUMN_SIZE"); + this.decimalDigits = getInteger(rs, "DECIMAL_DIGITS"); + this.numPrecRadix = rs.getInt("NUM_PREC_RADIX"); + this.isAllowNull = rs.getInt("NULLABLE") != DatabaseMetaData.columnNoNulls; + this.remarks = rs.getString("REMARKS"); + this.charOctetLength = rs.getInt("CHAR_OCTET_LENGTH"); + this.arrayDimensions = Optional.of(arrayDimensions); + } + public JdbcFieldSchema(ResultSet rs, Map dataTypeOverrides) throws SQLException { this.columnName = rs.getString("COLUMN_NAME"); this.dataType = getInteger(rs, "DATA_TYPE").orElseThrow(() -> new IllegalStateException("DATA_TYPE is null")); diff --git a/regression-test/data/external_table_p0/jdbc/test_pg_jdbc_catalog.out b/regression-test/data/external_table_p0/jdbc/test_pg_jdbc_catalog.out index 1950235745749e..e2993ffbaa7fcd 100644 --- a/regression-test/data/external_table_p0/jdbc/test_pg_jdbc_catalog.out +++ b/regression-test/data/external_table_p0/jdbc/test_pg_jdbc_catalog.out @@ -2234,6 +2234,17 @@ doris3 20 1 abc def 2022-10-11 1 2 3 2022-10-22T10:59:59 34.123 false 12.123456 10.16.10.14/32 10.16.10.14 ff:ff:ff:ff:ff:aa 1010101010 01010 1 {"id": 1} (1.0,1.0) {1.0,1.0,1.0} [(1.0,1.0),(2.0,2.0)] (2.0,2.0),(1.0,1.0) ((1.0,1.0),(2.0,2.0),(2.0,1.0)) ((1.0,1.0),(2.0,2.0),(2.0,1.0)) <(0.0,0.0),1.0> 2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N 2 \N \N \N \N \N \N \N \N +-- !select_all_arr_types -- +1 ["A", "B", "C"] ["Hello", "World"] ["Text1", "Text2"] ["2024-01-01", "2024-01-02"] [10, 20] [100, 200] [1000, 2000] ["2024-01-01 10:00:00.000000", "2024-01-02 12:00:00.000000"] ["2024-01-01 18:00:00.000000", "2024-01-02 20:00:00.000000"] [1, 0] [1.1, 2.2] [3.3, 4.4] +2 \N \N \N \N \N \N \N \N \N \N \N \N +3 ["A", "B", "C", null] ["Hello", "World", null] ["Text1", "Text2", null] ["2024-01-01", "2024-01-02", null] [10, 20, null] [100, 200, null] [1000, 2000, null] ["2024-01-01 10:00:00.000000", "2024-01-02 12:00:00.000000", null] ["2024-01-01 18:00:00.000000", "2024-01-02 20:00:00.000000", null] [1, 0, null] [1.1, 2.2, null] [3.3, 4.4, null] + +-- !select_all_arr2d_types -- +1 [["A", "B"], ["C", "D"]] [["Hello", "World"], ["Foo", "Bar"]] [["Text1", "Text2"], ["Text3", "Text4"]] [["2024-01-01", "2024-01-02"], ["2024-02-01", "2024-02-02"]] [[10, 20], [30, 40]] [[100, 200], [300, 400]] [[1000, 2000], [3000, 4000]] [["2024-01-01 10:00:00.000000", "2024-01-02 12:00:00.000000"], ["2024-03-01 10:00:00.000000", "2024-03-02 12:00:00.000000"]] [["2024-01-01 18:00:00.000000", "2024-01-02 20:00:00.000000"], ["2024-03-01 18:00:00.000000", "2024-03-02 20:00:00.000000"]] [[1, 0], [0, 1]] [[1.1, 2.2], [3.3, 4.4]] [[5.5, 6.6], [7.7, 8.8]] +2 \N \N \N \N \N \N \N \N \N \N \N \N +3 [["A", null], ["C", "D"]] [["Hello", null], ["Foo", "Bar"]] [["Text1", null], ["Text3", "Text4"]] [["2024-01-01", null], ["2024-02-01", "2024-02-02"]] [[10, null], [30, 40]] [[100, null], [300, 400]] [[1000, null], [3000, 4000]] [["2024-01-01 10:00:00.000000", null], ["2024-03-01 10:00:00.000000", "2024-03-02 12:00:00.000000"]] [["2024-01-01 18:00:00.000000", null], ["2024-03-01 18:00:00.000000", "2024-03-02 20:00:00.000000"]] [[1, null], [0, 1]] [[1.1, null], [3.3, 4.4]] [[5.5, null], [7.7, 8.8]] +4 [["A", "B"], [null, null]] [["Hello", "World"], [null, null]] [["Text1", "Text2"], [null, null]] [["2024-01-01", "2024-01-02"], [null, null]] [[10, 20], [null, null]] [[100, 200], [null, null]] [[1000, 2000], [null, null]] [["2024-01-01 10:00:00.000000", "2024-01-02 12:00:00.000000"], [null, null]] [["2024-01-01 18:00:00.000000", "2024-01-02 20:00:00.000000"], [null, null]] [[1, 0], [null, null]] [[1.1, 2.2], [null, null]] [[5.5, 6.6], [null, null]] + -- !ctas -- 1 abc def 2022-10-11 1 2 3 2022-10-22T10:59:59 34.123 false 12.123456 10.16.10.14/32 10.16.10.14 ff:ff:ff:ff:ff:aa 1010101010 01010 1 {"id": 1} (1.0,1.0) {1.0,1.0,1.0} [(1.0,1.0),(2.0,2.0)] (2.0,2.0),(1.0,1.0) ((1.0,1.0),(2.0,2.0),(2.0,1.0)) ((1.0,1.0),(2.0,2.0),(2.0,1.0)) <(0.0,0.0),1.0> 2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N 2 \N \N \N \N \N \N \N \N diff --git a/regression-test/suites/external_table_p0/jdbc/test_pg_jdbc_catalog.groovy b/regression-test/suites/external_table_p0/jdbc/test_pg_jdbc_catalog.groovy index 7f597493b9183d..c898f31ace65b3 100644 --- a/regression-test/suites/external_table_p0/jdbc/test_pg_jdbc_catalog.groovy +++ b/regression-test/suites/external_table_p0/jdbc/test_pg_jdbc_catalog.groovy @@ -146,6 +146,11 @@ suite("test_pg_jdbc_catalog", "p0,external,pg,external_docker,external_docker_pg // test select all types order_qt_select_all_types """select * from ${test_all_types}; """ + // test select all array types + order_qt_select_all_arr_types """select * from test_all_support_types_array order by 1;""" + + order_qt_select_all_arr2d_types """select * from test_all_support_types_array_2d order by 1;""" + // test test ctas sql """ drop table if exists internal.${internal_db_name}.${test_ctas} """ sql """ create table internal.${internal_db_name}.${test_ctas} From 8492f02559aa5b489fa4f927a0a12e950c1f1597 Mon Sep 17 00:00:00 2001 From: zy-kkk Date: Thu, 28 Nov 2024 23:49:29 +0800 Subject: [PATCH 071/399] [fix](jdbc catalog) set `enable_jdbc_cast_predicate_push_down` default true (#44548) ### What problem does this PR solve? Related PR: #42102 Problem Summary: We found that after disabling pushdown of predicates with implicit casts, some users experienced slower queries, so we temporarily changed this parameter back to the default behavior. --- .../src/main/java/org/apache/doris/qe/SessionVariable.java | 2 +- .../jdbc/test_clickhouse_jdbc_catalog.groovy | 6 ------ .../jdbc/test_jdbc_catalog_push_cast.groovy | 2 ++ 3 files changed, 3 insertions(+), 7 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java index f977c3ccb05f17..3c180be9d42802 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java @@ -726,7 +726,7 @@ public class SessionVariable implements Serializable, Writable { @VariableMgr.VarAttr(name = ENABLE_JDBC_CAST_PREDICATE_PUSH_DOWN, needForward = true, description = {"是否允许将带有 CAST 表达式的谓词下推到 JDBC 外部表。", "Whether to allow predicates with CAST expressions to be pushed down to JDBC external tables."}) - public boolean enableJdbcCastPredicatePushDown = false; + public boolean enableJdbcCastPredicatePushDown = true; @VariableMgr.VarAttr(name = ROUND_PRECISE_DECIMALV2_VALUE) public boolean roundPreciseDecimalV2Value = false; diff --git a/regression-test/suites/external_table_p0/jdbc/test_clickhouse_jdbc_catalog.groovy b/regression-test/suites/external_table_p0/jdbc/test_clickhouse_jdbc_catalog.groovy index fba50c3341e8a8..7875ac4244e372 100644 --- a/regression-test/suites/external_table_p0/jdbc/test_clickhouse_jdbc_catalog.groovy +++ b/regression-test/suites/external_table_p0/jdbc/test_clickhouse_jdbc_catalog.groovy @@ -83,17 +83,11 @@ suite("test_clickhouse_jdbc_catalog", "p0,external,clickhouse,external_docker,ex sql("select * from ts where from_unixtime(ts,'yyyyMMdd') >= '2022-01-01';") contains """QUERY: SELECT "id", "ts" FROM "doris_test"."ts" WHERE ((FROM_UNIXTIME("ts", '%Y%m%d') >= '2022-01-01'))""" } - explain { - sql("select * from ts where nvl(ts,null) >= '1';") - contains """QUERY: SELECT "id", "ts" FROM "doris_test"."ts""" - } order_qt_func_push2 """select * from ts where ts <= unix_timestamp(from_unixtime(ts,'yyyyMMdd'));""" - sql "set enable_jdbc_cast_predicate_push_down = true;" explain { sql("select * from ts where ts <= unix_timestamp(from_unixtime(ts,'yyyy-MM-dd'));") contains """QUERY: SELECT "id", "ts" FROM "doris_test"."ts" WHERE (("ts" <= toUnixTimestamp(FROM_UNIXTIME("ts", '%Y-%m-%d'))))""" } - sql "set enable_jdbc_cast_predicate_push_down = false;" order_qt_dt_with_tz """ select * from dt_with_tz order by id; """ diff --git a/regression-test/suites/external_table_p0/jdbc/test_jdbc_catalog_push_cast.groovy b/regression-test/suites/external_table_p0/jdbc/test_jdbc_catalog_push_cast.groovy index 6271a1619a5fac..3585f5fbb50941 100644 --- a/regression-test/suites/external_table_p0/jdbc/test_jdbc_catalog_push_cast.groovy +++ b/regression-test/suites/external_table_p0/jdbc/test_jdbc_catalog_push_cast.groovy @@ -34,6 +34,8 @@ suite("test_jdbc_catalog_push_cast", "p0,external,mysql,external_docker,external "driver_class" = "com.mysql.cj.jdbc.Driver" );""" + sql "set enable_jdbc_cast_predicate_push_down = false;" + sql "use jdbc_catalog_push_cast.doris_test" qt_sql """select * from test_cast where date(datetime_c) = '2022-01-01';""" From 2a5e992a3a67662d40c81933eb0568a7429eb026 Mon Sep 17 00:00:00 2001 From: abmdocrt Date: Fri, 29 Nov 2024 10:22:33 +0800 Subject: [PATCH 072/399] [Enhancement](cloud) Add concurrency for tablet stats processing (#43903) --- .../java/org/apache/doris/common/Config.java | 4 ++ .../doris/catalog/CloudTabletStatMgr.java | 54 ++++++++++++------- 2 files changed, 39 insertions(+), 19 deletions(-) diff --git a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java index 3caa137c858440..2ef391bb7ab390 100644 --- a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java +++ b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java @@ -3222,6 +3222,10 @@ public static int metaServiceRpcRetryTimes() { + "model is set to 300 times, which is approximately 5 minutes by default."}) public static int auto_start_wait_to_resume_times = 300; + @ConfField(description = {"Get tablet stat task的最大并发数。", + "Maximal concurrent num of get tablet stat job."}) + public static int max_get_tablet_stat_task_threads_num = 4; + // ATTN: DONOT add any config not related to cloud mode here // ATTN: DONOT add any config not related to cloud mode here // ATTN: DONOT add any config not related to cloud mode here diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/CloudTabletStatMgr.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/CloudTabletStatMgr.java index 0ba2686060e1f5..309b433572c942 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/CloudTabletStatMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/CloudTabletStatMgr.java @@ -36,6 +36,10 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; /* * CloudTabletStatMgr is for collecting tablet(replica) statistics from backends. @@ -47,6 +51,9 @@ public class CloudTabletStatMgr extends MasterDaemon { // <(dbId, tableId) -> OlapTable.Statistics> private volatile Map, OlapTable.Statistics> cloudTableStatsMap = new HashMap<>(); + private static final ExecutorService GET_TABLET_STATS_THREAD_POOL = Executors.newFixedThreadPool( + Config.max_get_tablet_stat_task_threads_num); + public CloudTabletStatMgr() { super("cloud tablet stat mgr", Config.tablet_stat_update_interval_second * 1000); } @@ -103,28 +110,37 @@ protected void runAfterCatalogReady() { reqList.add(builder.build()); } + List> futures = new ArrayList<>(); for (GetTabletStatsRequest req : reqList) { - GetTabletStatsResponse resp; - try { - resp = getTabletStats(req); - } catch (RpcException e) { - LOG.info("get tablet stats exception:", e); - continue; - } - - if (resp.getStatus().getCode() != MetaServiceCode.OK) { - continue; - } - - if (LOG.isDebugEnabled()) { - int i = 0; - for (TabletIndexPB idx : req.getTabletIdxList()) { - LOG.debug("db_id: {} table_id: {} index_id: {} tablet_id: {} size: {}", - idx.getDbId(), idx.getTableId(), idx.getIndexId(), idx.getTabletId(), - resp.getTabletStats(i++).getDataSize()); + futures.add(GET_TABLET_STATS_THREAD_POOL.submit(() -> { + GetTabletStatsResponse resp = GetTabletStatsResponse.newBuilder().build(); + try { + resp = getTabletStats(req); + } catch (RpcException e) { + LOG.warn("get tablet stats exception:", e); + } + if (resp.getStatus().getCode() != MetaServiceCode.OK) { + LOG.warn("get tablet stats return failed."); + } + if (LOG.isDebugEnabled()) { + int i = 0; + for (TabletIndexPB idx : req.getTabletIdxList()) { + LOG.debug("db_id: {} table_id: {} index_id: {} tablet_id: {} size: {}", + idx.getDbId(), idx.getTableId(), idx.getIndexId(), + idx.getTabletId(), resp.getTabletStats(i++).getDataSize()); + } } + updateTabletStat(resp); + return null; + })); + } + + try { + for (Future future : futures) { + future.get(); } - updateTabletStat(resp); + } catch (InterruptedException | ExecutionException e) { + LOG.error("Error waiting for get tablet stats tasks to complete", e); } LOG.info("finished to get tablet stat of all backends. cost: {} ms", From bc67fc9ef0e39507697c01cb74771f8a59b83acd Mon Sep 17 00:00:00 2001 From: zhengyu Date: Fri, 29 Nov 2024 10:40:01 +0800 Subject: [PATCH 073/399] [fix](cloud) serialize cache init to avoid unstable cache pick (#44429) The original paralleled cache init will causing unstable pick of cache base path because the choice depends on the order of init which could be different after each BE reboot. Thus, cause cache missing and duplicate cache block across multiple caches (disk space waste). This commit will serialize the init process of multiple cache and using fixed order, i.e. the order explicitly declared in be conf: file_cache_path. Signed-off-by: zhengyu --- be/src/runtime/exec_env_init.cpp | 26 +++++++------------------- 1 file changed, 7 insertions(+), 19 deletions(-) diff --git a/be/src/runtime/exec_env_init.cpp b/be/src/runtime/exec_env_init.cpp index 706fd7efd07d0f..b7f926cc3b4512 100644 --- a/be/src/runtime/exec_env_init.cpp +++ b/be/src/runtime/exec_env_init.cpp @@ -425,36 +425,24 @@ void ExecEnv::init_file_cache_factory(std::vector& cache_paths << ", reason=" << rest.msg(); exit(-1); } - std::vector file_cache_init_threads; - std::list cache_status; + doris::Status cache_status; for (auto& cache_path : cache_paths) { if (cache_path_set.find(cache_path.path) != cache_path_set.end()) { LOG(WARNING) << fmt::format("cache path {} is duplicate", cache_path.path); continue; } - file_cache_init_threads.emplace_back([&, status = &cache_status.emplace_back()]() { - *status = doris::io::FileCacheFactory::instance()->create_file_cache( - cache_path.path, cache_path.init_settings()); - }); - - cache_path_set.emplace(cache_path.path); - } - - for (std::thread& thread : file_cache_init_threads) { - if (thread.joinable()) { - thread.join(); - } - } - for (const auto& status : cache_status) { - if (!status.ok()) { + cache_status = doris::io::FileCacheFactory::instance()->create_file_cache( + cache_path.path, cache_path.init_settings()); + if (!cache_status.ok()) { if (!doris::config::ignore_broken_disk) { - LOG(FATAL) << "failed to init file cache, err: " << status; + LOG(FATAL) << "failed to init file cache, err: " << cache_status; exit(-1); } - LOG(WARNING) << "failed to init file cache, err: " << status; + LOG(WARNING) << "failed to init file cache, err: " << cache_status; } + cache_path_set.emplace(cache_path.path); } } From 797ac3b27e7aba275609a566d646fc56d925b958 Mon Sep 17 00:00:00 2001 From: Pxl Date: Fri, 29 Nov 2024 11:46:32 +0800 Subject: [PATCH 074/399] [Chore](runtime-filter) improve rf::debug_string and add some check about rf (#44697) improve rf::debug_string and add some check about rf --- be/src/exprs/runtime_filter.cpp | 14 +++++++------- be/src/exprs/runtime_filter.h | 9 ++++++--- be/src/exprs/runtime_filter_slots.h | 9 ++------- be/src/pipeline/exec/hashjoin_build_sink.cpp | 7 ++++++- 4 files changed, 21 insertions(+), 18 deletions(-) diff --git a/be/src/exprs/runtime_filter.cpp b/be/src/exprs/runtime_filter.cpp index efe96ae3395432..19789fbd743cbc 100644 --- a/be/src/exprs/runtime_filter.cpp +++ b/be/src/exprs/runtime_filter.cpp @@ -1535,11 +1535,13 @@ void IRuntimeFilter::update_runtime_filter_type_to_profile(uint64_t local_merge_ std::string IRuntimeFilter::debug_string() const { return fmt::format( - "RuntimeFilter: (id = {}, type = {}, is_broadcast: {}, " - "build_bf_cardinality: {}, ignored: {}, error_msg: {}", + "RuntimeFilter: (id = {}, type = {}, is_broadcast: {}, ignored: {}" + "build_bf_cardinality: {}, dependency: {}, synced_size: {}, has_local_target: {}, " + "has_remote_target: {},error_msg: [{}]", _filter_id, to_string(_runtime_filter_type), _is_broadcast_join, - _wrapper->get_build_bf_cardinality(), _wrapper->is_ignored(), - _wrapper->_context->err_msg); + _wrapper->_context->ignored, _wrapper->get_build_bf_cardinality(), + _dependency ? _dependency->debug_string() : "none", _synced_size, _has_local_target, + _has_remote_target, _wrapper->_context->err_msg); } Status IRuntimeFilter::merge_from(const RuntimePredicateWrapper* wrapper) { @@ -1592,9 +1594,7 @@ RuntimeFilterType IRuntimeFilter::get_real_type() { } bool IRuntimeFilter::need_sync_filter_size() { - return (type() == RuntimeFilterType::IN_OR_BLOOM_FILTER || - type() == RuntimeFilterType::BLOOM_FILTER) && - _wrapper->get_build_bf_cardinality() && !_is_broadcast_join; + return _wrapper->get_build_bf_cardinality() && !_is_broadcast_join; } void IRuntimeFilter::update_filter(std::shared_ptr wrapper, diff --git a/be/src/exprs/runtime_filter.h b/be/src/exprs/runtime_filter.h index 50ee52865be6d6..a1fdfbf07d239d 100644 --- a/be/src/exprs/runtime_filter.h +++ b/be/src/exprs/runtime_filter.h @@ -354,9 +354,12 @@ class IRuntimeFilter { void set_finish_dependency( const std::shared_ptr& dependency); - int64_t get_synced_size() const { return _synced_size; } - - bool isset_synced_size() const { return _synced_size != -1; } + int64_t get_synced_size() const { + if (_synced_size == -1) { + throw Status::InternalError("sync filter size meet error, filter: {}", debug_string()); + } + return _synced_size; + } protected: // serialize _wrapper to protobuf diff --git a/be/src/exprs/runtime_filter_slots.h b/be/src/exprs/runtime_filter_slots.h index 3c18735e4e82ce..7eb8c131c8a303 100644 --- a/be/src/exprs/runtime_filter_slots.h +++ b/be/src/exprs/runtime_filter_slots.h @@ -62,9 +62,8 @@ class VRuntimeFilterSlots { } // use synced size when this rf has global merged - static uint64_t get_real_size(IRuntimeFilter* runtime_filter, uint64_t hash_table_size) { - return runtime_filter->isset_synced_size() ? runtime_filter->get_synced_size() - : hash_table_size; + static uint64_t get_real_size(IRuntimeFilter* filter, uint64_t hash_table_size) { + return filter->need_sync_filter_size() ? filter->get_synced_size() : hash_table_size; } Status ignore_filters(RuntimeState* state) { @@ -119,10 +118,6 @@ class VRuntimeFilterSlots { } if (filter->get_real_type() == RuntimeFilterType::BLOOM_FILTER) { - if (filter->need_sync_filter_size() != filter->isset_synced_size()) { - return Status::InternalError("sync filter size meet error, filter: {}", - filter->debug_string()); - } RETURN_IF_ERROR(filter->init_bloom_filter( get_real_size(filter.get(), local_hash_table_size))); } diff --git a/be/src/pipeline/exec/hashjoin_build_sink.cpp b/be/src/pipeline/exec/hashjoin_build_sink.cpp index 5c1fa9daa0d837..a15dbb8dd03111 100644 --- a/be/src/pipeline/exec/hashjoin_build_sink.cpp +++ b/be/src/pipeline/exec/hashjoin_build_sink.cpp @@ -112,13 +112,14 @@ Status HashJoinBuildSinkLocalState::close(RuntimeState* state, Status exec_statu if (_closed) { return Status::OK(); } + auto p = _parent->cast(); Defer defer {[&]() { if (!_should_build_hash_table) { return; } // The build side hash key column maybe no need output, but we need to keep the column in block // because it is used to compare with probe side hash key column - auto p = _parent->cast(); + if (p._should_keep_hash_key_column && _build_col_ids.size() == 1) { p._should_keep_column_flags[_build_col_ids[0]] = true; } @@ -143,6 +144,10 @@ Status HashJoinBuildSinkLocalState::close(RuntimeState* state, Status exec_statu RETURN_IF_ERROR(_runtime_filter_slots->send_filter_size(state, 0, _finish_dependency)); RETURN_IF_ERROR(_runtime_filter_slots->ignore_all_filters()); } else { + if (p._shared_hashtable_controller && + !p._shared_hash_table_context->complete_build_stage) { + return Status::InternalError("close before sink meet eos"); + } auto* block = _shared_state->build_block.get(); uint64_t hash_table_size = block ? block->rows() : 0; { From c6cb3b646704acc6f52a7b837c24c555c85afc30 Mon Sep 17 00:00:00 2001 From: walter Date: Fri, 29 Nov 2024 11:48:55 +0800 Subject: [PATCH 075/399] [fix](clone) Fix wrong clone file path (#44746) The argument `clone_file` is a basename, not a full path --- be/src/olap/task/engine_clone_task.cpp | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/be/src/olap/task/engine_clone_task.cpp b/be/src/olap/task/engine_clone_task.cpp index 75cbcf68e956c1..bea1d3b1a91e89 100644 --- a/be/src/olap/task/engine_clone_task.cpp +++ b/be/src/olap/task/engine_clone_task.cpp @@ -82,8 +82,9 @@ namespace { /// if not equal, then return error /// return value: if binlog file not exist, then return to binlog file path Result check_dest_binlog_valid(const std::string& tablet_dir, + const std::string& clone_dir, const std::string& clone_file, bool* skip_link_file) { - std::string to; + std::string from, to; std::string new_clone_file = clone_file; if (clone_file.ends_with(".binlog")) { // change clone_file suffix from .binlog to .dat @@ -92,6 +93,7 @@ Result check_dest_binlog_valid(const std::string& tablet_dir, // change clone_file suffix from .binlog-index to .idx new_clone_file.replace(clone_file.size() - 13, 13, ".idx"); } + from = fmt::format("{}/{}", clone_dir, clone_file); to = fmt::format("{}/_binlog/{}", tablet_dir, new_clone_file); // check to to file exist @@ -106,10 +108,10 @@ Result check_dest_binlog_valid(const std::string& tablet_dir, } LOG(WARNING) << "binlog file already exist. " - << "tablet_dir=" << tablet_dir << ", clone_file=" << clone_file; + << "tablet_dir=" << tablet_dir << ", clone_file=" << from << ", to=" << to; std::string clone_file_md5sum; - status = io::global_local_filesystem()->md5sum(clone_file, &clone_file_md5sum); + status = io::global_local_filesystem()->md5sum(from, &clone_file_md5sum); if (!status.ok()) { return ResultError(std::move(status)); } @@ -697,7 +699,6 @@ Status EngineCloneTask::_finish_clone(Tablet* tablet, const std::string& clone_d continue; } - auto from = fmt::format("{}/{}", clone_dir, clone_file); std::string to; if (clone_file.ends_with(".binlog") || clone_file.ends_with(".binlog-index")) { if (!contain_binlog) { @@ -706,7 +707,8 @@ Status EngineCloneTask::_finish_clone(Tablet* tablet, const std::string& clone_d break; } - if (auto&& result = check_dest_binlog_valid(tablet_dir, clone_file, &skip_link_file); + if (auto&& result = + check_dest_binlog_valid(tablet_dir, clone_dir, clone_file, &skip_link_file); result) { to = std::move(result.value()); } else { @@ -718,6 +720,7 @@ Status EngineCloneTask::_finish_clone(Tablet* tablet, const std::string& clone_d } if (!skip_link_file) { + auto from = fmt::format("{}/{}", clone_dir, clone_file); status = io::global_local_filesystem()->link_file(from, to); if (!status.ok()) { return status; From 9daa3b77dc895d879e5ab62ff91b58d8473e6ffb Mon Sep 17 00:00:00 2001 From: seawinde Date: Fri, 29 Nov 2024 14:42:38 +0800 Subject: [PATCH 076/399] [test](mtmv) Add inject statistic when mv rewrite regression test to make sure rewrite result stable (#43785) ### What problem does this PR solve? The result of successful rewriting by the cbo optimizer depends on the statistics. The priority of the optimizer consumption statistics in descending order is 1. the injected statistics 2. the statistics reported by be 3. and the statistics analyzed actively. When the pipeline runs the case, the statistics reported by be may not be timely. Therefore, the outcome that leads to the cbo optimizer's successful selection of overwrites is not very certain, so the statistics are currently injected manually in the test cases --- ...AbstractMaterializedViewAggregateRule.java | 28 ++- .../data/mv_p0/ssb/q_1_1/q_1_1.out | 7 +- .../doris/regression/suite/Suite.groovy | 3 +- .../agg_have_dup_base.groovy | 2 + .../diffrent_serialize.groovy | 2 + .../agg_state/test_agg_state_max_by.groovy | 3 + .../mv_p0/case_ignore/case_ignore.groovy | 3 + .../contain_cast/mv_contains_cast.groovy | 2 + .../suites/mv_p0/count_star/count_star.groovy | 2 + .../suites/mv_p0/dis_26495/dis_26495.groovy | 2 + .../suites/mv_p0/k1ap2spa/k1ap2spa.groovy | 2 + .../suites/mv_p0/k1s2m3/k1s2m3.groovy | 1 + .../k1s2m3_auto_inc/k1s2m3_auto_inc.groovy | 1 + .../multi_agg_with_same_slot.groovy | 1 + .../multi_slot_k123p/multi_slot_k123p.groovy | 1 + .../multi_slot_k1a2p2ap3p.groovy | 1 + .../multi_slot_k1a2p2ap3ps.groovy | 1 + .../multi_slot_k1p2ap3p.groovy | 1 + .../multi_slot_k1p2ap3ps.groovy | 1 + .../multi_slot_multi_mv.groovy | 1 + .../mv_p0/mv_with_view/mv_with_view.groovy | 1 + .../mv_p0/null_insert/null_insert.groovy | 1 + .../routine_load_hll/routine_load_hll.groovy | 1 + .../ssb/multiple_ssb/multiple_ssb.groovy | 1 + .../multiple_ssb_between.groovy | 2 + .../suites/mv_p0/ssb/q_1_1/q_1_1.groovy | 20 +- .../suites/mv_p0/ssb/q_2_1/q_2_1.groovy | 2 + .../suites/mv_p0/ssb/q_3_1/q_3_1.groovy | 2 + .../suites/mv_p0/ssb/q_4_1/q_4_1.groovy | 2 + .../suites/mv_p0/ssb/q_4_1_r1/q_4_1_r1.groovy | 1 + .../suites/mv_p0/sum_count/sum_count.groovy | 1 + .../sum_divede_count/sum_devide_count.groovy | 2 + .../suites/mv_p0/test_28741/test_28741.groovy | 2 +- .../test_approx_count_distinct.groovy | 1 + .../suites/mv_p0/test_base/test_base.groovy | 1 + .../mv_p0/test_casewhen/test_casewhen.groovy | 1 + .../test_create_mv/test_create_mv.groovy | 2 + .../test_create_mv_complex_type.groovy | 2 + .../mv_p0/test_doc_e4/test_doc_e4.groovy | 1 + .../test_dup_group_by_mv_abs.groovy | 1 + .../test_dup_group_by_mv_plus.groovy | 1 + .../test_dup_mv_abs/test_dup_mv_abs.groovy | 1 + .../test_dup_mv_bin/test_dup_mv_bin.groovy | 1 + .../test_dup_mv_bitmap_hash.groovy | 3 + .../test_dup_mv_plus/test_dup_mv_plus.groovy | 2 +- .../test_dup_mv_repeat.groovy | 1 + .../test_dup_mv_year/test_dup_mv_year.groovy | 1 + .../test_duplicate_mv.groovy | 2 + .../suites/mv_p0/test_mv_dp/test_mv_dp.groovy | 2 + .../mv_p0/test_mv_mor/test_mv_mor.groovy | 2 + .../mv_p0/test_mv_mow/test_mv_mow.groovy | 1 + .../suites/mv_p0/test_ndv/test_ndv.groovy | 1 + .../suites/mv_p0/test_nvl/test_nvl.groovy | 1 + .../suites/mv_p0/test_o2/test_o2.groovy | 1 + .../mv_p0/test_substr/test_substr.groovy | 1 + .../mv_p0/test_tbl_name/test_tbl_name.groovy | 1 + .../test_upper_alias/test_upper_alias.groovy | 1 + .../test_user_activity.groovy | 1 + .../suites/mv_p0/unique/unique.groovy | 1 + .../testAggQueryOnAggMV1.groovy | 1 + .../testAggQueryOnAggMV10.groovy | 1 + .../testAggQueryOnAggMV11.groovy | 1 + .../testAggQueryOnAggMV2.groovy | 1 + .../testAggQueryOnAggMV3.groovy | 2 +- .../testAggQuqeryOnAggMV5.groovy | 1 + .../testAggQuqeryOnAggMV6.groovy | 1 + .../testAggQuqeryOnAggMV7.groovy | 2 + ...stAggTableCountDistinctInBitmapType.groovy | 1 + ...testAggregateMVCalcAggFunctionQuery.groovy | 1 + .../testBitmapUnionInQuery.groovy | 2 +- .../testCountDistinctToBitmap.groovy | 4 + .../testIncorrectMVRewriteInSubquery.groovy | 1 + .../testIncorrectRewriteCountDistinct.groovy | 1 + .../testJoinOnLeftProjectToJoin.groovy | 3 + .../mv_p0/ut/testNDVToHll/testNDVToHll.groovy | 1 + .../testOrderByQueryOnProjectView.groovy | 1 + .../testProjectionMV1.groovy | 1 + .../testProjectionMV2.groovy | 1 + .../testProjectionMV3.groovy | 1 + .../testProjectionMV4.groovy | 1 + .../ut/testQueryOnStar/testQueryOnStar.groovy | 1 + .../testSelectMVWithTableAlias.groovy | 2 + .../testSingleMVMultiUsage.groovy | 1 + .../mv_p0/ut/testSubQuery/testSubQuery.groovy | 2 +- .../testUnionDistinct.groovy | 2 + .../mv/agg_on_none_agg/agg_on_none_agg.groovy | 6 + .../agg_optimize_when_uniform.groovy | 4 + .../mv/agg_variety/agg_variety.groovy | 6 + .../aggregate_with_roll_up.groovy | 3 + .../aggregate_without_roll_up.groovy | 6 + .../mv/availability/grace_period.groovy | 2 + .../materialized_view_switch.groovy | 5 + .../range_date_datetrunc_part_up.groovy | 1 + .../range_date_part_up.groovy | 2 + .../range_date_part_up_rewrite.groovy | 3 + .../range_datetime_part_up_rewrite.groovy | 4 + .../mv/dimension/dimension_1.groovy | 3 + .../mv/dimension/dimension_2_3.groovy | 3 + .../mv/dimension/dimension_2_4.groovy | 4 + .../mv/dimension/dimension_2_5.groovy | 4 + .../mv/dimension/dimension_2_6.groovy | 6 + .../mv/dimension/dimension_2_full_join.groovy | 2 + .../dimension/dimension_2_inner_join.groovy | 3 + .../dimension_2_left_anti_join.groovy | 3 + .../mv/dimension/dimension_2_left_join.groovy | 3 + .../dimension_2_left_semi_join.groovy | 3 + .../dimension_2_right_anti_join.groovy | 3 + .../dimension/dimension_2_right_join.groovy | 3 + .../dimension_2_right_semi_join.groovy | 3 + .../mv/dimension/dimension_self_conn.groovy | 3 + .../dimension_2_join_agg.groovy | 15 +- .../dimension_join_agg_negative.groovy | 3 + .../filter_equal_or_notequal.groovy | 3 + .../full_join_filter.groovy | 3 + .../inner_join_filter.groovy | 3 + .../left_anti_join_filter.groovy | 3 + .../left_join_filter.groovy | 3 + .../left_semi_join_filter.groovy | 3 + .../right_anti_join_filter.groovy | 3 + .../right_join_filter.groovy | 3 + .../right_semi_join_filter.groovy | 3 + .../mv/direct_query/direct_query.groovy | 3 + .../mv/grouping_sets/grouping_sets.groovy | 3 + .../join/dphyp_inner/inner_join_dphyp.groovy | 4 + .../join/dphyp_outer/outer_join_dphyp.groovy | 4 + .../mv/join/inner/inner_join.groovy | 4 + .../mv/join/left_outer/outer_join.groovy | 11 + .../inner_join_infer_and_derive.groovy | 3 + .../left_join_infer_and_derive.groovy | 2 + .../right_join_infer_and_derive.groovy | 3 + .../mv/negative/negative_test.groovy | 3 + .../mv/nested/nested_materialized_view.groovy | 4 + .../mv/nested_mtmv/nested_mtmv.groovy | 202 +++++++----------- .../nested_mtmv_rewrite_switch.groovy | 3 + .../anti/other_join_conjuncts_anti.groovy | 4 + .../inner/other_join_conjuncts_inner.groovy | 4 + .../outer/other_join_conjuncts_outer.groovy | 4 + .../semi/other_join_conjuncts_semi.groovy | 4 + .../mv/partition_mv_rewrite.groovy | 4 + .../mv/same_name/sync_async_same_name.groovy | 2 + .../mv/scan/scan_table.groovy | 4 + .../single_table_without_aggregate.groovy | 1 + .../union_all_compensate.groovy | 3 + .../partition_curd_union_rewrite.groovy | 3 + .../partition_curd_union_rewrite_hive.groovy | 3 + .../usercase_union_rewrite.groovy | 3 + .../unsafe_equals/null_un_safe_equals.groovy | 2 + .../mv/variant/variant_mv.groovy | 3 + .../with_auth/with_select_table_auth.groovy | 3 + .../query_with_sql_limit.groovy | 4 + .../mv/newMv/aggHaveDupBase.groovy | 1 + .../mv/newMv/case_ignore.groovy | 2 + .../mv/newMv/dup_gb_mv_abs.groovy | 1 + .../mv/newMv/dup_gb_mv_plus.groovy | 1 + .../mv/newMv/dup_mv_abs.groovy | 2 +- .../mv/newMv/dup_mv_bin.groovy | 1 + .../mv/newMv/dup_mv_bm_hash.groovy | 1 + .../mv/newMv/dup_mv_plus.groovy | 2 +- .../mv/newMv/dup_mv_year.groovy | 1 + .../mv/newMv/multi_slot1.groovy | 1 + .../mv/newMv/multi_slot2.groovy | 1 + .../mv/newMv/multi_slot3.groovy | 1 + .../mv/newMv/multi_slot4.groovy | 1 + .../mv/newMv/multi_slot5.groovy | 1 + .../mv/newMv/multi_slot6.groovy | 1 + .../mv/newMv/mv_with_view.groovy | 1 + .../mv/newMv/single_slot.groovy | 2 +- .../mv/newMv/sum_devide_count.groovy | 2 +- .../mv/newMv/unique_mv.groovy | 1 + .../mv/ut/MVMultiUsage.groovy | 3 + .../nereids_syntax_p0/mv/ut/MVWithAs.groovy | 3 + .../mv/ut/aggCDInBitmap.groovy | 4 +- .../mv/ut/aggMVCalcAggFun.groovy | 2 +- .../mv/ut/aggOnAggMV1.groovy | 1 + .../mv/ut/aggOnAggMV10.groovy | 1 + .../mv/ut/aggOnAggMV11.groovy | 3 + .../mv/ut/aggOnAggMV2.groovy | 1 + .../mv/ut/aggOnAggMV3.groovy | 1 + .../mv/ut/aggOnAggMV5.groovy | 2 + .../mv/ut/aggOnAggMV6.groovy | 1 + .../mv/ut/aggOnAggMV7.groovy | 1 + .../mv/ut/bitmapUnionIn.groovy | 3 + .../mv/ut/distinctQuery.groovy | 2 + .../mv/ut/incMVReInSub.groovy | 3 + .../mv/ut/incRewriteCD.groovy | 3 + .../mv/ut/joinOnCalcToJoin.groovy | 5 + .../mv/ut/joinOnLeftPToJoin.groovy | 5 + .../mv/ut/onlyGroupBy.groovy | 3 + .../mv/ut/orderByOnPView.groovy | 2 + .../nereids_syntax_p0/mv/ut/projectMV1.groovy | 3 + .../nereids_syntax_p0/mv/ut/projectMV2.groovy | 3 + .../nereids_syntax_p0/mv/ut/projectMV3.groovy | 3 + .../nereids_syntax_p0/mv/ut/projectMV4.groovy | 3 + .../nereids_syntax_p0/mv/ut/subQuery.groovy | 2 +- .../nereids_syntax_p0/mv/ut/unionDis.groovy | 2 + .../nereids_syntax_p0/rollup/agg.groovy | 1 + .../nereids_syntax_p0/rollup/agg_date.groovy | 1 + .../nereids_syntax_p0/rollup/bitmap.groovy | 1 + .../nereids_syntax_p0/rollup/date.groovy | 1 + .../nereids_syntax_p0/rollup/hll/hll.groovy | 1 + .../hll_with_light_sc.groovy | 1 + 201 files changed, 532 insertions(+), 170 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewAggregateRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewAggregateRule.java index 743003dca6bb81..5a854f085b2f77 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewAggregateRule.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewAggregateRule.java @@ -487,27 +487,37 @@ private static boolean isGroupByEqualsByFunctionDependency( if (!viewShuttledExprQueryBasedSet.containsAll(queryGroupShuttledExpression)) { return false; } - Set viewShouldUniformExpressionSet = new HashSet<>(); + Set viewShouldRemovedExpressionSet = new HashSet<>(); + Set viewScanShouldReservedExpressionSet = new HashSet<>(); // calc the group by expr which is needed to roll up and should be uniform - for (Map.Entry expressionEntry : + for (Map.Entry expressionMappingEntry : viewShuttledExprQueryBasedToViewGroupByExprMap.entrySet()) { - if (queryGroupShuttledExpression.contains(expressionEntry.getKey())) { - // the group expr which query has, do not require uniform - continue; + if (queryGroupShuttledExpression.contains(expressionMappingEntry.getKey())) { + // the group expr which query has, do not require eliminate + viewScanShouldReservedExpressionSet.add( + viewShuttledExprToScanExprMapping.get(expressionMappingEntry.getValue())); + } else { + // the view expression which is more than query's expression, should try to eliminate + viewShouldRemovedExpressionSet.add(expressionMappingEntry.getValue()); } - viewShouldUniformExpressionSet.add(expressionEntry.getValue()); } DataTrait dataTrait = tempRewrittenPlan.computeDataTrait(); - for (Expression shouldUniformExpr : viewShouldUniformExpressionSet) { - Expression viewScanExpression = viewShuttledExprToScanExprMapping.get(shouldUniformExpr); + for (Expression viewShouldRemovedExpr : viewShouldRemovedExpressionSet) { + Expression viewScanExpression = viewShuttledExprToScanExprMapping.get(viewShouldRemovedExpr); if (viewScanExpression == null) { return false; } if (!(viewScanExpression instanceof Slot)) { return false; } - if (!dataTrait.isUniform((Slot) viewScanExpression)) { + if (!dataTrait.isUniform((Slot) viewScanExpression) + && Sets.intersection(dataTrait.calEqualSet((Slot) viewScanExpression), + viewScanShouldReservedExpressionSet).isEmpty()) { + // Such as query is l_orderkey#0, l_linenumber#1, o_custkey#17, l_partkey#2 + // view is ps_partkey#25, o_orderkey#16, l_orderkey#0, l_linenumber#1, o_custkey#17, l_partkey#2 + // If want to check the group by equality, ps_partkey#25, o_orderkey#16 should be uniform + // or should be equal any of [ l_orderkey#0, l_linenumber#1, o_custkey#17, l_partkey#2] return false; } } diff --git a/regression-test/data/mv_p0/ssb/q_1_1/q_1_1.out b/regression-test/data/mv_p0/ssb/q_1_1/q_1_1.out index 1da845620e8dd7..c00bbc9377409e 100644 --- a/regression-test/data/mv_p0/ssb/q_1_1/q_1_1.out +++ b/regression-test/data/mv_p0/ssb/q_1_1/q_1_1.out @@ -6,10 +6,13 @@ 19930101 1 1 1 1 1 1 1 1 1 1 100 1 1 1 2023-06-09 shipmode name address city nation AMERICA phone mktsegment name address city nation AMERICA phone name MFGR#1 category brand color type 4 container 19930101 1 1 1 1 1 1 1 1 1 1 100 1 1 1 2023-06-09 shipmode name address city nation AMERICA phone mktsegment name address city nation AMERICA phone name MFGR#1 category brand color type 4 container 19930101 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2023-06-09 shipmode name address city nation region phone mktsegment name address city nation region phone name mfgr category brand color type 4 container +19930101 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2023-06-09 shipmode name address city nation region phone mktsegment name address city nation region phone name mfgr category brand color type 4 container +19930101 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2023-06-09 shipmode name address city nation region phone mktsegment name address city nation region phone name mfgr category brand color type 4 container +19930101 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2023-06-09 shipmode name address city nation region phone mktsegment name address city nation region phone name mfgr category brand color type 4 container -- !select_mv -- -4 +16 -- !select -- -4 +16 diff --git a/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy b/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy index 2a5ba5acc414b7..62e6032578fd6a 100644 --- a/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy +++ b/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy @@ -1983,9 +1983,8 @@ class Suite implements GroovyInterceptable { check { result -> boolean success = true; for (String mv_name : mv_names) { - success = success && result.contains("${mv_name} chose") + Assert.assertEquals(true, result.contains("${mv_name} chose")) } - Assert.assertEquals(true, success) } } } diff --git a/regression-test/suites/mv_p0/agg_have_dup_base/agg_have_dup_base.groovy b/regression-test/suites/mv_p0/agg_have_dup_base/agg_have_dup_base.groovy index 8f3ec4171e32b0..ece526b1e24e7a 100644 --- a/regression-test/suites/mv_p0/agg_have_dup_base/agg_have_dup_base.groovy +++ b/regression-test/suites/mv_p0/agg_have_dup_base/agg_have_dup_base.groovy @@ -60,6 +60,8 @@ suite ("agg_have_dup_base") { qt_select_mv "select unix_timestamp(k1) tmp,sum(k2) from d_table group by tmp order by tmp;" sql """set enable_stats=true;""" + sql """alter table d_table modify column k4 set stats ('row_count'='5');""" + mv_rewrite_success("select k1,sum(k2),max(k2) from d_table group by k1;", "k12s3m") mv_rewrite_success("select k1,sum(k2) from d_table group by k1;", "k12s3m") diff --git a/regression-test/suites/mv_p0/agg_state/diffrent_serialize/diffrent_serialize.groovy b/regression-test/suites/mv_p0/agg_state/diffrent_serialize/diffrent_serialize.groovy index 78d26f476fa7a8..005646b89e47b2 100644 --- a/regression-test/suites/mv_p0/agg_state/diffrent_serialize/diffrent_serialize.groovy +++ b/regression-test/suites/mv_p0/agg_state/diffrent_serialize/diffrent_serialize.groovy @@ -37,6 +37,8 @@ suite ("diffrent_serialize") { sql "insert into d_table select 2,2,2,'b';" sql "insert into d_table select 3,3,null,'c';" + sql """alter table d_table modify column k4 set stats ('row_count'='7');""" + createMV("create materialized view mv1_1 as select k1,bitmap_intersect(to_bitmap(k2)) from d_table group by k1;") createMV("create materialized view mv1 as select k1,bitmap_agg(k2) from d_table group by k1;") createMV("create materialized view mv1_2 as select k1, multi_distinct_group_concat(k4) from d_table group by k1 order by k1;") diff --git a/regression-test/suites/mv_p0/agg_state/test_agg_state_max_by.groovy b/regression-test/suites/mv_p0/agg_state/test_agg_state_max_by.groovy index 1e34c70ccec900..29190396230dce 100644 --- a/regression-test/suites/mv_p0/agg_state/test_agg_state_max_by.groovy +++ b/regression-test/suites/mv_p0/agg_state/test_agg_state_max_by.groovy @@ -69,6 +69,7 @@ suite ("test_agg_state_max_by") { qt_select_star "select * from d_table order by 1,2;" mv_rewrite_success("select k1,max_by(k2,k3) from d_table group by k1 order by 1,2;", "k1mb") sql """set enable_stats=true;""" + sql """alter table d_table modify column k4 set stats ('row_count'='8');""" mv_rewrite_success("select k1,max_by(k2,k3) from d_table group by k1 order by 1,2;", "k1mb") qt_select_mv "select k1,max_by(k2,k3) from d_table group by k1 order by 1,2;" @@ -101,6 +102,7 @@ suite ("test_agg_state_max_by") { qt_select_star "select * from d_table order by 1,2;" sql """set enable_stats=true;""" + sql """alter table d_table modify column k4 set stats ('row_count'='8');""" sql "analyze table d_table with sync;" sql """set enable_stats=false;""" @@ -114,6 +116,7 @@ suite ("test_agg_state_max_by") { qt_select_mv "select k1,max_by(k2,abs(k3)) from d_table group by k1 order by 1,2;" sql """set enable_stats=true;""" + sql """alter table d_table modify column k4 set stats ('row_count'='8');""" mv_rewrite_success("select k1,max_by(k2+k3,abs(k3)) from d_table group by k1 order by 1,2;", "k1mbcp1") mv_rewrite_success("select k1,max_by(k2+k3,k3) from d_table group by k1 order by 1,2;", "k1mbcp2") mv_rewrite_success("select k1,max_by(k2,abs(k3)) from d_table group by k1 order by 1,2;", "k1mbcp3") diff --git a/regression-test/suites/mv_p0/case_ignore/case_ignore.groovy b/regression-test/suites/mv_p0/case_ignore/case_ignore.groovy index 1134d930185b9a..271eb0cb5ed190 100644 --- a/regression-test/suites/mv_p0/case_ignore/case_ignore.groovy +++ b/regression-test/suites/mv_p0/case_ignore/case_ignore.groovy @@ -37,6 +37,8 @@ suite ("case_ignore") { sql "insert into d_table select 2,2,2,'b';" sql "insert into d_table select 3,-3,null,'c';" + sql """alter table d_table modify column k4 set stats ('row_count'='4');""" + createMV ("create materialized view k12a as select K1,abs(K2) from d_table;") sql "insert into d_table select -4,-4,-4,'d';" @@ -53,6 +55,7 @@ suite ("case_ignore") { qt_select_mv "select K1,abs(K2) from d_table order by K1;" sql """set enable_stats=true;""" + sql """alter table d_table modify column k4 set stats ('row_count'='8');""" mv_rewrite_success("select k1,abs(k2) from d_table order by k1;", "k12a") mv_rewrite_success("select K1,abs(K2) from d_table order by K1;", "k12a") diff --git a/regression-test/suites/mv_p0/contain_cast/mv_contains_cast.groovy b/regression-test/suites/mv_p0/contain_cast/mv_contains_cast.groovy index e45798ba7984cd..af81035c3f9a6f 100644 --- a/regression-test/suites/mv_p0/contain_cast/mv_contains_cast.groovy +++ b/regression-test/suites/mv_p0/contain_cast/mv_contains_cast.groovy @@ -91,6 +91,8 @@ suite("mv_contains_cast") { order_qt_query_before "${query_sql}" + sql """alter table test modify column event_type set stats ('row_count'='10');""" + createMV(""" CREATE MATERIALIZED VIEW sync_mv AS diff --git a/regression-test/suites/mv_p0/count_star/count_star.groovy b/regression-test/suites/mv_p0/count_star/count_star.groovy index 52edda58276ef6..4d951d76fa98a9 100644 --- a/regression-test/suites/mv_p0/count_star/count_star.groovy +++ b/regression-test/suites/mv_p0/count_star/count_star.groovy @@ -64,6 +64,8 @@ suite ("count_star") { qt_select_mv "select count(*) from d_table where k3=1;" sql """set enable_stats=true;""" + + sql """alter table d_table modify column k4 set stats ('row_count'='8');""" mv_rewrite_success("select k1,k4,count(*) from d_table group by k1,k4;", "kstar") mv_rewrite_success("select k1,k4,count(*) from d_table where k1=1 group by k1,k4;", "kstar") mv_rewrite_fail("select k1,k4,count(*) from d_table where k3=1 group by k1,k4;", "kstar") diff --git a/regression-test/suites/mv_p0/dis_26495/dis_26495.groovy b/regression-test/suites/mv_p0/dis_26495/dis_26495.groovy index e30af70b0041b3..14ebb67ddc8168 100644 --- a/regression-test/suites/mv_p0/dis_26495/dis_26495.groovy +++ b/regression-test/suites/mv_p0/dis_26495/dis_26495.groovy @@ -28,6 +28,8 @@ suite ("dis_26495") { sql """insert into doris_test values (1,2,max_by_state(1,2));""" + sql """alter table doris_test modify column agg_st_1 set stats ('row_count'='1');""" + streamLoad { table "doris_test" set 'column_separator', ',' diff --git a/regression-test/suites/mv_p0/k1ap2spa/k1ap2spa.groovy b/regression-test/suites/mv_p0/k1ap2spa/k1ap2spa.groovy index cfe2a5307d7ed4..b4c2921669663a 100644 --- a/regression-test/suites/mv_p0/k1ap2spa/k1ap2spa.groovy +++ b/regression-test/suites/mv_p0/k1ap2spa/k1ap2spa.groovy @@ -52,6 +52,8 @@ suite ("k1ap2spa") { sql """set enable_stats=true;""" + sql """alter table d_table modify column k1 set stats ('row_count'='5');""" + mv_rewrite_success("select abs(k1)+1 t,sum(abs(k2+1)) from d_table group by t order by t;", "k1ap2spa") } diff --git a/regression-test/suites/mv_p0/k1s2m3/k1s2m3.groovy b/regression-test/suites/mv_p0/k1s2m3/k1s2m3.groovy index ecae837b4fe33c..25319e78807d7e 100644 --- a/regression-test/suites/mv_p0/k1s2m3/k1s2m3.groovy +++ b/regression-test/suites/mv_p0/k1s2m3/k1s2m3.groovy @@ -37,6 +37,7 @@ suite ("k1s2m3") { sql "insert into d_table select 2,2,2,'b';" sql "insert into d_table select 3,-3,null,'c';" + sql """alter table d_table modify column k1 set stats ('row_count'='6');""" createMV("create materialized view k1s2m3 as select k1,sum(k2*k3) from d_table group by k1;") sql "insert into d_table select -4,-4,-4,'d';" diff --git a/regression-test/suites/mv_p0/k1s2m3_auto_inc/k1s2m3_auto_inc.groovy b/regression-test/suites/mv_p0/k1s2m3_auto_inc/k1s2m3_auto_inc.groovy index 367cd9863c1e3e..17583741e68b91 100644 --- a/regression-test/suites/mv_p0/k1s2m3_auto_inc/k1s2m3_auto_inc.groovy +++ b/regression-test/suites/mv_p0/k1s2m3_auto_inc/k1s2m3_auto_inc.groovy @@ -57,5 +57,6 @@ suite ("k1s2m3_auto_inc") { qt_select_mv "select k3,sum(abs(k2+1)) from d_table group by k3 order by 1;" sql """set enable_stats=true;""" + sql """alter table d_table modify column k1 set stats ('row_count'='2');""" mv_rewrite_success("select k3,sum(abs(k2+1)) from d_table group by k3 order by 1;", "k3ap2spa") } diff --git a/regression-test/suites/mv_p0/multi_agg_with_same_slot/multi_agg_with_same_slot.groovy b/regression-test/suites/mv_p0/multi_agg_with_same_slot/multi_agg_with_same_slot.groovy index a1d254b333e211..538d050fec8a9d 100644 --- a/regression-test/suites/mv_p0/multi_agg_with_same_slot/multi_agg_with_same_slot.groovy +++ b/regression-test/suites/mv_p0/multi_agg_with_same_slot/multi_agg_with_same_slot.groovy @@ -64,6 +64,7 @@ suite ("multi_agg_with_same_slot") { sql """set enable_stats=true;""" + sql """alter table d_table modify column k1 set stats ('row_count'='5');""" mv_rewrite_success("select k1,k2,avg(k3),max(k3) from d_table group by k1,k2 order by 1,2;", "kmv") mv_rewrite_success("select k1,k2,avg(k3)+max(k3) from d_table group by k1,k2 order by 1,2;", "kmv") mv_rewrite_success("select k1,k2,avg(k3)+max(k3) from d_table group by grouping sets((k1),(k1,k2),()) order by 1,2;", "kmv") diff --git a/regression-test/suites/mv_p0/multi_slot_k123p/multi_slot_k123p.groovy b/regression-test/suites/mv_p0/multi_slot_k123p/multi_slot_k123p.groovy index a4a0be3ca8df65..154939a6bb5ef3 100644 --- a/regression-test/suites/mv_p0/multi_slot_k123p/multi_slot_k123p.groovy +++ b/regression-test/suites/mv_p0/multi_slot_k123p/multi_slot_k123p.groovy @@ -63,5 +63,6 @@ suite ("multi_slot_k123p") { qt_select_mv "select k1,version() from d_table order by k1;" sql """set enable_stats=true;""" + sql """alter table d_table modify column k1 set stats ('row_count'='5');""" mv_rewrite_success("select k1,k2+k3 from d_table order by k1;", "k123p") } diff --git a/regression-test/suites/mv_p0/multi_slot_k1a2p2ap3p/multi_slot_k1a2p2ap3p.groovy b/regression-test/suites/mv_p0/multi_slot_k1a2p2ap3p/multi_slot_k1a2p2ap3p.groovy index e64adbb06b2988..fc722c5a07fbe7 100644 --- a/regression-test/suites/mv_p0/multi_slot_k1a2p2ap3p/multi_slot_k1a2p2ap3p.groovy +++ b/regression-test/suites/mv_p0/multi_slot_k1a2p2ap3p/multi_slot_k1a2p2ap3p.groovy @@ -51,6 +51,7 @@ suite ("multi_slot_k1a2p2ap3p") { qt_select_mv "select abs(k1)+k2+1,abs(k2+2)+k3+3 from d_table order by abs(k1)+k2+1,abs(k2+2)+k3+3;" sql """set enable_stats=true;""" + sql """alter table d_table modify column k1 set stats ('row_count'='7');""" mv_rewrite_success("select abs(k1)+k2+1,abs(k2+2)+k3+3 from d_table order by abs(k1)+k2+1,abs(k2+2)+k3+3", "k1a2p2ap3p") } diff --git a/regression-test/suites/mv_p0/multi_slot_k1a2p2ap3ps/multi_slot_k1a2p2ap3ps.groovy b/regression-test/suites/mv_p0/multi_slot_k1a2p2ap3ps/multi_slot_k1a2p2ap3ps.groovy index 3a2f58f38d65a9..b2292084504199 100644 --- a/regression-test/suites/mv_p0/multi_slot_k1a2p2ap3ps/multi_slot_k1a2p2ap3ps.groovy +++ b/regression-test/suites/mv_p0/multi_slot_k1a2p2ap3ps/multi_slot_k1a2p2ap3ps.groovy @@ -61,6 +61,7 @@ suite ("multi_slot_k1a2p2ap3ps") { qt_select_base "select abs(k1)+k2+1,sum(abs(k2+2)+k3+3) from d_table group by abs(k1)+k2 order by abs(k1)+k2;" sql """set enable_stats=true;""" + sql """alter table d_table modify column k1 set stats ('row_count'='4');""" mv_rewrite_success("select abs(k1)+k2+1,sum(abs(k2+2)+k3+3) from d_table group by abs(k1)+k2+1 order by abs(k1)+k2+1", "k1a2p2ap3ps") mv_rewrite_fail("select abs(k1)+k2+1,sum(abs(k2+2)+k3+3) from d_table group by abs(k1)+k2 order by abs(k1)+k2", "k1a2p2ap3ps") diff --git a/regression-test/suites/mv_p0/multi_slot_k1p2ap3p/multi_slot_k1p2ap3p.groovy b/regression-test/suites/mv_p0/multi_slot_k1p2ap3p/multi_slot_k1p2ap3p.groovy index fa67fd3889d93e..75bbd3fb4b7bb8 100644 --- a/regression-test/suites/mv_p0/multi_slot_k1p2ap3p/multi_slot_k1p2ap3p.groovy +++ b/regression-test/suites/mv_p0/multi_slot_k1p2ap3p/multi_slot_k1p2ap3p.groovy @@ -50,5 +50,6 @@ suite ("multi_slot_k1p2ap3p") { qt_select_mv "select k1+1,abs(k2+2)+k3+3 from d_table order by k1+1;" sql """set enable_stats=true;""" + sql """alter table d_table modify column k1 set stats ('row_count'='4');""" mv_rewrite_success("select k1+1,abs(k2+2)+k3+3 from d_table order by k1+1;", "k1p2ap3p") } diff --git a/regression-test/suites/mv_p0/multi_slot_k1p2ap3ps/multi_slot_k1p2ap3ps.groovy b/regression-test/suites/mv_p0/multi_slot_k1p2ap3ps/multi_slot_k1p2ap3ps.groovy index 9895bc6872e740..2ed75247e8ea29 100644 --- a/regression-test/suites/mv_p0/multi_slot_k1p2ap3ps/multi_slot_k1p2ap3ps.groovy +++ b/regression-test/suites/mv_p0/multi_slot_k1p2ap3ps/multi_slot_k1p2ap3ps.groovy @@ -50,5 +50,6 @@ suite ("multi_slot_k1p2ap3ps") { qt_select_mv "select k1+1,sum(abs(k2+2)+k3+3) from d_table group by k1+1 order by k1+1;" sql """set enable_stats=true;""" + sql """alter table d_table modify column k1 set stats ('row_count'='5');""" mv_rewrite_success("select k1+1,sum(abs(k2+2)+k3+3) from d_table group by k1+1 order by k1+1;", "k1p2ap3ps") } diff --git a/regression-test/suites/mv_p0/multi_slot_multi_mv/multi_slot_multi_mv.groovy b/regression-test/suites/mv_p0/multi_slot_multi_mv/multi_slot_multi_mv.groovy index 5527a511e9df2a..172e3217dafe4d 100644 --- a/regression-test/suites/mv_p0/multi_slot_multi_mv/multi_slot_multi_mv.groovy +++ b/regression-test/suites/mv_p0/multi_slot_multi_mv/multi_slot_multi_mv.groovy @@ -82,6 +82,7 @@ suite ("multi_slot_multi_mv") { qt_select_mv "select abs(k1)+k2+1,abs(k2+2)+k3+3 from d_table order by abs(k1)+k2+1,abs(k2+2)+k3+3;" sql """set enable_stats=true;""" + sql """alter table d_table modify column k1 set stats ('row_count'='5');""" for (def i = 0; i < retry_times; ++i) { boolean is_k1a2p2ap3p = false boolean is_k1a2p2ap3ps = false diff --git a/regression-test/suites/mv_p0/mv_with_view/mv_with_view.groovy b/regression-test/suites/mv_p0/mv_with_view/mv_with_view.groovy index 236bde3bf6c828..f2e389e867a564 100644 --- a/regression-test/suites/mv_p0/mv_with_view/mv_with_view.groovy +++ b/regression-test/suites/mv_p0/mv_with_view/mv_with_view.groovy @@ -67,6 +67,7 @@ suite ("mv_with_view") { qt_select_mv "select * from v_k124 order by k1;" sql """set enable_stats=true;""" + sql """alter table d_table modify column k1 set stats ('row_count'='3');""" mv_rewrite_fail("select * from d_table order by k1;", "k312") sql """ diff --git a/regression-test/suites/mv_p0/null_insert/null_insert.groovy b/regression-test/suites/mv_p0/null_insert/null_insert.groovy index 015406d80a3796..7b6ea058f2d90b 100644 --- a/regression-test/suites/mv_p0/null_insert/null_insert.groovy +++ b/regression-test/suites/mv_p0/null_insert/null_insert.groovy @@ -76,6 +76,7 @@ suite ("null_insert") { GROUP BY date,vid,os,ver,ip_country;""" sql """set enable_stats=true;""" + sql """alter table test modify column date set stats ('row_count'='3');""" mv_rewrite_success("""SELECT date, vid, os, ver, ip_country, hll_union(hll_hash(uid)) FROM test GROUP BY date,vid,os,ver,ip_country;""", "mv_test") diff --git a/regression-test/suites/mv_p0/routine_load_hll/routine_load_hll.groovy b/regression-test/suites/mv_p0/routine_load_hll/routine_load_hll.groovy index a872e23efc4cce..d34024ebdf5068 100644 --- a/regression-test/suites/mv_p0/routine_load_hll/routine_load_hll.groovy +++ b/regression-test/suites/mv_p0/routine_load_hll/routine_load_hll.groovy @@ -58,5 +58,6 @@ suite ("routine_load_hll") { qt_select_mv "select time_stamp, hll_union_agg(device_id) from test group by time_stamp order by 1;" sql """set enable_stats=true;""" + sql """alter table test modify column event_id set stats ('row_count'='2');""" mv_rewrite_success("select time_stamp, hll_union_agg(device_id) from test group by time_stamp order by 1;", "m_view") } diff --git a/regression-test/suites/mv_p0/ssb/multiple_ssb/multiple_ssb.groovy b/regression-test/suites/mv_p0/ssb/multiple_ssb/multiple_ssb.groovy index eb8cfd1618fea3..e0dc916c05b66a 100644 --- a/regression-test/suites/mv_p0/ssb/multiple_ssb/multiple_ssb.groovy +++ b/regression-test/suites/mv_p0/ssb/multiple_ssb/multiple_ssb.groovy @@ -240,6 +240,7 @@ suite ("multiple_ssb") { qt_select_count_3 "select LO_ORDERPRIORITY, count(1) from lineorder_flat where LO_ORDERPRIORITY in ('1','2','3') group by LO_ORDERPRIORITY order by 1,2;" sql """set enable_stats=true;""" + sql """alter table lineorder_flat modify column LO_ORDERDATE set stats ('row_count'='8');""" mv_rewrite_success("""SELECT SUM(LO_EXTENDEDPRICE * LO_DISCOUNT) AS revenue FROM lineorder_flat WHERE diff --git a/regression-test/suites/mv_p0/ssb/multiple_ssb_between/multiple_ssb_between.groovy b/regression-test/suites/mv_p0/ssb/multiple_ssb_between/multiple_ssb_between.groovy index 71b737279f35bf..4dfdd748a5765b 100644 --- a/regression-test/suites/mv_p0/ssb/multiple_ssb_between/multiple_ssb_between.groovy +++ b/regression-test/suites/mv_p0/ssb/multiple_ssb_between/multiple_ssb_between.groovy @@ -151,6 +151,8 @@ suite ("multiple_ssb_between") { sql "analyze table lineorder_flat with sync;" sql """set enable_stats=true;""" + + sql """alter table lineorder_flat modify column LO_ORDERDATE set stats ('row_count'='8');""" mv_rewrite_success("""SELECT SUM(LO_EXTENDEDPRICE * LO_DISCOUNT) AS revenue FROM lineorder_flat diff --git a/regression-test/suites/mv_p0/ssb/q_1_1/q_1_1.groovy b/regression-test/suites/mv_p0/ssb/q_1_1/q_1_1.groovy index 30959801e72af1..c7aa6e67c6ee8e 100644 --- a/regression-test/suites/mv_p0/ssb/q_1_1/q_1_1.groovy +++ b/regression-test/suites/mv_p0/ssb/q_1_1/q_1_1.groovy @@ -75,10 +75,12 @@ suite ("mv_ssb_q_1_1") { // Add more data when insert into firstly to make sure use mv easier sql """INSERT INTO lineorder_flat (LO_ORDERDATE, LO_ORDERKEY, LO_LINENUMBER, LO_CUSTKEY, LO_PARTKEY, LO_SUPPKEY, LO_ORDERPRIORITY, LO_SHIPPRIORITY, LO_QUANTITY, LO_EXTENDEDPRICE, LO_ORDTOTALPRICE, LO_DISCOUNT, LO_REVENUE, LO_SUPPLYCOST, LO_TAX, LO_COMMITDATE, LO_SHIPMODE, C_NAME, C_ADDRESS, C_CITY, C_NATION, C_REGION, C_PHONE, C_MKTSEGMENT, S_NAME, S_ADDRESS, S_CITY, S_NATION, S_REGION, S_PHONE, P_NAME, P_MFGR, P_CATEGORY, P_BRAND, P_COLOR,P_TYPE,P_SIZE,P_CONTAINER) VALUES (19930101 , 1 , 1 , 1 , 1 , 1 , '1' , 1 , 1 , 1 , 1 , 100 , 1 , 1 , 1 , '2023-06-09' , 'shipmode' , 'name' , 'address' , 'city' , 'nation' , 'AMERICA' , 'phone' , 'mktsegment' , 'name' , 'address' , 'city' , 'nation' , 'AMERICA' ,'phone', 'name', 'MFGR#1', 'category', 'brand', 'color', 'type', 4 ,'container'), - (19930101 , 1 , 1 , 1 , 1 , 1 , '1' , 1 , 1 , 1 , 1 , 100 , 1 , 1 , 1 , '2023-06-09' , 'shipmode' , 'name' , 'address' , 'city' , 'nation' , 'AMERICA' , 'phone' , 'mktsegment' , 'name' , 'address' , 'city' , 'nation' , 'AMERICA' ,'phone', 'name', 'MFGR#1', 'category', 'brand', 'color', 'type', 4 ,'container'), - (19930101 , 1 , 1 , 1 , 1 , 1 , '1' , 1 , 1 , 1 , 1 , 100 , 1 , 1 , 1 , '2023-06-09' , 'shipmode' , 'name' , 'address' , 'city' , 'nation' , 'AMERICA' , 'phone' , 'mktsegment' , 'name' , 'address' , 'city' , 'nation' , 'AMERICA' ,'phone', 'name', 'MFGR#1', 'category', 'brand', 'color', 'type', 4 ,'container'), - (19930101 , 1 , 1 , 1 , 1 , 1 , '1' , 1 , 1 , 1 , 1 , 100 , 1 , 1 , 1 , '2023-06-09' , 'shipmode' , 'name' , 'address' , 'city' , 'nation' , 'AMERICA' , 'phone' , 'mktsegment' , 'name' , 'address' , 'city' , 'nation' , 'AMERICA' ,'phone', 'name', 'MFGR#1', 'category', 'brand', 'color', 'type', 4 ,'container'), - (19930101 , 1 , 1 , 1 , 1 , 1 , '1' , 1 , 1 , 1 , 1 , 100 , 1 , 1 , 1 , '2023-06-09' , 'shipmode' , 'name' , 'address' , 'city' , 'nation' , 'AMERICA' , 'phone' , 'mktsegment' , 'name' , 'address' , 'city' , 'nation' , 'AMERICA' ,'phone', 'name', 'MFGR#1', 'category', 'brand', 'color', 'type', 4 ,'container');""" + (19930101 , 1 , 1 , 1 , 1 , 1 , '1' , 1 , 1 , 1 , 1 , 100 , 1 , 1 , 1 , '2023-06-09' , 'shipmode' , 'name' , 'address' , 'city' , 'nation' , 'AMERICA' , 'phone' , 'mktsegment' , 'name' , 'address' , 'city' , 'nation' , 'AMERICA' ,'phone', 'name', 'MFGR#1', 'category', 'brand', 'color', 'type', 4 ,'container'), + (19930101 , 1 , 1 , 1 , 1 , 1 , '1' , 1 , 1 , 1 , 1 , 100 , 1 , 1 , 1 , '2023-06-09' , 'shipmode' , 'name' , 'address' , 'city' , 'nation' , 'AMERICA' , 'phone' , 'mktsegment' , 'name' , 'address' , 'city' , 'nation' , 'AMERICA' ,'phone', 'name', 'MFGR#1', 'category', 'brand', 'color', 'type', 4 ,'container'), + (19930101 , 1 , 1 , 1 , 1 , 1 , '1' , 1 , 1 , 1 , 1 , 100 , 1 , 1 , 1 , '2023-06-09' , 'shipmode' , 'name' , 'address' , 'city' , 'nation' , 'AMERICA' , 'phone' , 'mktsegment' , 'name' , 'address' , 'city' , 'nation' , 'AMERICA' ,'phone', 'name', 'MFGR#1', 'category', 'brand', 'color', 'type', 4 ,'container'), + (19930101 , 1 , 1 , 1 , 1 , 1 , '1' , 1 , 1 , 1 , 1 , 100 , 1 , 1 , 1 , '2023-06-09' , 'shipmode' , 'name' , 'address' , 'city' , 'nation' , 'AMERICA' , 'phone' , 'mktsegment' , 'name' , 'address' , 'city' , 'nation' , 'AMERICA' ,'phone', 'name', 'MFGR#1', 'category', 'brand', 'color', 'type', 4 ,'container');""" + + sql """alter table lineorder_flat modify column LO_ORDERDATE set stats ('row_count'='6');""" createMV ("""create materialized view lineorder_q_1_1 as SELECT LO_ORDERKEY, SUM(LO_EXTENDEDPRICE * LO_DISCOUNT) AS revenue @@ -88,12 +90,14 @@ suite ("mv_ssb_q_1_1") { AND LO_ORDERDATE <= 19931231 AND LO_DISCOUNT >= 1 AND LO_DISCOUNT <= 3 AND LO_QUANTITY < 25 - GROUP BY - LO_ORDERKEY;""") + GROUP BY LO_ORDERKEY;""") sql """INSERT INTO lineorder_flat (LO_ORDERDATE, LO_ORDERKEY, LO_LINENUMBER, LO_CUSTKEY, LO_PARTKEY, LO_SUPPKEY, LO_ORDERPRIORITY, LO_SHIPPRIORITY, LO_QUANTITY, LO_EXTENDEDPRICE, LO_ORDTOTALPRICE, LO_DISCOUNT, LO_REVENUE, LO_SUPPLYCOST, LO_TAX, LO_COMMITDATE, LO_SHIPMODE,C_NAME,C_ADDRESS,C_CITY,C_NATION,C_REGION,C_PHONE,C_MKTSEGMENT,S_NAME,S_ADDRESS,S_CITY,S_NATION,S_REGION,S_PHONE,P_NAME,P_MFGR,P_CATEGORY,P_BRAND,P_COLOR,P_TYPE,P_SIZE,P_CONTAINER) - VALUES (19930101 , 2 , 2 , 2 , 2 , 2 ,'2',2 ,2 ,2 ,2 ,2 ,2 ,2 ,2 ,'2023-06-09','shipmode','name','address','city','nation','region','phone','mktsegment','name','address','city','nation','region','phone','name','mfgr','category','brand','color','type',4,'container');""" - + VALUES + (19930101 , 2 , 2 , 2 , 2 , 2 ,'2',2 ,2 ,2 ,2 ,2 ,2 ,2 ,2 ,'2023-06-09','shipmode','name','address','city','nation','region','phone','mktsegment','name','address','city','nation','region','phone','name','mfgr','category','brand','color','type',4,'container'), + (19930101 , 2 , 2 , 2 , 2 , 2 ,'2',2 ,2 ,2 ,2 ,2 ,2 ,2 ,2 ,'2023-06-09','shipmode','name','address','city','nation','region','phone','mktsegment','name','address','city','nation','region','phone','name','mfgr','category','brand','color','type',4,'container'), + (19930101 , 2 , 2 , 2 , 2 , 2 ,'2',2 ,2 ,2 ,2 ,2 ,2 ,2 ,2 ,'2023-06-09','shipmode','name','address','city','nation','region','phone','mktsegment','name','address','city','nation','region','phone','name','mfgr','category','brand','color','type',4,'container'), + (19930101 , 2 , 2 , 2 , 2 , 2 ,'2',2 ,2 ,2 ,2 ,2 ,2 ,2 ,2 ,'2023-06-09','shipmode','name','address','city','nation','region','phone','mktsegment','name','address','city','nation','region','phone','name','mfgr','category','brand','color','type',4,'container');""" qt_select_star "select * from lineorder_flat order by 1,2, P_MFGR;" sql "analyze table lineorder_flat with sync;" diff --git a/regression-test/suites/mv_p0/ssb/q_2_1/q_2_1.groovy b/regression-test/suites/mv_p0/ssb/q_2_1/q_2_1.groovy index 6ead8380eed59b..0ac5f0245ef8ae 100644 --- a/regression-test/suites/mv_p0/ssb/q_2_1/q_2_1.groovy +++ b/regression-test/suites/mv_p0/ssb/q_2_1/q_2_1.groovy @@ -93,6 +93,8 @@ suite ("mv_ssb_q_2_1") { qt_select_star "select * from lineorder_flat order by 1,2,P_MFGR;" + sql """alter table lineorder_flat modify column LO_ORDERDATE set stats ('row_count'='2');""" + mv_rewrite_success("""SELECT SUM(LO_REVENUE), (LO_ORDERDATE DIV 10000) AS YEAR, P_BRAND diff --git a/regression-test/suites/mv_p0/ssb/q_3_1/q_3_1.groovy b/regression-test/suites/mv_p0/ssb/q_3_1/q_3_1.groovy index 2b170fe3b2fefd..6b333d97050630 100644 --- a/regression-test/suites/mv_p0/ssb/q_3_1/q_3_1.groovy +++ b/regression-test/suites/mv_p0/ssb/q_3_1/q_3_1.groovy @@ -98,6 +98,8 @@ suite ("mv_ssb_q_3_1") { sql """analyze table lineorder_flat with sync;""" + sql """alter table lineorder_flat modify column LO_ORDERDATE set stats ('row_count'='2');""" + mv_rewrite_success("""SELECT C_NATION, S_NATION, (LO_ORDERDATE DIV 10000) AS YEAR, diff --git a/regression-test/suites/mv_p0/ssb/q_4_1/q_4_1.groovy b/regression-test/suites/mv_p0/ssb/q_4_1/q_4_1.groovy index c094e8e00c9eaf..361f60997e3da6 100644 --- a/regression-test/suites/mv_p0/ssb/q_4_1/q_4_1.groovy +++ b/regression-test/suites/mv_p0/ssb/q_4_1/q_4_1.groovy @@ -92,6 +92,8 @@ suite ("mv_ssb_q_4_1") { sql """analyze table lineorder_flat with sync;""" + sql """alter table lineorder_flat modify column LO_ORDERDATE set stats ('row_count'='2');""" + mv_rewrite_success("""SELECT (LO_ORDERDATE DIV 10000) AS YEAR, C_NATION, SUM(LO_REVENUE - LO_SUPPLYCOST) AS profit diff --git a/regression-test/suites/mv_p0/ssb/q_4_1_r1/q_4_1_r1.groovy b/regression-test/suites/mv_p0/ssb/q_4_1_r1/q_4_1_r1.groovy index 1baf21b547f062..ebeb23d96e75e5 100644 --- a/regression-test/suites/mv_p0/ssb/q_4_1_r1/q_4_1_r1.groovy +++ b/regression-test/suites/mv_p0/ssb/q_4_1_r1/q_4_1_r1.groovy @@ -119,6 +119,7 @@ suite ("q_4_1_r1") { GROUP BY YEAR, C_NATION ORDER BY YEAR ASC, C_NATION ASC;""" sql """set enable_stats=true;""" + sql """alter table lineorder_flat modify column LO_ORDERDATE set stats ('row_count'='8');""" mv_rewrite_success("""SELECT (LO_ORDERDATE DIV 10000) AS YEAR, C_NATION, SUM(LO_REVENUE - LO_SUPPLYCOST) AS profit diff --git a/regression-test/suites/mv_p0/sum_count/sum_count.groovy b/regression-test/suites/mv_p0/sum_count/sum_count.groovy index b47e51999b7c42..137aabc4e16a89 100644 --- a/regression-test/suites/mv_p0/sum_count/sum_count.groovy +++ b/regression-test/suites/mv_p0/sum_count/sum_count.groovy @@ -51,6 +51,7 @@ suite ("sum_count") { sql "analyze table d_table with sync;" sql """set enable_stats=true;""" + sql """alter table d_table modify column k1 set stats ('row_count'='8');""" mv_rewrite_success("select k1,k4,sum(k2),count(k2) from d_table group by k1,k4;", "kavg") qt_select_mv "select k1,k4,sum(k2),count(k2) from d_table group by k1,k4 order by 1,2;" diff --git a/regression-test/suites/mv_p0/sum_divede_count/sum_devide_count.groovy b/regression-test/suites/mv_p0/sum_divede_count/sum_devide_count.groovy index 4e6374e300b09e..85a3f7137b9e40 100644 --- a/regression-test/suites/mv_p0/sum_divede_count/sum_devide_count.groovy +++ b/regression-test/suites/mv_p0/sum_divede_count/sum_devide_count.groovy @@ -46,6 +46,7 @@ suite ("sum_devide_count") { sql """analyze table d_table with sync;""" sql """set enable_stats=false;""" + mv_rewrite_success("select k1,k4,sum(k2)/count(k2) from d_table group by k1,k4 order by k1,k4;", "kavg") qt_select_mv "select k1,k4,sum(k2)/count(k2) from d_table group by k1,k4 order by k1,k4;" @@ -59,6 +60,7 @@ suite ("sum_devide_count") { qt_select_mv "select sum(k2)/count(k2) from d_table;" sql """set enable_stats=true;""" + sql """alter table d_table modify column k1 set stats ('row_count'='5');""" mv_rewrite_success("select k1,k4,sum(k2)/count(k2) from d_table group by k1,k4 order by k1,k4;", "kavg") mv_rewrite_success("select k1,sum(k2)/count(k2) from d_table group by k1 order by k1;", "kavg") diff --git a/regression-test/suites/mv_p0/test_28741/test_28741.groovy b/regression-test/suites/mv_p0/test_28741/test_28741.groovy index 9fabca41b943fc..88fb3c057da092 100644 --- a/regression-test/suites/mv_p0/test_28741/test_28741.groovy +++ b/regression-test/suites/mv_p0/test_28741/test_28741.groovy @@ -69,8 +69,8 @@ suite ("test_28741") { sql """set enable_stats=false;""" mv_rewrite_fail("select b1 from test where t >= '2023-12-20 17:21:00'", "mv_test") - qt_select "select b1 from test where t >= '2023-12-20 17:21:00'" sql """set enable_stats=true;""" + sql """alter table test modify column a set stats ('row_count'='2');""" mv_rewrite_fail("select b1 from test where t >= '2023-12-20 17:21:00'", "mv_test") } diff --git a/regression-test/suites/mv_p0/test_approx_count_distinct/test_approx_count_distinct.groovy b/regression-test/suites/mv_p0/test_approx_count_distinct/test_approx_count_distinct.groovy index 752cf40d9e5f6a..68952da602aa81 100644 --- a/regression-test/suites/mv_p0/test_approx_count_distinct/test_approx_count_distinct.groovy +++ b/regression-test/suites/mv_p0/test_approx_count_distinct/test_approx_count_distinct.groovy @@ -50,6 +50,7 @@ suite ("test_approx_count_distinct") { qt_select_mv "select user_id, approx_count_distinct(tag_id) a from user_tags group by user_id order by user_id;" sql """set enable_stats=true;""" + sql """alter table user_tags modify column time_col set stats ('row_count'='3');""" mv_rewrite_fail("select * from user_tags order by time_col;", "user_tags_mv") mv_rewrite_success("select user_id, ndv(tag_id) a from user_tags group by user_id order by user_id;", "user_tags_mv") diff --git a/regression-test/suites/mv_p0/test_base/test_base.groovy b/regression-test/suites/mv_p0/test_base/test_base.groovy index d2c018febd5ada..97ece8cd6cca68 100644 --- a/regression-test/suites/mv_p0/test_base/test_base.groovy +++ b/regression-test/suites/mv_p0/test_base/test_base.groovy @@ -53,6 +53,7 @@ suite ("test_base") { qt_select_mv "SELECT id,created_at FROM dwd order by 1, 2;" sql """set enable_stats=true;""" + sql """alter table dwd modify column id set stats ('row_count'='2');""" mv_rewrite_success("SELECT created_at, id FROM dwd where created_at = '2020-09-09 00:00:00' order by 1, 2;", "dwd_mv") mv_rewrite_success("SELECT id,created_at FROM dwd where id is not null order by 1, 2;", "dwd_mv") diff --git a/regression-test/suites/mv_p0/test_casewhen/test_casewhen.groovy b/regression-test/suites/mv_p0/test_casewhen/test_casewhen.groovy index eac65eeef22adf..0c9e0026cb7fbd 100644 --- a/regression-test/suites/mv_p0/test_casewhen/test_casewhen.groovy +++ b/regression-test/suites/mv_p0/test_casewhen/test_casewhen.groovy @@ -40,5 +40,6 @@ suite ("test_casewhen") { qt_select_mv "select store_id, sum(case when sale_amt>10 then 1 else 2 end) from sales_records group by store_id order by 1;" sql """set enable_stats=true;""" + sql """alter table sales_records modify column record_id set stats ('row_count'='4');""" mv_rewrite_success("select store_id, sum(case when sale_amt>10 then 1 else 2 end) from sales_records group by store_id order by 1;", "store_amt") } diff --git a/regression-test/suites/mv_p0/test_create_mv/test_create_mv.groovy b/regression-test/suites/mv_p0/test_create_mv/test_create_mv.groovy index 776bf2ecf0bd17..8ee24cb5ba2bd9 100644 --- a/regression-test/suites/mv_p0/test_create_mv/test_create_mv.groovy +++ b/regression-test/suites/mv_p0/test_create_mv/test_create_mv.groovy @@ -49,6 +49,8 @@ suite("test_create_mv") { sql """ insert into ${tableName} values ('2024-03-20 10:00:00', 'a', 'b', 1) """ + sql """alter table test_mv_10010 modify column load_time set stats ('row_count'='1');""" + sql """ create materialized view mv_1 as select diff --git a/regression-test/suites/mv_p0/test_create_mv_complex_type/test_create_mv_complex_type.groovy b/regression-test/suites/mv_p0/test_create_mv_complex_type/test_create_mv_complex_type.groovy index 12790500d2f98e..1e07d69ba82871 100644 --- a/regression-test/suites/mv_p0/test_create_mv_complex_type/test_create_mv_complex_type.groovy +++ b/regression-test/suites/mv_p0/test_create_mv_complex_type/test_create_mv_complex_type.groovy @@ -37,6 +37,8 @@ suite ("create_mv_complex_type") { sql """insert into base_table select 1, 100000, 1.0, '{"jsonk1": 123}', [100, 200], {"k1": 10}, {1, 2};""" + sql """alter table base_table modify column c_int set stats ('row_count'='1');""" + def success = false // 1. special column - mv dup key diff --git a/regression-test/suites/mv_p0/test_doc_e4/test_doc_e4.groovy b/regression-test/suites/mv_p0/test_doc_e4/test_doc_e4.groovy index 5e99464018e8d7..517b4415bd69ea 100644 --- a/regression-test/suites/mv_p0/test_doc_e4/test_doc_e4.groovy +++ b/regression-test/suites/mv_p0/test_doc_e4/test_doc_e4.groovy @@ -57,6 +57,7 @@ suite ("test_doc_e4") { qt_select_mv "select year(k4)+month(k4) from d_table where year(k4) = 2020 order by 1;" sql """set enable_stats=true;""" + sql """alter table d_table modify column k1 set stats ('row_count'='3');""" mv_rewrite_success("select abs(k1)+k2+1,sum(abs(k2+2)+k3+3) from d_table group by abs(k1)+k2+1 order by 1,2;", "k1a2p2ap3ps") mv_rewrite_success("select bin(abs(k1)+k2+1),sum(abs(k2+2)+k3+3) from d_table group by bin(abs(k1)+k2+1);", "k1a2p2ap3ps") diff --git a/regression-test/suites/mv_p0/test_dup_group_by_mv_abs/test_dup_group_by_mv_abs.groovy b/regression-test/suites/mv_p0/test_dup_group_by_mv_abs/test_dup_group_by_mv_abs.groovy index 5e917e9b0f5bd0..23aeeb207f6898 100644 --- a/regression-test/suites/mv_p0/test_dup_group_by_mv_abs/test_dup_group_by_mv_abs.groovy +++ b/regression-test/suites/mv_p0/test_dup_group_by_mv_abs/test_dup_group_by_mv_abs.groovy @@ -53,6 +53,7 @@ suite ("test_dup_group_by_mv_abs") { qt_select_mv_sub "select sum(abs(k2)) from d_table group by k1 order by k1;" sql """set enable_stats=true;""" + sql """alter table d_table modify column k1 set stats ('row_count'='4');""" mv_rewrite_success("select k1,sum(abs(k2)) from d_table group by k1;", "k12sa") mv_rewrite_success("select sum(abs(k2)) from d_table group by k1;", "k12sa") } diff --git a/regression-test/suites/mv_p0/test_dup_group_by_mv_plus/test_dup_group_by_mv_plus.groovy b/regression-test/suites/mv_p0/test_dup_group_by_mv_plus/test_dup_group_by_mv_plus.groovy index ba9f60f6df5026..3551bb8a7fb359 100644 --- a/regression-test/suites/mv_p0/test_dup_group_by_mv_plus/test_dup_group_by_mv_plus.groovy +++ b/regression-test/suites/mv_p0/test_dup_group_by_mv_plus/test_dup_group_by_mv_plus.groovy @@ -53,6 +53,7 @@ suite ("test_dup_group_by_mv_plus") { qt_select_mv_sub "select sum(k2+1) from d_table group by k1 order by k1;" sql """set enable_stats=true;""" + sql """alter table d_table modify column k1 set stats ('row_count'='4');""" mv_rewrite_success("select k1,sum(k2+1) from d_table group by k1;", "k12sp") mv_rewrite_success("select sum(k2+1) from d_table group by k1;", "k12sp") diff --git a/regression-test/suites/mv_p0/test_dup_mv_abs/test_dup_mv_abs.groovy b/regression-test/suites/mv_p0/test_dup_mv_abs/test_dup_mv_abs.groovy index 89088ffb9ea317..f44c9d134f91f0 100644 --- a/regression-test/suites/mv_p0/test_dup_mv_abs/test_dup_mv_abs.groovy +++ b/regression-test/suites/mv_p0/test_dup_mv_abs/test_dup_mv_abs.groovy @@ -65,6 +65,7 @@ suite ("test_dup_mv_abs") { qt_select_group_mv_not "select sum(abs(k2)) from d_table group by k3 order by k3;" sql """set enable_stats=true;""" + sql """alter table d_table modify column k1 set stats ('row_count'='4');""" mv_rewrite_success("select k1,abs(k2) from d_table order by k1;", "k12a") mv_rewrite_success("select abs(k2) from d_table order by k1;", "k12a") diff --git a/regression-test/suites/mv_p0/test_dup_mv_bin/test_dup_mv_bin.groovy b/regression-test/suites/mv_p0/test_dup_mv_bin/test_dup_mv_bin.groovy index a7db2617fe6067..4b8fa5aa6359e8 100644 --- a/regression-test/suites/mv_p0/test_dup_mv_bin/test_dup_mv_bin.groovy +++ b/regression-test/suites/mv_p0/test_dup_mv_bin/test_dup_mv_bin.groovy @@ -64,6 +64,7 @@ suite ("test_dup_mv_bin") { qt_select_group_mv_not "select group_concat(bin(k2)) from d_table group by k3 order by k3;" sql """set enable_stats=true;""" + sql """alter table d_table modify column k1 set stats ('row_count'='4');""" mv_rewrite_success("select k1,bin(k2) from d_table order by k1;", "k12b") mv_rewrite_success("select bin(k2) from d_table order by k1;", "k12b") diff --git a/regression-test/suites/mv_p0/test_dup_mv_bitmap_hash/test_dup_mv_bitmap_hash.groovy b/regression-test/suites/mv_p0/test_dup_mv_bitmap_hash/test_dup_mv_bitmap_hash.groovy index 659f44de1e93c4..6530fd84cac7a8 100644 --- a/regression-test/suites/mv_p0/test_dup_mv_bitmap_hash/test_dup_mv_bitmap_hash.groovy +++ b/regression-test/suites/mv_p0/test_dup_mv_bitmap_hash/test_dup_mv_bitmap_hash.groovy @@ -46,6 +46,7 @@ suite ("test_dup_mv_bitmap_hash") { mv_rewrite_success("select bitmap_union_count(to_bitmap(k2)) from d_table group by k1 order by k1;", "k1g2bm") qt_select_mv "select bitmap_union_count(to_bitmap(k2)) from d_table group by k1 order by k1;" sql """set enable_stats=true;""" + sql """alter table d_table modify column k1 set stats ('row_count'='4');""" mv_rewrite_success("select bitmap_union_count(to_bitmap(k2)) from d_table group by k1 order by k1;", "k1g2bm") createMV "create materialized view k1g3bm as select k1,bitmap_union(bitmap_hash(k3)) from d_table group by k1;" @@ -58,11 +59,13 @@ suite ("test_dup_mv_bitmap_hash") { qt_select_star "select * from d_table order by k1,k2,k3;" sql """set enable_stats=true;""" + sql """alter table d_table modify column k1 set stats ('row_count'='4');""" sql """analyze table d_table with sync;""" sql """set enable_stats=false;""" mv_rewrite_success("select k1,bitmap_union_count(bitmap_hash(k3)) from d_table group by k1;", "k1g3bm") qt_select_mv_sub "select k1,bitmap_union_count(bitmap_hash(k3)) from d_table group by k1 order by k1;" sql """set enable_stats=true;""" + sql """alter table d_table modify column k1 set stats ('row_count'='4');""" mv_rewrite_success("select k1,bitmap_union_count(bitmap_hash(k3)) from d_table group by k1;", "k1g3bm") } diff --git a/regression-test/suites/mv_p0/test_dup_mv_plus/test_dup_mv_plus.groovy b/regression-test/suites/mv_p0/test_dup_mv_plus/test_dup_mv_plus.groovy index 5abe03e736b641..ad437db464b3bb 100644 --- a/regression-test/suites/mv_p0/test_dup_mv_plus/test_dup_mv_plus.groovy +++ b/regression-test/suites/mv_p0/test_dup_mv_plus/test_dup_mv_plus.groovy @@ -70,7 +70,7 @@ suite ("test_dup_mv_plus") { qt_select_mv "select k1,k2+1 from d_table order by k2;" sql """set enable_stats=true;""" - + sql """alter table d_table modify column k4 set stats ('row_count'='3');""" mv_rewrite_success("select k1,k2+1 from d_table order by k1;", "k12p") mv_rewrite_success("select k2+1 from d_table order by k1;", "k12p") diff --git a/regression-test/suites/mv_p0/test_dup_mv_repeat/test_dup_mv_repeat.groovy b/regression-test/suites/mv_p0/test_dup_mv_repeat/test_dup_mv_repeat.groovy index 562fba25b2e8a9..5ccdae1bd8a7c5 100644 --- a/regression-test/suites/mv_p0/test_dup_mv_repeat/test_dup_mv_repeat.groovy +++ b/regression-test/suites/mv_p0/test_dup_mv_repeat/test_dup_mv_repeat.groovy @@ -47,6 +47,7 @@ suite ("test_dup_mv_repeat") { qt_select_mv "SELECT s AS s, sum(n) / count(DISTINCT dt) AS n FROM db1 GROUP BY GROUPING SETS((s)) order by 1;" sql """set enable_stats=true;""" + sql """alter table db1 modify column dt set stats ('row_count'='2');""" mv_rewrite_success("SELECT s AS s, sum(n) / count(DISTINCT dt) AS n FROM db1 GROUP BY GROUPING SETS((s)) order by 1;", "dbviwe") } diff --git a/regression-test/suites/mv_p0/test_dup_mv_year/test_dup_mv_year.groovy b/regression-test/suites/mv_p0/test_dup_mv_year/test_dup_mv_year.groovy index 92a8d6520f3122..18f798f8b6aac3 100644 --- a/regression-test/suites/mv_p0/test_dup_mv_year/test_dup_mv_year.groovy +++ b/regression-test/suites/mv_p0/test_dup_mv_year/test_dup_mv_year.groovy @@ -45,6 +45,7 @@ suite ("test_dup_mv_year") { qt_select_mv "select k1,year(k2) from d_table order by k1;" sql """set enable_stats=true;""" + sql """alter table d_table modify column k1 set stats ('row_count'='4');""" mv_rewrite_success("select k1,year(k2) from d_table order by k1;", "k12y") createMV "create materialized view k13y as select k1,year(k3) from d_table;" diff --git a/regression-test/suites/mv_p0/test_duplicate_mv/test_duplicate_mv.groovy b/regression-test/suites/mv_p0/test_duplicate_mv/test_duplicate_mv.groovy index ccd87d792348c3..c1a495a74e20bc 100644 --- a/regression-test/suites/mv_p0/test_duplicate_mv/test_duplicate_mv.groovy +++ b/regression-test/suites/mv_p0/test_duplicate_mv/test_duplicate_mv.groovy @@ -39,6 +39,8 @@ suite ("test_duplicate_mv") { sql "insert into duplicate_table select 2,3,4,5;" sql "insert into duplicate_table select 1,2,3,4;" + sql """alter table duplicate_table modify column k1 set stats ('row_count'='3');""" + qt_select_star "select * from duplicate_table order by k1;" mv_rewrite_success("select k1, k2, k3, k4 from duplicate_table group by k1, k2, k3, k4;", "deduplicate") diff --git a/regression-test/suites/mv_p0/test_mv_dp/test_mv_dp.groovy b/regression-test/suites/mv_p0/test_mv_dp/test_mv_dp.groovy index 50f1fd61806f2e..2fadc0b0da91a0 100644 --- a/regression-test/suites/mv_p0/test_mv_dp/test_mv_dp.groovy +++ b/regression-test/suites/mv_p0/test_mv_dp/test_mv_dp.groovy @@ -56,6 +56,7 @@ suite ("test_mv_dp") { time 10000 // limit inflight 10s } */ + mv_rewrite_success("""select d, bitmap_union_count(bitmap_from_array(cast(uid_list as array))), bitmap_union_count(bitmap_from_array(if(status='success', cast(uid_list as array), array()))) @@ -68,6 +69,7 @@ suite ("test_mv_dp") { from dp group by d order by 1;""" sql """set enable_stats=true;""" + sql """alter table dp modify column d set stats ('row_count'='4');""" mv_rewrite_success("""select d, bitmap_union_count(bitmap_from_array(cast(uid_list as array))), bitmap_union_count(bitmap_from_array(if(status='success', cast(uid_list as array), array()))) diff --git a/regression-test/suites/mv_p0/test_mv_mor/test_mv_mor.groovy b/regression-test/suites/mv_p0/test_mv_mor/test_mv_mor.groovy index b7f95123f5f9dd..072c7d905cbea0 100644 --- a/regression-test/suites/mv_p0/test_mv_mor/test_mv_mor.groovy +++ b/regression-test/suites/mv_p0/test_mv_mor/test_mv_mor.groovy @@ -39,6 +39,8 @@ suite ("test_mv_mor") { sql "insert into u_table select 1,1,1,2;" sql "insert into u_table select 1,2,1,2;" + sql """alter table u_table modify column k1 set stats ('row_count'='2');""" + // do not match mv coz preagg is off, mv need contains all key column to make row count correct mv_rewrite_success("select k1,k2+k3 from u_table order by k1;", "k123p") qt_select_mv "select k1,k2+k3 from u_table order by k1;" diff --git a/regression-test/suites/mv_p0/test_mv_mow/test_mv_mow.groovy b/regression-test/suites/mv_p0/test_mv_mow/test_mv_mow.groovy index f584730ad1fca3..3fe10c2e50a33a 100644 --- a/regression-test/suites/mv_p0/test_mv_mow/test_mv_mow.groovy +++ b/regression-test/suites/mv_p0/test_mv_mow/test_mv_mow.groovy @@ -50,5 +50,6 @@ suite ("test_mv_mow") { qt_select_mv "select mv_k1 from `u_table` index `k123p` order by 1;" qt_select_mv "select `mv_(k2 + k3)` from `u_table` index `k123p` order by 1;" sql """set enable_stats=true;""" + sql """alter table u_table modify column k1 set stats ('row_count'='2');""" mv_rewrite_success("select k1,k2+k3 from u_table order by k1;", "k123p") } diff --git a/regression-test/suites/mv_p0/test_ndv/test_ndv.groovy b/regression-test/suites/mv_p0/test_ndv/test_ndv.groovy index d2c9921edefe7a..2d43785ba6fafb 100644 --- a/regression-test/suites/mv_p0/test_ndv/test_ndv.groovy +++ b/regression-test/suites/mv_p0/test_ndv/test_ndv.groovy @@ -50,6 +50,7 @@ suite ("test_ndv") { qt_select_mv "select user_id, approx_count_distinct(tag_id) a from user_tags group by user_id order by user_id;" sql """set enable_stats=true;""" + sql """alter table user_tags modify column time_col set stats ('row_count'='3');""" mv_rewrite_fail("select * from user_tags order by time_col;", "user_tags_mv") mv_rewrite_success("select user_id, ndv(tag_id) a from user_tags group by user_id order by user_id;", "user_tags_mv") diff --git a/regression-test/suites/mv_p0/test_nvl/test_nvl.groovy b/regression-test/suites/mv_p0/test_nvl/test_nvl.groovy index c6b1dd2d8a9b1c..12f5765666ef28 100644 --- a/regression-test/suites/mv_p0/test_nvl/test_nvl.groovy +++ b/regression-test/suites/mv_p0/test_nvl/test_nvl.groovy @@ -51,6 +51,7 @@ suite ("test_nvl") { qt_select_mv "select ifnull(id,0) from dwd order by 1;" sql """set enable_stats=true;""" + sql """alter table dwd modify column id set stats ('row_count'='2');""" mv_rewrite_success("select nvl(id,0) from dwd order by 1;", "dwd_mv") mv_rewrite_success("select ifnull(id,0) from dwd order by 1;", "dwd_mv") diff --git a/regression-test/suites/mv_p0/test_o2/test_o2.groovy b/regression-test/suites/mv_p0/test_o2/test_o2.groovy index 665156056c0096..34d31a08cfae7b 100644 --- a/regression-test/suites/mv_p0/test_o2/test_o2.groovy +++ b/regression-test/suites/mv_p0/test_o2/test_o2.groovy @@ -60,6 +60,7 @@ suite ("test_o2") { qt_select_mv "select ts,metric_name,platform,sum(count_value) from o2_order_events group by ts,metric_name,platform;" sql """set enable_stats=true;""" + sql """alter table o2_order_events modify column ts set stats ('row_count'='2');""" mv_rewrite_success("select ts,metric_name,platform,sum(count_value) from o2_order_events group by ts,metric_name,platform;", "o2_order_events_mv") } diff --git a/regression-test/suites/mv_p0/test_substr/test_substr.groovy b/regression-test/suites/mv_p0/test_substr/test_substr.groovy index c43c3ae6483acd..7d6ae82634fd8c 100644 --- a/regression-test/suites/mv_p0/test_substr/test_substr.groovy +++ b/regression-test/suites/mv_p0/test_substr/test_substr.groovy @@ -56,6 +56,7 @@ suite ("test_substr") { qt_select_mv "SELECT substr(created_at,1,10) as statistic_date, max(dt) as dt FROM dwd group by substr(created_at,1,10);" sql """set enable_stats=true;""" + sql """alter table dwd modify column id set stats ('row_count'='2');""" mv_rewrite_success("SELECT substr(created_at,1,10) as statistic_date, max(dt) as dt FROM dwd group by substr(created_at,1,10);", "dwd_mv") } diff --git a/regression-test/suites/mv_p0/test_tbl_name/test_tbl_name.groovy b/regression-test/suites/mv_p0/test_tbl_name/test_tbl_name.groovy index f65e36484e7de9..bbfa8ea6a54d17 100644 --- a/regression-test/suites/mv_p0/test_tbl_name/test_tbl_name.groovy +++ b/regression-test/suites/mv_p0/test_tbl_name/test_tbl_name.groovy @@ -66,6 +66,7 @@ suite ("test_tbl_name") { group by id order by 1,2; """ sql """set enable_stats=true;""" + sql """alter table functionality_olap modify column id set stats ('row_count'='2');""" mv_rewrite_success("""select functionality_olap.id as id, sum(functionality_olap.score) as score_max diff --git a/regression-test/suites/mv_p0/test_upper_alias/test_upper_alias.groovy b/regression-test/suites/mv_p0/test_upper_alias/test_upper_alias.groovy index b62ec696224197..69264d838e4d24 100644 --- a/regression-test/suites/mv_p0/test_upper_alias/test_upper_alias.groovy +++ b/regression-test/suites/mv_p0/test_upper_alias/test_upper_alias.groovy @@ -62,6 +62,7 @@ suite ("test_upper_alias") { qt_select_mv "SELECT d_a AS d_b FROM test_0401 order by 1;" sql """set enable_stats=true;""" + sql """alter table test_0401 modify column d_b set stats ('row_count'='3');""" mv_rewrite_any_success("SELECT upper(d_b) AS d_b FROM test_0401 GROUP BY upper(d_b) order by 1;", ["test_0401_mv", "test_0401_mv2"]) diff --git a/regression-test/suites/mv_p0/test_user_activity/test_user_activity.groovy b/regression-test/suites/mv_p0/test_user_activity/test_user_activity.groovy index 3d139cbc01bbe7..f971673971a2aa 100644 --- a/regression-test/suites/mv_p0/test_user_activity/test_user_activity.groovy +++ b/regression-test/suites/mv_p0/test_user_activity/test_user_activity.groovy @@ -53,6 +53,7 @@ suite ("test_user_activity") { qt_select_group_mv "select n_dx, percentile_approx(n_duration, 0.5) as p50, percentile_approx(n_duration, 0.90) as p90 FROM u_axx GROUP BY n_dx;" sql """set enable_stats=true;""" + sql """alter table u_axx modify column r_xx set stats ('row_count'='3');""" mv_rewrite_success("select n_dx, percentile_approx(n_duration, 0.5) as p50, percentile_approx(n_duration, 0.90) as p90 FROM u_axx GROUP BY n_dx;", "session_distribution_2") } diff --git a/regression-test/suites/mv_p0/unique/unique.groovy b/regression-test/suites/mv_p0/unique/unique.groovy index e2a78c60bc2b0f..d054803678da4f 100644 --- a/regression-test/suites/mv_p0/unique/unique.groovy +++ b/regression-test/suites/mv_p0/unique/unique.groovy @@ -73,6 +73,7 @@ suite ("unique") { qt_select_star "select * from u_table order by k1;" sql """set enable_stats=true;""" + sql """alter table u_table modify column k1 set stats ('row_count'='3');""" mv_rewrite_success("select k3,length(k1),k2 from u_table order by 1,2,3;", "k31l42") // todo: support match query diff --git a/regression-test/suites/mv_p0/ut/testAggQueryOnAggMV1/testAggQueryOnAggMV1.groovy b/regression-test/suites/mv_p0/ut/testAggQueryOnAggMV1/testAggQueryOnAggMV1.groovy index 70b016de2da80f..f39e8df6cabe6a 100644 --- a/regression-test/suites/mv_p0/ut/testAggQueryOnAggMV1/testAggQueryOnAggMV1.groovy +++ b/regression-test/suites/mv_p0/ut/testAggQueryOnAggMV1/testAggQueryOnAggMV1.groovy @@ -36,6 +36,7 @@ suite ("testAggQueryOnAggMV1") { sql """insert into emps values("2020-01-02",2,"b",2,2,2);""" sql """insert into emps values("2020-01-03",3,"c",3,3,3);""" +sql """alter table emps modify column time_col set stats ('row_count'='4');""" createMV("create materialized view emps_mv as select deptno, sum(salary), max(commission) from emps group by deptno;") createMV("create materialized view emps_mv_count_key as select deptno, count(deptno) from emps group by deptno;") diff --git a/regression-test/suites/mv_p0/ut/testAggQueryOnAggMV10/testAggQueryOnAggMV10.groovy b/regression-test/suites/mv_p0/ut/testAggQueryOnAggMV10/testAggQueryOnAggMV10.groovy index a25ac24404e181..739dde33372cb6 100644 --- a/regression-test/suites/mv_p0/ut/testAggQueryOnAggMV10/testAggQueryOnAggMV10.groovy +++ b/regression-test/suites/mv_p0/ut/testAggQueryOnAggMV10/testAggQueryOnAggMV10.groovy @@ -51,6 +51,7 @@ suite ("testAggQueryOnAggMV10") { qt_select_mv "select deptno, commission, sum(salary) + 1 from emps group by rollup (deptno, commission) order by 1,2;" sql """set enable_stats=true;""" + sql """alter table emps modify column time_col set stats ('row_count'='4');""" mv_rewrite_fail("select * from emps order by empid;", "emps_mv") mv_rewrite_success("select deptno, commission, sum(salary) + 1 from emps group by rollup (deptno, commission);", diff --git a/regression-test/suites/mv_p0/ut/testAggQueryOnAggMV11/testAggQueryOnAggMV11.groovy b/regression-test/suites/mv_p0/ut/testAggQueryOnAggMV11/testAggQueryOnAggMV11.groovy index c26db8e034e66a..a4d69136bc03ca 100644 --- a/regression-test/suites/mv_p0/ut/testAggQueryOnAggMV11/testAggQueryOnAggMV11.groovy +++ b/regression-test/suites/mv_p0/ut/testAggQueryOnAggMV11/testAggQueryOnAggMV11.groovy @@ -50,6 +50,7 @@ suite ("testAggQueryOnAggMV11") { qt_select_mv "select deptno, count(salary) + count(1) from emps group by deptno order by 1;" sql """set enable_stats=true;""" + sql """alter table emps modify column time_col set stats ('row_count'='4');""" mv_rewrite_fail("select * from emps order by empid;", "emps_mv") mv_rewrite_fail("select deptno, count(salary) + count(1) from emps group by deptno;", "emps_mv") diff --git a/regression-test/suites/mv_p0/ut/testAggQueryOnAggMV2/testAggQueryOnAggMV2.groovy b/regression-test/suites/mv_p0/ut/testAggQueryOnAggMV2/testAggQueryOnAggMV2.groovy index 945e4e9194910f..5cf7d257fe8b73 100644 --- a/regression-test/suites/mv_p0/ut/testAggQueryOnAggMV2/testAggQueryOnAggMV2.groovy +++ b/regression-test/suites/mv_p0/ut/testAggQueryOnAggMV2/testAggQueryOnAggMV2.groovy @@ -53,6 +53,7 @@ suite ("testAggQueryOnAggMV2") { qt_select_mv "select * from (select deptno, sum(salary) as sum_salary from emps group by deptno) a where (sum_salary * 2) > 3 order by deptno ;" sql """set enable_stats=true;""" + sql """alter table emps modify column time_col set stats ('row_count'='3');""" mv_rewrite_fail("select * from emps order by empid;", "emps_mv") mv_rewrite_success("select * from (select deptno, sum(salary) as sum_salary from emps group by deptno) a where (sum_salary * 2) > 3 order by deptno ;", diff --git a/regression-test/suites/mv_p0/ut/testAggQueryOnAggMV3/testAggQueryOnAggMV3.groovy b/regression-test/suites/mv_p0/ut/testAggQueryOnAggMV3/testAggQueryOnAggMV3.groovy index 46457f0d0a81d4..d2f61e04762b2f 100644 --- a/regression-test/suites/mv_p0/ut/testAggQueryOnAggMV3/testAggQueryOnAggMV3.groovy +++ b/regression-test/suites/mv_p0/ut/testAggQueryOnAggMV3/testAggQueryOnAggMV3.groovy @@ -38,7 +38,6 @@ suite ("testAggQueryOnAggMV3") { sql """insert into emps values("2020-01-04",4,"d",21,4,4);""" - createMV("create materialized view emps_mv as select deptno, commission, sum(salary) from emps group by deptno, commission;") sql "analyze table emps with sync;" @@ -56,6 +55,7 @@ suite ("testAggQueryOnAggMV3") { qt_select_mv "select commission, sum(salary) from emps where commission = 100 group by commission order by commission;" sql """set enable_stats=true;""" + sql """alter table emps modify column time_col set stats ('row_count'='4');""" mv_rewrite_fail("select * from emps order by empid;", "emps_mv") mv_rewrite_success("select commission, sum(salary) from emps where deptno > 0 and commission * (deptno + commission) = 100 group by commission order by commission;", diff --git a/regression-test/suites/mv_p0/ut/testAggQuqeryOnAggMV5/testAggQuqeryOnAggMV5.groovy b/regression-test/suites/mv_p0/ut/testAggQuqeryOnAggMV5/testAggQuqeryOnAggMV5.groovy index 1fa4342952cad4..1f14728e3b7b2c 100644 --- a/regression-test/suites/mv_p0/ut/testAggQuqeryOnAggMV5/testAggQuqeryOnAggMV5.groovy +++ b/regression-test/suites/mv_p0/ut/testAggQuqeryOnAggMV5/testAggQuqeryOnAggMV5.groovy @@ -51,6 +51,7 @@ suite ("testAggQuqeryOnAggMV5") { qt_select_mv "select * from (select deptno, sum(salary) as sum_salary from emps group by deptno) a where sum_salary>10 order by 1;" sql """set enable_stats=true;""" + sql """alter table emps modify column time_col set stats ('row_count'='4');""" mv_rewrite_fail("select * from emps order by empid;", "emps_mv") mv_rewrite_success("select * from (select deptno, sum(salary) as sum_salary from emps group by deptno) a where sum_salary>0;", diff --git a/regression-test/suites/mv_p0/ut/testAggQuqeryOnAggMV6/testAggQuqeryOnAggMV6.groovy b/regression-test/suites/mv_p0/ut/testAggQuqeryOnAggMV6/testAggQuqeryOnAggMV6.groovy index 3cca0273051f0a..4cafeaa185f4a3 100644 --- a/regression-test/suites/mv_p0/ut/testAggQuqeryOnAggMV6/testAggQuqeryOnAggMV6.groovy +++ b/regression-test/suites/mv_p0/ut/testAggQuqeryOnAggMV6/testAggQuqeryOnAggMV6.groovy @@ -51,6 +51,7 @@ suite ("testAggQuqeryOnAggMV6") { qt_select_mv "select * from (select deptno, sum(salary) as sum_salary from emps where deptno>=20 group by deptno) a where sum_salary>10 order by 1;" sql """set enable_stats=true;""" + sql """alter table emps modify column time_col set stats ('row_count'='4');""" mv_rewrite_fail("select * from emps order by empid;", "emps_mv") mv_rewrite_success("select * from (select deptno, sum(salary) as sum_salary from emps where deptno>=0 group by deptno) a where sum_salary>10;", diff --git a/regression-test/suites/mv_p0/ut/testAggQuqeryOnAggMV7/testAggQuqeryOnAggMV7.groovy b/regression-test/suites/mv_p0/ut/testAggQuqeryOnAggMV7/testAggQuqeryOnAggMV7.groovy index 1a606141e38f04..d43b5796e6d7f9 100644 --- a/regression-test/suites/mv_p0/ut/testAggQuqeryOnAggMV7/testAggQuqeryOnAggMV7.groovy +++ b/regression-test/suites/mv_p0/ut/testAggQuqeryOnAggMV7/testAggQuqeryOnAggMV7.groovy @@ -50,6 +50,8 @@ suite ("testAggQuqeryOnAggMV7") { qt_select_mv "select deptno, sum(salary) from emps where deptno>=20 group by deptno order by 1;" sql """set enable_stats=true;""" + sql """alter table emps modify column time_col set stats ('row_count'='4');""" + sql """alter table emps modify column time_col set stats ('row_count'='4');""" mv_rewrite_fail("select * from emps order by empid;", "emps_mv") mv_rewrite_success("select deptno, sum(salary) from emps where deptno>=20 group by deptno;", "emps_mv") diff --git a/regression-test/suites/mv_p0/ut/testAggTableCountDistinctInBitmapType/testAggTableCountDistinctInBitmapType.groovy b/regression-test/suites/mv_p0/ut/testAggTableCountDistinctInBitmapType/testAggTableCountDistinctInBitmapType.groovy index 7e8199b9932e72..26f1fdb0219a2d 100644 --- a/regression-test/suites/mv_p0/ut/testAggTableCountDistinctInBitmapType/testAggTableCountDistinctInBitmapType.groovy +++ b/regression-test/suites/mv_p0/ut/testAggTableCountDistinctInBitmapType/testAggTableCountDistinctInBitmapType.groovy @@ -42,6 +42,7 @@ suite ("testAggTableCountDistinctInBitmapType") { qt_select_mv "select k1, count(distinct v1) from test_tb group by k1 order by k1;" sql """set enable_stats=true;""" + sql """alter table test_tb modify column k1 set stats ('row_count'='3');""" explain { sql("select k1, count(distinct v1) from test_tb group by k1;") contains "bitmap_union_count" diff --git a/regression-test/suites/mv_p0/ut/testAggregateMVCalcAggFunctionQuery/testAggregateMVCalcAggFunctionQuery.groovy b/regression-test/suites/mv_p0/ut/testAggregateMVCalcAggFunctionQuery/testAggregateMVCalcAggFunctionQuery.groovy index 7f2427e275de47..6dc78784956d32 100644 --- a/regression-test/suites/mv_p0/ut/testAggregateMVCalcAggFunctionQuery/testAggregateMVCalcAggFunctionQuery.groovy +++ b/regression-test/suites/mv_p0/ut/testAggregateMVCalcAggFunctionQuery/testAggregateMVCalcAggFunctionQuery.groovy @@ -50,6 +50,7 @@ suite ("testAggregateMVCalcAggFunctionQuery") { qt_select_mv "select deptno, sum(salary + 1) from emps where deptno > 10 group by deptno order by deptno;" sql """set enable_stats=true;""" + sql """alter table emps modify column time_col set stats ('row_count'='4');""" mv_rewrite_fail("select * from emps order by empid;", "emps_mv") mv_rewrite_fail("select deptno, sum(salary + 1) from emps where deptno > 10 group by deptno;", "emps_mv") diff --git a/regression-test/suites/mv_p0/ut/testBitmapUnionInQuery/testBitmapUnionInQuery.groovy b/regression-test/suites/mv_p0/ut/testBitmapUnionInQuery/testBitmapUnionInQuery.groovy index 69410ae7be5d0d..c1287bfa94bcc4 100644 --- a/regression-test/suites/mv_p0/ut/testBitmapUnionInQuery/testBitmapUnionInQuery.groovy +++ b/regression-test/suites/mv_p0/ut/testBitmapUnionInQuery/testBitmapUnionInQuery.groovy @@ -51,7 +51,7 @@ suite ("testBitmapUnionInQuery") { qt_select_mv "select user_id, bitmap_count(bitmap_union(to_bitmap(tag_id))) a from user_tags group by user_id having a>1 order by a;" sql """set enable_stats=true;""" - + sql """alter table user_tags modify column time_col set stats ('row_count'='3');""" mv_rewrite_fail("select * from user_tags order by time_col;", "user_tags_mv") mv_rewrite_success("select user_id, bitmap_union_count(to_bitmap(tag_id)) a from user_tags group by user_id having a>1 order by a;", diff --git a/regression-test/suites/mv_p0/ut/testCountDistinctToBitmap/testCountDistinctToBitmap.groovy b/regression-test/suites/mv_p0/ut/testCountDistinctToBitmap/testCountDistinctToBitmap.groovy index 12152aad369e1d..5cc05db62be1a7 100644 --- a/regression-test/suites/mv_p0/ut/testCountDistinctToBitmap/testCountDistinctToBitmap.groovy +++ b/regression-test/suites/mv_p0/ut/testCountDistinctToBitmap/testCountDistinctToBitmap.groovy @@ -66,6 +66,10 @@ suite ("testCountDistinctToBitmap") { sql """insert into user_tags2 values("2020-01-01",1,"a",1);""" sql """insert into user_tags2 values("2020-01-02",2,"b",2);""" + + sql """alter table user_tags modify column time_col set stats ('row_count'='3');""" + sql """alter table user_tags2 modify column time_col set stats ('row_count'='3');""" + createMV("create materialized view user_tags_mv as select user_id, bitmap_union(to_bitmap(tag_id)) from user_tags2 group by user_id;") sql """insert into user_tags2 values("2020-01-01",1,"a",2);""" diff --git a/regression-test/suites/mv_p0/ut/testIncorrectMVRewriteInSubquery/testIncorrectMVRewriteInSubquery.groovy b/regression-test/suites/mv_p0/ut/testIncorrectMVRewriteInSubquery/testIncorrectMVRewriteInSubquery.groovy index c671e106a10648..698490f0943e88 100644 --- a/regression-test/suites/mv_p0/ut/testIncorrectMVRewriteInSubquery/testIncorrectMVRewriteInSubquery.groovy +++ b/regression-test/suites/mv_p0/ut/testIncorrectMVRewriteInSubquery/testIncorrectMVRewriteInSubquery.groovy @@ -47,6 +47,7 @@ suite ("testIncorrectMVRewriteInSubquery") { qt_select_mv "select user_id, bitmap_union(to_bitmap(tag_id)) from user_tags where user_name in (select user_name from user_tags group by user_name having bitmap_union_count(to_bitmap(tag_id)) >1 ) group by user_id order by user_id;" sql """set enable_stats=true;""" + sql """alter table user_tags modify column time_col set stats ('row_count'='3');""" mv_rewrite_fail("select * from user_tags order by time_col;", "user_tags_mv") mv_rewrite_fail("select user_id, bitmap_union(to_bitmap(tag_id)) from user_tags where user_name in (select user_name from user_tags group by user_name having bitmap_union_count(to_bitmap(tag_id)) >1 ) group by user_id order by user_id;", diff --git a/regression-test/suites/mv_p0/ut/testIncorrectRewriteCountDistinct/testIncorrectRewriteCountDistinct.groovy b/regression-test/suites/mv_p0/ut/testIncorrectRewriteCountDistinct/testIncorrectRewriteCountDistinct.groovy index dbece0c9592364..45230c5b5587d8 100644 --- a/regression-test/suites/mv_p0/ut/testIncorrectRewriteCountDistinct/testIncorrectRewriteCountDistinct.groovy +++ b/regression-test/suites/mv_p0/ut/testIncorrectRewriteCountDistinct/testIncorrectRewriteCountDistinct.groovy @@ -45,6 +45,7 @@ suite ("testIncorrectRewriteCountDistinct") { qt_select_mv "select user_name, count(distinct tag_id) from user_tags group by user_name order by user_name;" sql """set enable_stats=true;""" + sql """alter table user_tags modify column time_col set stats ('row_count'='3');""" mv_rewrite_fail("select * from user_tags order by time_col;", "user_tags_mv") mv_rewrite_fail("select user_name, count(distinct tag_id) from user_tags group by user_name;", "user_tags_mv") diff --git a/regression-test/suites/mv_p0/ut/testJoinOnLeftProjectToJoin/testJoinOnLeftProjectToJoin.groovy b/regression-test/suites/mv_p0/ut/testJoinOnLeftProjectToJoin/testJoinOnLeftProjectToJoin.groovy index 3caf0000c1824d..8f41fc1ffb5e74 100644 --- a/regression-test/suites/mv_p0/ut/testJoinOnLeftProjectToJoin/testJoinOnLeftProjectToJoin.groovy +++ b/regression-test/suites/mv_p0/ut/testJoinOnLeftProjectToJoin/testJoinOnLeftProjectToJoin.groovy @@ -43,6 +43,8 @@ suite ("testJoinOnLeftProjectToJoin") { partition by range (time_col) (partition p1 values less than MAXVALUE) distributed by hash(time_col) buckets 3 properties('replication_num' = '1'); """ + sql """alter table depts modify column time_col set stats ('row_count'='3');""" + sql """insert into depts values("2020-01-02",2,"b",2);""" sql """insert into depts values("2020-01-03",3,"c",3);""" sql """insert into depts values("2020-01-02",2,"b",1);""" @@ -59,6 +61,7 @@ suite ("testJoinOnLeftProjectToJoin") { qt_select_mv "select * from (select deptno , sum(salary) from emps group by deptno) A join (select deptno, max(cost) from depts group by deptno ) B on A.deptno = B.deptno order by A.deptno;" sql """set enable_stats=true;""" + sql """alter table emps modify column time_col set stats ('row_count'='3');""" mv_rewrite_all_success("select * from (select deptno , sum(salary) from emps group by deptno) A join (select deptno, max(cost) from depts group by deptno ) B on A.deptno = B.deptno;", ["emps_mv", "depts_mv"]) } diff --git a/regression-test/suites/mv_p0/ut/testNDVToHll/testNDVToHll.groovy b/regression-test/suites/mv_p0/ut/testNDVToHll/testNDVToHll.groovy index a9456cb4e79c96..c77f58609ca0cb 100644 --- a/regression-test/suites/mv_p0/ut/testNDVToHll/testNDVToHll.groovy +++ b/regression-test/suites/mv_p0/ut/testNDVToHll/testNDVToHll.groovy @@ -50,6 +50,7 @@ suite ("testNDVToHll") { qt_select_mv "select user_id, approx_count_distinct(tag_id) a from user_tags group by user_id order by user_id;" sql """set enable_stats=true;""" + sql """alter table user_tags modify column time_col set stats ('row_count'='3');""" mv_rewrite_fail("select * from user_tags order by time_col;", "user_tags_mv") mv_rewrite_success("select user_id, ndv(tag_id) a from user_tags group by user_id order by user_id;", "user_tags_mv") diff --git a/regression-test/suites/mv_p0/ut/testOrderByQueryOnProjectView/testOrderByQueryOnProjectView.groovy b/regression-test/suites/mv_p0/ut/testOrderByQueryOnProjectView/testOrderByQueryOnProjectView.groovy index 2aa88d84f3d433..41cf480e880cd0 100644 --- a/regression-test/suites/mv_p0/ut/testOrderByQueryOnProjectView/testOrderByQueryOnProjectView.groovy +++ b/regression-test/suites/mv_p0/ut/testOrderByQueryOnProjectView/testOrderByQueryOnProjectView.groovy @@ -49,6 +49,7 @@ suite ("testOrderByQueryOnProjectView") { qt_select_mv "select empid from emps order by deptno;" sql """set enable_stats=true;""" + sql """alter table emps modify column time_col set stats ('row_count'='4');""" mv_rewrite_fail("select * from emps order by empid;", "emps_mv") mv_rewrite_success("select empid from emps where deptno > 0 order by deptno;", "emps_mv") diff --git a/regression-test/suites/mv_p0/ut/testProjectionMV1/testProjectionMV1.groovy b/regression-test/suites/mv_p0/ut/testProjectionMV1/testProjectionMV1.groovy index d95c3e2fc6de88..d578cbddba945d 100644 --- a/regression-test/suites/mv_p0/ut/testProjectionMV1/testProjectionMV1.groovy +++ b/regression-test/suites/mv_p0/ut/testProjectionMV1/testProjectionMV1.groovy @@ -61,6 +61,7 @@ suite ("testProjectionMV1") { qt_select_mv "select deptno, sum(empid) from emps group by deptno order by deptno;" sql """set enable_stats=true;""" + sql """alter table emps modify column time_col set stats ('row_count'='3');""" mv_rewrite_fail("select * from emps order by empid;", "emps_mv") mv_rewrite_success("select empid, deptno from emps where deptno > 0 order by empid;", "emps_mv") diff --git a/regression-test/suites/mv_p0/ut/testProjectionMV2/testProjectionMV2.groovy b/regression-test/suites/mv_p0/ut/testProjectionMV2/testProjectionMV2.groovy index 2f7548b14c8635..b663bfcd2b1738 100644 --- a/regression-test/suites/mv_p0/ut/testProjectionMV2/testProjectionMV2.groovy +++ b/regression-test/suites/mv_p0/ut/testProjectionMV2/testProjectionMV2.groovy @@ -51,6 +51,7 @@ suite ("testProjectionMV2") { qt_select_base "select name from emps where deptno -1 = 0 order by empid;" sql """set enable_stats=true;""" + sql """alter table emps modify column time_col set stats ('row_count'='3');""" mv_rewrite_fail("select * from emps order by empid;", "emps_mv") mv_rewrite_success("select empid + 1 from emps where deptno = 1 order by empid;", "emps_mv") diff --git a/regression-test/suites/mv_p0/ut/testProjectionMV3/testProjectionMV3.groovy b/regression-test/suites/mv_p0/ut/testProjectionMV3/testProjectionMV3.groovy index e4ea1aa4de023e..378ec80d6cfd72 100644 --- a/regression-test/suites/mv_p0/ut/testProjectionMV3/testProjectionMV3.groovy +++ b/regression-test/suites/mv_p0/ut/testProjectionMV3/testProjectionMV3.groovy @@ -53,6 +53,7 @@ suite ("testProjectionMV3") { qt_select_mv2 "select name from emps where deptno = 1 order by empid;" sql """set enable_stats=true;""" + sql """alter table emps modify column time_col set stats ('row_count'='3');""" mv_rewrite_fail("select * from emps order by empid;", "emps_mv") mv_rewrite_success("select empid + 1, name from emps where deptno = 1 order by empid;", "emps_mv") diff --git a/regression-test/suites/mv_p0/ut/testProjectionMV4/testProjectionMV4.groovy b/regression-test/suites/mv_p0/ut/testProjectionMV4/testProjectionMV4.groovy index 0c6f737deabb29..37a57f80d03cae 100644 --- a/regression-test/suites/mv_p0/ut/testProjectionMV4/testProjectionMV4.groovy +++ b/regression-test/suites/mv_p0/ut/testProjectionMV4/testProjectionMV4.groovy @@ -52,6 +52,7 @@ suite ("testProjectionMV4") { qt_select_base "select empid from emps where deptno > 1 and empid > 1 order by empid;" sql """set enable_stats=true;""" + sql """alter table emps modify column time_col set stats ('row_count'='3');""" mv_rewrite_fail("select * from emps order by empid;", "emps_mv") mv_rewrite_fail("select empid from emps where deptno > 1 and empid > 1 order by empid;", "emps_mv") diff --git a/regression-test/suites/mv_p0/ut/testQueryOnStar/testQueryOnStar.groovy b/regression-test/suites/mv_p0/ut/testQueryOnStar/testQueryOnStar.groovy index d81129d520729f..7d563883514f72 100644 --- a/regression-test/suites/mv_p0/ut/testQueryOnStar/testQueryOnStar.groovy +++ b/regression-test/suites/mv_p0/ut/testQueryOnStar/testQueryOnStar.groovy @@ -58,4 +58,5 @@ suite ("testQueryOnStar") { sql """insert into tpch_tiny_region values(1,'a','a');""" qt_select_mv "select ref_1.`empid` as c0 from tpch_tiny_region as ref_0 left join emps as ref_1 on (ref_0.`r_comment` = ref_1.`name` ) where true order by ref_0.`r_regionkey`,ref_0.`r_regionkey` desc ,ref_0.`r_regionkey`,ref_0.`r_regionkey`;" + } diff --git a/regression-test/suites/mv_p0/ut/testSelectMVWithTableAlias/testSelectMVWithTableAlias.groovy b/regression-test/suites/mv_p0/ut/testSelectMVWithTableAlias/testSelectMVWithTableAlias.groovy index 0cfd80fcb06b4e..9e3bd886b31ea5 100644 --- a/regression-test/suites/mv_p0/ut/testSelectMVWithTableAlias/testSelectMVWithTableAlias.groovy +++ b/regression-test/suites/mv_p0/ut/testSelectMVWithTableAlias/testSelectMVWithTableAlias.groovy @@ -29,6 +29,8 @@ suite ("testSelectMVWithTableAlias") { partition by range (time_col) (partition p1 values less than MAXVALUE) distributed by hash(time_col) buckets 3 properties('replication_num' = '1'); """ + sql """alter table user_tags modify column time_col set stats ('row_count'='3');""" + sql """insert into user_tags values("2020-01-01",1,"a",1);""" sql """insert into user_tags values("2020-01-02",2,"b",2);""" diff --git a/regression-test/suites/mv_p0/ut/testSingleMVMultiUsage/testSingleMVMultiUsage.groovy b/regression-test/suites/mv_p0/ut/testSingleMVMultiUsage/testSingleMVMultiUsage.groovy index 006b72885ec112..73ddb21109bb91 100644 --- a/regression-test/suites/mv_p0/ut/testSingleMVMultiUsage/testSingleMVMultiUsage.groovy +++ b/regression-test/suites/mv_p0/ut/testSingleMVMultiUsage/testSingleMVMultiUsage.groovy @@ -52,6 +52,7 @@ suite ("testSingleMVMultiUsage") { } qt_select_mv "select * from (select deptno, empid from emps where deptno>100) A join (select deptno, empid from emps where deptno >200) B using (deptno) order by 1;" sql """set enable_stats=true;""" + sql """alter table emps modify column time_col set stats ('row_count'='4');""" mv_rewrite_fail("select * from emps order by empid;", "emps_mv") explain { diff --git a/regression-test/suites/mv_p0/ut/testSubQuery/testSubQuery.groovy b/regression-test/suites/mv_p0/ut/testSubQuery/testSubQuery.groovy index 6c1747059d1ae3..cd134a5d48ea40 100644 --- a/regression-test/suites/mv_p0/ut/testSubQuery/testSubQuery.groovy +++ b/regression-test/suites/mv_p0/ut/testSubQuery/testSubQuery.groovy @@ -38,7 +38,6 @@ suite ("testSubQuery") { sql """insert into emps values("2020-01-02",2,"b",2,2,2);""" sql """insert into emps values("2020-01-03",3,"c",3,3,3);""" - createMV("create materialized view emps_mv as select deptno, empid from emps;") sql """insert into emps values("2020-01-01",1,"a",1,1,1);""" @@ -52,5 +51,6 @@ suite ("testSubQuery") { qt_select_mv "select empid, deptno, salary from emps e1 where empid = (select max(empid) from emps where deptno = e1.deptno) order by deptno;" sql """set enable_stats=true;""" + sql """alter table emps modify column time_col set stats ('row_count'='4');""" mv_rewrite_fail("select * from emps order by empid;", "emps_mv") } diff --git a/regression-test/suites/mv_p0/ut/testUnionDistinct/testUnionDistinct.groovy b/regression-test/suites/mv_p0/ut/testUnionDistinct/testUnionDistinct.groovy index f504c81f214e91..c6243a3f923616 100644 --- a/regression-test/suites/mv_p0/ut/testUnionDistinct/testUnionDistinct.groovy +++ b/regression-test/suites/mv_p0/ut/testUnionDistinct/testUnionDistinct.groovy @@ -35,6 +35,7 @@ suite ("testUnionDistinct") { sql """insert into emps values("2020-01-02",2,"b",2,2,2);""" sql """insert into emps values("2020-01-03",3,"c",3,3,3);""" + createMV("create materialized view emps_mv as select empid, deptno from emps order by empid, deptno;") sql """insert into emps values("2020-01-01",1,"a",1,1,1);""" @@ -52,6 +53,7 @@ suite ("testUnionDistinct") { } qt_select_mv "select * from (select empid, deptno from emps where empid >1 union select empid, deptno from emps where empid <0) t order by 1;" sql """set enable_stats=true;""" + sql """alter table emps modify column time_col set stats ('row_count'='4');""" mv_rewrite_fail("select * from emps order by empid;", "emps_mv") explain { diff --git a/regression-test/suites/nereids_rules_p0/mv/agg_on_none_agg/agg_on_none_agg.groovy b/regression-test/suites/nereids_rules_p0/mv/agg_on_none_agg/agg_on_none_agg.groovy index d10498705f60c6..db72f1174fdff2 100644 --- a/regression-test/suites/nereids_rules_p0/mv/agg_on_none_agg/agg_on_none_agg.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/agg_on_none_agg/agg_on_none_agg.groovy @@ -57,6 +57,8 @@ suite("agg_on_none_agg") { (5, 2, 'o', 1.2, '2023-12-12', 'c','d',2, 'mi'); """ + sql """alter table orders modify column O_COMMENT set stats ('row_count'='8');""" + sql """ drop table if exists lineitem """ @@ -95,6 +97,8 @@ suite("agg_on_none_agg") { (5, 2, 3, 6, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-12-12', '2023-12-12', '2023-12-13', 'c', 'd', 'xxxxxxxxx'); """ + sql """alter table lineitem modify column l_comment set stats ('row_count'='5');""" + sql """ drop table if exists partsupp """ @@ -120,6 +124,8 @@ suite("agg_on_none_agg") { (2, 3, 10, 11.01, 'supply2'); """ + sql """alter table partsupp modify column ps_comment set stats ('row_count'='2');""" + sql """analyze table orders with sync;""" sql """analyze table lineitem with sync;""" sql """analyze table partsupp with sync;""" diff --git a/regression-test/suites/nereids_rules_p0/mv/agg_optimize_when_uniform/agg_optimize_when_uniform.groovy b/regression-test/suites/nereids_rules_p0/mv/agg_optimize_when_uniform/agg_optimize_when_uniform.groovy index 355165055c8ef5..e0f33a2b0db8e5 100644 --- a/regression-test/suites/nereids_rules_p0/mv/agg_optimize_when_uniform/agg_optimize_when_uniform.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/agg_optimize_when_uniform/agg_optimize_when_uniform.groovy @@ -120,6 +120,10 @@ suite("agg_optimize_when_uniform") { (2, 3, 10, 11.01, 'supply2'); """ + sql """alter table lineitem modify column l_comment set stats ('row_count'='5');""" + sql """alter table orders modify column O_COMMENT set stats ('row_count'='8');""" + sql """alter table partsupp modify column ps_comment set stats ('row_count'='2');""" + sql """analyze table lineitem with sync;""" sql """analyze table orders with sync;""" sql """analyze table partsupp with sync;""" diff --git a/regression-test/suites/nereids_rules_p0/mv/agg_variety/agg_variety.groovy b/regression-test/suites/nereids_rules_p0/mv/agg_variety/agg_variety.groovy index 3fa6dffca01367..2de47b4bbc2f8e 100644 --- a/regression-test/suites/nereids_rules_p0/mv/agg_variety/agg_variety.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/agg_variety/agg_variety.groovy @@ -95,6 +95,8 @@ suite("agg_variety") { (5, 2, 3, 6, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-12-12', '2023-12-12', '2023-12-13', 'c', 'd', 'xxxxxxxxx'); """ + + sql """ drop table if exists partsupp """ @@ -124,6 +126,10 @@ suite("agg_variety") { sql """analyze table lineitem with sync;""" sql """analyze table partsupp with sync;""" + sql """alter table orders modify column O_COMMENT set stats ('row_count'='8');""" + sql """alter table lineitem modify column l_comment set stats ('row_count'='5');""" + sql """alter table partsupp modify column ps_comment set stats ('row_count'='2');""" + def check_rewrite_but_not_chose = { mv_sql, query_sql, mv_name -> sql """DROP MATERIALIZED VIEW IF EXISTS ${mv_name}""" diff --git a/regression-test/suites/nereids_rules_p0/mv/agg_with_roll_up/aggregate_with_roll_up.groovy b/regression-test/suites/nereids_rules_p0/mv/agg_with_roll_up/aggregate_with_roll_up.groovy index cda9b21fc4e51d..5b8aa6e00d2308 100644 --- a/regression-test/suites/nereids_rules_p0/mv/agg_with_roll_up/aggregate_with_roll_up.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/agg_with_roll_up/aggregate_with_roll_up.groovy @@ -132,6 +132,9 @@ suite("aggregate_with_roll_up") { sql """analyze table partsupp with sync""" sql """analyze table lineitem with sync""" sql """analyze table orders with sync""" + sql """alter table lineitem modify column l_comment set stats ('row_count'='5');""" + sql """alter table orders modify column O_COMMENT set stats ('row_count'='8');""" + sql """alter table partsupp modify column ps_comment set stats ('row_count'='2');""" // multi table // filter inside + left + use roll up dimension diff --git a/regression-test/suites/nereids_rules_p0/mv/agg_without_roll_up/aggregate_without_roll_up.groovy b/regression-test/suites/nereids_rules_p0/mv/agg_without_roll_up/aggregate_without_roll_up.groovy index 9d60280503cf74..c43bde59acaa3d 100644 --- a/regression-test/suites/nereids_rules_p0/mv/agg_without_roll_up/aggregate_without_roll_up.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/agg_without_roll_up/aggregate_without_roll_up.groovy @@ -121,6 +121,12 @@ suite("aggregate_without_roll_up") { (2, 3, 10, 11.01, 'supply2'); """ + sql """alter table lineitem modify column l_comment set stats ('row_count'='5');""" + + sql """alter table orders modify column o_comment set stats ('row_count'='8');""" + + sql """alter table partsupp modify column ps_comment set stats ('row_count'='2');""" + // single table // with filter def mv1_0 = """ diff --git a/regression-test/suites/nereids_rules_p0/mv/availability/grace_period.groovy b/regression-test/suites/nereids_rules_p0/mv/availability/grace_period.groovy index 5a6c75f6c59c6e..b9e671f2636b82 100644 --- a/regression-test/suites/nereids_rules_p0/mv/availability/grace_period.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/availability/grace_period.groovy @@ -49,6 +49,7 @@ suite("grace_period") { "replication_num" = "1" ); """ + sql """alter table orders_partition modify column o_comment set stats ('row_count'='3');""" sql """ drop table if exists lineitem_partition @@ -81,6 +82,7 @@ suite("grace_period") { "replication_num" = "1" ); """ + sql """alter table lineitem_partition modify column l_comment set stats ('row_count'='3');""" sql """ insert into orders_partition values diff --git a/regression-test/suites/nereids_rules_p0/mv/availability/materialized_view_switch.groovy b/regression-test/suites/nereids_rules_p0/mv/availability/materialized_view_switch.groovy index ae349c6514c469..4af1d778c002b2 100644 --- a/regression-test/suites/nereids_rules_p0/mv/availability/materialized_view_switch.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/availability/materialized_view_switch.groovy @@ -124,6 +124,11 @@ suite("materialized_view_switch") { analyze table orders with sync; analyze table partsupp with sync; """ + + sql """alter table lineitem modify column l_comment set stats ('row_count'='5');""" + sql """alter table orders modify column O_COMMENT set stats ('row_count'='8');""" + sql """alter table partsupp modify column ps_comment set stats ('row_count'='2');""" + def mv_name = """ select l_shipdate, o_orderdate, l_partkey, l_suppkey, o_orderkey from lineitem diff --git a/regression-test/suites/nereids_rules_p0/mv/create_part_and_up/range_date_datetrunc_part_up.groovy b/regression-test/suites/nereids_rules_p0/mv/create_part_and_up/range_date_datetrunc_part_up.groovy index c86f5d6a141f7f..a2fcb2eba15913 100644 --- a/regression-test/suites/nereids_rules_p0/mv/create_part_and_up/range_date_datetrunc_part_up.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/create_part_and_up/range_date_datetrunc_part_up.groovy @@ -71,6 +71,7 @@ suite("mtmv_range_date_datetrunc_date_part_up") { (1, 2, 3, null, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-17', '2023-10-17', 'a', 'b', 'yyyyyyyyy', '2023-10-29 03:00:00'), (2, 3, 2, 1, 5.5, 6.5, 7.5, 8.5, 'o', 'k', null, '2023-10-18', 'a', 'b', 'yyyyyyyyy', '2023-10-29 04:00:00'); """ + sql """alter table ${tb_name} modify column l_comment set stats ('row_count'='5');""" def compare_res = { def stmt -> sql "SET enable_materialized_view_rewrite=false" diff --git a/regression-test/suites/nereids_rules_p0/mv/create_part_and_up/range_date_part_up.groovy b/regression-test/suites/nereids_rules_p0/mv/create_part_and_up/range_date_part_up.groovy index 5be6b1afdcdc44..4c5b01ff59fd3a 100644 --- a/regression-test/suites/nereids_rules_p0/mv/create_part_and_up/range_date_part_up.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/create_part_and_up/range_date_part_up.groovy @@ -119,6 +119,8 @@ suite("mtmv_range_date_part_up") { (1, 3, 2, 2, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-17', '2023-10-17', 'a', 'b', 'yyyyyyyyy', '2023-10-29'); """ + sql """alter table lineitem_range_date modify column l_comment set stats ('row_count'='7');""" + def get_part = { def mv_name -> def part_res = sql """show partitions from ${mv_name}""" return part_res.size() diff --git a/regression-test/suites/nereids_rules_p0/mv/create_part_and_up/range_date_part_up_rewrite.groovy b/regression-test/suites/nereids_rules_p0/mv/create_part_and_up/range_date_part_up_rewrite.groovy index 8bf629d93841fb..88d8ad6ea2d84d 100644 --- a/regression-test/suites/nereids_rules_p0/mv/create_part_and_up/range_date_part_up_rewrite.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/create_part_and_up/range_date_part_up_rewrite.groovy @@ -113,6 +113,9 @@ suite("mtmv_range_date_part_up_rewrite") { (4, 5, 'k', 99.5, 'a', 'b', 1, 'yy', '2023-10-31'); """ + sql """alter table lineitem_range_date_union modify column l_comment set stats ('row_count'='7');""" + sql """alter table orders_range_date_union modify column o_comment set stats ('row_count'='10');""" + sql """DROP MATERIALIZED VIEW if exists ${mv_prefix}_mv1;""" sql """CREATE MATERIALIZED VIEW ${mv_prefix}_mv1 BUILD IMMEDIATE REFRESH AUTO ON MANUAL partition by(date_trunc(`col1`, 'month')) DISTRIBUTED BY RANDOM BUCKETS 2 PROPERTIES ('replication_num' = '1') AS select date_trunc(`l_shipdate`, 'day') as col1, l_shipdate, l_orderkey from lineitem_range_date_union as t1 left join orders_range_date_union as t2 on t1.l_orderkey = t2.o_orderkey group by col1, l_shipdate, l_orderkey;""" diff --git a/regression-test/suites/nereids_rules_p0/mv/create_part_and_up/range_datetime_part_up_rewrite.groovy b/regression-test/suites/nereids_rules_p0/mv/create_part_and_up/range_datetime_part_up_rewrite.groovy index b7e9562249d789..f8e601e64f5819 100644 --- a/regression-test/suites/nereids_rules_p0/mv/create_part_and_up/range_datetime_part_up_rewrite.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/create_part_and_up/range_datetime_part_up_rewrite.groovy @@ -98,6 +98,7 @@ suite("mtmv_range_datetime_part_up_rewrite") { (3, 1, 1, 2, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-10-19', null, 'c', 'd', 'xxxxxxxxx', '2023-10-29 02:00:00'), (1, 3, 2, 2, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-17', '2023-10-17', 'a', 'b', 'yyyyyyyyy', '2023-10-29 00:00:00'); """ + sql """alter table lineitem_range_datetime_union modify column l_comment set stats ('row_count'='7');""" sql """ insert into orders_range_datetime_union values @@ -112,6 +113,7 @@ suite("mtmv_range_datetime_part_up_rewrite") { (3, 2, 'k', 99.5, 'a', 'b', 1, 'yy', '2023-10-29 00:00:00'), (4, 5, 'k', 99.5, 'a', 'b', 1, 'yy', '2023-10-29 02:00:00'); """ + sql """alter table orders_range_datetime_union modify column o_comment set stats ('row_count'='10');""" sql """DROP MATERIALIZED VIEW if exists ${mv_prefix}_mv1;""" sql """CREATE MATERIALIZED VIEW ${mv_prefix}_mv1 BUILD IMMEDIATE REFRESH AUTO ON MANUAL partition by(date_trunc(`col1`, 'month')) DISTRIBUTED BY RANDOM BUCKETS 2 PROPERTIES ('replication_num' = '1') AS @@ -174,6 +176,7 @@ suite("mtmv_range_datetime_part_up_rewrite") { sql """alter table lineitem_range_datetime_union add partition p4 values [("2023-11-29 03:00:00"), ("2023-11-29 04:00:00"));""" sql """insert into lineitem_range_datetime_union values (1, null, 3, 1, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-18', '2023-10-18', 'a', 'b', 'yyyyyyyyy', '2023-11-29 03:00:00')""" + sql """alter table lineitem_range_datetime_union modify column l_comment set stats ('row_count'='8');""" for (int i = 0; i < mv_name_list.size(); i++) { mv_rewrite_success(query_stmt_list[i], mv_name_list[i]) compare_res(query_stmt_list[i] + " order by 1,2,3") @@ -187,6 +190,7 @@ suite("mtmv_range_datetime_part_up_rewrite") { sql """insert into lineitem_range_datetime_union values (3, null, 3, 1, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-18', '2023-10-18', 'a', 'b', 'yyyyyyyyy', '2023-11-29 03:00:00');""" + sql """alter table lineitem_range_datetime_union modify column l_comment set stats ('row_count'='9');""" for (int i = 0; i < mv_name_list.size(); i++) { mv_rewrite_success(query_stmt_list[i], mv_name_list[i]) compare_res(query_stmt_list[i] + " order by 1,2,3") diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_1.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_1.groovy index f2b5ad09c0b17a..3aed3b0f9e24df 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_1.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_1.groovy @@ -98,6 +98,9 @@ suite("partition_mv_rewrite_dimension_1") { (1, 3, 2, 2, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-17', '2023-10-17', 'a', 'b', 'yyyyyyyyy', '2023-10-17'); """ + sql """alter table orders_1 modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem_1 modify column l_comment set stats ('row_count'='7');""" + sql """analyze table orders_1 with sync;""" sql """analyze table lineitem_1 with sync;""" diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_3.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_3.groovy index 53f8777a1589d7..c7ee359cdef2e4 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_3.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_3.groovy @@ -102,6 +102,9 @@ suite("partition_mv_rewrite_dimension_2_3") { sql """analyze table orders_2_3 with sync;""" sql """analyze table lineitem_2_3 with sync;""" + sql """alter table orders_2_3 modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem_2_3 modify column l_comment set stats ('row_count'='7');""" + def compare_res = { def stmt -> sql "SET enable_materialized_view_rewrite=false" def origin_res = sql stmt diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_4.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_4.groovy index deb6255119a26e..e59b2771dd4e57 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_4.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_4.groovy @@ -124,6 +124,10 @@ suite("partition_mv_rewrite_dimension_2_4") { (3, null, 1, 99.5, 'yy'); """ + sql """alter table orders_2_4 modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem_2_4 modify column l_comment set stats ('row_count'='7');""" + sql """alter table partsupp_2_4 modify column ps_comment set stats ('row_count'='3');""" + sql """analyze table orders_2_4 with sync;""" sql """analyze table lineitem_2_4 with sync;""" sql """analyze table partsupp_2_4 with sync;""" diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_5.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_5.groovy index 5d8a6b5270285b..e5e43073bede3e 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_5.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_5.groovy @@ -124,6 +124,10 @@ suite("partition_mv_rewrite_dimension_2_5") { (3, null, 1, 99.5, 'yy'); """ + sql """alter table orders_2_5 modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem_2_5 modify column l_comment set stats ('row_count'='7');""" + sql """alter table partsupp_2_5 modify column ps_comment set stats ('row_count'='3');""" + sql """analyze table orders_2_5 with sync;""" sql """analyze table lineitem_2_5 with sync;""" sql """analyze table partsupp_2_5 with sync;""" diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_6.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_6.groovy index 1831e8de9a7a9d..f3289b007a0505 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_6.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_6.groovy @@ -124,6 +124,12 @@ suite("partition_mv_rewrite_dimension_2_6") { (3, null, 1, 99.5, 'yy'); """ + sql """alter table orders_2_6 modify column o_comment set stats ('row_count'='10');""" + + sql """alter table lineitem_2_6 modify column l_comment set stats ('row_count'='7');""" + + sql """alter table partsupp_2_6 modify column ps_comment set stats ('row_count'='3');""" + sql """analyze table orders_2_6 with sync;""" sql """analyze table lineitem_2_6 with sync;""" sql """analyze table partsupp_2_6 with sync;""" diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_full_join.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_full_join.groovy index 6def436e11c5c4..e36a5e277daca7 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_full_join.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_full_join.groovy @@ -102,6 +102,8 @@ suite("partition_mv_rewrite_dimension_2_full_join") { sql """analyze table orders_2_full_join with sync;""" sql """analyze table lineitem_2_full_join with sync;""" + sql """alter table orders_2_full_join modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem_2_full_join modify column l_comment set stats ('row_count'='7');""" def compare_res = { def stmt -> sql "SET enable_materialized_view_rewrite=false" diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_inner_join.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_inner_join.groovy index 526e05f0692fbe..a615c7316bdb56 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_inner_join.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_inner_join.groovy @@ -102,6 +102,9 @@ suite("partition_mv_rewrite_dimension_2_2") { sql """analyze table orders_2_2 with sync;""" sql """analyze table lineitem_2_2 with sync;""" + sql """alter table orders_2_2 modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem_2_2 modify column l_comment set stats ('row_count'='7');""" + def compare_res = { def stmt -> sql "SET enable_materialized_view_rewrite=false" def origin_res = sql stmt diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_left_anti_join.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_left_anti_join.groovy index 0db7b3bf008c68..35d48270574a6f 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_left_anti_join.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_left_anti_join.groovy @@ -102,6 +102,9 @@ suite("partition_mv_rewrite_dimension_2_left_anti_join") { sql """analyze table orders_2_left_anti_join with sync;""" sql """analyze table lineitem_2_left_anti_join with sync;""" + sql """alter table orders_2_left_anti_join modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem_2_left_anti_join modify column l_comment set stats ('row_count'='7');""" + def compare_res = { def stmt -> sql "SET enable_materialized_view_rewrite=false" def origin_res = sql stmt diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_left_join.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_left_join.groovy index f59dffd0091bb4..616707412954e4 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_left_join.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_left_join.groovy @@ -102,6 +102,9 @@ suite("partition_mv_rewrite_dimension_2_1") { sql """analyze table orders_2_1 with sync;""" sql """analyze table lineitem_2_1 with sync;""" + sql """alter table orders_2_1 modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem_2_1 modify column l_comment set stats ('row_count'='7');""" + def compare_res = { def stmt -> sql "SET enable_materialized_view_rewrite=false" def origin_res = sql stmt diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_left_semi_join.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_left_semi_join.groovy index 5e60250ef91d26..f203db1134f2f2 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_left_semi_join.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_left_semi_join.groovy @@ -102,6 +102,9 @@ suite("partition_mv_rewrite_dimension_2_left_semi_join") { sql """analyze table orders_2_left_semi_join with sync;""" sql """analyze table lineitem_2_left_semi_join with sync;""" + sql """alter table orders_2_left_semi_join modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem_2_left_semi_join modify column l_comment set stats ('row_count'='7');""" + def compare_res = { def stmt -> sql "SET enable_materialized_view_rewrite=false" def origin_res = sql stmt diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_right_anti_join.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_right_anti_join.groovy index 289ed57287f1d8..c6097d845f41c9 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_right_anti_join.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_right_anti_join.groovy @@ -102,6 +102,9 @@ suite("partition_mv_rewrite_dimension_2_right_anti_join") { sql """analyze table orders_2_right_anti_join with sync;""" sql """analyze table lineitem_2_right_anti_join with sync;""" + sql """alter table orders_2_right_anti_join modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem_2_right_anti_join modify column l_comment set stats ('row_count'='7');""" + def compare_res = { def stmt -> sql "SET enable_materialized_view_rewrite=false" def origin_res = sql stmt diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_right_join.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_right_join.groovy index 8d6abb6dc561f8..850de42fc696eb 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_right_join.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_right_join.groovy @@ -102,6 +102,9 @@ suite("partition_mv_rewrite_dimension_2_right_join") { sql """analyze table orders_2_right_join with sync;""" sql """analyze table lineitem_2_right_join with sync;""" + sql """alter table orders_2_right_join modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem_2_right_join modify column l_comment set stats ('row_count'='7');""" + def compare_res = { def stmt -> sql "SET enable_materialized_view_rewrite=false" def origin_res = sql stmt diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_right_semi_join.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_right_semi_join.groovy index 8f1f6f48a0ae91..97ddcd0eb1da29 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_right_semi_join.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_right_semi_join.groovy @@ -102,6 +102,9 @@ suite("partition_mv_rewrite_dimension_2_right_semi_join") { sql """analyze table orders_2_right_semi_join with sync;""" sql """analyze table lineitem_2_right_semi_join with sync;""" + sql """alter table orders_2_right_semi_join modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem_2_right_semi_join modify column l_comment set stats ('row_count'='7');""" + def compare_res = { def stmt -> sql "SET enable_materialized_view_rewrite=false" def origin_res = sql stmt diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_self_conn.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_self_conn.groovy index f433e18f48c062..776f1b996f1826 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_self_conn.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_self_conn.groovy @@ -101,6 +101,9 @@ suite("partition_mv_rewrite_dimension_self_conn") { sql """analyze table orders_self_conn with sync;""" sql """analyze table lineitem_self_conn with sync;""" + sql """alter table orders_self_conn modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem_self_conn modify column l_comment set stats ('row_count'='7');""" + def compare_res = { def stmt -> sql "SET enable_materialized_view_rewrite=false" def origin_res = sql stmt diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension_2_join_agg/dimension_2_join_agg.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension_2_join_agg/dimension_2_join_agg.groovy index dee43f20f4926a..fd41789d02661e 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension_2_join_agg/dimension_2_join_agg.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension_2_join_agg/dimension_2_join_agg.groovy @@ -101,6 +101,9 @@ suite("dimension_2_join_agg_replenish") { sql """analyze table orders with sync;""" sql """analyze table lineitem with sync;""" + sql """alter table orders modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem modify column l_comment set stats ('row_count'='7');""" + def compare_res = { def stmt -> sql "SET enable_materialized_view_rewrite=false" def origin_res = sql stmt @@ -649,19 +652,19 @@ suite("dimension_2_join_agg_replenish") { """ def right_semi_mv_stmt_8 = """select t1.l_orderkey, t1.l_shipdate, t1.l_partkey from orders - right semi join (select l_orderkey, l_partkey, l_suppkey, l_quantity, l_shipdate, sum(l_quantity) as col1 from lineitem where l_orderkey > 1 + 1 group by l_orderkey, l_partkey, l_suppkey, l_quantity, l_shipdate) as t1 + right semi join (select l_orderkey, l_partkey, l_suppkey, l_quantity, l_shipdate, sum(l_quantity) as col1 from lineitem where l_shipdate >= "2023-10-17" group by l_orderkey, l_partkey, l_suppkey, l_quantity, l_shipdate) as t1 on t1.l_orderkey = orders.o_orderkey group by t1.l_orderkey, t1.l_shipdate, t1.l_partkey """ def right_semi_mv_stmt_9 = """select t1.l_orderkey, t1.l_shipdate, t1.l_partkey from orders - right semi join (select l_orderkey, l_partkey, l_suppkey, l_quantity, l_shipdate, sum(l_quantity) as col1, count(*) as col2 from lineitem where l_orderkey > 1 + 1 group by l_orderkey, l_partkey, l_suppkey, l_quantity, l_shipdate) as t1 + right semi join (select l_orderkey, l_partkey, l_suppkey, l_quantity, l_shipdate, sum(l_quantity) as col1, count(*) as col2 from lineitem where l_shipdate >= "2023-10-17" group by l_orderkey, l_partkey, l_suppkey, l_quantity, l_shipdate) as t1 on t1.l_orderkey = orders.o_orderkey group by t1.l_orderkey, t1.l_shipdate, t1.l_partkey """ - def right_semi_mv_stmt_10 = """select t1.l_orderkey + def right_semi_mv_stmt_10 = """select t1.sum_total, max_total+min_total as col3, count_all from orders right semi join (select l_orderkey, sum(l_quantity) as sum_total, max(l_quantity) as max_total, @@ -670,10 +673,10 @@ suite("dimension_2_join_agg_replenish") { sum(l_quantity) + count(*) as col5, bitmap_union(to_bitmap(case when l_quantity > 1 and l_orderkey IN (1, 3) then l_partkey else null end)) as cnt_1, bitmap_union(to_bitmap(case when l_quantity > 2 and l_orderkey IN (2) then l_partkey else null end)) as cnt_2 - from lineitem where l_orderkey > 1 + 1 group by l_orderkey) as t1 + from lineitem where l_shipdate >= "2023-10-17" group by l_orderkey) as t1 on t1.l_orderkey = orders.o_orderkey group by - t1.l_orderkey + t1.sum_total, col3, count_all """ def left_anti_mv_stmt_1 = """select t1.o_orderdate, t1.o_shippriority, t1.o_orderkey @@ -790,7 +793,7 @@ suite("dimension_2_join_agg_replenish") { bitmap_union(to_bitmap(case when o_shippriority > 2 and o_orderkey IN (2) then o_custkey else null end)) as cnt_2 from orders where o_orderkey >= 1 + 1 group by o_orderkey) as t1 right anti join lineitem on lineitem.l_orderkey = t1.o_orderkey - group by l_orderkey, l_shipdate, l_partkey + group by l_orderkey, l_shipdate, l_partkey """ def right_anti_mv_stmt_6 = """select t1.l_shipdate, t1.l_quantity, t1.l_orderkey diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension_2_join_agg/dimension_join_agg_negative.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension_2_join_agg/dimension_join_agg_negative.groovy index 04bd6ac5c6ac8d..3afbe42fd2a1c3 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension_2_join_agg/dimension_join_agg_negative.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension_2_join_agg/dimension_join_agg_negative.groovy @@ -101,6 +101,9 @@ suite("dimension_join_agg_negative") { sql """analyze table orders_negative with sync;""" sql """analyze table lineitem_negative with sync;""" + sql """alter table orders_negative modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem_negative modify column l_comment set stats ('row_count'='7');""" + // left join + agg (function + group by + +-*/ + filter) def left_mv_stmt_1 = """select t1.o_orderdate, t1.o_shippriority, t1.o_orderkey from (select o_orderkey, o_custkey, o_orderstatus, o_orderdate, o_shippriority from orders_negative group by o_orderkey, o_custkey, o_orderstatus, o_orderdate, o_shippriority) as t1 diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension_equal/filter_equal_or_notequal.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension_equal/filter_equal_or_notequal.groovy index 763880ea94be44..2ca4a524989c99 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension_equal/filter_equal_or_notequal.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension_equal/filter_equal_or_notequal.groovy @@ -99,6 +99,9 @@ suite("filter_equal_or_notequal_case") { sql """analyze table orders_1 with sync;""" sql """analyze table lineitem_1 with sync;""" + sql """alter table orders_1 modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem_1 modify column l_comment set stats ('row_count'='7');""" + def compare_res = { def stmt -> sql "SET enable_materialized_view_rewrite=false" def origin_res = sql stmt diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/full_join_filter.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/full_join_filter.groovy index ce30fab1b6c71e..751e5d0056726f 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/full_join_filter.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/full_join_filter.groovy @@ -105,6 +105,9 @@ suite("full_join_filter") { sql """analyze table orders_full_join with sync;""" sql """analyze table lineitem_full_join with sync;""" + sql """alter table orders_full_join modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem_full_join modify column l_comment set stats ('row_count'='7');""" + def compare_res = { def stmt -> sql "SET enable_materialized_view_rewrite=false" def origin_res = sql stmt diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/inner_join_filter.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/inner_join_filter.groovy index 932c2608ad408a..8e6f06036aa7da 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/inner_join_filter.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/inner_join_filter.groovy @@ -105,6 +105,9 @@ suite("inner_join_filter") { sql """analyze table orders_inner_join with sync;""" sql """analyze table lineitem_inner_join with sync;""" + sql """alter table orders_inner_join modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem_inner_join modify column l_comment set stats ('row_count'='7');""" + def compare_res = { def stmt -> sql "SET enable_materialized_view_rewrite=false" def origin_res = sql stmt diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/left_anti_join_filter.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/left_anti_join_filter.groovy index d813e55f33286a..6522d7f2dc248a 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/left_anti_join_filter.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/left_anti_join_filter.groovy @@ -107,6 +107,9 @@ suite("left_anti_join_filter") { sql """analyze table orders_left_anti_join with sync;""" sql """analyze table lineitem_left_anti_join with sync;""" + sql """alter table orders_left_anti_join modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem_left_anti_join modify column l_comment set stats ('row_count'='9');""" + def compare_res = { def stmt -> sql "SET enable_materialized_view_rewrite=false" def origin_res = sql stmt diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/left_join_filter.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/left_join_filter.groovy index 608aec88c63621..37c9d3bfde4e47 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/left_join_filter.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/left_join_filter.groovy @@ -105,6 +105,9 @@ suite("left_join_filter") { sql """analyze table orders_left_join with sync;""" sql """analyze table lineitem_left_join with sync;""" + sql """alter table orders_left_join modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem_left_join modify column l_comment set stats ('row_count'='7');""" + def compare_res = { def stmt -> sql "SET enable_materialized_view_rewrite=false" def origin_res = sql stmt diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/left_semi_join_filter.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/left_semi_join_filter.groovy index da12c8fbeaef3f..2c9854eca82106 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/left_semi_join_filter.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/left_semi_join_filter.groovy @@ -105,6 +105,9 @@ suite("left_semi_join_filter") { sql """analyze table orders_left_semi_join with sync;""" sql """analyze table lineitem_left_semi_join with sync;""" + sql """alter table orders_left_semi_join modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem_left_semi_join modify column l_comment set stats ('row_count'='7');""" + def compare_res = { def stmt -> sql "SET enable_materialized_view_rewrite=false" def origin_res = sql stmt diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/right_anti_join_filter.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/right_anti_join_filter.groovy index b920b766b07aa4..547a36c424c8cb 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/right_anti_join_filter.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/right_anti_join_filter.groovy @@ -107,6 +107,9 @@ suite("right_anti_join_filter") { sql """analyze table orders_right_anti_join with sync;""" sql """analyze table lineitem_right_anti_join with sync;""" + sql """alter table orders_right_anti_join modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem_right_anti_join modify column l_comment set stats ('row_count'='9');""" + def compare_res = { def stmt -> sql "SET enable_materialized_view_rewrite=false" def origin_res = sql stmt diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/right_join_filter.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/right_join_filter.groovy index f3223855173f48..c29effd36cb701 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/right_join_filter.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/right_join_filter.groovy @@ -105,6 +105,9 @@ suite("right_join_filter") { sql """analyze table orders_right_join with sync;""" sql """analyze table lineitem_right_join with sync;""" + sql """alter table orders_right_join modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem_right_join modify column l_comment set stats ('row_count'='7');""" + def compare_res = { def stmt -> sql "SET enable_materialized_view_rewrite=false" def origin_res = sql stmt diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/right_semi_join_filter.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/right_semi_join_filter.groovy index bbf4e60afb64ec..6dc16554b26ca8 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/right_semi_join_filter.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/right_semi_join_filter.groovy @@ -107,6 +107,9 @@ suite("right_semi_join_filter") { sql """analyze table orders_right_semi_join with sync;""" sql """analyze table lineitem_right_semi_join with sync;""" + sql """alter table orders_right_semi_join modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem_right_semi_join modify column l_comment set stats ('row_count'='7');""" + def compare_res = { def stmt -> sql "SET enable_materialized_view_rewrite=false" def origin_res = sql stmt diff --git a/regression-test/suites/nereids_rules_p0/mv/direct_query/direct_query.groovy b/regression-test/suites/nereids_rules_p0/mv/direct_query/direct_query.groovy index f4fc91db26c3b8..0aaaebb64cf3a3 100644 --- a/regression-test/suites/nereids_rules_p0/mv/direct_query/direct_query.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/direct_query/direct_query.groovy @@ -131,6 +131,9 @@ suite("direct_query_mv") { sql """analyze table orders with sync;""" sql """analyze table partsupp with sync;""" + sql """alter table lineitem modify column l_comment set stats ('row_count'='5');""" + sql """alter table orders modify column o_comment set stats ('row_count'='15');""" + sql """alter table partsupp modify column ps_comment set stats ('row_count'='2');""" create_async_mv(db, "mv1_0", """ diff --git a/regression-test/suites/nereids_rules_p0/mv/grouping_sets/grouping_sets.groovy b/regression-test/suites/nereids_rules_p0/mv/grouping_sets/grouping_sets.groovy index ec8cecbaf517d0..4655f9d004bc7a 100644 --- a/regression-test/suites/nereids_rules_p0/mv/grouping_sets/grouping_sets.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/grouping_sets/grouping_sets.groovy @@ -127,6 +127,9 @@ suite("materialized_view_grouping_sets") { sql """analyze table orders with sync;""" sql """analyze table partsupp with sync;""" + sql """alter table orders modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem modify column l_comment set stats ('row_count'='7');""" + // query has group sets, and mv doesn't // single table grouping sets without grouping scalar function def mv1_0 = diff --git a/regression-test/suites/nereids_rules_p0/mv/join/dphyp_inner/inner_join_dphyp.groovy b/regression-test/suites/nereids_rules_p0/mv/join/dphyp_inner/inner_join_dphyp.groovy index 2b018ebc37373b..8c0674faeb8c32 100644 --- a/regression-test/suites/nereids_rules_p0/mv/join/dphyp_inner/inner_join_dphyp.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/join/dphyp_inner/inner_join_dphyp.groovy @@ -123,6 +123,10 @@ suite("inner_join_dphyp") { sql """analyze table orders with sync;""" sql """analyze table partsupp with sync;""" + sql """alter table lineitem modify column l_comment set stats ('row_count'='5');""" + sql """alter table orders modify column o_comment set stats ('row_count'='8');""" + sql """alter table partsupp modify column ps_comment set stats ('row_count'='2');""" + // without filter def mv1_0 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY " + "from lineitem " + diff --git a/regression-test/suites/nereids_rules_p0/mv/join/dphyp_outer/outer_join_dphyp.groovy b/regression-test/suites/nereids_rules_p0/mv/join/dphyp_outer/outer_join_dphyp.groovy index 45d9aee067c798..71154fbeb276cf 100644 --- a/regression-test/suites/nereids_rules_p0/mv/join/dphyp_outer/outer_join_dphyp.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/join/dphyp_outer/outer_join_dphyp.groovy @@ -123,6 +123,10 @@ suite("outer_join_dphyp") { sql """analyze table orders with sync;""" sql """analyze table partsupp with sync;""" + sql """alter table lineitem modify column l_comment set stats ('row_count'='5');""" + sql """alter table orders modify column o_comment set stats ('row_count'='8');""" + sql """alter table partsupp modify column ps_comment set stats ('row_count'='2');""" + // without filter def mv1_0 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY " + "from lineitem " + diff --git a/regression-test/suites/nereids_rules_p0/mv/join/inner/inner_join.groovy b/regression-test/suites/nereids_rules_p0/mv/join/inner/inner_join.groovy index b745f7d0197c28..3aebf92b7cec71 100644 --- a/regression-test/suites/nereids_rules_p0/mv/join/inner/inner_join.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/join/inner/inner_join.groovy @@ -122,6 +122,10 @@ suite("inner_join") { sql """analyze table orders with sync;""" sql """analyze table partsupp with sync;""" + sql """alter table lineitem modify column l_comment set stats ('row_count'='5');""" + sql """alter table orders modify column o_comment set stats ('row_count'='8');""" + sql """alter table partsupp modify column ps_comment set stats ('row_count'='2');""" + // without filter def mv1_0 = """ diff --git a/regression-test/suites/nereids_rules_p0/mv/join/left_outer/outer_join.groovy b/regression-test/suites/nereids_rules_p0/mv/join/left_outer/outer_join.groovy index faa2c747a837c5..cdc4632ff7379b 100644 --- a/regression-test/suites/nereids_rules_p0/mv/join/left_outer/outer_join.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/join/left_outer/outer_join.groovy @@ -278,9 +278,20 @@ suite("outer_join") { sql """analyze table lineitem with sync;""" sql """analyze table orders with sync;""" sql """analyze table partsupp with sync;""" + sql """analyze table lineitem_null with sync;""" + sql """analyze table orders_null with sync;""" sql """analyze table orders_same_col with sync;""" sql """analyze table lineitem_same_col with sync;""" + sql """alter table lineitem modify column l_comment set stats ('row_count'='5');""" + sql """alter table lineitem_same_col modify column l_comment set stats ('row_count'='5');""" + sql """alter table orders modify column o_comment set stats ('row_count'='8');""" + sql """alter table orders_same_col modify column o_comment set stats ('row_count'='18');""" + sql """alter table partsupp modify column ps_comment set stats ('row_count'='2');""" + sql """alter table lineitem_null modify column l_comment set stats ('row_count'='5');""" + sql """alter table orders_null modify column o_comment set stats ('row_count'='5');""" + + // without filter def mv1_0 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY " + "from lineitem " + diff --git a/regression-test/suites/nereids_rules_p0/mv/join_infer_derive/inner_join_infer_and_derive.groovy b/regression-test/suites/nereids_rules_p0/mv/join_infer_derive/inner_join_infer_and_derive.groovy index 5688c2bbe21d6a..cce090b2a940fb 100644 --- a/regression-test/suites/nereids_rules_p0/mv/join_infer_derive/inner_join_infer_and_derive.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/join_infer_derive/inner_join_infer_and_derive.groovy @@ -106,6 +106,9 @@ suite("inner_join_infer_and_derive") { sql """analyze table orders_inner with sync;""" sql """analyze table lineitem_inner with sync;""" + sql """alter table orders_inner modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem_inner modify column l_comment set stats ('row_count'='7');""" + def compare_res = { def stmt -> sql "SET enable_materialized_view_rewrite=false" def origin_res = sql stmt diff --git a/regression-test/suites/nereids_rules_p0/mv/join_infer_derive/left_join_infer_and_derive.groovy b/regression-test/suites/nereids_rules_p0/mv/join_infer_derive/left_join_infer_and_derive.groovy index dbfaa783383af8..7e51847c3b1a07 100644 --- a/regression-test/suites/nereids_rules_p0/mv/join_infer_derive/left_join_infer_and_derive.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/join_infer_derive/left_join_infer_and_derive.groovy @@ -106,6 +106,8 @@ suite("left_join_infer_and_derive") { sql """analyze table orders_left with sync;""" sql """analyze table lineitem_left with sync;""" + sql """alter table orders_left modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem_left modify column l_comment set stats ('row_count'='7');""" def compare_res = { def stmt -> sql "SET enable_materialized_view_rewrite=false" diff --git a/regression-test/suites/nereids_rules_p0/mv/join_infer_derive/right_join_infer_and_derive.groovy b/regression-test/suites/nereids_rules_p0/mv/join_infer_derive/right_join_infer_and_derive.groovy index 4b5bdf467a5d02..1761efa5ef4475 100644 --- a/regression-test/suites/nereids_rules_p0/mv/join_infer_derive/right_join_infer_and_derive.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/join_infer_derive/right_join_infer_and_derive.groovy @@ -106,6 +106,9 @@ suite("right_join_infer_and_derive") { sql """analyze table orders_right with sync;""" sql """analyze table lineitem_right with sync;""" + sql """alter table orders_right modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem_right modify column l_comment set stats ('row_count'='7');""" + def compare_res = { def stmt -> sql "SET enable_materialized_view_rewrite=false" def origin_res = sql stmt diff --git a/regression-test/suites/nereids_rules_p0/mv/negative/negative_test.groovy b/regression-test/suites/nereids_rules_p0/mv/negative/negative_test.groovy index ba23c29ab9abc4..ea45b806529d8d 100644 --- a/regression-test/suites/nereids_rules_p0/mv/negative/negative_test.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/negative/negative_test.groovy @@ -127,6 +127,9 @@ suite("negative_partition_mv_rewrite") { sql """analyze table lineitem_1 with sync;""" sql """analyze table partsupp_1 with sync;""" + sql """alter table orders_1 modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem_1 modify column l_comment set stats ('row_count'='7');""" + def mv_name = "mv_1" def mtmv_sql = """ select l_shipdate, o_orderdate, l_partkey, l_suppkey diff --git a/regression-test/suites/nereids_rules_p0/mv/nested/nested_materialized_view.groovy b/regression-test/suites/nereids_rules_p0/mv/nested/nested_materialized_view.groovy index 3a130d489eb036..692ea03fc61806 100644 --- a/regression-test/suites/nereids_rules_p0/mv/nested/nested_materialized_view.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/nested/nested_materialized_view.groovy @@ -199,6 +199,10 @@ suite("nested_materialized_view") { (2, 3, 10, 11.01, 'supply2'); """ + sql """alter table lineitem modify column l_comment set stats ('row_count'='5');""" + sql """alter table orders modify column o_comment set stats ('row_count'='8');""" + sql """alter table partsupp modify column ps_comment set stats ('row_count'='2');""" + // simple nested materialized view def mv1_0_inner_mv = """ select diff --git a/regression-test/suites/nereids_rules_p0/mv/nested_mtmv/nested_mtmv.groovy b/regression-test/suites/nereids_rules_p0/mv/nested_mtmv/nested_mtmv.groovy index 700c17606d2539..4870ec99e659c0 100644 --- a/regression-test/suites/nereids_rules_p0/mv/nested_mtmv/nested_mtmv.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/nested_mtmv/nested_mtmv.groovy @@ -96,12 +96,20 @@ suite("nested_mtmv") { insert into orders_1 values (null, 1, 'k', 99.5, 'a', 'b', 1, 'yy', '2023-10-17'), (1, null, 'o', 109.2, 'c','d',2, 'mm', '2023-10-17'), + (1, null, 'o', 109.2, 'c','d',2, 'mm', '2023-10-17'), + (3, 3, null, 99.5, 'a', 'b', 1, 'yy', '2023-10-19'), (3, 3, null, 99.5, 'a', 'b', 1, 'yy', '2023-10-19'), (1, 2, 'o', null, 'a', 'b', 1, 'yy', '2023-10-20'), + (1, 2, 'o', null, 'a', 'b', 1, 'yy', '2023-10-20'), (2, 3, 'k', 109.2, null,'d',2, 'mm', '2023-10-21'), + (2, 3, 'k', 109.2, null,'d',2, 'mm', '2023-10-21'), + (3, 1, 'k', 99.5, 'a', null, 1, 'yy', '2023-10-22'), (3, 1, 'k', 99.5, 'a', null, 1, 'yy', '2023-10-22'), (1, 3, 'o', 99.5, 'a', 'b', null, 'yy', '2023-10-19'), + (1, 3, 'o', 99.5, 'a', 'b', null, 'yy', '2023-10-19'), (2, 1, 'o', 109.2, 'c','d',2, null, '2023-10-18'), + (2, 1, 'o', 109.2, 'c','d',2, null, '2023-10-18'), + (3, 2, 'k', 99.5, 'a', 'b', 1, 'yy', '2023-10-17'), (3, 2, 'k', 99.5, 'a', 'b', 1, 'yy', '2023-10-17'), (4, 5, 'k', 99.5, 'a', 'b', 1, 'yy', '2023-10-19'); """ @@ -109,22 +117,29 @@ suite("nested_mtmv") { sql """ insert into lineitem_1 values (null, 1, 2, 3, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-17', '2023-10-17', 'a', 'b', 'yyyyyyyyy', '2023-10-17'), + (null, 1, 2, 3, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-17', '2023-10-17', 'a', 'b', 'yyyyyyyyy', '2023-10-17'), (1, 1, 3, 1, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-18', '2023-10-18', 'a', 'b', 'yyyyyyyyy', '2023-10-17'), (3, 3, 3, 2, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-10-19', '2023-10-19', 'c', 'd', 'xxxxxxxxx', '2023-10-19'), + (3, 3, 3, 2, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-10-19', '2023-10-19', 'c', 'd', 'xxxxxxxxx', '2023-10-19'), (1, 2, 3, 2, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-17', '2023-10-17', 'a', 'b', 'yyyyyyyyy', '2023-10-17'), (2, 1, 2, 1, 5.5, 6.5, 7.5, 8.5, 'o', 'k', null, '2023-10-18', 'a', 'b', 'yyyyyyyyy', '2023-10-18'), + (2, 1, 2, 1, 5.5, 6.5, 7.5, 8.5, 'o', 'k', null, '2023-10-18', 'a', 'b', 'yyyyyyyyy', '2023-10-18'), (3, 1, 3, 1, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-10-19', null, 'c', 'd', 'xxxxxxxxx', '2023-10-19'), (1, 2, 1, 2, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-17', '2023-10-17', 'a', 'b', 'yyyyyyyyy', '2023-10-17'), (2, 2, 2, 2, 5.5, 6.5, 7.5, 8.5, 'o', 'k', null, '2023-10-18', 'a', 'b', 'yyyyyyyyy', '2023-10-18'), (3, 3, 3, 3, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-10-19', null, 'c', 'd', 'xxxxxxxxx', '2023-10-19'), + (3, 3, 3, 3, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-10-19', null, 'c', 'd', 'xxxxxxxxx', '2023-10-19'), (1, 1, 1, 1, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-17', '2023-10-17', 'a', 'b', 'yyyyyyyyy', '2023-10-17'); """ sql""" insert into partsupp_1 values (1, 1, 1, 99.5, 'yy'), + (1, 1, 1, 99.5, 'yy'), + (2, 2, 2, 109.2, 'mm'), (2, 2, 2, 109.2, 'mm'), (3, 3, 1, 99.5, 'yy'), + (3, 3, 1, 99.5, 'yy'), (3, null, 1, 99.5, 'yy'); """ @@ -132,6 +147,10 @@ suite("nested_mtmv") { sql """analyze table lineitem_1 with sync;""" sql """analyze table partsupp_1 with sync;""" + sql """alter table orders_1 modify column o_orderdate set stats ('row_count'='17');""" + sql """alter table lineitem_1 modify column l_shipdate set stats ('row_count'='14');""" + sql """alter table partsupp_1 modify column ps_comment set stats ('row_count'='7');""" + def compare_res = { def stmt -> sql "SET enable_materialized_view_rewrite=false" def origin_res = sql stmt @@ -659,154 +678,77 @@ suite("nested_mtmv") { left join ( select t1.l_orderkey, t2.l_partkey, t1.l_suppkey, t2.o_orderkey, t1.o_custkey, t2.ps_partkey, t1.ps_suppkey, t2.agg1, t1.agg2, t2.agg3, t1.agg4, t2.agg5, t1.agg6 from ( - select t1.l_orderkey, t2.l_partkey, t1.l_suppkey, t2.o_orderkey, t1.o_custkey, t2.ps_partkey, t1.ps_suppkey, t2.agg1, t1.agg2, t2.agg3, t1.agg4, t2.agg5, t1.agg6 + select l_orderkey, l_partkey, l_suppkey, o_orderkey, o_custkey, ps_partkey, ps_suppkey, + t.agg1 as agg1, + t.sum_total as agg3, + t.max_total as agg4, + t.min_total as agg5, + t.count_all as agg6, + cast(sum(IFNULL(ps_suppkey, 0) * IFNULL(ps_partkey, 0)) as decimal(28, 8)) as agg2 from ( - select l_orderkey, l_partkey, l_suppkey, o_orderkey, o_custkey, ps_partkey, ps_suppkey, - t.agg1 as agg1, - t.sum_total as agg3, - t.max_total as agg4, - t.min_total as agg5, - t.count_all as agg6, - cast(sum(IFNULL(ps_suppkey, 0) * IFNULL(ps_partkey, 0)) as decimal(28, 8)) as agg2 - from ( - select l_orderkey, l_partkey, l_suppkey, o_orderkey, o_custkey, cast(sum(IFNULL(o_orderkey, 0) * IFNULL(o_custkey, 0)) as decimal(28, 8)) as agg1, - sum(o_totalprice) as sum_total, - max(o_totalprice) as max_total, - min(o_totalprice) as min_total, - count(*) as count_all, - bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) cnt_1, - bitmap_union(to_bitmap(case when o_shippriority > 2 and o_orderkey IN (2) then o_custkey else null end)) as cnt_2 - from lineitem_1 - inner join orders_1 - on lineitem_1.l_orderkey = orders_1.o_orderkey - where lineitem_1.l_shipdate >= "2023-10-17" - group by l_orderkey, l_partkey, l_suppkey, o_orderkey, o_custkey - ) as t - inner join partsupp_1 - on t.l_partkey = partsupp_1.ps_partkey and t.l_suppkey = partsupp_1.ps_suppkey - where partsupp_1.ps_suppkey > 1 - group by l_orderkey, l_partkey, l_suppkey, o_orderkey, o_custkey, ps_partkey, ps_suppkey, agg1, agg3, agg4, agg5, agg6 - ) as t1 - left join ( - select l_orderkey, l_partkey, l_suppkey, o_orderkey, o_custkey, ps_partkey, ps_suppkey, - t.agg1 as agg1, - t.sum_total as agg3, - t.max_total as agg4, - t.min_total as agg5, - t.count_all as agg6, - cast(sum(IFNULL(ps_suppkey, 0) * IFNULL(ps_partkey, 0)) as decimal(28, 8)) as agg2 - from ( - select l_orderkey, l_partkey, l_suppkey, o_orderkey, o_custkey, cast(sum(IFNULL(o_orderkey, 0) * IFNULL(o_custkey, 0)) as decimal(28, 8)) as agg1, - sum(o_totalprice) as sum_total, - max(o_totalprice) as max_total, - min(o_totalprice) as min_total, - count(*) as count_all, - bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) cnt_1, - bitmap_union(to_bitmap(case when o_shippriority > 2 and o_orderkey IN (2) then o_custkey else null end)) as cnt_2 - from lineitem_1 - inner join orders_1 - on lineitem_1.l_orderkey = orders_1.o_orderkey - where lineitem_1.l_shipdate >= "2023-10-17" - group by l_orderkey, l_partkey, l_suppkey, o_orderkey, o_custkey - ) as t - inner join partsupp_1 - on t.l_partkey = partsupp_1.ps_partkey and t.l_suppkey = partsupp_1.ps_suppkey - where partsupp_1.ps_suppkey > 1 - group by l_orderkey, l_partkey, l_suppkey, o_orderkey, o_custkey, ps_partkey, ps_suppkey, agg1, agg3, agg4, agg5, agg6 - ) as t2 - on t1.l_orderkey = t2.l_orderkey - where t1.l_orderkey > 1 - group by t1.l_orderkey, t2.l_partkey, t1.l_suppkey, t2.o_orderkey, t1.o_custkey, t2.ps_partkey, t1.ps_suppkey, t2.agg1, t1.agg2, t2.agg3, t1.agg4, t2.agg5, t1.agg6 + select l_orderkey, l_partkey, l_suppkey, o_orderkey, o_custkey, cast(sum(IFNULL(o_orderkey, 0) * IFNULL(o_custkey, 0)) as decimal(28, 8)) as agg1, + sum(o_totalprice) as sum_total, + max(o_totalprice) as max_total, + min(o_totalprice) as min_total, + count(*) as count_all, + bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) cnt_1, + bitmap_union(to_bitmap(case when o_shippriority > 2 and o_orderkey IN (2) then o_custkey else null end)) as cnt_2 + from lineitem_1 + inner join orders_1 + on lineitem_1.l_orderkey = orders_1.o_orderkey + where lineitem_1.l_shipdate >= "2023-10-17" + group by l_orderkey, l_partkey, l_suppkey, o_orderkey, o_custkey + ) as t + inner join partsupp_1 + on t.l_partkey = partsupp_1.ps_partkey and t.l_suppkey = partsupp_1.ps_suppkey + where partsupp_1.ps_suppkey > 1 + group by l_orderkey, l_partkey, l_suppkey, o_orderkey, o_custkey, ps_partkey, ps_suppkey, agg1, agg3, agg4, agg5, agg6 ) as t1 left join ( - select t1.l_orderkey, t2.l_partkey, t1.l_suppkey, t2.o_orderkey, t1.o_custkey, t2.ps_partkey, t1.ps_suppkey, t2.agg1, t1.agg2, t2.agg3, t1.agg4, t2.agg5, t1.agg6 + select l_orderkey, l_partkey, l_suppkey, o_orderkey, o_custkey, ps_partkey, ps_suppkey, + t.agg1 as agg1, + t.sum_total as agg3, + t.max_total as agg4, + t.min_total as agg5, + t.count_all as agg6, + cast(sum(IFNULL(ps_suppkey, 0) * IFNULL(ps_partkey, 0)) as decimal(28, 8)) as agg2 from ( - select l_orderkey, l_partkey, l_suppkey, o_orderkey, o_custkey, ps_partkey, ps_suppkey, - t.agg1 as agg1, - t.sum_total as agg3, - t.max_total as agg4, - t.min_total as agg5, - t.count_all as agg6, - cast(sum(IFNULL(ps_suppkey, 0) * IFNULL(ps_partkey, 0)) as decimal(28, 8)) as agg2 - from ( - select l_orderkey, l_partkey, l_suppkey, o_orderkey, o_custkey, cast(sum(IFNULL(o_orderkey, 0) * IFNULL(o_custkey, 0)) as decimal(28, 8)) as agg1, - sum(o_totalprice) as sum_total, - max(o_totalprice) as max_total, - min(o_totalprice) as min_total, - count(*) as count_all, - bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) cnt_1, - bitmap_union(to_bitmap(case when o_shippriority > 2 and o_orderkey IN (2) then o_custkey else null end)) as cnt_2 - from lineitem_1 - inner join orders_1 - on lineitem_1.l_orderkey = orders_1.o_orderkey - where lineitem_1.l_shipdate >= "2023-10-17" - group by l_orderkey, l_partkey, l_suppkey, o_orderkey, o_custkey - ) as t - inner join partsupp_1 - on t.l_partkey = partsupp_1.ps_partkey and t.l_suppkey = partsupp_1.ps_suppkey - where partsupp_1.ps_suppkey > 1 - group by l_orderkey, l_partkey, l_suppkey, o_orderkey, o_custkey, ps_partkey, ps_suppkey, agg1, agg3, agg4, agg5, agg6 - ) as t1 - left join ( - select l_orderkey, l_partkey, l_suppkey, o_orderkey, o_custkey, ps_partkey, ps_suppkey, - t.agg1 as agg1, - t.sum_total as agg3, - t.max_total as agg4, - t.min_total as agg5, - t.count_all as agg6, - cast(sum(IFNULL(ps_suppkey, 0) * IFNULL(ps_partkey, 0)) as decimal(28, 8)) as agg2 - from ( - select l_orderkey, l_partkey, l_suppkey, o_orderkey, o_custkey, cast(sum(IFNULL(o_orderkey, 0) * IFNULL(o_custkey, 0)) as decimal(28, 8)) as agg1, - sum(o_totalprice) as sum_total, - max(o_totalprice) as max_total, - min(o_totalprice) as min_total, - count(*) as count_all, - bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) cnt_1, - bitmap_union(to_bitmap(case when o_shippriority > 2 and o_orderkey IN (2) then o_custkey else null end)) as cnt_2 - from lineitem_1 - inner join orders_1 - on lineitem_1.l_orderkey = orders_1.o_orderkey - where lineitem_1.l_shipdate >= "2023-10-17" - group by l_orderkey, l_partkey, l_suppkey, o_orderkey, o_custkey - ) as t - inner join partsupp_1 - on t.l_partkey = partsupp_1.ps_partkey and t.l_suppkey = partsupp_1.ps_suppkey - where partsupp_1.ps_suppkey > 1 - group by l_orderkey, l_partkey, l_suppkey, o_orderkey, o_custkey, ps_partkey, ps_suppkey, agg1, agg3, agg4, agg5, agg6 - ) as t2 - on t1.l_orderkey = t2.l_orderkey - where t1.l_orderkey > 1 - group by t1.l_orderkey, t2.l_partkey, t1.l_suppkey, t2.o_orderkey, t1.o_custkey, t2.ps_partkey, t1.ps_suppkey, t2.agg1, t1.agg2, t2.agg3, t1.agg4, t2.agg5, t1.agg6 + select l_orderkey, l_partkey, l_suppkey, o_orderkey, o_custkey, cast(sum(IFNULL(o_orderkey, 0) * IFNULL(o_custkey, 0)) as decimal(28, 8)) as agg1, + sum(o_totalprice) as sum_total, + max(o_totalprice) as max_total, + min(o_totalprice) as min_total, + count(*) as count_all, + bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) cnt_1, + bitmap_union(to_bitmap(case when o_shippriority > 2 and o_orderkey IN (2) then o_custkey else null end)) as cnt_2 + from lineitem_1 + inner join orders_1 + on lineitem_1.l_orderkey = orders_1.o_orderkey + where lineitem_1.l_shipdate >= "2023-10-17" + group by l_orderkey, l_partkey, l_suppkey, o_orderkey, o_custkey + ) as t + inner join partsupp_1 + on t.l_partkey = partsupp_1.ps_partkey and t.l_suppkey = partsupp_1.ps_suppkey + where partsupp_1.ps_suppkey > 1 + group by l_orderkey, l_partkey, l_suppkey, o_orderkey, o_custkey, ps_partkey, ps_suppkey, agg1, agg3, agg4, agg5, agg6 ) as t2 on t1.l_orderkey = t2.l_orderkey + where t1.l_orderkey > 1 group by t1.l_orderkey, t2.l_partkey, t1.l_suppkey, t2.o_orderkey, t1.o_custkey, t2.ps_partkey, t1.ps_suppkey, t2.agg1, t1.agg2, t2.agg3, t1.agg4, t2.agg5, t1.agg6 ) as t2 on t1.l_orderkey = t2.l_orderkey group by t1.l_orderkey, t2.l_partkey, t1.l_suppkey, t2.o_orderkey, t1.o_custkey, t2.ps_partkey, t1.ps_suppkey, t2.agg1, t1.agg2, t2.agg3, t1.agg4, t2.agg5, t1.agg6 """ - explain { - sql("${sql_2}") - contains "${mv_2}(${mv_2})" - } + mv_rewrite_any_success(sql_2, [mv_1, mv_2]) compare_res(sql_2 + " order by 1,2,3,4,5,6,7,8,9,10,11,12,13") - explain { - sql("${sql_3}") - contains "${mv_3}(${mv_3})" - } + mv_rewrite_any_success(sql_3, [mv_3, mv_4]) compare_res(sql_3 + " order by 1,2,3,4,5,6,7,8,9,10,11,12,13") - explain { - sql("${sql_4}") - contains "${mv_4}(${mv_4})" - } + mv_rewrite_any_success(sql_4, [mv_3, mv_4]) compare_res(sql_4 + " order by 1,2,3,4,5,6,7,8,9,10,11,12,13") - explain { - sql("${sql_5}") - contains "${mv_5}(${mv_5})" - } + mv_rewrite_any_success(sql_5, [mv_3, mv_4, mv_5]) compare_res(sql_5 + " order by 1,2,3,4,5,6,7,8,9,10,11,12,13") } diff --git a/regression-test/suites/nereids_rules_p0/mv/nested_mtmv_switch/nested_mtmv_rewrite_switch.groovy b/regression-test/suites/nereids_rules_p0/mv/nested_mtmv_switch/nested_mtmv_rewrite_switch.groovy index f9a84fa5250179..64354267a039bd 100644 --- a/regression-test/suites/nereids_rules_p0/mv/nested_mtmv_switch/nested_mtmv_rewrite_switch.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/nested_mtmv_switch/nested_mtmv_rewrite_switch.groovy @@ -99,6 +99,9 @@ suite("nested_mtmv_rewrite_switch") { sql """analyze table orders_2 with sync;""" sql """analyze table lineitem_2 with sync;""" + sql """alter table orders_2 modify column o_orderdate set stats ('row_count'='10');""" + sql """alter table lineitem_2 modify column l_shipdate set stats ('row_count'='7');""" + def compare_res = { def stmt -> sql "SET enable_materialized_view_rewrite=false" diff --git a/regression-test/suites/nereids_rules_p0/mv/other_join_conjuncts/anti/other_join_conjuncts_anti.groovy b/regression-test/suites/nereids_rules_p0/mv/other_join_conjuncts/anti/other_join_conjuncts_anti.groovy index 68337b00d09da8..be5301cdf2ec17 100644 --- a/regression-test/suites/nereids_rules_p0/mv/other_join_conjuncts/anti/other_join_conjuncts_anti.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/other_join_conjuncts/anti/other_join_conjuncts_anti.groovy @@ -140,6 +140,10 @@ suite("other_join_conjuncts_anti") { sql """analyze table lineitem with sync""" sql """analyze table orders with sync""" + sql """alter table orders modify column lo_orderdate set stats ('row_count'='18');""" + sql """alter table lineitem modify column lo_orderdate set stats ('row_count'='10');""" +sql """alter table partsupp modify column ps_comment set stats ('row_count'='2');""" + // =, !=, >, <, <=, >= // left anti join other conjuncts in join condition def mv1_0 = diff --git a/regression-test/suites/nereids_rules_p0/mv/other_join_conjuncts/inner/other_join_conjuncts_inner.groovy b/regression-test/suites/nereids_rules_p0/mv/other_join_conjuncts/inner/other_join_conjuncts_inner.groovy index b79e5808932044..0f07b7390cb3d7 100644 --- a/regression-test/suites/nereids_rules_p0/mv/other_join_conjuncts/inner/other_join_conjuncts_inner.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/other_join_conjuncts/inner/other_join_conjuncts_inner.groovy @@ -140,6 +140,10 @@ suite("other_join_conjuncts_inner") { sql """analyze table lineitem with sync""" sql """analyze table orders with sync""" + sql """alter table orders modify column lo_orderdate set stats ('row_count'='18');""" + sql """alter table lineitem modify column lo_orderdate set stats ('row_count'='10');""" + sql """alter table partsupp modify column ps_comment set stats ('row_count'='2');""" + // =, !=, >, <, <=, >= // other conjuncts in join condition def mv1_0 = diff --git a/regression-test/suites/nereids_rules_p0/mv/other_join_conjuncts/outer/other_join_conjuncts_outer.groovy b/regression-test/suites/nereids_rules_p0/mv/other_join_conjuncts/outer/other_join_conjuncts_outer.groovy index 8a434364c89168..87fd620c704baf 100644 --- a/regression-test/suites/nereids_rules_p0/mv/other_join_conjuncts/outer/other_join_conjuncts_outer.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/other_join_conjuncts/outer/other_join_conjuncts_outer.groovy @@ -140,6 +140,10 @@ suite("other_join_conjuncts_outer") { sql """analyze table lineitem with sync""" sql """analyze table orders with sync""" + sql """alter table orders modify column lo_orderdate set stats ('row_count'='18');""" + sql """alter table lineitem modify column lo_orderdate set stats ('row_count'='10');""" + sql """alter table partsupp modify column ps_comment set stats ('row_count'='2');""" + // =, !=, >, <, <=, >= // left outer join // other conjuncts in join condition diff --git a/regression-test/suites/nereids_rules_p0/mv/other_join_conjuncts/semi/other_join_conjuncts_semi.groovy b/regression-test/suites/nereids_rules_p0/mv/other_join_conjuncts/semi/other_join_conjuncts_semi.groovy index 30c21c77269e1c..4bbfa7e7c0a15c 100644 --- a/regression-test/suites/nereids_rules_p0/mv/other_join_conjuncts/semi/other_join_conjuncts_semi.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/other_join_conjuncts/semi/other_join_conjuncts_semi.groovy @@ -140,6 +140,10 @@ suite("other_join_conjuncts_semi") { sql """analyze table lineitem with sync""" sql """analyze table orders with sync""" + sql """alter table orders modify column lo_orderdate set stats ('row_count'='18');""" + sql """alter table lineitem modify column lo_orderdate set stats ('row_count'='10');""" +sql """alter table partsupp modify column ps_comment set stats ('row_count'='2');""" + // left self join def mv1_0 = """ diff --git a/regression-test/suites/nereids_rules_p0/mv/partition_mv_rewrite.groovy b/regression-test/suites/nereids_rules_p0/mv/partition_mv_rewrite.groovy index 35a983c264c0e2..f8023a9e1e4c49 100644 --- a/regression-test/suites/nereids_rules_p0/mv/partition_mv_rewrite.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/partition_mv_rewrite.groovy @@ -480,6 +480,10 @@ suite("partition_mv_rewrite") { analyze table orders with sync; """ + sql """alter table orders modify column o_comment set stats ('row_count'='3');""" + sql """alter table lineitem modify column l_comment set stats ('row_count'='6');""" + sql """alter table lineitem_static modify column l_comment set stats ('row_count'='4');""" + // should rewrite successful when union rewrite enalbe if base table add new partition mv_rewrite_success(roll_up_all_partition_sql, "mv_10086", true, is_partition_statistics_ready(db, ["lineitem", "orders", "mv_10086"])) diff --git a/regression-test/suites/nereids_rules_p0/mv/same_name/sync_async_same_name.groovy b/regression-test/suites/nereids_rules_p0/mv/same_name/sync_async_same_name.groovy index e518c016d2b641..4dda0f9c08414b 100644 --- a/regression-test/suites/nereids_rules_p0/mv/same_name/sync_async_same_name.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/same_name/sync_async_same_name.groovy @@ -57,6 +57,8 @@ suite("sync_async_same_name") { (5, 2, 'o', 1.2, '2023-12-12', 'c','d',2, 'mi'); """ + sql """alter table orders modify column o_comment set stats ('row_count'='8');""" + sql """analyze table orders with sync;""" def common_mv_name = 'common_mv_name' diff --git a/regression-test/suites/nereids_rules_p0/mv/scan/scan_table.groovy b/regression-test/suites/nereids_rules_p0/mv/scan/scan_table.groovy index 6b5c2dc45e875f..c42888e35abc0a 100644 --- a/regression-test/suites/nereids_rules_p0/mv/scan/scan_table.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/scan/scan_table.groovy @@ -123,6 +123,10 @@ suite("mv_scan_table") { sql """analyze table lineitem with sync;""" sql """analyze table partsupp with sync;""" + sql """alter table orders modify column o_comment set stats ('row_count'='8');""" + sql """alter table lineitem modify column l_comment set stats ('row_count'='5');""" +sql """alter table partsupp modify column ps_comment set stats ('row_count'='2');""" + // with filter def mv1_0 = """ diff --git a/regression-test/suites/nereids_rules_p0/mv/single_table_without_agg/single_table_without_aggregate.groovy b/regression-test/suites/nereids_rules_p0/mv/single_table_without_agg/single_table_without_aggregate.groovy index b6f44bba80139c..450fb9c0ea3187 100644 --- a/regression-test/suites/nereids_rules_p0/mv/single_table_without_agg/single_table_without_aggregate.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/single_table_without_agg/single_table_without_aggregate.groovy @@ -52,6 +52,7 @@ suite("single_table_without_aggregate") { """ sql "analyze table orders with sync;" + sql """alter table orders modify column o_comment set stats ('row_count'='2');""" sql """set enable_stats=false;""" def check_rewrite = { mv_sql, query_sql, mv_name -> diff --git a/regression-test/suites/nereids_rules_p0/mv/union_all_compensate/union_all_compensate.groovy b/regression-test/suites/nereids_rules_p0/mv/union_all_compensate/union_all_compensate.groovy index e72e7575053b5b..192f9c420f18b5 100644 --- a/regression-test/suites/nereids_rules_p0/mv/union_all_compensate/union_all_compensate.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/union_all_compensate/union_all_compensate.groovy @@ -180,6 +180,9 @@ suite("union_all_compensate") { mv_rewrite_fail(query1_0, "test_agg_mv") order_qt_query1_1_after "${query1_0}" + sql """alter table test_table1 modify column num set stats ('row_count'='20');""" + sql """alter table test_table2 modify column num set stats ('row_count'='16');""" + // Aggregate, if query group by expression doesn't use the partition column, but the invalid partition is in the // grace_period, should not compensate union all, but should rewritten successfully diff --git a/regression-test/suites/nereids_rules_p0/mv/union_rewrite/partition_curd_union_rewrite.groovy b/regression-test/suites/nereids_rules_p0/mv/union_rewrite/partition_curd_union_rewrite.groovy index 4f6717dfe59255..28a8f2b60f653c 100644 --- a/regression-test/suites/nereids_rules_p0/mv/union_rewrite/partition_curd_union_rewrite.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/union_rewrite/partition_curd_union_rewrite.groovy @@ -78,6 +78,9 @@ suite ("partition_curd_union_rewrite") { ); """ + sql """alter table orders modify column o_comment set stats ('row_count'='3');""" + sql """alter table lineitem modify column l_comment set stats ('row_count'='3');""" + sql""" insert into orders values (1, 1, 'ok', 99.5, '2023-10-17', 'a', 'b', 1, 'yy'), diff --git a/regression-test/suites/nereids_rules_p0/mv/union_rewrite/partition_curd_union_rewrite_hive.groovy b/regression-test/suites/nereids_rules_p0/mv/union_rewrite/partition_curd_union_rewrite_hive.groovy index 14170bf21f40fa..d82eaefde509e6 100644 --- a/regression-test/suites/nereids_rules_p0/mv/union_rewrite/partition_curd_union_rewrite_hive.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/union_rewrite/partition_curd_union_rewrite_hive.groovy @@ -155,6 +155,9 @@ suite ("partition_curd_union_rewrite_hive") { sql """create database if not exists ${db}""" sql """use ${db}""" + sql """alter table ${orders_tb_name} modify column o_comment set stats ('row_count'='3');""" +sql """alter table ${lineitem_tb_name} modify column l_comment set stats ('row_count'='3');""" + def mv_def_sql = """ select l_shipdate, o_orderdate, l_partkey, l_suppkey, sum(o_totalprice) as sum_total diff --git a/regression-test/suites/nereids_rules_p0/mv/union_rewrite/usercase_union_rewrite.groovy b/regression-test/suites/nereids_rules_p0/mv/union_rewrite/usercase_union_rewrite.groovy index 702431e5e391ae..4e74090b1a2bb2 100644 --- a/regression-test/suites/nereids_rules_p0/mv/union_rewrite/usercase_union_rewrite.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/union_rewrite/usercase_union_rewrite.groovy @@ -90,6 +90,9 @@ suite ("usercase_union_rewrite") { sql """analyze table orders_user with sync;""" sql """analyze table lineitem_user with sync;""" + sql """alter table orders_user modify column o_comment set stats ('row_count'='4');""" + sql """alter table lineitem_user modify column l_comment set stats ('row_count'='3');""" + def create_mv_orders = { mv_name, mv_sql -> sql """DROP MATERIALIZED VIEW IF EXISTS ${mv_name};""" sql """DROP TABLE IF EXISTS ${mv_name}""" diff --git a/regression-test/suites/nereids_rules_p0/mv/unsafe_equals/null_un_safe_equals.groovy b/regression-test/suites/nereids_rules_p0/mv/unsafe_equals/null_un_safe_equals.groovy index 5c64379950f942..cba2509483b6dc 100644 --- a/regression-test/suites/nereids_rules_p0/mv/unsafe_equals/null_un_safe_equals.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/unsafe_equals/null_un_safe_equals.groovy @@ -57,6 +57,8 @@ suite("null_unsafe_equals") { (5, 2, 'o', 1.2, '2023-12-12', 'c','d', null, 'mi'); """ + sql """alter table orders modify column o_comment set stats ('row_count'='8');""" + def mv1_0 = """ select count(*), o_orderstatus, o_comment diff --git a/regression-test/suites/nereids_rules_p0/mv/variant/variant_mv.groovy b/regression-test/suites/nereids_rules_p0/mv/variant/variant_mv.groovy index 866f82af156c69..fae32e914ed031 100644 --- a/regression-test/suites/nereids_rules_p0/mv/variant/variant_mv.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/variant/variant_mv.groovy @@ -87,6 +87,9 @@ suite("variant_mv") { sql """analyze table github_events1 with sync;""" sql """analyze table github_events2 with sync;""" + sql """alter table github_events1 modify column created_at set stats ('row_count'='3');""" + sql """alter table github_events2 modify column created_at set stats ('row_count'='3');""" + // variant appear in where both slot and in expression def mv1_0 = """ SELECT diff --git a/regression-test/suites/nereids_rules_p0/mv/with_auth/with_select_table_auth.groovy b/regression-test/suites/nereids_rules_p0/mv/with_auth/with_select_table_auth.groovy index 400e113b6f3ae5..c51dea2f5a7403 100644 --- a/regression-test/suites/nereids_rules_p0/mv/with_auth/with_select_table_auth.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/with_auth/with_select_table_auth.groovy @@ -112,6 +112,9 @@ suite("with_select_table_auth","p0,auth") { sql """analyze table lineitem with sync""" sql """analyze table orders with sync""" + sql """alter table orders modify column o_comment set stats ('row_count'='18');""" + sql """alter table lineitem modify column l_comment set stats ('row_count'='5');""" + sql """grant select_priv on ${db}.orders to ${user_name}""" sql """grant select_priv on ${db}.lineitem to ${user_name}""" sql """grant select_priv on regression_test to ${user_name}""" diff --git a/regression-test/suites/nereids_rules_p0/mv/with_sql_limit/query_with_sql_limit.groovy b/regression-test/suites/nereids_rules_p0/mv/with_sql_limit/query_with_sql_limit.groovy index b7c6ecbd8ae008..757a4c42e5ad32 100644 --- a/regression-test/suites/nereids_rules_p0/mv/with_sql_limit/query_with_sql_limit.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/with_sql_limit/query_with_sql_limit.groovy @@ -143,6 +143,10 @@ suite("query_with_sql_limit") { sql """analyze table lineitem with sync""" sql """analyze table orders with sync""" + sql """alter table orders modify column o_comment set stats ('row_count'='18');""" + sql """alter table lineitem modify column l_comment set stats ('row_count'='5');""" + sql """alter table partsupp modify column ps_comment set stats ('row_count'='3');""" + // test sql_select_limit default, default 9223372036854775807 sql """set sql_select_limit = 2;""" def mv1_0 = diff --git a/regression-test/suites/nereids_syntax_p0/mv/newMv/aggHaveDupBase.groovy b/regression-test/suites/nereids_syntax_p0/mv/newMv/aggHaveDupBase.groovy index 76696e1c62b305..54ad59943d77bd 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/newMv/aggHaveDupBase.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/newMv/aggHaveDupBase.groovy @@ -36,6 +36,7 @@ suite ("aggHaveDupBase") { distributed BY hash(k1) buckets 3 properties("replication_num" = "1"); """ + sql """alter table agg_have_dup_base modify column k1 set stats ('row_count'='5');""" sql "insert into agg_have_dup_base select 1,1,1,'a';" sql "insert into agg_have_dup_base select 2,2,2,'b';" diff --git a/regression-test/suites/nereids_syntax_p0/mv/newMv/case_ignore.groovy b/regression-test/suites/nereids_syntax_p0/mv/newMv/case_ignore.groovy index 91293bb747aa64..e2f334566e9c24 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/newMv/case_ignore.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/newMv/case_ignore.groovy @@ -36,6 +36,7 @@ suite ("case_ignore") { sql "insert into case_ignore select 2,2,2,'b';" sql "insert into case_ignore select 3,-3,null,'c';" + createMV ("create materialized view k12a as select K1,abs(K2) from case_ignore;") sleep(3000) @@ -55,6 +56,7 @@ suite ("case_ignore") { order_qt_select_mv "select K1,abs(K2) from case_ignore order by K1;" sql """set enable_stats=true;""" + sql """alter table case_ignore modify column k1 set stats ('row_count'='4');""" mv_rewrite_success("select k1,abs(k2) from case_ignore order by k1;", "k12a") mv_rewrite_success("select K1,abs(K2) from case_ignore order by K1;", "k12a") diff --git a/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_gb_mv_abs.groovy b/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_gb_mv_abs.groovy index 132a4c3f054571..3537777d312c58 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_gb_mv_abs.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_gb_mv_abs.groovy @@ -57,6 +57,7 @@ suite ("dup_gb_mv_abs") { order_qt_select_mv_sub "select sum(abs(k2)) from dup_gb_mv_abs group by k1 order by k1;" sql """set enable_stats=true;""" + sql """alter table dup_gb_mv_abs modify column k1 set stats ('row_count'='4');""" mv_rewrite_success("select k1,sum(abs(k2)) from dup_gb_mv_abs group by k1;", "k12sa") mv_rewrite_success("select sum(abs(k2)) from dup_gb_mv_abs group by k1;", "k12sa") diff --git a/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_gb_mv_plus.groovy b/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_gb_mv_plus.groovy index 45271415b98ef5..2e6e9c6b627850 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_gb_mv_plus.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_gb_mv_plus.groovy @@ -57,6 +57,7 @@ suite ("dup_gb_mv_plus") { order_qt_select_mv_sub "select sum(k2+1) from dup_gb_mv_plus group by k1 order by k1;" sql """set enable_stats=true;""" + sql """alter table dup_gb_mv_plus modify column k1 set stats ('row_count'='4');""" mv_rewrite_success("select k1,sum(k2+1) from dup_gb_mv_plus group by k1;", "k12sp") mv_rewrite_success("select sum(k2+1) from dup_gb_mv_plus group by k1;", "k12sp") diff --git a/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_mv_abs.groovy b/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_mv_abs.groovy index c4770d6dcd0262..6cef52fd96edc7 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_mv_abs.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_mv_abs.groovy @@ -31,7 +31,6 @@ suite ("dup_mv_abs") { distributed BY hash(k1) buckets 3 properties("replication_num" = "1"); """ - sql "insert into dup_mv_abs select 1,1,1,'a';" sql "insert into dup_mv_abs select 2,2,2,'b';" sql "insert into dup_mv_abs select 3,-3,null,'c';" @@ -69,6 +68,7 @@ suite ("dup_mv_abs") { order_qt_select_group_mv_not "select sum(abs(k2)) from dup_mv_abs group by k3 order by k3;" sql """set enable_stats=true;""" + sql """alter table dup_mv_abs modify column k1 set stats ('row_count'='4');""" mv_rewrite_success("select k1,abs(k2) from dup_mv_abs order by k1;", "k12a") mv_rewrite_success("select abs(k2) from dup_mv_abs order by k1;", "k12a") diff --git a/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_mv_bin.groovy b/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_mv_bin.groovy index 788647ae224d98..7fd22f5b71892c 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_mv_bin.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_mv_bin.groovy @@ -69,6 +69,7 @@ suite ("dup_mv_bin") { order_qt_select_group_mv_not "select group_concat(bin(k2)) from dup_mv_bin group by k3 order by k3;" sql """set enable_stats=true;""" + sql """alter table dup_mv_bin modify column k1 set stats ('row_count'='4');""" mv_rewrite_success("select k1,bin(k2) from dup_mv_bin order by k1;", "k12b") mv_rewrite_success("select bin(k2) from dup_mv_bin order by k1;", "k12b") diff --git a/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_mv_bm_hash.groovy b/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_mv_bm_hash.groovy index d049a0aeb7c1b0..f5784e7a5fdaa6 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_mv_bm_hash.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_mv_bm_hash.groovy @@ -45,6 +45,7 @@ suite ("dup_mv_bm_hash") { order_qt_select_mv "select bitmap_union_count(to_bitmap(k2)) from dup_mv_bm_hash group by k1 order by k1;" sql """set enable_stats=true;""" + sql """alter table dup_mv_bm_hash modify column k1 set stats ('row_count'='5');""" mv_rewrite_success("select bitmap_union_count(to_bitmap(k2)) from dup_mv_bm_hash group by k1 order by k1;", "dup_mv_bm_hash_mv1") createMV("create materialized view dup_mv_bm_hash_mv2 as select k1,bitmap_union(bitmap_hash(k3)) from dup_mv_bm_hash group by k1;") diff --git a/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_mv_plus.groovy b/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_mv_plus.groovy index b661e75ad967ab..d96d2e07843dd9 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_mv_plus.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_mv_plus.groovy @@ -46,7 +46,6 @@ suite ("dup_mv_plus") { sql "analyze table dup_mv_plus with sync;" sql """set enable_stats=false;""" - order_qt_select_star "select * from dup_mv_plus order by k1;" mv_rewrite_success("select k1,k2+1 from dup_mv_plus order by k1;", "k12p") @@ -86,6 +85,7 @@ suite ("dup_mv_plus") { order_qt_select_mv "select k1,k2+1 from dup_mv_plus order by k2;" sql """set enable_stats=true;""" + sql """alter table dup_mv_plus modify column k1 set stats ('row_count'='4');""" mv_rewrite_success("select k1,k2+1 from dup_mv_plus order by k1;", "k12p") diff --git a/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_mv_year.groovy b/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_mv_year.groovy index c5098c55074c7d..95902c27866481 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_mv_year.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_mv_year.groovy @@ -45,6 +45,7 @@ suite ("dup_mv_year") { order_qt_select_mv "select k1,year(k2) from dup_mv_year order by k1;" sql """set enable_stats=true;""" + sql """alter table dup_mv_year modify column k1 set stats ('row_count'='4');""" mv_rewrite_success("select k1,year(k2) from dup_mv_year order by k1;", "k12y") createMV "create materialized view k13y as select k1,year(k3) from dup_mv_year;" diff --git a/regression-test/suites/nereids_syntax_p0/mv/newMv/multi_slot1.groovy b/regression-test/suites/nereids_syntax_p0/mv/newMv/multi_slot1.groovy index 7e317d501bea07..d0fa214e98f042 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/newMv/multi_slot1.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/newMv/multi_slot1.groovy @@ -52,5 +52,6 @@ suite ("multi_slot1") { order_qt_select_mv "select abs(k1)+k2+1,abs(k2+2)+k3+3 from multi_slot1 order by abs(k1)+k2+1,abs(k2+2)+k3+3;" sql """set enable_stats=true;""" + sql """alter table multi_slot1 modify column k1 set stats ('row_count'='4');""" mv_rewrite_success("select abs(k1)+k2+1,abs(k2+2)+k3+3 from multi_slot1 order by abs(k1)+k2+1,abs(k2+2)+k3+3", "k1a2p2ap3p") } diff --git a/regression-test/suites/nereids_syntax_p0/mv/newMv/multi_slot2.groovy b/regression-test/suites/nereids_syntax_p0/mv/newMv/multi_slot2.groovy index 6bbded534a77ee..0fa36b77b44114 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/newMv/multi_slot2.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/newMv/multi_slot2.groovy @@ -65,6 +65,7 @@ suite ("multi_slot2") { order_qt_select_base "select abs(k1)+k2+1,sum(abs(k2+2)+k3+3) from multi_slot2 group by abs(k1)+k2 order by abs(k1)+k2;" sql """set enable_stats=true;""" + sql """alter table multi_slot2 modify column k1 set stats ('row_count'='4');""" mv_rewrite_success("select abs(k1)+k2+1,sum(abs(k2+2)+k3+3) from multi_slot2 group by abs(k1)+k2+1 order by abs(k1)+k2+1", "k1a2p2ap3ps") diff --git a/regression-test/suites/nereids_syntax_p0/mv/newMv/multi_slot3.groovy b/regression-test/suites/nereids_syntax_p0/mv/newMv/multi_slot3.groovy index c6016296e8b352..51cc90aaf6ae26 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/newMv/multi_slot3.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/newMv/multi_slot3.groovy @@ -53,5 +53,6 @@ suite ("multi_slot3") { order_qt_select_mv "select k1+1,abs(k2+2)+k3+3 from multi_slot3 order by k1+1;" sql """set enable_stats=true;""" + sql """alter table multi_slot3 modify column k1 set stats ('row_count'='4');""" mv_rewrite_success("select k1+1,abs(k2+2)+k3+3 from multi_slot3 order by k1+1;", "k1p2ap3p") } diff --git a/regression-test/suites/nereids_syntax_p0/mv/newMv/multi_slot4.groovy b/regression-test/suites/nereids_syntax_p0/mv/newMv/multi_slot4.groovy index a09f3937aee302..49aba77aad05ff 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/newMv/multi_slot4.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/newMv/multi_slot4.groovy @@ -56,5 +56,6 @@ suite ("multi_slot4") { order_qt_select_mv "select k1+1,sum(abs(k2+2)+k3+3) from multi_slot4 group by k1+1 order by k1+1;" sql """set enable_stats=true;""" + sql """alter table multi_slot4 modify column k1 set stats ('row_count'='5');""" mv_rewrite_success("select k1+1,sum(abs(k2+2)+k3+3) from multi_slot4 group by k1+1 order by k1+1;", "k1p2ap3ps") } diff --git a/regression-test/suites/nereids_syntax_p0/mv/newMv/multi_slot5.groovy b/regression-test/suites/nereids_syntax_p0/mv/newMv/multi_slot5.groovy index 55ea991913cb3e..ebe6c8e57800a4 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/newMv/multi_slot5.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/newMv/multi_slot5.groovy @@ -58,5 +58,6 @@ suite ("multi_slot5") { order_qt_select_mv "select k1,version() from multi_slot5 order by k1;" sql """set enable_stats=true;""" + sql """alter table multi_slot5 modify column k1 set stats ('row_count'='5');""" mv_rewrite_success("select k1,k2+k3 from multi_slot5 order by k1;", "k123p") } diff --git a/regression-test/suites/nereids_syntax_p0/mv/newMv/multi_slot6.groovy b/regression-test/suites/nereids_syntax_p0/mv/newMv/multi_slot6.groovy index 9201a1cea4d321..2ed66fc8477a37 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/newMv/multi_slot6.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/newMv/multi_slot6.groovy @@ -84,5 +84,6 @@ suite ("multi_slot6") { order_qt_select_mv "select abs(k1)+k2+1,abs(k2+2)+k3+3 from multi_slot6 order by abs(k1)+k2+1,abs(k2+2)+k3+3;" sql """set enable_stats=true;""" + sql """alter table multi_slot6 modify column k1 set stats ('row_count'='4');""" mv_rewrite_success("select abs(k1)+k2+1,abs(k2+2)+k3+3 from multi_slot6 order by abs(k1)+k2+1,abs(k2+2)+k3+3", "k1a2p2ap3p") } diff --git a/regression-test/suites/nereids_syntax_p0/mv/newMv/mv_with_view.groovy b/regression-test/suites/nereids_syntax_p0/mv/newMv/mv_with_view.groovy index 236bde3bf6c828..60e7f6fefed232 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/newMv/mv_with_view.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/newMv/mv_with_view.groovy @@ -67,6 +67,7 @@ suite ("mv_with_view") { qt_select_mv "select * from v_k124 order by k1;" sql """set enable_stats=true;""" + sql """alter table d_table modify column k1 set stats ('row_count'='4');""" mv_rewrite_fail("select * from d_table order by k1;", "k312") sql """ diff --git a/regression-test/suites/nereids_syntax_p0/mv/newMv/single_slot.groovy b/regression-test/suites/nereids_syntax_p0/mv/newMv/single_slot.groovy index d6cd0d5a24421a..387c155a5e45c8 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/newMv/single_slot.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/newMv/single_slot.groovy @@ -47,7 +47,6 @@ suite ("single_slot") { sql "analyze table single_slot with sync;" sql """set enable_stats=false;""" - order_qt_select_star "select * from single_slot order by k1;" explain { @@ -58,5 +57,6 @@ suite ("single_slot") { order_qt_select_mv "select abs(k1)+1 t,sum(abs(k2+1)) from single_slot group by t order by t;" sql """set enable_stats=true;""" + sql """alter table single_slot modify column k1 set stats ('row_count'='4');""" mv_rewrite_success("select abs(k1)+1 t,sum(abs(k2+1)) from single_slot group by t order by t;", "k1ap2spa") } diff --git a/regression-test/suites/nereids_syntax_p0/mv/newMv/sum_devide_count.groovy b/regression-test/suites/nereids_syntax_p0/mv/newMv/sum_devide_count.groovy index 031560a41ef502..be8bc50532ff81 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/newMv/sum_devide_count.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/newMv/sum_devide_count.groovy @@ -62,7 +62,7 @@ suite ("sum_devide_count") { order_qt_select_mv "select sum(k2)/count(k2) from sum_devide_count;" sql """set enable_stats=true;""" - + sql """alter table sum_devide_count modify column k1 set stats ('row_count'='5');""" mv_rewrite_success("select k1,k4,sum(k2)/count(k2) from sum_devide_count group by k1,k4 order by k1,k4;", "kavg") mv_rewrite_success("select k1,sum(k2)/count(k2) from sum_devide_count group by k1 order by k1;", "kavg") diff --git a/regression-test/suites/nereids_syntax_p0/mv/newMv/unique_mv.groovy b/regression-test/suites/nereids_syntax_p0/mv/newMv/unique_mv.groovy index df390b26551054..1ebaa8884ab383 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/newMv/unique_mv.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/newMv/unique_mv.groovy @@ -47,6 +47,7 @@ suite ("unique_mv") { mv_rewrite_success("SELECT * FROM c5816_t WHERE call_uuid='adc';", "mv_1") sql """set enable_stats=true;""" + sql """alter table c5816_t modify column org_id set stats ('row_count'='1');""" mv_rewrite_success("SELECT * FROM c5816_t WHERE call_uuid='adc';", "mv_1") } diff --git a/regression-test/suites/nereids_syntax_p0/mv/ut/MVMultiUsage.groovy b/regression-test/suites/nereids_syntax_p0/mv/ut/MVMultiUsage.groovy index 9a5e42034be3cb..2b6b0dd354e4cd 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/ut/MVMultiUsage.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/ut/MVMultiUsage.groovy @@ -37,6 +37,7 @@ suite ("MVMultiUsage") { sql """insert into MVMultiUsage values("2020-01-02",2,"b",2,2,2);""" sql """insert into MVMultiUsage values("2020-01-03",3,"c",3,3,3);""" + createMV("create materialized view MVMultiUsage_mv as select deptno, empid, salary from MVMultiUsage order by deptno;") sleep(3000) @@ -58,6 +59,8 @@ suite ("MVMultiUsage") { order_qt_select_mv "select * from (select deptno, empid from MVMultiUsage where deptno>100) A join (select deptno, empid from MVMultiUsage where deptno >200) B using (deptno) order by 1;" sql """set enable_stats=true;""" + sql """alter table MVMultiUsage modify column time_col set stats ('row_count'='4');""" + mv_rewrite_fail("select * from MVMultiUsage order by empid;", "MVMultiUsage_mv") explain { sql("select * from (select deptno, empid from MVMultiUsage where deptno>100) A join (select deptno, empid from MVMultiUsage where deptno >200) B using (deptno);") diff --git a/regression-test/suites/nereids_syntax_p0/mv/ut/MVWithAs.groovy b/regression-test/suites/nereids_syntax_p0/mv/ut/MVWithAs.groovy index 246c97022e9e85..86a85dcc647443 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/ut/MVWithAs.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/ut/MVWithAs.groovy @@ -30,6 +30,7 @@ suite ("MVWithAs") { partition by range (time_col) (partition p1 values less than MAXVALUE) distributed by hash(time_col) buckets 3 properties('replication_num' = '1'); """ + sql """insert into MVWithAs values("2020-01-01",1,"a",1);""" sql """insert into MVWithAs values("2020-01-01",1,"a",1);""" sql """insert into MVWithAs values("2020-01-01",1,"a",1);""" @@ -53,6 +54,8 @@ suite ("MVWithAs") { order_qt_select_mv "select count(tag_id) from MVWithAs t;" sql """set enable_stats=true;""" + sql """alter table MVWithAs modify column time_col set stats ('row_count'='7');""" + mv_rewrite_fail("select * from MVWithAs order by time_col;", "MVWithAs_mv") mv_rewrite_success("select count(tag_id) from MVWithAs t;", "MVWithAs_mv") diff --git a/regression-test/suites/nereids_syntax_p0/mv/ut/aggCDInBitmap.groovy b/regression-test/suites/nereids_syntax_p0/mv/ut/aggCDInBitmap.groovy index 79ede78f0d3752..f45d523d423527 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/ut/aggCDInBitmap.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/ut/aggCDInBitmap.groovy @@ -34,7 +34,6 @@ suite ("aggCDInBitmap") { sql "analyze table aggCDInBitmap with sync;" sql """set enable_stats=false;""" - order_qt_select_star "select * from aggCDInBitmap order by 1;" @@ -45,9 +44,10 @@ suite ("aggCDInBitmap") { order_qt_select_mv "select k1, count(distinct v1) from aggCDInBitmap group by k1 order by k1;" sql """set enable_stats=true;""" + sql """alter table aggCDInBitmap modify column k1 set stats ('row_count'='3');""" explain { sql("select k1, count(distinct v1) from aggCDInBitmap group by k1;") contains "bitmap_union_count" } - + } diff --git a/regression-test/suites/nereids_syntax_p0/mv/ut/aggMVCalcAggFun.groovy b/regression-test/suites/nereids_syntax_p0/mv/ut/aggMVCalcAggFun.groovy index 1a7ccd851503d8..a234661150e732 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/ut/aggMVCalcAggFun.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/ut/aggMVCalcAggFun.groovy @@ -38,7 +38,6 @@ suite ("aggMVCalcAggFun") { sql """insert into aggMVCalcAggFun values("2020-01-02",2,"b",2,2,2);""" sql """insert into aggMVCalcAggFun values("2020-01-03",3,"c",3,3,3);""" - createMV("create materialized view aggMVCalcAggFunMv as select deptno, empid, sum(salary) from aggMVCalcAggFun group by empid, deptno;") sleep(3000) @@ -55,6 +54,7 @@ suite ("aggMVCalcAggFun") { order_qt_select_mv "select deptno, sum(salary + 1) from aggMVCalcAggFun where deptno > 10 group by deptno order by deptno;" sql """set enable_stats=true;""" + sql """alter table aggMVCalcAggFun modify column time_col set stats ('row_count'='4');""" mv_rewrite_fail("select * from aggMVCalcAggFun order by empid;", "aggMVCalcAggFunMv")\ mv_rewrite_fail("select deptno, sum(salary + 1) from aggMVCalcAggFun where deptno > 10 group by deptno;", "aggMVCalcAggFunMv") diff --git a/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV1.groovy b/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV1.groovy index 0cb12cbcde0b51..c29eec40c8da93 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV1.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV1.groovy @@ -54,6 +54,7 @@ suite ("aggOnAggMV1") { order_qt_select_mv "select sum(salary), deptno from aggOnAggMV1 group by deptno order by deptno;" sql """set enable_stats=true;""" + sql """alter table aggOnAggMV1 modify column time_col set stats ('row_count'='4');""" mv_rewrite_fail("select * from aggOnAggMV1 order by empid;", "aggOnAggMV1_mv") mv_rewrite_success("select sum(salary), deptno from aggOnAggMV1 group by deptno order by deptno;", "aggOnAggMV1_mv") diff --git a/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV10.groovy b/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV10.groovy index bd826537ff88f7..e38a27c1db3c53 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV10.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV10.groovy @@ -54,6 +54,7 @@ suite ("aggOnAggMV10") { order_qt_select_mv "select deptno, commission, sum(salary) + 1 from aggOnAggMV10 group by rollup (deptno, commission) order by 1,2;" sql """set enable_stats=true;""" + sql """alter table aggOnAggMV10 modify column time_col set stats ('row_count'='4');""" mv_rewrite_fail("select * from aggOnAggMV10 order by empid;", "aggOnAggMV10_mv") mv_rewrite_success("select deptno, commission, sum(salary) + 1 from aggOnAggMV10 group by rollup (deptno, commission);", diff --git a/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV11.groovy b/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV11.groovy index 793389ec68e4ae..b0f379b19476b8 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV11.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV11.groovy @@ -33,6 +33,7 @@ suite ("aggOnAggMV11") { partition by range (time_col) (partition p1 values less than MAXVALUE) distributed by hash(time_col) buckets 3 properties('replication_num' = '1'); """ + sql """insert into aggOnAggMV11 values("2020-01-01",1,"a",1,1,1);""" sql """insert into aggOnAggMV11 values("2020-01-02",2,"b",2,2,2);""" sql """insert into aggOnAggMV11 values("2020-01-03",3,"c",3,3,3);""" @@ -54,6 +55,8 @@ suite ("aggOnAggMV11") { order_qt_select_mv "select deptno, count(salary) + count(1) from aggOnAggMV11 group by deptno order by 1;" sql """set enable_stats=true;""" + sql """alter table aggOnAggMV11 modify column time_col set stats ('row_count'='4');""" + mv_rewrite_fail("select * from aggOnAggMV11 order by empid;", "aggOnAggMV11_mv") mv_rewrite_fail("select deptno, count(salary) + count(1) from aggOnAggMV11 group by deptno;", diff --git a/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV2.groovy b/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV2.groovy index 32d897939ea652..179e22584d32a1 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV2.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV2.groovy @@ -56,6 +56,7 @@ suite ("aggOnAggMV2") { order_qt_select_mv "select * from (select deptno, sum(salary) as sum_salary from aggOnAggMV2 group by deptno) a where (sum_salary * 2) > 3 order by deptno ;" sql """set enable_stats=true;""" + sql """alter table aggOnAggMV2 modify column time_col set stats ('row_count'='3');""" mv_rewrite_fail("select * from aggOnAggMV2 order by empid;", "aggOnAggMV2_mv") mv_rewrite_success("select * from (select deptno, sum(salary) as sum_salary from aggOnAggMV2 group by deptno) a where (sum_salary * 2) > 3 order by deptno ;", "aggOnAggMV2_mv") diff --git a/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV3.groovy b/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV3.groovy index dfd0ce3e67503c..0b54345099bcb9 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV3.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV3.groovy @@ -56,6 +56,7 @@ suite ("aggOnAggMV3") { order_qt_select_mv "select commission, sum(salary) from aggOnAggMV3 where commission * (deptno + commission) = 100 group by commission order by commission;" sql """set enable_stats=true;""" + sql """alter table aggOnAggMV3 modify column time_col set stats ('row_count'='5');""" mv_rewrite_fail("select * from aggOnAggMV3 order by empid;", "aggOnAggMV3_mv") mv_rewrite_success("select commission, sum(salary) from aggOnAggMV3 where commission * (deptno + commission) = 100 group by commission order by commission;", diff --git a/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV5.groovy b/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV5.groovy index 481d8f19b5a133..d90057b8bf5120 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV5.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV5.groovy @@ -33,6 +33,8 @@ suite ("aggOnAggMV5") { partition by range (time_col) (partition p1 values less than MAXVALUE) distributed by hash(time_col) buckets 3 properties('replication_num' = '1'); """ + sql """alter table aggOnAggMV5 modify column time_col set stats ('row_count'='4');""" + sql """insert into aggOnAggMV5 values("2020-01-01",1,"a",1,1,1);""" sql """insert into aggOnAggMV5 values("2020-01-02",2,"b",2,2,2);""" sql """insert into aggOnAggMV5 values("2020-01-03",3,"c",3,3,3);""" diff --git a/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV6.groovy b/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV6.groovy index abd66e1ab13ccd..7e2e87718b8e76 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV6.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV6.groovy @@ -54,6 +54,7 @@ suite ("aggOnAggMV6") { order_qt_select_mv "select * from (select deptno, sum(salary) as sum_salary from aggOnAggMV6 where deptno>=20 group by deptno) a where sum_salary>10 order by 1;" sql """set enable_stats=true;""" + sql """alter table aggOnAggMV6 modify column time_col set stats ('row_count'='4');""" mv_rewrite_fail("select * from aggOnAggMV6 order by empid;", "aggOnAggMV6_mv") mv_rewrite_success("select * from (select deptno, sum(salary) as sum_salary from aggOnAggMV6 where deptno>=20 group by deptno) a where sum_salary>10;", diff --git a/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV7.groovy b/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV7.groovy index 53b75d03d85167..386871a8a5a98f 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV7.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV7.groovy @@ -54,6 +54,7 @@ suite ("aggOnAggMV7") { order_qt_select_mv "select deptno, sum(salary) from aggOnAggMV7 where deptno>=20 group by deptno order by 1;" sql """set enable_stats=true;""" + sql """alter table aggOnAggMV7 modify column time_col set stats ('row_count'='4');""" mv_rewrite_fail("select * from aggOnAggMV7 order by empid;", "aggOnAggMV7_mv") mv_rewrite_success("select deptno, sum(salary) from aggOnAggMV7 where deptno>=20 group by deptno;", diff --git a/regression-test/suites/nereids_syntax_p0/mv/ut/bitmapUnionIn.groovy b/regression-test/suites/nereids_syntax_p0/mv/ut/bitmapUnionIn.groovy index 13e2097cdb46ba..8b2697cb1f78f7 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/ut/bitmapUnionIn.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/ut/bitmapUnionIn.groovy @@ -30,6 +30,7 @@ suite ("bitmapUnionIn") { partition by range (time_col) (partition p1 values less than MAXVALUE) distributed by hash(time_col) buckets 3 properties('replication_num' = '1'); """ + sql """insert into bitmapUnionIn values("2020-01-01",1,"a",1);""" sql """insert into bitmapUnionIn values("2020-01-02",2,"b",2);""" @@ -50,6 +51,8 @@ suite ("bitmapUnionIn") { order_qt_select_mv "select user_id, bitmap_union_count(to_bitmap(tag_id)) a from bitmapUnionIn group by user_id having a>1 order by a;" sql """set enable_stats=true;""" + sql """alter table bitmapUnionIn modify column time_col set stats ('row_count'='3');""" + mv_rewrite_fail("select * from bitmapUnionIn order by time_col;", "bitmapUnionIn_mv") mv_rewrite_success("select user_id, bitmap_union_count(to_bitmap(tag_id)) a from bitmapUnionIn group by user_id having a>1 order by a;", diff --git a/regression-test/suites/nereids_syntax_p0/mv/ut/distinctQuery.groovy b/regression-test/suites/nereids_syntax_p0/mv/ut/distinctQuery.groovy index bfd7c159dade89..eb314fecccbb4b 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/ut/distinctQuery.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/ut/distinctQuery.groovy @@ -37,6 +37,8 @@ suite ("distinctQuery") { sql """insert into distinctQuery values("2020-01-02",2,"b",2,2,2);""" sql """insert into distinctQuery values("2020-01-03",3,"c",3,3,3);""" + sql """alter table distinctQuery modify column time_col set stats ('row_count'='5');""" + createMV("create materialized view distinctQuery_mv as select deptno, count(salary) from distinctQuery group by deptno;") createMV("create materialized view distinctQuery_mv2 as select empid, deptno, count(salary) from distinctQuery group by empid, deptno;") diff --git a/regression-test/suites/nereids_syntax_p0/mv/ut/incMVReInSub.groovy b/regression-test/suites/nereids_syntax_p0/mv/ut/incMVReInSub.groovy index c28dac1d227aab..52abff46678bb0 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/ut/incMVReInSub.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/ut/incMVReInSub.groovy @@ -31,6 +31,7 @@ suite ("incMVReInSub") { partition by range (time_col) (partition p1 values less than MAXVALUE) distributed by hash(time_col) buckets 3 properties('replication_num' = '1'); """ + sql """insert into incMVReInSub values("2020-01-01",1,"a",1);""" sql """insert into incMVReInSub values("2020-01-02",2,"b",2);""" @@ -52,6 +53,8 @@ suite ("incMVReInSub") { order_qt_select_mv "select user_id, bitmap_union(to_bitmap(tag_id)) from incMVReInSub where user_name in (select user_name from incMVReInSub group by user_name having bitmap_union_count(to_bitmap(tag_id)) >1 ) group by user_id order by user_id;" sql """set enable_stats=true;""" + sql """alter table incMVReInSub modify column time_col set stats ('row_count'='3');""" + mv_rewrite_fail("select * from incMVReInSub order by time_col;", "incMVReInSub_mv") mv_rewrite_fail("select user_id, bitmap_union(to_bitmap(tag_id)) from incMVReInSub where user_name in (select user_name from incMVReInSub group by user_name having bitmap_union_count(to_bitmap(tag_id)) >1 ) group by user_id order by user_id;", diff --git a/regression-test/suites/nereids_syntax_p0/mv/ut/incRewriteCD.groovy b/regression-test/suites/nereids_syntax_p0/mv/ut/incRewriteCD.groovy index ee8e688c0e7313..6172773ca7a43b 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/ut/incRewriteCD.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/ut/incRewriteCD.groovy @@ -31,6 +31,7 @@ suite ("incRewriteCD") { partition by range (time_col) (partition p1 values less than MAXVALUE) distributed by hash(time_col) buckets 3 properties('replication_num' = '1'); """ + sql """insert into incRewriteCD values("2020-01-01",1,"a",1);""" sql """insert into incRewriteCD values("2020-01-02",2,"b",2);""" @@ -50,6 +51,8 @@ suite ("incRewriteCD") { order_qt_select_mv "select user_name, count(distinct tag_id) from incRewriteCD group by user_name order by user_name;" sql """set enable_stats=true;""" + sql """alter table incRewriteCD modify column time_col set stats ('row_count'='3');""" + mv_rewrite_fail("select * from incRewriteCD order by time_col;", "incRewriteCD_mv") mv_rewrite_fail("select user_name, count(distinct tag_id) from incRewriteCD group by user_name;", "incRewriteCD_mv") diff --git a/regression-test/suites/nereids_syntax_p0/mv/ut/joinOnCalcToJoin.groovy b/regression-test/suites/nereids_syntax_p0/mv/ut/joinOnCalcToJoin.groovy index 8ed3e09bdf0df3..378731dbb97a62 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/ut/joinOnCalcToJoin.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/ut/joinOnCalcToJoin.groovy @@ -35,6 +35,8 @@ suite ("joinOnCalcToJoin") { sql """insert into joinOnCalcToJoin values("2020-01-03",3,"c",3,3,3);""" sql """insert into joinOnCalcToJoin values("2020-01-02",2,"b",2,7,2);""" + sql """alter table joinOnCalcToJoin modify column time_col set stats ('row_count'='3');""" + sql """ DROP TABLE IF EXISTS joinOnCalcToJoin_1; """ sql """ create table joinOnCalcToJoin_1 ( @@ -49,6 +51,7 @@ suite ("joinOnCalcToJoin") { sql """insert into joinOnCalcToJoin_1 values("2020-01-03",3,"c",3);""" sql """insert into joinOnCalcToJoin_1 values("2020-01-02",2,"b",1);""" + createMV("create materialized view joinOnLeftPToJoin_mv as select empid, deptno from joinOnCalcToJoin;") sleep(3000) createMV("create materialized view joinOnLeftPToJoin_1_mv as select deptno, cost from joinOnCalcToJoin_1;") @@ -62,6 +65,8 @@ suite ("joinOnCalcToJoin") { ["joinOnLeftPToJoin_mv", "joinOnLeftPToJoin_1_mv"]) sql """set enable_stats=true;""" + sql """alter table joinOnCalcToJoin_1 modify column time_col set stats ('row_count'='3');""" + mv_rewrite_all_success("select * from (select empid, deptno from joinOnCalcToJoin where empid = 0) A join (select deptno, cost from joinOnCalcToJoin_1 where deptno > 0) B on A.deptno = B.deptno;", ["joinOnLeftPToJoin_mv", "joinOnLeftPToJoin_1_mv"]) } diff --git a/regression-test/suites/nereids_syntax_p0/mv/ut/joinOnLeftPToJoin.groovy b/regression-test/suites/nereids_syntax_p0/mv/ut/joinOnLeftPToJoin.groovy index f2f7a014d98fc8..69d1c39931c17f 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/ut/joinOnLeftPToJoin.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/ut/joinOnLeftPToJoin.groovy @@ -31,6 +31,8 @@ suite ("joinOnLeftPToJoin") { partition by range (time_col) (partition p1 values less than MAXVALUE) distributed by hash(time_col) buckets 3 properties('replication_num' = '1'); """ + sql """alter table joinOnLeftPToJoin modify column time_col set stats ('row_count'='3');""" + sql """insert into joinOnLeftPToJoin values("2020-01-02",2,"b",2,2,2);""" sql """insert into joinOnLeftPToJoin values("2020-01-03",3,"c",3,3,3);""" sql """insert into joinOnLeftPToJoin values("2020-01-02",2,"b",2,7,2);""" @@ -45,6 +47,7 @@ suite ("joinOnLeftPToJoin") { partition by range (time_col) (partition p1 values less than MAXVALUE) distributed by hash(time_col) buckets 3 properties('replication_num' = '1'); """ + sql """insert into joinOnLeftPToJoin_1 values("2020-01-02",2,"b",2);""" sql """insert into joinOnLeftPToJoin_1 values("2020-01-03",3,"c",3);""" sql """insert into joinOnLeftPToJoin_1 values("2020-01-02",2,"b",1);""" @@ -64,6 +67,8 @@ suite ("joinOnLeftPToJoin") { order_qt_select_mv "select * from (select deptno , sum(salary) from joinOnLeftPToJoin group by deptno) A join (select deptno, max(cost) from joinOnLeftPToJoin_1 group by deptno ) B on A.deptno = B.deptno order by A.deptno;" sql """set enable_stats=true;""" + sql """alter table joinOnLeftPToJoin_1 modify column time_col set stats ('row_count'='3');""" + mv_rewrite_all_success("select * from (select deptno , sum(salary) from joinOnLeftPToJoin group by deptno) A join (select deptno, max(cost) from joinOnLeftPToJoin_1 group by deptno ) B on A.deptno = B.deptno;", ["joinOnLeftPToJoin_mv", "joinOnLeftPToJoin_1_mv"]) } diff --git a/regression-test/suites/nereids_syntax_p0/mv/ut/onlyGroupBy.groovy b/regression-test/suites/nereids_syntax_p0/mv/ut/onlyGroupBy.groovy index 698f4ae13f328d..e36ae20e9bcb11 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/ut/onlyGroupBy.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/ut/onlyGroupBy.groovy @@ -33,6 +33,7 @@ suite ("onlyGroupBy") { partition by range (time_col) (partition p1 values less than MAXVALUE) distributed by hash(time_col) buckets 3 properties('replication_num' = '1'); """ + sql """insert into onlyGroupBy values("2020-01-01",1,"a",1,1,1);""" sql """insert into onlyGroupBy values("2020-01-02",2,"b",2,2,2);""" sql """insert into onlyGroupBy values("2020-01-03",3,"c",3,3,3);""" @@ -47,5 +48,7 @@ suite ("onlyGroupBy") { mv_rewrite_success("select deptno from onlyGroupBy group by deptno;", "onlyGroupBy_mv") sql """set enable_stats=true;""" + sql """alter table onlyGroupBy modify column time_col set stats ('row_count'='4');""" + mv_rewrite_success("select deptno from onlyGroupBy group by deptno;", "onlyGroupBy_mv") } diff --git a/regression-test/suites/nereids_syntax_p0/mv/ut/orderByOnPView.groovy b/regression-test/suites/nereids_syntax_p0/mv/ut/orderByOnPView.groovy index 7f455da429e6c3..21864f973ee8c0 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/ut/orderByOnPView.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/ut/orderByOnPView.groovy @@ -54,6 +54,8 @@ suite ("orderByOnPView") { order_qt_select_mv "select empid from orderByOnPView order by deptno;" sql """set enable_stats=true;""" + sql """alter table orderByOnPView modify column time_col set stats ('row_count'='4');""" + mv_rewrite_fail("select * from orderByOnPView where time_col='2020-01-01' order by empid;", "orderByOnPView_mv") mv_rewrite_success("select empid from orderByOnPView where deptno = 0 order by deptno;", "orderByOnPView_mv") diff --git a/regression-test/suites/nereids_syntax_p0/mv/ut/projectMV1.groovy b/regression-test/suites/nereids_syntax_p0/mv/ut/projectMV1.groovy index 384137471d827f..157a15498e717c 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/ut/projectMV1.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/ut/projectMV1.groovy @@ -33,6 +33,7 @@ suite ("projectMV1") { partition by range (time_col) (partition p1 values less than MAXVALUE) distributed by hash(time_col) buckets 3 properties('replication_num' = '1'); """ + sql """insert into projectMV1 values("2020-01-01",1,"a",1,1,1);""" sql """insert into projectMV1 values("2020-01-02",2,"b",2,2,2);""" @@ -52,6 +53,8 @@ suite ("projectMV1") { order_qt_select_mv "select empid, deptno from projectMV1 order by empid;" sql """set enable_stats=true;""" + sql """alter table projectMV1 modify column time_col set stats ('row_count'='3');""" + mv_rewrite_fail("select * from projectMV1 where time_col='2020-01-01' order by empid;", "projectMV1_mv") mv_rewrite_success("select empid, deptno from projectMV1 where deptno=0 order by empid;", "projectMV1_mv") diff --git a/regression-test/suites/nereids_syntax_p0/mv/ut/projectMV2.groovy b/regression-test/suites/nereids_syntax_p0/mv/ut/projectMV2.groovy index 672bda198b01aa..4d85fcc9c8aeae 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/ut/projectMV2.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/ut/projectMV2.groovy @@ -33,6 +33,7 @@ suite ("projectMV2") { partition by range (time_col) (partition p1 values less than MAXVALUE) distributed by hash(time_col) buckets 3 properties('replication_num' = '1'); """ + sql """insert into projectMV2 values("2020-01-01",1,"a",1,1,1);""" sql """insert into projectMV2 values("2020-01-02",2,"b",2,2,2);""" @@ -55,6 +56,8 @@ suite ("projectMV2") { order_qt_select_base "select name from projectMV2 where deptno -1 = 0 order by empid;" sql """set enable_stats=true;""" + sql """alter table projectMV2 modify column time_col set stats ('row_count'='3');""" + mv_rewrite_fail("select * from projectMV2 order by empid;", "projectMV2_mv") mv_rewrite_success("select empid + 1 from projectMV2 where deptno = 1 order by empid;", "projectMV2_mv") diff --git a/regression-test/suites/nereids_syntax_p0/mv/ut/projectMV3.groovy b/regression-test/suites/nereids_syntax_p0/mv/ut/projectMV3.groovy index 37b73a9a81aff4..9ad301fbbc6a97 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/ut/projectMV3.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/ut/projectMV3.groovy @@ -33,6 +33,7 @@ suite ("projectMV3") { partition by range (time_col) (partition p1 values less than MAXVALUE) distributed by hash(time_col) buckets 3 properties('replication_num' = '1'); """ + sql """insert into projectMV3 values("2020-01-01",1,"a",1,1,1);""" sql """insert into projectMV3 values("2020-01-02",2,"b",2,2,2);""" @@ -57,6 +58,8 @@ suite ("projectMV3") { order_qt_select_mv2 "select name from projectMV3 where deptno -1 = 0 order by empid;" sql """set enable_stats=true;""" + sql """alter table projectMV3 modify column time_col set stats ('row_count'='3');""" + mv_rewrite_fail("select * from projectMV3 order by empid;", "projectMV3_mv") mv_rewrite_success("select empid + 1, name from projectMV3 where deptno = 1 order by empid;", "projectMV3_mv") diff --git a/regression-test/suites/nereids_syntax_p0/mv/ut/projectMV4.groovy b/regression-test/suites/nereids_syntax_p0/mv/ut/projectMV4.groovy index ea87551dfa1249..ddcdada549ad8f 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/ut/projectMV4.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/ut/projectMV4.groovy @@ -33,6 +33,7 @@ suite ("projectMV4") { partition by range (time_col) (partition p1 values less than MAXVALUE) distributed by hash(time_col) buckets 3 properties('replication_num' = '1'); """ + sql """insert into projectMV4 values("2020-01-01",1,"a",1,1,1);""" sql """insert into projectMV4 values("2020-01-02",2,"b",2,2,2);""" @@ -57,6 +58,8 @@ suite ("projectMV4") { order_qt_select_base "select empid from projectMV4 where deptno > 1 and empid > 1 order by empid;" sql """set enable_stats=true;""" + sql """alter table projectMV4 modify column time_col set stats ('row_count'='3');""" + mv_rewrite_fail("select * from projectMV4 order by empid;", "projectMV4_mv") mv_rewrite_success("select name from projectMV4 where deptno > 1 and salary > 1 and name = 'a' order by name;", "projectMV4_mv") diff --git a/regression-test/suites/nereids_syntax_p0/mv/ut/subQuery.groovy b/regression-test/suites/nereids_syntax_p0/mv/ut/subQuery.groovy index 25300c9d602681..67312eed088ed4 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/ut/subQuery.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/ut/subQuery.groovy @@ -37,7 +37,6 @@ suite ("subQuery") { sql """insert into subQuery values("2020-01-02",2,"b",2,2,2);""" sql """insert into subQuery values("2020-01-03",3,"c",3,3,3);""" - createMV("create materialized view subQuery_mv as select deptno, empid from subQuery;") sleep(3000) @@ -60,6 +59,7 @@ suite ("subQuery") { */ sql """set enable_stats=true;""" + sql """alter table subQuery modify column time_col set stats ('row_count'='4');""" mv_rewrite_fail("select * from subQuery order by empid;", "subQuery_mv") } diff --git a/regression-test/suites/nereids_syntax_p0/mv/ut/unionDis.groovy b/regression-test/suites/nereids_syntax_p0/mv/ut/unionDis.groovy index 0f57b5c6c1687e..9feb1aa1c975f5 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/ut/unionDis.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/ut/unionDis.groovy @@ -58,6 +58,8 @@ suite ("unionDis") { order_qt_select_mv "select * from (select empid, deptno from unionDis where empid >1 union select empid, deptno from unionDis where empid <0) t order by 1;" sql """set enable_stats=true;""" + sql """alter table unionDis modify column time_col set stats ('row_count'='4');""" + mv_rewrite_fail("select * from unionDis order by empid;", "unionDis_mv") explain { diff --git a/regression-test/suites/nereids_syntax_p0/rollup/agg.groovy b/regression-test/suites/nereids_syntax_p0/rollup/agg.groovy index 0f682cacd8be96..3d65f3a85063d4 100644 --- a/regression-test/suites/nereids_syntax_p0/rollup/agg.groovy +++ b/regression-test/suites/nereids_syntax_p0/rollup/agg.groovy @@ -44,6 +44,7 @@ suite("agg") { AGGREGATE KEY (siteid,citycode,username) DISTRIBUTED BY HASH(siteid) BUCKETS 5 properties("replication_num" = "1"); """ + sql """alter table test_rollup_agg1 modify column siteid set stats ('row_count'='3');""" sql """ALTER TABLE ${tbName} ADD ROLLUP rollup_city(citycode, pv);""" int max_try_secs = 60 String res = "NOT_FINISHED" diff --git a/regression-test/suites/nereids_syntax_p0/rollup/agg_date.groovy b/regression-test/suites/nereids_syntax_p0/rollup/agg_date.groovy index b711728f769129..07af0da35098e1 100644 --- a/regression-test/suites/nereids_syntax_p0/rollup/agg_date.groovy +++ b/regression-test/suites/nereids_syntax_p0/rollup/agg_date.groovy @@ -44,6 +44,7 @@ suite("agg_date", "rollup") { AGGREGATE KEY (datek1, datetimek1, datetimek2, datetimek3) DISTRIBUTED BY HASH(datek1) BUCKETS 5 properties("replication_num" = "1"); """ + sql """alter table test_rollup_agg_date1 modify column datek1 set stats ('row_count'='2');""" sql """ALTER TABLE ${tbName} ADD ROLLUP rollup_date(datek1,datetimek2,datetimek1,datetimek3,datev1,datetimev1,datetimev2,datetimev3);""" int max_try_secs = 60 while (max_try_secs--) { diff --git a/regression-test/suites/nereids_syntax_p0/rollup/bitmap.groovy b/regression-test/suites/nereids_syntax_p0/rollup/bitmap.groovy index a3917dee28a75a..2407c70d8a451c 100644 --- a/regression-test/suites/nereids_syntax_p0/rollup/bitmap.groovy +++ b/regression-test/suites/nereids_syntax_p0/rollup/bitmap.groovy @@ -34,6 +34,7 @@ suite("bitmap", "rollup") { ) DISTRIBUTED BY HASH(k1) properties("replication_num" = "1"); """ + sql """alter table test_materialized_view_bitmap1 modify column k1 set stats ('row_count'='2');""" sql "CREATE MATERIALIZED VIEW test_neg as select k1,bitmap_union(to_bitmap(k2)), bitmap_union(to_bitmap(k3)) FROM ${tbName1} GROUP BY k1;" max_try_secs = 60 diff --git a/regression-test/suites/nereids_syntax_p0/rollup/date.groovy b/regression-test/suites/nereids_syntax_p0/rollup/date.groovy index 1415a442952ab6..138b8775778a6c 100644 --- a/regression-test/suites/nereids_syntax_p0/rollup/date.groovy +++ b/regression-test/suites/nereids_syntax_p0/rollup/date.groovy @@ -61,6 +61,7 @@ suite("date", "rollup") { mv_rewrite_success("SELECT store_id, max(sale_datetime3) FROM ${tbName1} GROUP BY store_id", "amt_max4") sql """set enable_stats=true;""" + sql """alter table test_materialized_view_date1 modify column record_id set stats ('row_count'='2');""" mv_rewrite_success("SELECT store_id, max(sale_date1) FROM ${tbName1} GROUP BY store_id", "amt_max1") mv_rewrite_success("SELECT store_id, max(sale_datetime1) FROM ${tbName1} GROUP BY store_id", "amt_max2") diff --git a/regression-test/suites/nereids_syntax_p0/rollup/hll/hll.groovy b/regression-test/suites/nereids_syntax_p0/rollup/hll/hll.groovy index 2ba3540832945a..5002c509a20617 100644 --- a/regression-test/suites/nereids_syntax_p0/rollup/hll/hll.groovy +++ b/regression-test/suites/nereids_syntax_p0/rollup/hll/hll.groovy @@ -46,6 +46,7 @@ suite("hll", "rollup") { "amt_count") sql """set enable_stats=true;""" + sql """alter table test_materialized_view_hll1 modify column record_id set stats ('row_count'='2');""" mv_rewrite_success("SELECT store_id, hll_union_agg(hll_hash(sale_amt)) FROM test_materialized_view_hll1 GROUP BY store_id;", "amt_count") } diff --git a/regression-test/suites/nereids_syntax_p0/rollup/hll_with_light_sc/hll_with_light_sc.groovy b/regression-test/suites/nereids_syntax_p0/rollup/hll_with_light_sc/hll_with_light_sc.groovy index 51599f294d1e62..d7dec7696d95eb 100644 --- a/regression-test/suites/nereids_syntax_p0/rollup/hll_with_light_sc/hll_with_light_sc.groovy +++ b/regression-test/suites/nereids_syntax_p0/rollup/hll_with_light_sc/hll_with_light_sc.groovy @@ -43,6 +43,7 @@ suite("hll_with_light_sc", "rollup") { "amt_count1") sql """set enable_stats=true;""" + sql """alter table test_materialized_view_hll_with_light_sc1 modify column record_id set stats ('row_count'='2');""" mv_rewrite_success("SELECT store_id, hll_union_agg(hll_hash(sale_amt)) FROM test_materialized_view_hll_with_light_sc1 GROUP BY store_id;", "amt_count1") } From b9afb48383f5e10b228f4e8d64cb16ce332ef6bf Mon Sep 17 00:00:00 2001 From: zhangstar333 Date: Fri, 29 Nov 2024 15:12:43 +0800 Subject: [PATCH 077/399] [log](function) print type name when input is invaild (#44758) ### What problem does this PR solve? Problem Summary: add log to print data type when input is invalid --- .../vec/functions/function_date_or_datetime_computation.h | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/be/src/vec/functions/function_date_or_datetime_computation.h b/be/src/vec/functions/function_date_or_datetime_computation.h index 90221e66c213a9..330ea75cba96c8 100644 --- a/be/src/vec/functions/function_date_or_datetime_computation.h +++ b/be/src/vec/functions/function_date_or_datetime_computation.h @@ -684,9 +684,10 @@ struct DateTimeAddIntervalImpl { block.replace_by_position(result, std::move(col_to)); } } else { - return Status::RuntimeError("Illegal column {} of first argument of function {}", - block.get_by_position(arguments[0]).column->get_name(), - Transform::name); + return Status::RuntimeError( + "Illegal column {} of first argument and type {} of function {}", + block.get_by_position(arguments[0]).column->get_name(), + block.get_by_position(arguments[0]).type->get_name(), Transform::name); } return Status::OK(); } From cb04281d63f475902062311c5daf6e155204856e Mon Sep 17 00:00:00 2001 From: Mryange Date: Fri, 29 Nov 2024 15:35:29 +0800 Subject: [PATCH 078/399] [minor](column) remove function "remove_low_cardinality" (#44766) remove function "remove_low_cardinality" --- be/src/vec/columns/column.h | 4 ---- be/src/vec/columns/column_const.cpp | 4 ---- be/src/vec/columns/column_const.h | 2 -- be/src/vec/functions/function.cpp | 8 -------- be/src/vec/functions/nullif.cpp | 8 -------- 5 files changed, 26 deletions(-) diff --git a/be/src/vec/columns/column.h b/be/src/vec/columns/column.h index 96408579a843c4..593666568aa375 100644 --- a/be/src/vec/columns/column.h +++ b/be/src/vec/columns/column.h @@ -100,10 +100,6 @@ class IColumn : public COW { */ virtual Ptr convert_column_if_overflow() { return get_ptr(); } - /// If column isn't ColumnLowCardinality, return itself. - /// If column is ColumnLowCardinality, transforms is to full column. - virtual Ptr convert_to_full_column_if_low_cardinality() const { return get_ptr(); } - /// If column isn't ColumnDictionary, return itself. /// If column is ColumnDictionary, transforms is to predicate column. virtual MutablePtr convert_to_predicate_column_if_dictionary() { return get_ptr(); } diff --git a/be/src/vec/columns/column_const.cpp b/be/src/vec/columns/column_const.cpp index fd05127f6d72aa..feeaec497295ba 100644 --- a/be/src/vec/columns/column_const.cpp +++ b/be/src/vec/columns/column_const.cpp @@ -72,10 +72,6 @@ ColumnPtr ColumnConst::convert_to_full_column() const { return data->replicate(Offsets(1, cast_set(s))); } -ColumnPtr ColumnConst::remove_low_cardinality() const { - return ColumnConst::create(data->convert_to_full_column_if_low_cardinality(), s); -} - ColumnPtr ColumnConst::filter(const Filter& filt, ssize_t /*result_size_hint*/) const { column_match_filter_size(s, filt.size()); diff --git a/be/src/vec/columns/column_const.h b/be/src/vec/columns/column_const.h index 0fa22ca23bf20b..69177eb2ca377a 100644 --- a/be/src/vec/columns/column_const.h +++ b/be/src/vec/columns/column_const.h @@ -118,8 +118,6 @@ class ColumnConst final : public COWHelper { bool is_variable_length() const override { return data->is_variable_length(); } - ColumnPtr remove_low_cardinality() const; - std::string get_name() const override { return "Const(" + data->get_name() + ")"; } void resize(size_t new_size) override { s = new_size; } diff --git a/be/src/vec/functions/function.cpp b/be/src/vec/functions/function.cpp index 5b935444710e25..851e430d2f0407 100644 --- a/be/src/vec/functions/function.cpp +++ b/be/src/vec/functions/function.cpp @@ -285,14 +285,6 @@ DataTypePtr FunctionBuilderImpl::get_return_type_without_low_cardinality( DataTypePtr FunctionBuilderImpl::get_return_type(const ColumnsWithTypeAndName& arguments) const { if (use_default_implementation_for_low_cardinality_columns()) { ColumnsWithTypeAndName args_without_low_cardinality(arguments); - - for (ColumnWithTypeAndName& arg : args_without_low_cardinality) { - bool is_const = arg.column && is_column_const(*arg.column); - if (is_const) { - arg.column = assert_cast(*arg.column).remove_low_cardinality(); - } - } - auto type_without_low_cardinality = get_return_type_without_low_cardinality(args_without_low_cardinality); diff --git a/be/src/vec/functions/nullif.cpp b/be/src/vec/functions/nullif.cpp index 4d85cab6b1cbc7..d8ab344acc47a4 100644 --- a/be/src/vec/functions/nullif.cpp +++ b/be/src/vec/functions/nullif.cpp @@ -67,14 +67,6 @@ class FunctionNullIf : public IFunction { static DataTypePtr get_return_type_for_equal(const ColumnsWithTypeAndName& arguments) { ColumnsWithTypeAndName args_without_low_cardinality(arguments); - - for (ColumnWithTypeAndName& arg : args_without_low_cardinality) { - bool is_const = arg.column && is_column_const(*arg.column); - if (is_const) { - arg.column = assert_cast(*arg.column).remove_low_cardinality(); - } - } - if (!arguments.empty()) { if (have_null_column(arguments)) { return make_nullable(std::make_shared()); From 08b187b2687837921dddc22cd2235e186c428f0a Mon Sep 17 00:00:00 2001 From: Socrates Date: Fri, 29 Nov 2024 16:15:04 +0800 Subject: [PATCH 079/399] [fix](orc) check all the cases before build_search_argument (#44615) ### What problem does this PR solve? In the old logic, the `check_expr_can_push_down` function does not check whether the `orc::Literal` are constructed successfully, but only checks during `build_search_argument`. However, if it is found that the `orc::Literal` fails to be constructed after `builder->startNot`, it will fail because the builder cannot end `startNot`. Therefore, we advance the behavior of constructing `orc::Literal` to the `check_expr_can_push_down` function and save the result to the map, so that it will never fail in the `build_search_argument` phase. Related PR: #43255 --- be/src/vec/exec/format/orc/vorc_reader.cpp | 176 ++++++++++----------- be/src/vec/exec/format/orc/vorc_reader.h | 18 ++- be/test/vec/exec/orc_reader_test.cpp | 57 +++---- 3 files changed, 131 insertions(+), 120 deletions(-) diff --git a/be/src/vec/exec/format/orc/vorc_reader.cpp b/be/src/vec/exec/format/orc/vorc_reader.cpp index 70f3f6f003f611..df8ec52efc2896 100644 --- a/be/src/vec/exec/format/orc/vorc_reader.cpp +++ b/be/src/vec/exec/format/orc/vorc_reader.cpp @@ -34,6 +34,7 @@ #include #include #include +#include #include "cctz/civil_time.h" #include "cctz/time_zone.h" @@ -567,12 +568,14 @@ std::tuple convert_to_orc_literal(const orc::Type* type, std::tuple OrcReader::_make_orc_literal( const VSlotRef* slot_ref, const VLiteral* literal) { + DCHECK(_col_name_to_file_col_name_low_case.contains(slot_ref->expr_name())); auto file_col_name_low_case = _col_name_to_file_col_name_low_case[slot_ref->expr_name()]; if (!_type_map.contains(file_col_name_low_case)) { // TODO: this is for acid table LOG(WARNING) << "Column " << slot_ref->expr_name() << " not found in _type_map"; return std::make_tuple(false, orc::Literal(false), orc::PredicateDataType::LONG); } + DCHECK(_type_map.contains(file_col_name_low_case)); const auto* orc_type = _type_map[file_col_name_low_case]; if (!TYPEKIND_TO_PREDICATE_TYPE.contains(orc_type->getKind())) { LOG(WARNING) << "Unsupported Push Down Orc Type [TypeKind=" << orc_type->getKind() << "]"; @@ -624,15 +627,37 @@ std::tuple OrcReader::_make_orc_lite } } -// check if the slot of expr can be pushed down to orc reader +// check if the slot of expr can be pushed down to orc reader and make orc predicate type bool OrcReader::_check_slot_can_push_down(const VExprSPtr& expr) { if (!expr->children()[0]->is_slot_ref()) { return false; } const auto* slot_ref = static_cast(expr->children()[0].get()); // check if the slot exists in orc file and not partition column - return _col_name_to_file_col_name.contains(slot_ref->expr_name()) && - !_lazy_read_ctx.predicate_partition_columns.contains(slot_ref->expr_name()); + if (!_col_name_to_file_col_name.contains(slot_ref->expr_name()) || + _lazy_read_ctx.predicate_partition_columns.contains(slot_ref->expr_name())) { + return false; + } + auto [valid, _, predicate_type] = _make_orc_literal(slot_ref, nullptr); + if (valid) { + _vslot_ref_to_orc_predicate_data_type[slot_ref] = predicate_type; + } + return valid; +} + +// check if the literal of expr can be pushed down to orc reader and make orc literal +bool OrcReader::_check_literal_can_push_down(const VExprSPtr& expr, uint16_t child_id) { + if (!expr->children()[child_id]->is_literal()) { + return false; + } + // the slot has been checked in _check_slot_can_push_down before calling this function + const auto* slot_ref = static_cast(expr->children()[0].get()); + const auto* literal = static_cast(expr->children()[child_id].get()); + auto [valid, orc_literal, _] = _make_orc_literal(slot_ref, literal); + if (valid) { + _vliteral_to_orc_literal.insert(std::make_pair(literal, orc_literal)); + } + return valid; } // check if there are rest children of expr can be pushed down to orc reader @@ -642,7 +667,7 @@ bool OrcReader::_check_rest_children_can_push_down(const VExprSPtr& expr) { } for (size_t i = 1; i < expr->children().size(); ++i) { - if (!expr->children()[i]->is_literal()) { + if (!_check_literal_can_push_down(expr, i)) { return false; } } @@ -651,7 +676,10 @@ bool OrcReader::_check_rest_children_can_push_down(const VExprSPtr& expr) { // check if the expr can be pushed down to orc reader bool OrcReader::_check_expr_can_push_down(const VExprSPtr& expr) { - DCHECK(expr != nullptr); + if (expr == nullptr) { + return false; + } + switch (expr->op()) { case TExprOpcode::COMPOUND_AND: // at least one child can be pushed down @@ -693,198 +721,167 @@ bool OrcReader::_check_expr_can_push_down(const VExprSPtr& expr) { } } -bool OrcReader::_build_less_than(const VExprSPtr& expr, +void OrcReader::_build_less_than(const VExprSPtr& expr, std::unique_ptr& builder) { DCHECK(expr->children().size() == 2); DCHECK(expr->children()[0]->is_slot_ref()); DCHECK(expr->children()[1]->is_literal()); const auto* slot_ref = static_cast(expr->children()[0].get()); const auto* literal = static_cast(expr->children()[1].get()); - auto [valid, orc_literal, predicate_type] = _make_orc_literal(slot_ref, literal); - if (!valid) { - return false; - } + DCHECK(_vslot_ref_to_orc_predicate_data_type.contains(slot_ref)); + auto predicate_type = _vslot_ref_to_orc_predicate_data_type[slot_ref]; + DCHECK(_vliteral_to_orc_literal.contains(literal)); + auto orc_literal = _vliteral_to_orc_literal.find(literal)->second; builder->lessThan(slot_ref->expr_name(), predicate_type, orc_literal); - return true; } -bool OrcReader::_build_less_than_equals(const VExprSPtr& expr, +void OrcReader::_build_less_than_equals(const VExprSPtr& expr, std::unique_ptr& builder) { DCHECK(expr->children().size() == 2); DCHECK(expr->children()[0]->is_slot_ref()); DCHECK(expr->children()[1]->is_literal()); const auto* slot_ref = static_cast(expr->children()[0].get()); const auto* literal = static_cast(expr->children()[1].get()); - auto [valid, orc_literal, predicate_type] = _make_orc_literal(slot_ref, literal); - if (!valid) { - return false; - } + DCHECK(_vslot_ref_to_orc_predicate_data_type.contains(slot_ref)); + auto predicate_type = _vslot_ref_to_orc_predicate_data_type[slot_ref]; + DCHECK(_vliteral_to_orc_literal.contains(literal)); + auto orc_literal = _vliteral_to_orc_literal.find(literal)->second; builder->lessThanEquals(slot_ref->expr_name(), predicate_type, orc_literal); - return true; } -bool OrcReader::_build_equals(const VExprSPtr& expr, +void OrcReader::_build_equals(const VExprSPtr& expr, std::unique_ptr& builder) { DCHECK(expr->children().size() == 2); DCHECK(expr->children()[0]->is_slot_ref()); DCHECK(expr->children()[1]->is_literal()); const auto* slot_ref = static_cast(expr->children()[0].get()); const auto* literal = static_cast(expr->children()[1].get()); - auto [valid, orc_literal, predicate_type] = _make_orc_literal(slot_ref, literal); - if (!valid) { - return false; - } + DCHECK(_vslot_ref_to_orc_predicate_data_type.contains(slot_ref)); + auto predicate_type = _vslot_ref_to_orc_predicate_data_type[slot_ref]; + DCHECK(_vliteral_to_orc_literal.contains(literal)); + auto orc_literal = _vliteral_to_orc_literal.find(literal)->second; builder->equals(slot_ref->expr_name(), predicate_type, orc_literal); - return true; } -bool OrcReader::_build_filter_in(const VExprSPtr& expr, +void OrcReader::_build_filter_in(const VExprSPtr& expr, std::unique_ptr& builder) { DCHECK(expr->children().size() >= 2); DCHECK(expr->children()[0]->is_slot_ref()); const auto* slot_ref = static_cast(expr->children()[0].get()); std::vector literals; - orc::PredicateDataType predicate_type = orc::PredicateDataType::LONG; + DCHECK(_vslot_ref_to_orc_predicate_data_type.contains(slot_ref)); + orc::PredicateDataType predicate_type = _vslot_ref_to_orc_predicate_data_type[slot_ref]; for (size_t i = 1; i < expr->children().size(); ++i) { DCHECK(expr->children()[i]->is_literal()); const auto* literal = static_cast(expr->children()[i].get()); - auto [valid, orc_literal, type] = _make_orc_literal(slot_ref, literal); - if (!valid) { - return false; - } + DCHECK(_vliteral_to_orc_literal.contains(literal)); + auto orc_literal = _vliteral_to_orc_literal.find(literal)->second; literals.emplace_back(orc_literal); - predicate_type = type; } DCHECK(!literals.empty()); builder->in(slot_ref->expr_name(), predicate_type, literals); - return true; } -bool OrcReader::_build_is_null(const VExprSPtr& expr, +void OrcReader::_build_is_null(const VExprSPtr& expr, std::unique_ptr& builder) { DCHECK(expr->children().size() == 1); DCHECK(expr->children()[0]->is_slot_ref()); const auto* slot_ref = static_cast(expr->children()[0].get()); - auto [valid, _, predicate_type] = _make_orc_literal(slot_ref, nullptr); + DCHECK(_vslot_ref_to_orc_predicate_data_type.contains(slot_ref)); + auto predicate_type = _vslot_ref_to_orc_predicate_data_type[slot_ref]; builder->isNull(slot_ref->expr_name(), predicate_type); - return true; } bool OrcReader::_build_search_argument(const VExprSPtr& expr, std::unique_ptr& builder) { - if (expr == nullptr) { - return false; - } - - // if expr can not be pushed down, skip it and continue to next expr + // OPTIMIZE: check expr only once if (!_check_expr_can_push_down(expr)) { return false; } - switch (expr->op()) { case TExprOpcode::COMPOUND_AND: { - bool at_least_one_can_push_down = false; builder->startAnd(); + bool at_least_one_can_push_down = false; for (const auto& child : expr->children()) { if (_build_search_argument(child, builder)) { at_least_one_can_push_down = true; } } - if (!at_least_one_can_push_down) { - // if all exprs can not be pushed down, builder->end() will throw exception - return false; - } + DCHECK(at_least_one_can_push_down); builder->end(); break; } - case TExprOpcode::COMPOUND_OR: + case TExprOpcode::COMPOUND_OR: { builder->startOr(); + bool all_can_push_down = true; for (const auto& child : expr->children()) { if (!_build_search_argument(child, builder)) { - return false; + all_can_push_down = false; } } + DCHECK(all_can_push_down); builder->end(); break; - case TExprOpcode::COMPOUND_NOT: - builder->startNot(); + } + case TExprOpcode::COMPOUND_NOT: { DCHECK_EQ(expr->children().size(), 1); - if (!_build_search_argument(expr->children()[0], builder)) { - return false; - } + builder->startNot(); + auto res = _build_search_argument(expr->children()[0], builder); + DCHECK(res); builder->end(); break; + } case TExprOpcode::GE: builder->startNot(); - if (!_build_less_than(expr, builder)) { - return false; - } + _build_less_than(expr, builder); builder->end(); break; case TExprOpcode::GT: builder->startNot(); - if (!_build_less_than_equals(expr, builder)) { - return false; - } + _build_less_than_equals(expr, builder); builder->end(); break; case TExprOpcode::LE: - if (!_build_less_than_equals(expr, builder)) { - return false; - } + _build_less_than_equals(expr, builder); break; case TExprOpcode::LT: - if (!_build_less_than(expr, builder)) { - return false; - } + _build_less_than(expr, builder); break; case TExprOpcode::EQ: - if (!_build_equals(expr, builder)) { - return false; - } + _build_equals(expr, builder); break; case TExprOpcode::NE: builder->startNot(); - if (!_build_equals(expr, builder)) { - return false; - } + _build_equals(expr, builder); builder->end(); break; case TExprOpcode::FILTER_IN: - if (!_build_filter_in(expr, builder)) { - return false; - } + _build_filter_in(expr, builder); break; case TExprOpcode::FILTER_NOT_IN: builder->startNot(); - if (!_build_filter_in(expr, builder)) { - return false; - } + _build_filter_in(expr, builder); builder->end(); break; // is null and is not null is represented as function call - case TExprOpcode::INVALID_OPCODE: { + case TExprOpcode::INVALID_OPCODE: DCHECK(expr->node_type() == TExprNodeType::FUNCTION_CALL); if (expr->fn().name.function_name == "is_null_pred") { - if (!_build_is_null(expr, builder)) { - return false; - } + _build_is_null(expr, builder); } else if (expr->fn().name.function_name == "is_not_null_pred") { builder->startNot(); - if (!_build_is_null(expr, builder)) { - return false; - } + _build_is_null(expr, builder); builder->end(); } else { + // should not reach here, because _check_expr_can_push_down has already checked __builtin_unreachable(); } break; - } - default: { + + default: // should not reach here, because _check_expr_can_push_down has already checked __builtin_unreachable(); } - } return true; } @@ -898,6 +895,8 @@ bool OrcReader::_init_search_argument(const VExprContextSPtrs& conjuncts) { bool at_least_one_can_push_down = false; builder->startAnd(); for (const auto& expr_ctx : conjuncts) { + _vslot_ref_to_orc_predicate_data_type.clear(); + _vliteral_to_orc_literal.clear(); if (_build_search_argument(expr_ctx->root(), builder)) { at_least_one_can_push_down = true; } @@ -943,7 +942,7 @@ Status OrcReader::set_fill_columns( visit_slot(child.get()); } } else if (VInPredicate* in_predicate = typeid_cast(filter_impl)) { - if (in_predicate->get_num_children() > 0) { + if (!in_predicate->children().empty()) { visit_slot(in_predicate->children()[0].get()); } } else { @@ -1179,7 +1178,8 @@ Status OrcReader::_fill_partition_columns( if (num_deserialized != rows) { return Status::InternalError( "Failed to fill partition column: {}={} ." - "Number of rows expected to be written : {}, number of rows actually written : " + "Number of rows expected to be written : {}, number of rows actually " + "written : " "{}", slot_desc->col_name(), value, num_deserialized, rows); } diff --git a/be/src/vec/exec/format/orc/vorc_reader.h b/be/src/vec/exec/format/orc/vorc_reader.h index 8c73957e79e4e0..0dd19077bcf0af 100644 --- a/be/src/vec/exec/format/orc/vorc_reader.h +++ b/be/src/vec/exec/format/orc/vorc_reader.h @@ -41,6 +41,7 @@ #include "orc/Reader.hh" #include "orc/Type.hh" #include "orc/Vector.hh" +#include "orc/sargs/Literal.hh" #include "runtime/types.h" #include "util/runtime_profile.h" #include "vec/aggregate_functions/aggregate_function.h" @@ -288,23 +289,27 @@ class OrcReader : public GenericReader { bool* is_hive1_orc); static bool _check_acid_schema(const orc::Type& type); static const orc::Type& _remove_acid(const orc::Type& type); + + // functions for building search argument until _init_search_argument std::tuple _make_orc_literal( const VSlotRef* slot_ref, const VLiteral* literal); bool _check_slot_can_push_down(const VExprSPtr& expr); + bool _check_literal_can_push_down(const VExprSPtr& expr, uint16_t child_id); bool _check_rest_children_can_push_down(const VExprSPtr& expr); bool _check_expr_can_push_down(const VExprSPtr& expr); - bool _build_less_than(const VExprSPtr& expr, + void _build_less_than(const VExprSPtr& expr, std::unique_ptr& builder); - bool _build_less_than_equals(const VExprSPtr& expr, + void _build_less_than_equals(const VExprSPtr& expr, std::unique_ptr& builder); - bool _build_equals(const VExprSPtr& expr, std::unique_ptr& builder); - bool _build_filter_in(const VExprSPtr& expr, + void _build_equals(const VExprSPtr& expr, std::unique_ptr& builder); + void _build_filter_in(const VExprSPtr& expr, std::unique_ptr& builder); - bool _build_is_null(const VExprSPtr& expr, + void _build_is_null(const VExprSPtr& expr, std::unique_ptr& builder); bool _build_search_argument(const VExprSPtr& expr, std::unique_ptr& builder); bool _init_search_argument(const VExprContextSPtrs& conjuncts); + void _init_bloom_filter( std::unordered_map* colname_to_value_range); void _init_system_properties(); @@ -644,6 +649,9 @@ class OrcReader : public GenericReader { std::unordered_map _table_col_to_file_col; //support iceberg position delete . std::vector* _position_delete_ordered_rowids = nullptr; + std::unordered_map + _vslot_ref_to_orc_predicate_data_type; + std::unordered_map _vliteral_to_orc_literal; }; class ORCFileInputStream : public orc::InputStream, public ProfileCollector { diff --git a/be/test/vec/exec/orc_reader_test.cpp b/be/test/vec/exec/orc_reader_test.cpp index ec5bd9b519de63..f2bba434368ae1 100644 --- a/be/test/vec/exec/orc_reader_test.cpp +++ b/be/test/vec/exec/orc_reader_test.cpp @@ -94,33 +94,34 @@ class OrcReaderTest : public testing::Test { TEST_F(OrcReaderTest, test_build_search_argument) { ExecEnv::GetInstance()->set_orc_memory_pool(new ORCMemoryPool()); - std:: - vector - exprs = - { - // select count(o_orderkey) from tpch1_orc.orders where o_orderkey < 100 or o_orderkey > 5999900 or o_orderkey in (1000000, 2000000, 3000000); - R"|({"1":{"lst":["rec",13,{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":3},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":3},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":11},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"lt"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"lt(int, int)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":5},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":100}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":13},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"gt"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"gt(int, int)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":5},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":5999900}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":11},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":5},"4":{"i32":4},"11":{"rec":{"1":{"tf":0}}},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":1000000}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":2000000}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":3000000}}},"20":{"i32":-1},"29":{"tf":0}}]}})|", - // select count(o_orderkey) from tpch1_orc.orders where o_orderkey is null or (o_orderkey between 100 and 1000 and o_orderkey not in (200, 300, 400)); - R"|({"1":{"lst":["rec",16,{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":3},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":20},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"4":{"i32":1},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"is_null_pred"}}},"2":{"i32":0},"3":{"lst":["rec",1,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"is_null_pred(int)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"29":{"tf":0}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":2},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":2},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":14},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"ge"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"ge(int, int)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":5},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":100}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":12},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"le"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"le(int, int)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":5},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":1000}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":11},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":6},"4":{"i32":4},"11":{"rec":{"1":{"tf":1}}},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":200}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":300}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":400}}},"20":{"i32":-1},"29":{"tf":0}}]}})|", - // select count(o_orderkey) from tpch1_orc.orders where o_orderkey is null or (o_orderkey between 1000000 and 1200000 and o_orderkey != 1100000); - R"|({"1":{"lst":["rec",14,{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":3},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":20},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"4":{"i32":1},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"is_null_pred"}}},"2":{"i32":0},"3":{"lst":["rec",1,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"is_null_pred(int)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"29":{"tf":0}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":2},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":2},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":14},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"ge"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"ge(int, int)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":5},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":1000000}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":12},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"le"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"le(int, int)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":5},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":1200000}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":10},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"ne"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"ne(int, int)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":5},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":1100000}}},"20":{"i32":-1},"29":{"tf":0}}]}})|", - // SELECT count(o_orderkey) FROM tpch1_orc.orders WHERE o_orderkey IN (1000000, 2000000, 3000000) OR (o_orderdate >= '1994-01-01' AND o_orderdate <= '1994-12-31'); - R"|({"1":{"lst":["rec",13,{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":3},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":11},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":5},"4":{"i32":4},"11":{"rec":{"1":{"tf":0}}},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":1000000}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":2000000}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":3000000}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":2},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":14},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"ge"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":26}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":26}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"ge(datev2, datev2)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":26},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":26}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":4},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderdate"}},{"1":{"i32":7},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":26}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"8":{"rec":{"1":{"str":"1994-01-01"}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":12},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"le"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":26}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":26}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"le(datev2, datev2)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":26},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":26}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":4},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderdate"}},{"1":{"i32":7},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":26}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"8":{"rec":{"1":{"str":"1994-12-31"}}},"20":{"i32":-1},"29":{"tf":0}}]}})|", - // select count(o_orderkey) from tpch1_orc.orders where o_orderkey < 2 or (o_comment like '%delayed%' and o_orderpriority = '1-URGENT'); - R"|({"1":{"lst":["rec",11,{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":3},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":11},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"lt"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"lt(int, int)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":5},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":2}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":2},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":20},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"like"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":23},"2":{"i32":2147483643}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":15},"2":{"i32":9}}}}]},"3":{"i64":9}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"like(text, varchar(9))"},"9":{"rec":{"1":{"str":""}}},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":23},"2":{"i32":2147483643}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":8},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_comment"}},{"1":{"i32":17},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":15},"2":{"i32":65533}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"16":{"rec":{"1":{"str":"%delayed%"}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":9},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"eq"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":23},"2":{"i32":2147483643}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":15},"2":{"i32":65533}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"eq(text, varchar(65533))"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":23},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":23},"2":{"i32":2147483643}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":5},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderpriority"}},{"1":{"i32":17},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":15},"2":{"i32":65533}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"16":{"rec":{"1":{"str":"1-URGENT"}}},"20":{"i32":-1},"29":{"tf":0}}]}})|", - // select count(o_orderkey) from tpch1_orc.orders where o_orderkey < 1 + 1; - R"|({"1":{"lst":["rec",3,{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":11},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"lt"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"lt(int, int)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":5},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":2}}},"20":{"i32":-1},"29":{"tf":0}}]}})|", - // SELECT count(o_orderkey) FROM tpch1_orc.orders WHERE o_comment LIKE '%delayed%' OR o_orderpriority = '1-URGENT'; - R"|({"1":{"lst":["rec",7,{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":3},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":20},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"like"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":23},"2":{"i32":2147483643}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":15},"2":{"i32":9}}}}]},"3":{"i64":9}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"like(text, varchar(9))"},"9":{"rec":{"1":{"str":""}}},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":23},"2":{"i32":2147483643}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":8},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_comment"}},{"1":{"i32":17},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":15},"2":{"i32":65533}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"16":{"rec":{"1":{"str":"%delayed%"}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":9},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"eq"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":23},"2":{"i32":2147483643}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":15},"2":{"i32":65533}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"eq(text, varchar(65533))"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":23},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":23},"2":{"i32":2147483643}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":5},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderpriority"}},{"1":{"i32":17},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":15},"2":{"i32":65533}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"16":{"rec":{"1":{"str":"1-URGENT"}}},"20":{"i32":-1},"29":{"tf":0}}]}})|", - // select count(o_orderkey) from tpch1_orc.orders where o_orderkey between 1 and 100 or random() > 0.5; - R"|({"1":{"lst":["rec",11,{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":3},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":2},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":14},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"ge"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"ge(int, int)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":5},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":1}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":12},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"le"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"le(int, int)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":5},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":100}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":13},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"gt"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":8}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":8}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"gt(double, double)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":8},"29":{"tf":0}},{"1":{"i32":20},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":8}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"random"}}},"2":{"i32":0},"3":{"lst":["rec",0]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":8}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"random()"},"9":{"rec":{"1":{"str":""}}},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"29":{"tf":0}},{"1":{"i32":8},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":8}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"9":{"rec":{"1":{"dbl":0.5}}},"20":{"i32":-1},"29":{"tf":0}}]}})|", - // select count(o_orderkey) from tpch1_orc.orders where lower(o_orderpriority) = '1-urgent'; - R"|({"1":{"lst":["rec",4,{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":9},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"eq"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":15},"2":{"i32":65533}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":15},"2":{"i32":65533}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"eq(varchar(65533), varchar(65533))"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":15},"29":{"tf":1}},{"1":{"i32":20},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":15},"2":{"i32":65533}}}}]},"3":{"i64":-1}}},"4":{"i32":1},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"lower"}}},"2":{"i32":0},"3":{"lst":["rec",1,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":23},"2":{"i32":2147483643}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":15},"2":{"i32":65533}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"lower(text)"},"9":{"rec":{"1":{"str":""}}},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":23},"2":{"i32":2147483643}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":5},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderpriority"}},{"1":{"i32":17},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":15},"2":{"i32":65533}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"16":{"rec":{"1":{"str":"1-urgent"}}},"20":{"i32":-1},"29":{"tf":0}}]}})|", - // select count(o_orderkey) from tpch1_orc.orders where o_orderkey * 2 < 60; - R"|({"1":{"lst":["rec",5,{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":11},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"lt"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":6}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":6}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"lt(bigint, bigint)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":6},"29":{"tf":1}},{"1":{"i32":1},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":6}}}}]},"3":{"i64":-1}}},"3":{"i32":55},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"multiply"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":3}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":6}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"multiply(int, tinyint)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":3}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":2}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":6}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":60}}},"20":{"i32":-1},"29":{"tf":0}}]}})|", - // select count(o_orderkey) from tpch1_orc.orders where o_orderdate is not null; - R"|({"1":{"lst":["rec",4,{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":9},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"eq"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":27},"3":{"i32":18},"4":{"i32":0}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":27},"3":{"i32":18},"4":{"i32":0}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"eq(datetimev2(0), datetimev2(0))"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":27},"29":{"tf":1}},{"1":{"i32":5},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":27},"3":{"i32":18},"4":{"i32":0}}}}]},"3":{"i64":-1}}},"3":{"i32":4},"4":{"i32":1},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"casttodatetimev2"}}},"2":{"i32":0},"3":{"lst":["rec",1,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":26}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":27},"3":{"i32":18},"4":{"i32":0}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"casttodatetimev2(datev2)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":26}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":4},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderdate"}},{"1":{"i32":7},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":27},"3":{"i32":18},"4":{"i32":0}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"8":{"rec":{"1":{"str":"2024-11-12 21:13:02"}}},"20":{"i32":-1},"29":{"tf":0}}]}})|", - }; + std::vector + exprs = + { + // select count(o_orderkey) from tpch1_orc.orders where o_orderkey < 100 or o_orderkey > 5999900 or o_orderkey in (1000000, 2000000, 3000000); + R"|({"1":{"lst":["rec",13,{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":3},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":3},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":11},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"lt"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"lt(int, int)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":5},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":100}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":13},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"gt"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"gt(int, int)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":5},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":5999900}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":11},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":5},"4":{"i32":4},"11":{"rec":{"1":{"tf":0}}},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":1000000}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":2000000}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":3000000}}},"20":{"i32":-1},"29":{"tf":0}}]}})|", + // select count(o_orderkey) from tpch1_orc.orders where o_orderkey is null or (o_orderkey between 100 and 1000 and o_orderkey not in (200, 300, 400)); + R"|({"1":{"lst":["rec",16,{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":3},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":20},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"4":{"i32":1},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"is_null_pred"}}},"2":{"i32":0},"3":{"lst":["rec",1,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"is_null_pred(int)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"29":{"tf":0}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":2},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":2},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":14},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"ge"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"ge(int, int)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":5},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":100}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":12},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"le"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"le(int, int)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":5},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":1000}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":11},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":6},"4":{"i32":4},"11":{"rec":{"1":{"tf":1}}},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":200}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":300}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":400}}},"20":{"i32":-1},"29":{"tf":0}}]}})|", + // select count(o_orderkey) from tpch1_orc.orders where o_orderkey is null or (o_orderkey between 1000000 and 1200000 and o_orderkey != 1100000); + R"|({"1":{"lst":["rec",14,{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":3},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":20},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"4":{"i32":1},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"is_null_pred"}}},"2":{"i32":0},"3":{"lst":["rec",1,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"is_null_pred(int)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"29":{"tf":0}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":2},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":2},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":14},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"ge"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"ge(int, int)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":5},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":1000000}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":12},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"le"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"le(int, int)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":5},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":1200000}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":10},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"ne"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"ne(int, int)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":5},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":1100000}}},"20":{"i32":-1},"29":{"tf":0}}]}})|", + // SELECT count(o_orderkey) FROM tpch1_orc.orders WHERE o_orderkey IN (1000000, 2000000, 3000000) OR (o_orderdate >= '1994-01-01' AND o_orderdate <= '1994-12-31'); + R"|({"1":{"lst":["rec",13,{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":3},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":11},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":5},"4":{"i32":4},"11":{"rec":{"1":{"tf":0}}},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":1000000}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":2000000}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":3000000}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":2},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":14},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"ge"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":26}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":26}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"ge(datev2, datev2)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":26},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":26}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":4},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderdate"}},{"1":{"i32":7},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":26}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"8":{"rec":{"1":{"str":"1994-01-01"}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":12},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"le"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":26}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":26}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"le(datev2, datev2)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":26},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":26}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":4},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderdate"}},{"1":{"i32":7},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":26}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"8":{"rec":{"1":{"str":"1994-12-31"}}},"20":{"i32":-1},"29":{"tf":0}}]}})|", + // select count(o_orderkey) from tpch1_orc.orders where o_orderkey < 2 or (o_comment like '%delayed%' and o_orderpriority = '1-URGENT'); + R"|({"1":{"lst":["rec",11,{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":3},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":11},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"lt"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"lt(int, int)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":5},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":2}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":2},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":20},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"like"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":23},"2":{"i32":2147483643}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":15},"2":{"i32":9}}}}]},"3":{"i64":9}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"like(text, varchar(9))"},"9":{"rec":{"1":{"str":""}}},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":23},"2":{"i32":2147483643}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":8},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_comment"}},{"1":{"i32":17},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":15},"2":{"i32":65533}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"16":{"rec":{"1":{"str":"%delayed%"}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":9},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"eq"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":23},"2":{"i32":2147483643}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":15},"2":{"i32":65533}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"eq(text, varchar(65533))"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":23},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":23},"2":{"i32":2147483643}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":5},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderpriority"}},{"1":{"i32":17},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":15},"2":{"i32":65533}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"16":{"rec":{"1":{"str":"1-URGENT"}}},"20":{"i32":-1},"29":{"tf":0}}]}})|", + // select count(o_orderkey) from tpch1_orc.orders where o_orderkey < 2 or (o_totalprice < 173665.47 and o_custkey >= 36901); + R"|({"1":{"lst":["rec",11,{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":3},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":11},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"lt"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"lt(int, int)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":5},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":2}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":2},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":11},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"lt"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":30},"3":{"i32":12},"4":{"i32":2}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":30},"3":{"i32":12},"4":{"i32":2}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"lt(decimalv3(12,2), decimalv3(12,2))"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":30},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":30},"3":{"i32":12},"4":{"i32":2}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":3},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_totalprice"}},{"1":{"i32":10},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":30},"3":{"i32":12},"4":{"i32":2}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"19":{"rec":{"1":{"str":"173665.47"}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":14},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"ge"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"ge(int, int)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":5},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":1},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_custkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":36901}}},"20":{"i32":-1},"29":{"tf":0}}]}})|", + // select count(o_orderkey) from tpch1_orc.orders where o_orderkey < 1 + 1; + R"|({"1":{"lst":["rec",3,{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":11},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"lt"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"lt(int, int)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":5},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":2}}},"20":{"i32":-1},"29":{"tf":0}}]}})|", + // SELECT count(o_orderkey) FROM tpch1_orc.orders WHERE o_comment LIKE '%delayed%' OR o_orderpriority = '1-URGENT'; + R"|({"1":{"lst":["rec",7,{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":3},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":20},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"like"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":23},"2":{"i32":2147483643}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":15},"2":{"i32":9}}}}]},"3":{"i64":9}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"like(text, varchar(9))"},"9":{"rec":{"1":{"str":""}}},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":23},"2":{"i32":2147483643}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":8},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_comment"}},{"1":{"i32":17},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":15},"2":{"i32":65533}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"16":{"rec":{"1":{"str":"%delayed%"}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":9},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"eq"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":23},"2":{"i32":2147483643}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":15},"2":{"i32":65533}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"eq(text, varchar(65533))"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":23},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":23},"2":{"i32":2147483643}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":5},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderpriority"}},{"1":{"i32":17},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":15},"2":{"i32":65533}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"16":{"rec":{"1":{"str":"1-URGENT"}}},"20":{"i32":-1},"29":{"tf":0}}]}})|", + // select count(o_orderkey) from tpch1_orc.orders where o_orderkey between 1 and 100 or random() > 0.5; + R"|({"1":{"lst":["rec",11,{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":3},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":2},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":14},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"ge"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"ge(int, int)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":5},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":1}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":12},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"le"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"le(int, int)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":5},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":100}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":13},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"gt"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":8}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":8}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"gt(double, double)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":8},"29":{"tf":0}},{"1":{"i32":20},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":8}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"random"}}},"2":{"i32":0},"3":{"lst":["rec",0]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":8}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"random()"},"9":{"rec":{"1":{"str":""}}},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"29":{"tf":0}},{"1":{"i32":8},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":8}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"9":{"rec":{"1":{"dbl":0.5}}},"20":{"i32":-1},"29":{"tf":0}}]}})|", + // select count(o_orderkey) from tpch1_orc.orders where lower(o_orderpriority) = '1-urgent'; + R"|({"1":{"lst":["rec",4,{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":9},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"eq"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":15},"2":{"i32":65533}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":15},"2":{"i32":65533}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"eq(varchar(65533), varchar(65533))"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":15},"29":{"tf":1}},{"1":{"i32":20},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":15},"2":{"i32":65533}}}}]},"3":{"i64":-1}}},"4":{"i32":1},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"lower"}}},"2":{"i32":0},"3":{"lst":["rec",1,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":23},"2":{"i32":2147483643}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":15},"2":{"i32":65533}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"lower(text)"},"9":{"rec":{"1":{"str":""}}},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":23},"2":{"i32":2147483643}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":5},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderpriority"}},{"1":{"i32":17},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":15},"2":{"i32":65533}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"16":{"rec":{"1":{"str":"1-urgent"}}},"20":{"i32":-1},"29":{"tf":0}}]}})|", + // select count(o_orderkey) from tpch1_orc.orders where o_orderkey * 2 < 60; + R"|({"1":{"lst":["rec",5,{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":11},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"lt"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":6}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":6}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"lt(bigint, bigint)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":6},"29":{"tf":1}},{"1":{"i32":1},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":6}}}}]},"3":{"i64":-1}}},"3":{"i32":55},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"multiply"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":3}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":6}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"multiply(int, tinyint)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":3}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":2}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":6}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":60}}},"20":{"i32":-1},"29":{"tf":0}}]}})|", + // select count(o_orderkey) from tpch1_orc.orders where o_orderdate is not null; + R"|({"1":{"lst":["rec",4,{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":9},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"eq"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":27},"3":{"i32":18},"4":{"i32":0}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":27},"3":{"i32":18},"4":{"i32":0}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"eq(datetimev2(0), datetimev2(0))"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":27},"29":{"tf":1}},{"1":{"i32":5},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":27},"3":{"i32":18},"4":{"i32":0}}}}]},"3":{"i64":-1}}},"3":{"i32":4},"4":{"i32":1},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"casttodatetimev2"}}},"2":{"i32":0},"3":{"lst":["rec",1,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":26}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":27},"3":{"i32":18},"4":{"i32":0}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"casttodatetimev2(datev2)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":26}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":4},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderdate"}},{"1":{"i32":7},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":27},"3":{"i32":18},"4":{"i32":0}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"8":{"rec":{"1":{"str":"2024-11-12 21:13:02"}}},"20":{"i32":-1},"29":{"tf":0}}]}})|", + }; std::vector result_search_arguments = { "leaf-0 = (o_orderkey < 100), leaf-1 = (o_orderkey <= 5999900), leaf-2 " "= (o_orderkey " @@ -139,6 +140,8 @@ TEST_F(OrcReaderTest, test_build_search_argument) { "(or leaf-0 leaf-2))", "leaf-0 = (o_orderkey < 2), leaf-1 = (o_orderpriority = 1-URGENT), expr = (or leaf-0 " "leaf-1)", + "leaf-0 = (o_orderkey < 2), leaf-1 = (o_custkey < 36901), expr = (or leaf-0 (not " + "leaf-1))", "leaf-0 = (o_orderkey < 2), expr = leaf-0", CANNOT_PUSH_DOWN_ERROR, CANNOT_PUSH_DOWN_ERROR, From 46575e59bac2c39088ecd925d1d20097d86667a3 Mon Sep 17 00:00:00 2001 From: HappenLee Date: Fri, 29 Nov 2024 17:52:49 +0800 Subject: [PATCH 080/399] [Refactor](vec) Remove the unless api in IColumn (#44619) Remove the unless api in IColumn --- be/src/vec/columns/column.h | 12 +----------- be/src/vec/columns/column_complex.h | 5 ----- be/src/vec/columns/column_const.h | 1 - be/src/vec/columns/column_decimal.h | 3 --- be/src/vec/columns/column_nullable.h | 3 --- be/src/vec/columns/column_vector.h | 2 -- be/src/vec/columns/predicate_column.h | 2 -- be/src/vec/data_types/data_type.h | 1 + be/src/vec/functions/functions_comparison.h | 6 +++--- 9 files changed, 5 insertions(+), 30 deletions(-) diff --git a/be/src/vec/columns/column.h b/be/src/vec/columns/column.h index 593666568aa375..24653f878555f5 100644 --- a/be/src/vec/columns/column.h +++ b/be/src/vec/columns/column.h @@ -565,10 +565,6 @@ class IColumn : public COW { /// It's true for ColumnNullable, can be true or false for ColumnConst, etc. virtual bool is_concrete_nullable() const { return false; } - virtual bool is_bitmap() const { return false; } - - virtual bool is_hll() const { return false; } - // true if column has null element virtual bool has_null() const { return false; } @@ -607,10 +603,6 @@ class IColumn : public COW { /// Checks only @sample_ratio ratio of rows. virtual double get_ratio_of_default_rows(double sample_ratio = 1.0) const { return 0.0; } - /// Column is ColumnVector of numbers or ColumnConst of it. Note that Nullable columns are not numeric. - /// Implies is_fixed_and_contiguous. - virtual bool is_numeric() const { return false; } - // Column is ColumnString/ColumnArray/ColumnMap or other variable length column at every row virtual bool is_variable_length() const { return false; } @@ -618,8 +610,6 @@ class IColumn : public COW { virtual bool is_column_string64() const { return false; } - virtual bool is_column_decimal() const { return false; } - virtual bool is_column_dictionary() const { return false; } virtual bool is_column_array() const { return false; } @@ -752,4 +742,4 @@ struct ColumnPtrWrapper { ColumnPtrWrapper(vectorized::ColumnPtr col) : column_ptr(std::move(col)) {} }; -} // namespace doris +} // namespace doris \ No newline at end of file diff --git a/be/src/vec/columns/column_complex.h b/be/src/vec/columns/column_complex.h index 14ae940c9d7ba0..f8513d54f57934 100644 --- a/be/src/vec/columns/column_complex.h +++ b/be/src/vec/columns/column_complex.h @@ -48,11 +48,6 @@ class ColumnComplexType final : public COWHelper> using value_type = T; using Container = std::vector; - bool is_numeric() const override { return false; } - - bool is_bitmap() const override { return std::is_same_v; } - bool is_hll() const override { return std::is_same_v; } - size_t size() const override { return data.size(); } StringRef get_data_at(size_t n) const override { diff --git a/be/src/vec/columns/column_const.h b/be/src/vec/columns/column_const.h index 69177eb2ca377a..80d50e26e16ead 100644 --- a/be/src/vec/columns/column_const.h +++ b/be/src/vec/columns/column_const.h @@ -253,7 +253,6 @@ class ColumnConst final : public COWHelper { // ColumnConst is not nullable, but may be concrete nullable. bool is_concrete_nullable() const override { return is_column_nullable(*data); } bool only_null() const override { return data->is_null_at(0); } - bool is_numeric() const override { return data->is_numeric(); } StringRef get_raw_data() const override { return data->get_raw_data(); } /// Not part of the common interface. diff --git a/be/src/vec/columns/column_decimal.h b/be/src/vec/columns/column_decimal.h index 946b268436ef6c..a8b1f9573ef185 100644 --- a/be/src/vec/columns/column_decimal.h +++ b/be/src/vec/columns/column_decimal.h @@ -105,9 +105,6 @@ class ColumnDecimal final : public COWHelper> { public: std::string get_name() const override { return TypeName::get(); } - bool is_numeric() const override { return false; } - bool is_column_decimal() const override { return true; } - size_t size() const override { return data.size(); } size_t byte_size() const override { return data.size() * sizeof(data[0]); } size_t allocated_bytes() const override { return data.allocated_bytes(); } diff --git a/be/src/vec/columns/column_nullable.h b/be/src/vec/columns/column_nullable.h index 83d5e6af35aeb4..84b3ce0f82aadb 100644 --- a/be/src/vec/columns/column_nullable.h +++ b/be/src/vec/columns/column_nullable.h @@ -328,9 +328,6 @@ class ColumnNullable final : public COWHelper, public N bool is_nullable() const override { return true; } bool is_concrete_nullable() const override { return true; } - bool is_bitmap() const override { return get_nested_column().is_bitmap(); } - bool is_hll() const override { return get_nested_column().is_hll(); } - bool is_column_decimal() const override { return get_nested_column().is_column_decimal(); } bool is_column_string() const override { return get_nested_column().is_column_string(); } bool is_column_array() const override { return get_nested_column().is_column_array(); } bool is_column_map() const override { return get_nested_column().is_column_map(); } diff --git a/be/src/vec/columns/column_vector.h b/be/src/vec/columns/column_vector.h index 970997a91864cc..8006d76c1f98be 100644 --- a/be/src/vec/columns/column_vector.h +++ b/be/src/vec/columns/column_vector.h @@ -152,8 +152,6 @@ class ColumnVector final : public COWHelper> { ColumnVector(std::initializer_list il) : data {il} {} public: - bool is_numeric() const override { return IsNumber; } - size_t size() const override { return data.size(); } StringRef get_data_at(size_t n) const override { diff --git a/be/src/vec/columns/predicate_column.h b/be/src/vec/columns/predicate_column.h index 7e15656fe1d204..ab9648c3ae620f 100644 --- a/be/src/vec/columns/predicate_column.h +++ b/be/src/vec/columns/predicate_column.h @@ -102,8 +102,6 @@ class PredicateColumnType final : public COWHelper; - bool is_numeric() const override { return false; } - size_t size() const override { return data.size(); } StringRef get_data_at(size_t n) const override { diff --git a/be/src/vec/data_types/data_type.h b/be/src/vec/data_types/data_type.h index 86f7cf36fa8dc7..6c54241ea18a2a 100644 --- a/be/src/vec/data_types/data_type.h +++ b/be/src/vec/data_types/data_type.h @@ -289,6 +289,7 @@ struct WhichDataType { bool is_aggregate_function() const { return idx == TypeIndex::AggregateFunction; } bool is_variant_type() const { return idx == TypeIndex::VARIANT; } bool is_simple() const { return is_int() || is_uint() || is_float() || is_string(); } + bool is_num_can_compare() const { return is_int_or_uint() || is_float() || is_ip(); } }; /// IDataType helpers (alternative for IDataType virtual methods with single point of truth) diff --git a/be/src/vec/functions/functions_comparison.h b/be/src/vec/functions/functions_comparison.h index 78a89071c76efd..cb56f176c71eb1 100644 --- a/be/src/vec/functions/functions_comparison.h +++ b/be/src/vec/functions/functions_comparison.h @@ -637,8 +637,8 @@ class FunctionComparison : public IFunction { WhichDataType which_left {left_type}; WhichDataType which_right {right_type}; - const bool left_is_num = col_left_untyped->is_numeric(); - const bool right_is_num = col_right_untyped->is_numeric(); + const bool left_is_num_can_compare = which_left.is_num_can_compare(); + const bool right_is_num_can_compare = which_right.is_num_can_compare(); const bool left_is_string = which_left.is_string_or_fixed_string(); const bool right_is_string = which_right.is_string_or_fixed_string(); @@ -648,7 +648,7 @@ class FunctionComparison : public IFunction { // bool date_and_datetime = (left_type != right_type) && which_left.is_date_or_datetime() && // which_right.is_date_or_datetime(); - if (left_is_num && right_is_num) { + if (left_is_num_can_compare && right_is_num_can_compare) { if (!(execute_num_left_type(block, result, col_left_untyped, col_right_untyped) || execute_num_left_type(block, result, col_left_untyped, From 0ffb71b558949840bc3d83b073177b0952b82cbb Mon Sep 17 00:00:00 2001 From: James Date: Fri, 29 Nov 2024 18:09:48 +0800 Subject: [PATCH 081/399] [improvement](mysql)Support mysql COM_RESET_CONNECTION command. (#44747) ### What problem does this PR solve? Support mysql COM_RESET_CONNECTION command. Doris server side reset default catalog and return ctx.getState().setOk() Python test code: ``` import mysql.connector connection = mysql.connector.connect( host="172.20.32.136", port=29030, user="root", password="", database="" ) print(connection.cmd_reset_connection()) ``` Issue Number: close #xxx Related PR: #xxx Problem Summary: ### Release note None --- .../src/main/java/org/apache/doris/qe/ConnectContext.java | 5 +++++ .../main/java/org/apache/doris/qe/ConnectProcessor.java | 7 +++++++ .../java/org/apache/doris/qe/MysqlConnectProcessor.java | 3 +++ 3 files changed, 15 insertions(+) diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectContext.java b/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectContext.java index a16422ba9e5979..c21c9ee3f86db9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectContext.java @@ -342,6 +342,11 @@ public String removeLastDBOfCatalog(String catalog) { return lastDBOfCatalog.get(catalog); } + // Used by COM_RESET_CONNECTION + public void clearLastDBOfCatalog() { + lastDBOfCatalog.clear(); + } + public void setNotEvalNondeterministicFunction(boolean notEvalNondeterministicFunction) { this.notEvalNondeterministicFunction = notEvalNondeterministicFunction; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectProcessor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectProcessor.java index 2340aa37aebb55..fcc6c2362cf276 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectProcessor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectProcessor.java @@ -44,6 +44,7 @@ import org.apache.doris.common.util.SqlUtils; import org.apache.doris.common.util.Util; import org.apache.doris.datasource.CatalogIf; +import org.apache.doris.datasource.InternalCatalog; import org.apache.doris.metric.MetricRepo; import org.apache.doris.mysql.MysqlChannel; import org.apache.doris.mysql.MysqlPacket; @@ -198,6 +199,12 @@ protected void handleDebug() { ctx.getState().setOk(); } + protected void handleResetConnection() { + ctx.changeDefaultCatalog(InternalCatalog.INTERNAL_CATALOG_NAME); + ctx.clearLastDBOfCatalog(); + ctx.getState().setOk(); + } + protected void handleStmtReset() { ctx.getState().setOk(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/MysqlConnectProcessor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/MysqlConnectProcessor.java index 376d4740e632a1..da3baced5fad78 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/MysqlConnectProcessor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/MysqlConnectProcessor.java @@ -264,6 +264,9 @@ private void dispatch() throws IOException { case COM_SET_OPTION: handleSetOption(); break; + case COM_RESET_CONNECTION: + handleResetConnection(); + break; default: ctx.getState().setError(ErrorCode.ERR_UNKNOWN_COM_ERROR, "Unsupported command(" + command + ")"); LOG.warn("Unsupported command(" + command + ")"); From 851d780c14d9e880547181b826ee57953869944d Mon Sep 17 00:00:00 2001 From: Pxl Date: Fri, 29 Nov 2024 19:06:37 +0800 Subject: [PATCH 082/399] [Chore](check) add check for bf not inited and not ignored (#44781) add check for bf not inited and not ignored --- be/src/exprs/bloom_filter_func.h | 2 ++ be/src/exprs/runtime_filter.cpp | 7 +++++++ 2 files changed, 9 insertions(+) diff --git a/be/src/exprs/bloom_filter_func.h b/be/src/exprs/bloom_filter_func.h index 674a429581643b..ff749420ad286e 100644 --- a/be/src/exprs/bloom_filter_func.h +++ b/be/src/exprs/bloom_filter_func.h @@ -233,6 +233,8 @@ class BloomFilterFuncBase : public RuntimeFilterFuncBase { uint16_t* offsets, int number, bool is_parse_column) = 0; + bool inited() const { return _inited; } + private: void _limit_length() { if (_runtime_bloom_filter_min_size > 0) { diff --git a/be/src/exprs/runtime_filter.cpp b/be/src/exprs/runtime_filter.cpp index 19789fbd743cbc..e26452c9ef69e5 100644 --- a/be/src/exprs/runtime_filter.cpp +++ b/be/src/exprs/runtime_filter.cpp @@ -1283,6 +1283,13 @@ PrimitiveType IRuntimeFilter::column_type() const { void IRuntimeFilter::signal() { DCHECK(is_consumer()); + + if (!_wrapper->is_ignored() && _wrapper->is_bloomfilter() && + !_wrapper->get_bloomfilter()->inited()) { + throw Exception(ErrorCode::INTERNAL_ERROR, "bf not inited and not ignored, rf: {}", + debug_string()); + } + COUNTER_SET(_wait_timer, int64_t((MonotonicMillis() - registration_time_) * NANOS_PER_MILLIS)); _rf_state_atomic.store(RuntimeFilterState::READY); if (!_filter_timer.empty()) { From fed46d2d8467148c87e88ca8bec69280125dd8e3 Mon Sep 17 00:00:00 2001 From: hui lai Date: Fri, 29 Nov 2024 20:43:43 +0800 Subject: [PATCH 083/399] [improve](routine load) ensure load balance after scaling up BE nodes (#44693) We scaling up from three BE nodes to five BE nodes, but from the monitoring perspective, only the previous three nodes have written traffic. ![image](https://github.com/user-attachments/assets/947fe8a9-745d-4c37-93d4-15b0e27f12fc) This pr aims to ensure load balance after scaling up BE nodes. --- .../org/apache/doris/load/routineload/RoutineLoadManager.java | 1 + 1 file changed, 1 insertion(+) diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadManager.java b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadManager.java index 8b0ebab3fbe7d5..97f6aba8c589c6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadManager.java @@ -487,6 +487,7 @@ public long getAvailableBeForTask(long jobId, long previousBeId) throws LoadExce // check if be has idle slot readLock(); try { + updateBeIdToMaxConcurrentTasks(); Map beIdToConcurrentTasks = getBeCurrentTasksNumMap(); int previousBeIdleTaskNum = 0; From 5f8051ace2947f510cda40afb3ed825f0d0c3236 Mon Sep 17 00:00:00 2001 From: TengJianPing Date: Fri, 29 Nov 2024 20:46:45 +0800 Subject: [PATCH 084/399] [improvemment](profile) use one counter to represent MemoryUsage and MemoryUsagePeak (#44645) --- .../exec/aggregation_sink_operator.cpp | 2 - .../pipeline/exec/analytic_sink_operator.cpp | 1 - be/src/pipeline/exec/exchange_sink_buffer.cpp | 2 - .../pipeline/exec/exchange_sink_operator.cpp | 10 ---- be/src/pipeline/exec/hashjoin_build_sink.cpp | 2 - .../pipeline/exec/hashjoin_probe_operator.cpp | 2 - .../exec/join/process_hash_table_probe_impl.h | 1 - be/src/pipeline/exec/operator.cpp | 22 ++------ be/src/pipeline/exec/operator.h | 14 +---- be/src/pipeline/exec/scan_operator.cpp | 1 - be/src/pipeline/exec/scan_operator.h | 1 - be/src/pipeline/exec/sort_sink_operator.cpp | 1 - .../exec/spill_sort_sink_operator.cpp | 1 - .../exec/streaming_aggregation_operator.cpp | 3 - be/src/runtime/load_channel.cpp | 1 - be/src/runtime/load_channel.h | 1 - be/src/util/runtime_profile.cpp | 52 ++++++++---------- be/src/util/runtime_profile.h | 55 ++++++++++++++++--- be/src/vec/exec/scan/scanner_context.cpp | 10 +--- be/src/vec/exec/scan/scanner_context.h | 2 - be/src/vec/exec/scan/scanner_scheduler.cpp | 6 -- be/src/vec/runtime/vdata_stream_recvr.cpp | 4 -- 22 files changed, 79 insertions(+), 115 deletions(-) diff --git a/be/src/pipeline/exec/aggregation_sink_operator.cpp b/be/src/pipeline/exec/aggregation_sink_operator.cpp index 27400fba474eef..44e58535b75b71 100644 --- a/be/src/pipeline/exec/aggregation_sink_operator.cpp +++ b/be/src/pipeline/exec/aggregation_sink_operator.cpp @@ -231,7 +231,6 @@ void AggSinkLocalState::_update_memusage_with_serialized_key() { COUNTER_SET(_memory_used_counter, arena_memory_usage + hash_table_memory_usage); - COUNTER_SET(_peak_memory_usage_counter, _memory_used_counter->value()); COUNTER_SET(_serialize_key_arena_memory_usage, arena_memory_usage); COUNTER_SET(_hash_table_memory_usage, hash_table_memory_usage); @@ -415,7 +414,6 @@ Status AggSinkLocalState::_merge_without_key(vectorized::Block* block) { void AggSinkLocalState::_update_memusage_without_key() { int64_t arena_memory_usage = _agg_arena_pool->size(); COUNTER_SET(_memory_used_counter, arena_memory_usage); - COUNTER_SET(_peak_memory_usage_counter, arena_memory_usage); COUNTER_SET(_serialize_key_arena_memory_usage, arena_memory_usage); } diff --git a/be/src/pipeline/exec/analytic_sink_operator.cpp b/be/src/pipeline/exec/analytic_sink_operator.cpp index 377aeb6fa12be4..7cc25eef9446d6 100644 --- a/be/src/pipeline/exec/analytic_sink_operator.cpp +++ b/be/src/pipeline/exec/analytic_sink_operator.cpp @@ -323,7 +323,6 @@ Status AnalyticSinkOperatorX::sink(doris::RuntimeState* state, vectorized::Block } COUNTER_UPDATE(local_state._memory_used_counter, input_block->allocated_bytes()); - COUNTER_SET(local_state._peak_memory_usage_counter, local_state._memory_used_counter->value()); //TODO: if need improvement, the is a tips to maintain a free queue, //so the memory could reuse, no need to new/delete again; diff --git a/be/src/pipeline/exec/exchange_sink_buffer.cpp b/be/src/pipeline/exec/exchange_sink_buffer.cpp index 0f02ffc2b9a4b1..6e6108d13a919f 100644 --- a/be/src/pipeline/exec/exchange_sink_buffer.cpp +++ b/be/src/pipeline/exec/exchange_sink_buffer.cpp @@ -159,8 +159,6 @@ Status ExchangeSinkBuffer::add_block(TransmitInfo&& request) { RETURN_IF_ERROR( BeExecVersionManager::check_be_exec_version(request.block->be_exec_version())); COUNTER_UPDATE(_parent->memory_used_counter(), request.block->ByteSizeLong()); - COUNTER_SET(_parent->peak_memory_usage_counter(), - _parent->memory_used_counter()->value()); } _instance_to_package_queue[ins_id].emplace(std::move(request)); _total_queue_size++; diff --git a/be/src/pipeline/exec/exchange_sink_operator.cpp b/be/src/pipeline/exec/exchange_sink_operator.cpp index 85f58417197d52..dfa6df392b74ba 100644 --- a/be/src/pipeline/exec/exchange_sink_operator.cpp +++ b/be/src/pipeline/exec/exchange_sink_operator.cpp @@ -385,10 +385,6 @@ Status ExchangeSinkOperatorX::sink(RuntimeState* state, vectorized::Block* block if (all_receiver_eof) { return Status::EndOfFile("all data stream channels EOF"); } - Defer defer([&]() { - COUNTER_SET(local_state._peak_memory_usage_counter, - local_state._memory_used_counter->value()); - }); if (_part_type == TPartitionType::UNPARTITIONED || local_state.channels.size() == 1) { // 1. serialize depends on it is not local exchange @@ -505,8 +501,6 @@ Status ExchangeSinkOperatorX::sink(RuntimeState* state, vectorized::Block* block } COUNTER_UPDATE(local_state.memory_used_counter(), new_channel_mem_usage - old_channel_mem_usage); - COUNTER_SET(local_state.peak_memory_usage_counter(), - local_state.memory_used_counter()->value()); } else if (_part_type == TPartitionType::TABLET_SINK_SHUFFLE_PARTITIONED) { int64_t old_channel_mem_usage = 0; for (const auto& channel : local_state.channels) { @@ -555,8 +549,6 @@ Status ExchangeSinkOperatorX::sink(RuntimeState* state, vectorized::Block* block } COUNTER_UPDATE(local_state.memory_used_counter(), new_channel_mem_usage - old_channel_mem_usage); - COUNTER_SET(local_state.peak_memory_usage_counter(), - local_state.memory_used_counter()->value()); } else if (_part_type == TPartitionType::TABLE_SINK_HASH_PARTITIONED) { int64_t old_channel_mem_usage = 0; for (const auto& channel : local_state.channels) { @@ -581,8 +573,6 @@ Status ExchangeSinkOperatorX::sink(RuntimeState* state, vectorized::Block* block } COUNTER_UPDATE(local_state.memory_used_counter(), new_channel_mem_usage - old_channel_mem_usage); - COUNTER_SET(local_state.peak_memory_usage_counter(), - local_state.memory_used_counter()->value()); } else if (_part_type == TPartitionType::TABLE_SINK_RANDOM_PARTITIONED) { // Control the number of channels according to the flow, thereby controlling the number of table sink writers. // 1. select channel diff --git a/be/src/pipeline/exec/hashjoin_build_sink.cpp b/be/src/pipeline/exec/hashjoin_build_sink.cpp index a15dbb8dd03111..0a71b86bed0e86 100644 --- a/be/src/pipeline/exec/hashjoin_build_sink.cpp +++ b/be/src/pipeline/exec/hashjoin_build_sink.cpp @@ -310,7 +310,6 @@ Status HashJoinBuildSinkLocalState::process_build_block(RuntimeState* state, _build_blocks_memory_usage->value() + (int64_t)(arg.hash_table->get_byte_size() + arg.serialized_keys_size(true))); - COUNTER_SET(_peak_memory_usage_counter, _memory_used_counter->value()); return st; }}, _shared_state->hash_table_variants->method_variant, _shared_state->join_op_variants, @@ -492,7 +491,6 @@ Status HashJoinBuildSinkOperatorX::sink(RuntimeState* state, vectorized::Block* std::move(*in_block))); int64_t blocks_mem_usage = local_state._build_side_mutable_block.allocated_bytes(); COUNTER_SET(local_state._memory_used_counter, blocks_mem_usage); - COUNTER_SET(local_state._peak_memory_usage_counter, blocks_mem_usage); COUNTER_SET(local_state._build_blocks_memory_usage, blocks_mem_usage); } } diff --git a/be/src/pipeline/exec/hashjoin_probe_operator.cpp b/be/src/pipeline/exec/hashjoin_probe_operator.cpp index 426bfcb219dc04..7c663b256832ed 100644 --- a/be/src/pipeline/exec/hashjoin_probe_operator.cpp +++ b/be/src/pipeline/exec/hashjoin_probe_operator.cpp @@ -483,8 +483,6 @@ Status HashJoinProbeOperatorX::push(RuntimeState* state, vectorized::Block* inpu input_block->swap(local_state._probe_block); COUNTER_SET(local_state._memory_used_counter, (int64_t)local_state._probe_block.allocated_bytes()); - COUNTER_SET(local_state._peak_memory_usage_counter, - local_state._memory_used_counter->value()); } } return Status::OK(); diff --git a/be/src/pipeline/exec/join/process_hash_table_probe_impl.h b/be/src/pipeline/exec/join/process_hash_table_probe_impl.h index fc1153b3419d3e..24a9a7f67431da 100644 --- a/be/src/pipeline/exec/join/process_hash_table_probe_impl.h +++ b/be/src/pipeline/exec/join/process_hash_table_probe_impl.h @@ -184,7 +184,6 @@ typename HashTableType::State ProcessHashTableProbe::_init_probe_sid int64_t arena_memory_usage = hash_table_ctx.serialized_keys_size(false); COUNTER_SET(_parent->_probe_arena_memory_usage, arena_memory_usage); COUNTER_UPDATE(_parent->_memory_used_counter, arena_memory_usage); - COUNTER_SET(_parent->_peak_memory_usage_counter, _parent->_memory_used_counter->value()); } return typename HashTableType::State(_parent->_probe_columns); diff --git a/be/src/pipeline/exec/operator.cpp b/be/src/pipeline/exec/operator.cpp index 3b5174d87c0f7f..09d0eef0f04340 100644 --- a/be/src/pipeline/exec/operator.cpp +++ b/be/src/pipeline/exec/operator.cpp @@ -342,7 +342,6 @@ Status OperatorXBase::get_block_after_projects(RuntimeState* state, vectorized:: return status; } status = get_block(state, block, eos); - local_state->_peak_memory_usage_counter->set(local_state->_memory_used_counter->value()); return status; } @@ -441,11 +440,7 @@ PipelineXSinkLocalStateBase::PipelineXSinkLocalStateBase(DataSinkOperatorXBase* } PipelineXLocalStateBase::PipelineXLocalStateBase(RuntimeState* state, OperatorXBase* parent) - : _num_rows_returned(0), - _rows_returned_counter(nullptr), - _peak_memory_usage_counter(nullptr), - _parent(parent), - _state(state) { + : _num_rows_returned(0), _rows_returned_counter(nullptr), _parent(parent), _state(state) { _query_statistics = std::make_shared(); } @@ -484,9 +479,8 @@ Status PipelineXLocalState::init(RuntimeState* state, LocalState _open_timer = ADD_TIMER_WITH_LEVEL(_runtime_profile, "OpenTime", 1); _close_timer = ADD_TIMER_WITH_LEVEL(_runtime_profile, "CloseTime", 1); _exec_timer = ADD_TIMER_WITH_LEVEL(_runtime_profile, "ExecTime", 1); - _memory_used_counter = ADD_COUNTER_WITH_LEVEL(_runtime_profile, "MemoryUsage", TUnit::BYTES, 1); - _peak_memory_usage_counter = - _runtime_profile->AddHighWaterMarkCounter("MemoryUsagePeak", TUnit::BYTES, "", 1); + _memory_used_counter = + _runtime_profile->AddHighWaterMarkCounter("MemoryUsage", TUnit::BYTES, "", 1); return Status::OK(); } @@ -519,9 +513,6 @@ Status PipelineXLocalState::close(RuntimeState* state) { if constexpr (!std::is_same_v) { COUNTER_SET(_wait_for_dependency_timer, _dependency->watcher_elapse_time()); } - if (_peak_memory_usage_counter) { - _peak_memory_usage_counter->set(_memory_used_counter->value()); - } _closed = true; // Some kinds of source operators has a 1-1 relationship with a sink operator (such as AnalyticOperator). // We must ensure AnalyticSinkOperator will not be blocked if AnalyticSourceOperator already closed. @@ -560,9 +551,7 @@ Status PipelineXSinkLocalState::init(RuntimeState* state, LocalSink _close_timer = ADD_TIMER_WITH_LEVEL(_profile, "CloseTime", 1); _exec_timer = ADD_TIMER_WITH_LEVEL(_profile, "ExecTime", 1); info.parent_profile->add_child(_profile, true, nullptr); - _memory_used_counter = ADD_COUNTER_WITH_LEVEL(_profile, "MemoryUsage", TUnit::BYTES, 1); - _peak_memory_usage_counter = - _profile->AddHighWaterMarkCounter("MemoryUsagePeak", TUnit::BYTES, "", 1); + _memory_used_counter = _profile->AddHighWaterMarkCounter("MemoryUsage", TUnit::BYTES, "", 1); return Status::OK(); } @@ -574,9 +563,6 @@ Status PipelineXSinkLocalState::close(RuntimeState* state, Status e if constexpr (!std::is_same_v) { COUNTER_SET(_wait_for_dependency_timer, _dependency->watcher_elapse_time()); } - if (_peak_memory_usage_counter) { - _peak_memory_usage_counter->set(_memory_used_counter->value()); - } _closed = true; return Status::OK(); } diff --git a/be/src/pipeline/exec/operator.h b/be/src/pipeline/exec/operator.h index 6053b1a2f48e87..c84c4e7b43f981 100644 --- a/be/src/pipeline/exec/operator.h +++ b/be/src/pipeline/exec/operator.h @@ -165,9 +165,6 @@ class PipelineXLocalStateBase { RuntimeProfile::Counter* exec_time_counter() { return _exec_timer; } RuntimeProfile::Counter* memory_used_counter() { return _memory_used_counter; } - RuntimeProfile::HighWaterMarkCounter* peak_memory_usage_counter() { - return _peak_memory_usage_counter; - } OperatorXBase* parent() { return _parent; } RuntimeState* state() { return _state; } vectorized::VExprContextSPtrs& conjuncts() { return _conjuncts; } @@ -202,11 +199,10 @@ class PipelineXLocalStateBase { RuntimeProfile::Counter* _rows_returned_counter = nullptr; RuntimeProfile::Counter* _blocks_returned_counter = nullptr; RuntimeProfile::Counter* _wait_for_dependency_timer = nullptr; - RuntimeProfile::Counter* _memory_used_counter = nullptr; + // Account for current memory and peak memory used by this node + RuntimeProfile::HighWaterMarkCounter* _memory_used_counter = nullptr; RuntimeProfile::Counter* _projection_timer = nullptr; RuntimeProfile::Counter* _exec_timer = nullptr; - // Account for peak memory used by this node - RuntimeProfile::HighWaterMarkCounter* _peak_memory_usage_counter = nullptr; RuntimeProfile::Counter* _init_timer = nullptr; RuntimeProfile::Counter* _open_timer = nullptr; RuntimeProfile::Counter* _close_timer = nullptr; @@ -348,9 +344,6 @@ class PipelineXSinkLocalStateBase { RuntimeProfile::Counter* rows_input_counter() { return _rows_input_counter; } RuntimeProfile::Counter* exec_time_counter() { return _exec_timer; } RuntimeProfile::Counter* memory_used_counter() { return _memory_used_counter; } - RuntimeProfile::HighWaterMarkCounter* peak_memory_usage_counter() { - return _peak_memory_usage_counter; - } virtual std::vector dependencies() const { return {nullptr}; } // override in exchange sink , AsyncWriterSink @@ -380,8 +373,7 @@ class PipelineXSinkLocalStateBase { RuntimeProfile::Counter* _wait_for_dependency_timer = nullptr; RuntimeProfile::Counter* _wait_for_finish_dependency_timer = nullptr; RuntimeProfile::Counter* _exec_timer = nullptr; - RuntimeProfile::Counter* _memory_used_counter = nullptr; - RuntimeProfile::HighWaterMarkCounter* _peak_memory_usage_counter = nullptr; + RuntimeProfile::HighWaterMarkCounter* _memory_used_counter = nullptr; std::shared_ptr _query_statistics = nullptr; }; diff --git a/be/src/pipeline/exec/scan_operator.cpp b/be/src/pipeline/exec/scan_operator.cpp index 21c3103fe5a708..ae4396b22c7eec 100644 --- a/be/src/pipeline/exec/scan_operator.cpp +++ b/be/src/pipeline/exec/scan_operator.cpp @@ -1037,7 +1037,6 @@ Status ScanLocalState::_init_profile() { _total_throughput_counter = profile()->add_rate_counter("TotalReadThroughput", _rows_read_counter); _num_scanners = ADD_COUNTER(_runtime_profile, "NumScanners", TUnit::UNIT); - _scanner_peak_memory_usage = _peak_memory_usage_counter; //_runtime_profile->AddHighWaterMarkCounter("PeakMemoryUsage", TUnit::BYTES); // 2. counters for scanners diff --git a/be/src/pipeline/exec/scan_operator.h b/be/src/pipeline/exec/scan_operator.h index 5d41c800383bd0..4519a3ca283f6f 100644 --- a/be/src/pipeline/exec/scan_operator.h +++ b/be/src/pipeline/exec/scan_operator.h @@ -108,7 +108,6 @@ class ScanLocalStateBase : public PipelineXLocalState<>, public RuntimeFilterCon // Max num of scanner thread RuntimeProfile::Counter* _max_scanner_thread_num = nullptr; RuntimeProfile::HighWaterMarkCounter* _peak_running_scanner = nullptr; - RuntimeProfile::HighWaterMarkCounter* _scanner_peak_memory_usage = nullptr; // time of get block from scanner RuntimeProfile::Counter* _scan_timer = nullptr; RuntimeProfile::Counter* _scan_cpu_timer = nullptr; diff --git a/be/src/pipeline/exec/sort_sink_operator.cpp b/be/src/pipeline/exec/sort_sink_operator.cpp index faec4961af93b7..072f28723a36ea 100644 --- a/be/src/pipeline/exec/sort_sink_operator.cpp +++ b/be/src/pipeline/exec/sort_sink_operator.cpp @@ -128,7 +128,6 @@ Status SortSinkOperatorX::sink(doris::RuntimeState* state, vectorized::Block* in int64_t data_size = local_state._shared_state->sorter->data_size(); COUNTER_SET(local_state._sort_blocks_memory_usage, data_size); COUNTER_SET(local_state._memory_used_counter, data_size); - COUNTER_SET(local_state._peak_memory_usage_counter, data_size); RETURN_IF_CANCELLED(state); diff --git a/be/src/pipeline/exec/spill_sort_sink_operator.cpp b/be/src/pipeline/exec/spill_sort_sink_operator.cpp index 267bcc83aad92c..6e6689d4134deb 100644 --- a/be/src/pipeline/exec/spill_sort_sink_operator.cpp +++ b/be/src/pipeline/exec/spill_sort_sink_operator.cpp @@ -160,7 +160,6 @@ Status SpillSortSinkOperatorX::sink(doris::RuntimeState* state, vectorized::Bloc int64_t data_size = local_state._shared_state->in_mem_shared_state->sorter->data_size(); COUNTER_SET(local_state._sort_blocks_memory_usage, data_size); COUNTER_SET(local_state._memory_used_counter, data_size); - COUNTER_SET(local_state._peak_memory_usage_counter, data_size); if (eos) { if (local_state._shared_state->is_spilled) { diff --git a/be/src/pipeline/exec/streaming_aggregation_operator.cpp b/be/src/pipeline/exec/streaming_aggregation_operator.cpp index cf5071d62e4737..1c8d2c47bc698a 100644 --- a/be/src/pipeline/exec/streaming_aggregation_operator.cpp +++ b/be/src/pipeline/exec/streaming_aggregation_operator.cpp @@ -353,7 +353,6 @@ Status StreamingAggLocalState::_merge_without_key(vectorized::Block* block) { void StreamingAggLocalState::_update_memusage_without_key() { int64_t arena_memory_usage = _agg_arena_pool->size(); COUNTER_SET(_memory_used_counter, arena_memory_usage); - COUNTER_SET(_peak_memory_usage_counter, arena_memory_usage); COUNTER_SET(_serialize_key_arena_memory_usage, arena_memory_usage); } @@ -378,8 +377,6 @@ void StreamingAggLocalState::_update_memusage_with_serialized_key() { COUNTER_SET(_memory_used_counter, arena_memory_usage + hash_table_memory_usage); - COUNTER_SET(_peak_memory_usage_counter, - arena_memory_usage + hash_table_memory_usage); COUNTER_SET(_serialize_key_arena_memory_usage, arena_memory_usage); COUNTER_SET(_hash_table_memory_usage, hash_table_memory_usage); diff --git a/be/src/runtime/load_channel.cpp b/be/src/runtime/load_channel.cpp index 9369c0c833c53c..0cb313747b0373 100644 --- a/be/src/runtime/load_channel.cpp +++ b/be/src/runtime/load_channel.cpp @@ -98,7 +98,6 @@ void LoadChannel::_init_profile() { _load_id.to_string(), _sender_ip, _backend_id), true, true); _add_batch_number_counter = ADD_COUNTER(_self_profile, "NumberBatchAdded", TUnit::UNIT); - _peak_memory_usage_counter = ADD_COUNTER(_self_profile, "PeakMemoryUsage", TUnit::BYTES); _add_batch_timer = ADD_TIMER(_self_profile, "AddBatchTime"); _handle_eos_timer = ADD_CHILD_TIMER(_self_profile, "HandleEosTime", "AddBatchTime"); _add_batch_times = ADD_COUNTER(_self_profile, "AddBatchTimes", TUnit::UNIT); diff --git a/be/src/runtime/load_channel.h b/be/src/runtime/load_channel.h index 36a8f363ba9bac..2889bcf256515b 100644 --- a/be/src/runtime/load_channel.h +++ b/be/src/runtime/load_channel.h @@ -91,7 +91,6 @@ class LoadChannel { std::unique_ptr _profile; RuntimeProfile* _self_profile = nullptr; RuntimeProfile::Counter* _add_batch_number_counter = nullptr; - RuntimeProfile::Counter* _peak_memory_usage_counter = nullptr; RuntimeProfile::Counter* _add_batch_timer = nullptr; RuntimeProfile::Counter* _add_batch_times = nullptr; RuntimeProfile::Counter* _mgr_add_batch_timer = nullptr; diff --git a/be/src/util/runtime_profile.cpp b/be/src/util/runtime_profile.cpp index e87301880d2479..45db607a342743 100644 --- a/be/src/util/runtime_profile.cpp +++ b/be/src/util/runtime_profile.cpp @@ -363,28 +363,24 @@ const std::string* RuntimeProfile::get_info_string(const std::string& key) { return &it->second; } -#define ADD_COUNTER_IMPL(NAME, T) \ - RuntimeProfile::T* RuntimeProfile::NAME(const std::string& name, TUnit::type unit, \ - const std::string& parent_counter_name, \ - int64_t level) { \ - DCHECK_EQ(_is_averaged_profile, false); \ - std::lock_guard l(_counter_map_lock); \ - if (_counter_map.find(name) != _counter_map.end()) { \ - return reinterpret_cast(_counter_map[name]); \ - } \ - DCHECK(parent_counter_name == ROOT_COUNTER || \ - _counter_map.find(parent_counter_name) != _counter_map.end()); \ - T* counter = _pool->add(new T(unit, level)); \ - _counter_map[name] = counter; \ - std::set* child_counters = \ - find_or_insert(&_child_counter_map, parent_counter_name, std::set()); \ - child_counters->insert(name); \ - return counter; \ - } - -//ADD_COUNTER_IMPL(AddCounter, Counter); -ADD_COUNTER_IMPL(AddHighWaterMarkCounter, HighWaterMarkCounter); -//ADD_COUNTER_IMPL(AddConcurrentTimerCounter, ConcurrentTimerCounter); +RuntimeProfile::HighWaterMarkCounter* RuntimeProfile::AddHighWaterMarkCounter( + const std::string& name, TUnit::type unit, const std::string& parent_counter_name, + int64_t level) { + DCHECK_EQ(_is_averaged_profile, false); + std::lock_guard l(_counter_map_lock); + if (_counter_map.find(name) != _counter_map.end()) { + return reinterpret_cast(_counter_map[name]); + } + DCHECK(parent_counter_name == ROOT_COUNTER || + _counter_map.find(parent_counter_name) != _counter_map.end()); + RuntimeProfile::HighWaterMarkCounter* counter = + _pool->add(new RuntimeProfile::HighWaterMarkCounter(unit, level, parent_counter_name)); + _counter_map[name] = counter; + std::set* child_counters = + find_or_insert(&_child_counter_map, parent_counter_name, std::set()); + child_counters->insert(name); + return counter; +} std::shared_ptr RuntimeProfile::AddSharedHighWaterMarkCounter( const std::string& name, TUnit::type unit, const std::string& parent_counter_name) { @@ -395,7 +391,8 @@ std::shared_ptr RuntimeProfile::AddSharedH } DCHECK(parent_counter_name == ROOT_COUNTER || _counter_map.find(parent_counter_name) != _counter_map.end()); - std::shared_ptr counter = std::make_shared(unit); + std::shared_ptr counter = + std::make_shared(unit, 2, parent_counter_name); _shared_counter_pool[name] = counter; DCHECK(_counter_map.find(name) == _counter_map.end()) @@ -697,17 +694,14 @@ void RuntimeProfile::print_child_counters(const std::string& prefix, const CounterMap& counter_map, const ChildCounterMap& child_counter_map, std::ostream* s) { - std::ostream& stream = *s; - ChildCounterMap::const_iterator itr = child_counter_map.find(counter_name); + auto itr = child_counter_map.find(counter_name); if (itr != child_counter_map.end()) { const std::set& child_counters = itr->second; for (const std::string& child_counter : child_counters) { - CounterMap::const_iterator iter = counter_map.find(child_counter); + auto iter = counter_map.find(child_counter); DCHECK(iter != counter_map.end()); - stream << prefix << " - " << iter->first << ": " - << PrettyPrinter::print(iter->second->value(), iter->second->type()) - << std::endl; + iter->second->pretty_print(s, prefix, iter->first); RuntimeProfile::print_child_counters(prefix + " ", child_counter, counter_map, child_counter_map, s); } diff --git a/be/src/util/runtime_profile.h b/be/src/util/runtime_profile.h index 955d77b72aa51c..6e393ac673a628 100644 --- a/be/src/util/runtime_profile.h +++ b/be/src/util/runtime_profile.h @@ -39,6 +39,7 @@ #include "common/compiler_util.h" // IWYU pragma: keep #include "util/binary_cast.hpp" +#include "util/container_util.hpp" #include "util/pretty_printer.h" #include "util/stopwatch.hpp" @@ -126,6 +127,14 @@ class RuntimeProfile { tcounters.push_back(std::move(counter)); } + virtual void pretty_print(std::ostream* s, const std::string& prefix, + const std::string& name) const { + std::ostream& stream = *s; + stream << prefix << " - " << name << ": " + << PrettyPrinter::print(_value.load(std::memory_order_relaxed), type()) + << std::endl; + } + TUnit::type type() const { return _type; } virtual int64_t level() { return _level; } @@ -142,15 +151,49 @@ class RuntimeProfile { /// as value()) and the current value. class HighWaterMarkCounter : public Counter { public: - HighWaterMarkCounter(TUnit::type unit, int64_t level = 2) - : Counter(unit, 0, level), current_value_(0) {} + HighWaterMarkCounter(TUnit::type unit, int64_t level, const std::string& parent_name) + : Counter(unit, 0, level), current_value_(0), _parent_name(parent_name) {} - virtual void add(int64_t delta) { + void add(int64_t delta) { current_value_.fetch_add(delta, std::memory_order_relaxed); if (delta > 0) { UpdateMax(current_value_); } } + virtual void update(int64_t delta) override { add(delta); } + + virtual void to_thrift( + const std::string& name, std::vector& tcounters, + std::map>& child_counters_map) override { + { + TCounter counter; + counter.name = name; + counter.value = this->current_value(); + counter.type = this->type(); + counter.__set_level(this->level()); + tcounters.push_back(std::move(counter)); + } + { + TCounter counter; + std::string peak_name = name + "Peak"; + counter.name = peak_name; + counter.value = this->value(); + counter.type = this->type(); + counter.__set_level(this->level()); + tcounters.push_back(std::move(counter)); + child_counters_map[_parent_name].insert(peak_name); + } + } + + virtual void pretty_print(std::ostream* s, const std::string& prefix, + const std::string& name) const override { + std::ostream& stream = *s; + stream << prefix << " - " << name << ": " + << PrettyPrinter::print(current_value(), type()) << std::endl; + stream << prefix << " - " << name << "Peak: " + << PrettyPrinter::print(_value.load(std::memory_order_relaxed), type()) + << std::endl; + } /// Tries to increase the current value by delta. If current_value() + delta /// exceeds max, return false and current_value is not changed. @@ -194,6 +237,8 @@ class RuntimeProfile { /// The current value of the counter. _value in the super class represents /// the high water mark. std::atomic current_value_; + + const std::string _parent_name; }; using DerivedCounterFunction = std::function; @@ -561,10 +606,6 @@ class RuntimeProfile { static void print_child_counters(const std::string& prefix, const std::string& counter_name, const CounterMap& counter_map, const ChildCounterMap& child_counter_map, std::ostream* s); - - static std::string print_counter(Counter* counter) { - return PrettyPrinter::print(counter->value(), counter->type()); - } }; // Utility class to update the counter at object construction and destruction. diff --git a/be/src/vec/exec/scan/scanner_context.cpp b/be/src/vec/exec/scan/scanner_context.cpp index d37d26b09f7815..65812cb428a8f8 100644 --- a/be/src/vec/exec/scan/scanner_context.cpp +++ b/be/src/vec/exec/scan/scanner_context.cpp @@ -236,7 +236,6 @@ vectorized::BlockUPtr ScannerContext::get_free_block(bool force) { _scanner_memory_used_counter->set(_block_memory_usage); // A free block is reused, so the memory usage should be decreased // The caller of get_free_block will increase the memory usage - update_peak_memory_usage(-block->allocated_bytes()); } else if (_block_memory_usage < _max_bytes_in_queue || force) { _newly_create_free_blocks_num->update(1); block = vectorized::Block::create_unique(_output_tuple_desc->slots(), 0, @@ -251,9 +250,7 @@ void ScannerContext::return_free_block(vectorized::BlockUPtr block) { _block_memory_usage += block_size_to_reuse; _scanner_memory_used_counter->set(_block_memory_usage); block->clear_column_data(); - if (_free_blocks.enqueue(std::move(block))) { - update_peak_memory_usage(block_size_to_reuse); - } + _free_blocks.enqueue(std::move(block)); } } @@ -324,7 +321,6 @@ Status ScannerContext::get_block_from_queue(RuntimeState* state, vectorized::Blo _estimated_block_size = block_size; } _block_memory_usage -= block_size; - update_peak_memory_usage(-current_block->allocated_bytes()); // consume current block block->swap(*current_block); return_free_block(std::move(current_block)); @@ -540,8 +536,4 @@ void ScannerContext::update_peak_running_scanner(int num) { _local_state->_peak_running_scanner->add(num); } -void ScannerContext::update_peak_memory_usage(int64_t usage) { - _local_state->_scanner_peak_memory_usage->add(usage); -} - } // namespace doris::vectorized diff --git a/be/src/vec/exec/scan/scanner_context.h b/be/src/vec/exec/scan/scanner_context.h index 82e0a06799940b..d1cf06d56686ac 100644 --- a/be/src/vec/exec/scan/scanner_context.h +++ b/be/src/vec/exec/scan/scanner_context.h @@ -127,8 +127,6 @@ class ScannerContext : public std::enable_shared_from_this, // Caller should make sure the pipeline task is still running when calling this function void update_peak_running_scanner(int num); - // Caller should make sure the pipeline task is still running when calling this function - void update_peak_memory_usage(int64_t usage); // Get next block from blocks queue. Called by ScanNode/ScanOperator // Set eos to true if there is no more data to read. diff --git a/be/src/vec/exec/scan/scanner_scheduler.cpp b/be/src/vec/exec/scan/scanner_scheduler.cpp index 385b581d2a5725..7c5aa8db0a77af 100644 --- a/be/src/vec/exec/scan/scanner_scheduler.cpp +++ b/be/src/vec/exec/scan/scanner_scheduler.cpp @@ -283,8 +283,6 @@ void ScannerScheduler::_scanner_scan(std::shared_ptr ctx, break; } // We got a new created block or a reused block. - ctx->update_peak_memory_usage(free_block->allocated_bytes()); - ctx->update_peak_memory_usage(-free_block->allocated_bytes()); status = scanner->get_block_after_projects(state, free_block.get(), &eos); first_read = false; if (!status.ok()) { @@ -293,7 +291,6 @@ void ScannerScheduler::_scanner_scan(std::shared_ptr ctx, } // Projection will truncate useless columns, makes block size change. auto free_block_bytes = free_block->allocated_bytes(); - ctx->update_peak_memory_usage(free_block_bytes); raw_bytes_read += free_block_bytes; if (!scan_task->cached_blocks.empty() && scan_task->cached_blocks.back().first->rows() + free_block->rows() <= @@ -301,9 +298,7 @@ void ScannerScheduler::_scanner_scan(std::shared_ptr ctx, size_t block_size = scan_task->cached_blocks.back().first->allocated_bytes(); vectorized::MutableBlock mutable_block( scan_task->cached_blocks.back().first.get()); - ctx->update_peak_memory_usage(-mutable_block.allocated_bytes()); status = mutable_block.merge(*free_block); - ctx->update_peak_memory_usage(mutable_block.allocated_bytes()); if (!status.ok()) { LOG(WARNING) << "Block merge failed: " << status.to_string(); break; @@ -313,7 +308,6 @@ void ScannerScheduler::_scanner_scan(std::shared_ptr ctx, std::move(mutable_block.mutable_columns())); // Return block succeed or not, this free_block is not used by this scan task any more. - ctx->update_peak_memory_usage(-free_block_bytes); // If block can be reused, its memory usage will be added back. ctx->return_free_block(std::move(free_block)); ctx->inc_block_usage(scan_task->cached_blocks.back().first->allocated_bytes() - diff --git a/be/src/vec/runtime/vdata_stream_recvr.cpp b/be/src/vec/runtime/vdata_stream_recvr.cpp index 81e4e1cd5f037e..c3277b0917e84c 100644 --- a/be/src/vec/runtime/vdata_stream_recvr.cpp +++ b/be/src/vec/runtime/vdata_stream_recvr.cpp @@ -199,8 +199,6 @@ Status VDataStreamRecvr::SenderQueue::add_block(std::unique_ptr pblock, *done = nullptr; } _recvr->_parent->memory_used_counter()->update(block_byte_size); - _recvr->_parent->peak_memory_usage_counter()->set( - _recvr->_parent->memory_used_counter()->value()); add_blocks_memory_usage(block_byte_size); return Status::OK(); } @@ -240,8 +238,6 @@ void VDataStreamRecvr::SenderQueue::add_block(Block* block, bool use_move) { try_set_dep_ready_without_lock(); COUNTER_UPDATE(_recvr->_local_bytes_received_counter, block_mem_size); _recvr->_parent->memory_used_counter()->update(block_mem_size); - _recvr->_parent->peak_memory_usage_counter()->set( - _recvr->_parent->memory_used_counter()->value()); add_blocks_memory_usage(block_mem_size); } } From cd0cd55f11d21a9bf25a02c1567e378c1f672e07 Mon Sep 17 00:00:00 2001 From: bobhan1 Date: Fri, 29 Nov 2024 20:52:54 +0800 Subject: [PATCH 085/399] [opt](auto-inc) Allow to miss auto-increment column and other value columns in partial update (#44528) Previously, if a partial update misses auto-increment key columns and other value columns, the load will fail with error `distributed column not found, column=xxx`. Considering the uniqueness of the generated value of auto-increment column, this PR converts the partial update load to upsert in this situation to make it llegal. ### Release note Allow to miss auto-increment key column and other value columns in partial update load --- .../plans/commands/insert/InsertUtils.java | 10 +++++ .../test_partial_update_auto_inc.out | 8 ++++ .../test_partial_update_auto_inc.groovy | 42 ++++++++++++++++++- 3 files changed, 59 insertions(+), 1 deletion(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertUtils.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertUtils.java index 6b301c16f73aa2..60e7e5bf805a64 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertUtils.java @@ -297,6 +297,7 @@ public static Plan normalizePlan(Plan plan, TableIf table, Optional insertCol = unboundLogicalSink.getColNames().stream() .filter(c -> c.equalsIgnoreCase(col.getName())).findFirst(); @@ -313,9 +314,18 @@ public static Plan normalizePlan(Plan plan, TableIf table, Optional) unboundLogicalSink).setPartialUpdate(false); + } else { + if (hasMissingAutoIncKey) { + // becuase of the uniqueness of genetaed value of auto-increment column, + // we convert this load to upsert when is misses auto-increment key column + ((UnboundTableSink) unboundLogicalSink).setPartialUpdate(false); + } } } } diff --git a/regression-test/data/unique_with_mow_p0/partial_update/test_partial_update_auto_inc.out b/regression-test/data/unique_with_mow_p0/partial_update/test_partial_update_auto_inc.out index 19b192c95facb9..b672eb10570508 100644 --- a/regression-test/data/unique_with_mow_p0/partial_update/test_partial_update_auto_inc.out +++ b/regression-test/data/unique_with_mow_p0/partial_update/test_partial_update_auto_inc.out @@ -35,3 +35,11 @@ doris9 3 888 888 30 4 40 40 40 +-- !sql -- +test1 15 +test2 29 +test3 49 + +-- !sql -- +3 + diff --git a/regression-test/suites/unique_with_mow_p0/partial_update/test_partial_update_auto_inc.groovy b/regression-test/suites/unique_with_mow_p0/partial_update/test_partial_update_auto_inc.groovy index 1ec60fbb10e837..d689deeb8afe02 100644 --- a/regression-test/suites/unique_with_mow_p0/partial_update/test_partial_update_auto_inc.groovy +++ b/regression-test/suites/unique_with_mow_p0/partial_update/test_partial_update_auto_inc.groovy @@ -103,6 +103,46 @@ suite("test_partial_update_auto_inc") { time 10000 } order_qt_select_6 "select * from test_primary_key_partial_update_auto_inc2" - sql """ DROP TABLE IF EXISTS test_primary_key_partial_update_auto_inc2 """ + + + sql """ DROP TABLE IF EXISTS test_primary_key_partial_update_auto_inc3 force; """ + sql """ create table test_primary_key_partial_update_auto_inc3 + ( + `id` bigint not null AUTO_INCREMENT, + `project_code` varchar(20) not null, + `period_num` int, + `c2` int + ) unique KEY(`id`) + DISTRIBUTED BY HASH(`id`) BUCKETS auto + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1", + "enable_unique_key_merge_on_write" = "true" + ); """ + sql "set enable_unique_key_partial_update=true;" + sql "set enable_insert_strict=false;" + sql "sync;" + + sql "insert into test_primary_key_partial_update_auto_inc3(project_code,period_num) values ('test1',15),('test2',29),('test3',49);" + qt_sql "select project_code,period_num from test_primary_key_partial_update_auto_inc3 order by project_code,period_num;" + qt_sql "select count(distinct id) from test_primary_key_partial_update_auto_inc3;" + + + sql """ DROP TABLE IF EXISTS test_primary_key_partial_update_auto_inc4 """ + sql """ CREATE TABLE test_primary_key_partial_update_auto_inc4 ( + `k1` BIGINT NOT NULL AUTO_INCREMENT, + `k2` int, + `c1` int, + `c2` int, + `c3` int) + UNIQUE KEY(`k1`,`k2`) DISTRIBUTED BY HASH(`k1`,`k2`) BUCKETS 1 + PROPERTIES("replication_num" = "1", "enable_unique_key_merge_on_write" = "true"); """ + sql "set enable_unique_key_partial_update=true;" + sql "set enable_insert_strict=false;" + sql "sync;" + + test { + sql "insert into test_primary_key_partial_update_auto_inc4(c1,c2) values(1,1),(2,2),(3,3)" + exception "Partial update should include all key columns, missing: k2" + } } } From 7c6ece0fd1d47f56f31e4bf82ac86d527fa3f1af Mon Sep 17 00:00:00 2001 From: bobhan1 Date: Fri, 29 Nov 2024 21:30:36 +0800 Subject: [PATCH 086/399] [fix](case) exclude some case when `random_add_cluster_keys_for_mow` is enabled (#44775) --- .../src/main/groovy/org/apache/doris/regression/Config.groovy | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/regression-test/framework/src/main/groovy/org/apache/doris/regression/Config.groovy b/regression-test/framework/src/main/groovy/org/apache/doris/regression/Config.groovy index 5aa6d7ce7e7218..379d177fbc280d 100644 --- a/regression-test/framework/src/main/groovy/org/apache/doris/regression/Config.groovy +++ b/regression-test/framework/src/main/groovy/org/apache/doris/regression/Config.groovy @@ -1025,8 +1025,10 @@ class Config { if (isCKEnabled) { excludeDirectorySet.add("unique_with_mow_p0/partial_update") excludeDirectorySet.add("unique_with_mow_p0/flexible") + excludeDirectorySet.add("fault_injection_p0/partial_update") + excludeDirectorySet.add("fault_injection_p0/flexible") excludeDirectorySet.add("doc") - List excludeCases = ["test_table_properties", "test_default_hll", "test_default_pi", "test_full_compaction", "test_full_compaction_by_table_id", "test_create_table", "txn_insert", "test_update_mow", "test_new_update", "test_update_unique", "test_partial_update_generated_column", "nereids_partial_update_native_insert_stmt", "partial_update", "nereids_update_on_current_timestamp", "update_on_current_timestamp", "test_default_bitmap_empty", "nereids_delete_mow_partial_update", "delete_mow_partial_update", "partial_update_seq_col", "nereids_partial_update_native_insert_stmt_complex", "regression_test_variant_delete_and_update", "test_unique_table_auto_inc_partial_update_correct_stream_load", "test_unique_table_auto_inc", "test_unique_table_auto_inc_partial_update_correct_insert", "test_update_schema_change"] + List excludeCases = ["test_table_properties", "test_default_hll", "test_default_pi", "test_full_compaction", "test_full_compaction_by_table_id", "test_create_table", "txn_insert", "test_update_mow", "test_new_update", "test_update_unique", "test_partial_update_generated_column", "nereids_partial_update_native_insert_stmt", "partial_update", "nereids_update_on_current_timestamp", "update_on_current_timestamp", "test_default_bitmap_empty", "nereids_delete_mow_partial_update", "delete_mow_partial_update", "partial_update_seq_col", "nereids_partial_update_native_insert_stmt_complex", "regression_test_variant_delete_and_update", "test_unique_table_auto_inc_partial_update_correct_stream_load", "test_unique_table_auto_inc", "test_unique_table_auto_inc_partial_update_correct_insert", "test_update_schema_change", "test_partial_update_rowset_not_found_fault_injection"] for (def excludeCase in excludeCases) { excludeSuiteWildcard.add(excludeCase) } From c34c8899f0bbcd27a4b4e8288195727c618b4112 Mon Sep 17 00:00:00 2001 From: airborne12 Date: Fri, 29 Nov 2024 23:20:57 +0800 Subject: [PATCH 087/399] [feat](test) add invalid params check and add ut case for bf (#44778) add bf ut --- be/src/olap/rowset/segment_v2/bloom_filter.h | 10 +- be/test/olap/date_bloom_filter_test.cpp | 2 +- .../rowset/segment_v2/bloom_filter_test.cpp | 302 ++++++++++++++++++ 3 files changed, 311 insertions(+), 3 deletions(-) create mode 100644 be/test/olap/rowset/segment_v2/bloom_filter_test.cpp diff --git a/be/src/olap/rowset/segment_v2/bloom_filter.h b/be/src/olap/rowset/segment_v2/bloom_filter.h index 20a903e65c1b3b..a7845d1ca36704 100644 --- a/be/src/olap/rowset/segment_v2/bloom_filter.h +++ b/be/src/olap/rowset/segment_v2/bloom_filter.h @@ -132,14 +132,20 @@ class BloomFilter { // for read // use deep copy to acquire the data virtual Status init(const char* buf, uint32_t size, HashStrategyPB strategy) { + if (size <= 1) { + return Status::InvalidArgument("invalid size:{}", size); + } DCHECK(size > 1); if (strategy == HASH_MURMUR3_X64_64) { _hash_func = murmur_hash3_x64_64; } else { return Status::InvalidArgument("invalid strategy:{}", strategy); } - if (size == 0) { - return Status::InvalidArgument("invalid size:{}", size); + if (buf == nullptr) { + return Status::InvalidArgument("buf is nullptr"); + } + if (((size - 1) & (size - 2)) != 0) { + return Status::InvalidArgument("size - 1 must be power of two"); } _data = new char[size]; memcpy(_data, buf, size); diff --git a/be/test/olap/date_bloom_filter_test.cpp b/be/test/olap/date_bloom_filter_test.cpp index 1678f5ee71a9e1..715301419e228f 100644 --- a/be/test/olap/date_bloom_filter_test.cpp +++ b/be/test/olap/date_bloom_filter_test.cpp @@ -59,7 +59,7 @@ class DateBloomFilterTest : public ::testing::Test { construct_column(schema_pb.add_column(), 0, "DATE", "date_column"); construct_column(schema_pb.add_column(), 1, "DATETIME", "datetime_column"); - + schema_pb.set_bf_fpp(0.05); _tablet_schema.reset(new TabletSchema); _tablet_schema->init_from_pb(schema_pb); diff --git a/be/test/olap/rowset/segment_v2/bloom_filter_test.cpp b/be/test/olap/rowset/segment_v2/bloom_filter_test.cpp new file mode 100644 index 00000000000000..355482173f28b0 --- /dev/null +++ b/be/test/olap/rowset/segment_v2/bloom_filter_test.cpp @@ -0,0 +1,302 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. +#include "olap/rowset/segment_v2/bloom_filter.h" + +#include + +#include +#include +#include +#include + +using namespace doris; +using namespace segment_v2; + +// Test class for BloomFilter +class BloomFilterTest : public testing::Test { +public: + void SetUp() override { + // Any setup required before each test + } + void TearDown() override { + // Any cleanup required after each test + } +}; + +// Test the creation of BloomFilter instances +TEST_F(BloomFilterTest, TestCreateBloomFilter) { + // Create a BlockSplitBloomFilter + std::unique_ptr bf; + Status st = BloomFilter::create(BLOCK_BLOOM_FILTER, &bf); + EXPECT_TRUE(st.ok()); + EXPECT_NE(bf, nullptr); + + // Create an NGramBloomFilter with a specific size + std::unique_ptr ngram_bf; + size_t bf_size = 1024; + st = BloomFilter::create(NGRAM_BLOOM_FILTER, &ngram_bf, bf_size); + EXPECT_TRUE(st.ok()); + EXPECT_NE(ngram_bf, nullptr); +} + +// Test the initialization of the BloomFilter with filter size +TEST_F(BloomFilterTest, TestInitBloomFilterWithFilterSize) { + std::unique_ptr bf; + Status st = BloomFilter::create(BLOCK_BLOOM_FILTER, &bf); + EXPECT_TRUE(st.ok()); + + // Initialize with a specific filter size + uint64_t filter_size = 1024; // Filter size in bytes + st = bf->init(filter_size); + EXPECT_TRUE(st.ok()); + + // Verify the internal size is set correctly + EXPECT_EQ(bf->num_bytes(), static_cast(filter_size)); +} + +// Test the initialization of the BloomFilter with expected items and FPP +TEST_F(BloomFilterTest, TestInitBloomFilter) { + std::unique_ptr bf; + Status st = BloomFilter::create(BLOCK_BLOOM_FILTER, &bf); + EXPECT_TRUE(st.ok()); + + // Initialize with expected number of items and FPP + uint64_t n = 1000; // Expected number of items + double fpp = 0.01; // Desired false positive probability + st = bf->init(n, fpp, HASH_MURMUR3_X64_64); + EXPECT_TRUE(st.ok()); + + // Verify the internal size is set correctly + uint32_t expected_num_bits = BloomFilter::optimal_bit_num(n, fpp); + uint32_t expected_num_bytes = expected_num_bits / 8; + EXPECT_EQ(bf->num_bytes(), expected_num_bytes); +} + +// Test the num_bytes() function +TEST_F(BloomFilterTest, TestNumBytesFunction) { + std::unique_ptr bf; + Status st = BloomFilter::create(BLOCK_BLOOM_FILTER, &bf); + EXPECT_TRUE(st.ok()); + + // Initialize with a specific filter size + uint64_t filter_size = 2048; // Filter size in bytes + st = bf->init(filter_size); + EXPECT_TRUE(st.ok()); + + // Verify num_bytes() returns the correct value + EXPECT_EQ(bf->num_bytes(), static_cast(filter_size)); +} + +// Test the contains() function +TEST_F(BloomFilterTest, TestContainsFunction) { + // Create and initialize two BloomFilters + std::unique_ptr bf1; + Status st = BloomFilter::create(BLOCK_BLOOM_FILTER, &bf1); + EXPECT_TRUE(st.ok()); + uint64_t n = 1000; + double fpp = 0.01; + st = bf1->init(n, fpp, HASH_MURMUR3_X64_64); + EXPECT_TRUE(st.ok()); + + std::unique_ptr bf2; + st = BloomFilter::create(BLOCK_BLOOM_FILTER, &bf2); + EXPECT_TRUE(st.ok()); + st = bf2->init(n, fpp, HASH_MURMUR3_X64_64); + EXPECT_TRUE(st.ok()); + + // Add elements to bf1 + for (int i = 0; i < n; ++i) { + std::string elem = "bf1_element_" + std::to_string(i); + bf1->add_bytes(elem.c_str(), elem.size()); + } + + // Add elements to bf2 + for (int i = 0; i < n; ++i) { + std::string elem = "bf2_element_" + std::to_string(i); + bf2->add_bytes(elem.c_str(), elem.size()); + } + + // Test the contains() function + // Since the base class implementation always returns true, we expect true + bool result = bf1->contains(*bf2); + EXPECT_TRUE(result); +} + +// Test adding elements and checking them +TEST_F(BloomFilterTest, TestAddAndTestElements) { + std::unique_ptr bf; + Status st = BloomFilter::create(BLOCK_BLOOM_FILTER, &bf); + EXPECT_TRUE(st.ok()); + + uint64_t n = 1000; + double fpp = 0.01; + st = bf->init(n, fpp, HASH_MURMUR3_X64_64); + EXPECT_TRUE(st.ok()); + + // Add elements to the BloomFilter + std::vector inserted_elements; + for (int i = 0; i < n; ++i) { + std::string elem = "element_" + std::to_string(i); + bf->add_bytes(elem.c_str(), elem.size()); + inserted_elements.push_back(elem); + } + + // Test that inserted elements are found + for (const auto& elem : inserted_elements) { + bool exists = bf->test_bytes(elem.c_str(), elem.size()); + EXPECT_TRUE(exists); + } + + // Test that non-inserted elements are not found (allowing for false positives) + int false_positives = 0; + int test_count = 10000; + for (int i = 0; i < test_count; ++i) { + std::string elem = "nonexistent_element_" + std::to_string(i); + bool exists = bf->test_bytes(elem.c_str(), elem.size()); + if (exists) { + false_positives++; + } + } + double actual_fpp = static_cast(false_positives) / test_count; + std::cout << "Expected FPP: " << fpp << ", Actual FPP: " << actual_fpp << std::endl; + // Allow some margin in FPP + EXPECT_LE(actual_fpp, fpp * 1.2); +} + +// Test the merge function +TEST_F(BloomFilterTest, TestMergeBloomFilters) { + // Create and initialize two BloomFilters + std::unique_ptr bf1; + Status st = BloomFilter::create(BLOCK_BLOOM_FILTER, &bf1); + EXPECT_TRUE(st.ok()); + uint64_t n = 1000; + double fpp = 0.01; + st = bf1->init(n, fpp, HASH_MURMUR3_X64_64); + EXPECT_TRUE(st.ok()); + + std::unique_ptr bf2; + st = BloomFilter::create(BLOCK_BLOOM_FILTER, &bf2); + EXPECT_TRUE(st.ok()); + st = bf2->init(n, fpp, HASH_MURMUR3_X64_64); + EXPECT_TRUE(st.ok()); + + // Add elements to bf1 + std::vector elements_bf1; + for (int i = 0; i < n; ++i) { + std::string elem = "bf1_element_" + std::to_string(i); + bf1->add_bytes(elem.c_str(), elem.size()); + elements_bf1.push_back(elem); + } + + // Add elements to bf2 + std::vector elements_bf2; + for (int i = 0; i < n; ++i) { + std::string elem = "bf2_element_" + std::to_string(i); + bf2->add_bytes(elem.c_str(), elem.size()); + elements_bf2.push_back(elem); + } + + // Merge bf2 into bf1 + st = bf1->merge(bf2.get()); + EXPECT_TRUE(st.ok()); + + // Test that elements from bf1 are found + for (const auto& elem : elements_bf1) { + bool exists = bf1->test_bytes(elem.c_str(), elem.size()); + EXPECT_TRUE(exists); + } + + // Test that elements from bf2 are found in merged bf1 + for (const auto& elem : elements_bf2) { + bool exists = bf1->test_bytes(elem.c_str(), elem.size()); + EXPECT_TRUE(exists); + } +} + +// Test null value handling +TEST_F(BloomFilterTest, TestNullValueHandling) { + std::unique_ptr bf; + Status st = BloomFilter::create(BLOCK_BLOOM_FILTER, &bf); + EXPECT_TRUE(st.ok()); + + uint64_t n = 1000; + double fpp = 0.01; + st = bf->init(n, fpp, HASH_MURMUR3_X64_64); + EXPECT_TRUE(st.ok()); + + // Initially, has_null should be false + EXPECT_FALSE(bf->has_null()); + + // Add null value + bf->add_bytes(nullptr, 0); + + // Now, has_null should be true + EXPECT_TRUE(bf->has_null()); + + // Test for null value + bool exists = bf->test_bytes(nullptr, 0); + EXPECT_TRUE(exists); +} + +// Test the optimal_bit_num function +TEST_F(BloomFilterTest, TestOptimalBitNum) { + uint64_t n = 1000; + double fpp = 0.01; + uint32_t num_bits = BloomFilter::optimal_bit_num(n, fpp); + + // Expected num_bits should be within a reasonable range + EXPECT_GT(num_bits, 0u); + EXPECT_LE(num_bits, BloomFilter::MAXIMUM_BYTES * 8); + + // Verify that num_bits is a power of 2 + EXPECT_EQ(num_bits & (num_bits - 1), 0u); +} + +// Test init function with invalid inputs to cover exception branches +TEST_F(BloomFilterTest, TestInitWithInvalidInputs) { + std::unique_ptr bf; + Status st = BloomFilter::create(BLOCK_BLOOM_FILTER, &bf); + EXPECT_TRUE(st.ok()); + + // Test with invalid hash strategy + const char* buffer = "test_buffer"; + uint32_t size = 1024; + HashStrategyPB invalid_strategy = static_cast(-1); // Invalid strategy + st = bf->init(buffer, size, invalid_strategy); + EXPECT_FALSE(st.ok()); + EXPECT_EQ(st.code(), TStatusCode::INVALID_ARGUMENT); + + // Test with size zero + st = bf->init(buffer, 0, HASH_MURMUR3_X64_64); + EXPECT_FALSE(st.ok()); + EXPECT_EQ(st.code(), TStatusCode::INVALID_ARGUMENT); + + // Test with nullptr buffer + st = bf->init(nullptr, size, HASH_MURMUR3_X64_64); + EXPECT_EQ(st.code(), TStatusCode::INVALID_ARGUMENT); + + // Test with size less than minimum allowed + uint32_t invalid_size = 1; // Less than minimum bytes + st = bf->init(buffer, invalid_size, HASH_MURMUR3_X64_64); + EXPECT_FALSE(st.ok()); + EXPECT_EQ(st.code(), TStatusCode::INVALID_ARGUMENT); + + // Test with size not a power of two + uint32_t non_power_of_two_size = 1000; // Not a power of two + st = bf->init(buffer, non_power_of_two_size, HASH_MURMUR3_X64_64); + EXPECT_EQ(st.code(), TStatusCode::INVALID_ARGUMENT); +} \ No newline at end of file From cf40dba7fba7ca258d3ee7b6275ded1a4cc345ea Mon Sep 17 00:00:00 2001 From: Tiewei Fang Date: Sat, 30 Nov 2024 09:18:53 +0800 Subject: [PATCH 088/399] [fix](outfile) add regression test for outfile (#44734) ### What problem does this PR solve? Problem Summary: add regression test for this pr: #43929 TODO: Support to export data of doris NULL type to orc file format. --- .../outfile/test_outfile_null_type.out | 37 ++++++ .../outfile/test_outfile_null_type.groovy | 111 ++++++++++++++++++ 2 files changed, 148 insertions(+) create mode 100644 regression-test/data/export_p0/outfile/test_outfile_null_type.out create mode 100644 regression-test/suites/export_p0/outfile/test_outfile_null_type.groovy diff --git a/regression-test/data/export_p0/outfile/test_outfile_null_type.out b/regression-test/data/export_p0/outfile/test_outfile_null_type.out new file mode 100644 index 00000000000000..1777bba72c1b5c --- /dev/null +++ b/regression-test/data/export_p0/outfile/test_outfile_null_type.out @@ -0,0 +1,37 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !select_export -- +1 ftw-1 19 +2 ftw-2 20 +3 ftw-3 21 +4 ftw-4 22 +5 ftw-5 23 +6 ftw-6 24 +7 ftw-7 25 +8 ftw-8 26 +9 ftw-9 27 +10 \N \N + +-- !select_load_parquet -- +1 ftw-1 19 \N +10 \N \N \N +2 ftw-2 20 \N +3 ftw-3 21 \N +4 ftw-4 22 \N +5 ftw-5 23 \N +6 ftw-6 24 \N +7 ftw-7 25 \N +8 ftw-8 26 \N +9 ftw-9 27 \N + +-- !select_load_csv -- +1 ftw-1 19 \N +10 \N \N \N +2 ftw-2 20 \N +3 ftw-3 21 \N +4 ftw-4 22 \N +5 ftw-5 23 \N +6 ftw-6 24 \N +7 ftw-7 25 \N +8 ftw-8 26 \N +9 ftw-9 27 \N + diff --git a/regression-test/suites/export_p0/outfile/test_outfile_null_type.groovy b/regression-test/suites/export_p0/outfile/test_outfile_null_type.groovy new file mode 100644 index 00000000000000..492b6770195341 --- /dev/null +++ b/regression-test/suites/export_p0/outfile/test_outfile_null_type.groovy @@ -0,0 +1,111 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("test_outfile_null_type", "p0") { + String ak = getS3AK() + String sk = getS3SK() + String s3_endpoint = getS3Endpoint() + String region = getS3Region() + String bucket = context.config.otherConfigs.get("s3BucketName"); + + def export_table_name = "test_outfile_null_type" + def outFilePath = "${bucket}/outfile/null_type/exp_" + + def outfile_to_S3 = { format -> + // select ... into outfile ... + def res = sql """ + SELECT *, NULL AS null_col FROM ${export_table_name} t + INTO OUTFILE "s3://${outFilePath}" + FORMAT AS ${format} + PROPERTIES ( + "s3.endpoint" = "${s3_endpoint}", + "s3.region" = "${region}", + "s3.secret_key"="${sk}", + "s3.access_key" = "${ak}" + ); + """ + + return res[0][3] + } + + sql """ DROP TABLE IF EXISTS ${export_table_name} """ + sql """ + CREATE TABLE `${export_table_name}` ( + `id` int(11) NULL, + `Name` string NULL, + `age` int(11) NULL + ) ENGINE=OLAP + DISTRIBUTED BY HASH(`id`) + PROPERTIES ( + "replication_num" = "1" + ); + """ + + StringBuilder sb = new StringBuilder() + int i = 1 + for (; i < 10; i ++) { + sb.append(""" + (${i}, 'ftw-${i}', ${i + 18}), + """) + } + sb.append(""" + (${i}, NULL, NULL) + """) + sql """ INSERT INTO ${export_table_name} VALUES + ${sb.toString()} + """ + def insert_res = sql "show last insert;" + logger.info("insert result: " + insert_res.toString()) + qt_select_export """ SELECT * FROM ${export_table_name} t ORDER BY id; """ + + // parquet file format + def format = "parquet" + def outfile_url = outfile_to_S3("${format}") + order_qt_select_load_parquet """ SELECT * FROM S3 ( + "uri" = "http://${bucket}.${s3_endpoint}${outfile_url.substring(5 + bucket.length(), outfile_url.length() - 1)}0.${format}", + "ACCESS_KEY"= "${ak}", + "SECRET_KEY" = "${sk}", + "format" = "${format}", + "region" = "${region}" + ); + """ + + // TODO(ftw): orc supports to export data of NULL type. + // orc file foramt + // format = "orc" + // outfile_url = outfile_to_S3("${format}") + // qt_select_load_orc """ SELECT * FROM S3 ( + // "uri" = "http://${bucket}.${s3_endpoint}${outfile_url.substring(5 + bucket.length(), outfile_url.length() - 1)}0.${format}", + // "ACCESS_KEY"= "${ak}", + // "SECRET_KEY" = "${sk}", + // "format" = "${format}", + // "region" = "${region}" + // ); + // """ + + // csv file foramt + format = "csv" + outfile_url = outfile_to_S3("${format}") + order_qt_select_load_csv """ SELECT * FROM S3 ( + "uri" = "http://${bucket}.${s3_endpoint}${outfile_url.substring(5 + bucket.length(), outfile_url.length() - 1)}0.${format}", + "ACCESS_KEY"= "${ak}", + "SECRET_KEY" = "${sk}", + "format" = "${format}", + "region" = "${region}" + ); + """ +} \ No newline at end of file From 42a7734007dc3d20713cfb89b3aaf83e6d25608e Mon Sep 17 00:00:00 2001 From: minghong Date: Sat, 30 Nov 2024 14:17:12 +0800 Subject: [PATCH 089/399] [opt](nereids)flattern and/or (#44574) ### What problem does this PR solve? And/or expressions are represented by binary trees. The depth of the tree causes stack overflow in recursive program calls. To solve this problem, this pr will flatten the binary tree when processing and/or, reducing the number of recursions. At the same time, we also rewrite some recursive programs into non-recursive forms to avoid stack overflow. Issue --- .../apache/doris/job/manager/JobManager.java | 8 +- .../java/org/apache/doris/load/ExportMgr.java | 15 ++- .../apache/doris/load/loadv2/LoadManager.java | 6 +- .../doris/nereids/cost/ExprCostModel.java | 12 ++ .../glue/translator/ExpressionTranslator.java | 81 +++++++++++-- .../nereids/parser/LogicalPlanBuilder.java | 10 +- .../rules/analysis/ExpressionAnalyzer.java | 64 +++++++++- .../rules/analysis/SubqueryToApply.java | 24 ++-- .../rules/ExtractCommonFactorRule.java | 20 +++- .../rules/FoldConstantRuleOnFE.java | 16 +-- .../PartitionPruneExpressionExtractor.java | 80 +++++++------ .../expression/rules/SimplifyNotExprRule.java | 5 +- .../TrySimplifyPredicateWithMarkJoinSlot.java | 42 ++++--- .../rewrite/InferPredicateByReplace.java | 6 - .../stats/ColumnStatsAdjustVisitor.java | 11 +- .../nereids/stats/ExpressionEstimation.java | 27 ++++- .../doris/nereids/stats/FilterEstimation.java | 49 +++++--- .../doris/nereids/trees/expressions/And.java | 36 ++++-- .../trees/expressions/CompoundPredicate.java | 82 +++++++++++-- .../nereids/trees/expressions/Expression.java | 18 +-- .../doris/nereids/trees/expressions/Or.java | 35 ++++-- .../visitor/ExpressionVisitor.java | 2 +- .../trees/plans/commands/CancelCommand.java | 6 +- .../plans/commands/CancelExportCommand.java | 3 +- .../plans/commands/CancelLoadCommand.java | 5 +- .../plans/commands/DeleteFromCommand.java | 4 +- .../doris/nereids/util/ExpressionUtils.java | 42 +++++-- .../doris/nereids/util/TypeCoercionUtils.java | 23 ---- .../rules/expression/SimplifyRangeTest.java | 2 +- .../ArrayContainsToArrayOverlapTest.java | 3 +- .../nereids/rules/rewrite/OrToInTest.java | 19 +-- .../doris/nereids/sqltest/InferTest.java | 6 +- .../nereids/stats/FilterEstimationTest.java | 2 +- .../UpdateMvByPartitionCommandTest.java | 4 +- .../nereids_hint_tpcds_p0/shape/query10.out | 2 +- .../nereids_hint_tpcds_p0/shape/query13.out | 6 +- .../nereids_hint_tpcds_p0/shape/query15.out | 2 +- .../nereids_hint_tpcds_p0/shape/query26.out | 2 +- .../nereids_hint_tpcds_p0/shape/query28.out | 12 +- .../nereids_hint_tpcds_p0/shape/query34.out | 2 +- .../nereids_hint_tpcds_p0/shape/query41.out | 2 +- .../nereids_hint_tpcds_p0/shape/query45.out | 2 +- .../nereids_hint_tpcds_p0/shape/query46.out | 2 +- .../nereids_hint_tpcds_p0/shape/query47.out | 2 +- .../nereids_hint_tpcds_p0/shape/query48.out | 6 +- .../nereids_hint_tpcds_p0/shape/query49.out | 6 +- .../nereids_hint_tpcds_p0/shape/query53.out | 2 +- .../nereids_hint_tpcds_p0/shape/query57.out | 2 +- .../nereids_hint_tpcds_p0/shape/query61.out | 2 +- .../nereids_hint_tpcds_p0/shape/query63.out | 2 +- .../nereids_hint_tpcds_p0/shape/query68.out | 2 +- .../nereids_hint_tpcds_p0/shape/query7.out | 2 +- .../nereids_hint_tpcds_p0/shape/query78.out | 2 +- .../nereids_hint_tpcds_p0/shape/query79.out | 2 +- .../nereids_hint_tpcds_p0/shape/query85.out | 6 +- .../nereids_hint_tpcds_p0/shape/query88.out | 16 +-- .../nereids_hint_tpcds_p0/shape/query89.out | 2 +- .../nereids_hint_tpcds_p0/shape/query91.out | 2 +- .../data/nereids_hint_tpch_p0/shape/q19.out | 4 +- .../data/nereids_hint_tpch_p0/shape/q7.out | 2 +- .../huge_compound/huge_compound.out | 4 + .../eliminate_not_null/eliminate_not_null.out | 2 +- .../eliminate_outer_join.out | 2 +- .../push_down_expression_in_hash_join.out | 2 +- .../push_down_filter_other_condition.out | 6 +- .../filter_push_down/push_filter_through.out | 14 +-- .../extend_infer_equal_predicate.out | 4 +- .../pull_up_predicate_set_op.out | 2 +- .../predicate_infer/infer_predicate.out | 6 +- .../extract_from_disjunction_in_join.out | 16 +-- .../push_down_filter_through_window.out | 2 +- .../bs_downgrade_shape/query13.out | 6 +- .../bs_downgrade_shape/query45.out | 2 +- .../bs_downgrade_shape/query61.out | 2 +- .../bs_downgrade_shape/query68.out | 2 +- .../bs_downgrade_shape/query91.out | 2 +- .../eliminate_empty/query10_empty.out | 2 +- .../shape/query10.out | 2 +- .../shape/query13.out | 6 +- .../shape/query15.out | 2 +- .../shape/query26.out | 2 +- .../shape/query28.out | 12 +- .../shape/query34.out | 2 +- .../shape/query35.out | 2 +- .../shape/query41.out | 2 +- .../shape/query45.out | 2 +- .../shape/query46.out | 2 +- .../shape/query47.out | 2 +- .../shape/query48.out | 6 +- .../shape/query49.out | 6 +- .../shape/query53.out | 2 +- .../shape/query57.out | 2 +- .../shape/query61.out | 2 +- .../shape/query63.out | 2 +- .../shape/query68.out | 2 +- .../shape/query7.out | 2 +- .../shape/query78.out | 2 +- .../shape/query79.out | 2 +- .../shape/query85.out | 6 +- .../shape/query88.out | 16 +-- .../shape/query89.out | 2 +- .../shape/query91.out | 2 +- .../noStatsRfPrune/query10.out | 2 +- .../noStatsRfPrune/query13.out | 6 +- .../noStatsRfPrune/query15.out | 2 +- .../noStatsRfPrune/query26.out | 2 +- .../noStatsRfPrune/query28.out | 12 +- .../noStatsRfPrune/query34.out | 2 +- .../noStatsRfPrune/query35.out | 2 +- .../noStatsRfPrune/query41.out | 2 +- .../noStatsRfPrune/query45.out | 2 +- .../noStatsRfPrune/query46.out | 2 +- .../noStatsRfPrune/query47.out | 2 +- .../noStatsRfPrune/query48.out | 6 +- .../noStatsRfPrune/query49.out | 6 +- .../noStatsRfPrune/query53.out | 2 +- .../noStatsRfPrune/query57.out | 2 +- .../noStatsRfPrune/query61.out | 2 +- .../noStatsRfPrune/query63.out | 2 +- .../noStatsRfPrune/query68.out | 2 +- .../noStatsRfPrune/query7.out | 2 +- .../noStatsRfPrune/query78.out | 2 +- .../noStatsRfPrune/query79.out | 2 +- .../noStatsRfPrune/query85.out | 6 +- .../noStatsRfPrune/query88.out | 16 +-- .../noStatsRfPrune/query89.out | 2 +- .../noStatsRfPrune/query91.out | 2 +- .../no_stats_shape/query10.out | 2 +- .../no_stats_shape/query13.out | 6 +- .../no_stats_shape/query15.out | 2 +- .../no_stats_shape/query26.out | 2 +- .../no_stats_shape/query28.out | 12 +- .../no_stats_shape/query34.out | 2 +- .../no_stats_shape/query35.out | 2 +- .../no_stats_shape/query41.out | 2 +- .../no_stats_shape/query45.out | 2 +- .../no_stats_shape/query46.out | 2 +- .../no_stats_shape/query47.out | 2 +- .../no_stats_shape/query48.out | 6 +- .../no_stats_shape/query49.out | 6 +- .../no_stats_shape/query53.out | 2 +- .../no_stats_shape/query57.out | 2 +- .../no_stats_shape/query61.out | 2 +- .../no_stats_shape/query63.out | 2 +- .../no_stats_shape/query68.out | 2 +- .../no_stats_shape/query7.out | 2 +- .../no_stats_shape/query78.out | 2 +- .../no_stats_shape/query79.out | 2 +- .../no_stats_shape/query85.out | 6 +- .../no_stats_shape/query88.out | 16 +-- .../no_stats_shape/query89.out | 2 +- .../no_stats_shape/query91.out | 2 +- .../rf_prune/query10.out | 2 +- .../rf_prune/query13.out | 6 +- .../rf_prune/query15.out | 2 +- .../rf_prune/query26.out | 2 +- .../rf_prune/query28.out | 12 +- .../rf_prune/query34.out | 2 +- .../rf_prune/query35.out | 2 +- .../rf_prune/query41.out | 2 +- .../rf_prune/query45.out | 2 +- .../rf_prune/query46.out | 2 +- .../rf_prune/query47.out | 2 +- .../rf_prune/query48.out | 6 +- .../rf_prune/query49.out | 6 +- .../rf_prune/query53.out | 2 +- .../rf_prune/query57.out | 2 +- .../rf_prune/query61.out | 2 +- .../rf_prune/query63.out | 2 +- .../rf_prune/query68.out | 2 +- .../rf_prune/query7.out | 2 +- .../rf_prune/query78.out | 2 +- .../rf_prune/query79.out | 2 +- .../rf_prune/query85.out | 6 +- .../rf_prune/query88.out | 16 +-- .../rf_prune/query89.out | 2 +- .../rf_prune/query91.out | 2 +- .../shape/query10.out | 2 +- .../shape/query13.out | 6 +- .../shape/query15.out | 2 +- .../shape/query26.out | 2 +- .../shape/query28.out | 12 +- .../shape/query34.out | 2 +- .../shape/query35.out | 2 +- .../shape/query41.out | 2 +- .../shape/query45.out | 2 +- .../shape/query46.out | 2 +- .../shape/query47.out | 2 +- .../shape/query48.out | 6 +- .../shape/query49.out | 6 +- .../shape/query53.out | 2 +- .../shape/query57.out | 2 +- .../shape/query61.out | 2 +- .../shape/query63.out | 2 +- .../shape/query68.out | 2 +- .../shape/query7.out | 2 +- .../shape/query78.out | 2 +- .../shape/query79.out | 2 +- .../shape/query85.out | 6 +- .../shape/query88.out | 16 +-- .../shape/query89.out | 2 +- .../shape/query91.out | 2 +- .../shape/query10.out | 2 +- .../shape/query13.out | 6 +- .../shape/query15.out | 2 +- .../shape/query26.out | 2 +- .../shape/query28.out | 12 +- .../shape/query34.out | 2 +- .../shape/query35.out | 2 +- .../shape/query41.out | 2 +- .../shape/query45.out | 2 +- .../shape/query46.out | 2 +- .../shape/query47.out | 2 +- .../shape/query48.out | 6 +- .../shape/query49.out | 6 +- .../shape/query53.out | 16 +-- .../shape/query57.out | 2 +- .../shape/query61.out | 2 +- .../shape/query63.out | 16 +-- .../shape/query68.out | 2 +- .../shape/query7.out | 2 +- .../shape/query78.out | 2 +- .../shape/query79.out | 2 +- .../shape/query85.out | 6 +- .../shape/query88.out | 16 +-- .../shape/query89.out | 2 +- .../shape/query91.out | 2 +- .../nostats_rf_prune/q19.out | 4 +- .../nostats_rf_prune/q7.out | 2 +- .../rf_prune/q19.out | 4 +- .../rf_prune/q7.out | 2 +- .../shape/q19.out | 4 +- .../nereids_tpch_shape_sf1000_p0/shape/q7.out | 2 +- .../shape_no_stats/q19.out | 4 +- .../shape_no_stats/q7.out | 2 +- .../hint_tpcds/shape/query78.out | 2 +- .../new_shapes_p0/hint_tpch/shape/q19.out | 4 +- .../data/new_shapes_p0/hint_tpch/shape/q7.out | 2 +- .../tpcds_sf100/noStatsRfPrune/query10.out | 2 +- .../tpcds_sf100/noStatsRfPrune/query13.out | 6 +- .../tpcds_sf100/noStatsRfPrune/query15.out | 2 +- .../tpcds_sf100/noStatsRfPrune/query26.out | 2 +- .../tpcds_sf100/noStatsRfPrune/query28.out | 12 +- .../tpcds_sf100/noStatsRfPrune/query34.out | 2 +- .../tpcds_sf100/noStatsRfPrune/query35.out | 2 +- .../tpcds_sf100/noStatsRfPrune/query41.out | 2 +- .../tpcds_sf100/noStatsRfPrune/query45.out | 2 +- .../tpcds_sf100/noStatsRfPrune/query46.out | 2 +- .../tpcds_sf100/noStatsRfPrune/query47.out | 2 +- .../tpcds_sf100/noStatsRfPrune/query48.out | 6 +- .../tpcds_sf100/noStatsRfPrune/query49.out | 6 +- .../tpcds_sf100/noStatsRfPrune/query53.out | 2 +- .../tpcds_sf100/noStatsRfPrune/query57.out | 2 +- .../tpcds_sf100/noStatsRfPrune/query61.out | 2 +- .../tpcds_sf100/noStatsRfPrune/query63.out | 2 +- .../tpcds_sf100/noStatsRfPrune/query68.out | 2 +- .../tpcds_sf100/noStatsRfPrune/query7.out | 2 +- .../tpcds_sf100/noStatsRfPrune/query78.out | 2 +- .../tpcds_sf100/noStatsRfPrune/query79.out | 2 +- .../tpcds_sf100/noStatsRfPrune/query85.out | 6 +- .../tpcds_sf100/noStatsRfPrune/query88.out | 16 +-- .../tpcds_sf100/noStatsRfPrune/query89.out | 2 +- .../tpcds_sf100/noStatsRfPrune/query91.out | 2 +- .../tpcds_sf100/no_stats_shape/query10.out | 2 +- .../tpcds_sf100/no_stats_shape/query13.out | 6 +- .../tpcds_sf100/no_stats_shape/query15.out | 2 +- .../tpcds_sf100/no_stats_shape/query26.out | 2 +- .../tpcds_sf100/no_stats_shape/query28.out | 12 +- .../tpcds_sf100/no_stats_shape/query34.out | 2 +- .../tpcds_sf100/no_stats_shape/query35.out | 2 +- .../tpcds_sf100/no_stats_shape/query41.out | 2 +- .../tpcds_sf100/no_stats_shape/query45.out | 2 +- .../tpcds_sf100/no_stats_shape/query46.out | 2 +- .../tpcds_sf100/no_stats_shape/query47.out | 2 +- .../tpcds_sf100/no_stats_shape/query48.out | 6 +- .../tpcds_sf100/no_stats_shape/query49.out | 6 +- .../tpcds_sf100/no_stats_shape/query53.out | 2 +- .../tpcds_sf100/no_stats_shape/query57.out | 2 +- .../tpcds_sf100/no_stats_shape/query61.out | 2 +- .../tpcds_sf100/no_stats_shape/query63.out | 2 +- .../tpcds_sf100/no_stats_shape/query68.out | 2 +- .../tpcds_sf100/no_stats_shape/query7.out | 2 +- .../tpcds_sf100/no_stats_shape/query78.out | 2 +- .../tpcds_sf100/no_stats_shape/query79.out | 2 +- .../tpcds_sf100/no_stats_shape/query85.out | 6 +- .../tpcds_sf100/no_stats_shape/query88.out | 16 +-- .../tpcds_sf100/no_stats_shape/query89.out | 2 +- .../tpcds_sf100/no_stats_shape/query91.out | 2 +- .../tpcds_sf100/rf_prune/query10.out | 2 +- .../tpcds_sf100/rf_prune/query13.out | 6 +- .../tpcds_sf100/rf_prune/query15.out | 2 +- .../tpcds_sf100/rf_prune/query26.out | 2 +- .../tpcds_sf100/rf_prune/query28.out | 12 +- .../tpcds_sf100/rf_prune/query34.out | 2 +- .../tpcds_sf100/rf_prune/query35.out | 2 +- .../tpcds_sf100/rf_prune/query41.out | 2 +- .../tpcds_sf100/rf_prune/query45.out | 2 +- .../tpcds_sf100/rf_prune/query46.out | 2 +- .../tpcds_sf100/rf_prune/query47.out | 2 +- .../tpcds_sf100/rf_prune/query48.out | 16 +-- .../tpcds_sf100/rf_prune/query49.out | 6 +- .../tpcds_sf100/rf_prune/query53.out | 2 +- .../tpcds_sf100/rf_prune/query57.out | 2 +- .../tpcds_sf100/rf_prune/query61.out | 2 +- .../tpcds_sf100/rf_prune/query63.out | 2 +- .../tpcds_sf100/rf_prune/query68.out | 2 +- .../tpcds_sf100/rf_prune/query7.out | 2 +- .../tpcds_sf100/rf_prune/query78.out | 2 +- .../tpcds_sf100/rf_prune/query79.out | 2 +- .../tpcds_sf100/rf_prune/query85.out | 6 +- .../tpcds_sf100/rf_prune/query88.out | 16 +-- .../tpcds_sf100/rf_prune/query89.out | 2 +- .../tpcds_sf100/rf_prune/query91.out | 2 +- .../tpcds_sf100/shape/query10.out | 2 +- .../tpcds_sf100/shape/query13.out | 6 +- .../tpcds_sf100/shape/query15.out | 2 +- .../tpcds_sf100/shape/query26.out | 2 +- .../tpcds_sf100/shape/query28.out | 12 +- .../tpcds_sf100/shape/query34.out | 2 +- .../tpcds_sf100/shape/query35.out | 2 +- .../tpcds_sf100/shape/query41.out | 2 +- .../tpcds_sf100/shape/query45.out | 2 +- .../tpcds_sf100/shape/query46.out | 2 +- .../tpcds_sf100/shape/query47.out | 2 +- .../tpcds_sf100/shape/query48.out | 16 +-- .../tpcds_sf100/shape/query49.out | 6 +- .../tpcds_sf100/shape/query53.out | 2 +- .../tpcds_sf100/shape/query57.out | 2 +- .../tpcds_sf100/shape/query61.out | 2 +- .../tpcds_sf100/shape/query63.out | 2 +- .../tpcds_sf100/shape/query68.out | 2 +- .../tpcds_sf100/shape/query7.out | 2 +- .../tpcds_sf100/shape/query78.out | 2 +- .../tpcds_sf100/shape/query79.out | 2 +- .../tpcds_sf100/shape/query85.out | 6 +- .../tpcds_sf100/shape/query88.out | 16 +-- .../tpcds_sf100/shape/query89.out | 2 +- .../tpcds_sf100/shape/query91.out | 2 +- .../bs_downgrade_shape/query13.out | 6 +- .../bs_downgrade_shape/query45.out | 2 +- .../bs_downgrade_shape/query61.out | 2 +- .../bs_downgrade_shape/query68.out | 2 +- .../bs_downgrade_shape/query91.out | 2 +- .../eliminate_empty/query10_empty.out | 2 +- .../tpcds_sf1000/shape/query10.out | 2 +- .../tpcds_sf1000/shape/query13.out | 6 +- .../tpcds_sf1000/shape/query15.out | 2 +- .../tpcds_sf1000/shape/query26.out | 2 +- .../tpcds_sf1000/shape/query28.out | 12 +- .../tpcds_sf1000/shape/query34.out | 2 +- .../tpcds_sf1000/shape/query35.out | 2 +- .../tpcds_sf1000/shape/query41.out | 2 +- .../tpcds_sf1000/shape/query45.out | 2 +- .../tpcds_sf1000/shape/query46.out | 2 +- .../tpcds_sf1000/shape/query47.out | 2 +- .../tpcds_sf1000/shape/query48.out | 6 +- .../tpcds_sf1000/shape/query49.out | 6 +- .../tpcds_sf1000/shape/query53.out | 2 +- .../tpcds_sf1000/shape/query57.out | 2 +- .../tpcds_sf1000/shape/query61.out | 2 +- .../tpcds_sf1000/shape/query63.out | 2 +- .../tpcds_sf1000/shape/query68.out | 2 +- .../tpcds_sf1000/shape/query7.out | 2 +- .../tpcds_sf1000/shape/query78.out | 2 +- .../tpcds_sf1000/shape/query79.out | 2 +- .../tpcds_sf1000/shape/query85.out | 6 +- .../tpcds_sf1000/shape/query88.out | 16 +-- .../tpcds_sf1000/shape/query89.out | 2 +- .../tpcds_sf1000/shape/query91.out | 2 +- .../tpch_sf1000/nostats_rf_prune/q19.out | 4 +- .../tpch_sf1000/nostats_rf_prune/q7.out | 2 +- .../tpch_sf1000/rf_prune/q19.out | 4 +- .../new_shapes_p0/tpch_sf1000/rf_prune/q7.out | 2 +- .../new_shapes_p0/tpch_sf1000/shape/q19.out | 4 +- .../new_shapes_p0/tpch_sf1000/shape/q7.out | 2 +- .../tpch_sf1000/shape_no_stats/q19.out | 4 +- .../tpch_sf1000/shape_no_stats/q7.out | 2 +- .../huge_compound/huge_compound.groovy | 69 +++++++++++ .../pull_up_predicate_set_op.groovy | 2 +- .../test_compoundpredicate_explain.groovy | 111 +++++++++++------- 380 files changed, 1300 insertions(+), 930 deletions(-) create mode 100644 regression-test/data/nereids_p0/huge_compound/huge_compound.out create mode 100644 regression-test/suites/nereids_p0/huge_compound/huge_compound.groovy diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/manager/JobManager.java b/fe/fe-core/src/main/java/org/apache/doris/job/manager/JobManager.java index db2ca6eb81d156..2a957775e113b8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/manager/JobManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/manager/JobManager.java @@ -42,7 +42,7 @@ import org.apache.doris.load.loadv2.JobState; import org.apache.doris.mysql.privilege.PrivPredicate; import org.apache.doris.nereids.trees.expressions.And; -import org.apache.doris.nereids.trees.expressions.BinaryOperator; +import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.qe.ConnectContext; import com.google.common.collect.Lists; @@ -528,7 +528,8 @@ private static void addNeedCancelLoadJob(String label, String state, /** * used for nereids planner */ - public void cancelLoadJob(String dbName, String label, String state, BinaryOperator operator) + public void cancelLoadJob(String dbName, String label, String state, + Expression operator) throws JobException, AnalysisException, DdlException { Database db = Env.getCurrentInternalCatalog().getDbOrDdlException(dbName); // List of load jobs waiting to be cancelled @@ -582,7 +583,8 @@ public void cancelLoadJob(String dbName, String label, String state, BinaryOpera } private static void addNeedCancelLoadJob(String label, String state, - BinaryOperator operator, List loadJobs, + Expression operator, + List loadJobs, List matchLoadJobs) throws AnalysisException { PatternMatcher matcher = PatternMatcherWrapper.createMysqlPattern(label, diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/ExportMgr.java b/fe/fe-core/src/main/java/org/apache/doris/load/ExportMgr.java index 398cc8f4ac2518..eddd5fb27eeef1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/ExportMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/ExportMgr.java @@ -38,7 +38,7 @@ import org.apache.doris.common.util.TimeUtils; import org.apache.doris.datasource.InternalCatalog; import org.apache.doris.mysql.privilege.PrivPredicate; -import org.apache.doris.nereids.trees.expressions.BinaryOperator; +import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.Or; import org.apache.doris.qe.ConnectContext; import org.apache.doris.scheduler.exception.JobException; @@ -162,7 +162,9 @@ public void cancelExportJob(CancelExportStmt stmt) throws DdlException, Analysis } } - private List getWaitingCancelJobs(String label, String state, BinaryOperator operator) + private List getWaitingCancelJobs( + String label, String state, + Expression operator) throws AnalysisException { Predicate jobFilter = buildCancelJobFilter(label, state, operator); readLock(); @@ -174,7 +176,9 @@ private List getWaitingCancelJobs(String label, String state, BinaryO } @VisibleForTesting - public static Predicate buildCancelJobFilter(String label, String state, BinaryOperator operator) + public static Predicate buildCancelJobFilter( + String label, String state, + Expression operator) throws AnalysisException { PatternMatcher matcher = PatternMatcherWrapper.createMysqlPattern(label, CaseSensibility.LABEL.getCaseSensibility()); @@ -201,7 +205,10 @@ public static Predicate buildCancelJobFilter(String label, String sta /** * used for Nereids planner */ - public void cancelExportJob(String label, String state, BinaryOperator operator, String dbName) + public void cancelExportJob( + String label, + String state, + Expression operator, String dbName) throws DdlException, AnalysisException { // List of export jobs waiting to be cancelled List matchExportJobs = getWaitingCancelJobs(label, state, operator); diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadManager.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadManager.java index 5c39a27e24f828..eb9b3f0e831a22 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadManager.java @@ -47,7 +47,7 @@ import org.apache.doris.load.Load; import org.apache.doris.mysql.privilege.PrivPredicate; import org.apache.doris.nereids.trees.expressions.And; -import org.apache.doris.nereids.trees.expressions.BinaryOperator; +import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.persist.CleanLabelOperationLog; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.OriginStatement; @@ -248,7 +248,7 @@ public void recordFinishedLoadJob(String label, long transactionId, String dbNam * Match need cancel loadJob by stmt. **/ @VisibleForTesting - public static void addNeedCancelLoadJob(String label, String state, BinaryOperator operator, + public static void addNeedCancelLoadJob(String label, String state, Expression operator, List loadJobs, List matchLoadJobs) throws AnalysisException { PatternMatcher matcher = PatternMatcherWrapper.createMysqlPattern(label, @@ -281,7 +281,7 @@ public static void addNeedCancelLoadJob(String label, String state, BinaryOperat /** * Cancel load job by stmt. **/ - public void cancelLoadJob(String dbName, String label, String state, BinaryOperator operator) + public void cancelLoadJob(String dbName, String label, String state, Expression operator) throws DdlException, AnalysisException { Database db = Env.getCurrentInternalCatalog().getDbOrDdlException(dbName); // List of load jobs waiting to be cancelled diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/cost/ExprCostModel.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/cost/ExprCostModel.java index 4a8f9ccd029276..43c98969b9b2a1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/cost/ExprCostModel.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/cost/ExprCostModel.java @@ -18,7 +18,9 @@ package org.apache.doris.nereids.cost; import org.apache.doris.nereids.trees.expressions.Alias; +import org.apache.doris.nereids.trees.expressions.And; import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.Or; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.SlotReference; import org.apache.doris.nereids.trees.expressions.literal.Literal; @@ -70,4 +72,14 @@ public Double visitLiteral(Literal literal, Void context) { return 0.0; } + @Override + public Double visitAnd(And and, Void context) { + return 0.0; + } + + @Override + public Double visitOr(Or or, Void context) { + return 0.0; + } + } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/ExpressionTranslator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/ExpressionTranslator.java index 627a983c123b8d..166892f48a9db6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/ExpressionTranslator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/ExpressionTranslator.java @@ -104,7 +104,9 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; +import java.util.ArrayDeque; import java.util.ArrayList; +import java.util.Deque; import java.util.List; import java.util.Optional; import java.util.stream.Collectors; @@ -322,22 +324,85 @@ public Expr visitNullLiteral(NullLiteral nullLiteral, PlanTranslatorContext cont return nullLit; } + private static class Frame { + int low; + int high; + CompoundPredicate.Operator op; + boolean processed; + + Frame(int low, int high, CompoundPredicate.Operator op) { + this.low = low; + this.high = high; + this.op = op; + this.processed = false; + } + } + + private Expr toBalancedTree(int low, int high, List children, + CompoundPredicate.Operator op) { + Deque stack = new ArrayDeque<>(); + Deque results = new ArrayDeque<>(); + + stack.push(new Frame(low, high, op)); + + while (!stack.isEmpty()) { + Frame currentFrame = stack.peek(); + + if (!currentFrame.processed) { + int l = currentFrame.low; + int h = currentFrame.high; + int diff = h - l; + + if (diff == 0) { + results.push(children.get(l)); + stack.pop(); + } else if (diff == 1) { + Expr left = children.get(l); + Expr right = children.get(h); + CompoundPredicate cp = new CompoundPredicate(op, left, right); + results.push(cp); + stack.pop(); + } else { + int mid = l + (h - l) / 2; + + currentFrame.processed = true; + + stack.push(new Frame(mid + 1, h, op)); + stack.push(new Frame(l, mid, op)); + } + } else { + stack.pop(); + if (results.size() >= 2) { + Expr right = results.pop(); + Expr left = results.pop(); + CompoundPredicate cp = new CompoundPredicate(currentFrame.op, left, right); + results.push(cp); + } + } + } + return results.pop(); + } + @Override public Expr visitAnd(And and, PlanTranslatorContext context) { - org.apache.doris.analysis.CompoundPredicate cp = new org.apache.doris.analysis.CompoundPredicate( - org.apache.doris.analysis.CompoundPredicate.Operator.AND, - and.child(0).accept(this, context), - and.child(1).accept(this, context)); + List children = and.children().stream().map( + e -> e.accept(this, context) + ).collect(Collectors.toList()); + CompoundPredicate cp = (CompoundPredicate) toBalancedTree(0, children.size() - 1, + children, CompoundPredicate.Operator.AND); + cp.setNullableFromNereids(and.nullable()); return cp; } @Override public Expr visitOr(Or or, PlanTranslatorContext context) { - org.apache.doris.analysis.CompoundPredicate cp = new org.apache.doris.analysis.CompoundPredicate( - org.apache.doris.analysis.CompoundPredicate.Operator.OR, - or.child(0).accept(this, context), - or.child(1).accept(this, context)); + List children = or.children().stream().map( + e -> e.accept(this, context) + ).collect(Collectors.toList()); + CompoundPredicate cp = (CompoundPredicate) toBalancedTree(0, children.size() - 1, + children, CompoundPredicate.Operator.OR); + cp.setNullableFromNereids(or.nullable()); return cp; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index 2db26dc15011f0..e5f4322225de53 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -2081,8 +2081,14 @@ public Expression visitLogicalBinary(LogicalBinaryContext ctx) { // into expressions. Collections.reverse(contexts); List expressions = contexts.stream().map(this::getExpression).collect(Collectors.toList()); - // Create a balanced tree. - return reduceToExpressionTree(0, expressions.size() - 1, expressions, ctx); + if (ctx.operator.getType() == DorisParser.AND) { + return new And(expressions); + } else if (ctx.operator.getType() == DorisParser.OR) { + return new Or(expressions); + } else { + // Create a balanced tree. + return reduceToExpressionTree(0, expressions.size() - 1, expressions, ctx); + } }); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/ExpressionAnalyzer.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/ExpressionAnalyzer.java index 49789aa66e1ff8..adc68ac6ecac1b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/ExpressionAnalyzer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/ExpressionAnalyzer.java @@ -39,6 +39,7 @@ import org.apache.doris.nereids.rules.expression.ExpressionRewriteContext; import org.apache.doris.nereids.rules.expression.rules.FoldConstantRuleOnFE; import org.apache.doris.nereids.trees.expressions.Alias; +import org.apache.doris.nereids.trees.expressions.And; import org.apache.doris.nereids.trees.expressions.ArrayItemReference; import org.apache.doris.nereids.trees.expressions.BinaryArithmetic; import org.apache.doris.nereids.trees.expressions.BitNot; @@ -46,7 +47,6 @@ import org.apache.doris.nereids.trees.expressions.CaseWhen; import org.apache.doris.nereids.trees.expressions.Cast; import org.apache.doris.nereids.trees.expressions.ComparisonPredicate; -import org.apache.doris.nereids.trees.expressions.CompoundPredicate; import org.apache.doris.nereids.trees.expressions.Divide; import org.apache.doris.nereids.trees.expressions.EqualTo; import org.apache.doris.nereids.trees.expressions.Expression; @@ -57,6 +57,7 @@ import org.apache.doris.nereids.trees.expressions.Match; import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.expressions.Not; +import org.apache.doris.nereids.trees.expressions.Or; import org.apache.doris.nereids.trees.expressions.Placeholder; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.SlotReference; @@ -73,6 +74,7 @@ import org.apache.doris.nereids.trees.expressions.functions.udf.UdfBuilder; import org.apache.doris.nereids.trees.expressions.literal.IntegerLikeLiteral; import org.apache.doris.nereids.trees.expressions.literal.Literal; +import org.apache.doris.nereids.trees.expressions.literal.NullLiteral; import org.apache.doris.nereids.trees.expressions.literal.StringLiteral; import org.apache.doris.nereids.trees.expressions.typecoercion.ImplicitCastInputTypes; import org.apache.doris.nereids.trees.plans.Plan; @@ -81,6 +83,7 @@ import org.apache.doris.nereids.types.BigIntType; import org.apache.doris.nereids.types.BooleanType; import org.apache.doris.nereids.types.DataType; +import org.apache.doris.nereids.util.ExpressionUtils; import org.apache.doris.nereids.util.TypeCoercionUtils; import org.apache.doris.nereids.util.Utils; import org.apache.doris.qe.ConnectContext; @@ -95,6 +98,7 @@ import com.google.common.base.Strings; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList.Builder; +import com.google.common.collect.Lists; import org.apache.commons.lang3.StringUtils; import java.util.ArrayList; @@ -498,11 +502,59 @@ public Expression visitBinaryArithmetic(BinaryArithmetic binaryArithmetic, Expre } @Override - public Expression visitCompoundPredicate(CompoundPredicate compoundPredicate, ExpressionRewriteContext context) { - Expression left = compoundPredicate.left().accept(this, context); - Expression right = compoundPredicate.right().accept(this, context); - CompoundPredicate ret = (CompoundPredicate) compoundPredicate.withChildren(left, right); - return TypeCoercionUtils.processCompoundPredicate(ret); + public Expression visitOr(Or or, ExpressionRewriteContext context) { + List children = ExpressionUtils.extractDisjunction(or); + List newChildren = Lists.newArrayListWithCapacity(children.size()); + boolean hasNewChild = false; + for (Expression child : children) { + Expression newChild = child.accept(this, context); + if (newChild == null) { + newChild = child; + } + if (newChild.getDataType().isNullType()) { + newChild = new NullLiteral(BooleanType.INSTANCE); + } else { + newChild = TypeCoercionUtils.castIfNotSameType(newChild, BooleanType.INSTANCE); + } + + if (! child.equals(newChild)) { + hasNewChild = true; + } + newChildren.add(newChild); + } + if (hasNewChild) { + return ExpressionUtils.or(newChildren); + } else { + return or; + } + } + + @Override + public Expression visitAnd(And and, ExpressionRewriteContext context) { + List children = ExpressionUtils.extractConjunction(and); + List newChildren = Lists.newArrayListWithCapacity(children.size()); + boolean hasNewChild = false; + for (Expression child : children) { + Expression newChild = child.accept(this, context); + if (newChild == null) { + newChild = child; + } + if (newChild.getDataType().isNullType()) { + newChild = new NullLiteral(BooleanType.INSTANCE); + } else { + newChild = TypeCoercionUtils.castIfNotSameType(newChild, BooleanType.INSTANCE); + } + + if (! child.equals(newChild)) { + hasNewChild = true; + } + newChildren.add(newChild); + } + if (hasNewChild) { + return ExpressionUtils.and(newChildren); + } else { + return and; + } } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/SubqueryToApply.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/SubqueryToApply.java index 17e7d098cad552..14700b030d68b7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/SubqueryToApply.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/SubqueryToApply.java @@ -27,7 +27,7 @@ import org.apache.doris.nereids.trees.TreeNode; import org.apache.doris.nereids.trees.expressions.Alias; import org.apache.doris.nereids.trees.expressions.And; -import org.apache.doris.nereids.trees.expressions.BinaryOperator; +import org.apache.doris.nereids.trees.expressions.CompoundPredicate; import org.apache.doris.nereids.trees.expressions.Exists; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.InSubquery; @@ -589,17 +589,19 @@ public Expression visitScalarSubquery(ScalarSubquery scalar, SubqueryContext con } @Override - public Expression visitBinaryOperator(BinaryOperator binaryOperator, SubqueryContext context) { + public Expression visitCompoundPredicate(CompoundPredicate compound, SubqueryContext context) { // update isMarkJoin flag - isMarkJoin = - isMarkJoin || ((binaryOperator.left().anyMatch(SubqueryExpr.class::isInstance) - || binaryOperator.right().anyMatch(SubqueryExpr.class::isInstance)) - && (binaryOperator instanceof Or)); - - Expression left = replace(binaryOperator.left(), context); - Expression right = replace(binaryOperator.right(), context); - - return binaryOperator.withChildren(left, right); + if (compound instanceof Or) { + for (Expression child : compound.children()) { + if (child.anyMatch(SubqueryExpr.class::isInstance)) { + isMarkJoin = true; + break; + } + } + } + return compound.withChildren( + compound.children().stream().map(c -> replace(c, context)).collect(Collectors.toList()) + ); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/ExtractCommonFactorRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/ExtractCommonFactorRule.java index 4032db4aadf550..d79fafa80a71b9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/ExtractCommonFactorRule.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/ExtractCommonFactorRule.java @@ -40,6 +40,7 @@ import java.util.List; import java.util.Map.Entry; import java.util.Set; +import java.util.stream.Collectors; /** * Extract common expr for `CompoundPredicate`. @@ -60,11 +61,24 @@ public List> buildRules() { private static Expression extractCommonFactor(CompoundPredicate originExpr) { // fast return - if (!(originExpr.left() instanceof CompoundPredicate || originExpr.left() instanceof BooleanLiteral) - && !(originExpr.right() instanceof CompoundPredicate || originExpr.right() instanceof BooleanLiteral)) { + boolean canExtract = false; + Set childrenSet = new LinkedHashSet<>(); + for (Expression child : originExpr.children()) { + if ((child instanceof CompoundPredicate || child instanceof BooleanLiteral)) { + canExtract = true; + } + childrenSet.add(child); + } + if (!canExtract) { + if (childrenSet.size() != originExpr.children().size()) { + if (childrenSet.size() == 1) { + return childrenSet.iterator().next(); + } else { + return originExpr.withChildren(childrenSet.stream().collect(Collectors.toList())); + } + } return originExpr; } - // flatten same type to a list // e.g. ((a and (b or c)) and c) -> [a, (b or c), c] List flatten = ExpressionUtils.extract(originExpr); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/FoldConstantRuleOnFE.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/FoldConstantRuleOnFE.java index 1e01e544a2510d..d1c385ec621062 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/FoldConstantRuleOnFE.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/FoldConstantRuleOnFE.java @@ -401,15 +401,15 @@ public Expression visitAnd(And and, ExpressionRewriteContext context) { // x and y return and.withChildren(nonTrueLiteral); } - } else if (nullCount == 1) { + } else if (nullCount < and.children().size()) { if (nonTrueLiteral.size() == 1) { - // null and true - return new NullLiteral(BooleanType.INSTANCE); + return nonTrueLiteral.get(0); + } else { + // null and x + return and.withChildren(nonTrueLiteral); } - // null and x - return and.withChildren(nonTrueLiteral); } else { - // null and null + // null and null and null and ... return new NullLiteral(BooleanType.INSTANCE); } } @@ -442,10 +442,10 @@ public Expression visitOr(Or or, ExpressionRewriteContext context) { // x or y return or.withChildren(nonFalseLiteral); } - } else if (nullCount == 1) { + } else if (nullCount < nonFalseLiteral.size()) { if (nonFalseLiteral.size() == 1) { // null or false - return new NullLiteral(BooleanType.INSTANCE); + return nonFalseLiteral.get(0); } // null or x return or.withChildren(nonFalseLiteral); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/PartitionPruneExpressionExtractor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/PartitionPruneExpressionExtractor.java index 322016fd45c4a9..90375e0c0e6a36 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/PartitionPruneExpressionExtractor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/PartitionPruneExpressionExtractor.java @@ -29,7 +29,9 @@ import org.apache.doris.nereids.trees.expressions.visitor.DefaultExpressionRewriter; import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.Lists; +import java.util.List; import java.util.Objects; import java.util.Set; @@ -100,49 +102,55 @@ public Expression visit(Expression originExpr, Context parentContext) { } @Override - public Expression visitAnd(And node, Context parentContext) { - // handle left node - Context leftContext = new Context(); - Expression newLeft = node.left().accept(this, leftContext); - // handle right node - Context rightContext = new Context(); - Expression newRight = node.right().accept(this, rightContext); - - // if anyone of them is FALSE, the whole expression should be FALSE. - if (newLeft == BooleanLiteral.FALSE || newRight == BooleanLiteral.FALSE) { - return BooleanLiteral.FALSE; + public Expression visitAnd(And and, Context parentContext) { + List children = and.children(); + List newChildren = Lists.newArrayListWithCapacity(children.size()); + boolean changed = false; + for (Expression child : children) { + Context childContext = new Context(); + Expression newChild = child.accept(this, childContext); + // if anyone of them is FALSE, the whole expression should be FALSE. + if (newChild == BooleanLiteral.FALSE) { + return BooleanLiteral.FALSE; + } + if (newChild != BooleanLiteral.TRUE && !childContext.containsUnEvaluableExpression) { + newChildren.add(newChild); + changed |= !newChild.equals(child); + } else { + changed = true; + } } - - // If left node contains non-partition slot or is TURE, just discard it. - if (newLeft == BooleanLiteral.TRUE || leftContext.containsUnEvaluableExpression) { - return rightContext.containsUnEvaluableExpression ? BooleanLiteral.TRUE : newRight; + if (newChildren.isEmpty()) { + return BooleanLiteral.TRUE; } - - // If right node contains non-partition slot or is TURE, just discard it. - if (newRight == BooleanLiteral.TRUE || rightContext.containsUnEvaluableExpression) { - return newLeft; + if (newChildren.size() == 1) { + return newChildren.get(0); + } + if (changed) { + if (newChildren.isEmpty()) { + return BooleanLiteral.TRUE; + } else { + return and.withChildren(newChildren); + } + } else { + return and; } - - // both does not contains non-partition slot. - return new And(newLeft, newRight); } @Override - public Expression visitOr(Or node, Context parentContext) { - // handle left node - Context leftContext = new Context(); - Expression newLeft = node.left().accept(this, leftContext); - // handle right node - Context rightContext = new Context(); - Expression newRight = node.right().accept(this, rightContext); - - // if anyone of them is TRUE or contains non-partition slot, just return TRUE. - if (newLeft == BooleanLiteral.TRUE || newRight == BooleanLiteral.TRUE - || leftContext.containsUnEvaluableExpression || rightContext.containsUnEvaluableExpression) { - return BooleanLiteral.TRUE; + public Expression visitOr(Or or, Context parentContext) { + List children = or.children(); + List newChildren = Lists.newArrayListWithCapacity(children.size()); + for (Expression child : children) { + Context childContext = new Context(); + Expression newChild = child.accept(this, childContext); + // if anyone of them is TRUE or contains non-partition slot, just return TRUE. + if (newChild == BooleanLiteral.TRUE || childContext.containsUnEvaluableExpression) { + return BooleanLiteral.TRUE; + } + newChildren.add(newChild); } - - return new Or(newLeft, newRight); + return or.withChildren(newChildren); } /** diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifyNotExprRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifyNotExprRule.java index 484d68f0d7317d..92ea5ddfd5c9ed 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifyNotExprRule.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifyNotExprRule.java @@ -31,6 +31,7 @@ import com.google.common.collect.ImmutableList; import java.util.List; +import java.util.stream.Collectors; /** * Rewrite rule of NOT expression. @@ -76,9 +77,7 @@ public static Expression simplify(Not not) { } } else if (child instanceof CompoundPredicate) { CompoundPredicate cp = (CompoundPredicate) child; - Not left = new Not(cp.left()); - Not right = new Not(cp.right()); - return cp.flip(left, right); + return cp.flip(cp.children().stream().map(c -> new Not(c)).collect(Collectors.toList())); } else if (child instanceof Not) { return child.child(0); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/TrySimplifyPredicateWithMarkJoinSlot.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/TrySimplifyPredicateWithMarkJoinSlot.java index d4dc6697d80aaa..323a6951688290 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/TrySimplifyPredicateWithMarkJoinSlot.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/TrySimplifyPredicateWithMarkJoinSlot.java @@ -25,6 +25,10 @@ import org.apache.doris.nereids.trees.expressions.Or; import org.apache.doris.nereids.trees.expressions.literal.BooleanLiteral; +import com.google.common.collect.Lists; + +import java.util.List; + /** * TrySimplifyPredicateWithMarkJoinSlot */ @@ -56,19 +60,15 @@ public Expression visitAnd(And and, ExpressionRewriteContext context) { * we change 'predicate(with mark slot) and predicate(no mark slot)' -> predicate(with mark slot) and true * to evaluate the predicate */ - Expression left = and.left(); - Expression newLeft = left.accept(this, context); - - if (newLeft.getInputSlots().stream().noneMatch(MarkJoinSlotReference.class::isInstance)) { - newLeft = BooleanLiteral.TRUE; + List newChildren = Lists.newArrayListWithCapacity(and.children().size()); + for (Expression child : and.children()) { + Expression newChild = child.accept(this, context); + if (newChild.getInputSlots().stream().noneMatch(MarkJoinSlotReference.class::isInstance)) { + newChild = BooleanLiteral.TRUE; + } + newChildren.add(newChild); } - - Expression right = and.right(); - Expression newRight = right.accept(this, context); - if (newRight.getInputSlots().stream().noneMatch(MarkJoinSlotReference.class::isInstance)) { - newRight = BooleanLiteral.TRUE; - } - Expression expr = new And(newLeft, newRight); + Expression expr = new And(newChildren); return expr; } @@ -94,19 +94,17 @@ public Expression visitOr(Or or, ExpressionRewriteContext context) { * we change 'predicate(with mark slot) or predicate(no mark slot)' -> predicate(with mark slot) or false * to evaluate the predicate */ - Expression left = or.left(); - Expression newLeft = left.accept(this, context); - if (newLeft.getInputSlots().stream().noneMatch(MarkJoinSlotReference.class::isInstance)) { - newLeft = BooleanLiteral.FALSE; + List newChildren = Lists.newArrayListWithCapacity(or.children().size()); + for (Expression child : or.children()) { + Expression newChild = child.accept(this, context); + if (newChild.getInputSlots().stream().noneMatch(MarkJoinSlotReference.class::isInstance)) { + newChild = BooleanLiteral.FALSE; + } + newChildren.add(newChild); } - Expression right = or.right(); - Expression newRight = right.accept(this, context); - if (newRight.getInputSlots().stream().noneMatch(MarkJoinSlotReference.class::isInstance)) { - newRight = BooleanLiteral.FALSE; - } - Expression expr = new Or(newLeft, newRight); + Expression expr = new Or(newChildren); return expr; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/InferPredicateByReplace.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/InferPredicateByReplace.java index d6f4925c7adeb7..4fc9efc1943ba6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/InferPredicateByReplace.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/InferPredicateByReplace.java @@ -29,7 +29,6 @@ import org.apache.doris.nereids.trees.expressions.InPredicate; import org.apache.doris.nereids.trees.expressions.Like; import org.apache.doris.nereids.trees.expressions.Not; -import org.apache.doris.nereids.trees.expressions.Or; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.functions.ExpressionTrait; import org.apache.doris.nereids.trees.expressions.literal.Literal; @@ -81,11 +80,6 @@ public Void visit(Expression expr, Map> context) { return null; } - @Override - public Void visitOr(Or expr, Map> context) { - return null; - } - @Override public Void visitInPredicate(InPredicate inPredicate, Map> context) { if (!validInPredicate(inPredicate)) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/ColumnStatsAdjustVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/ColumnStatsAdjustVisitor.java index eaf2ce3734408d..d91cb5b4a6883b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/ColumnStatsAdjustVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/ColumnStatsAdjustVisitor.java @@ -20,9 +20,8 @@ import org.apache.doris.analysis.LiteralExpr; import org.apache.doris.catalog.Type; import org.apache.doris.nereids.trees.expressions.Cast; -import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.literal.Literal; -import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; +import org.apache.doris.nereids.trees.expressions.visitor.DefaultExpressionVisitor; import org.apache.doris.nereids.types.DataType; import org.apache.doris.nereids.types.coercion.CharacterType; import org.apache.doris.statistics.ColumnStatistic; @@ -57,16 +56,10 @@ * for other expressions(except cast), we also need to adjust their input column stats. * */ -public class ColumnStatsAdjustVisitor extends ExpressionVisitor { +public class ColumnStatsAdjustVisitor extends DefaultExpressionVisitor { private static final Logger LOG = LogManager.getLogger(ColumnStatsAdjustVisitor.class); - @Override - public ColumnStatistic visit(Expression expr, Statistics context) { - expr.children().forEach(child -> child.accept(this, context)); - return null; - } - @Override public ColumnStatistic visitCast(Cast cast, Statistics context) { ColumnStatistic colStats = context.findColumnStatistics(cast); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/ExpressionEstimation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/ExpressionEstimation.java index 2307a6dfba3525..7d1b5439bace23 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/ExpressionEstimation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/ExpressionEstimation.java @@ -24,17 +24,18 @@ import org.apache.doris.nereids.trees.expressions.Add; import org.apache.doris.nereids.trees.expressions.AggregateExpression; import org.apache.doris.nereids.trees.expressions.Alias; +import org.apache.doris.nereids.trees.expressions.And; import org.apache.doris.nereids.trees.expressions.BinaryArithmetic; import org.apache.doris.nereids.trees.expressions.CaseWhen; import org.apache.doris.nereids.trees.expressions.Cast; import org.apache.doris.nereids.trees.expressions.ComparisonPredicate; -import org.apache.doris.nereids.trees.expressions.CompoundPredicate; import org.apache.doris.nereids.trees.expressions.Divide; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.IntegralDivide; import org.apache.doris.nereids.trees.expressions.MarkJoinSlotReference; import org.apache.doris.nereids.trees.expressions.Mod; import org.apache.doris.nereids.trees.expressions.Multiply; +import org.apache.doris.nereids.trees.expressions.Or; import org.apache.doris.nereids.trees.expressions.SlotReference; import org.apache.doris.nereids.trees.expressions.Subtract; import org.apache.doris.nereids.trees.expressions.TimestampArithmetic; @@ -451,12 +452,26 @@ public ColumnStatistic visitComparisonPredicate(ComparisonPredicate cp, Statisti } @Override - public ColumnStatistic visitCompoundPredicate(CompoundPredicate compoundPredicate, Statistics context) { - List childExprs = compoundPredicate.children(); - ColumnStatistic firstChild = childExprs.get(0).accept(this, context); + public ColumnStatistic visitOr(Or or, Statistics inputStats) { + List children = or.children(); + // TODO: this algorithm is not right, fix it latter + ColumnStatistic firstChild = children.get(0).accept(this, inputStats); double maxNull = StatsMathUtil.maxNonNaN(firstChild.numNulls, 1); - for (int i = 1; i < childExprs.size(); i++) { - ColumnStatistic columnStatistic = childExprs.get(i).accept(this, context); + for (int i = 1; i < children.size(); i++) { + ColumnStatistic columnStatistic = children.get(i).accept(this, inputStats); + maxNull = StatsMathUtil.maxNonNaN(maxNull, columnStatistic.numNulls); + } + return new ColumnStatisticBuilder(firstChild).setNumNulls(maxNull).setNdv(2).build(); + } + + @Override + public ColumnStatistic visitAnd(And and, Statistics inputStats) { + List children = and.children(); + // TODO: this algorithm is not right, fix it latter + ColumnStatistic firstChild = children.get(0).accept(this, inputStats); + double maxNull = StatsMathUtil.maxNonNaN(firstChild.numNulls, 1); + for (int i = 1; i < children.size(); i++) { + ColumnStatistic columnStatistic = children.get(i).accept(this, inputStats); maxNull = StatsMathUtil.maxNonNaN(maxNull, columnStatistic.numNulls); } return new ColumnStatisticBuilder(firstChild).setNumNulls(maxNull).setNdv(2).build(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/FilterEstimation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/FilterEstimation.java index e050ae9fe0feec..afd2bdaf25f43a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/FilterEstimation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/FilterEstimation.java @@ -23,7 +23,6 @@ import org.apache.doris.nereids.stats.FilterEstimation.EstimationContext; import org.apache.doris.nereids.trees.expressions.And; import org.apache.doris.nereids.trees.expressions.ComparisonPredicate; -import org.apache.doris.nereids.trees.expressions.CompoundPredicate; import org.apache.doris.nereids.trees.expressions.EqualPredicate; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.GreaterThan; @@ -113,22 +112,36 @@ public Statistics visit(Expression expr, EstimationContext context) { } @Override - public Statistics visitCompoundPredicate(CompoundPredicate predicate, EstimationContext context) { - Expression leftExpr = predicate.child(0); - Expression rightExpr = predicate.child(1); - Statistics leftStats = leftExpr.accept(this, context); + public Statistics visitAnd(And and, EstimationContext context) { + List children = and.children(); + Statistics inputStats = context.statistics; + Statistics outputStats = inputStats; + Preconditions.checkArgument(children.size() > 1, "and expression abnormal: " + and); + for (Expression child : children) { + outputStats = child.accept(this, new EstimationContext(inputStats)); + outputStats.normalizeColumnStatistics(inputStats.getRowCount(), true); + inputStats = outputStats; + } + return outputStats; + } + + @Override + public Statistics visitOr(Or or, EstimationContext context) { + List children = or.children(); + Set leftInputSlots = Sets.newHashSet(children.get(0).getInputSlots()); + Statistics leftStats = children.get(0).accept(this, context); leftStats.normalizeColumnStatistics(context.statistics.getRowCount(), true); - Statistics andStats = rightExpr.accept(this, new EstimationContext(leftStats)); - if (predicate instanceof And) { - andStats.normalizeColumnStatistics(context.statistics.getRowCount(), true); - return andStats; - } else if (predicate instanceof Or) { - Statistics rightStats = rightExpr.accept(this, context); + Statistics outputStats = leftStats; + Preconditions.checkArgument(children.size() > 1, "and expression abnormal: " + or); + for (int i = 1; i < children.size(); i++) { + Expression child = children.get(i); + Statistics andStats = child.accept(this, new EstimationContext(leftStats)); + Statistics rightStats = child.accept(this, context); rightStats.normalizeColumnStatistics(context.statistics.getRowCount(), true); double rowCount = leftStats.getRowCount() + rightStats.getRowCount() - andStats.getRowCount(); Statistics orStats = context.statistics.withRowCount(rowCount); - Set leftInputSlots = leftExpr.getInputSlots(); - Set rightInputSlots = rightExpr.getInputSlots(); + + Set rightInputSlots = child.getInputSlots(); for (Slot slot : context.keyColumns) { if (leftInputSlots.contains(slot) && rightInputSlots.contains(slot)) { ColumnStatistic leftColStats = leftStats.findColumnStatistics(slot); @@ -146,13 +159,11 @@ public Statistics visitCompoundPredicate(CompoundPredicate predicate, Estimation orStats.addColumnStats(slot, colBuilder.build()); } } - return orStats; + leftStats = orStats; + outputStats = orStats; + leftInputSlots.addAll(child.getInputSlots()); } - // should not come here - Preconditions.checkArgument(false, - "unsupported compound operator: %s in %s", - predicate.getClass().getName(), predicate.toSql()); - return context.statistics; + return outputStats; } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/And.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/And.java index 5e76c3afa1a6b3..cc302e76061cfa 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/And.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/And.java @@ -18,9 +18,9 @@ package org.apache.doris.nereids.trees.expressions; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; +import org.apache.doris.nereids.util.ExpressionUtils; import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableList; import java.util.List; @@ -28,7 +28,6 @@ * And predicate expression. */ public class And extends CompoundPredicate { - /** * Desc: Constructor for CompoundPredicate. * @@ -36,16 +35,18 @@ public class And extends CompoundPredicate { * @param right right child of comparison predicate */ public And(Expression left, Expression right) { - super(ImmutableList.of(left, right), "AND"); + super(ExpressionUtils.mergeList( + ExpressionUtils.extractConjunction(left), + ExpressionUtils.extractConjunction(right)), "AND"); } - private And(List children) { + public And(List children) { super(children, "AND"); } @Override public Expression withChildren(List children) { - Preconditions.checkArgument(children.size() == 2); + Preconditions.checkArgument(children.size() >= 2); return new And(children); } @@ -56,16 +57,35 @@ public R accept(ExpressionVisitor visitor, C context) { @Override public CompoundPredicate flip() { - return new Or(left(), right()); + return new Or(children); } @Override - public CompoundPredicate flip(Expression left, Expression right) { - return new Or(left, right); + public CompoundPredicate flip(List children) { + return new Or(children); } @Override public Class flipType() { return Or.class; } + + @Override + protected List extract() { + return ExpressionUtils.extractConjunction(this); + } + + @Override + public List children() { + if (flattenChildren.isEmpty()) { + for (Expression child : children) { + if (child instanceof And) { + flattenChildren.addAll(((And) child).extract()); + } else { + flattenChildren.add(child); + } + } + } + return flattenChildren; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/CompoundPredicate.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/CompoundPredicate.java index ccc55122125784..d58d1ba8193de5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/CompoundPredicate.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/CompoundPredicate.java @@ -18,25 +18,32 @@ package org.apache.doris.nereids.trees.expressions; import org.apache.doris.nereids.exceptions.UnboundException; +import org.apache.doris.nereids.trees.expressions.functions.ExpressionTrait; +import org.apache.doris.nereids.trees.expressions.typecoercion.ExpectsInputTypes; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; import org.apache.doris.nereids.types.BooleanType; import org.apache.doris.nereids.types.DataType; +import java.util.ArrayList; import java.util.List; +import java.util.stream.Collectors; /** * Compound predicate expression. - * Such as &&,||,AND,OR. + * Such as AND,OR. */ -public abstract class CompoundPredicate extends BinaryOperator { +public abstract class CompoundPredicate extends Expression implements ExpectsInputTypes { + protected final List flattenChildren = new ArrayList<>(); + private String symbol; public CompoundPredicate(List children, String symbol) { - super(children, symbol); + super(children); + this.symbol = symbol; } @Override public boolean nullable() throws UnboundException { - return left().nullable() || right().nullable(); + return children.stream().anyMatch(ExpressionTrait::nullable); } @Override @@ -50,8 +57,8 @@ public R accept(ExpressionVisitor visitor, C context) { } @Override - public DataType inputType() { - return BooleanType.INSTANCE; + public List expectedInputTypes() { + return children.stream().map(c -> BooleanType.INSTANCE).collect(Collectors.toList()); } /** @@ -62,8 +69,69 @@ public DataType inputType() { /** * Flip logical `and` and `or` operator with new children. */ - public abstract CompoundPredicate flip(Expression left, Expression right); + public abstract CompoundPredicate flip(List children); public abstract Class flipType(); + protected abstract List extract(); + + @Override + public boolean equals(Object o) { + if (compareWidthAndDepth) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + List thisChildren = this.children(); + List thatChildren = ((CompoundPredicate) o).children(); + if (thisChildren.size() != thatChildren.size()) { + return false; + } + for (int i = 0; i < thisChildren.size(); i++) { + if (!thisChildren.get(i).equals(thatChildren.get(i))) { + return false; + } + } + return true; + } else { + return super.equals(o); + } + } + + @Override + public String toSql() { + StringBuilder sb = new StringBuilder(); + children().forEach(c -> sb.append(c.toSql()).append(",")); + sb.deleteCharAt(sb.length() - 1); + return symbol + "[" + sb + "]"; + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + children().forEach(c -> sb.append(c.toString()).append(",")); + sb.deleteCharAt(sb.length() - 1); + return symbol + "[" + sb + "]"; + } + + @Override + public String shapeInfo() { + StringBuilder sb = new StringBuilder(); + children().forEach(c -> sb.append(c.shapeInfo()).append(",")); + sb.deleteCharAt(sb.length() - 1); + return symbol + "[" + sb + "]"; + } + + @Override + public int arity() { + // get flattern children + return children().size(); + } + + @Override + public Expression child(int index) { + return children().get(index); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Expression.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Expression.java index 6063ad2b1cd6d1..e20290e8b59c41 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Expression.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Expression.java @@ -17,7 +17,6 @@ package org.apache.doris.nereids.trees.expressions; -import org.apache.doris.common.Config; import org.apache.doris.nereids.analyzer.Unbound; import org.apache.doris.nereids.analyzer.UnboundVariable; import org.apache.doris.nereids.exceptions.AnalysisException; @@ -60,12 +59,12 @@ public abstract class Expression extends AbstractTreeNode implements public static final String DEFAULT_EXPRESSION_NAME = "expression"; // Mask this expression is generated by rule, should be removed. protected Optional exprName = Optional.empty(); + protected final boolean compareWidthAndDepth; private final int depth; private final int width; // Mark this expression is from predicate infer or something else infer private final boolean inferred; private final boolean hasUnbound; - private final boolean compareWidthAndDepth; private final Supplier> inputSlots = Suppliers.memoize( () -> collect(e -> e instanceof Slot && !(e instanceof ArrayItemSlot))); private final int fastChildrenHashCode; @@ -115,8 +114,6 @@ protected Expression(Expression... children) { this.compareWidthAndDepth = compareWidthAndDepth; this.fastChildrenHashCode = fastChildrenHashCode; } - - checkLimit(); this.inferred = false; this.hasUnbound = hasUnbound || this instanceof Unbound; } @@ -170,23 +167,10 @@ protected Expression(List children, boolean inferred) { this.compareWidthAndDepth = compareWidthAndDepth && supportCompareWidthAndDepth(); this.fastChildrenHashCode = fastChildrenhashCode; } - - checkLimit(); this.inferred = inferred; this.hasUnbound = hasUnbound || this instanceof Unbound; } - private void checkLimit() { - if (depth > Config.expr_depth_limit) { - throw new AnalysisException(String.format("Exceeded the maximum depth of an " - + "expression tree (%s).", Config.expr_depth_limit)); - } - if (width > Config.expr_children_limit) { - throw new AnalysisException(String.format("Exceeded the maximum children of an " - + "expression tree (%s).", Config.expr_children_limit)); - } - } - public Alias alias(String alias) { return new Alias(this, alias); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Or.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Or.java index 61249fb91c48ce..cf6c46c3ea4215 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Or.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Or.java @@ -18,9 +18,9 @@ package org.apache.doris.nereids.trees.expressions; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; +import org.apache.doris.nereids.util.ExpressionUtils; import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableList; import java.util.List; @@ -36,16 +36,18 @@ public class Or extends CompoundPredicate { * @param right right child of comparison predicate */ public Or(Expression left, Expression right) { - super(ImmutableList.of(left, right), "OR"); + super(ExpressionUtils.mergeList( + ExpressionUtils.extractDisjunction(left), + ExpressionUtils.extractDisjunction(right)), "OR"); } - private Or(List children) { + public Or(List children) { super(children, "OR"); } @Override public Expression withChildren(List children) { - Preconditions.checkArgument(children.size() == 2); + Preconditions.checkArgument(children.size() >= 2); return new Or(children); } @@ -56,16 +58,35 @@ public R accept(ExpressionVisitor visitor, C context) { @Override public CompoundPredicate flip() { - return new And(left(), right()); + return new And(children); } @Override - public CompoundPredicate flip(Expression left, Expression right) { - return new And(left, right); + public CompoundPredicate flip(List children) { + return new And(children); } @Override public Class flipType() { return And.class; } + + @Override + protected List extract() { + return ExpressionUtils.extractDisjunction(this); + } + + @Override + public List children() { + if (flattenChildren.isEmpty()) { + for (Expression child : children) { + if (child instanceof Or) { + flattenChildren.addAll(((Or) child).extract()); + } else { + flattenChildren.add(child); + } + } + } + return flattenChildren; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ExpressionVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ExpressionVisitor.java index ab367a2bf7398e..406d0835610a17 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ExpressionVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ExpressionVisitor.java @@ -334,7 +334,7 @@ public R visitStructLiteral(StructLiteral structLiteral, C context) { } public R visitCompoundPredicate(CompoundPredicate compoundPredicate, C context) { - return visitBinaryOperator(compoundPredicate, context); + return visit(compoundPredicate, context); } public R visitAnd(And and, C context) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CancelCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CancelCommand.java index 88ecf3cba1e6d8..e82191c3648e97 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CancelCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CancelCommand.java @@ -144,18 +144,18 @@ private void binaryCheck(Expression expr, Map supportedColumns) private void compoundCheck(Expression expr, Map supportedColumns) throws AnalysisException { // current only support label and state if (expr instanceof Not) { - throw new AnalysisException("Current not support NOT operator"); + throw new AnalysisException("not support NOT operator"); } for (int i = 0; i < 2; i++) { Expression child = expr.child(i); if (child instanceof CompoundPredicate) { - throw new AnalysisException("Current not support nested clause"); + throw new AnalysisException("not support where clause: " + expr.toSql()); } else if (child instanceof Like) { likeCheck(child, supportedColumns); } else if (child instanceof BinaryOperator) { binaryCheck(child, supportedColumns); } else { - throw new AnalysisException("Only support like/binary predicate"); + throw new AnalysisException("Only support like/equalTo/And/Or predicate"); } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CancelExportCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CancelExportCommand.java index f7defefb422476..c7dc786a733a85 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CancelExportCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CancelExportCommand.java @@ -20,7 +20,6 @@ import org.apache.doris.common.AnalysisException; import org.apache.doris.common.UserException; import org.apache.doris.load.ExportJobState; -import org.apache.doris.nereids.trees.expressions.BinaryOperator; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.plans.PlanType; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; @@ -55,7 +54,7 @@ public CancelExportCommand(String dbName, Expression whereClause) { @Override public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { validate(ctx); - ctx.getEnv().getExportMgr().cancelExportJob(label, state, (BinaryOperator) whereClause, dbName); + ctx.getEnv().getExportMgr().cancelExportJob(label, state, whereClause, dbName); } private void validate(ConnectContext ctx) throws UserException { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CancelLoadCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CancelLoadCommand.java index 4d3068ed08f559..80c622a1957a18 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CancelLoadCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CancelLoadCommand.java @@ -21,7 +21,6 @@ import org.apache.doris.common.UserException; import org.apache.doris.job.exception.JobException; import org.apache.doris.load.ExportJobState; -import org.apache.doris.nereids.trees.expressions.BinaryOperator; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.plans.PlanType; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; @@ -57,9 +56,9 @@ public CancelLoadCommand(String dbName, Expression whereClause) { public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { validate(ctx); try { - ctx.getEnv().getJobManager().cancelLoadJob(dbName, label, state, (BinaryOperator) whereClause); + ctx.getEnv().getJobManager().cancelLoadJob(dbName, label, state, whereClause); } catch (JobException e) { - ctx.getEnv().getLoadManager().cancelLoadJob(dbName, label, state, (BinaryOperator) whereClause); + ctx.getEnv().getLoadManager().cancelLoadJob(dbName, label, state, whereClause); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DeleteFromCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DeleteFromCommand.java index 34a54ec2651f65..ab2a7d3f7414c1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DeleteFromCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DeleteFromCommand.java @@ -334,8 +334,8 @@ private void checkInPredicate(InPredicate in) { private void checkPredicate(Expression predicate) { if (predicate instanceof And) { - checkPredicate(((And) predicate).left()); - checkPredicate(((And) predicate).right()); + And and = (And) predicate; + and.children().forEach(child -> checkPredicate(child)); } else if (predicate instanceof ComparisonPredicate) { checkComparisonPredicate((ComparisonPredicate) predicate); } else if (predicate instanceof IsNull) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java index bf4d6e084795f1..e3909b87bd3726 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java @@ -78,10 +78,12 @@ import com.google.common.collect.Maps; import com.google.common.collect.Sets; +import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Arrays; import java.util.BitSet; import java.util.Collection; +import java.util.Deque; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -131,18 +133,24 @@ public static List extract(CompoundPredicate expr) { private static List extract(Class type, Expression expr) { List result = Lists.newArrayList(); - extract(type, expr, result); + Deque stack = new ArrayDeque<>(); + stack.push(expr); + while (!stack.isEmpty()) { + Expression current = stack.pop(); + if (type.isInstance(current)) { + for (Expression child : current.children()) { + stack.push(child); + } + } else { + result.add(current); + } + } + result = Lists.reverse(result); return result; } private static void extract(Class type, Expression expr, Collection result) { - if (type.isInstance(expr)) { - CompoundPredicate predicate = (CompoundPredicate) expr; - extract(type, predicate.left(), result); - extract(type, predicate.right(), result); - } else { - result.add(expr); - } + result.addAll(extract(type, expr)); } public static Optional> extractEqualSlot(Expression expr) { @@ -1002,6 +1010,24 @@ public static Literal analyzeAndFoldToLiteral(ConnectContext ctx, Expression exp } } + /** + * mergeList + */ + public static List mergeList(List list1, List list2) { + ImmutableList.Builder builder = ImmutableList.builder(); + for (Expression expression : list1) { + if (expression != null) { + builder.add(expression); + } + } + for (Expression expression : list2) { + if (expression != null) { + builder.add(expression); + } + } + return builder.build(); + } + private static class UnboundSlotRewriter extends DefaultExpressionRewriter { public static final UnboundSlotRewriter INSTANCE = new UnboundSlotRewriter(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/TypeCoercionUtils.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/TypeCoercionUtils.java index d7f9fc83baf288..603a891d2d2a49 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/TypeCoercionUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/TypeCoercionUtils.java @@ -32,7 +32,6 @@ import org.apache.doris.nereids.trees.expressions.CaseWhen; import org.apache.doris.nereids.trees.expressions.Cast; import org.apache.doris.nereids.trees.expressions.ComparisonPredicate; -import org.apache.doris.nereids.trees.expressions.CompoundPredicate; import org.apache.doris.nereids.trees.expressions.Divide; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.InPredicate; @@ -1192,28 +1191,6 @@ public static Expression processCaseWhen(CaseWhen caseWhen) { .orElseThrow(() -> new AnalysisException("Cannot find common type for case when " + caseWhen)); } - /** - * process compound predicate type coercion. - */ - public static Expression processCompoundPredicate(CompoundPredicate compoundPredicate) { - // check - compoundPredicate.checkLegalityBeforeTypeCoercion(); - ImmutableList.Builder newChildren - = ImmutableList.builderWithExpectedSize(compoundPredicate.arity()); - boolean changed = false; - for (Expression child : compoundPredicate.children()) { - Expression newChild; - if (child.getDataType().isNullType()) { - newChild = new NullLiteral(BooleanType.INSTANCE); - } else { - newChild = castIfNotSameType(child, BooleanType.INSTANCE); - } - changed |= child != newChild; - newChildren.add(newChild); - } - return changed ? compoundPredicate.withChildren(newChildren.build()) : compoundPredicate; - } - private static boolean canCompareDate(DataType t1, DataType t2) { DataType dateType = t1; DataType anotherType = t2; diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/SimplifyRangeTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/SimplifyRangeTest.java index d6a7b9e5585d14..2848965fb06b7a 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/SimplifyRangeTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/SimplifyRangeTest.java @@ -189,7 +189,7 @@ public void testSimplify() { assertRewrite("TA + TC = 1 and TA + TC = 3", "(TA + TC) is null and null"); assertRewriteNotNull("TA + TC in (1) and TA + TC in (3)", "FALSE"); assertRewrite("TA + TC in (1) and TA + TC in (3)", "(TA + TC) is null and null"); - assertRewrite("TA + TC in (1) and TA + TC in (1)", "TA + TC = 1"); + assertRewrite("TA + TC in (1) and TA + TC in (1)", "(TA + TC) IN (1)"); assertRewriteNotNull("(TA + TC > 3 and TA + TC < 1) and TB < 5", "FALSE"); assertRewrite("(TA + TC > 3 and TA + TC < 1) and TB < 5", "(TA + TC) is null and null and TB < 5"); assertRewrite("(TA + TC > 3 and TA + TC < 1) or TB < 5", "((TA + TC) is null and null) OR TB < 5"); diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/ArrayContainsToArrayOverlapTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/ArrayContainsToArrayOverlapTest.java index 028d85ce097fe6..ed56bd2a13c3bc 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/ArrayContainsToArrayOverlapTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/ArrayContainsToArrayOverlapTest.java @@ -89,8 +89,7 @@ void testAndOverlap() { .rewrite() .getPlan(); Expression expression = plan.child(0).getExpressions().get(0).child(0); - Assertions.assertEquals("(array_contains([1], 0) OR " - + "(array_contains([1], 1) AND arrays_overlap([1], [2, 3, 4])))", + Assertions.assertEquals("OR[array_contains([1], 0),AND[array_contains([1], 1),arrays_overlap([1], [2, 3, 4])]]", expression.toSql()); } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/OrToInTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/OrToInTest.java index 5ce7871db144bc..774be68643feb2 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/OrToInTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/OrToInTest.java @@ -38,7 +38,7 @@ void test1() { String expr = "col1 = 1 or col1 = 2 or col1 = 3 and (col2 = 4)"; Expression expression = PARSER.parseExpression(expr); Expression rewritten = OrToIn.INSTANCE.rewriteTree(expression, context); - Assertions.assertEquals("(col1 IN (1, 2, 3) AND (col1 IN (1, 2) OR ((col1 = 3) AND (col2 = 4))))", + Assertions.assertEquals("AND[col1 IN (1, 2, 3),OR[col1 IN (1, 2),AND[(col1 = 3),(col2 = 4)]]]", rewritten.toSql()); } @@ -106,7 +106,8 @@ void test7() { String expr = "A = 1 or A = 2 or abs(A)=5 or A in (1, 2, 3) or B = 1 or B = 2 or B in (1, 2, 3) or B+1 in (4, 5, 7)"; Expression expression = PARSER.parseExpression(expr); Expression rewritten = OrToIn.INSTANCE.rewriteTree(expression, context); - Assertions.assertEquals("(((A IN (1, 2, 3) OR (abs(A) = 5)) OR B IN (1, 2, 3)) OR (B + 1) IN (4, 5, 7))", rewritten.toSql()); + Assertions.assertEquals("OR[A IN (1, 2, 3),(abs(A) = 5),B IN (1, 2, 3),(B + 1) IN (4, 5, 7)]", + rewritten.toSql()); } @Test @@ -114,7 +115,7 @@ void test8() { String expr = "col = 1 or (col = 2 and (col = 3 or col = '4' or col = 5.0))"; Expression expression = PARSER.parseExpression(expr); Expression rewritten = OrToIn.INSTANCE.rewriteTree(expression, context); - Assertions.assertEquals("((col = 1) OR ((col = 2) AND col IN ('4', 3, 5.0)))", + Assertions.assertEquals("OR[(col = 1),AND[(col = 2),col IN ('4', 3, 5.0)]]", rewritten.toSql()); } @@ -124,7 +125,7 @@ void testEnsureOrder() { String expr = "col1 IN (1, 2) OR col2 IN (1, 2)"; Expression expression = PARSER.parseExpression(expr); Expression rewritten = OrToIn.INSTANCE.rewriteTree(expression, context); - Assertions.assertEquals("(col1 IN (1, 2) OR col2 IN (1, 2))", + Assertions.assertEquals("OR[col1 IN (1, 2),col2 IN (1, 2)]", rewritten.toSql()); } @@ -133,7 +134,7 @@ void test9() { String expr = "col1=1 and (col2=1 or col2=2)"; Expression expression = PARSER.parseExpression(expr); Expression rewritten = OrToIn.INSTANCE.rewriteTree(expression, context); - Assertions.assertEquals("((col1 = 1) AND col2 IN (1, 2))", + Assertions.assertEquals("AND[(col1 = 1),col2 IN (1, 2)]", rewritten.toSql()); } @@ -143,7 +144,7 @@ void test10() { String expr = "col1=1 or (col2 = 2 and (col3=4 or col3=5))"; Expression expression = PARSER.parseExpression(expr); Expression rewritten = OrToIn.INSTANCE.rewriteTree(expression, context); - Assertions.assertEquals("((col1 = 1) OR ((col2 = 2) AND col3 IN (4, 5)))", + Assertions.assertEquals("OR[(col1 = 1),AND[(col2 = 2),col3 IN (4, 5)]]", rewritten.toSql()); } @@ -153,7 +154,7 @@ void test11() { String expr = "(a=1 and b=2 and c=3) or (a=2 and b=2 and c=4)"; Expression expression = PARSER.parseExpression(expr); Expression rewritten = OrToIn.INSTANCE.rewriteTree(expression, context); - Assertions.assertEquals("((b = 2) AND ((a IN (1, 2) AND c IN (3, 4)) AND (((a = 1) AND (c = 3)) OR ((a = 2) AND (c = 4)))))", + Assertions.assertEquals("AND[(b = 2),a IN (1, 2),c IN (3, 4),OR[AND[(a = 1),(c = 3)],AND[(a = 2),(c = 4)]]]", rewritten.toSql()); } @@ -183,7 +184,7 @@ void test14() { String expr = "(a=1 and f(a)=2) or a=3"; Expression expression = PARSER.parseExpression(expr); Expression rewritten = OrToIn.INSTANCE.rewriteTree(expression, context); - Assertions.assertEquals("(((a = 1) AND (f(a) = 2)) OR (a = 3))", + Assertions.assertEquals("OR[AND[(a = 1),(f(a) = 2)],(a = 3)]", rewritten.toSql()); } @@ -193,7 +194,7 @@ void test15() { String expr = "x=1 or (a=1 and b=2) or (a=2 and c=3)"; Expression expression = PARSER.parseExpression(expr); Expression rewritten = OrToIn.INSTANCE.rewriteTree(expression, context); - Assertions.assertEquals("(((x = 1) OR ((a = 1) AND (b = 2))) OR ((a = 2) AND (c = 3)))", + Assertions.assertEquals("OR[(x = 1),AND[(a = 1),(b = 2)],AND[(a = 2),(c = 3)]]", rewritten.toSql()); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/sqltest/InferTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/sqltest/InferTest.java index 494183d196c1ad..613965b1238e27 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/sqltest/InferTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/sqltest/InferTest.java @@ -51,7 +51,7 @@ void testInferNotNullFromFilterAndEliminateOuter2() { innerLogicalJoin( logicalOlapScan(), logicalFilter().when( - f -> f.getPredicate().toString().equals("((id#0 = 4) OR (id#0 > 4))")) + f -> f.getPredicate().toString().equals("OR[(id#0 = 4),(id#0 > 4)]")) ) ); @@ -69,11 +69,11 @@ void testInferNotNullFromFilterAndEliminateOuter3() { logicalFilter( leftOuterLogicalJoin( logicalFilter().when( - f -> f.getPredicate().toString().equals("((id#0 = 4) OR (id#0 > 4))")), + f -> f.getPredicate().toString().equals("OR[(id#0 = 4),(id#0 > 4)]")), logicalOlapScan() ) ).when(f -> f.getPredicate().toString() - .equals("((id#0 = 4) OR ((id#0 > 4) AND score#3 IS NULL))")) + .equals("OR[(id#0 = 4),AND[(id#0 > 4),score#3 IS NULL]]")) ); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/stats/FilterEstimationTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/stats/FilterEstimationTest.java index 9b0fdc3880dd47..2009a98fbc1c8a 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/stats/FilterEstimationTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/stats/FilterEstimationTest.java @@ -1214,7 +1214,7 @@ void testMultiAndWithNull() { FilterEstimation filterEstimation = new FilterEstimation(); Statistics result = filterEstimation.estimate(allAnd, stats); - Assertions.assertEquals(result.getRowCount(), 2109.16, 0.01); + Assertions.assertEquals(result.getRowCount(), 2109.0, 10); } /** diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommandTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommandTest.java index a2909f85a76129..26a1b7cabd3eae 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommandTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommandTest.java @@ -45,7 +45,7 @@ void testFirstPartWithoutLowerBound() throws AnalysisException { RangePartitionItem item1 = new RangePartitionItem(range1); Set predicates = UpdateMvByPartitionCommand.constructPredicates(Sets.newHashSet(item1), "s"); - Assertions.assertEquals("((s < 1) OR s IS NULL)", predicates.iterator().next().toSql()); + Assertions.assertEquals("OR[(s < 1),s IS NULL]", predicates.iterator().next().toSql()); } @@ -82,6 +82,6 @@ void testNull() throws AnalysisException { listPartitionItem = new ListPartitionItem(ImmutableList.of(v1, v2)); expr = UpdateMvByPartitionCommand.constructPredicates(Sets.newHashSet(listPartitionItem), "s").iterator() .next(); - Assertions.assertEquals("(s IS NULL OR s IN (1))", expr.toSql()); + Assertions.assertEquals("OR[s IS NULL,s IN (1)]", expr.toSql()); } } diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query10.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query10.out index bdf370f6b1793f..f12195a12a34f7 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query10.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query10.out @@ -9,7 +9,7 @@ PhysicalResultSink ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) +------------------filter(OR[ifnull($c$1, FALSE),ifnull($c$2, FALSE)]) --------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() ----------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() ------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF5 ss_customer_sk->[c_customer_sk] diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query13.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query13.out index 1c8d917a8612c1..5a2ada8884b3a1 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query13.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query13.out @@ -7,16 +7,16 @@ PhysicalResultSink --------PhysicalProject ----------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF4 s_store_sk->[ss_store_sk] ------------PhysicalProject ---------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)) otherCondition=((((((household_demographics.hd_dep_count = 1) AND cd_marital_status IN ('D', 'W')) AND cd_education_status IN ('2 yr Degree', 'Primary')) AND ((((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = 'Primary')) AND ((store_sales.ss_sales_price >= 50.00) AND (store_sales.ss_sales_price <= 100.00))) OR (((customer_demographics.cd_marital_status = 'W') AND (customer_demographics.cd_education_status = '2 yr Degree')) AND ((store_sales.ss_sales_price >= 150.00) AND (store_sales.ss_sales_price <= 200.00))))) OR ((((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = 'College')) AND ((store_sales.ss_sales_price >= 100.00) AND (store_sales.ss_sales_price <= 150.00))) AND (household_demographics.hd_dep_count = 3)))) build RFs:RF3 ss_cdemo_sk->[cd_demo_sk] +--------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)) otherCondition=(OR[AND[(household_demographics.hd_dep_count = 1),cd_marital_status IN ('D', 'W'),cd_education_status IN ('2 yr Degree', 'Primary'),OR[AND[(customer_demographics.cd_marital_status = 'D'),(customer_demographics.cd_education_status = 'Primary'),(store_sales.ss_sales_price >= 50.00),(store_sales.ss_sales_price <= 100.00)],AND[(customer_demographics.cd_marital_status = 'W'),(customer_demographics.cd_education_status = '2 yr Degree'),(store_sales.ss_sales_price >= 150.00),(store_sales.ss_sales_price <= 200.00)]]],AND[(customer_demographics.cd_marital_status = 'M'),(customer_demographics.cd_education_status = 'College'),(store_sales.ss_sales_price >= 100.00),(store_sales.ss_sales_price <= 150.00),(household_demographics.hd_dep_count = 3)]]) build RFs:RF3 ss_cdemo_sk->[cd_demo_sk] ----------------PhysicalProject -------------------filter(((((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = 'College')) OR ((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = 'Primary'))) OR ((customer_demographics.cd_marital_status = 'W') AND (customer_demographics.cd_education_status = '2 yr Degree'))) and cd_education_status IN ('2 yr Degree', 'College', 'Primary') and cd_marital_status IN ('D', 'M', 'W')) +------------------filter(OR[AND[(customer_demographics.cd_marital_status = 'M'),(customer_demographics.cd_education_status = 'College')],AND[(customer_demographics.cd_marital_status = 'D'),(customer_demographics.cd_education_status = 'Primary')],AND[(customer_demographics.cd_marital_status = 'W'),(customer_demographics.cd_education_status = '2 yr Degree')]] and cd_education_status IN ('2 yr Degree', 'College', 'Primary') and cd_marital_status IN ('D', 'M', 'W')) --------------------PhysicalOlapScan[customer_demographics] apply RFs: RF3 ----------------PhysicalProject ------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF2 hd_demo_sk->[ss_hdemo_sk] --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=((((ca_state IN ('IL', 'TN', 'TX') AND ((store_sales.ss_net_profit >= 100.00) AND (store_sales.ss_net_profit <= 200.00))) OR (ca_state IN ('ID', 'OH', 'WY') AND ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 300.00)))) OR (ca_state IN ('IA', 'MS', 'SC') AND ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 250.00))))) build RFs:RF0 ca_address_sk->[ss_addr_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=(OR[AND[ca_state IN ('IL', 'TN', 'TX'),(store_sales.ss_net_profit >= 100.00),(store_sales.ss_net_profit <= 200.00)],AND[ca_state IN ('ID', 'OH', 'WY'),(store_sales.ss_net_profit >= 150.00),(store_sales.ss_net_profit <= 300.00)],AND[ca_state IN ('IA', 'MS', 'SC'),(store_sales.ss_net_profit >= 50.00),(store_sales.ss_net_profit <= 250.00)]]) build RFs:RF0 ca_address_sk->[ss_addr_sk] ----------------------------PhysicalProject ------------------------------filter((store_sales.ss_net_profit <= 300.00) and (store_sales.ss_net_profit >= 50.00) and (store_sales.ss_sales_price <= 200.00) and (store_sales.ss_sales_price >= 50.00)) --------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF4 diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query15.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query15.out index 212fd68ca76555..46aab2d9f4aa45 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query15.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query15.out @@ -8,7 +8,7 @@ PhysicalResultSink ----------PhysicalDistribute[DistributionSpecHash] ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=(((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR ca_state IN ('CA', 'GA', 'WA')) OR (catalog_sales.cs_sales_price > 500.00))) build RFs:RF2 c_customer_sk->[cs_bill_customer_sk] +----------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),ca_state IN ('CA', 'GA', 'WA'),(catalog_sales.cs_sales_price > 500.00)]) build RFs:RF2 c_customer_sk->[cs_bill_customer_sk] ------------------PhysicalProject --------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] ----------------------PhysicalProject diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query26.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query26.out index 6687b771fe4cf2..ce6e2a80111885 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query26.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query26.out @@ -24,7 +24,7 @@ PhysicalResultSink ----------------------------filter((date_dim.d_year = 2002)) ------------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalProject -------------------------filter(((promotion.p_channel_email = 'N') OR (promotion.p_channel_event = 'N'))) +------------------------filter(OR[(promotion.p_channel_email = 'N'),(promotion.p_channel_event = 'N')]) --------------------------PhysicalOlapScan[promotion] ------------------PhysicalProject --------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query28.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query28.out index 2fdcc86b103f71..36ec7305d96abb 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query28.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query28.out @@ -17,41 +17,41 @@ PhysicalResultSink ----------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------hashAgg[LOCAL] --------------------------------PhysicalProject -----------------------------------filter(((((store_sales.ss_list_price >= 107.00) AND (store_sales.ss_list_price <= 117.00)) OR ((store_sales.ss_coupon_amt >= 1319.00) AND (store_sales.ss_coupon_amt <= 2319.00))) OR ((store_sales.ss_wholesale_cost >= 60.00) AND (store_sales.ss_wholesale_cost <= 80.00))) and (store_sales.ss_quantity <= 5) and (store_sales.ss_quantity >= 0)) +----------------------------------filter((store_sales.ss_quantity <= 5) and (store_sales.ss_quantity >= 0) and OR[AND[(store_sales.ss_list_price >= 107.00),(store_sales.ss_list_price <= 117.00)],AND[(store_sales.ss_coupon_amt >= 1319.00),(store_sales.ss_coupon_amt <= 2319.00)],AND[(store_sales.ss_wholesale_cost >= 60.00),(store_sales.ss_wholesale_cost <= 80.00)]]) ------------------------------------PhysicalOlapScan[store_sales] ------------------------PhysicalLimit[LOCAL] --------------------------hashAgg[GLOBAL] ----------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------hashAgg[LOCAL] --------------------------------PhysicalProject -----------------------------------filter(((((store_sales.ss_list_price >= 23.00) AND (store_sales.ss_list_price <= 33.00)) OR ((store_sales.ss_coupon_amt >= 825.00) AND (store_sales.ss_coupon_amt <= 1825.00))) OR ((store_sales.ss_wholesale_cost >= 43.00) AND (store_sales.ss_wholesale_cost <= 63.00))) and (store_sales.ss_quantity <= 10) and (store_sales.ss_quantity >= 6)) +----------------------------------filter((store_sales.ss_quantity <= 10) and (store_sales.ss_quantity >= 6) and OR[AND[(store_sales.ss_list_price >= 23.00),(store_sales.ss_list_price <= 33.00)],AND[(store_sales.ss_coupon_amt >= 825.00),(store_sales.ss_coupon_amt <= 1825.00)],AND[(store_sales.ss_wholesale_cost >= 43.00),(store_sales.ss_wholesale_cost <= 63.00)]]) ------------------------------------PhysicalOlapScan[store_sales] --------------------PhysicalLimit[LOCAL] ----------------------hashAgg[GLOBAL] ------------------------PhysicalDistribute[DistributionSpecGather] --------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------filter(((((store_sales.ss_list_price >= 74.00) AND (store_sales.ss_list_price <= 84.00)) OR ((store_sales.ss_coupon_amt >= 4381.00) AND (store_sales.ss_coupon_amt <= 5381.00))) OR ((store_sales.ss_wholesale_cost >= 57.00) AND (store_sales.ss_wholesale_cost <= 77.00))) and (store_sales.ss_quantity <= 15) and (store_sales.ss_quantity >= 11)) +------------------------------filter((store_sales.ss_quantity <= 15) and (store_sales.ss_quantity >= 11) and OR[AND[(store_sales.ss_list_price >= 74.00),(store_sales.ss_list_price <= 84.00)],AND[(store_sales.ss_coupon_amt >= 4381.00),(store_sales.ss_coupon_amt <= 5381.00)],AND[(store_sales.ss_wholesale_cost >= 57.00),(store_sales.ss_wholesale_cost <= 77.00)]]) --------------------------------PhysicalOlapScan[store_sales] ----------------PhysicalLimit[LOCAL] ------------------hashAgg[GLOBAL] --------------------PhysicalDistribute[DistributionSpecGather] ----------------------hashAgg[LOCAL] ------------------------PhysicalProject ---------------------------filter(((((store_sales.ss_list_price >= 89.00) AND (store_sales.ss_list_price <= 99.00)) OR ((store_sales.ss_coupon_amt >= 3117.00) AND (store_sales.ss_coupon_amt <= 4117.00))) OR ((store_sales.ss_wholesale_cost >= 68.00) AND (store_sales.ss_wholesale_cost <= 88.00))) and (store_sales.ss_quantity <= 20) and (store_sales.ss_quantity >= 16)) +--------------------------filter((store_sales.ss_quantity <= 20) and (store_sales.ss_quantity >= 16) and OR[AND[(store_sales.ss_list_price >= 89.00),(store_sales.ss_list_price <= 99.00)],AND[(store_sales.ss_coupon_amt >= 3117.00),(store_sales.ss_coupon_amt <= 4117.00)],AND[(store_sales.ss_wholesale_cost >= 68.00),(store_sales.ss_wholesale_cost <= 88.00)]]) ----------------------------PhysicalOlapScan[store_sales] ------------PhysicalLimit[LOCAL] --------------hashAgg[GLOBAL] ----------------PhysicalDistribute[DistributionSpecGather] ------------------hashAgg[LOCAL] --------------------PhysicalProject -----------------------filter(((((store_sales.ss_list_price >= 58.00) AND (store_sales.ss_list_price <= 68.00)) OR ((store_sales.ss_coupon_amt >= 9402.00) AND (store_sales.ss_coupon_amt <= 10402.00))) OR ((store_sales.ss_wholesale_cost >= 38.00) AND (store_sales.ss_wholesale_cost <= 58.00))) and (store_sales.ss_quantity <= 25) and (store_sales.ss_quantity >= 21)) +----------------------filter((store_sales.ss_quantity <= 25) and (store_sales.ss_quantity >= 21) and OR[AND[(store_sales.ss_list_price >= 58.00),(store_sales.ss_list_price <= 68.00)],AND[(store_sales.ss_coupon_amt >= 9402.00),(store_sales.ss_coupon_amt <= 10402.00)],AND[(store_sales.ss_wholesale_cost >= 38.00),(store_sales.ss_wholesale_cost <= 58.00)]]) ------------------------PhysicalOlapScan[store_sales] --------PhysicalLimit[LOCAL] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecGather] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------filter(((((store_sales.ss_list_price >= 64.00) AND (store_sales.ss_list_price <= 74.00)) OR ((store_sales.ss_coupon_amt >= 5792.00) AND (store_sales.ss_coupon_amt <= 6792.00))) OR ((store_sales.ss_wholesale_cost >= 73.00) AND (store_sales.ss_wholesale_cost <= 93.00))) and (store_sales.ss_quantity <= 30) and (store_sales.ss_quantity >= 26)) +------------------filter((store_sales.ss_quantity <= 30) and (store_sales.ss_quantity >= 26) and OR[AND[(store_sales.ss_list_price >= 64.00),(store_sales.ss_list_price <= 74.00)],AND[(store_sales.ss_coupon_amt >= 5792.00),(store_sales.ss_coupon_amt <= 6792.00)],AND[(store_sales.ss_wholesale_cost >= 73.00),(store_sales.ss_wholesale_cost <= 93.00)]]) --------------------PhysicalOlapScan[store_sales] diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query34.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query34.out index 4b86edab02cf41..12e90cc7312c45 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query34.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query34.out @@ -25,7 +25,7 @@ PhysicalResultSink ------------------------------------filter((store.s_county = 'Williamson County')) --------------------------------------PhysicalOlapScan[store] ------------------------------PhysicalProject ---------------------------------filter((((date_dim.d_dom >= 1) AND (date_dim.d_dom <= 3)) OR ((date_dim.d_dom >= 25) AND (date_dim.d_dom <= 28))) and d_year IN (2000, 2001, 2002)) +--------------------------------filter(OR[AND[(date_dim.d_dom >= 1),(date_dim.d_dom <= 3)],AND[(date_dim.d_dom >= 25),(date_dim.d_dom <= 28)]] and d_year IN (2000, 2001, 2002)) ----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalProject ----------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('0-500', '1001-5000')) diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query41.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query41.out index ea5acccd883550..0bba60d4cdac39 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query41.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query41.out @@ -18,6 +18,6 @@ PhysicalResultSink ------------------------PhysicalDistribute[DistributionSpecHash] --------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------filter((((i_color IN ('forest', 'lime', 'maroon', 'navy', 'powder', 'sky', 'slate', 'smoke') AND i_units IN ('Bunch', 'Case', 'Dozen', 'Gross', 'Lb', 'Ounce', 'Pallet', 'Pound')) AND (((((((item.i_category = 'Women') AND i_color IN ('forest', 'lime')) AND i_units IN ('Pallet', 'Pound')) AND i_size IN ('economy', 'small')) OR ((((item.i_category = 'Women') AND i_color IN ('navy', 'slate')) AND i_units IN ('Bunch', 'Gross')) AND i_size IN ('extra large', 'petite'))) OR ((((item.i_category = 'Men') AND i_color IN ('powder', 'sky')) AND i_units IN ('Dozen', 'Lb')) AND i_size IN ('N/A', 'large'))) OR ((((item.i_category = 'Men') AND i_color IN ('maroon', 'smoke')) AND i_units IN ('Case', 'Ounce')) AND i_size IN ('economy', 'small')))) OR ((i_color IN ('aquamarine', 'dark', 'firebrick', 'frosted', 'papaya', 'peach', 'plum', 'sienna') AND i_units IN ('Box', 'Bundle', 'Carton', 'Cup', 'Dram', 'Each', 'Tbl', 'Ton')) AND (((((((item.i_category = 'Women') AND i_color IN ('aquamarine', 'dark')) AND i_units IN ('Tbl', 'Ton')) AND i_size IN ('economy', 'small')) OR ((((item.i_category = 'Women') AND i_color IN ('frosted', 'plum')) AND i_units IN ('Box', 'Dram')) AND i_size IN ('extra large', 'petite'))) OR ((((item.i_category = 'Men') AND i_color IN ('papaya', 'peach')) AND i_units IN ('Bundle', 'Carton')) AND i_size IN ('N/A', 'large'))) OR ((((item.i_category = 'Men') AND i_color IN ('firebrick', 'sienna')) AND i_units IN ('Cup', 'Each')) AND i_size IN ('economy', 'small'))))) and i_category IN ('Men', 'Women') and i_size IN ('N/A', 'economy', 'extra large', 'large', 'petite', 'small')) +------------------------------filter(OR[AND[i_color IN ('forest', 'lime', 'maroon', 'navy', 'powder', 'sky', 'slate', 'smoke'),i_units IN ('Bunch', 'Case', 'Dozen', 'Gross', 'Lb', 'Ounce', 'Pallet', 'Pound'),OR[AND[(item.i_category = 'Women'),i_color IN ('forest', 'lime'),i_units IN ('Pallet', 'Pound'),i_size IN ('economy', 'small')],AND[(item.i_category = 'Women'),i_color IN ('navy', 'slate'),i_units IN ('Bunch', 'Gross'),i_size IN ('extra large', 'petite')],AND[(item.i_category = 'Men'),i_color IN ('powder', 'sky'),i_units IN ('Dozen', 'Lb'),i_size IN ('N/A', 'large')],AND[(item.i_category = 'Men'),i_color IN ('maroon', 'smoke'),i_units IN ('Case', 'Ounce'),i_size IN ('economy', 'small')]]],AND[i_color IN ('aquamarine', 'dark', 'firebrick', 'frosted', 'papaya', 'peach', 'plum', 'sienna'),i_units IN ('Box', 'Bundle', 'Carton', 'Cup', 'Dram', 'Each', 'Tbl', 'Ton'),OR[AND[(item.i_category = 'Women'),i_color IN ('aquamarine', 'dark'),i_units IN ('Tbl', 'Ton'),i_size IN ('economy', 'small')],AND[(item.i_category = 'Women'),i_color IN ('frosted', 'plum'),i_units IN ('Box', 'Dram'),i_size IN ('extra large', 'petite')],AND[(item.i_category = 'Men'),i_color IN ('papaya', 'peach'),i_units IN ('Bundle', 'Carton'),i_size IN ('N/A', 'large')],AND[(item.i_category = 'Men'),i_color IN ('firebrick', 'sienna'),i_units IN ('Cup', 'Each'),i_size IN ('economy', 'small')]]]] and i_category IN ('Men', 'Women') and i_size IN ('N/A', 'economy', 'extra large', 'large', 'petite', 'small')) --------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query45.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query45.out index b65fa9047c0c87..dcc2e202e23752 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query45.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query45.out @@ -8,7 +8,7 @@ PhysicalResultSink ----------PhysicalDistribute[DistributionSpecHash] ------------hashAgg[LOCAL] --------------PhysicalProject -----------------filter((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR $c$1)) +----------------filter(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),$c$1]) ------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ws_bill_customer_sk] diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query46.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query46.out index 2f09cd1407d04e..441387467eaf87 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query46.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query46.out @@ -29,7 +29,7 @@ PhysicalResultSink ----------------------------------filter(d_dow IN (0, 6) and d_year IN (2000, 2001, 2002)) ------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalProject -------------------------------filter(((household_demographics.hd_dep_count = 8) OR (household_demographics.hd_vehicle_count = 0))) +------------------------------filter(OR[(household_demographics.hd_dep_count = 8),(household_demographics.hd_vehicle_count = 0)]) --------------------------------PhysicalOlapScan[household_demographics] ------------------------PhysicalProject --------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query47.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query47.out index 5554797a7b9dc3..b1694f8470571f 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query47.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query47.out @@ -21,7 +21,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------------------------PhysicalProject --------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 ------------------------------------PhysicalProject ---------------------------------------filter((((date_dim.d_year = 2000) OR ((date_dim.d_year = 1999) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2001) AND (date_dim.d_moy = 1))) and d_year IN (1999, 2000, 2001)) +--------------------------------------filter(OR[(date_dim.d_year = 2000),AND[(date_dim.d_year = 1999),(date_dim.d_moy = 12)],AND[(date_dim.d_year = 2001),(date_dim.d_moy = 1)]] and d_year IN (1999, 2000, 2001)) ----------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query48.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query48.out index 06ecefed7e0624..6b34ba86efb1b0 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query48.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query48.out @@ -9,14 +9,14 @@ PhysicalResultSink ------------PhysicalProject --------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=((((ca_state IN ('ND', 'NY', 'SD') AND ((store_sales.ss_net_profit >= 0.00) AND (store_sales.ss_net_profit <= 2000.00))) OR (ca_state IN ('GA', 'KS', 'MD') AND ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 3000.00)))) OR (ca_state IN ('CO', 'MN', 'NC') AND ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 25000.00))))) build RFs:RF1 ca_address_sk->[ss_addr_sk] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=(OR[AND[ca_state IN ('ND', 'NY', 'SD'),(store_sales.ss_net_profit >= 0.00),(store_sales.ss_net_profit <= 2000.00)],AND[ca_state IN ('GA', 'KS', 'MD'),(store_sales.ss_net_profit >= 150.00),(store_sales.ss_net_profit <= 3000.00)],AND[ca_state IN ('CO', 'MN', 'NC'),(store_sales.ss_net_profit >= 50.00),(store_sales.ss_net_profit <= 25000.00)]]) build RFs:RF1 ca_address_sk->[ss_addr_sk] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)) otherCondition=((((((customer_demographics.cd_marital_status = 'S') AND (customer_demographics.cd_education_status = 'Secondary')) AND ((store_sales.ss_sales_price >= 100.00) AND (store_sales.ss_sales_price <= 150.00))) OR (((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = '2 yr Degree')) AND ((store_sales.ss_sales_price >= 50.00) AND (store_sales.ss_sales_price <= 100.00)))) OR (((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = 'Advanced Degree')) AND ((store_sales.ss_sales_price >= 150.00) AND (store_sales.ss_sales_price <= 200.00))))) build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)) otherCondition=(OR[AND[(customer_demographics.cd_marital_status = 'S'),(customer_demographics.cd_education_status = 'Secondary'),(store_sales.ss_sales_price >= 100.00),(store_sales.ss_sales_price <= 150.00)],AND[(customer_demographics.cd_marital_status = 'M'),(customer_demographics.cd_education_status = '2 yr Degree'),(store_sales.ss_sales_price >= 50.00),(store_sales.ss_sales_price <= 100.00)],AND[(customer_demographics.cd_marital_status = 'D'),(customer_demographics.cd_education_status = 'Advanced Degree'),(store_sales.ss_sales_price >= 150.00),(store_sales.ss_sales_price <= 200.00)]]) build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] ------------------------PhysicalProject --------------------------filter((store_sales.ss_net_profit <= 25000.00) and (store_sales.ss_net_profit >= 0.00) and (store_sales.ss_sales_price <= 200.00) and (store_sales.ss_sales_price >= 50.00)) ----------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 ------------------------PhysicalProject ---------------------------filter(((((customer_demographics.cd_marital_status = 'S') AND (customer_demographics.cd_education_status = 'Secondary')) OR ((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = '2 yr Degree'))) OR ((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = 'Advanced Degree'))) and cd_education_status IN ('2 yr Degree', 'Advanced Degree', 'Secondary') and cd_marital_status IN ('D', 'M', 'S')) +--------------------------filter(OR[AND[(customer_demographics.cd_marital_status = 'S'),(customer_demographics.cd_education_status = 'Secondary')],AND[(customer_demographics.cd_marital_status = 'M'),(customer_demographics.cd_education_status = '2 yr Degree')],AND[(customer_demographics.cd_marital_status = 'D'),(customer_demographics.cd_education_status = 'Advanced Degree')]] and cd_education_status IN ('2 yr Degree', 'Advanced Degree', 'Secondary') and cd_marital_status IN ('D', 'M', 'S')) ----------------------------PhysicalOlapScan[customer_demographics] --------------------PhysicalProject ----------------------filter((customer_address.ca_country = 'United States') and ca_state IN ('CO', 'GA', 'KS', 'MD', 'MN', 'NC', 'ND', 'NY', 'SD')) diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query49.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query49.out index 889d5069ff9751..8b807baf5409e4 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query49.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query49.out @@ -16,7 +16,7 @@ PhysicalResultSink --------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) ----------------------------------PhysicalWindow ------------------------------------PhysicalQuickSort[LOCAL_SORT] --------------------------------------PhysicalWindow @@ -48,7 +48,7 @@ PhysicalResultSink --------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) ----------------------------------PhysicalWindow ------------------------------------PhysicalQuickSort[LOCAL_SORT] --------------------------------------PhysicalWindow @@ -80,7 +80,7 @@ PhysicalResultSink --------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) ----------------------------------PhysicalWindow ------------------------------------PhysicalQuickSort[LOCAL_SORT] --------------------------------------PhysicalWindow diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query53.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query53.out index 5e987921a5d3f7..cdbe5ee8705ae0 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query53.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query53.out @@ -22,7 +22,7 @@ PhysicalResultSink --------------------------------------PhysicalProject ----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 --------------------------------------PhysicalProject -----------------------------------------filter((((i_category IN ('Books', 'Children', 'Electronics') AND i_class IN ('personal', 'portable', 'reference', 'self-help')) AND i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')) OR ((i_category IN ('Men', 'Music', 'Women') AND i_class IN ('accessories', 'classical', 'fragrances', 'pants')) AND i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1'))) and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) +----------------------------------------filter(OR[AND[i_category IN ('Books', 'Children', 'Electronics'),i_class IN ('personal', 'portable', 'reference', 'self-help'),i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')],AND[i_category IN ('Men', 'Music', 'Women'),i_class IN ('accessories', 'classical', 'fragrances', 'pants'),i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')]] and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) ------------------------------------------PhysicalOlapScan[item] ----------------------------------PhysicalProject ------------------------------------filter(d_month_seq IN (1186, 1187, 1188, 1189, 1190, 1191, 1192, 1193, 1194, 1195, 1196, 1197)) diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query57.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query57.out index 91234f18164852..f1d71f63e53c41 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query57.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query57.out @@ -21,7 +21,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------------------------PhysicalProject --------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 ------------------------------------PhysicalProject ---------------------------------------filter((((date_dim.d_year = 2001) OR ((date_dim.d_year = 2000) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2002) AND (date_dim.d_moy = 1))) and d_year IN (2000, 2001, 2002)) +--------------------------------------filter(OR[(date_dim.d_year = 2001),AND[(date_dim.d_year = 2000),(date_dim.d_moy = 12)],AND[(date_dim.d_year = 2002),(date_dim.d_moy = 1)]] and d_year IN (2000, 2001, 2002)) ----------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[call_center] diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query61.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query61.out index 4e0a1573b53eb0..4f8ab146d22a36 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query61.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query61.out @@ -34,7 +34,7 @@ PhysicalResultSink --------------------------------------filter((item.i_category = 'Home')) ----------------------------------------PhysicalOlapScan[item] --------------------------------PhysicalProject -----------------------------------filter((((promotion.p_channel_dmail = 'Y') OR (promotion.p_channel_email = 'Y')) OR (promotion.p_channel_tv = 'Y'))) +----------------------------------filter(OR[(promotion.p_channel_dmail = 'Y'),(promotion.p_channel_email = 'Y'),(promotion.p_channel_tv = 'Y')]) ------------------------------------PhysicalOlapScan[promotion] ----------------------------PhysicalProject ------------------------------filter((store.s_gmt_offset = -7.00)) diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query63.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query63.out index 89a666497689a1..f9d54e03f75154 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query63.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query63.out @@ -22,7 +22,7 @@ PhysicalResultSink --------------------------------------PhysicalProject ----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 --------------------------------------PhysicalProject -----------------------------------------filter((((i_category IN ('Books', 'Children', 'Electronics') AND i_class IN ('personal', 'portable', 'reference', 'self-help')) AND i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')) OR ((i_category IN ('Men', 'Music', 'Women') AND i_class IN ('accessories', 'classical', 'fragrances', 'pants')) AND i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1'))) and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) +----------------------------------------filter(OR[AND[i_category IN ('Books', 'Children', 'Electronics'),i_class IN ('personal', 'portable', 'reference', 'self-help'),i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')],AND[i_category IN ('Men', 'Music', 'Women'),i_class IN ('accessories', 'classical', 'fragrances', 'pants'),i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')]] and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) ------------------------------------------PhysicalOlapScan[item] ----------------------------------PhysicalProject ------------------------------------filter(d_month_seq IN (1222, 1223, 1224, 1225, 1226, 1227, 1228, 1229, 1230, 1231, 1232, 1233)) diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query68.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query68.out index 084a62e24b6954..9e66e56512ebc7 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query68.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query68.out @@ -33,7 +33,7 @@ PhysicalResultSink ----------------------------------filter(s_city IN ('Fairview', 'Midway')) ------------------------------------PhysicalOlapScan[store] ----------------------------PhysicalProject -------------------------------filter(((household_demographics.hd_dep_count = 3) OR (household_demographics.hd_vehicle_count = 4))) +------------------------------filter(OR[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count = 4)]) --------------------------------PhysicalOlapScan[household_demographics] Hint log: diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query7.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query7.out index 1ccce08635ab65..89f1c0f0b3a615 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query7.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query7.out @@ -24,7 +24,7 @@ PhysicalResultSink ----------------------------filter((date_dim.d_year = 2001)) ------------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalProject -------------------------filter(((promotion.p_channel_email = 'N') OR (promotion.p_channel_event = 'N'))) +------------------------filter(OR[(promotion.p_channel_email = 'N'),(promotion.p_channel_event = 'N')]) --------------------------PhysicalOlapScan[promotion] ------------------PhysicalProject --------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query78.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query78.out index 52ab844c3f1bbc..e57834b15ff42a 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query78.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query78.out @@ -5,7 +5,7 @@ PhysicalResultSink ----PhysicalDistribute[DistributionSpecGather] ------PhysicalTopN[LOCAL_SORT] --------PhysicalProject -----------filter(((coalesce(ws_qty, 0) > 0) OR (coalesce(cs_qty, 0) > 0))) +----------filter(OR[(coalesce(ws_qty, 0) > 0),(coalesce(cs_qty, 0) > 0)]) ------------hashJoin[LEFT_OUTER_JOIN colocated] hashCondition=((cs.cs_customer_sk = ss.ss_customer_sk) and (cs.cs_item_sk = ss.ss_item_sk) and (cs.cs_sold_year = ss.ss_sold_year)) otherCondition=() --------------PhysicalProject ----------------hashJoin[LEFT_OUTER_JOIN colocated] hashCondition=((ws.ws_customer_sk = ss.ss_customer_sk) and (ws.ws_item_sk = ss.ss_item_sk) and (ws.ws_sold_year = ss.ss_sold_year)) otherCondition=() diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query79.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query79.out index a03517f058c033..62c0b640f35218 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query79.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query79.out @@ -22,7 +22,7 @@ PhysicalResultSink ----------------------------------filter((date_dim.d_dow = 1) and d_year IN (2000, 2001, 2002)) ------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalProject -------------------------------filter(((household_demographics.hd_dep_count = 7) OR (household_demographics.hd_vehicle_count > -1))) +------------------------------filter(OR[(household_demographics.hd_dep_count = 7),(household_demographics.hd_vehicle_count > -1)]) --------------------------------PhysicalOlapScan[household_demographics] ------------------------PhysicalProject --------------------------filter((store.s_number_employees <= 295) and (store.s_number_employees >= 200)) diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query85.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query85.out index aca2a36b6a8e3a..2826be96b9c8c8 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query85.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query85.out @@ -20,12 +20,12 @@ PhysicalResultSink ------------------------------filter(cd_education_status IN ('Advanced Degree', 'College', 'Primary') and cd_marital_status IN ('D', 'S', 'U')) --------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF5 RF6 RF7 ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((customer_address.ca_address_sk = web_returns.wr_refunded_addr_sk)) otherCondition=((((ca_state IN ('IA', 'NC', 'TX') AND ((web_sales.ws_net_profit >= 100.00) AND (web_sales.ws_net_profit <= 200.00))) OR (ca_state IN ('GA', 'WI', 'WV') AND ((web_sales.ws_net_profit >= 150.00) AND (web_sales.ws_net_profit <= 300.00)))) OR (ca_state IN ('KY', 'OK', 'VA') AND ((web_sales.ws_net_profit >= 50.00) AND (web_sales.ws_net_profit <= 250.00))))) build RFs:RF4 wr_refunded_addr_sk->[ca_address_sk] +------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((customer_address.ca_address_sk = web_returns.wr_refunded_addr_sk)) otherCondition=(OR[AND[ca_state IN ('IA', 'NC', 'TX'),(web_sales.ws_net_profit >= 100.00),(web_sales.ws_net_profit <= 200.00)],AND[ca_state IN ('GA', 'WI', 'WV'),(web_sales.ws_net_profit >= 150.00),(web_sales.ws_net_profit <= 300.00)],AND[ca_state IN ('KY', 'OK', 'VA'),(web_sales.ws_net_profit >= 50.00),(web_sales.ws_net_profit <= 250.00)]]) build RFs:RF4 wr_refunded_addr_sk->[ca_address_sk] --------------------------------PhysicalProject ----------------------------------filter((customer_address.ca_country = 'United States') and ca_state IN ('GA', 'IA', 'KY', 'NC', 'OK', 'TX', 'VA', 'WI', 'WV')) ------------------------------------PhysicalOlapScan[customer_address] apply RFs: RF4 --------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cd1.cd_demo_sk = web_returns.wr_refunded_cdemo_sk)) otherCondition=((((((cd1.cd_marital_status = 'D') AND (cd1.cd_education_status = 'Primary')) AND ((web_sales.ws_sales_price >= 100.00) AND (web_sales.ws_sales_price <= 150.00))) OR (((cd1.cd_marital_status = 'S') AND (cd1.cd_education_status = 'College')) AND ((web_sales.ws_sales_price >= 50.00) AND (web_sales.ws_sales_price <= 100.00)))) OR (((cd1.cd_marital_status = 'U') AND (cd1.cd_education_status = 'Advanced Degree')) AND ((web_sales.ws_sales_price >= 150.00) AND (web_sales.ws_sales_price <= 200.00))))) build RFs:RF3 cd_demo_sk->[wr_refunded_cdemo_sk] +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cd1.cd_demo_sk = web_returns.wr_refunded_cdemo_sk)) otherCondition=(OR[AND[(cd1.cd_marital_status = 'D'),(cd1.cd_education_status = 'Primary'),(web_sales.ws_sales_price >= 100.00),(web_sales.ws_sales_price <= 150.00)],AND[(cd1.cd_marital_status = 'S'),(cd1.cd_education_status = 'College'),(web_sales.ws_sales_price >= 50.00),(web_sales.ws_sales_price <= 100.00)],AND[(cd1.cd_marital_status = 'U'),(cd1.cd_education_status = 'Advanced Degree'),(web_sales.ws_sales_price >= 150.00),(web_sales.ws_sales_price <= 200.00)]]) build RFs:RF3 cd_demo_sk->[wr_refunded_cdemo_sk] ------------------------------------PhysicalProject --------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((web_sales.ws_item_sk = web_returns.wr_item_sk) and (web_sales.ws_order_number = web_returns.wr_order_number)) otherCondition=() build RFs:RF1 ws_item_sk->[wr_item_sk];RF2 ws_order_number->[wr_order_number] ----------------------------------------PhysicalProject @@ -39,7 +39,7 @@ PhysicalResultSink ----------------------------------------------filter((date_dim.d_year = 1998)) ------------------------------------------------PhysicalOlapScan[date_dim] ------------------------------------PhysicalProject ---------------------------------------filter(((((cd1.cd_marital_status = 'D') AND (cd1.cd_education_status = 'Primary')) OR ((cd1.cd_marital_status = 'S') AND (cd1.cd_education_status = 'College'))) OR ((cd1.cd_marital_status = 'U') AND (cd1.cd_education_status = 'Advanced Degree'))) and cd_education_status IN ('Advanced Degree', 'College', 'Primary') and cd_marital_status IN ('D', 'S', 'U')) +--------------------------------------filter(OR[AND[(cd1.cd_marital_status = 'D'),(cd1.cd_education_status = 'Primary')],AND[(cd1.cd_marital_status = 'S'),(cd1.cd_education_status = 'College')],AND[(cd1.cd_marital_status = 'U'),(cd1.cd_education_status = 'Advanced Degree')]] and cd_education_status IN ('Advanced Degree', 'College', 'Primary') and cd_marital_status IN ('D', 'S', 'U')) ----------------------------------------PhysicalOlapScan[customer_demographics] ------------------------PhysicalProject --------------------------PhysicalOlapScan[reason] diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query88.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query88.out index 49e27f92b64a61..8f366f3d8c0202 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query88.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query88.out @@ -24,7 +24,7 @@ PhysicalResultSink --------------------------------------filter((time_dim.t_hour = 8) and (time_dim.t_minute >= 30)) ----------------------------------------PhysicalOlapScan[time_dim] --------------------------------PhysicalProject -----------------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3)) +----------------------------------filter(OR[AND[(household_demographics.hd_dep_count = 0),(household_demographics.hd_vehicle_count <= 2)],AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 0, 3)) ------------------------------------PhysicalOlapScan[household_demographics] ----------------------------PhysicalProject ------------------------------filter((store.s_store_name = 'ese')) @@ -45,7 +45,7 @@ PhysicalResultSink --------------------------------------filter((time_dim.t_hour = 9) and (time_dim.t_minute < 30)) ----------------------------------------PhysicalOlapScan[time_dim] --------------------------------PhysicalProject -----------------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3)) +----------------------------------filter(OR[AND[(household_demographics.hd_dep_count = 0),(household_demographics.hd_vehicle_count <= 2)],AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 0, 3)) ------------------------------------PhysicalOlapScan[household_demographics] ----------------------------PhysicalProject ------------------------------filter((store.s_store_name = 'ese')) @@ -66,7 +66,7 @@ PhysicalResultSink ------------------------------------filter((time_dim.t_hour = 9) and (time_dim.t_minute >= 30)) --------------------------------------PhysicalOlapScan[time_dim] ------------------------------PhysicalProject ---------------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3)) +--------------------------------filter(OR[AND[(household_demographics.hd_dep_count = 0),(household_demographics.hd_vehicle_count <= 2)],AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 0, 3)) ----------------------------------PhysicalOlapScan[household_demographics] --------------------------PhysicalProject ----------------------------filter((store.s_store_name = 'ese')) @@ -87,7 +87,7 @@ PhysicalResultSink ----------------------------------filter((time_dim.t_hour = 10) and (time_dim.t_minute < 30)) ------------------------------------PhysicalOlapScan[time_dim] ----------------------------PhysicalProject -------------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3)) +------------------------------filter(OR[AND[(household_demographics.hd_dep_count = 0),(household_demographics.hd_vehicle_count <= 2)],AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 0, 3)) --------------------------------PhysicalOlapScan[household_demographics] ------------------------PhysicalProject --------------------------filter((store.s_store_name = 'ese')) @@ -108,7 +108,7 @@ PhysicalResultSink --------------------------------filter((time_dim.t_hour = 10) and (time_dim.t_minute >= 30)) ----------------------------------PhysicalOlapScan[time_dim] --------------------------PhysicalProject -----------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3)) +----------------------------filter(OR[AND[(household_demographics.hd_dep_count = 0),(household_demographics.hd_vehicle_count <= 2)],AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 0, 3)) ------------------------------PhysicalOlapScan[household_demographics] ----------------------PhysicalProject ------------------------filter((store.s_store_name = 'ese')) @@ -129,7 +129,7 @@ PhysicalResultSink ------------------------------filter((time_dim.t_hour = 11) and (time_dim.t_minute < 30)) --------------------------------PhysicalOlapScan[time_dim] ------------------------PhysicalProject ---------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3)) +--------------------------filter(OR[AND[(household_demographics.hd_dep_count = 0),(household_demographics.hd_vehicle_count <= 2)],AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 0, 3)) ----------------------------PhysicalOlapScan[household_demographics] --------------------PhysicalProject ----------------------filter((store.s_store_name = 'ese')) @@ -150,7 +150,7 @@ PhysicalResultSink ----------------------------filter((time_dim.t_hour = 11) and (time_dim.t_minute >= 30)) ------------------------------PhysicalOlapScan[time_dim] ----------------------PhysicalProject -------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3)) +------------------------filter(OR[AND[(household_demographics.hd_dep_count = 0),(household_demographics.hd_vehicle_count <= 2)],AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 0, 3)) --------------------------PhysicalOlapScan[household_demographics] ------------------PhysicalProject --------------------filter((store.s_store_name = 'ese')) @@ -171,7 +171,7 @@ PhysicalResultSink --------------------------filter((time_dim.t_hour = 12) and (time_dim.t_minute < 30)) ----------------------------PhysicalOlapScan[time_dim] --------------------PhysicalProject -----------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3)) +----------------------filter(OR[AND[(household_demographics.hd_dep_count = 0),(household_demographics.hd_vehicle_count <= 2)],AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 0, 3)) ------------------------PhysicalOlapScan[household_demographics] ----------------PhysicalProject ------------------filter((store.s_store_name = 'ese')) diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query89.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query89.out index 91ff8765933815..e3f71a702a8ae1 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query89.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query89.out @@ -23,7 +23,7 @@ PhysicalResultSink ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 ----------------------------------------PhysicalProject -------------------------------------------filter(((i_category IN ('Books', 'Children', 'Electronics') AND i_class IN ('audio', 'history', 'school-uniforms')) OR (i_category IN ('Men', 'Shoes', 'Sports') AND i_class IN ('pants', 'tennis', 'womens'))) and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Shoes', 'Sports') and i_class IN ('audio', 'history', 'pants', 'school-uniforms', 'tennis', 'womens')) +------------------------------------------filter(OR[AND[i_category IN ('Books', 'Children', 'Electronics'),i_class IN ('audio', 'history', 'school-uniforms')],AND[i_category IN ('Men', 'Shoes', 'Sports'),i_class IN ('pants', 'tennis', 'womens')]] and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Shoes', 'Sports') and i_class IN ('audio', 'history', 'pants', 'school-uniforms', 'tennis', 'womens')) --------------------------------------------PhysicalOlapScan[item] ------------------------------------PhysicalProject --------------------------------------filter((date_dim.d_year = 2001)) diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query91.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query91.out index 95b8e947317d70..3c8761017d349b 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query91.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query91.out @@ -28,7 +28,7 @@ PhysicalResultSink ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 RF1 ----------------------------------------PhysicalProject -------------------------------------------filter((((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = 'Unknown')) OR ((customer_demographics.cd_marital_status = 'W') AND (customer_demographics.cd_education_status = 'Advanced Degree'))) and cd_education_status IN ('Advanced Degree', 'Unknown') and cd_marital_status IN ('M', 'W')) +------------------------------------------filter(OR[AND[(customer_demographics.cd_marital_status = 'M'),(customer_demographics.cd_education_status = 'Unknown')],AND[(customer_demographics.cd_marital_status = 'W'),(customer_demographics.cd_education_status = 'Advanced Degree')]] and cd_education_status IN ('Advanced Degree', 'Unknown') and cd_marital_status IN ('M', 'W')) --------------------------------------------PhysicalOlapScan[customer_demographics] ------------------------------------PhysicalProject --------------------------------------filter((hd_buy_potential like 'Unknown%')) diff --git a/regression-test/data/nereids_hint_tpch_p0/shape/q19.out b/regression-test/data/nereids_hint_tpch_p0/shape/q19.out index 940b16c90e40d5..5cabdf9f163c56 100644 --- a/regression-test/data/nereids_hint_tpch_p0/shape/q19.out +++ b/regression-test/data/nereids_hint_tpch_p0/shape/q19.out @@ -5,12 +5,12 @@ PhysicalResultSink ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] --------PhysicalProject -----------hashJoin[INNER_JOIN broadcast] hashCondition=((part.p_partkey = lineitem.l_partkey)) otherCondition=(((((((part.p_brand = 'Brand#12') AND p_container IN ('SM BOX', 'SM CASE', 'SM PACK', 'SM PKG')) AND ((lineitem.l_quantity >= 1.00) AND (lineitem.l_quantity <= 11.00))) AND (part.p_size <= 5)) OR ((((part.p_brand = 'Brand#23') AND p_container IN ('MED BAG', 'MED BOX', 'MED PACK', 'MED PKG')) AND ((lineitem.l_quantity >= 10.00) AND (lineitem.l_quantity <= 20.00))) AND (part.p_size <= 10))) OR ((((part.p_brand = 'Brand#34') AND p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG')) AND ((lineitem.l_quantity >= 20.00) AND (lineitem.l_quantity <= 30.00))) AND (part.p_size <= 15)))) +----------hashJoin[INNER_JOIN broadcast] hashCondition=((part.p_partkey = lineitem.l_partkey)) otherCondition=(OR[AND[(part.p_brand = 'Brand#12'),p_container IN ('SM BOX', 'SM CASE', 'SM PACK', 'SM PKG'),(lineitem.l_quantity >= 1.00),(lineitem.l_quantity <= 11.00),(part.p_size <= 5)],AND[(part.p_brand = 'Brand#23'),p_container IN ('MED BAG', 'MED BOX', 'MED PACK', 'MED PKG'),(lineitem.l_quantity >= 10.00),(lineitem.l_quantity <= 20.00),(part.p_size <= 10)],AND[(part.p_brand = 'Brand#34'),p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG'),(lineitem.l_quantity >= 20.00),(lineitem.l_quantity <= 30.00),(part.p_size <= 15)]]) ------------PhysicalProject --------------filter((lineitem.l_quantity <= 30.00) and (lineitem.l_quantity >= 1.00) and (lineitem.l_shipinstruct = 'DELIVER IN PERSON') and l_shipmode IN ('AIR REG', 'AIR')) ----------------PhysicalOlapScan[lineitem] ------------PhysicalProject ---------------filter((((((part.p_brand = 'Brand#12') AND p_container IN ('SM BOX', 'SM CASE', 'SM PACK', 'SM PKG')) AND (part.p_size <= 5)) OR (((part.p_brand = 'Brand#23') AND p_container IN ('MED BAG', 'MED BOX', 'MED PACK', 'MED PKG')) AND (part.p_size <= 10))) OR (((part.p_brand = 'Brand#34') AND p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG')) AND (part.p_size <= 15))) and (part.p_size >= 1) and p_brand IN ('Brand#12', 'Brand#23', 'Brand#34') and p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG', 'MED BAG', 'MED BOX', 'MED PACK', 'MED PKG', 'SM BOX', 'SM CASE', 'SM PACK', 'SM PKG')) +--------------filter((part.p_size >= 1) and OR[AND[(part.p_brand = 'Brand#12'),p_container IN ('SM BOX', 'SM CASE', 'SM PACK', 'SM PKG'),(part.p_size <= 5)],AND[(part.p_brand = 'Brand#23'),p_container IN ('MED BAG', 'MED BOX', 'MED PACK', 'MED PKG'),(part.p_size <= 10)],AND[(part.p_brand = 'Brand#34'),p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG'),(part.p_size <= 15)]] and p_brand IN ('Brand#12', 'Brand#23', 'Brand#34') and p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG', 'MED BAG', 'MED BOX', 'MED PACK', 'MED PKG', 'SM BOX', 'SM CASE', 'SM PACK', 'SM PKG')) ----------------PhysicalOlapScan[part] Hint log: diff --git a/regression-test/data/nereids_hint_tpch_p0/shape/q7.out b/regression-test/data/nereids_hint_tpch_p0/shape/q7.out index 9cffd25181dbe0..d9fb0e0791a203 100644 --- a/regression-test/data/nereids_hint_tpch_p0/shape/q7.out +++ b/regression-test/data/nereids_hint_tpch_p0/shape/q7.out @@ -8,7 +8,7 @@ PhysicalResultSink ----------PhysicalDistribute[DistributionSpecHash] ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=((((n1.n_name = 'FRANCE') AND (n2.n_name = 'GERMANY')) OR ((n1.n_name = 'GERMANY') AND (n2.n_name = 'FRANCE')))) +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=(OR[AND[(n1.n_name = 'FRANCE'),(n2.n_name = 'GERMANY')],AND[(n1.n_name = 'GERMANY'),(n2.n_name = 'FRANCE')]]) ------------------PhysicalProject --------------------hashJoin[INNER_JOIN broadcast] hashCondition=((supplier.s_suppkey = lineitem.l_suppkey)) otherCondition=() ----------------------PhysicalProject diff --git a/regression-test/data/nereids_p0/huge_compound/huge_compound.out b/regression-test/data/nereids_p0/huge_compound/huge_compound.out new file mode 100644 index 00000000000000..e0843bea1df999 --- /dev/null +++ b/regression-test/data/nereids_p0/huge_compound/huge_compound.out @@ -0,0 +1,4 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !1 -- +72 93 + diff --git a/regression-test/data/nereids_rules_p0/eliminate_not_null/eliminate_not_null.out b/regression-test/data/nereids_rules_p0/eliminate_not_null/eliminate_not_null.out index 8f1bab13986b7d..877492a2eed6ad 100644 --- a/regression-test/data/nereids_rules_p0/eliminate_not_null/eliminate_not_null.out +++ b/regression-test/data/nereids_rules_p0/eliminate_not_null/eliminate_not_null.out @@ -37,7 +37,7 @@ PhysicalResultSink -- !eliminate_not_null_complex_logic -- PhysicalResultSink ---filter(( not score IS NULL) and ((t.score > 5) OR (t.id < 10))) +--filter(( not score IS NULL) and OR[(t.score > 5),(t.id < 10)]) ----PhysicalOlapScan[t] -- !eliminate_not_null_date_function -- diff --git a/regression-test/data/nereids_rules_p0/eliminate_outer_join/eliminate_outer_join.out b/regression-test/data/nereids_rules_p0/eliminate_outer_join/eliminate_outer_join.out index d92655e4e73e0b..48b3db99631679 100644 --- a/regression-test/data/nereids_rules_p0/eliminate_outer_join/eliminate_outer_join.out +++ b/regression-test/data/nereids_rules_p0/eliminate_outer_join/eliminate_outer_join.out @@ -128,7 +128,7 @@ PhysicalResultSink -- !full_outer_join_compound_conditions -- PhysicalResultSink --PhysicalDistribute[DistributionSpecGather] -----filter(((t1.score > 5) OR (t2.score > 5))) +----filter(OR[(t1.score > 5),(t2.score > 5)]) ------hashJoin[FULL_OUTER_JOIN colocated] hashCondition=((t1.id = t2.id)) otherCondition=() --------PhysicalOlapScan[t] --------PhysicalOlapScan[t] diff --git a/regression-test/data/nereids_rules_p0/filter_push_down/push_down_expression_in_hash_join.out b/regression-test/data/nereids_rules_p0/filter_push_down/push_down_expression_in_hash_join.out index ec63cd6aabdc4f..b4bddac5a5e077 100644 --- a/regression-test/data/nereids_rules_p0/filter_push_down/push_down_expression_in_hash_join.out +++ b/regression-test/data/nereids_rules_p0/filter_push_down/push_down_expression_in_hash_join.out @@ -70,7 +70,7 @@ PhysicalResultSink -- !pushdown_left_outer_join_subquery -- PhysicalResultSink ---NestedLoopJoin[INNER_JOIN]((cast(id as BIGINT) = (sum(id) - 1)) OR id IS NULL) +--NestedLoopJoin[INNER_JOIN]OR[(cast(id as BIGINT) = (sum(id) - 1)),id IS NULL] ----PhysicalOlapScan[t1] ----hashAgg[GLOBAL] ------hashAgg[LOCAL] diff --git a/regression-test/data/nereids_rules_p0/filter_push_down/push_down_filter_other_condition.out b/regression-test/data/nereids_rules_p0/filter_push_down/push_down_filter_other_condition.out index afa4e4cb6ae315..7bc1aded5877cc 100644 --- a/regression-test/data/nereids_rules_p0/filter_push_down/push_down_filter_other_condition.out +++ b/regression-test/data/nereids_rules_p0/filter_push_down/push_down_filter_other_condition.out @@ -190,7 +190,7 @@ PhysicalResultSink -- !pushdown_left_outer_join_subquery -- PhysicalResultSink ---filter(((cast(id as BIGINT) = sum(id)) OR id IS NULL)) +--filter(OR[(cast(id as BIGINT) = sum(id)),id IS NULL]) ----NestedLoopJoin[LEFT_OUTER_JOIN](id = 1) ------PhysicalOlapScan[t1] ------hashAgg[GLOBAL] @@ -199,7 +199,7 @@ PhysicalResultSink -- !pushdown_left_anti_join_subquery -- PhysicalResultSink ---NestedLoopJoin[LEFT_ANTI_JOIN](((t1.id = t2.id) OR id IS NULL) OR id IS NULL)(id > 1) +--NestedLoopJoin[LEFT_ANTI_JOIN]OR[(t1.id = t2.id),id IS NULL,id IS NULL](id > 1) ----PhysicalOlapScan[t1] ----PhysicalOlapScan[t2] @@ -229,7 +229,7 @@ PhysicalResultSink -- !pushdown_left_outer_join_subquery_outer -- PhysicalResultSink ---NestedLoopJoin[INNER_JOIN]((t1.id = t2.id) OR (id IS NULL AND (id > 1))) +--NestedLoopJoin[INNER_JOIN]OR[(t1.id = t2.id),AND[id IS NULL,(id > 1)]] ----PhysicalOlapScan[t1] ----PhysicalAssertNumRows ------PhysicalOlapScan[t2] diff --git a/regression-test/data/nereids_rules_p0/filter_push_down/push_filter_through.out b/regression-test/data/nereids_rules_p0/filter_push_down/push_filter_through.out index 906d59fa9016dc..a9f9442ee62559 100644 --- a/regression-test/data/nereids_rules_p0/filter_push_down/push_filter_through.out +++ b/regression-test/data/nereids_rules_p0/filter_push_down/push_filter_through.out @@ -59,7 +59,7 @@ PhysicalResultSink -- !filter_join_inner -- PhysicalResultSink ---hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=(((t1.id = 1) OR (t2.id = 2))) +--hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=(OR[(t1.id = 1),(t2.id = 2)]) ----PhysicalOlapScan[t1] ----PhysicalOlapScan[t2] @@ -77,7 +77,7 @@ PhysicalResultSink -- !filter_join_left -- PhysicalResultSink ---filter(((t1.id = 1) OR (t2.id = 2))) +--filter(OR[(t1.id = 1),(t2.id = 2)]) ----hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ------PhysicalOlapScan[t1] ------PhysicalOlapScan[t2] @@ -104,7 +104,7 @@ PhysicalResultSink -- !filter_join_left -- PhysicalResultSink ---filter(((t1.id = 1) OR (t2.id = 2))) +--filter(OR[(t1.id = 1),(t2.id = 2)]) ----hashJoin[FULL_OUTER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ------PhysicalOlapScan[t1] ------PhysicalOlapScan[t2] @@ -169,7 +169,7 @@ PhysicalResultSink -- !filter_mixed_inner_left -- PhysicalResultSink ---filter((((t1.id = 1) AND (t2.id = 2)) OR (t3.id = 2))) +--filter(OR[AND[(t1.id = 1),(t2.id = 2)],(t3.id = 2)]) ----hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.id = t3.id)) otherCondition=() ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------PhysicalOlapScan[t1] @@ -178,7 +178,7 @@ PhysicalResultSink -- !filter_multi_left -- PhysicalResultSink ---filter((((t1.id = 1) AND (t2.id > 1)) OR (t3.id < 4))) +--filter(OR[AND[(t1.id = 1),(t2.id > 1)],(t3.id < 4)]) ----hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.id = t3.id)) otherCondition=() ------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------PhysicalOlapScan[t1] @@ -236,7 +236,7 @@ PhysicalResultSink -- !filter_aggregation_group_set -- PhysicalResultSink ---filter(((t1.id > 10) OR (cast(msg as DOUBLE) = 1.0))) +--filter(OR[(t1.id > 10),(cast(msg as DOUBLE) = 1.0)]) ----hashAgg[LOCAL] ------PhysicalRepeat --------PhysicalOlapScan[t1] @@ -372,6 +372,6 @@ PhysicalResultSink ----PhysicalQuickSort[LOCAL_SORT] ------PhysicalWindow --------PhysicalQuickSort[LOCAL_SORT] -----------filter(((t1.msg = '') OR (t1.id = 2))) +----------filter(OR[(t1.msg = ''),(t1.id = 2)]) ------------PhysicalOlapScan[t1] diff --git a/regression-test/data/nereids_rules_p0/infer_predicate/extend_infer_equal_predicate.out b/regression-test/data/nereids_rules_p0/infer_predicate/extend_infer_equal_predicate.out index 79f3ad0b7939ea..ed43d254b5063f 100644 --- a/regression-test/data/nereids_rules_p0/infer_predicate/extend_infer_equal_predicate.out +++ b/regression-test/data/nereids_rules_p0/infer_predicate/extend_infer_equal_predicate.out @@ -112,14 +112,14 @@ PhysicalResultSink -- !test_or1 -- PhysicalResultSink --hashJoin[INNER_JOIN] hashCondition=((t1.a = t2.a)) otherCondition=() -----filter(((t1.a < 2) OR (t1.a > 10))) +----filter(OR[(t1.a < 2),(t1.a > 10)]) ------PhysicalOlapScan[extend_infer_t3] ----PhysicalOlapScan[extend_infer_t4] -- !test_or2 -- PhysicalResultSink --hashJoin[INNER_JOIN] hashCondition=((t1.a = t2.a)) otherCondition=() -----filter(((t1.a < 2) OR (t1.a > 10))) +----filter(OR[(t1.a < 2),(t1.a > 10)]) ------PhysicalOlapScan[extend_infer_t3] ----PhysicalOlapScan[extend_infer_t4] diff --git a/regression-test/data/nereids_rules_p0/infer_predicate/pull_up_predicate_set_op.out b/regression-test/data/nereids_rules_p0/infer_predicate/pull_up_predicate_set_op.out index 3e7889f460b585..db72758ec618fd 100644 --- a/regression-test/data/nereids_rules_p0/infer_predicate/pull_up_predicate_set_op.out +++ b/regression-test/data/nereids_rules_p0/infer_predicate/pull_up_predicate_set_op.out @@ -202,7 +202,7 @@ PhysicalResultSink ----hashAgg[GLOBAL] ------hashAgg[LOCAL] --------PhysicalUnion -----------filter((a IN (1, 2) OR b IN ('2d', '3'))) +----------filter(OR[a IN (1, 2),b IN ('2d', '3')]) ------------PhysicalOlapScan[test_pull_up_predicate_set_op1] ----PhysicalOlapScan[test_pull_up_predicate_set_op3] diff --git a/regression-test/data/nereids_rules_p0/predicate_infer/infer_predicate.out b/regression-test/data/nereids_rules_p0/predicate_infer/infer_predicate.out index 14817af2ee3200..288c30bb28c1cf 100644 --- a/regression-test/data/nereids_rules_p0/predicate_infer/infer_predicate.out +++ b/regression-test/data/nereids_rules_p0/predicate_infer/infer_predicate.out @@ -30,7 +30,7 @@ PhysicalResultSink -- !infer_predicate_full_outer_join -- PhysicalResultSink ---filter(((t1.name = 'Test') OR (t2.name = 'Test'))) +--filter(OR[(t1.name = 'Test'),(t2.name = 'Test')]) ----hashJoin[FULL_OUTER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ------PhysicalOlapScan[t] ------PhysicalOlapScan[t] @@ -128,8 +128,8 @@ PhysicalResultSink -- !infer_predicate_complex_and_or_logic -- PhysicalResultSink ---hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=(((t1.score > 80) OR ((t2.name = 'Dave') AND (t1.id < 50)))) -----filter(((t1.score > 80) OR (t1.id < 50))) +--hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=(OR[(t1.score > 80),AND[(t2.name = 'Dave'),(t1.id < 50)]]) +----filter(OR[(t1.score > 80),(t1.id < 50)]) ------PhysicalOlapScan[t] ----PhysicalOlapScan[t] diff --git a/regression-test/data/nereids_rules_p0/push_down_filter/extract_from_disjunction_in_join.out b/regression-test/data/nereids_rules_p0/push_down_filter/extract_from_disjunction_in_join.out index 898621c7da765c..1c9c25ba337c78 100644 --- a/regression-test/data/nereids_rules_p0/push_down_filter/extract_from_disjunction_in_join.out +++ b/regression-test/data/nereids_rules_p0/push_down_filter/extract_from_disjunction_in_join.out @@ -1,7 +1,7 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !left_semi -- PhysicalResultSink ---hashJoin[LEFT_SEMI_JOIN] hashCondition=((t1.b = t2.b)) otherCondition=((((t2.a = 9) AND (t1.a = 1)) OR ((t1.a = 2) AND (t2.a = 8)))) +--hashJoin[LEFT_SEMI_JOIN] hashCondition=((t1.b = t2.b)) otherCondition=(OR[AND[(t2.a = 9),(t1.a = 1)],AND[(t1.a = 2),(t2.a = 8)]]) ----filter(a IN (1, 2)) ------PhysicalOlapScan[extract_from_disjunction_in_join_t1] ----filter(a IN (8, 9)) @@ -9,7 +9,7 @@ PhysicalResultSink -- !right_semi -- PhysicalResultSink ---hashJoin[RIGHT_SEMI_JOIN] hashCondition=((t1.b = t2.b)) otherCondition=((((t2.a = 9) AND (t1.a = 1)) OR ((t1.a = 2) AND (t2.a = 8)))) +--hashJoin[RIGHT_SEMI_JOIN] hashCondition=((t1.b = t2.b)) otherCondition=(OR[AND[(t2.a = 9),(t1.a = 1)],AND[(t1.a = 2),(t2.a = 8)]]) ----filter(a IN (1, 2)) ------PhysicalOlapScan[extract_from_disjunction_in_join_t1] ----filter(a IN (8, 9)) @@ -17,35 +17,35 @@ PhysicalResultSink -- !left -- PhysicalResultSink ---hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.b = t2.b)) otherCondition=((((t2.a = 9) AND (t1.a = 1)) OR ((t1.a = 2) AND (t2.a = 8))) and a IN (1, 2)) +--hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.b = t2.b)) otherCondition=(OR[AND[(t2.a = 9),(t1.a = 1)],AND[(t1.a = 2),(t2.a = 8)]] and a IN (1, 2)) ----PhysicalOlapScan[extract_from_disjunction_in_join_t1] ----filter(a IN (8, 9)) ------PhysicalOlapScan[extract_from_disjunction_in_join_t2] -- !right -- PhysicalResultSink ---hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.b = t2.b)) otherCondition=((((t2.a = 9) AND (t1.a = 1)) OR ((t1.a = 2) AND (t2.a = 8))) and a IN (8, 9)) +--hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.b = t2.b)) otherCondition=(OR[AND[(t2.a = 9),(t1.a = 1)],AND[(t1.a = 2),(t2.a = 8)]] and a IN (8, 9)) ----filter(a IN (1, 2)) ------PhysicalOlapScan[extract_from_disjunction_in_join_t1] ----PhysicalOlapScan[extract_from_disjunction_in_join_t2] -- !left_anti -- PhysicalResultSink ---hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.b = t2.b)) otherCondition=((((t2.a = 9) AND (t1.a = 1)) OR ((t1.a = 2) AND (t2.a = 8))) and a IN (1, 2)) +--hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.b = t2.b)) otherCondition=(OR[AND[(t2.a = 9),(t1.a = 1)],AND[(t1.a = 2),(t2.a = 8)]] and a IN (1, 2)) ----PhysicalOlapScan[extract_from_disjunction_in_join_t1] ----filter(a IN (8, 9)) ------PhysicalOlapScan[extract_from_disjunction_in_join_t2] -- !right_anti -- PhysicalResultSink ---hashJoin[RIGHT_ANTI_JOIN] hashCondition=((t1.b = t2.b)) otherCondition=((((t2.a = 9) AND (t1.a = 1)) OR ((t1.a = 2) AND (t2.a = 8))) and a IN (8, 9)) +--hashJoin[RIGHT_ANTI_JOIN] hashCondition=((t1.b = t2.b)) otherCondition=(OR[AND[(t2.a = 9),(t1.a = 1)],AND[(t1.a = 2),(t2.a = 8)]] and a IN (8, 9)) ----filter(a IN (1, 2)) ------PhysicalOlapScan[extract_from_disjunction_in_join_t1] ----PhysicalOlapScan[extract_from_disjunction_in_join_t2] -- !inner -- PhysicalResultSink ---hashJoin[INNER_JOIN] hashCondition=((t1.b = t2.b)) otherCondition=((((t2.a = 9) AND (t1.a = 1)) OR ((t1.a = 2) AND (t2.a = 8)))) +--hashJoin[INNER_JOIN] hashCondition=((t1.b = t2.b)) otherCondition=(OR[AND[(t2.a = 9),(t1.a = 1)],AND[(t1.a = 2),(t2.a = 8)]]) ----filter(a IN (1, 2)) ------PhysicalOlapScan[extract_from_disjunction_in_join_t1] ----filter(a IN (8, 9)) @@ -53,7 +53,7 @@ PhysicalResultSink -- !outer -- PhysicalResultSink ---hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.b = t2.b)) otherCondition=((((t2.a = 9) AND (t1.a = 1)) OR ((t1.a = 2) AND (t2.a = 8))) and a IN (1, 2)) +--hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.b = t2.b)) otherCondition=(OR[AND[(t2.a = 9),(t1.a = 1)],AND[(t1.a = 2),(t2.a = 8)]] and a IN (1, 2)) ----filter((t1.c = 3)) ------PhysicalOlapScan[extract_from_disjunction_in_join_t1] ----filter(a IN (8, 9)) diff --git a/regression-test/data/nereids_rules_p0/push_down_filter/push_down_filter_through_window.out b/regression-test/data/nereids_rules_p0/push_down_filter/push_down_filter_through_window.out index 9ecb96e1fc0b74..a200972a0fa5a3 100644 --- a/regression-test/data/nereids_rules_p0/push_down_filter/push_down_filter_through_window.out +++ b/regression-test/data/nereids_rules_p0/push_down_filter/push_down_filter_through_window.out @@ -20,7 +20,7 @@ PhysicalResultSink ------PhysicalQuickSort[LOCAL_SORT] --------PhysicalPartitionTopN ----------PhysicalProject -------------filter(((t.id > 1) OR (t.value1 > 2))) +------------filter(OR[(t.id > 1),(t.value1 > 2)]) --------------PhysicalOlapScan[push_down_multi_column_predicate_through_window_t] -- !multi_column_or_predicate_push_down_window -- diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/bs_downgrade_shape/query13.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/bs_downgrade_shape/query13.out index 461f54e0e1767c..8138a8e20ed7e5 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/bs_downgrade_shape/query13.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/bs_downgrade_shape/query13.out @@ -7,21 +7,21 @@ PhysicalResultSink --------PhysicalProject ----------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF4 s_store_sk->[ss_store_sk] ------------PhysicalProject ---------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=((((ca_state IN ('IL', 'TN', 'TX') AND ((store_sales.ss_net_profit >= 100.00) AND (store_sales.ss_net_profit <= 200.00))) OR (ca_state IN ('ID', 'OH', 'WY') AND ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 300.00)))) OR (ca_state IN ('IA', 'MS', 'SC') AND ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 250.00))))) build RFs:RF3 ss_addr_sk->[ca_address_sk] +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=(OR[AND[ca_state IN ('IL', 'TN', 'TX'),(store_sales.ss_net_profit >= 100.00),(store_sales.ss_net_profit <= 200.00)],AND[ca_state IN ('ID', 'OH', 'WY'),(store_sales.ss_net_profit >= 150.00),(store_sales.ss_net_profit <= 300.00)],AND[ca_state IN ('IA', 'MS', 'SC'),(store_sales.ss_net_profit >= 50.00),(store_sales.ss_net_profit <= 250.00)]]) build RFs:RF3 ss_addr_sk->[ca_address_sk] ----------------PhysicalProject ------------------filter((customer_address.ca_country = 'United States') and ca_state IN ('IA', 'ID', 'IL', 'MS', 'OH', 'SC', 'TN', 'TX', 'WY')) --------------------PhysicalOlapScan[customer_address] apply RFs: RF3 ----------------PhysicalProject ------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=((((((household_demographics.hd_dep_count = 1) AND cd_marital_status IN ('D', 'W')) AND cd_education_status IN ('2 yr Degree', 'Primary')) AND ((((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = 'Primary')) AND ((store_sales.ss_sales_price >= 50.00) AND (store_sales.ss_sales_price <= 100.00))) OR (((customer_demographics.cd_marital_status = 'W') AND (customer_demographics.cd_education_status = '2 yr Degree')) AND ((store_sales.ss_sales_price >= 150.00) AND (store_sales.ss_sales_price <= 200.00))))) OR ((((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = 'College')) AND ((store_sales.ss_sales_price >= 100.00) AND (store_sales.ss_sales_price <= 150.00))) AND (household_demographics.hd_dep_count = 3)))) build RFs:RF1 hd_demo_sk->[ss_hdemo_sk] +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=(OR[AND[(household_demographics.hd_dep_count = 1),cd_marital_status IN ('D', 'W'),cd_education_status IN ('2 yr Degree', 'Primary'),OR[AND[(customer_demographics.cd_marital_status = 'D'),(customer_demographics.cd_education_status = 'Primary'),(store_sales.ss_sales_price >= 50.00),(store_sales.ss_sales_price <= 100.00)],AND[(customer_demographics.cd_marital_status = 'W'),(customer_demographics.cd_education_status = '2 yr Degree'),(store_sales.ss_sales_price >= 150.00),(store_sales.ss_sales_price <= 200.00)]]],AND[(customer_demographics.cd_marital_status = 'M'),(customer_demographics.cd_education_status = 'College'),(store_sales.ss_sales_price >= 100.00),(store_sales.ss_sales_price <= 150.00),(household_demographics.hd_dep_count = 3)]]) build RFs:RF1 hd_demo_sk->[ss_hdemo_sk] ------------------------PhysicalProject --------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] ----------------------------PhysicalProject ------------------------------filter((store_sales.ss_net_profit <= 300.00) and (store_sales.ss_net_profit >= 50.00) and (store_sales.ss_sales_price <= 200.00) and (store_sales.ss_sales_price >= 50.00)) --------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF4 ----------------------------PhysicalProject -------------------------------filter(((((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = 'College')) OR ((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = 'Primary'))) OR ((customer_demographics.cd_marital_status = 'W') AND (customer_demographics.cd_education_status = '2 yr Degree'))) and cd_education_status IN ('2 yr Degree', 'College', 'Primary') and cd_marital_status IN ('D', 'M', 'W')) +------------------------------filter(OR[AND[(customer_demographics.cd_marital_status = 'M'),(customer_demographics.cd_education_status = 'College')],AND[(customer_demographics.cd_marital_status = 'D'),(customer_demographics.cd_education_status = 'Primary')],AND[(customer_demographics.cd_marital_status = 'W'),(customer_demographics.cd_education_status = '2 yr Degree')]] and cd_education_status IN ('2 yr Degree', 'College', 'Primary') and cd_marital_status IN ('D', 'M', 'W')) --------------------------------PhysicalOlapScan[customer_demographics] ------------------------PhysicalProject --------------------------filter(hd_dep_count IN (1, 3)) diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/bs_downgrade_shape/query45.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/bs_downgrade_shape/query45.out index b65fa9047c0c87..dcc2e202e23752 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/bs_downgrade_shape/query45.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/bs_downgrade_shape/query45.out @@ -8,7 +8,7 @@ PhysicalResultSink ----------PhysicalDistribute[DistributionSpecHash] ------------hashAgg[LOCAL] --------------PhysicalProject -----------------filter((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR $c$1)) +----------------filter(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),$c$1]) ------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ws_bill_customer_sk] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/bs_downgrade_shape/query61.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/bs_downgrade_shape/query61.out index 7453f3ef1a7080..e768a09ec1494d 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/bs_downgrade_shape/query61.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/bs_downgrade_shape/query61.out @@ -33,7 +33,7 @@ PhysicalResultSink ------------------------------------filter((item.i_category = 'Home')) --------------------------------------PhysicalOlapScan[item] ------------------------------PhysicalProject ---------------------------------filter((((promotion.p_channel_dmail = 'Y') OR (promotion.p_channel_email = 'Y')) OR (promotion.p_channel_tv = 'Y'))) +--------------------------------filter(OR[(promotion.p_channel_dmail = 'Y'),(promotion.p_channel_email = 'Y'),(promotion.p_channel_tv = 'Y')]) ----------------------------------PhysicalOlapScan[promotion] --------------------------PhysicalProject ----------------------------filter((store.s_gmt_offset = -7.00)) diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/bs_downgrade_shape/query68.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/bs_downgrade_shape/query68.out index f4503a600d6272..2f4fbe401f1315 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/bs_downgrade_shape/query68.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/bs_downgrade_shape/query68.out @@ -33,6 +33,6 @@ PhysicalResultSink ----------------------------------filter(s_city IN ('Fairview', 'Midway')) ------------------------------------PhysicalOlapScan[store] ----------------------------PhysicalProject -------------------------------filter(((household_demographics.hd_dep_count = 3) OR (household_demographics.hd_vehicle_count = 4))) +------------------------------filter(OR[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count = 4)]) --------------------------------PhysicalOlapScan[household_demographics] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/bs_downgrade_shape/query91.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/bs_downgrade_shape/query91.out index 624685e9dab961..9d3c77acb23ca8 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/bs_downgrade_shape/query91.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/bs_downgrade_shape/query91.out @@ -28,7 +28,7 @@ PhysicalResultSink ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 RF1 ----------------------------------------PhysicalProject -------------------------------------------filter((((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = 'Unknown')) OR ((customer_demographics.cd_marital_status = 'W') AND (customer_demographics.cd_education_status = 'Advanced Degree'))) and cd_education_status IN ('Advanced Degree', 'Unknown') and cd_marital_status IN ('M', 'W')) +------------------------------------------filter(OR[AND[(customer_demographics.cd_marital_status = 'M'),(customer_demographics.cd_education_status = 'Unknown')],AND[(customer_demographics.cd_marital_status = 'W'),(customer_demographics.cd_education_status = 'Advanced Degree')]] and cd_education_status IN ('Advanced Degree', 'Unknown') and cd_marital_status IN ('M', 'W')) --------------------------------------------PhysicalOlapScan[customer_demographics] ------------------------------------PhysicalProject --------------------------------------filter((hd_buy_potential like 'Unknown%')) diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/eliminate_empty/query10_empty.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/eliminate_empty/query10_empty.out index f68d46db52fbb8..78fd7c847c29ed 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/eliminate_empty/query10_empty.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/eliminate_empty/query10_empty.out @@ -9,7 +9,7 @@ PhysicalResultSink ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) +------------------filter(OR[ifnull($c$1, FALSE),ifnull($c$2, FALSE)]) --------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() ----------------------PhysicalProject ------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query10.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query10.out index f68d46db52fbb8..78fd7c847c29ed 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query10.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query10.out @@ -9,7 +9,7 @@ PhysicalResultSink ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) +------------------filter(OR[ifnull($c$1, FALSE),ifnull($c$2, FALSE)]) --------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() ----------------------PhysicalProject ------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query13.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query13.out index 461f54e0e1767c..8138a8e20ed7e5 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query13.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query13.out @@ -7,21 +7,21 @@ PhysicalResultSink --------PhysicalProject ----------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF4 s_store_sk->[ss_store_sk] ------------PhysicalProject ---------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=((((ca_state IN ('IL', 'TN', 'TX') AND ((store_sales.ss_net_profit >= 100.00) AND (store_sales.ss_net_profit <= 200.00))) OR (ca_state IN ('ID', 'OH', 'WY') AND ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 300.00)))) OR (ca_state IN ('IA', 'MS', 'SC') AND ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 250.00))))) build RFs:RF3 ss_addr_sk->[ca_address_sk] +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=(OR[AND[ca_state IN ('IL', 'TN', 'TX'),(store_sales.ss_net_profit >= 100.00),(store_sales.ss_net_profit <= 200.00)],AND[ca_state IN ('ID', 'OH', 'WY'),(store_sales.ss_net_profit >= 150.00),(store_sales.ss_net_profit <= 300.00)],AND[ca_state IN ('IA', 'MS', 'SC'),(store_sales.ss_net_profit >= 50.00),(store_sales.ss_net_profit <= 250.00)]]) build RFs:RF3 ss_addr_sk->[ca_address_sk] ----------------PhysicalProject ------------------filter((customer_address.ca_country = 'United States') and ca_state IN ('IA', 'ID', 'IL', 'MS', 'OH', 'SC', 'TN', 'TX', 'WY')) --------------------PhysicalOlapScan[customer_address] apply RFs: RF3 ----------------PhysicalProject ------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=((((((household_demographics.hd_dep_count = 1) AND cd_marital_status IN ('D', 'W')) AND cd_education_status IN ('2 yr Degree', 'Primary')) AND ((((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = 'Primary')) AND ((store_sales.ss_sales_price >= 50.00) AND (store_sales.ss_sales_price <= 100.00))) OR (((customer_demographics.cd_marital_status = 'W') AND (customer_demographics.cd_education_status = '2 yr Degree')) AND ((store_sales.ss_sales_price >= 150.00) AND (store_sales.ss_sales_price <= 200.00))))) OR ((((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = 'College')) AND ((store_sales.ss_sales_price >= 100.00) AND (store_sales.ss_sales_price <= 150.00))) AND (household_demographics.hd_dep_count = 3)))) build RFs:RF1 hd_demo_sk->[ss_hdemo_sk] +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=(OR[AND[(household_demographics.hd_dep_count = 1),cd_marital_status IN ('D', 'W'),cd_education_status IN ('2 yr Degree', 'Primary'),OR[AND[(customer_demographics.cd_marital_status = 'D'),(customer_demographics.cd_education_status = 'Primary'),(store_sales.ss_sales_price >= 50.00),(store_sales.ss_sales_price <= 100.00)],AND[(customer_demographics.cd_marital_status = 'W'),(customer_demographics.cd_education_status = '2 yr Degree'),(store_sales.ss_sales_price >= 150.00),(store_sales.ss_sales_price <= 200.00)]]],AND[(customer_demographics.cd_marital_status = 'M'),(customer_demographics.cd_education_status = 'College'),(store_sales.ss_sales_price >= 100.00),(store_sales.ss_sales_price <= 150.00),(household_demographics.hd_dep_count = 3)]]) build RFs:RF1 hd_demo_sk->[ss_hdemo_sk] ------------------------PhysicalProject --------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] ----------------------------PhysicalProject ------------------------------filter((store_sales.ss_net_profit <= 300.00) and (store_sales.ss_net_profit >= 50.00) and (store_sales.ss_sales_price <= 200.00) and (store_sales.ss_sales_price >= 50.00)) --------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF4 ----------------------------PhysicalProject -------------------------------filter(((((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = 'College')) OR ((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = 'Primary'))) OR ((customer_demographics.cd_marital_status = 'W') AND (customer_demographics.cd_education_status = '2 yr Degree'))) and cd_education_status IN ('2 yr Degree', 'College', 'Primary') and cd_marital_status IN ('D', 'M', 'W')) +------------------------------filter(OR[AND[(customer_demographics.cd_marital_status = 'M'),(customer_demographics.cd_education_status = 'College')],AND[(customer_demographics.cd_marital_status = 'D'),(customer_demographics.cd_education_status = 'Primary')],AND[(customer_demographics.cd_marital_status = 'W'),(customer_demographics.cd_education_status = '2 yr Degree')]] and cd_education_status IN ('2 yr Degree', 'College', 'Primary') and cd_marital_status IN ('D', 'M', 'W')) --------------------------------PhysicalOlapScan[customer_demographics] ------------------------PhysicalProject --------------------------filter(hd_dep_count IN (1, 3)) diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query15.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query15.out index 2de8417f7ab831..06c1b08293ef85 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query15.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query15.out @@ -8,7 +8,7 @@ PhysicalResultSink ----------PhysicalDistribute[DistributionSpecHash] ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=(((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR ca_state IN ('CA', 'GA', 'WA')) OR (catalog_sales.cs_sales_price > 500.00))) build RFs:RF2 c_customer_sk->[cs_bill_customer_sk] +----------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),ca_state IN ('CA', 'GA', 'WA'),(catalog_sales.cs_sales_price > 500.00)]) build RFs:RF2 c_customer_sk->[cs_bill_customer_sk] ------------------PhysicalProject --------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] ----------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query26.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query26.out index dc5aee071856c1..383242890f9dd4 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query26.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query26.out @@ -24,7 +24,7 @@ PhysicalResultSink ----------------------------filter((date_dim.d_year = 2002)) ------------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalProject -------------------------filter(((promotion.p_channel_email = 'N') OR (promotion.p_channel_event = 'N'))) +------------------------filter(OR[(promotion.p_channel_email = 'N'),(promotion.p_channel_event = 'N')]) --------------------------PhysicalOlapScan[promotion] ------------------PhysicalProject --------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query28.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query28.out index 2fdcc86b103f71..36ec7305d96abb 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query28.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query28.out @@ -17,41 +17,41 @@ PhysicalResultSink ----------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------hashAgg[LOCAL] --------------------------------PhysicalProject -----------------------------------filter(((((store_sales.ss_list_price >= 107.00) AND (store_sales.ss_list_price <= 117.00)) OR ((store_sales.ss_coupon_amt >= 1319.00) AND (store_sales.ss_coupon_amt <= 2319.00))) OR ((store_sales.ss_wholesale_cost >= 60.00) AND (store_sales.ss_wholesale_cost <= 80.00))) and (store_sales.ss_quantity <= 5) and (store_sales.ss_quantity >= 0)) +----------------------------------filter((store_sales.ss_quantity <= 5) and (store_sales.ss_quantity >= 0) and OR[AND[(store_sales.ss_list_price >= 107.00),(store_sales.ss_list_price <= 117.00)],AND[(store_sales.ss_coupon_amt >= 1319.00),(store_sales.ss_coupon_amt <= 2319.00)],AND[(store_sales.ss_wholesale_cost >= 60.00),(store_sales.ss_wholesale_cost <= 80.00)]]) ------------------------------------PhysicalOlapScan[store_sales] ------------------------PhysicalLimit[LOCAL] --------------------------hashAgg[GLOBAL] ----------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------hashAgg[LOCAL] --------------------------------PhysicalProject -----------------------------------filter(((((store_sales.ss_list_price >= 23.00) AND (store_sales.ss_list_price <= 33.00)) OR ((store_sales.ss_coupon_amt >= 825.00) AND (store_sales.ss_coupon_amt <= 1825.00))) OR ((store_sales.ss_wholesale_cost >= 43.00) AND (store_sales.ss_wholesale_cost <= 63.00))) and (store_sales.ss_quantity <= 10) and (store_sales.ss_quantity >= 6)) +----------------------------------filter((store_sales.ss_quantity <= 10) and (store_sales.ss_quantity >= 6) and OR[AND[(store_sales.ss_list_price >= 23.00),(store_sales.ss_list_price <= 33.00)],AND[(store_sales.ss_coupon_amt >= 825.00),(store_sales.ss_coupon_amt <= 1825.00)],AND[(store_sales.ss_wholesale_cost >= 43.00),(store_sales.ss_wholesale_cost <= 63.00)]]) ------------------------------------PhysicalOlapScan[store_sales] --------------------PhysicalLimit[LOCAL] ----------------------hashAgg[GLOBAL] ------------------------PhysicalDistribute[DistributionSpecGather] --------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------filter(((((store_sales.ss_list_price >= 74.00) AND (store_sales.ss_list_price <= 84.00)) OR ((store_sales.ss_coupon_amt >= 4381.00) AND (store_sales.ss_coupon_amt <= 5381.00))) OR ((store_sales.ss_wholesale_cost >= 57.00) AND (store_sales.ss_wholesale_cost <= 77.00))) and (store_sales.ss_quantity <= 15) and (store_sales.ss_quantity >= 11)) +------------------------------filter((store_sales.ss_quantity <= 15) and (store_sales.ss_quantity >= 11) and OR[AND[(store_sales.ss_list_price >= 74.00),(store_sales.ss_list_price <= 84.00)],AND[(store_sales.ss_coupon_amt >= 4381.00),(store_sales.ss_coupon_amt <= 5381.00)],AND[(store_sales.ss_wholesale_cost >= 57.00),(store_sales.ss_wholesale_cost <= 77.00)]]) --------------------------------PhysicalOlapScan[store_sales] ----------------PhysicalLimit[LOCAL] ------------------hashAgg[GLOBAL] --------------------PhysicalDistribute[DistributionSpecGather] ----------------------hashAgg[LOCAL] ------------------------PhysicalProject ---------------------------filter(((((store_sales.ss_list_price >= 89.00) AND (store_sales.ss_list_price <= 99.00)) OR ((store_sales.ss_coupon_amt >= 3117.00) AND (store_sales.ss_coupon_amt <= 4117.00))) OR ((store_sales.ss_wholesale_cost >= 68.00) AND (store_sales.ss_wholesale_cost <= 88.00))) and (store_sales.ss_quantity <= 20) and (store_sales.ss_quantity >= 16)) +--------------------------filter((store_sales.ss_quantity <= 20) and (store_sales.ss_quantity >= 16) and OR[AND[(store_sales.ss_list_price >= 89.00),(store_sales.ss_list_price <= 99.00)],AND[(store_sales.ss_coupon_amt >= 3117.00),(store_sales.ss_coupon_amt <= 4117.00)],AND[(store_sales.ss_wholesale_cost >= 68.00),(store_sales.ss_wholesale_cost <= 88.00)]]) ----------------------------PhysicalOlapScan[store_sales] ------------PhysicalLimit[LOCAL] --------------hashAgg[GLOBAL] ----------------PhysicalDistribute[DistributionSpecGather] ------------------hashAgg[LOCAL] --------------------PhysicalProject -----------------------filter(((((store_sales.ss_list_price >= 58.00) AND (store_sales.ss_list_price <= 68.00)) OR ((store_sales.ss_coupon_amt >= 9402.00) AND (store_sales.ss_coupon_amt <= 10402.00))) OR ((store_sales.ss_wholesale_cost >= 38.00) AND (store_sales.ss_wholesale_cost <= 58.00))) and (store_sales.ss_quantity <= 25) and (store_sales.ss_quantity >= 21)) +----------------------filter((store_sales.ss_quantity <= 25) and (store_sales.ss_quantity >= 21) and OR[AND[(store_sales.ss_list_price >= 58.00),(store_sales.ss_list_price <= 68.00)],AND[(store_sales.ss_coupon_amt >= 9402.00),(store_sales.ss_coupon_amt <= 10402.00)],AND[(store_sales.ss_wholesale_cost >= 38.00),(store_sales.ss_wholesale_cost <= 58.00)]]) ------------------------PhysicalOlapScan[store_sales] --------PhysicalLimit[LOCAL] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecGather] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------filter(((((store_sales.ss_list_price >= 64.00) AND (store_sales.ss_list_price <= 74.00)) OR ((store_sales.ss_coupon_amt >= 5792.00) AND (store_sales.ss_coupon_amt <= 6792.00))) OR ((store_sales.ss_wholesale_cost >= 73.00) AND (store_sales.ss_wholesale_cost <= 93.00))) and (store_sales.ss_quantity <= 30) and (store_sales.ss_quantity >= 26)) +------------------filter((store_sales.ss_quantity <= 30) and (store_sales.ss_quantity >= 26) and OR[AND[(store_sales.ss_list_price >= 64.00),(store_sales.ss_list_price <= 74.00)],AND[(store_sales.ss_coupon_amt >= 5792.00),(store_sales.ss_coupon_amt <= 6792.00)],AND[(store_sales.ss_wholesale_cost >= 73.00),(store_sales.ss_wholesale_cost <= 93.00)]]) --------------------PhysicalOlapScan[store_sales] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query34.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query34.out index 135b461a144762..ad28c2d46104d0 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query34.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query34.out @@ -24,7 +24,7 @@ PhysicalResultSink ----------------------------------filter((store.s_county = 'Williamson County')) ------------------------------------PhysicalOlapScan[store] ----------------------------PhysicalProject -------------------------------filter((((date_dim.d_dom >= 1) AND (date_dim.d_dom <= 3)) OR ((date_dim.d_dom >= 25) AND (date_dim.d_dom <= 28))) and d_year IN (2000, 2001, 2002)) +------------------------------filter(OR[AND[(date_dim.d_dom >= 1),(date_dim.d_dom <= 3)],AND[(date_dim.d_dom >= 25),(date_dim.d_dom <= 28)]] and d_year IN (2000, 2001, 2002)) --------------------------------PhysicalOlapScan[date_dim] ------------------------PhysicalProject --------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('0-500', '1001-5000')) diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query35.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query35.out index a69bca459222b5..2f4be8c2912555 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query35.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query35.out @@ -9,7 +9,7 @@ PhysicalResultSink ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) +------------------filter(OR[ifnull($c$1, FALSE),ifnull($c$2, FALSE)]) --------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() ----------------------PhysicalProject ------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF5 cd_demo_sk->[c_current_cdemo_sk] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query41.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query41.out index ea5acccd883550..0bba60d4cdac39 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query41.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query41.out @@ -18,6 +18,6 @@ PhysicalResultSink ------------------------PhysicalDistribute[DistributionSpecHash] --------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------filter((((i_color IN ('forest', 'lime', 'maroon', 'navy', 'powder', 'sky', 'slate', 'smoke') AND i_units IN ('Bunch', 'Case', 'Dozen', 'Gross', 'Lb', 'Ounce', 'Pallet', 'Pound')) AND (((((((item.i_category = 'Women') AND i_color IN ('forest', 'lime')) AND i_units IN ('Pallet', 'Pound')) AND i_size IN ('economy', 'small')) OR ((((item.i_category = 'Women') AND i_color IN ('navy', 'slate')) AND i_units IN ('Bunch', 'Gross')) AND i_size IN ('extra large', 'petite'))) OR ((((item.i_category = 'Men') AND i_color IN ('powder', 'sky')) AND i_units IN ('Dozen', 'Lb')) AND i_size IN ('N/A', 'large'))) OR ((((item.i_category = 'Men') AND i_color IN ('maroon', 'smoke')) AND i_units IN ('Case', 'Ounce')) AND i_size IN ('economy', 'small')))) OR ((i_color IN ('aquamarine', 'dark', 'firebrick', 'frosted', 'papaya', 'peach', 'plum', 'sienna') AND i_units IN ('Box', 'Bundle', 'Carton', 'Cup', 'Dram', 'Each', 'Tbl', 'Ton')) AND (((((((item.i_category = 'Women') AND i_color IN ('aquamarine', 'dark')) AND i_units IN ('Tbl', 'Ton')) AND i_size IN ('economy', 'small')) OR ((((item.i_category = 'Women') AND i_color IN ('frosted', 'plum')) AND i_units IN ('Box', 'Dram')) AND i_size IN ('extra large', 'petite'))) OR ((((item.i_category = 'Men') AND i_color IN ('papaya', 'peach')) AND i_units IN ('Bundle', 'Carton')) AND i_size IN ('N/A', 'large'))) OR ((((item.i_category = 'Men') AND i_color IN ('firebrick', 'sienna')) AND i_units IN ('Cup', 'Each')) AND i_size IN ('economy', 'small'))))) and i_category IN ('Men', 'Women') and i_size IN ('N/A', 'economy', 'extra large', 'large', 'petite', 'small')) +------------------------------filter(OR[AND[i_color IN ('forest', 'lime', 'maroon', 'navy', 'powder', 'sky', 'slate', 'smoke'),i_units IN ('Bunch', 'Case', 'Dozen', 'Gross', 'Lb', 'Ounce', 'Pallet', 'Pound'),OR[AND[(item.i_category = 'Women'),i_color IN ('forest', 'lime'),i_units IN ('Pallet', 'Pound'),i_size IN ('economy', 'small')],AND[(item.i_category = 'Women'),i_color IN ('navy', 'slate'),i_units IN ('Bunch', 'Gross'),i_size IN ('extra large', 'petite')],AND[(item.i_category = 'Men'),i_color IN ('powder', 'sky'),i_units IN ('Dozen', 'Lb'),i_size IN ('N/A', 'large')],AND[(item.i_category = 'Men'),i_color IN ('maroon', 'smoke'),i_units IN ('Case', 'Ounce'),i_size IN ('economy', 'small')]]],AND[i_color IN ('aquamarine', 'dark', 'firebrick', 'frosted', 'papaya', 'peach', 'plum', 'sienna'),i_units IN ('Box', 'Bundle', 'Carton', 'Cup', 'Dram', 'Each', 'Tbl', 'Ton'),OR[AND[(item.i_category = 'Women'),i_color IN ('aquamarine', 'dark'),i_units IN ('Tbl', 'Ton'),i_size IN ('economy', 'small')],AND[(item.i_category = 'Women'),i_color IN ('frosted', 'plum'),i_units IN ('Box', 'Dram'),i_size IN ('extra large', 'petite')],AND[(item.i_category = 'Men'),i_color IN ('papaya', 'peach'),i_units IN ('Bundle', 'Carton'),i_size IN ('N/A', 'large')],AND[(item.i_category = 'Men'),i_color IN ('firebrick', 'sienna'),i_units IN ('Cup', 'Each'),i_size IN ('economy', 'small')]]]] and i_category IN ('Men', 'Women') and i_size IN ('N/A', 'economy', 'extra large', 'large', 'petite', 'small')) --------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query45.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query45.out index b65fa9047c0c87..dcc2e202e23752 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query45.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query45.out @@ -8,7 +8,7 @@ PhysicalResultSink ----------PhysicalDistribute[DistributionSpecHash] ------------hashAgg[LOCAL] --------------PhysicalProject -----------------filter((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR $c$1)) +----------------filter(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),$c$1]) ------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ws_bill_customer_sk] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query46.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query46.out index e84100fe65fac8..cb7d73da5838ed 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query46.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query46.out @@ -33,6 +33,6 @@ PhysicalResultSink ----------------------------------filter(d_dow IN (0, 6) and d_year IN (2000, 2001, 2002)) ------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalProject -------------------------------filter(((household_demographics.hd_dep_count = 8) OR (household_demographics.hd_vehicle_count = 0))) +------------------------------filter(OR[(household_demographics.hd_dep_count = 8),(household_demographics.hd_vehicle_count = 0)]) --------------------------------PhysicalOlapScan[household_demographics] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query47.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query47.out index d51d48c5ab5677..0e9f713243773a 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query47.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query47.out @@ -21,7 +21,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------------------------PhysicalProject --------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 ------------------------------------PhysicalProject ---------------------------------------filter((((date_dim.d_year = 2000) OR ((date_dim.d_year = 1999) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2001) AND (date_dim.d_moy = 1))) and d_year IN (1999, 2000, 2001)) +--------------------------------------filter(OR[(date_dim.d_year = 2000),AND[(date_dim.d_year = 1999),(date_dim.d_moy = 12)],AND[(date_dim.d_year = 2001),(date_dim.d_moy = 1)]] and d_year IN (1999, 2000, 2001)) ----------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query48.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query48.out index 3fcfef0d8f68ba..6ed75b1e21eca8 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query48.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query48.out @@ -9,14 +9,14 @@ PhysicalResultSink ------------PhysicalProject --------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=((((ca_state IN ('ND', 'NY', 'SD') AND ((store_sales.ss_net_profit >= 0.00) AND (store_sales.ss_net_profit <= 2000.00))) OR (ca_state IN ('GA', 'KS', 'MD') AND ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 3000.00)))) OR (ca_state IN ('CO', 'MN', 'NC') AND ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 25000.00))))) build RFs:RF1 ca_address_sk->[ss_addr_sk] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=(OR[AND[ca_state IN ('ND', 'NY', 'SD'),(store_sales.ss_net_profit >= 0.00),(store_sales.ss_net_profit <= 2000.00)],AND[ca_state IN ('GA', 'KS', 'MD'),(store_sales.ss_net_profit >= 150.00),(store_sales.ss_net_profit <= 3000.00)],AND[ca_state IN ('CO', 'MN', 'NC'),(store_sales.ss_net_profit >= 50.00),(store_sales.ss_net_profit <= 25000.00)]]) build RFs:RF1 ca_address_sk->[ss_addr_sk] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)) otherCondition=((((((customer_demographics.cd_marital_status = 'S') AND (customer_demographics.cd_education_status = 'Secondary')) AND ((store_sales.ss_sales_price >= 100.00) AND (store_sales.ss_sales_price <= 150.00))) OR (((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = '2 yr Degree')) AND ((store_sales.ss_sales_price >= 50.00) AND (store_sales.ss_sales_price <= 100.00)))) OR (((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = 'Advanced Degree')) AND ((store_sales.ss_sales_price >= 150.00) AND (store_sales.ss_sales_price <= 200.00))))) build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)) otherCondition=(OR[AND[(customer_demographics.cd_marital_status = 'S'),(customer_demographics.cd_education_status = 'Secondary'),(store_sales.ss_sales_price >= 100.00),(store_sales.ss_sales_price <= 150.00)],AND[(customer_demographics.cd_marital_status = 'M'),(customer_demographics.cd_education_status = '2 yr Degree'),(store_sales.ss_sales_price >= 50.00),(store_sales.ss_sales_price <= 100.00)],AND[(customer_demographics.cd_marital_status = 'D'),(customer_demographics.cd_education_status = 'Advanced Degree'),(store_sales.ss_sales_price >= 150.00),(store_sales.ss_sales_price <= 200.00)]]) build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] ------------------------PhysicalProject --------------------------filter((store_sales.ss_net_profit <= 25000.00) and (store_sales.ss_net_profit >= 0.00) and (store_sales.ss_sales_price <= 200.00) and (store_sales.ss_sales_price >= 50.00)) ----------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 ------------------------PhysicalProject ---------------------------filter(((((customer_demographics.cd_marital_status = 'S') AND (customer_demographics.cd_education_status = 'Secondary')) OR ((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = '2 yr Degree'))) OR ((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = 'Advanced Degree'))) and cd_education_status IN ('2 yr Degree', 'Advanced Degree', 'Secondary') and cd_marital_status IN ('D', 'M', 'S')) +--------------------------filter(OR[AND[(customer_demographics.cd_marital_status = 'S'),(customer_demographics.cd_education_status = 'Secondary')],AND[(customer_demographics.cd_marital_status = 'M'),(customer_demographics.cd_education_status = '2 yr Degree')],AND[(customer_demographics.cd_marital_status = 'D'),(customer_demographics.cd_education_status = 'Advanced Degree')]] and cd_education_status IN ('2 yr Degree', 'Advanced Degree', 'Secondary') and cd_marital_status IN ('D', 'M', 'S')) ----------------------------PhysicalOlapScan[customer_demographics] --------------------PhysicalProject ----------------------filter((customer_address.ca_country = 'United States') and ca_state IN ('CO', 'GA', 'KS', 'MD', 'MN', 'NC', 'ND', 'NY', 'SD')) diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query49.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query49.out index 889d5069ff9751..8b807baf5409e4 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query49.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query49.out @@ -16,7 +16,7 @@ PhysicalResultSink --------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) ----------------------------------PhysicalWindow ------------------------------------PhysicalQuickSort[LOCAL_SORT] --------------------------------------PhysicalWindow @@ -48,7 +48,7 @@ PhysicalResultSink --------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) ----------------------------------PhysicalWindow ------------------------------------PhysicalQuickSort[LOCAL_SORT] --------------------------------------PhysicalWindow @@ -80,7 +80,7 @@ PhysicalResultSink --------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) ----------------------------------PhysicalWindow ------------------------------------PhysicalQuickSort[LOCAL_SORT] --------------------------------------PhysicalWindow diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query53.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query53.out index 07d64a35b071ca..d2467a65e93e09 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query53.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query53.out @@ -22,7 +22,7 @@ PhysicalResultSink --------------------------------------PhysicalProject ----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 --------------------------------------PhysicalProject -----------------------------------------filter((((i_category IN ('Books', 'Children', 'Electronics') AND i_class IN ('personal', 'portable', 'reference', 'self-help')) AND i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')) OR ((i_category IN ('Men', 'Music', 'Women') AND i_class IN ('accessories', 'classical', 'fragrances', 'pants')) AND i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1'))) and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) +----------------------------------------filter(OR[AND[i_category IN ('Books', 'Children', 'Electronics'),i_class IN ('personal', 'portable', 'reference', 'self-help'),i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')],AND[i_category IN ('Men', 'Music', 'Women'),i_class IN ('accessories', 'classical', 'fragrances', 'pants'),i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')]] and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) ------------------------------------------PhysicalOlapScan[item] ----------------------------------PhysicalProject ------------------------------------filter(d_month_seq IN (1186, 1187, 1188, 1189, 1190, 1191, 1192, 1193, 1194, 1195, 1196, 1197)) diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query57.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query57.out index c1b802ebfc4498..2e8174812f69ea 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query57.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query57.out @@ -21,7 +21,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------------------------PhysicalProject --------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 ------------------------------------PhysicalProject ---------------------------------------filter((((date_dim.d_year = 2001) OR ((date_dim.d_year = 2000) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2002) AND (date_dim.d_moy = 1))) and d_year IN (2000, 2001, 2002)) +--------------------------------------filter(OR[(date_dim.d_year = 2001),AND[(date_dim.d_year = 2000),(date_dim.d_moy = 12)],AND[(date_dim.d_year = 2002),(date_dim.d_moy = 1)]] and d_year IN (2000, 2001, 2002)) ----------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[call_center] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query61.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query61.out index 7453f3ef1a7080..e768a09ec1494d 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query61.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query61.out @@ -33,7 +33,7 @@ PhysicalResultSink ------------------------------------filter((item.i_category = 'Home')) --------------------------------------PhysicalOlapScan[item] ------------------------------PhysicalProject ---------------------------------filter((((promotion.p_channel_dmail = 'Y') OR (promotion.p_channel_email = 'Y')) OR (promotion.p_channel_tv = 'Y'))) +--------------------------------filter(OR[(promotion.p_channel_dmail = 'Y'),(promotion.p_channel_email = 'Y'),(promotion.p_channel_tv = 'Y')]) ----------------------------------PhysicalOlapScan[promotion] --------------------------PhysicalProject ----------------------------filter((store.s_gmt_offset = -7.00)) diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query63.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query63.out index d1807a29db2f0d..bbbb80bc4b68e0 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query63.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query63.out @@ -22,7 +22,7 @@ PhysicalResultSink --------------------------------------PhysicalProject ----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 --------------------------------------PhysicalProject -----------------------------------------filter((((i_category IN ('Books', 'Children', 'Electronics') AND i_class IN ('personal', 'portable', 'reference', 'self-help')) AND i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')) OR ((i_category IN ('Men', 'Music', 'Women') AND i_class IN ('accessories', 'classical', 'fragrances', 'pants')) AND i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1'))) and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) +----------------------------------------filter(OR[AND[i_category IN ('Books', 'Children', 'Electronics'),i_class IN ('personal', 'portable', 'reference', 'self-help'),i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')],AND[i_category IN ('Men', 'Music', 'Women'),i_class IN ('accessories', 'classical', 'fragrances', 'pants'),i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')]] and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) ------------------------------------------PhysicalOlapScan[item] ----------------------------------PhysicalProject ------------------------------------filter(d_month_seq IN (1222, 1223, 1224, 1225, 1226, 1227, 1228, 1229, 1230, 1231, 1232, 1233)) diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query68.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query68.out index f4503a600d6272..2f4fbe401f1315 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query68.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query68.out @@ -33,6 +33,6 @@ PhysicalResultSink ----------------------------------filter(s_city IN ('Fairview', 'Midway')) ------------------------------------PhysicalOlapScan[store] ----------------------------PhysicalProject -------------------------------filter(((household_demographics.hd_dep_count = 3) OR (household_demographics.hd_vehicle_count = 4))) +------------------------------filter(OR[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count = 4)]) --------------------------------PhysicalOlapScan[household_demographics] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query7.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query7.out index aadb31a78e012f..2d63af9e61b19e 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query7.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query7.out @@ -24,7 +24,7 @@ PhysicalResultSink ----------------------------filter((date_dim.d_year = 2001)) ------------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalProject -------------------------filter(((promotion.p_channel_email = 'N') OR (promotion.p_channel_event = 'N'))) +------------------------filter(OR[(promotion.p_channel_email = 'N'),(promotion.p_channel_event = 'N')]) --------------------------PhysicalOlapScan[promotion] ------------------PhysicalProject --------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query78.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query78.out index 3b2d980a6ac2c7..0663ee2198a5fe 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query78.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query78.out @@ -5,7 +5,7 @@ PhysicalResultSink ----PhysicalDistribute[DistributionSpecGather] ------PhysicalTopN[LOCAL_SORT] --------PhysicalProject -----------filter(((coalesce(ws_qty, 0) > 0) OR (coalesce(cs_qty, 0) > 0))) +----------filter(OR[(coalesce(ws_qty, 0) > 0),(coalesce(cs_qty, 0) > 0)]) ------------hashJoin[LEFT_OUTER_JOIN colocated] hashCondition=((cs.cs_customer_sk = ss.ss_customer_sk) and (cs.cs_item_sk = ss.ss_item_sk) and (cs.cs_sold_year = ss.ss_sold_year)) otherCondition=() --------------PhysicalProject ----------------hashJoin[LEFT_OUTER_JOIN colocated] hashCondition=((ws.ws_customer_sk = ss.ss_customer_sk) and (ws.ws_item_sk = ss.ss_item_sk) and (ws.ws_sold_year = ss.ss_sold_year)) otherCondition=() diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query79.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query79.out index b8b18b2657766b..e3d8f3af326d91 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query79.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query79.out @@ -22,7 +22,7 @@ PhysicalResultSink ----------------------------------filter((date_dim.d_dow = 1) and d_year IN (2000, 2001, 2002)) ------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalProject -------------------------------filter(((household_demographics.hd_dep_count = 7) OR (household_demographics.hd_vehicle_count > -1))) +------------------------------filter(OR[(household_demographics.hd_dep_count = 7),(household_demographics.hd_vehicle_count > -1)]) --------------------------------PhysicalOlapScan[household_demographics] ------------------------PhysicalProject --------------------------filter((store.s_number_employees <= 295) and (store.s_number_employees >= 200)) diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query85.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query85.out index 1781cd24d5b9f5..c734d7945f3a56 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query85.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query85.out @@ -18,12 +18,12 @@ PhysicalResultSink ------------------------PhysicalProject --------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_web_page_sk = web_page.wp_web_page_sk)) otherCondition=() build RFs:RF5 wp_web_page_sk->[ws_web_page_sk] ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((customer_address.ca_address_sk = web_returns.wr_refunded_addr_sk)) otherCondition=((((ca_state IN ('IA', 'NC', 'TX') AND ((web_sales.ws_net_profit >= 100.00) AND (web_sales.ws_net_profit <= 200.00))) OR (ca_state IN ('GA', 'WI', 'WV') AND ((web_sales.ws_net_profit >= 150.00) AND (web_sales.ws_net_profit <= 300.00)))) OR (ca_state IN ('KY', 'OK', 'VA') AND ((web_sales.ws_net_profit >= 50.00) AND (web_sales.ws_net_profit <= 250.00))))) build RFs:RF4 wr_refunded_addr_sk->[ca_address_sk] +------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((customer_address.ca_address_sk = web_returns.wr_refunded_addr_sk)) otherCondition=(OR[AND[ca_state IN ('IA', 'NC', 'TX'),(web_sales.ws_net_profit >= 100.00),(web_sales.ws_net_profit <= 200.00)],AND[ca_state IN ('GA', 'WI', 'WV'),(web_sales.ws_net_profit >= 150.00),(web_sales.ws_net_profit <= 300.00)],AND[ca_state IN ('KY', 'OK', 'VA'),(web_sales.ws_net_profit >= 50.00),(web_sales.ws_net_profit <= 250.00)]]) build RFs:RF4 wr_refunded_addr_sk->[ca_address_sk] --------------------------------PhysicalProject ----------------------------------filter((customer_address.ca_country = 'United States') and ca_state IN ('GA', 'IA', 'KY', 'NC', 'OK', 'TX', 'VA', 'WI', 'WV')) ------------------------------------PhysicalOlapScan[customer_address] apply RFs: RF4 --------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cd1.cd_demo_sk = web_returns.wr_refunded_cdemo_sk)) otherCondition=((((((cd1.cd_marital_status = 'D') AND (cd1.cd_education_status = 'Primary')) AND ((web_sales.ws_sales_price >= 100.00) AND (web_sales.ws_sales_price <= 150.00))) OR (((cd1.cd_marital_status = 'S') AND (cd1.cd_education_status = 'College')) AND ((web_sales.ws_sales_price >= 50.00) AND (web_sales.ws_sales_price <= 100.00)))) OR (((cd1.cd_marital_status = 'U') AND (cd1.cd_education_status = 'Advanced Degree')) AND ((web_sales.ws_sales_price >= 150.00) AND (web_sales.ws_sales_price <= 200.00))))) build RFs:RF3 cd_demo_sk->[wr_refunded_cdemo_sk] +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cd1.cd_demo_sk = web_returns.wr_refunded_cdemo_sk)) otherCondition=(OR[AND[(cd1.cd_marital_status = 'D'),(cd1.cd_education_status = 'Primary'),(web_sales.ws_sales_price >= 100.00),(web_sales.ws_sales_price <= 150.00)],AND[(cd1.cd_marital_status = 'S'),(cd1.cd_education_status = 'College'),(web_sales.ws_sales_price >= 50.00),(web_sales.ws_sales_price <= 100.00)],AND[(cd1.cd_marital_status = 'U'),(cd1.cd_education_status = 'Advanced Degree'),(web_sales.ws_sales_price >= 150.00),(web_sales.ws_sales_price <= 200.00)]]) build RFs:RF3 cd_demo_sk->[wr_refunded_cdemo_sk] ------------------------------------PhysicalProject --------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((web_sales.ws_item_sk = web_returns.wr_item_sk) and (web_sales.ws_order_number = web_returns.wr_order_number)) otherCondition=() build RFs:RF1 ws_item_sk->[wr_item_sk];RF2 ws_order_number->[wr_order_number] ----------------------------------------PhysicalProject @@ -37,7 +37,7 @@ PhysicalResultSink ----------------------------------------------filter((date_dim.d_year = 1998)) ------------------------------------------------PhysicalOlapScan[date_dim] ------------------------------------PhysicalProject ---------------------------------------filter(((((cd1.cd_marital_status = 'D') AND (cd1.cd_education_status = 'Primary')) OR ((cd1.cd_marital_status = 'S') AND (cd1.cd_education_status = 'College'))) OR ((cd1.cd_marital_status = 'U') AND (cd1.cd_education_status = 'Advanced Degree'))) and cd_education_status IN ('Advanced Degree', 'College', 'Primary') and cd_marital_status IN ('D', 'S', 'U')) +--------------------------------------filter(OR[AND[(cd1.cd_marital_status = 'D'),(cd1.cd_education_status = 'Primary')],AND[(cd1.cd_marital_status = 'S'),(cd1.cd_education_status = 'College')],AND[(cd1.cd_marital_status = 'U'),(cd1.cd_education_status = 'Advanced Degree')]] and cd_education_status IN ('Advanced Degree', 'College', 'Primary') and cd_marital_status IN ('D', 'S', 'U')) ----------------------------------------PhysicalOlapScan[customer_demographics] ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[web_page] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query88.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query88.out index 3c8f0335f1f415..d016dca48db832 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query88.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query88.out @@ -23,7 +23,7 @@ PhysicalResultSink ------------------------------------filter((time_dim.t_hour = 8) and (time_dim.t_minute >= 30)) --------------------------------------PhysicalOlapScan[time_dim] ------------------------------PhysicalProject ---------------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3)) +--------------------------------filter(OR[AND[(household_demographics.hd_dep_count = 0),(household_demographics.hd_vehicle_count <= 2)],AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 0, 3)) ----------------------------------PhysicalOlapScan[household_demographics] --------------------------PhysicalProject ----------------------------filter((store.s_store_name = 'ese')) @@ -43,7 +43,7 @@ PhysicalResultSink ------------------------------------filter((time_dim.t_hour = 9) and (time_dim.t_minute < 30)) --------------------------------------PhysicalOlapScan[time_dim] ------------------------------PhysicalProject ---------------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3)) +--------------------------------filter(OR[AND[(household_demographics.hd_dep_count = 0),(household_demographics.hd_vehicle_count <= 2)],AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 0, 3)) ----------------------------------PhysicalOlapScan[household_demographics] --------------------------PhysicalProject ----------------------------filter((store.s_store_name = 'ese')) @@ -63,7 +63,7 @@ PhysicalResultSink ----------------------------------filter((time_dim.t_hour = 9) and (time_dim.t_minute >= 30)) ------------------------------------PhysicalOlapScan[time_dim] ----------------------------PhysicalProject -------------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3)) +------------------------------filter(OR[AND[(household_demographics.hd_dep_count = 0),(household_demographics.hd_vehicle_count <= 2)],AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 0, 3)) --------------------------------PhysicalOlapScan[household_demographics] ------------------------PhysicalProject --------------------------filter((store.s_store_name = 'ese')) @@ -83,7 +83,7 @@ PhysicalResultSink --------------------------------filter((time_dim.t_hour = 10) and (time_dim.t_minute < 30)) ----------------------------------PhysicalOlapScan[time_dim] --------------------------PhysicalProject -----------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3)) +----------------------------filter(OR[AND[(household_demographics.hd_dep_count = 0),(household_demographics.hd_vehicle_count <= 2)],AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 0, 3)) ------------------------------PhysicalOlapScan[household_demographics] ----------------------PhysicalProject ------------------------filter((store.s_store_name = 'ese')) @@ -103,7 +103,7 @@ PhysicalResultSink ------------------------------filter((time_dim.t_hour = 10) and (time_dim.t_minute >= 30)) --------------------------------PhysicalOlapScan[time_dim] ------------------------PhysicalProject ---------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3)) +--------------------------filter(OR[AND[(household_demographics.hd_dep_count = 0),(household_demographics.hd_vehicle_count <= 2)],AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 0, 3)) ----------------------------PhysicalOlapScan[household_demographics] --------------------PhysicalProject ----------------------filter((store.s_store_name = 'ese')) @@ -123,7 +123,7 @@ PhysicalResultSink ----------------------------filter((time_dim.t_hour = 11) and (time_dim.t_minute < 30)) ------------------------------PhysicalOlapScan[time_dim] ----------------------PhysicalProject -------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3)) +------------------------filter(OR[AND[(household_demographics.hd_dep_count = 0),(household_demographics.hd_vehicle_count <= 2)],AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 0, 3)) --------------------------PhysicalOlapScan[household_demographics] ------------------PhysicalProject --------------------filter((store.s_store_name = 'ese')) @@ -143,7 +143,7 @@ PhysicalResultSink --------------------------filter((time_dim.t_hour = 11) and (time_dim.t_minute >= 30)) ----------------------------PhysicalOlapScan[time_dim] --------------------PhysicalProject -----------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3)) +----------------------filter(OR[AND[(household_demographics.hd_dep_count = 0),(household_demographics.hd_vehicle_count <= 2)],AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 0, 3)) ------------------------PhysicalOlapScan[household_demographics] ----------------PhysicalProject ------------------filter((store.s_store_name = 'ese')) @@ -163,7 +163,7 @@ PhysicalResultSink ------------------------filter((time_dim.t_hour = 12) and (time_dim.t_minute < 30)) --------------------------PhysicalOlapScan[time_dim] ------------------PhysicalProject ---------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3)) +--------------------filter(OR[AND[(household_demographics.hd_dep_count = 0),(household_demographics.hd_vehicle_count <= 2)],AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 0, 3)) ----------------------PhysicalOlapScan[household_demographics] --------------PhysicalProject ----------------filter((store.s_store_name = 'ese')) diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query89.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query89.out index 786de940dad5d8..b8751687a0ff29 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query89.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query89.out @@ -23,7 +23,7 @@ PhysicalResultSink ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 ----------------------------------------PhysicalProject -------------------------------------------filter(((i_category IN ('Books', 'Children', 'Electronics') AND i_class IN ('audio', 'history', 'school-uniforms')) OR (i_category IN ('Men', 'Shoes', 'Sports') AND i_class IN ('pants', 'tennis', 'womens'))) and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Shoes', 'Sports') and i_class IN ('audio', 'history', 'pants', 'school-uniforms', 'tennis', 'womens')) +------------------------------------------filter(OR[AND[i_category IN ('Books', 'Children', 'Electronics'),i_class IN ('audio', 'history', 'school-uniforms')],AND[i_category IN ('Men', 'Shoes', 'Sports'),i_class IN ('pants', 'tennis', 'womens')]] and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Shoes', 'Sports') and i_class IN ('audio', 'history', 'pants', 'school-uniforms', 'tennis', 'womens')) --------------------------------------------PhysicalOlapScan[item] ------------------------------------PhysicalProject --------------------------------------filter((date_dim.d_year = 2001)) diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query91.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query91.out index 624685e9dab961..9d3c77acb23ca8 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query91.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query91.out @@ -28,7 +28,7 @@ PhysicalResultSink ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 RF1 ----------------------------------------PhysicalProject -------------------------------------------filter((((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = 'Unknown')) OR ((customer_demographics.cd_marital_status = 'W') AND (customer_demographics.cd_education_status = 'Advanced Degree'))) and cd_education_status IN ('Advanced Degree', 'Unknown') and cd_marital_status IN ('M', 'W')) +------------------------------------------filter(OR[AND[(customer_demographics.cd_marital_status = 'M'),(customer_demographics.cd_education_status = 'Unknown')],AND[(customer_demographics.cd_marital_status = 'W'),(customer_demographics.cd_education_status = 'Advanced Degree')]] and cd_education_status IN ('Advanced Degree', 'Unknown') and cd_marital_status IN ('M', 'W')) --------------------------------------------PhysicalOlapScan[customer_demographics] ------------------------------------PhysicalProject --------------------------------------filter((hd_buy_potential like 'Unknown%')) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query10.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query10.out index bc11b07f3a3809..5fb40519b131d5 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query10.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query10.out @@ -9,7 +9,7 @@ PhysicalResultSink ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) +------------------filter(OR[ifnull($c$1, FALSE),ifnull($c$2, FALSE)]) --------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() ----------------------PhysicalProject ------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF5 ca_address_sk->[c_current_addr_sk] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query13.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query13.out index 6df93fe3046d5c..9c7d9382777bee 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query13.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query13.out @@ -7,9 +7,9 @@ PhysicalResultSink --------PhysicalProject ----------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] ------------PhysicalProject ---------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=((((ca_state IN ('KS', 'MI', 'SD') AND ((store_sales.ss_net_profit >= 100.00) AND (store_sales.ss_net_profit <= 200.00))) OR (ca_state IN ('CO', 'MO', 'ND') AND ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 300.00)))) OR (ca_state IN ('NH', 'OH', 'TX') AND ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 250.00))))) build RFs:RF3 ca_address_sk->[ss_addr_sk] +--------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=(OR[AND[ca_state IN ('KS', 'MI', 'SD'),(store_sales.ss_net_profit >= 100.00),(store_sales.ss_net_profit <= 200.00)],AND[ca_state IN ('CO', 'MO', 'ND'),(store_sales.ss_net_profit >= 150.00),(store_sales.ss_net_profit <= 300.00)],AND[ca_state IN ('NH', 'OH', 'TX'),(store_sales.ss_net_profit >= 50.00),(store_sales.ss_net_profit <= 250.00)]]) build RFs:RF3 ca_address_sk->[ss_addr_sk] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=((((((household_demographics.hd_dep_count = 1) AND cd_marital_status IN ('M', 'S')) AND cd_education_status IN ('4 yr Degree', 'College')) AND ((((customer_demographics.cd_marital_status = 'S') AND (customer_demographics.cd_education_status = 'College')) AND ((store_sales.ss_sales_price >= 50.00) AND (store_sales.ss_sales_price <= 100.00))) OR (((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = '4 yr Degree')) AND ((store_sales.ss_sales_price >= 150.00) AND (store_sales.ss_sales_price <= 200.00))))) OR ((((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = 'Unknown')) AND ((store_sales.ss_sales_price >= 100.00) AND (store_sales.ss_sales_price <= 150.00))) AND (household_demographics.hd_dep_count = 3)))) build RFs:RF2 hd_demo_sk->[ss_hdemo_sk] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=(OR[AND[(household_demographics.hd_dep_count = 1),cd_marital_status IN ('M', 'S'),cd_education_status IN ('4 yr Degree', 'College'),OR[AND[(customer_demographics.cd_marital_status = 'S'),(customer_demographics.cd_education_status = 'College'),(store_sales.ss_sales_price >= 50.00),(store_sales.ss_sales_price <= 100.00)],AND[(customer_demographics.cd_marital_status = 'M'),(customer_demographics.cd_education_status = '4 yr Degree'),(store_sales.ss_sales_price >= 150.00),(store_sales.ss_sales_price <= 200.00)]]],AND[(customer_demographics.cd_marital_status = 'D'),(customer_demographics.cd_education_status = 'Unknown'),(store_sales.ss_sales_price >= 100.00),(store_sales.ss_sales_price <= 150.00),(household_demographics.hd_dep_count = 3)]]) build RFs:RF2 hd_demo_sk->[ss_hdemo_sk] --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)) otherCondition=() build RFs:RF1 cd_demo_sk->[ss_cdemo_sk] ------------------------PhysicalProject @@ -20,7 +20,7 @@ PhysicalResultSink ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[store] ------------------------PhysicalProject ---------------------------filter(((((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = 'Unknown')) OR ((customer_demographics.cd_marital_status = 'S') AND (customer_demographics.cd_education_status = 'College'))) OR ((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = '4 yr Degree'))) and cd_education_status IN ('4 yr Degree', 'College', 'Unknown') and cd_marital_status IN ('D', 'M', 'S')) +--------------------------filter(OR[AND[(customer_demographics.cd_marital_status = 'D'),(customer_demographics.cd_education_status = 'Unknown')],AND[(customer_demographics.cd_marital_status = 'S'),(customer_demographics.cd_education_status = 'College')],AND[(customer_demographics.cd_marital_status = 'M'),(customer_demographics.cd_education_status = '4 yr Degree')]] and cd_education_status IN ('4 yr Degree', 'College', 'Unknown') and cd_marital_status IN ('D', 'M', 'S')) ----------------------------PhysicalOlapScan[customer_demographics] --------------------PhysicalProject ----------------------filter(hd_dep_count IN (1, 3)) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query15.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query15.out index 7b0fd9b10fd861..5825559155b8e7 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query15.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query15.out @@ -10,7 +10,7 @@ PhysicalResultSink --------------PhysicalProject ----------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=(((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR ca_state IN ('CA', 'GA', 'WA')) OR (catalog_sales.cs_sales_price > 500.00))) +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),ca_state IN ('CA', 'GA', 'WA'),(catalog_sales.cs_sales_price > 500.00)]) ----------------------PhysicalProject ------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() --------------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query26.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query26.out index 3762fabaeead07..52f628f8b600a2 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query26.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query26.out @@ -26,6 +26,6 @@ PhysicalResultSink ------------------------filter((date_dim.d_year = 2001)) --------------------------PhysicalOlapScan[date_dim] ------------------PhysicalProject ---------------------filter(((promotion.p_channel_email = 'N') OR (promotion.p_channel_event = 'N'))) +--------------------filter(OR[(promotion.p_channel_email = 'N'),(promotion.p_channel_event = 'N')]) ----------------------PhysicalOlapScan[promotion] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query28.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query28.out index dae7cb1c23a402..7a6bdd8868ef00 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query28.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query28.out @@ -17,41 +17,41 @@ PhysicalResultSink ----------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------hashAgg[LOCAL] --------------------------------PhysicalProject -----------------------------------filter(((((store_sales.ss_list_price >= 131.00) AND (store_sales.ss_list_price <= 141.00)) OR ((store_sales.ss_coupon_amt >= 16798.00) AND (store_sales.ss_coupon_amt <= 17798.00))) OR ((store_sales.ss_wholesale_cost >= 25.00) AND (store_sales.ss_wholesale_cost <= 45.00))) and (store_sales.ss_quantity <= 5) and (store_sales.ss_quantity >= 0)) +----------------------------------filter((store_sales.ss_quantity <= 5) and (store_sales.ss_quantity >= 0) and OR[AND[(store_sales.ss_list_price >= 131.00),(store_sales.ss_list_price <= 141.00)],AND[(store_sales.ss_coupon_amt >= 16798.00),(store_sales.ss_coupon_amt <= 17798.00)],AND[(store_sales.ss_wholesale_cost >= 25.00),(store_sales.ss_wholesale_cost <= 45.00)]]) ------------------------------------PhysicalOlapScan[store_sales] ------------------------PhysicalLimit[LOCAL] --------------------------hashAgg[GLOBAL] ----------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------hashAgg[LOCAL] --------------------------------PhysicalProject -----------------------------------filter(((((store_sales.ss_list_price >= 145.00) AND (store_sales.ss_list_price <= 155.00)) OR ((store_sales.ss_coupon_amt >= 14792.00) AND (store_sales.ss_coupon_amt <= 15792.00))) OR ((store_sales.ss_wholesale_cost >= 46.00) AND (store_sales.ss_wholesale_cost <= 66.00))) and (store_sales.ss_quantity <= 10) and (store_sales.ss_quantity >= 6)) +----------------------------------filter((store_sales.ss_quantity <= 10) and (store_sales.ss_quantity >= 6) and OR[AND[(store_sales.ss_list_price >= 145.00),(store_sales.ss_list_price <= 155.00)],AND[(store_sales.ss_coupon_amt >= 14792.00),(store_sales.ss_coupon_amt <= 15792.00)],AND[(store_sales.ss_wholesale_cost >= 46.00),(store_sales.ss_wholesale_cost <= 66.00)]]) ------------------------------------PhysicalOlapScan[store_sales] --------------------PhysicalLimit[LOCAL] ----------------------hashAgg[GLOBAL] ------------------------PhysicalDistribute[DistributionSpecGather] --------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------filter(((((store_sales.ss_list_price >= 150.00) AND (store_sales.ss_list_price <= 160.00)) OR ((store_sales.ss_coupon_amt >= 6600.00) AND (store_sales.ss_coupon_amt <= 7600.00))) OR ((store_sales.ss_wholesale_cost >= 9.00) AND (store_sales.ss_wholesale_cost <= 29.00))) and (store_sales.ss_quantity <= 15) and (store_sales.ss_quantity >= 11)) +------------------------------filter((store_sales.ss_quantity <= 15) and (store_sales.ss_quantity >= 11) and OR[AND[(store_sales.ss_list_price >= 150.00),(store_sales.ss_list_price <= 160.00)],AND[(store_sales.ss_coupon_amt >= 6600.00),(store_sales.ss_coupon_amt <= 7600.00)],AND[(store_sales.ss_wholesale_cost >= 9.00),(store_sales.ss_wholesale_cost <= 29.00)]]) --------------------------------PhysicalOlapScan[store_sales] ----------------PhysicalLimit[LOCAL] ------------------hashAgg[GLOBAL] --------------------PhysicalDistribute[DistributionSpecGather] ----------------------hashAgg[LOCAL] ------------------------PhysicalProject ---------------------------filter(((((store_sales.ss_list_price >= 91.00) AND (store_sales.ss_list_price <= 101.00)) OR ((store_sales.ss_coupon_amt >= 13493.00) AND (store_sales.ss_coupon_amt <= 14493.00))) OR ((store_sales.ss_wholesale_cost >= 36.00) AND (store_sales.ss_wholesale_cost <= 56.00))) and (store_sales.ss_quantity <= 20) and (store_sales.ss_quantity >= 16)) +--------------------------filter((store_sales.ss_quantity <= 20) and (store_sales.ss_quantity >= 16) and OR[AND[(store_sales.ss_list_price >= 91.00),(store_sales.ss_list_price <= 101.00)],AND[(store_sales.ss_coupon_amt >= 13493.00),(store_sales.ss_coupon_amt <= 14493.00)],AND[(store_sales.ss_wholesale_cost >= 36.00),(store_sales.ss_wholesale_cost <= 56.00)]]) ----------------------------PhysicalOlapScan[store_sales] ------------PhysicalLimit[LOCAL] --------------hashAgg[GLOBAL] ----------------PhysicalDistribute[DistributionSpecGather] ------------------hashAgg[LOCAL] --------------------PhysicalProject -----------------------filter(((((store_sales.ss_list_price >= 0.00) AND (store_sales.ss_list_price <= 10.00)) OR ((store_sales.ss_coupon_amt >= 7629.00) AND (store_sales.ss_coupon_amt <= 8629.00))) OR ((store_sales.ss_wholesale_cost >= 6.00) AND (store_sales.ss_wholesale_cost <= 26.00))) and (store_sales.ss_quantity <= 25) and (store_sales.ss_quantity >= 21)) +----------------------filter((store_sales.ss_quantity <= 25) and (store_sales.ss_quantity >= 21) and OR[AND[(store_sales.ss_list_price >= 0.00),(store_sales.ss_list_price <= 10.00)],AND[(store_sales.ss_coupon_amt >= 7629.00),(store_sales.ss_coupon_amt <= 8629.00)],AND[(store_sales.ss_wholesale_cost >= 6.00),(store_sales.ss_wholesale_cost <= 26.00)]]) ------------------------PhysicalOlapScan[store_sales] --------PhysicalLimit[LOCAL] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecGather] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------filter(((((store_sales.ss_list_price >= 89.00) AND (store_sales.ss_list_price <= 99.00)) OR ((store_sales.ss_coupon_amt >= 15257.00) AND (store_sales.ss_coupon_amt <= 16257.00))) OR ((store_sales.ss_wholesale_cost >= 31.00) AND (store_sales.ss_wholesale_cost <= 51.00))) and (store_sales.ss_quantity <= 30) and (store_sales.ss_quantity >= 26)) +------------------filter((store_sales.ss_quantity <= 30) and (store_sales.ss_quantity >= 26) and OR[AND[(store_sales.ss_list_price >= 89.00),(store_sales.ss_list_price <= 99.00)],AND[(store_sales.ss_coupon_amt >= 15257.00),(store_sales.ss_coupon_amt <= 16257.00)],AND[(store_sales.ss_wholesale_cost >= 31.00),(store_sales.ss_wholesale_cost <= 51.00)]]) --------------------PhysicalOlapScan[store_sales] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query34.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query34.out index dad99a6b6465dd..6c90e3a48d7cba 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query34.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query34.out @@ -19,7 +19,7 @@ PhysicalResultSink --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 --------------------------------PhysicalProject -----------------------------------filter((((date_dim.d_dom >= 1) AND (date_dim.d_dom <= 3)) OR ((date_dim.d_dom >= 25) AND (date_dim.d_dom <= 28))) and d_year IN (1998, 1999, 2000)) +----------------------------------filter(OR[AND[(date_dim.d_dom >= 1),(date_dim.d_dom <= 3)],AND[(date_dim.d_dom >= 25),(date_dim.d_dom <= 28)]] and d_year IN (1998, 1999, 2000)) ------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalProject ------------------------------filter(s_county IN ('Barrow County', 'Daviess County', 'Franklin Parish', 'Luce County', 'Richland County', 'Walker County', 'Williamson County', 'Ziebach County')) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query35.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query35.out index 9f3bb8dab0f9e3..c14ce7550a3069 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query35.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query35.out @@ -9,7 +9,7 @@ PhysicalResultSink ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) +------------------filter(OR[ifnull($c$1, FALSE),ifnull($c$2, FALSE)]) --------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() ----------------------PhysicalProject ------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query41.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query41.out index c9c1ffff499741..3034a77fe0897a 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query41.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query41.out @@ -18,6 +18,6 @@ PhysicalResultSink ------------------------PhysicalDistribute[DistributionSpecHash] --------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------filter((((i_color IN ('aquamarine', 'blue', 'chartreuse', 'chiffon', 'dodger', 'gainsboro', 'tan', 'violet') AND i_units IN ('Bunch', 'Dozen', 'Each', 'Ounce', 'Oz', 'Pound', 'Ton', 'Tsp')) AND (((((((item.i_category = 'Women') AND i_color IN ('aquamarine', 'gainsboro')) AND i_units IN ('Dozen', 'Ounce')) AND i_size IN ('economy', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('chiffon', 'violet')) AND i_units IN ('Pound', 'Ton')) AND i_size IN ('extra large', 'small'))) OR ((((item.i_category = 'Men') AND i_color IN ('blue', 'chartreuse')) AND i_units IN ('Each', 'Oz')) AND i_size IN ('N/A', 'large'))) OR ((((item.i_category = 'Men') AND i_color IN ('dodger', 'tan')) AND i_units IN ('Bunch', 'Tsp')) AND i_size IN ('economy', 'medium')))) OR ((i_color IN ('almond', 'blanched', 'indian', 'lime', 'peru', 'saddle', 'spring', 'tomato') AND i_units IN ('Box', 'Carton', 'Case', 'Dram', 'Gram', 'Pallet', 'Tbl', 'Unknown')) AND (((((((item.i_category = 'Women') AND i_color IN ('blanched', 'tomato')) AND i_units IN ('Case', 'Tbl')) AND i_size IN ('economy', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('almond', 'lime')) AND i_units IN ('Box', 'Dram')) AND i_size IN ('extra large', 'small'))) OR ((((item.i_category = 'Men') AND i_color IN ('peru', 'saddle')) AND i_units IN ('Gram', 'Pallet')) AND i_size IN ('N/A', 'large'))) OR ((((item.i_category = 'Men') AND i_color IN ('indian', 'spring')) AND i_units IN ('Carton', 'Unknown')) AND i_size IN ('economy', 'medium'))))) and i_category IN ('Men', 'Women') and i_size IN ('N/A', 'economy', 'extra large', 'large', 'medium', 'small')) +------------------------------filter(OR[AND[i_color IN ('aquamarine', 'blue', 'chartreuse', 'chiffon', 'dodger', 'gainsboro', 'tan', 'violet'),i_units IN ('Bunch', 'Dozen', 'Each', 'Ounce', 'Oz', 'Pound', 'Ton', 'Tsp'),OR[AND[(item.i_category = 'Women'),i_color IN ('aquamarine', 'gainsboro'),i_units IN ('Dozen', 'Ounce'),i_size IN ('economy', 'medium')],AND[(item.i_category = 'Women'),i_color IN ('chiffon', 'violet'),i_units IN ('Pound', 'Ton'),i_size IN ('extra large', 'small')],AND[(item.i_category = 'Men'),i_color IN ('blue', 'chartreuse'),i_units IN ('Each', 'Oz'),i_size IN ('N/A', 'large')],AND[(item.i_category = 'Men'),i_color IN ('dodger', 'tan'),i_units IN ('Bunch', 'Tsp'),i_size IN ('economy', 'medium')]]],AND[i_color IN ('almond', 'blanched', 'indian', 'lime', 'peru', 'saddle', 'spring', 'tomato'),i_units IN ('Box', 'Carton', 'Case', 'Dram', 'Gram', 'Pallet', 'Tbl', 'Unknown'),OR[AND[(item.i_category = 'Women'),i_color IN ('blanched', 'tomato'),i_units IN ('Case', 'Tbl'),i_size IN ('economy', 'medium')],AND[(item.i_category = 'Women'),i_color IN ('almond', 'lime'),i_units IN ('Box', 'Dram'),i_size IN ('extra large', 'small')],AND[(item.i_category = 'Men'),i_color IN ('peru', 'saddle'),i_units IN ('Gram', 'Pallet'),i_size IN ('N/A', 'large')],AND[(item.i_category = 'Men'),i_color IN ('indian', 'spring'),i_units IN ('Carton', 'Unknown'),i_size IN ('economy', 'medium')]]]] and i_category IN ('Men', 'Women') and i_size IN ('N/A', 'economy', 'extra large', 'large', 'medium', 'small')) --------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query45.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query45.out index c6b4daafdce0df..7270fe9092a53b 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query45.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query45.out @@ -8,7 +8,7 @@ PhysicalResultSink ----------PhysicalDistribute[DistributionSpecHash] ------------hashAgg[LOCAL] --------------PhysicalProject -----------------filter((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR $c$1)) +----------------filter(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),$c$1]) ------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query46.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query46.out index be021ce25b68ae..48e2eae68f5c61 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query46.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query46.out @@ -27,7 +27,7 @@ PhysicalResultSink ----------------------------------filter(s_city IN ('Centerville', 'Fairview', 'Five Points', 'Liberty', 'Oak Grove')) ------------------------------------PhysicalOlapScan[store] ----------------------------PhysicalProject -------------------------------filter(((household_demographics.hd_dep_count = 6) OR (household_demographics.hd_vehicle_count = 0))) +------------------------------filter(OR[(household_demographics.hd_dep_count = 6),(household_demographics.hd_vehicle_count = 0)]) --------------------------------PhysicalOlapScan[household_demographics] ------------------------PhysicalProject --------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query47.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query47.out index 0182bbf46eb817..77dcc9357f98b8 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query47.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query47.out @@ -23,7 +23,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------------------------PhysicalProject --------------------------------------PhysicalOlapScan[item] --------------------------------PhysicalProject -----------------------------------filter((((date_dim.d_year = 2001) OR ((date_dim.d_year = 2000) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2002) AND (date_dim.d_moy = 1))) and d_year IN (2000, 2001, 2002)) +----------------------------------filter(OR[(date_dim.d_year = 2001),AND[(date_dim.d_year = 2000),(date_dim.d_moy = 12)],AND[(date_dim.d_year = 2002),(date_dim.d_moy = 1)]] and d_year IN (2000, 2001, 2002)) ------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query48.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query48.out index 338a70100eb21b..57618c21f263d3 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query48.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query48.out @@ -7,9 +7,9 @@ PhysicalResultSink --------PhysicalProject ----------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ss_sold_date_sk] ------------PhysicalProject ---------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=((((ca_state IN ('IA', 'MD', 'MN') AND ((store_sales.ss_net_profit >= 0.00) AND (store_sales.ss_net_profit <= 2000.00))) OR (ca_state IN ('IL', 'TX', 'VA') AND ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 3000.00)))) OR (ca_state IN ('IN', 'MI', 'WI') AND ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 25000.00))))) build RFs:RF2 ca_address_sk->[ss_addr_sk] +--------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=(OR[AND[ca_state IN ('IA', 'MD', 'MN'),(store_sales.ss_net_profit >= 0.00),(store_sales.ss_net_profit <= 2000.00)],AND[ca_state IN ('IL', 'TX', 'VA'),(store_sales.ss_net_profit >= 150.00),(store_sales.ss_net_profit <= 3000.00)],AND[ca_state IN ('IN', 'MI', 'WI'),(store_sales.ss_net_profit >= 50.00),(store_sales.ss_net_profit <= 25000.00)]]) build RFs:RF2 ca_address_sk->[ss_addr_sk] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)) otherCondition=((((((customer_demographics.cd_marital_status = 'U') AND (customer_demographics.cd_education_status = 'Primary')) AND ((store_sales.ss_sales_price >= 100.00) AND (store_sales.ss_sales_price <= 150.00))) OR (((customer_demographics.cd_marital_status = 'W') AND (customer_demographics.cd_education_status = 'College')) AND ((store_sales.ss_sales_price >= 50.00) AND (store_sales.ss_sales_price <= 100.00)))) OR (((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = '2 yr Degree')) AND ((store_sales.ss_sales_price >= 150.00) AND (store_sales.ss_sales_price <= 200.00))))) build RFs:RF1 cd_demo_sk->[ss_cdemo_sk] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)) otherCondition=(OR[AND[(customer_demographics.cd_marital_status = 'U'),(customer_demographics.cd_education_status = 'Primary'),(store_sales.ss_sales_price >= 100.00),(store_sales.ss_sales_price <= 150.00)],AND[(customer_demographics.cd_marital_status = 'W'),(customer_demographics.cd_education_status = 'College'),(store_sales.ss_sales_price >= 50.00),(store_sales.ss_sales_price <= 100.00)],AND[(customer_demographics.cd_marital_status = 'D'),(customer_demographics.cd_education_status = '2 yr Degree'),(store_sales.ss_sales_price >= 150.00),(store_sales.ss_sales_price <= 200.00)]]) build RFs:RF1 cd_demo_sk->[ss_cdemo_sk] --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() ------------------------PhysicalProject @@ -18,7 +18,7 @@ PhysicalResultSink ------------------------PhysicalProject --------------------------PhysicalOlapScan[store] --------------------PhysicalProject -----------------------filter(((((customer_demographics.cd_marital_status = 'U') AND (customer_demographics.cd_education_status = 'Primary')) OR ((customer_demographics.cd_marital_status = 'W') AND (customer_demographics.cd_education_status = 'College'))) OR ((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = '2 yr Degree'))) and cd_education_status IN ('2 yr Degree', 'College', 'Primary') and cd_marital_status IN ('D', 'U', 'W')) +----------------------filter(OR[AND[(customer_demographics.cd_marital_status = 'U'),(customer_demographics.cd_education_status = 'Primary')],AND[(customer_demographics.cd_marital_status = 'W'),(customer_demographics.cd_education_status = 'College')],AND[(customer_demographics.cd_marital_status = 'D'),(customer_demographics.cd_education_status = '2 yr Degree')]] and cd_education_status IN ('2 yr Degree', 'College', 'Primary') and cd_marital_status IN ('D', 'U', 'W')) ------------------------PhysicalOlapScan[customer_demographics] ----------------PhysicalProject ------------------filter((customer_address.ca_country = 'United States') and ca_state IN ('IA', 'IL', 'IN', 'MD', 'MI', 'MN', 'TX', 'VA', 'WI')) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query49.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query49.out index dc56d7f9e81411..0db3fa841189b2 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query49.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query49.out @@ -16,7 +16,7 @@ PhysicalResultSink --------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) ----------------------------------PhysicalWindow ------------------------------------PhysicalQuickSort[LOCAL_SORT] --------------------------------------PhysicalWindow @@ -48,7 +48,7 @@ PhysicalResultSink --------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) ----------------------------------PhysicalWindow ------------------------------------PhysicalQuickSort[LOCAL_SORT] --------------------------------------PhysicalWindow @@ -80,7 +80,7 @@ PhysicalResultSink --------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) ----------------------------------PhysicalWindow ------------------------------------PhysicalQuickSort[LOCAL_SORT] --------------------------------------PhysicalWindow diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query53.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query53.out index 93289fca48e4ef..89dc632eb527c4 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query53.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query53.out @@ -22,7 +22,7 @@ PhysicalResultSink --------------------------------------PhysicalProject ----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 --------------------------------------PhysicalProject -----------------------------------------filter((((i_category IN ('Books', 'Children', 'Electronics') AND i_class IN ('personal', 'portable', 'reference', 'self-help')) AND i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')) OR ((i_category IN ('Men', 'Music', 'Women') AND i_class IN ('accessories', 'classical', 'fragrances', 'pants')) AND i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1'))) and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) +----------------------------------------filter(OR[AND[i_category IN ('Books', 'Children', 'Electronics'),i_class IN ('personal', 'portable', 'reference', 'self-help'),i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')],AND[i_category IN ('Men', 'Music', 'Women'),i_class IN ('accessories', 'classical', 'fragrances', 'pants'),i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')]] and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) ------------------------------------------PhysicalOlapScan[item] ----------------------------------PhysicalProject ------------------------------------filter(d_month_seq IN (1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211)) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query57.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query57.out index 842b18a4eac9e6..96d8f68090e5de 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query57.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query57.out @@ -23,7 +23,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------------------------PhysicalProject --------------------------------------PhysicalOlapScan[item] --------------------------------PhysicalProject -----------------------------------filter((((date_dim.d_year = 1999) OR ((date_dim.d_year = 1998) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2000) AND (date_dim.d_moy = 1))) and d_year IN (1998, 1999, 2000)) +----------------------------------filter(OR[(date_dim.d_year = 1999),AND[(date_dim.d_year = 1998),(date_dim.d_moy = 12)],AND[(date_dim.d_year = 2000),(date_dim.d_moy = 1)]] and d_year IN (1998, 1999, 2000)) ------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[call_center] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query61.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query61.out index f9dcb3c9460ca4..62da8c9cb21a0f 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query61.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query61.out @@ -27,7 +27,7 @@ PhysicalResultSink ------------------------------------filter((store.s_gmt_offset = -7.00)) --------------------------------------PhysicalOlapScan[store] ------------------------------PhysicalProject ---------------------------------filter((((promotion.p_channel_dmail = 'Y') OR (promotion.p_channel_email = 'Y')) OR (promotion.p_channel_tv = 'Y'))) +--------------------------------filter(OR[(promotion.p_channel_dmail = 'Y'),(promotion.p_channel_email = 'Y'),(promotion.p_channel_tv = 'Y')]) ----------------------------------PhysicalOlapScan[promotion] --------------------------PhysicalProject ----------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 1999)) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query63.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query63.out index 3bd844ce72c196..9653f6c52199aa 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query63.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query63.out @@ -22,7 +22,7 @@ PhysicalResultSink --------------------------------------PhysicalProject ----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 --------------------------------------PhysicalProject -----------------------------------------filter((((i_category IN ('Books', 'Children', 'Electronics') AND i_class IN ('personal', 'portable', 'reference', 'self-help')) AND i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')) OR ((i_category IN ('Men', 'Music', 'Women') AND i_class IN ('accessories', 'classical', 'fragrances', 'pants')) AND i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1'))) and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) +----------------------------------------filter(OR[AND[i_category IN ('Books', 'Children', 'Electronics'),i_class IN ('personal', 'portable', 'reference', 'self-help'),i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')],AND[i_category IN ('Men', 'Music', 'Women'),i_class IN ('accessories', 'classical', 'fragrances', 'pants'),i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')]] and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) ------------------------------------------PhysicalOlapScan[item] ----------------------------------PhysicalProject ------------------------------------filter(d_month_seq IN (1181, 1182, 1183, 1184, 1185, 1186, 1187, 1188, 1189, 1190, 1191, 1192)) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query68.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query68.out index 52f4671d103618..0fd8ddd97ecd2c 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query68.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query68.out @@ -27,7 +27,7 @@ PhysicalResultSink ----------------------------------filter(s_city IN ('Five Points', 'Pleasant Hill')) ------------------------------------PhysicalOlapScan[store] ----------------------------PhysicalProject -------------------------------filter(((household_demographics.hd_dep_count = 8) OR (household_demographics.hd_vehicle_count = -1))) +------------------------------filter(OR[(household_demographics.hd_dep_count = 8),(household_demographics.hd_vehicle_count = -1)]) --------------------------------PhysicalOlapScan[household_demographics] ------------------------PhysicalProject --------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query7.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query7.out index ff218ca1256a94..f47da720468166 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query7.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query7.out @@ -26,6 +26,6 @@ PhysicalResultSink ------------------------filter((date_dim.d_year = 2001)) --------------------------PhysicalOlapScan[date_dim] ------------------PhysicalProject ---------------------filter(((promotion.p_channel_email = 'N') OR (promotion.p_channel_event = 'N'))) +--------------------filter(OR[(promotion.p_channel_email = 'N'),(promotion.p_channel_event = 'N')]) ----------------------PhysicalOlapScan[promotion] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query78.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query78.out index 30527883a49018..a6034ca86ac5c5 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query78.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query78.out @@ -5,7 +5,7 @@ PhysicalResultSink ----PhysicalDistribute[DistributionSpecGather] ------PhysicalTopN[LOCAL_SORT] --------PhysicalProject -----------filter(((coalesce(ws_qty, 0) > 0) OR (coalesce(cs_qty, 0) > 0))) +----------filter(OR[(coalesce(ws_qty, 0) > 0),(coalesce(cs_qty, 0) > 0)]) ------------hashJoin[LEFT_OUTER_JOIN colocated] hashCondition=((cs.cs_customer_sk = ss.ss_customer_sk) and (cs.cs_item_sk = ss.ss_item_sk) and (cs.cs_sold_year = ss.ss_sold_year)) otherCondition=() --------------PhysicalProject ----------------hashJoin[LEFT_OUTER_JOIN colocated] hashCondition=((ws.ws_customer_sk = ss.ss_customer_sk) and (ws.ws_item_sk = ss.ss_item_sk) and (ws.ws_sold_year = ss.ss_sold_year)) otherCondition=() diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query79.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query79.out index 77168450e857f5..8f9b721f08ee59 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query79.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query79.out @@ -25,7 +25,7 @@ PhysicalResultSink ------------------------------filter((store.s_number_employees <= 295) and (store.s_number_employees >= 200)) --------------------------------PhysicalOlapScan[store] ------------------------PhysicalProject ---------------------------filter(((household_demographics.hd_dep_count = 5) OR (household_demographics.hd_vehicle_count > 4))) +--------------------------filter(OR[(household_demographics.hd_dep_count = 5),(household_demographics.hd_vehicle_count > 4)]) ----------------------------PhysicalOlapScan[household_demographics] ------------PhysicalProject --------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query85.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query85.out index 590589c61a42a0..9f12763cec5114 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query85.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query85.out @@ -13,11 +13,11 @@ PhysicalResultSink --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF8 d_date_sk->[ws_sold_date_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = web_returns.wr_refunded_addr_sk)) otherCondition=((((ca_state IN ('DE', 'FL', 'TX') AND ((web_sales.ws_net_profit >= 100.00) AND (web_sales.ws_net_profit <= 200.00))) OR (ca_state IN ('ID', 'IN', 'ND') AND ((web_sales.ws_net_profit >= 150.00) AND (web_sales.ws_net_profit <= 300.00)))) OR (ca_state IN ('IL', 'MT', 'OH') AND ((web_sales.ws_net_profit >= 50.00) AND (web_sales.ws_net_profit <= 250.00))))) build RFs:RF7 ca_address_sk->[wr_refunded_addr_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = web_returns.wr_refunded_addr_sk)) otherCondition=(OR[AND[ca_state IN ('DE', 'FL', 'TX'),(web_sales.ws_net_profit >= 100.00),(web_sales.ws_net_profit <= 200.00)],AND[ca_state IN ('ID', 'IN', 'ND'),(web_sales.ws_net_profit >= 150.00),(web_sales.ws_net_profit <= 300.00)],AND[ca_state IN ('IL', 'MT', 'OH'),(web_sales.ws_net_profit >= 50.00),(web_sales.ws_net_profit <= 250.00)]]) build RFs:RF7 ca_address_sk->[wr_refunded_addr_sk] ----------------------------PhysicalProject ------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cd1.cd_education_status = cd2.cd_education_status) and (cd1.cd_marital_status = cd2.cd_marital_status) and (cd2.cd_demo_sk = web_returns.wr_returning_cdemo_sk)) otherCondition=() build RFs:RF4 cd_demo_sk->[wr_returning_cdemo_sk];RF5 cd_marital_status->[cd_marital_status];RF6 cd_education_status->[cd_education_status] --------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cd1.cd_demo_sk = web_returns.wr_refunded_cdemo_sk)) otherCondition=((((((cd1.cd_marital_status = 'M') AND (cd1.cd_education_status = '4 yr Degree')) AND ((web_sales.ws_sales_price >= 100.00) AND (web_sales.ws_sales_price <= 150.00))) OR (((cd1.cd_marital_status = 'S') AND (cd1.cd_education_status = 'Secondary')) AND ((web_sales.ws_sales_price >= 50.00) AND (web_sales.ws_sales_price <= 100.00)))) OR (((cd1.cd_marital_status = 'W') AND (cd1.cd_education_status = 'Advanced Degree')) AND ((web_sales.ws_sales_price >= 150.00) AND (web_sales.ws_sales_price <= 200.00))))) build RFs:RF3 cd_demo_sk->[wr_refunded_cdemo_sk] +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cd1.cd_demo_sk = web_returns.wr_refunded_cdemo_sk)) otherCondition=(OR[AND[(cd1.cd_marital_status = 'M'),(cd1.cd_education_status = '4 yr Degree'),(web_sales.ws_sales_price >= 100.00),(web_sales.ws_sales_price <= 150.00)],AND[(cd1.cd_marital_status = 'S'),(cd1.cd_education_status = 'Secondary'),(web_sales.ws_sales_price >= 50.00),(web_sales.ws_sales_price <= 100.00)],AND[(cd1.cd_marital_status = 'W'),(cd1.cd_education_status = 'Advanced Degree'),(web_sales.ws_sales_price >= 150.00),(web_sales.ws_sales_price <= 200.00)]]) build RFs:RF3 cd_demo_sk->[wr_refunded_cdemo_sk] ------------------------------------PhysicalProject --------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_web_page_sk = web_page.wp_web_page_sk)) otherCondition=() ----------------------------------------PhysicalProject @@ -30,7 +30,7 @@ PhysicalResultSink ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[web_page] ------------------------------------PhysicalProject ---------------------------------------filter(((((cd1.cd_marital_status = 'M') AND (cd1.cd_education_status = '4 yr Degree')) OR ((cd1.cd_marital_status = 'S') AND (cd1.cd_education_status = 'Secondary'))) OR ((cd1.cd_marital_status = 'W') AND (cd1.cd_education_status = 'Advanced Degree'))) and cd_education_status IN ('4 yr Degree', 'Advanced Degree', 'Secondary') and cd_marital_status IN ('M', 'S', 'W')) +--------------------------------------filter(OR[AND[(cd1.cd_marital_status = 'M'),(cd1.cd_education_status = '4 yr Degree')],AND[(cd1.cd_marital_status = 'S'),(cd1.cd_education_status = 'Secondary')],AND[(cd1.cd_marital_status = 'W'),(cd1.cd_education_status = 'Advanced Degree')]] and cd_education_status IN ('4 yr Degree', 'Advanced Degree', 'Secondary') and cd_marital_status IN ('M', 'S', 'W')) ----------------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF5 RF6 --------------------------------PhysicalProject ----------------------------------filter(cd_education_status IN ('4 yr Degree', 'Advanced Degree', 'Secondary') and cd_marital_status IN ('M', 'S', 'W')) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query88.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query88.out index dc879d6daa5a4b..a20acf339431ea 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query88.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query88.out @@ -20,7 +20,7 @@ PhysicalResultSink ----------------------------------PhysicalProject ------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF21 RF22 RF23 ----------------------------------PhysicalProject -------------------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +------------------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) --------------------------------------PhysicalOlapScan[household_demographics] ------------------------------PhysicalProject --------------------------------filter((time_dim.t_hour = 8) and (time_dim.t_minute >= 30)) @@ -40,7 +40,7 @@ PhysicalResultSink ----------------------------------PhysicalProject ------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF18 RF19 RF20 ----------------------------------PhysicalProject -------------------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +------------------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) --------------------------------------PhysicalOlapScan[household_demographics] ------------------------------PhysicalProject --------------------------------filter((time_dim.t_hour = 9) and (time_dim.t_minute < 30)) @@ -60,7 +60,7 @@ PhysicalResultSink --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF15 RF16 RF17 --------------------------------PhysicalProject -----------------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +----------------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) ------------------------------------PhysicalOlapScan[household_demographics] ----------------------------PhysicalProject ------------------------------filter((time_dim.t_hour = 9) and (time_dim.t_minute >= 30)) @@ -80,7 +80,7 @@ PhysicalResultSink ------------------------------PhysicalProject --------------------------------PhysicalOlapScan[store_sales] apply RFs: RF12 RF13 RF14 ------------------------------PhysicalProject ---------------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +--------------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) ----------------------------------PhysicalOlapScan[household_demographics] --------------------------PhysicalProject ----------------------------filter((time_dim.t_hour = 10) and (time_dim.t_minute < 30)) @@ -100,7 +100,7 @@ PhysicalResultSink ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[store_sales] apply RFs: RF9 RF10 RF11 ----------------------------PhysicalProject -------------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +------------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) --------------------------------PhysicalOlapScan[household_demographics] ------------------------PhysicalProject --------------------------filter((time_dim.t_hour = 10) and (time_dim.t_minute >= 30)) @@ -120,7 +120,7 @@ PhysicalResultSink --------------------------PhysicalProject ----------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 RF7 RF8 --------------------------PhysicalProject -----------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +----------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) ------------------------------PhysicalOlapScan[household_demographics] ----------------------PhysicalProject ------------------------filter((time_dim.t_hour = 11) and (time_dim.t_minute < 30)) @@ -140,7 +140,7 @@ PhysicalResultSink ------------------------PhysicalProject --------------------------PhysicalOlapScan[store_sales] apply RFs: RF3 RF4 RF5 ------------------------PhysicalProject ---------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +--------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) ----------------------------PhysicalOlapScan[household_demographics] --------------------PhysicalProject ----------------------filter((time_dim.t_hour = 11) and (time_dim.t_minute >= 30)) @@ -160,7 +160,7 @@ PhysicalResultSink ----------------------PhysicalProject ------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 ----------------------PhysicalProject -------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) --------------------------PhysicalOlapScan[household_demographics] ------------------PhysicalProject --------------------filter((time_dim.t_hour = 12) and (time_dim.t_minute < 30)) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query89.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query89.out index c709c4df5fe097..2e9294bce91fd2 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query89.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query89.out @@ -23,7 +23,7 @@ PhysicalResultSink ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 ----------------------------------------PhysicalProject -------------------------------------------filter(((i_category IN ('Electronics', 'Jewelry', 'Shoes') AND i_class IN ('athletic', 'portable', 'semi-precious')) OR (i_category IN ('Men', 'Music', 'Women') AND i_class IN ('accessories', 'maternity', 'rock'))) and i_category IN ('Electronics', 'Jewelry', 'Men', 'Music', 'Shoes', 'Women') and i_class IN ('accessories', 'athletic', 'maternity', 'portable', 'rock', 'semi-precious')) +------------------------------------------filter(OR[AND[i_category IN ('Electronics', 'Jewelry', 'Shoes'),i_class IN ('athletic', 'portable', 'semi-precious')],AND[i_category IN ('Men', 'Music', 'Women'),i_class IN ('accessories', 'maternity', 'rock')]] and i_category IN ('Electronics', 'Jewelry', 'Men', 'Music', 'Shoes', 'Women') and i_class IN ('accessories', 'athletic', 'maternity', 'portable', 'rock', 'semi-precious')) --------------------------------------------PhysicalOlapScan[item] ------------------------------------PhysicalProject --------------------------------------filter((date_dim.d_year = 1999)) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query91.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query91.out index aa211a9f494891..6a4b369a5fe1ed 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query91.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query91.out @@ -33,7 +33,7 @@ PhysicalResultSink ------------------------------filter((customer_address.ca_gmt_offset = -6.00)) --------------------------------PhysicalOlapScan[customer_address] ------------------------PhysicalProject ---------------------------filter((((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = 'Unknown')) OR ((customer_demographics.cd_marital_status = 'W') AND (customer_demographics.cd_education_status = 'Advanced Degree'))) and cd_education_status IN ('Advanced Degree', 'Unknown') and cd_marital_status IN ('M', 'W')) +--------------------------filter(OR[AND[(customer_demographics.cd_marital_status = 'M'),(customer_demographics.cd_education_status = 'Unknown')],AND[(customer_demographics.cd_marital_status = 'W'),(customer_demographics.cd_education_status = 'Advanced Degree')]] and cd_education_status IN ('Advanced Degree', 'Unknown') and cd_marital_status IN ('M', 'W')) ----------------------------PhysicalOlapScan[customer_demographics] --------------------PhysicalProject ----------------------filter((hd_buy_potential like '1001-5000%')) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query10.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query10.out index 36a2c69f6b9516..4fdff8b37961c5 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query10.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query10.out @@ -9,7 +9,7 @@ PhysicalResultSink ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) +------------------filter(OR[ifnull($c$1, FALSE),ifnull($c$2, FALSE)]) --------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() ----------------------PhysicalProject ------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF5 ca_address_sk->[c_current_addr_sk] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query13.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query13.out index 7f109c5dece1a7..a16b428cbc5159 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query13.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query13.out @@ -7,9 +7,9 @@ PhysicalResultSink --------PhysicalProject ----------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] ------------PhysicalProject ---------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=((((ca_state IN ('KS', 'MI', 'SD') AND ((store_sales.ss_net_profit >= 100.00) AND (store_sales.ss_net_profit <= 200.00))) OR (ca_state IN ('CO', 'MO', 'ND') AND ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 300.00)))) OR (ca_state IN ('NH', 'OH', 'TX') AND ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 250.00))))) build RFs:RF3 ca_address_sk->[ss_addr_sk] +--------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=(OR[AND[ca_state IN ('KS', 'MI', 'SD'),(store_sales.ss_net_profit >= 100.00),(store_sales.ss_net_profit <= 200.00)],AND[ca_state IN ('CO', 'MO', 'ND'),(store_sales.ss_net_profit >= 150.00),(store_sales.ss_net_profit <= 300.00)],AND[ca_state IN ('NH', 'OH', 'TX'),(store_sales.ss_net_profit >= 50.00),(store_sales.ss_net_profit <= 250.00)]]) build RFs:RF3 ca_address_sk->[ss_addr_sk] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=((((((household_demographics.hd_dep_count = 1) AND cd_marital_status IN ('M', 'S')) AND cd_education_status IN ('4 yr Degree', 'College')) AND ((((customer_demographics.cd_marital_status = 'S') AND (customer_demographics.cd_education_status = 'College')) AND ((store_sales.ss_sales_price >= 50.00) AND (store_sales.ss_sales_price <= 100.00))) OR (((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = '4 yr Degree')) AND ((store_sales.ss_sales_price >= 150.00) AND (store_sales.ss_sales_price <= 200.00))))) OR ((((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = 'Unknown')) AND ((store_sales.ss_sales_price >= 100.00) AND (store_sales.ss_sales_price <= 150.00))) AND (household_demographics.hd_dep_count = 3)))) build RFs:RF2 hd_demo_sk->[ss_hdemo_sk] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=(OR[AND[(household_demographics.hd_dep_count = 1),cd_marital_status IN ('M', 'S'),cd_education_status IN ('4 yr Degree', 'College'),OR[AND[(customer_demographics.cd_marital_status = 'S'),(customer_demographics.cd_education_status = 'College'),(store_sales.ss_sales_price >= 50.00),(store_sales.ss_sales_price <= 100.00)],AND[(customer_demographics.cd_marital_status = 'M'),(customer_demographics.cd_education_status = '4 yr Degree'),(store_sales.ss_sales_price >= 150.00),(store_sales.ss_sales_price <= 200.00)]]],AND[(customer_demographics.cd_marital_status = 'D'),(customer_demographics.cd_education_status = 'Unknown'),(store_sales.ss_sales_price >= 100.00),(store_sales.ss_sales_price <= 150.00),(household_demographics.hd_dep_count = 3)]]) build RFs:RF2 hd_demo_sk->[ss_hdemo_sk] --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)) otherCondition=() build RFs:RF1 cd_demo_sk->[ss_cdemo_sk] ------------------------PhysicalProject @@ -20,7 +20,7 @@ PhysicalResultSink ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[store] ------------------------PhysicalProject ---------------------------filter(((((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = 'Unknown')) OR ((customer_demographics.cd_marital_status = 'S') AND (customer_demographics.cd_education_status = 'College'))) OR ((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = '4 yr Degree'))) and cd_education_status IN ('4 yr Degree', 'College', 'Unknown') and cd_marital_status IN ('D', 'M', 'S')) +--------------------------filter(OR[AND[(customer_demographics.cd_marital_status = 'D'),(customer_demographics.cd_education_status = 'Unknown')],AND[(customer_demographics.cd_marital_status = 'S'),(customer_demographics.cd_education_status = 'College')],AND[(customer_demographics.cd_marital_status = 'M'),(customer_demographics.cd_education_status = '4 yr Degree')]] and cd_education_status IN ('4 yr Degree', 'College', 'Unknown') and cd_marital_status IN ('D', 'M', 'S')) ----------------------------PhysicalOlapScan[customer_demographics] --------------------PhysicalProject ----------------------filter(hd_dep_count IN (1, 3)) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query15.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query15.out index 1fba646d62b40a..81b0bae51498c1 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query15.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query15.out @@ -10,7 +10,7 @@ PhysicalResultSink --------------PhysicalProject ----------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=(((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR ca_state IN ('CA', 'GA', 'WA')) OR (catalog_sales.cs_sales_price > 500.00))) build RFs:RF1 ca_address_sk->[c_current_addr_sk] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),ca_state IN ('CA', 'GA', 'WA'),(catalog_sales.cs_sales_price > 500.00)]) build RFs:RF1 ca_address_sk->[c_current_addr_sk] ----------------------PhysicalProject ------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF0 c_customer_sk->[cs_bill_customer_sk] --------------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query26.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query26.out index 4561d649bd82c2..edbed407b77921 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query26.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query26.out @@ -26,6 +26,6 @@ PhysicalResultSink ------------------------filter((date_dim.d_year = 2001)) --------------------------PhysicalOlapScan[date_dim] ------------------PhysicalProject ---------------------filter(((promotion.p_channel_email = 'N') OR (promotion.p_channel_event = 'N'))) +--------------------filter(OR[(promotion.p_channel_email = 'N'),(promotion.p_channel_event = 'N')]) ----------------------PhysicalOlapScan[promotion] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query28.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query28.out index dae7cb1c23a402..7a6bdd8868ef00 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query28.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query28.out @@ -17,41 +17,41 @@ PhysicalResultSink ----------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------hashAgg[LOCAL] --------------------------------PhysicalProject -----------------------------------filter(((((store_sales.ss_list_price >= 131.00) AND (store_sales.ss_list_price <= 141.00)) OR ((store_sales.ss_coupon_amt >= 16798.00) AND (store_sales.ss_coupon_amt <= 17798.00))) OR ((store_sales.ss_wholesale_cost >= 25.00) AND (store_sales.ss_wholesale_cost <= 45.00))) and (store_sales.ss_quantity <= 5) and (store_sales.ss_quantity >= 0)) +----------------------------------filter((store_sales.ss_quantity <= 5) and (store_sales.ss_quantity >= 0) and OR[AND[(store_sales.ss_list_price >= 131.00),(store_sales.ss_list_price <= 141.00)],AND[(store_sales.ss_coupon_amt >= 16798.00),(store_sales.ss_coupon_amt <= 17798.00)],AND[(store_sales.ss_wholesale_cost >= 25.00),(store_sales.ss_wholesale_cost <= 45.00)]]) ------------------------------------PhysicalOlapScan[store_sales] ------------------------PhysicalLimit[LOCAL] --------------------------hashAgg[GLOBAL] ----------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------hashAgg[LOCAL] --------------------------------PhysicalProject -----------------------------------filter(((((store_sales.ss_list_price >= 145.00) AND (store_sales.ss_list_price <= 155.00)) OR ((store_sales.ss_coupon_amt >= 14792.00) AND (store_sales.ss_coupon_amt <= 15792.00))) OR ((store_sales.ss_wholesale_cost >= 46.00) AND (store_sales.ss_wholesale_cost <= 66.00))) and (store_sales.ss_quantity <= 10) and (store_sales.ss_quantity >= 6)) +----------------------------------filter((store_sales.ss_quantity <= 10) and (store_sales.ss_quantity >= 6) and OR[AND[(store_sales.ss_list_price >= 145.00),(store_sales.ss_list_price <= 155.00)],AND[(store_sales.ss_coupon_amt >= 14792.00),(store_sales.ss_coupon_amt <= 15792.00)],AND[(store_sales.ss_wholesale_cost >= 46.00),(store_sales.ss_wholesale_cost <= 66.00)]]) ------------------------------------PhysicalOlapScan[store_sales] --------------------PhysicalLimit[LOCAL] ----------------------hashAgg[GLOBAL] ------------------------PhysicalDistribute[DistributionSpecGather] --------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------filter(((((store_sales.ss_list_price >= 150.00) AND (store_sales.ss_list_price <= 160.00)) OR ((store_sales.ss_coupon_amt >= 6600.00) AND (store_sales.ss_coupon_amt <= 7600.00))) OR ((store_sales.ss_wholesale_cost >= 9.00) AND (store_sales.ss_wholesale_cost <= 29.00))) and (store_sales.ss_quantity <= 15) and (store_sales.ss_quantity >= 11)) +------------------------------filter((store_sales.ss_quantity <= 15) and (store_sales.ss_quantity >= 11) and OR[AND[(store_sales.ss_list_price >= 150.00),(store_sales.ss_list_price <= 160.00)],AND[(store_sales.ss_coupon_amt >= 6600.00),(store_sales.ss_coupon_amt <= 7600.00)],AND[(store_sales.ss_wholesale_cost >= 9.00),(store_sales.ss_wholesale_cost <= 29.00)]]) --------------------------------PhysicalOlapScan[store_sales] ----------------PhysicalLimit[LOCAL] ------------------hashAgg[GLOBAL] --------------------PhysicalDistribute[DistributionSpecGather] ----------------------hashAgg[LOCAL] ------------------------PhysicalProject ---------------------------filter(((((store_sales.ss_list_price >= 91.00) AND (store_sales.ss_list_price <= 101.00)) OR ((store_sales.ss_coupon_amt >= 13493.00) AND (store_sales.ss_coupon_amt <= 14493.00))) OR ((store_sales.ss_wholesale_cost >= 36.00) AND (store_sales.ss_wholesale_cost <= 56.00))) and (store_sales.ss_quantity <= 20) and (store_sales.ss_quantity >= 16)) +--------------------------filter((store_sales.ss_quantity <= 20) and (store_sales.ss_quantity >= 16) and OR[AND[(store_sales.ss_list_price >= 91.00),(store_sales.ss_list_price <= 101.00)],AND[(store_sales.ss_coupon_amt >= 13493.00),(store_sales.ss_coupon_amt <= 14493.00)],AND[(store_sales.ss_wholesale_cost >= 36.00),(store_sales.ss_wholesale_cost <= 56.00)]]) ----------------------------PhysicalOlapScan[store_sales] ------------PhysicalLimit[LOCAL] --------------hashAgg[GLOBAL] ----------------PhysicalDistribute[DistributionSpecGather] ------------------hashAgg[LOCAL] --------------------PhysicalProject -----------------------filter(((((store_sales.ss_list_price >= 0.00) AND (store_sales.ss_list_price <= 10.00)) OR ((store_sales.ss_coupon_amt >= 7629.00) AND (store_sales.ss_coupon_amt <= 8629.00))) OR ((store_sales.ss_wholesale_cost >= 6.00) AND (store_sales.ss_wholesale_cost <= 26.00))) and (store_sales.ss_quantity <= 25) and (store_sales.ss_quantity >= 21)) +----------------------filter((store_sales.ss_quantity <= 25) and (store_sales.ss_quantity >= 21) and OR[AND[(store_sales.ss_list_price >= 0.00),(store_sales.ss_list_price <= 10.00)],AND[(store_sales.ss_coupon_amt >= 7629.00),(store_sales.ss_coupon_amt <= 8629.00)],AND[(store_sales.ss_wholesale_cost >= 6.00),(store_sales.ss_wholesale_cost <= 26.00)]]) ------------------------PhysicalOlapScan[store_sales] --------PhysicalLimit[LOCAL] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecGather] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------filter(((((store_sales.ss_list_price >= 89.00) AND (store_sales.ss_list_price <= 99.00)) OR ((store_sales.ss_coupon_amt >= 15257.00) AND (store_sales.ss_coupon_amt <= 16257.00))) OR ((store_sales.ss_wholesale_cost >= 31.00) AND (store_sales.ss_wholesale_cost <= 51.00))) and (store_sales.ss_quantity <= 30) and (store_sales.ss_quantity >= 26)) +------------------filter((store_sales.ss_quantity <= 30) and (store_sales.ss_quantity >= 26) and OR[AND[(store_sales.ss_list_price >= 89.00),(store_sales.ss_list_price <= 99.00)],AND[(store_sales.ss_coupon_amt >= 15257.00),(store_sales.ss_coupon_amt <= 16257.00)],AND[(store_sales.ss_wholesale_cost >= 31.00),(store_sales.ss_wholesale_cost <= 51.00)]]) --------------------PhysicalOlapScan[store_sales] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query34.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query34.out index 25429183c8fe4a..9a53073479fb8c 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query34.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query34.out @@ -19,7 +19,7 @@ PhysicalResultSink --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 --------------------------------PhysicalProject -----------------------------------filter((((date_dim.d_dom >= 1) AND (date_dim.d_dom <= 3)) OR ((date_dim.d_dom >= 25) AND (date_dim.d_dom <= 28))) and d_year IN (1998, 1999, 2000)) +----------------------------------filter(OR[AND[(date_dim.d_dom >= 1),(date_dim.d_dom <= 3)],AND[(date_dim.d_dom >= 25),(date_dim.d_dom <= 28)]] and d_year IN (1998, 1999, 2000)) ------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalProject ------------------------------filter(s_county IN ('Barrow County', 'Daviess County', 'Franklin Parish', 'Luce County', 'Richland County', 'Walker County', 'Williamson County', 'Ziebach County')) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query35.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query35.out index f5f9508f9b6cfb..b414e22cb15150 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query35.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query35.out @@ -9,7 +9,7 @@ PhysicalResultSink ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) +------------------filter(OR[ifnull($c$1, FALSE),ifnull($c$2, FALSE)]) --------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() ----------------------PhysicalProject ------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF5 cd_demo_sk->[c_current_cdemo_sk] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query41.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query41.out index c9c1ffff499741..3034a77fe0897a 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query41.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query41.out @@ -18,6 +18,6 @@ PhysicalResultSink ------------------------PhysicalDistribute[DistributionSpecHash] --------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------filter((((i_color IN ('aquamarine', 'blue', 'chartreuse', 'chiffon', 'dodger', 'gainsboro', 'tan', 'violet') AND i_units IN ('Bunch', 'Dozen', 'Each', 'Ounce', 'Oz', 'Pound', 'Ton', 'Tsp')) AND (((((((item.i_category = 'Women') AND i_color IN ('aquamarine', 'gainsboro')) AND i_units IN ('Dozen', 'Ounce')) AND i_size IN ('economy', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('chiffon', 'violet')) AND i_units IN ('Pound', 'Ton')) AND i_size IN ('extra large', 'small'))) OR ((((item.i_category = 'Men') AND i_color IN ('blue', 'chartreuse')) AND i_units IN ('Each', 'Oz')) AND i_size IN ('N/A', 'large'))) OR ((((item.i_category = 'Men') AND i_color IN ('dodger', 'tan')) AND i_units IN ('Bunch', 'Tsp')) AND i_size IN ('economy', 'medium')))) OR ((i_color IN ('almond', 'blanched', 'indian', 'lime', 'peru', 'saddle', 'spring', 'tomato') AND i_units IN ('Box', 'Carton', 'Case', 'Dram', 'Gram', 'Pallet', 'Tbl', 'Unknown')) AND (((((((item.i_category = 'Women') AND i_color IN ('blanched', 'tomato')) AND i_units IN ('Case', 'Tbl')) AND i_size IN ('economy', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('almond', 'lime')) AND i_units IN ('Box', 'Dram')) AND i_size IN ('extra large', 'small'))) OR ((((item.i_category = 'Men') AND i_color IN ('peru', 'saddle')) AND i_units IN ('Gram', 'Pallet')) AND i_size IN ('N/A', 'large'))) OR ((((item.i_category = 'Men') AND i_color IN ('indian', 'spring')) AND i_units IN ('Carton', 'Unknown')) AND i_size IN ('economy', 'medium'))))) and i_category IN ('Men', 'Women') and i_size IN ('N/A', 'economy', 'extra large', 'large', 'medium', 'small')) +------------------------------filter(OR[AND[i_color IN ('aquamarine', 'blue', 'chartreuse', 'chiffon', 'dodger', 'gainsboro', 'tan', 'violet'),i_units IN ('Bunch', 'Dozen', 'Each', 'Ounce', 'Oz', 'Pound', 'Ton', 'Tsp'),OR[AND[(item.i_category = 'Women'),i_color IN ('aquamarine', 'gainsboro'),i_units IN ('Dozen', 'Ounce'),i_size IN ('economy', 'medium')],AND[(item.i_category = 'Women'),i_color IN ('chiffon', 'violet'),i_units IN ('Pound', 'Ton'),i_size IN ('extra large', 'small')],AND[(item.i_category = 'Men'),i_color IN ('blue', 'chartreuse'),i_units IN ('Each', 'Oz'),i_size IN ('N/A', 'large')],AND[(item.i_category = 'Men'),i_color IN ('dodger', 'tan'),i_units IN ('Bunch', 'Tsp'),i_size IN ('economy', 'medium')]]],AND[i_color IN ('almond', 'blanched', 'indian', 'lime', 'peru', 'saddle', 'spring', 'tomato'),i_units IN ('Box', 'Carton', 'Case', 'Dram', 'Gram', 'Pallet', 'Tbl', 'Unknown'),OR[AND[(item.i_category = 'Women'),i_color IN ('blanched', 'tomato'),i_units IN ('Case', 'Tbl'),i_size IN ('economy', 'medium')],AND[(item.i_category = 'Women'),i_color IN ('almond', 'lime'),i_units IN ('Box', 'Dram'),i_size IN ('extra large', 'small')],AND[(item.i_category = 'Men'),i_color IN ('peru', 'saddle'),i_units IN ('Gram', 'Pallet'),i_size IN ('N/A', 'large')],AND[(item.i_category = 'Men'),i_color IN ('indian', 'spring'),i_units IN ('Carton', 'Unknown'),i_size IN ('economy', 'medium')]]]] and i_category IN ('Men', 'Women') and i_size IN ('N/A', 'economy', 'extra large', 'large', 'medium', 'small')) --------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query45.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query45.out index 038ece83d7c044..68d1ef7855a7fd 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query45.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query45.out @@ -8,7 +8,7 @@ PhysicalResultSink ----------PhysicalDistribute[DistributionSpecHash] ------------hashAgg[LOCAL] --------------PhysicalProject -----------------filter((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR $c$1)) +----------------filter(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),$c$1]) ------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query46.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query46.out index 71f3746264834f..802c3969afc6e1 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query46.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query46.out @@ -27,7 +27,7 @@ PhysicalResultSink ----------------------------------filter(s_city IN ('Centerville', 'Fairview', 'Five Points', 'Liberty', 'Oak Grove')) ------------------------------------PhysicalOlapScan[store] ----------------------------PhysicalProject -------------------------------filter(((household_demographics.hd_dep_count = 6) OR (household_demographics.hd_vehicle_count = 0))) +------------------------------filter(OR[(household_demographics.hd_dep_count = 6),(household_demographics.hd_vehicle_count = 0)]) --------------------------------PhysicalOlapScan[household_demographics] ------------------------PhysicalProject --------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query47.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query47.out index b6f046eb01fe81..29aa8e6ae22e6c 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query47.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query47.out @@ -23,7 +23,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------------------------PhysicalProject --------------------------------------PhysicalOlapScan[item] --------------------------------PhysicalProject -----------------------------------filter((((date_dim.d_year = 2001) OR ((date_dim.d_year = 2000) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2002) AND (date_dim.d_moy = 1))) and d_year IN (2000, 2001, 2002)) +----------------------------------filter(OR[(date_dim.d_year = 2001),AND[(date_dim.d_year = 2000),(date_dim.d_moy = 12)],AND[(date_dim.d_year = 2002),(date_dim.d_moy = 1)]] and d_year IN (2000, 2001, 2002)) ------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query48.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query48.out index ed7ed3b4bdf669..fa9006ce24f673 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query48.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query48.out @@ -7,9 +7,9 @@ PhysicalResultSink --------PhysicalProject ----------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ss_sold_date_sk] ------------PhysicalProject ---------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=((((ca_state IN ('IA', 'MD', 'MN') AND ((store_sales.ss_net_profit >= 0.00) AND (store_sales.ss_net_profit <= 2000.00))) OR (ca_state IN ('IL', 'TX', 'VA') AND ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 3000.00)))) OR (ca_state IN ('IN', 'MI', 'WI') AND ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 25000.00))))) build RFs:RF2 ca_address_sk->[ss_addr_sk] +--------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=(OR[AND[ca_state IN ('IA', 'MD', 'MN'),(store_sales.ss_net_profit >= 0.00),(store_sales.ss_net_profit <= 2000.00)],AND[ca_state IN ('IL', 'TX', 'VA'),(store_sales.ss_net_profit >= 150.00),(store_sales.ss_net_profit <= 3000.00)],AND[ca_state IN ('IN', 'MI', 'WI'),(store_sales.ss_net_profit >= 50.00),(store_sales.ss_net_profit <= 25000.00)]]) build RFs:RF2 ca_address_sk->[ss_addr_sk] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)) otherCondition=((((((customer_demographics.cd_marital_status = 'U') AND (customer_demographics.cd_education_status = 'Primary')) AND ((store_sales.ss_sales_price >= 100.00) AND (store_sales.ss_sales_price <= 150.00))) OR (((customer_demographics.cd_marital_status = 'W') AND (customer_demographics.cd_education_status = 'College')) AND ((store_sales.ss_sales_price >= 50.00) AND (store_sales.ss_sales_price <= 100.00)))) OR (((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = '2 yr Degree')) AND ((store_sales.ss_sales_price >= 150.00) AND (store_sales.ss_sales_price <= 200.00))))) build RFs:RF1 cd_demo_sk->[ss_cdemo_sk] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)) otherCondition=(OR[AND[(customer_demographics.cd_marital_status = 'U'),(customer_demographics.cd_education_status = 'Primary'),(store_sales.ss_sales_price >= 100.00),(store_sales.ss_sales_price <= 150.00)],AND[(customer_demographics.cd_marital_status = 'W'),(customer_demographics.cd_education_status = 'College'),(store_sales.ss_sales_price >= 50.00),(store_sales.ss_sales_price <= 100.00)],AND[(customer_demographics.cd_marital_status = 'D'),(customer_demographics.cd_education_status = '2 yr Degree'),(store_sales.ss_sales_price >= 150.00),(store_sales.ss_sales_price <= 200.00)]]) build RFs:RF1 cd_demo_sk->[ss_cdemo_sk] --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF0 s_store_sk->[ss_store_sk] ------------------------PhysicalProject @@ -18,7 +18,7 @@ PhysicalResultSink ------------------------PhysicalProject --------------------------PhysicalOlapScan[store] --------------------PhysicalProject -----------------------filter(((((customer_demographics.cd_marital_status = 'U') AND (customer_demographics.cd_education_status = 'Primary')) OR ((customer_demographics.cd_marital_status = 'W') AND (customer_demographics.cd_education_status = 'College'))) OR ((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = '2 yr Degree'))) and cd_education_status IN ('2 yr Degree', 'College', 'Primary') and cd_marital_status IN ('D', 'U', 'W')) +----------------------filter(OR[AND[(customer_demographics.cd_marital_status = 'U'),(customer_demographics.cd_education_status = 'Primary')],AND[(customer_demographics.cd_marital_status = 'W'),(customer_demographics.cd_education_status = 'College')],AND[(customer_demographics.cd_marital_status = 'D'),(customer_demographics.cd_education_status = '2 yr Degree')]] and cd_education_status IN ('2 yr Degree', 'College', 'Primary') and cd_marital_status IN ('D', 'U', 'W')) ------------------------PhysicalOlapScan[customer_demographics] ----------------PhysicalProject ------------------filter((customer_address.ca_country = 'United States') and ca_state IN ('IA', 'IL', 'IN', 'MD', 'MI', 'MN', 'TX', 'VA', 'WI')) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query49.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query49.out index dc56d7f9e81411..0db3fa841189b2 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query49.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query49.out @@ -16,7 +16,7 @@ PhysicalResultSink --------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) ----------------------------------PhysicalWindow ------------------------------------PhysicalQuickSort[LOCAL_SORT] --------------------------------------PhysicalWindow @@ -48,7 +48,7 @@ PhysicalResultSink --------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) ----------------------------------PhysicalWindow ------------------------------------PhysicalQuickSort[LOCAL_SORT] --------------------------------------PhysicalWindow @@ -80,7 +80,7 @@ PhysicalResultSink --------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) ----------------------------------PhysicalWindow ------------------------------------PhysicalQuickSort[LOCAL_SORT] --------------------------------------PhysicalWindow diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query53.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query53.out index 81e6091b663bf7..04920e65ac6894 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query53.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query53.out @@ -22,7 +22,7 @@ PhysicalResultSink --------------------------------------PhysicalProject ----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 --------------------------------------PhysicalProject -----------------------------------------filter((((i_category IN ('Books', 'Children', 'Electronics') AND i_class IN ('personal', 'portable', 'reference', 'self-help')) AND i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')) OR ((i_category IN ('Men', 'Music', 'Women') AND i_class IN ('accessories', 'classical', 'fragrances', 'pants')) AND i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1'))) and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) +----------------------------------------filter(OR[AND[i_category IN ('Books', 'Children', 'Electronics'),i_class IN ('personal', 'portable', 'reference', 'self-help'),i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')],AND[i_category IN ('Men', 'Music', 'Women'),i_class IN ('accessories', 'classical', 'fragrances', 'pants'),i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')]] and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) ------------------------------------------PhysicalOlapScan[item] ----------------------------------PhysicalProject ------------------------------------filter(d_month_seq IN (1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211)) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query57.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query57.out index e640ec24d7b3e3..88777bc1ff548d 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query57.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query57.out @@ -23,7 +23,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------------------------PhysicalProject --------------------------------------PhysicalOlapScan[item] --------------------------------PhysicalProject -----------------------------------filter((((date_dim.d_year = 1999) OR ((date_dim.d_year = 1998) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2000) AND (date_dim.d_moy = 1))) and d_year IN (1998, 1999, 2000)) +----------------------------------filter(OR[(date_dim.d_year = 1999),AND[(date_dim.d_year = 1998),(date_dim.d_moy = 12)],AND[(date_dim.d_year = 2000),(date_dim.d_moy = 1)]] and d_year IN (1998, 1999, 2000)) ------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[call_center] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query61.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query61.out index f9dcb3c9460ca4..62da8c9cb21a0f 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query61.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query61.out @@ -27,7 +27,7 @@ PhysicalResultSink ------------------------------------filter((store.s_gmt_offset = -7.00)) --------------------------------------PhysicalOlapScan[store] ------------------------------PhysicalProject ---------------------------------filter((((promotion.p_channel_dmail = 'Y') OR (promotion.p_channel_email = 'Y')) OR (promotion.p_channel_tv = 'Y'))) +--------------------------------filter(OR[(promotion.p_channel_dmail = 'Y'),(promotion.p_channel_email = 'Y'),(promotion.p_channel_tv = 'Y')]) ----------------------------------PhysicalOlapScan[promotion] --------------------------PhysicalProject ----------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 1999)) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query63.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query63.out index 2cd49bd8c40786..d4fb4990da98b8 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query63.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query63.out @@ -22,7 +22,7 @@ PhysicalResultSink --------------------------------------PhysicalProject ----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 --------------------------------------PhysicalProject -----------------------------------------filter((((i_category IN ('Books', 'Children', 'Electronics') AND i_class IN ('personal', 'portable', 'reference', 'self-help')) AND i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')) OR ((i_category IN ('Men', 'Music', 'Women') AND i_class IN ('accessories', 'classical', 'fragrances', 'pants')) AND i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1'))) and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) +----------------------------------------filter(OR[AND[i_category IN ('Books', 'Children', 'Electronics'),i_class IN ('personal', 'portable', 'reference', 'self-help'),i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')],AND[i_category IN ('Men', 'Music', 'Women'),i_class IN ('accessories', 'classical', 'fragrances', 'pants'),i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')]] and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) ------------------------------------------PhysicalOlapScan[item] ----------------------------------PhysicalProject ------------------------------------filter(d_month_seq IN (1181, 1182, 1183, 1184, 1185, 1186, 1187, 1188, 1189, 1190, 1191, 1192)) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query68.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query68.out index 04dab40ade6bb2..20e70268ebb862 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query68.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query68.out @@ -27,7 +27,7 @@ PhysicalResultSink ----------------------------------filter(s_city IN ('Five Points', 'Pleasant Hill')) ------------------------------------PhysicalOlapScan[store] ----------------------------PhysicalProject -------------------------------filter(((household_demographics.hd_dep_count = 8) OR (household_demographics.hd_vehicle_count = -1))) +------------------------------filter(OR[(household_demographics.hd_dep_count = 8),(household_demographics.hd_vehicle_count = -1)]) --------------------------------PhysicalOlapScan[household_demographics] ------------------------PhysicalProject --------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query7.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query7.out index 12c9bd7d3a0b25..2b6615e0b93b84 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query7.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query7.out @@ -26,6 +26,6 @@ PhysicalResultSink ------------------------filter((date_dim.d_year = 2001)) --------------------------PhysicalOlapScan[date_dim] ------------------PhysicalProject ---------------------filter(((promotion.p_channel_email = 'N') OR (promotion.p_channel_event = 'N'))) +--------------------filter(OR[(promotion.p_channel_email = 'N'),(promotion.p_channel_event = 'N')]) ----------------------PhysicalOlapScan[promotion] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query78.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query78.out index 30527883a49018..a6034ca86ac5c5 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query78.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query78.out @@ -5,7 +5,7 @@ PhysicalResultSink ----PhysicalDistribute[DistributionSpecGather] ------PhysicalTopN[LOCAL_SORT] --------PhysicalProject -----------filter(((coalesce(ws_qty, 0) > 0) OR (coalesce(cs_qty, 0) > 0))) +----------filter(OR[(coalesce(ws_qty, 0) > 0),(coalesce(cs_qty, 0) > 0)]) ------------hashJoin[LEFT_OUTER_JOIN colocated] hashCondition=((cs.cs_customer_sk = ss.ss_customer_sk) and (cs.cs_item_sk = ss.ss_item_sk) and (cs.cs_sold_year = ss.ss_sold_year)) otherCondition=() --------------PhysicalProject ----------------hashJoin[LEFT_OUTER_JOIN colocated] hashCondition=((ws.ws_customer_sk = ss.ss_customer_sk) and (ws.ws_item_sk = ss.ss_item_sk) and (ws.ws_sold_year = ss.ss_sold_year)) otherCondition=() diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query79.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query79.out index 5d935352c491c6..1a19308d991441 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query79.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query79.out @@ -25,7 +25,7 @@ PhysicalResultSink ------------------------------filter((store.s_number_employees <= 295) and (store.s_number_employees >= 200)) --------------------------------PhysicalOlapScan[store] ------------------------PhysicalProject ---------------------------filter(((household_demographics.hd_dep_count = 5) OR (household_demographics.hd_vehicle_count > 4))) +--------------------------filter(OR[(household_demographics.hd_dep_count = 5),(household_demographics.hd_vehicle_count > 4)]) ----------------------------PhysicalOlapScan[household_demographics] ------------PhysicalProject --------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query85.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query85.out index 08e59c1e36507e..424102e34021a1 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query85.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query85.out @@ -13,11 +13,11 @@ PhysicalResultSink --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF8 d_date_sk->[ws_sold_date_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = web_returns.wr_refunded_addr_sk)) otherCondition=((((ca_state IN ('DE', 'FL', 'TX') AND ((web_sales.ws_net_profit >= 100.00) AND (web_sales.ws_net_profit <= 200.00))) OR (ca_state IN ('ID', 'IN', 'ND') AND ((web_sales.ws_net_profit >= 150.00) AND (web_sales.ws_net_profit <= 300.00)))) OR (ca_state IN ('IL', 'MT', 'OH') AND ((web_sales.ws_net_profit >= 50.00) AND (web_sales.ws_net_profit <= 250.00))))) build RFs:RF7 ca_address_sk->[wr_refunded_addr_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = web_returns.wr_refunded_addr_sk)) otherCondition=(OR[AND[ca_state IN ('DE', 'FL', 'TX'),(web_sales.ws_net_profit >= 100.00),(web_sales.ws_net_profit <= 200.00)],AND[ca_state IN ('ID', 'IN', 'ND'),(web_sales.ws_net_profit >= 150.00),(web_sales.ws_net_profit <= 300.00)],AND[ca_state IN ('IL', 'MT', 'OH'),(web_sales.ws_net_profit >= 50.00),(web_sales.ws_net_profit <= 250.00)]]) build RFs:RF7 ca_address_sk->[wr_refunded_addr_sk] ----------------------------PhysicalProject ------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cd1.cd_education_status = cd2.cd_education_status) and (cd1.cd_marital_status = cd2.cd_marital_status) and (cd2.cd_demo_sk = web_returns.wr_returning_cdemo_sk)) otherCondition=() build RFs:RF4 cd_demo_sk->[wr_returning_cdemo_sk];RF5 cd_marital_status->[cd_marital_status];RF6 cd_education_status->[cd_education_status] --------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cd1.cd_demo_sk = web_returns.wr_refunded_cdemo_sk)) otherCondition=((((((cd1.cd_marital_status = 'M') AND (cd1.cd_education_status = '4 yr Degree')) AND ((web_sales.ws_sales_price >= 100.00) AND (web_sales.ws_sales_price <= 150.00))) OR (((cd1.cd_marital_status = 'S') AND (cd1.cd_education_status = 'Secondary')) AND ((web_sales.ws_sales_price >= 50.00) AND (web_sales.ws_sales_price <= 100.00)))) OR (((cd1.cd_marital_status = 'W') AND (cd1.cd_education_status = 'Advanced Degree')) AND ((web_sales.ws_sales_price >= 150.00) AND (web_sales.ws_sales_price <= 200.00))))) build RFs:RF3 cd_demo_sk->[wr_refunded_cdemo_sk] +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cd1.cd_demo_sk = web_returns.wr_refunded_cdemo_sk)) otherCondition=(OR[AND[(cd1.cd_marital_status = 'M'),(cd1.cd_education_status = '4 yr Degree'),(web_sales.ws_sales_price >= 100.00),(web_sales.ws_sales_price <= 150.00)],AND[(cd1.cd_marital_status = 'S'),(cd1.cd_education_status = 'Secondary'),(web_sales.ws_sales_price >= 50.00),(web_sales.ws_sales_price <= 100.00)],AND[(cd1.cd_marital_status = 'W'),(cd1.cd_education_status = 'Advanced Degree'),(web_sales.ws_sales_price >= 150.00),(web_sales.ws_sales_price <= 200.00)]]) build RFs:RF3 cd_demo_sk->[wr_refunded_cdemo_sk] ------------------------------------PhysicalProject --------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_web_page_sk = web_page.wp_web_page_sk)) otherCondition=() build RFs:RF2 wp_web_page_sk->[ws_web_page_sk] ----------------------------------------PhysicalProject @@ -30,7 +30,7 @@ PhysicalResultSink ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[web_page] ------------------------------------PhysicalProject ---------------------------------------filter(((((cd1.cd_marital_status = 'M') AND (cd1.cd_education_status = '4 yr Degree')) OR ((cd1.cd_marital_status = 'S') AND (cd1.cd_education_status = 'Secondary'))) OR ((cd1.cd_marital_status = 'W') AND (cd1.cd_education_status = 'Advanced Degree'))) and cd_education_status IN ('4 yr Degree', 'Advanced Degree', 'Secondary') and cd_marital_status IN ('M', 'S', 'W')) +--------------------------------------filter(OR[AND[(cd1.cd_marital_status = 'M'),(cd1.cd_education_status = '4 yr Degree')],AND[(cd1.cd_marital_status = 'S'),(cd1.cd_education_status = 'Secondary')],AND[(cd1.cd_marital_status = 'W'),(cd1.cd_education_status = 'Advanced Degree')]] and cd_education_status IN ('4 yr Degree', 'Advanced Degree', 'Secondary') and cd_marital_status IN ('M', 'S', 'W')) ----------------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF5 RF6 --------------------------------PhysicalProject ----------------------------------filter(cd_education_status IN ('4 yr Degree', 'Advanced Degree', 'Secondary') and cd_marital_status IN ('M', 'S', 'W')) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query88.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query88.out index dc879d6daa5a4b..a20acf339431ea 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query88.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query88.out @@ -20,7 +20,7 @@ PhysicalResultSink ----------------------------------PhysicalProject ------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF21 RF22 RF23 ----------------------------------PhysicalProject -------------------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +------------------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) --------------------------------------PhysicalOlapScan[household_demographics] ------------------------------PhysicalProject --------------------------------filter((time_dim.t_hour = 8) and (time_dim.t_minute >= 30)) @@ -40,7 +40,7 @@ PhysicalResultSink ----------------------------------PhysicalProject ------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF18 RF19 RF20 ----------------------------------PhysicalProject -------------------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +------------------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) --------------------------------------PhysicalOlapScan[household_demographics] ------------------------------PhysicalProject --------------------------------filter((time_dim.t_hour = 9) and (time_dim.t_minute < 30)) @@ -60,7 +60,7 @@ PhysicalResultSink --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF15 RF16 RF17 --------------------------------PhysicalProject -----------------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +----------------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) ------------------------------------PhysicalOlapScan[household_demographics] ----------------------------PhysicalProject ------------------------------filter((time_dim.t_hour = 9) and (time_dim.t_minute >= 30)) @@ -80,7 +80,7 @@ PhysicalResultSink ------------------------------PhysicalProject --------------------------------PhysicalOlapScan[store_sales] apply RFs: RF12 RF13 RF14 ------------------------------PhysicalProject ---------------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +--------------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) ----------------------------------PhysicalOlapScan[household_demographics] --------------------------PhysicalProject ----------------------------filter((time_dim.t_hour = 10) and (time_dim.t_minute < 30)) @@ -100,7 +100,7 @@ PhysicalResultSink ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[store_sales] apply RFs: RF9 RF10 RF11 ----------------------------PhysicalProject -------------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +------------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) --------------------------------PhysicalOlapScan[household_demographics] ------------------------PhysicalProject --------------------------filter((time_dim.t_hour = 10) and (time_dim.t_minute >= 30)) @@ -120,7 +120,7 @@ PhysicalResultSink --------------------------PhysicalProject ----------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 RF7 RF8 --------------------------PhysicalProject -----------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +----------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) ------------------------------PhysicalOlapScan[household_demographics] ----------------------PhysicalProject ------------------------filter((time_dim.t_hour = 11) and (time_dim.t_minute < 30)) @@ -140,7 +140,7 @@ PhysicalResultSink ------------------------PhysicalProject --------------------------PhysicalOlapScan[store_sales] apply RFs: RF3 RF4 RF5 ------------------------PhysicalProject ---------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +--------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) ----------------------------PhysicalOlapScan[household_demographics] --------------------PhysicalProject ----------------------filter((time_dim.t_hour = 11) and (time_dim.t_minute >= 30)) @@ -160,7 +160,7 @@ PhysicalResultSink ----------------------PhysicalProject ------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 ----------------------PhysicalProject -------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) --------------------------PhysicalOlapScan[household_demographics] ------------------PhysicalProject --------------------filter((time_dim.t_hour = 12) and (time_dim.t_minute < 30)) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query89.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query89.out index 661b47d0ea980a..e4d2ae3435f174 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query89.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query89.out @@ -23,7 +23,7 @@ PhysicalResultSink ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 ----------------------------------------PhysicalProject -------------------------------------------filter(((i_category IN ('Electronics', 'Jewelry', 'Shoes') AND i_class IN ('athletic', 'portable', 'semi-precious')) OR (i_category IN ('Men', 'Music', 'Women') AND i_class IN ('accessories', 'maternity', 'rock'))) and i_category IN ('Electronics', 'Jewelry', 'Men', 'Music', 'Shoes', 'Women') and i_class IN ('accessories', 'athletic', 'maternity', 'portable', 'rock', 'semi-precious')) +------------------------------------------filter(OR[AND[i_category IN ('Electronics', 'Jewelry', 'Shoes'),i_class IN ('athletic', 'portable', 'semi-precious')],AND[i_category IN ('Men', 'Music', 'Women'),i_class IN ('accessories', 'maternity', 'rock')]] and i_category IN ('Electronics', 'Jewelry', 'Men', 'Music', 'Shoes', 'Women') and i_class IN ('accessories', 'athletic', 'maternity', 'portable', 'rock', 'semi-precious')) --------------------------------------------PhysicalOlapScan[item] ------------------------------------PhysicalProject --------------------------------------filter((date_dim.d_year = 1999)) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query91.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query91.out index c19b43be01a9df..a2e5d4a7660114 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query91.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query91.out @@ -33,7 +33,7 @@ PhysicalResultSink ------------------------------filter((customer_address.ca_gmt_offset = -6.00)) --------------------------------PhysicalOlapScan[customer_address] ------------------------PhysicalProject ---------------------------filter((((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = 'Unknown')) OR ((customer_demographics.cd_marital_status = 'W') AND (customer_demographics.cd_education_status = 'Advanced Degree'))) and cd_education_status IN ('Advanced Degree', 'Unknown') and cd_marital_status IN ('M', 'W')) +--------------------------filter(OR[AND[(customer_demographics.cd_marital_status = 'M'),(customer_demographics.cd_education_status = 'Unknown')],AND[(customer_demographics.cd_marital_status = 'W'),(customer_demographics.cd_education_status = 'Advanced Degree')]] and cd_education_status IN ('Advanced Degree', 'Unknown') and cd_marital_status IN ('M', 'W')) ----------------------------PhysicalOlapScan[customer_demographics] --------------------PhysicalProject ----------------------filter((hd_buy_potential like '1001-5000%')) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query10.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query10.out index 6027b8b2684b4b..4dfc2de4cf3fe5 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query10.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query10.out @@ -9,7 +9,7 @@ PhysicalResultSink ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) +------------------filter(OR[ifnull($c$1, FALSE),ifnull($c$2, FALSE)]) --------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() ----------------------PhysicalProject ------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query13.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query13.out index e92ddac6d97742..4b72ea24b2c114 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query13.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query13.out @@ -7,16 +7,16 @@ PhysicalResultSink --------PhysicalProject ----------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() ------------PhysicalProject ---------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)) otherCondition=((((((household_demographics.hd_dep_count = 1) AND cd_marital_status IN ('M', 'S')) AND cd_education_status IN ('4 yr Degree', 'College')) AND ((((customer_demographics.cd_marital_status = 'S') AND (customer_demographics.cd_education_status = 'College')) AND ((store_sales.ss_sales_price >= 50.00) AND (store_sales.ss_sales_price <= 100.00))) OR (((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = '4 yr Degree')) AND ((store_sales.ss_sales_price >= 150.00) AND (store_sales.ss_sales_price <= 200.00))))) OR ((((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = 'Unknown')) AND ((store_sales.ss_sales_price >= 100.00) AND (store_sales.ss_sales_price <= 150.00))) AND (household_demographics.hd_dep_count = 3)))) build RFs:RF3 ss_cdemo_sk->[cd_demo_sk] +--------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)) otherCondition=(OR[AND[(household_demographics.hd_dep_count = 1),cd_marital_status IN ('M', 'S'),cd_education_status IN ('4 yr Degree', 'College'),OR[AND[(customer_demographics.cd_marital_status = 'S'),(customer_demographics.cd_education_status = 'College'),(store_sales.ss_sales_price >= 50.00),(store_sales.ss_sales_price <= 100.00)],AND[(customer_demographics.cd_marital_status = 'M'),(customer_demographics.cd_education_status = '4 yr Degree'),(store_sales.ss_sales_price >= 150.00),(store_sales.ss_sales_price <= 200.00)]]],AND[(customer_demographics.cd_marital_status = 'D'),(customer_demographics.cd_education_status = 'Unknown'),(store_sales.ss_sales_price >= 100.00),(store_sales.ss_sales_price <= 150.00),(household_demographics.hd_dep_count = 3)]]) build RFs:RF3 ss_cdemo_sk->[cd_demo_sk] ----------------PhysicalProject -------------------filter(((((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = 'Unknown')) OR ((customer_demographics.cd_marital_status = 'S') AND (customer_demographics.cd_education_status = 'College'))) OR ((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = '4 yr Degree'))) and cd_education_status IN ('4 yr Degree', 'College', 'Unknown') and cd_marital_status IN ('D', 'M', 'S')) +------------------filter(OR[AND[(customer_demographics.cd_marital_status = 'D'),(customer_demographics.cd_education_status = 'Unknown')],AND[(customer_demographics.cd_marital_status = 'S'),(customer_demographics.cd_education_status = 'College')],AND[(customer_demographics.cd_marital_status = 'M'),(customer_demographics.cd_education_status = '4 yr Degree')]] and cd_education_status IN ('4 yr Degree', 'College', 'Unknown') and cd_marital_status IN ('D', 'M', 'S')) --------------------PhysicalOlapScan[customer_demographics] apply RFs: RF3 ----------------PhysicalProject ------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF2 hd_demo_sk->[ss_hdemo_sk] --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=((((ca_state IN ('KS', 'MI', 'SD') AND ((store_sales.ss_net_profit >= 100.00) AND (store_sales.ss_net_profit <= 200.00))) OR (ca_state IN ('CO', 'MO', 'ND') AND ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 300.00)))) OR (ca_state IN ('NH', 'OH', 'TX') AND ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 250.00))))) build RFs:RF0 ca_address_sk->[ss_addr_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=(OR[AND[ca_state IN ('KS', 'MI', 'SD'),(store_sales.ss_net_profit >= 100.00),(store_sales.ss_net_profit <= 200.00)],AND[ca_state IN ('CO', 'MO', 'ND'),(store_sales.ss_net_profit >= 150.00),(store_sales.ss_net_profit <= 300.00)],AND[ca_state IN ('NH', 'OH', 'TX'),(store_sales.ss_net_profit >= 50.00),(store_sales.ss_net_profit <= 250.00)]]) build RFs:RF0 ca_address_sk->[ss_addr_sk] ----------------------------PhysicalProject ------------------------------filter((store_sales.ss_net_profit <= 300.00) and (store_sales.ss_net_profit >= 50.00) and (store_sales.ss_sales_price <= 200.00) and (store_sales.ss_sales_price >= 50.00)) --------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query15.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query15.out index 685f61ffed3387..f5813cde38de7d 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query15.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query15.out @@ -8,7 +8,7 @@ PhysicalResultSink ----------PhysicalDistribute[DistributionSpecHash] ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=(((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR ca_state IN ('CA', 'GA', 'WA')) OR (catalog_sales.cs_sales_price > 500.00))) build RFs:RF2 c_customer_sk->[cs_bill_customer_sk] +----------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),ca_state IN ('CA', 'GA', 'WA'),(catalog_sales.cs_sales_price > 500.00)]) build RFs:RF2 c_customer_sk->[cs_bill_customer_sk] ------------------PhysicalProject --------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] ----------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query26.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query26.out index 2ee30b92ac642f..e2b0cdc5351213 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query26.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query26.out @@ -26,6 +26,6 @@ PhysicalResultSink ----------------------------filter((date_dim.d_year = 2001)) ------------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalProject -------------------------filter(((promotion.p_channel_email = 'N') OR (promotion.p_channel_event = 'N'))) +------------------------filter(OR[(promotion.p_channel_email = 'N'),(promotion.p_channel_event = 'N')]) --------------------------PhysicalOlapScan[promotion] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query28.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query28.out index dae7cb1c23a402..7a6bdd8868ef00 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query28.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query28.out @@ -17,41 +17,41 @@ PhysicalResultSink ----------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------hashAgg[LOCAL] --------------------------------PhysicalProject -----------------------------------filter(((((store_sales.ss_list_price >= 131.00) AND (store_sales.ss_list_price <= 141.00)) OR ((store_sales.ss_coupon_amt >= 16798.00) AND (store_sales.ss_coupon_amt <= 17798.00))) OR ((store_sales.ss_wholesale_cost >= 25.00) AND (store_sales.ss_wholesale_cost <= 45.00))) and (store_sales.ss_quantity <= 5) and (store_sales.ss_quantity >= 0)) +----------------------------------filter((store_sales.ss_quantity <= 5) and (store_sales.ss_quantity >= 0) and OR[AND[(store_sales.ss_list_price >= 131.00),(store_sales.ss_list_price <= 141.00)],AND[(store_sales.ss_coupon_amt >= 16798.00),(store_sales.ss_coupon_amt <= 17798.00)],AND[(store_sales.ss_wholesale_cost >= 25.00),(store_sales.ss_wholesale_cost <= 45.00)]]) ------------------------------------PhysicalOlapScan[store_sales] ------------------------PhysicalLimit[LOCAL] --------------------------hashAgg[GLOBAL] ----------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------hashAgg[LOCAL] --------------------------------PhysicalProject -----------------------------------filter(((((store_sales.ss_list_price >= 145.00) AND (store_sales.ss_list_price <= 155.00)) OR ((store_sales.ss_coupon_amt >= 14792.00) AND (store_sales.ss_coupon_amt <= 15792.00))) OR ((store_sales.ss_wholesale_cost >= 46.00) AND (store_sales.ss_wholesale_cost <= 66.00))) and (store_sales.ss_quantity <= 10) and (store_sales.ss_quantity >= 6)) +----------------------------------filter((store_sales.ss_quantity <= 10) and (store_sales.ss_quantity >= 6) and OR[AND[(store_sales.ss_list_price >= 145.00),(store_sales.ss_list_price <= 155.00)],AND[(store_sales.ss_coupon_amt >= 14792.00),(store_sales.ss_coupon_amt <= 15792.00)],AND[(store_sales.ss_wholesale_cost >= 46.00),(store_sales.ss_wholesale_cost <= 66.00)]]) ------------------------------------PhysicalOlapScan[store_sales] --------------------PhysicalLimit[LOCAL] ----------------------hashAgg[GLOBAL] ------------------------PhysicalDistribute[DistributionSpecGather] --------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------filter(((((store_sales.ss_list_price >= 150.00) AND (store_sales.ss_list_price <= 160.00)) OR ((store_sales.ss_coupon_amt >= 6600.00) AND (store_sales.ss_coupon_amt <= 7600.00))) OR ((store_sales.ss_wholesale_cost >= 9.00) AND (store_sales.ss_wholesale_cost <= 29.00))) and (store_sales.ss_quantity <= 15) and (store_sales.ss_quantity >= 11)) +------------------------------filter((store_sales.ss_quantity <= 15) and (store_sales.ss_quantity >= 11) and OR[AND[(store_sales.ss_list_price >= 150.00),(store_sales.ss_list_price <= 160.00)],AND[(store_sales.ss_coupon_amt >= 6600.00),(store_sales.ss_coupon_amt <= 7600.00)],AND[(store_sales.ss_wholesale_cost >= 9.00),(store_sales.ss_wholesale_cost <= 29.00)]]) --------------------------------PhysicalOlapScan[store_sales] ----------------PhysicalLimit[LOCAL] ------------------hashAgg[GLOBAL] --------------------PhysicalDistribute[DistributionSpecGather] ----------------------hashAgg[LOCAL] ------------------------PhysicalProject ---------------------------filter(((((store_sales.ss_list_price >= 91.00) AND (store_sales.ss_list_price <= 101.00)) OR ((store_sales.ss_coupon_amt >= 13493.00) AND (store_sales.ss_coupon_amt <= 14493.00))) OR ((store_sales.ss_wholesale_cost >= 36.00) AND (store_sales.ss_wholesale_cost <= 56.00))) and (store_sales.ss_quantity <= 20) and (store_sales.ss_quantity >= 16)) +--------------------------filter((store_sales.ss_quantity <= 20) and (store_sales.ss_quantity >= 16) and OR[AND[(store_sales.ss_list_price >= 91.00),(store_sales.ss_list_price <= 101.00)],AND[(store_sales.ss_coupon_amt >= 13493.00),(store_sales.ss_coupon_amt <= 14493.00)],AND[(store_sales.ss_wholesale_cost >= 36.00),(store_sales.ss_wholesale_cost <= 56.00)]]) ----------------------------PhysicalOlapScan[store_sales] ------------PhysicalLimit[LOCAL] --------------hashAgg[GLOBAL] ----------------PhysicalDistribute[DistributionSpecGather] ------------------hashAgg[LOCAL] --------------------PhysicalProject -----------------------filter(((((store_sales.ss_list_price >= 0.00) AND (store_sales.ss_list_price <= 10.00)) OR ((store_sales.ss_coupon_amt >= 7629.00) AND (store_sales.ss_coupon_amt <= 8629.00))) OR ((store_sales.ss_wholesale_cost >= 6.00) AND (store_sales.ss_wholesale_cost <= 26.00))) and (store_sales.ss_quantity <= 25) and (store_sales.ss_quantity >= 21)) +----------------------filter((store_sales.ss_quantity <= 25) and (store_sales.ss_quantity >= 21) and OR[AND[(store_sales.ss_list_price >= 0.00),(store_sales.ss_list_price <= 10.00)],AND[(store_sales.ss_coupon_amt >= 7629.00),(store_sales.ss_coupon_amt <= 8629.00)],AND[(store_sales.ss_wholesale_cost >= 6.00),(store_sales.ss_wholesale_cost <= 26.00)]]) ------------------------PhysicalOlapScan[store_sales] --------PhysicalLimit[LOCAL] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecGather] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------filter(((((store_sales.ss_list_price >= 89.00) AND (store_sales.ss_list_price <= 99.00)) OR ((store_sales.ss_coupon_amt >= 15257.00) AND (store_sales.ss_coupon_amt <= 16257.00))) OR ((store_sales.ss_wholesale_cost >= 31.00) AND (store_sales.ss_wholesale_cost <= 51.00))) and (store_sales.ss_quantity <= 30) and (store_sales.ss_quantity >= 26)) +------------------filter((store_sales.ss_quantity <= 30) and (store_sales.ss_quantity >= 26) and OR[AND[(store_sales.ss_list_price >= 89.00),(store_sales.ss_list_price <= 99.00)],AND[(store_sales.ss_coupon_amt >= 15257.00),(store_sales.ss_coupon_amt <= 16257.00)],AND[(store_sales.ss_wholesale_cost >= 31.00),(store_sales.ss_wholesale_cost <= 51.00)]]) --------------------PhysicalOlapScan[store_sales] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query34.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query34.out index bb286ee190e816..489abcd4c2e33c 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query34.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query34.out @@ -21,7 +21,7 @@ PhysicalResultSink --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 --------------------------------PhysicalProject -----------------------------------filter((((date_dim.d_dom >= 1) AND (date_dim.d_dom <= 3)) OR ((date_dim.d_dom >= 25) AND (date_dim.d_dom <= 28))) and d_year IN (1998, 1999, 2000)) +----------------------------------filter(OR[AND[(date_dim.d_dom >= 1),(date_dim.d_dom <= 3)],AND[(date_dim.d_dom >= 25),(date_dim.d_dom <= 28)]] and d_year IN (1998, 1999, 2000)) ------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalProject ------------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('0-500', '1001-5000')) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query35.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query35.out index 49d318314dee3a..dea7b62c38003d 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query35.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query35.out @@ -9,7 +9,7 @@ PhysicalResultSink ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) +------------------filter(OR[ifnull($c$1, FALSE),ifnull($c$2, FALSE)]) --------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() ----------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() ------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF5 ss_customer_sk->[c_customer_sk] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query41.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query41.out index c9c1ffff499741..3034a77fe0897a 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query41.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query41.out @@ -18,6 +18,6 @@ PhysicalResultSink ------------------------PhysicalDistribute[DistributionSpecHash] --------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------filter((((i_color IN ('aquamarine', 'blue', 'chartreuse', 'chiffon', 'dodger', 'gainsboro', 'tan', 'violet') AND i_units IN ('Bunch', 'Dozen', 'Each', 'Ounce', 'Oz', 'Pound', 'Ton', 'Tsp')) AND (((((((item.i_category = 'Women') AND i_color IN ('aquamarine', 'gainsboro')) AND i_units IN ('Dozen', 'Ounce')) AND i_size IN ('economy', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('chiffon', 'violet')) AND i_units IN ('Pound', 'Ton')) AND i_size IN ('extra large', 'small'))) OR ((((item.i_category = 'Men') AND i_color IN ('blue', 'chartreuse')) AND i_units IN ('Each', 'Oz')) AND i_size IN ('N/A', 'large'))) OR ((((item.i_category = 'Men') AND i_color IN ('dodger', 'tan')) AND i_units IN ('Bunch', 'Tsp')) AND i_size IN ('economy', 'medium')))) OR ((i_color IN ('almond', 'blanched', 'indian', 'lime', 'peru', 'saddle', 'spring', 'tomato') AND i_units IN ('Box', 'Carton', 'Case', 'Dram', 'Gram', 'Pallet', 'Tbl', 'Unknown')) AND (((((((item.i_category = 'Women') AND i_color IN ('blanched', 'tomato')) AND i_units IN ('Case', 'Tbl')) AND i_size IN ('economy', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('almond', 'lime')) AND i_units IN ('Box', 'Dram')) AND i_size IN ('extra large', 'small'))) OR ((((item.i_category = 'Men') AND i_color IN ('peru', 'saddle')) AND i_units IN ('Gram', 'Pallet')) AND i_size IN ('N/A', 'large'))) OR ((((item.i_category = 'Men') AND i_color IN ('indian', 'spring')) AND i_units IN ('Carton', 'Unknown')) AND i_size IN ('economy', 'medium'))))) and i_category IN ('Men', 'Women') and i_size IN ('N/A', 'economy', 'extra large', 'large', 'medium', 'small')) +------------------------------filter(OR[AND[i_color IN ('aquamarine', 'blue', 'chartreuse', 'chiffon', 'dodger', 'gainsboro', 'tan', 'violet'),i_units IN ('Bunch', 'Dozen', 'Each', 'Ounce', 'Oz', 'Pound', 'Ton', 'Tsp'),OR[AND[(item.i_category = 'Women'),i_color IN ('aquamarine', 'gainsboro'),i_units IN ('Dozen', 'Ounce'),i_size IN ('economy', 'medium')],AND[(item.i_category = 'Women'),i_color IN ('chiffon', 'violet'),i_units IN ('Pound', 'Ton'),i_size IN ('extra large', 'small')],AND[(item.i_category = 'Men'),i_color IN ('blue', 'chartreuse'),i_units IN ('Each', 'Oz'),i_size IN ('N/A', 'large')],AND[(item.i_category = 'Men'),i_color IN ('dodger', 'tan'),i_units IN ('Bunch', 'Tsp'),i_size IN ('economy', 'medium')]]],AND[i_color IN ('almond', 'blanched', 'indian', 'lime', 'peru', 'saddle', 'spring', 'tomato'),i_units IN ('Box', 'Carton', 'Case', 'Dram', 'Gram', 'Pallet', 'Tbl', 'Unknown'),OR[AND[(item.i_category = 'Women'),i_color IN ('blanched', 'tomato'),i_units IN ('Case', 'Tbl'),i_size IN ('economy', 'medium')],AND[(item.i_category = 'Women'),i_color IN ('almond', 'lime'),i_units IN ('Box', 'Dram'),i_size IN ('extra large', 'small')],AND[(item.i_category = 'Men'),i_color IN ('peru', 'saddle'),i_units IN ('Gram', 'Pallet'),i_size IN ('N/A', 'large')],AND[(item.i_category = 'Men'),i_color IN ('indian', 'spring'),i_units IN ('Carton', 'Unknown'),i_size IN ('economy', 'medium')]]]] and i_category IN ('Men', 'Women') and i_size IN ('N/A', 'economy', 'extra large', 'large', 'medium', 'small')) --------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query45.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query45.out index 6f437edc5cb1a9..baafdf90cdef7a 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query45.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query45.out @@ -8,7 +8,7 @@ PhysicalResultSink ----------PhysicalDistribute[DistributionSpecHash] ------------hashAgg[LOCAL] --------------PhysicalProject -----------------filter((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR $c$1)) +----------------filter(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),$c$1]) ------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ws_bill_customer_sk] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query46.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query46.out index 23864cbc2a5e14..63a6814934ecb4 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query46.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query46.out @@ -22,7 +22,7 @@ PhysicalResultSink ----------------------------------filter(d_dow IN (0, 6) and d_year IN (1999, 2000, 2001)) ------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalProject -------------------------------filter(((household_demographics.hd_dep_count = 6) OR (household_demographics.hd_vehicle_count = 0))) +------------------------------filter(OR[(household_demographics.hd_dep_count = 6),(household_demographics.hd_vehicle_count = 0)]) --------------------------------PhysicalOlapScan[household_demographics] ------------------------PhysicalProject --------------------------filter(s_city IN ('Centerville', 'Fairview', 'Five Points', 'Liberty', 'Oak Grove')) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query47.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query47.out index e115c8751281e0..048f93392595ec 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query47.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query47.out @@ -21,7 +21,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------------------------PhysicalProject --------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 ------------------------------------PhysicalProject ---------------------------------------filter((((date_dim.d_year = 2001) OR ((date_dim.d_year = 2000) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2002) AND (date_dim.d_moy = 1))) and d_year IN (2000, 2001, 2002)) +--------------------------------------filter(OR[(date_dim.d_year = 2001),AND[(date_dim.d_year = 2000),(date_dim.d_moy = 12)],AND[(date_dim.d_year = 2002),(date_dim.d_moy = 1)]] and d_year IN (2000, 2001, 2002)) ----------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query48.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query48.out index 476aa9dd8793af..a0f0e2107fcda4 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query48.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query48.out @@ -9,14 +9,14 @@ PhysicalResultSink ------------PhysicalProject --------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=((((ca_state IN ('IA', 'MD', 'MN') AND ((store_sales.ss_net_profit >= 0.00) AND (store_sales.ss_net_profit <= 2000.00))) OR (ca_state IN ('IL', 'TX', 'VA') AND ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 3000.00)))) OR (ca_state IN ('IN', 'MI', 'WI') AND ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 25000.00))))) build RFs:RF1 ca_address_sk->[ss_addr_sk] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=(OR[AND[ca_state IN ('IA', 'MD', 'MN'),(store_sales.ss_net_profit >= 0.00),(store_sales.ss_net_profit <= 2000.00)],AND[ca_state IN ('IL', 'TX', 'VA'),(store_sales.ss_net_profit >= 150.00),(store_sales.ss_net_profit <= 3000.00)],AND[ca_state IN ('IN', 'MI', 'WI'),(store_sales.ss_net_profit >= 50.00),(store_sales.ss_net_profit <= 25000.00)]]) build RFs:RF1 ca_address_sk->[ss_addr_sk] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)) otherCondition=((((((customer_demographics.cd_marital_status = 'U') AND (customer_demographics.cd_education_status = 'Primary')) AND ((store_sales.ss_sales_price >= 100.00) AND (store_sales.ss_sales_price <= 150.00))) OR (((customer_demographics.cd_marital_status = 'W') AND (customer_demographics.cd_education_status = 'College')) AND ((store_sales.ss_sales_price >= 50.00) AND (store_sales.ss_sales_price <= 100.00)))) OR (((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = '2 yr Degree')) AND ((store_sales.ss_sales_price >= 150.00) AND (store_sales.ss_sales_price <= 200.00))))) build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)) otherCondition=(OR[AND[(customer_demographics.cd_marital_status = 'U'),(customer_demographics.cd_education_status = 'Primary'),(store_sales.ss_sales_price >= 100.00),(store_sales.ss_sales_price <= 150.00)],AND[(customer_demographics.cd_marital_status = 'W'),(customer_demographics.cd_education_status = 'College'),(store_sales.ss_sales_price >= 50.00),(store_sales.ss_sales_price <= 100.00)],AND[(customer_demographics.cd_marital_status = 'D'),(customer_demographics.cd_education_status = '2 yr Degree'),(store_sales.ss_sales_price >= 150.00),(store_sales.ss_sales_price <= 200.00)]]) build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] ------------------------PhysicalProject --------------------------filter((store_sales.ss_net_profit <= 25000.00) and (store_sales.ss_net_profit >= 0.00) and (store_sales.ss_sales_price <= 200.00) and (store_sales.ss_sales_price >= 50.00)) ----------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 ------------------------PhysicalProject ---------------------------filter(((((customer_demographics.cd_marital_status = 'U') AND (customer_demographics.cd_education_status = 'Primary')) OR ((customer_demographics.cd_marital_status = 'W') AND (customer_demographics.cd_education_status = 'College'))) OR ((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = '2 yr Degree'))) and cd_education_status IN ('2 yr Degree', 'College', 'Primary') and cd_marital_status IN ('D', 'U', 'W')) +--------------------------filter(OR[AND[(customer_demographics.cd_marital_status = 'U'),(customer_demographics.cd_education_status = 'Primary')],AND[(customer_demographics.cd_marital_status = 'W'),(customer_demographics.cd_education_status = 'College')],AND[(customer_demographics.cd_marital_status = 'D'),(customer_demographics.cd_education_status = '2 yr Degree')]] and cd_education_status IN ('2 yr Degree', 'College', 'Primary') and cd_marital_status IN ('D', 'U', 'W')) ----------------------------PhysicalOlapScan[customer_demographics] --------------------PhysicalProject ----------------------filter((customer_address.ca_country = 'United States') and ca_state IN ('IA', 'IL', 'IN', 'MD', 'MI', 'MN', 'TX', 'VA', 'WI')) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query49.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query49.out index e34962e0847b0d..7d0f6b1ce22d0f 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query49.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query49.out @@ -16,7 +16,7 @@ PhysicalResultSink --------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) ----------------------------------PhysicalWindow ------------------------------------PhysicalQuickSort[LOCAL_SORT] --------------------------------------PhysicalWindow @@ -48,7 +48,7 @@ PhysicalResultSink --------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) ----------------------------------PhysicalWindow ------------------------------------PhysicalQuickSort[LOCAL_SORT] --------------------------------------PhysicalWindow @@ -80,7 +80,7 @@ PhysicalResultSink --------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) ----------------------------------PhysicalWindow ------------------------------------PhysicalQuickSort[LOCAL_SORT] --------------------------------------PhysicalWindow diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query53.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query53.out index 93289fca48e4ef..89dc632eb527c4 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query53.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query53.out @@ -22,7 +22,7 @@ PhysicalResultSink --------------------------------------PhysicalProject ----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 --------------------------------------PhysicalProject -----------------------------------------filter((((i_category IN ('Books', 'Children', 'Electronics') AND i_class IN ('personal', 'portable', 'reference', 'self-help')) AND i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')) OR ((i_category IN ('Men', 'Music', 'Women') AND i_class IN ('accessories', 'classical', 'fragrances', 'pants')) AND i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1'))) and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) +----------------------------------------filter(OR[AND[i_category IN ('Books', 'Children', 'Electronics'),i_class IN ('personal', 'portable', 'reference', 'self-help'),i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')],AND[i_category IN ('Men', 'Music', 'Women'),i_class IN ('accessories', 'classical', 'fragrances', 'pants'),i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')]] and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) ------------------------------------------PhysicalOlapScan[item] ----------------------------------PhysicalProject ------------------------------------filter(d_month_seq IN (1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211)) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query57.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query57.out index 4a7de98eb56b50..4f23fac89cf958 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query57.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query57.out @@ -21,7 +21,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------------------------PhysicalProject --------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 ------------------------------------PhysicalProject ---------------------------------------filter((((date_dim.d_year = 1999) OR ((date_dim.d_year = 1998) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2000) AND (date_dim.d_moy = 1))) and d_year IN (1998, 1999, 2000)) +--------------------------------------filter(OR[(date_dim.d_year = 1999),AND[(date_dim.d_year = 1998),(date_dim.d_moy = 12)],AND[(date_dim.d_year = 2000),(date_dim.d_moy = 1)]] and d_year IN (1998, 1999, 2000)) ----------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query61.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query61.out index d034c5b43231d7..faf30604b86926 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query61.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query61.out @@ -29,7 +29,7 @@ PhysicalResultSink ------------------------------PhysicalProject --------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_promo_sk = promotion.p_promo_sk)) otherCondition=() build RFs:RF6 ss_promo_sk->[p_promo_sk] ----------------------------------PhysicalProject -------------------------------------filter((((promotion.p_channel_dmail = 'Y') OR (promotion.p_channel_email = 'Y')) OR (promotion.p_channel_tv = 'Y'))) +------------------------------------filter(OR[(promotion.p_channel_dmail = 'Y'),(promotion.p_channel_email = 'Y'),(promotion.p_channel_tv = 'Y')]) --------------------------------------PhysicalOlapScan[promotion] apply RFs: RF6 ----------------------------------PhysicalProject ------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF5 s_store_sk->[ss_store_sk] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query63.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query63.out index 3bd844ce72c196..9653f6c52199aa 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query63.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query63.out @@ -22,7 +22,7 @@ PhysicalResultSink --------------------------------------PhysicalProject ----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 --------------------------------------PhysicalProject -----------------------------------------filter((((i_category IN ('Books', 'Children', 'Electronics') AND i_class IN ('personal', 'portable', 'reference', 'self-help')) AND i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')) OR ((i_category IN ('Men', 'Music', 'Women') AND i_class IN ('accessories', 'classical', 'fragrances', 'pants')) AND i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1'))) and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) +----------------------------------------filter(OR[AND[i_category IN ('Books', 'Children', 'Electronics'),i_class IN ('personal', 'portable', 'reference', 'self-help'),i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')],AND[i_category IN ('Men', 'Music', 'Women'),i_class IN ('accessories', 'classical', 'fragrances', 'pants'),i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')]] and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) ------------------------------------------PhysicalOlapScan[item] ----------------------------------PhysicalProject ------------------------------------filter(d_month_seq IN (1181, 1182, 1183, 1184, 1185, 1186, 1187, 1188, 1189, 1190, 1191, 1192)) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query68.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query68.out index 8975dc78ab76e5..aa07d1b2a42d9b 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query68.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query68.out @@ -33,6 +33,6 @@ PhysicalResultSink ----------------------------------filter(s_city IN ('Five Points', 'Pleasant Hill')) ------------------------------------PhysicalOlapScan[store] ----------------------------PhysicalProject -------------------------------filter(((household_demographics.hd_dep_count = 8) OR (household_demographics.hd_vehicle_count = -1))) +------------------------------filter(OR[(household_demographics.hd_dep_count = 8),(household_demographics.hd_vehicle_count = -1)]) --------------------------------PhysicalOlapScan[household_demographics] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query7.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query7.out index 911daeee06f6a4..ad074d92dcc43c 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query7.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query7.out @@ -24,7 +24,7 @@ PhysicalResultSink ----------------------------filter((date_dim.d_year = 2001)) ------------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalProject -------------------------filter(((promotion.p_channel_email = 'N') OR (promotion.p_channel_event = 'N'))) +------------------------filter(OR[(promotion.p_channel_email = 'N'),(promotion.p_channel_event = 'N')]) --------------------------PhysicalOlapScan[promotion] ------------------PhysicalProject --------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query78.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query78.out index 30527883a49018..a6034ca86ac5c5 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query78.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query78.out @@ -5,7 +5,7 @@ PhysicalResultSink ----PhysicalDistribute[DistributionSpecGather] ------PhysicalTopN[LOCAL_SORT] --------PhysicalProject -----------filter(((coalesce(ws_qty, 0) > 0) OR (coalesce(cs_qty, 0) > 0))) +----------filter(OR[(coalesce(ws_qty, 0) > 0),(coalesce(cs_qty, 0) > 0)]) ------------hashJoin[LEFT_OUTER_JOIN colocated] hashCondition=((cs.cs_customer_sk = ss.ss_customer_sk) and (cs.cs_item_sk = ss.ss_item_sk) and (cs.cs_sold_year = ss.ss_sold_year)) otherCondition=() --------------PhysicalProject ----------------hashJoin[LEFT_OUTER_JOIN colocated] hashCondition=((ws.ws_customer_sk = ss.ss_customer_sk) and (ws.ws_item_sk = ss.ss_item_sk) and (ws.ws_sold_year = ss.ss_sold_year)) otherCondition=() diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query79.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query79.out index 0ceac85291e12d..f57418546e7fb9 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query79.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query79.out @@ -22,7 +22,7 @@ PhysicalResultSink ----------------------------------filter((date_dim.d_dow = 1) and d_year IN (1998, 1999, 2000)) ------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalProject -------------------------------filter(((household_demographics.hd_dep_count = 5) OR (household_demographics.hd_vehicle_count > 4))) +------------------------------filter(OR[(household_demographics.hd_dep_count = 5),(household_demographics.hd_vehicle_count > 4)]) --------------------------------PhysicalOlapScan[household_demographics] ------------------------PhysicalProject --------------------------filter((store.s_number_employees <= 295) and (store.s_number_employees >= 200)) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query85.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query85.out index d08017427d5c94..91ce7e962e317d 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query85.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query85.out @@ -18,12 +18,12 @@ PhysicalResultSink ------------------------------filter(cd_education_status IN ('4 yr Degree', 'Advanced Degree', 'Secondary') and cd_marital_status IN ('M', 'S', 'W')) --------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF5 RF6 RF7 ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cd1.cd_demo_sk = web_returns.wr_refunded_cdemo_sk)) otherCondition=((((((cd1.cd_marital_status = 'M') AND (cd1.cd_education_status = '4 yr Degree')) AND ((web_sales.ws_sales_price >= 100.00) AND (web_sales.ws_sales_price <= 150.00))) OR (((cd1.cd_marital_status = 'S') AND (cd1.cd_education_status = 'Secondary')) AND ((web_sales.ws_sales_price >= 50.00) AND (web_sales.ws_sales_price <= 100.00)))) OR (((cd1.cd_marital_status = 'W') AND (cd1.cd_education_status = 'Advanced Degree')) AND ((web_sales.ws_sales_price >= 150.00) AND (web_sales.ws_sales_price <= 200.00))))) build RFs:RF4 wr_refunded_cdemo_sk->[cd_demo_sk] +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cd1.cd_demo_sk = web_returns.wr_refunded_cdemo_sk)) otherCondition=(OR[AND[(cd1.cd_marital_status = 'M'),(cd1.cd_education_status = '4 yr Degree'),(web_sales.ws_sales_price >= 100.00),(web_sales.ws_sales_price <= 150.00)],AND[(cd1.cd_marital_status = 'S'),(cd1.cd_education_status = 'Secondary'),(web_sales.ws_sales_price >= 50.00),(web_sales.ws_sales_price <= 100.00)],AND[(cd1.cd_marital_status = 'W'),(cd1.cd_education_status = 'Advanced Degree'),(web_sales.ws_sales_price >= 150.00),(web_sales.ws_sales_price <= 200.00)]]) build RFs:RF4 wr_refunded_cdemo_sk->[cd_demo_sk] --------------------------------PhysicalProject -----------------------------------filter(((((cd1.cd_marital_status = 'M') AND (cd1.cd_education_status = '4 yr Degree')) OR ((cd1.cd_marital_status = 'S') AND (cd1.cd_education_status = 'Secondary'))) OR ((cd1.cd_marital_status = 'W') AND (cd1.cd_education_status = 'Advanced Degree'))) and cd_education_status IN ('4 yr Degree', 'Advanced Degree', 'Secondary') and cd_marital_status IN ('M', 'S', 'W')) +----------------------------------filter(OR[AND[(cd1.cd_marital_status = 'M'),(cd1.cd_education_status = '4 yr Degree')],AND[(cd1.cd_marital_status = 'S'),(cd1.cd_education_status = 'Secondary')],AND[(cd1.cd_marital_status = 'W'),(cd1.cd_education_status = 'Advanced Degree')]] and cd_education_status IN ('4 yr Degree', 'Advanced Degree', 'Secondary') and cd_marital_status IN ('M', 'S', 'W')) ------------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF4 --------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((customer_address.ca_address_sk = web_returns.wr_refunded_addr_sk)) otherCondition=((((ca_state IN ('DE', 'FL', 'TX') AND ((web_sales.ws_net_profit >= 100.00) AND (web_sales.ws_net_profit <= 200.00))) OR (ca_state IN ('ID', 'IN', 'ND') AND ((web_sales.ws_net_profit >= 150.00) AND (web_sales.ws_net_profit <= 300.00)))) OR (ca_state IN ('IL', 'MT', 'OH') AND ((web_sales.ws_net_profit >= 50.00) AND (web_sales.ws_net_profit <= 250.00))))) build RFs:RF3 wr_refunded_addr_sk->[ca_address_sk] +----------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((customer_address.ca_address_sk = web_returns.wr_refunded_addr_sk)) otherCondition=(OR[AND[ca_state IN ('DE', 'FL', 'TX'),(web_sales.ws_net_profit >= 100.00),(web_sales.ws_net_profit <= 200.00)],AND[ca_state IN ('ID', 'IN', 'ND'),(web_sales.ws_net_profit >= 150.00),(web_sales.ws_net_profit <= 300.00)],AND[ca_state IN ('IL', 'MT', 'OH'),(web_sales.ws_net_profit >= 50.00),(web_sales.ws_net_profit <= 250.00)]]) build RFs:RF3 wr_refunded_addr_sk->[ca_address_sk] ------------------------------------PhysicalProject --------------------------------------filter((customer_address.ca_country = 'United States') and ca_state IN ('DE', 'FL', 'ID', 'IL', 'IN', 'MT', 'ND', 'OH', 'TX')) ----------------------------------------PhysicalOlapScan[customer_address] apply RFs: RF3 diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query88.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query88.out index bca8b0d069014b..09c630cef443ef 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query88.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query88.out @@ -23,7 +23,7 @@ PhysicalResultSink ------------------------------------filter((time_dim.t_hour = 8) and (time_dim.t_minute >= 30)) --------------------------------------PhysicalOlapScan[time_dim] ------------------------------PhysicalProject ---------------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +--------------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) ----------------------------------PhysicalOlapScan[household_demographics] --------------------------PhysicalProject ----------------------------filter((store.s_store_name = 'ese')) @@ -43,7 +43,7 @@ PhysicalResultSink ------------------------------------filter((time_dim.t_hour = 9) and (time_dim.t_minute < 30)) --------------------------------------PhysicalOlapScan[time_dim] ------------------------------PhysicalProject ---------------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +--------------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) ----------------------------------PhysicalOlapScan[household_demographics] --------------------------PhysicalProject ----------------------------filter((store.s_store_name = 'ese')) @@ -63,7 +63,7 @@ PhysicalResultSink ----------------------------------filter((time_dim.t_hour = 9) and (time_dim.t_minute >= 30)) ------------------------------------PhysicalOlapScan[time_dim] ----------------------------PhysicalProject -------------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +------------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) --------------------------------PhysicalOlapScan[household_demographics] ------------------------PhysicalProject --------------------------filter((store.s_store_name = 'ese')) @@ -83,7 +83,7 @@ PhysicalResultSink --------------------------------filter((time_dim.t_hour = 10) and (time_dim.t_minute < 30)) ----------------------------------PhysicalOlapScan[time_dim] --------------------------PhysicalProject -----------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +----------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) ------------------------------PhysicalOlapScan[household_demographics] ----------------------PhysicalProject ------------------------filter((store.s_store_name = 'ese')) @@ -103,7 +103,7 @@ PhysicalResultSink ------------------------------filter((time_dim.t_hour = 10) and (time_dim.t_minute >= 30)) --------------------------------PhysicalOlapScan[time_dim] ------------------------PhysicalProject ---------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +--------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) ----------------------------PhysicalOlapScan[household_demographics] --------------------PhysicalProject ----------------------filter((store.s_store_name = 'ese')) @@ -123,7 +123,7 @@ PhysicalResultSink ----------------------------filter((time_dim.t_hour = 11) and (time_dim.t_minute < 30)) ------------------------------PhysicalOlapScan[time_dim] ----------------------PhysicalProject -------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) --------------------------PhysicalOlapScan[household_demographics] ------------------PhysicalProject --------------------filter((store.s_store_name = 'ese')) @@ -143,7 +143,7 @@ PhysicalResultSink --------------------------filter((time_dim.t_hour = 11) and (time_dim.t_minute >= 30)) ----------------------------PhysicalOlapScan[time_dim] --------------------PhysicalProject -----------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +----------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) ------------------------PhysicalOlapScan[household_demographics] ----------------PhysicalProject ------------------filter((store.s_store_name = 'ese')) @@ -163,7 +163,7 @@ PhysicalResultSink ------------------------filter((time_dim.t_hour = 12) and (time_dim.t_minute < 30)) --------------------------PhysicalOlapScan[time_dim] ------------------PhysicalProject ---------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +--------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) ----------------------PhysicalOlapScan[household_demographics] --------------PhysicalProject ----------------filter((store.s_store_name = 'ese')) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query89.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query89.out index c709c4df5fe097..2e9294bce91fd2 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query89.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query89.out @@ -23,7 +23,7 @@ PhysicalResultSink ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 ----------------------------------------PhysicalProject -------------------------------------------filter(((i_category IN ('Electronics', 'Jewelry', 'Shoes') AND i_class IN ('athletic', 'portable', 'semi-precious')) OR (i_category IN ('Men', 'Music', 'Women') AND i_class IN ('accessories', 'maternity', 'rock'))) and i_category IN ('Electronics', 'Jewelry', 'Men', 'Music', 'Shoes', 'Women') and i_class IN ('accessories', 'athletic', 'maternity', 'portable', 'rock', 'semi-precious')) +------------------------------------------filter(OR[AND[i_category IN ('Electronics', 'Jewelry', 'Shoes'),i_class IN ('athletic', 'portable', 'semi-precious')],AND[i_category IN ('Men', 'Music', 'Women'),i_class IN ('accessories', 'maternity', 'rock')]] and i_category IN ('Electronics', 'Jewelry', 'Men', 'Music', 'Shoes', 'Women') and i_class IN ('accessories', 'athletic', 'maternity', 'portable', 'rock', 'semi-precious')) --------------------------------------------PhysicalOlapScan[item] ------------------------------------PhysicalProject --------------------------------------filter((date_dim.d_year = 1999)) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query91.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query91.out index e2a4ed89af442d..6593bb83f07994 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query91.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query91.out @@ -28,7 +28,7 @@ PhysicalResultSink ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 RF1 ----------------------------------------PhysicalProject -------------------------------------------filter((((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = 'Unknown')) OR ((customer_demographics.cd_marital_status = 'W') AND (customer_demographics.cd_education_status = 'Advanced Degree'))) and cd_education_status IN ('Advanced Degree', 'Unknown') and cd_marital_status IN ('M', 'W')) +------------------------------------------filter(OR[AND[(customer_demographics.cd_marital_status = 'M'),(customer_demographics.cd_education_status = 'Unknown')],AND[(customer_demographics.cd_marital_status = 'W'),(customer_demographics.cd_education_status = 'Advanced Degree')]] and cd_education_status IN ('Advanced Degree', 'Unknown') and cd_marital_status IN ('M', 'W')) --------------------------------------------PhysicalOlapScan[customer_demographics] ------------------------------------PhysicalProject --------------------------------------filter((hd_buy_potential like '1001-5000%')) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query10.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query10.out index 6027b8b2684b4b..4dfc2de4cf3fe5 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query10.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query10.out @@ -9,7 +9,7 @@ PhysicalResultSink ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) +------------------filter(OR[ifnull($c$1, FALSE),ifnull($c$2, FALSE)]) --------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() ----------------------PhysicalProject ------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query13.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query13.out index 7528312c6dae6b..e8a4d92c4f7bf1 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query13.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query13.out @@ -7,16 +7,16 @@ PhysicalResultSink --------PhysicalProject ----------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF4 s_store_sk->[ss_store_sk] ------------PhysicalProject ---------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)) otherCondition=((((((household_demographics.hd_dep_count = 1) AND cd_marital_status IN ('M', 'S')) AND cd_education_status IN ('4 yr Degree', 'College')) AND ((((customer_demographics.cd_marital_status = 'S') AND (customer_demographics.cd_education_status = 'College')) AND ((store_sales.ss_sales_price >= 50.00) AND (store_sales.ss_sales_price <= 100.00))) OR (((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = '4 yr Degree')) AND ((store_sales.ss_sales_price >= 150.00) AND (store_sales.ss_sales_price <= 200.00))))) OR ((((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = 'Unknown')) AND ((store_sales.ss_sales_price >= 100.00) AND (store_sales.ss_sales_price <= 150.00))) AND (household_demographics.hd_dep_count = 3)))) build RFs:RF3 ss_cdemo_sk->[cd_demo_sk] +--------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)) otherCondition=(OR[AND[(household_demographics.hd_dep_count = 1),cd_marital_status IN ('M', 'S'),cd_education_status IN ('4 yr Degree', 'College'),OR[AND[(customer_demographics.cd_marital_status = 'S'),(customer_demographics.cd_education_status = 'College'),(store_sales.ss_sales_price >= 50.00),(store_sales.ss_sales_price <= 100.00)],AND[(customer_demographics.cd_marital_status = 'M'),(customer_demographics.cd_education_status = '4 yr Degree'),(store_sales.ss_sales_price >= 150.00),(store_sales.ss_sales_price <= 200.00)]]],AND[(customer_demographics.cd_marital_status = 'D'),(customer_demographics.cd_education_status = 'Unknown'),(store_sales.ss_sales_price >= 100.00),(store_sales.ss_sales_price <= 150.00),(household_demographics.hd_dep_count = 3)]]) build RFs:RF3 ss_cdemo_sk->[cd_demo_sk] ----------------PhysicalProject -------------------filter(((((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = 'Unknown')) OR ((customer_demographics.cd_marital_status = 'S') AND (customer_demographics.cd_education_status = 'College'))) OR ((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = '4 yr Degree'))) and cd_education_status IN ('4 yr Degree', 'College', 'Unknown') and cd_marital_status IN ('D', 'M', 'S')) +------------------filter(OR[AND[(customer_demographics.cd_marital_status = 'D'),(customer_demographics.cd_education_status = 'Unknown')],AND[(customer_demographics.cd_marital_status = 'S'),(customer_demographics.cd_education_status = 'College')],AND[(customer_demographics.cd_marital_status = 'M'),(customer_demographics.cd_education_status = '4 yr Degree')]] and cd_education_status IN ('4 yr Degree', 'College', 'Unknown') and cd_marital_status IN ('D', 'M', 'S')) --------------------PhysicalOlapScan[customer_demographics] apply RFs: RF3 ----------------PhysicalProject ------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF2 hd_demo_sk->[ss_hdemo_sk] --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=((((ca_state IN ('KS', 'MI', 'SD') AND ((store_sales.ss_net_profit >= 100.00) AND (store_sales.ss_net_profit <= 200.00))) OR (ca_state IN ('CO', 'MO', 'ND') AND ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 300.00)))) OR (ca_state IN ('NH', 'OH', 'TX') AND ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 250.00))))) build RFs:RF0 ca_address_sk->[ss_addr_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=(OR[AND[ca_state IN ('KS', 'MI', 'SD'),(store_sales.ss_net_profit >= 100.00),(store_sales.ss_net_profit <= 200.00)],AND[ca_state IN ('CO', 'MO', 'ND'),(store_sales.ss_net_profit >= 150.00),(store_sales.ss_net_profit <= 300.00)],AND[ca_state IN ('NH', 'OH', 'TX'),(store_sales.ss_net_profit >= 50.00),(store_sales.ss_net_profit <= 250.00)]]) build RFs:RF0 ca_address_sk->[ss_addr_sk] ----------------------------PhysicalProject ------------------------------filter((store_sales.ss_net_profit <= 300.00) and (store_sales.ss_net_profit >= 50.00) and (store_sales.ss_sales_price <= 200.00) and (store_sales.ss_sales_price >= 50.00)) --------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF4 diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query15.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query15.out index 685f61ffed3387..f5813cde38de7d 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query15.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query15.out @@ -8,7 +8,7 @@ PhysicalResultSink ----------PhysicalDistribute[DistributionSpecHash] ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=(((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR ca_state IN ('CA', 'GA', 'WA')) OR (catalog_sales.cs_sales_price > 500.00))) build RFs:RF2 c_customer_sk->[cs_bill_customer_sk] +----------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),ca_state IN ('CA', 'GA', 'WA'),(catalog_sales.cs_sales_price > 500.00)]) build RFs:RF2 c_customer_sk->[cs_bill_customer_sk] ------------------PhysicalProject --------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] ----------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query26.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query26.out index 2ee30b92ac642f..e2b0cdc5351213 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query26.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query26.out @@ -26,6 +26,6 @@ PhysicalResultSink ----------------------------filter((date_dim.d_year = 2001)) ------------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalProject -------------------------filter(((promotion.p_channel_email = 'N') OR (promotion.p_channel_event = 'N'))) +------------------------filter(OR[(promotion.p_channel_email = 'N'),(promotion.p_channel_event = 'N')]) --------------------------PhysicalOlapScan[promotion] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query28.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query28.out index dae7cb1c23a402..7a6bdd8868ef00 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query28.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query28.out @@ -17,41 +17,41 @@ PhysicalResultSink ----------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------hashAgg[LOCAL] --------------------------------PhysicalProject -----------------------------------filter(((((store_sales.ss_list_price >= 131.00) AND (store_sales.ss_list_price <= 141.00)) OR ((store_sales.ss_coupon_amt >= 16798.00) AND (store_sales.ss_coupon_amt <= 17798.00))) OR ((store_sales.ss_wholesale_cost >= 25.00) AND (store_sales.ss_wholesale_cost <= 45.00))) and (store_sales.ss_quantity <= 5) and (store_sales.ss_quantity >= 0)) +----------------------------------filter((store_sales.ss_quantity <= 5) and (store_sales.ss_quantity >= 0) and OR[AND[(store_sales.ss_list_price >= 131.00),(store_sales.ss_list_price <= 141.00)],AND[(store_sales.ss_coupon_amt >= 16798.00),(store_sales.ss_coupon_amt <= 17798.00)],AND[(store_sales.ss_wholesale_cost >= 25.00),(store_sales.ss_wholesale_cost <= 45.00)]]) ------------------------------------PhysicalOlapScan[store_sales] ------------------------PhysicalLimit[LOCAL] --------------------------hashAgg[GLOBAL] ----------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------hashAgg[LOCAL] --------------------------------PhysicalProject -----------------------------------filter(((((store_sales.ss_list_price >= 145.00) AND (store_sales.ss_list_price <= 155.00)) OR ((store_sales.ss_coupon_amt >= 14792.00) AND (store_sales.ss_coupon_amt <= 15792.00))) OR ((store_sales.ss_wholesale_cost >= 46.00) AND (store_sales.ss_wholesale_cost <= 66.00))) and (store_sales.ss_quantity <= 10) and (store_sales.ss_quantity >= 6)) +----------------------------------filter((store_sales.ss_quantity <= 10) and (store_sales.ss_quantity >= 6) and OR[AND[(store_sales.ss_list_price >= 145.00),(store_sales.ss_list_price <= 155.00)],AND[(store_sales.ss_coupon_amt >= 14792.00),(store_sales.ss_coupon_amt <= 15792.00)],AND[(store_sales.ss_wholesale_cost >= 46.00),(store_sales.ss_wholesale_cost <= 66.00)]]) ------------------------------------PhysicalOlapScan[store_sales] --------------------PhysicalLimit[LOCAL] ----------------------hashAgg[GLOBAL] ------------------------PhysicalDistribute[DistributionSpecGather] --------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------filter(((((store_sales.ss_list_price >= 150.00) AND (store_sales.ss_list_price <= 160.00)) OR ((store_sales.ss_coupon_amt >= 6600.00) AND (store_sales.ss_coupon_amt <= 7600.00))) OR ((store_sales.ss_wholesale_cost >= 9.00) AND (store_sales.ss_wholesale_cost <= 29.00))) and (store_sales.ss_quantity <= 15) and (store_sales.ss_quantity >= 11)) +------------------------------filter((store_sales.ss_quantity <= 15) and (store_sales.ss_quantity >= 11) and OR[AND[(store_sales.ss_list_price >= 150.00),(store_sales.ss_list_price <= 160.00)],AND[(store_sales.ss_coupon_amt >= 6600.00),(store_sales.ss_coupon_amt <= 7600.00)],AND[(store_sales.ss_wholesale_cost >= 9.00),(store_sales.ss_wholesale_cost <= 29.00)]]) --------------------------------PhysicalOlapScan[store_sales] ----------------PhysicalLimit[LOCAL] ------------------hashAgg[GLOBAL] --------------------PhysicalDistribute[DistributionSpecGather] ----------------------hashAgg[LOCAL] ------------------------PhysicalProject ---------------------------filter(((((store_sales.ss_list_price >= 91.00) AND (store_sales.ss_list_price <= 101.00)) OR ((store_sales.ss_coupon_amt >= 13493.00) AND (store_sales.ss_coupon_amt <= 14493.00))) OR ((store_sales.ss_wholesale_cost >= 36.00) AND (store_sales.ss_wholesale_cost <= 56.00))) and (store_sales.ss_quantity <= 20) and (store_sales.ss_quantity >= 16)) +--------------------------filter((store_sales.ss_quantity <= 20) and (store_sales.ss_quantity >= 16) and OR[AND[(store_sales.ss_list_price >= 91.00),(store_sales.ss_list_price <= 101.00)],AND[(store_sales.ss_coupon_amt >= 13493.00),(store_sales.ss_coupon_amt <= 14493.00)],AND[(store_sales.ss_wholesale_cost >= 36.00),(store_sales.ss_wholesale_cost <= 56.00)]]) ----------------------------PhysicalOlapScan[store_sales] ------------PhysicalLimit[LOCAL] --------------hashAgg[GLOBAL] ----------------PhysicalDistribute[DistributionSpecGather] ------------------hashAgg[LOCAL] --------------------PhysicalProject -----------------------filter(((((store_sales.ss_list_price >= 0.00) AND (store_sales.ss_list_price <= 10.00)) OR ((store_sales.ss_coupon_amt >= 7629.00) AND (store_sales.ss_coupon_amt <= 8629.00))) OR ((store_sales.ss_wholesale_cost >= 6.00) AND (store_sales.ss_wholesale_cost <= 26.00))) and (store_sales.ss_quantity <= 25) and (store_sales.ss_quantity >= 21)) +----------------------filter((store_sales.ss_quantity <= 25) and (store_sales.ss_quantity >= 21) and OR[AND[(store_sales.ss_list_price >= 0.00),(store_sales.ss_list_price <= 10.00)],AND[(store_sales.ss_coupon_amt >= 7629.00),(store_sales.ss_coupon_amt <= 8629.00)],AND[(store_sales.ss_wholesale_cost >= 6.00),(store_sales.ss_wholesale_cost <= 26.00)]]) ------------------------PhysicalOlapScan[store_sales] --------PhysicalLimit[LOCAL] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecGather] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------filter(((((store_sales.ss_list_price >= 89.00) AND (store_sales.ss_list_price <= 99.00)) OR ((store_sales.ss_coupon_amt >= 15257.00) AND (store_sales.ss_coupon_amt <= 16257.00))) OR ((store_sales.ss_wholesale_cost >= 31.00) AND (store_sales.ss_wholesale_cost <= 51.00))) and (store_sales.ss_quantity <= 30) and (store_sales.ss_quantity >= 26)) +------------------filter((store_sales.ss_quantity <= 30) and (store_sales.ss_quantity >= 26) and OR[AND[(store_sales.ss_list_price >= 89.00),(store_sales.ss_list_price <= 99.00)],AND[(store_sales.ss_coupon_amt >= 15257.00),(store_sales.ss_coupon_amt <= 16257.00)],AND[(store_sales.ss_wholesale_cost >= 31.00),(store_sales.ss_wholesale_cost <= 51.00)]]) --------------------PhysicalOlapScan[store_sales] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query34.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query34.out index bb286ee190e816..489abcd4c2e33c 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query34.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query34.out @@ -21,7 +21,7 @@ PhysicalResultSink --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 --------------------------------PhysicalProject -----------------------------------filter((((date_dim.d_dom >= 1) AND (date_dim.d_dom <= 3)) OR ((date_dim.d_dom >= 25) AND (date_dim.d_dom <= 28))) and d_year IN (1998, 1999, 2000)) +----------------------------------filter(OR[AND[(date_dim.d_dom >= 1),(date_dim.d_dom <= 3)],AND[(date_dim.d_dom >= 25),(date_dim.d_dom <= 28)]] and d_year IN (1998, 1999, 2000)) ------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalProject ------------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('0-500', '1001-5000')) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query35.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query35.out index dcc86d24029059..9728d7d30707cd 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query35.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query35.out @@ -9,7 +9,7 @@ PhysicalResultSink ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) +------------------filter(OR[ifnull($c$1, FALSE),ifnull($c$2, FALSE)]) --------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() ----------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() ------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF5 ss_customer_sk->[c_customer_sk] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query41.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query41.out index c9c1ffff499741..3034a77fe0897a 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query41.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query41.out @@ -18,6 +18,6 @@ PhysicalResultSink ------------------------PhysicalDistribute[DistributionSpecHash] --------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------filter((((i_color IN ('aquamarine', 'blue', 'chartreuse', 'chiffon', 'dodger', 'gainsboro', 'tan', 'violet') AND i_units IN ('Bunch', 'Dozen', 'Each', 'Ounce', 'Oz', 'Pound', 'Ton', 'Tsp')) AND (((((((item.i_category = 'Women') AND i_color IN ('aquamarine', 'gainsboro')) AND i_units IN ('Dozen', 'Ounce')) AND i_size IN ('economy', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('chiffon', 'violet')) AND i_units IN ('Pound', 'Ton')) AND i_size IN ('extra large', 'small'))) OR ((((item.i_category = 'Men') AND i_color IN ('blue', 'chartreuse')) AND i_units IN ('Each', 'Oz')) AND i_size IN ('N/A', 'large'))) OR ((((item.i_category = 'Men') AND i_color IN ('dodger', 'tan')) AND i_units IN ('Bunch', 'Tsp')) AND i_size IN ('economy', 'medium')))) OR ((i_color IN ('almond', 'blanched', 'indian', 'lime', 'peru', 'saddle', 'spring', 'tomato') AND i_units IN ('Box', 'Carton', 'Case', 'Dram', 'Gram', 'Pallet', 'Tbl', 'Unknown')) AND (((((((item.i_category = 'Women') AND i_color IN ('blanched', 'tomato')) AND i_units IN ('Case', 'Tbl')) AND i_size IN ('economy', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('almond', 'lime')) AND i_units IN ('Box', 'Dram')) AND i_size IN ('extra large', 'small'))) OR ((((item.i_category = 'Men') AND i_color IN ('peru', 'saddle')) AND i_units IN ('Gram', 'Pallet')) AND i_size IN ('N/A', 'large'))) OR ((((item.i_category = 'Men') AND i_color IN ('indian', 'spring')) AND i_units IN ('Carton', 'Unknown')) AND i_size IN ('economy', 'medium'))))) and i_category IN ('Men', 'Women') and i_size IN ('N/A', 'economy', 'extra large', 'large', 'medium', 'small')) +------------------------------filter(OR[AND[i_color IN ('aquamarine', 'blue', 'chartreuse', 'chiffon', 'dodger', 'gainsboro', 'tan', 'violet'),i_units IN ('Bunch', 'Dozen', 'Each', 'Ounce', 'Oz', 'Pound', 'Ton', 'Tsp'),OR[AND[(item.i_category = 'Women'),i_color IN ('aquamarine', 'gainsboro'),i_units IN ('Dozen', 'Ounce'),i_size IN ('economy', 'medium')],AND[(item.i_category = 'Women'),i_color IN ('chiffon', 'violet'),i_units IN ('Pound', 'Ton'),i_size IN ('extra large', 'small')],AND[(item.i_category = 'Men'),i_color IN ('blue', 'chartreuse'),i_units IN ('Each', 'Oz'),i_size IN ('N/A', 'large')],AND[(item.i_category = 'Men'),i_color IN ('dodger', 'tan'),i_units IN ('Bunch', 'Tsp'),i_size IN ('economy', 'medium')]]],AND[i_color IN ('almond', 'blanched', 'indian', 'lime', 'peru', 'saddle', 'spring', 'tomato'),i_units IN ('Box', 'Carton', 'Case', 'Dram', 'Gram', 'Pallet', 'Tbl', 'Unknown'),OR[AND[(item.i_category = 'Women'),i_color IN ('blanched', 'tomato'),i_units IN ('Case', 'Tbl'),i_size IN ('economy', 'medium')],AND[(item.i_category = 'Women'),i_color IN ('almond', 'lime'),i_units IN ('Box', 'Dram'),i_size IN ('extra large', 'small')],AND[(item.i_category = 'Men'),i_color IN ('peru', 'saddle'),i_units IN ('Gram', 'Pallet'),i_size IN ('N/A', 'large')],AND[(item.i_category = 'Men'),i_color IN ('indian', 'spring'),i_units IN ('Carton', 'Unknown'),i_size IN ('economy', 'medium')]]]] and i_category IN ('Men', 'Women') and i_size IN ('N/A', 'economy', 'extra large', 'large', 'medium', 'small')) --------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query45.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query45.out index 6f437edc5cb1a9..baafdf90cdef7a 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query45.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query45.out @@ -8,7 +8,7 @@ PhysicalResultSink ----------PhysicalDistribute[DistributionSpecHash] ------------hashAgg[LOCAL] --------------PhysicalProject -----------------filter((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR $c$1)) +----------------filter(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),$c$1]) ------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ws_bill_customer_sk] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query46.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query46.out index 8279b1c53b5c39..136c96ba2856c9 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query46.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query46.out @@ -22,7 +22,7 @@ PhysicalResultSink ----------------------------------filter(d_dow IN (0, 6) and d_year IN (1999, 2000, 2001)) ------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalProject -------------------------------filter(((household_demographics.hd_dep_count = 6) OR (household_demographics.hd_vehicle_count = 0))) +------------------------------filter(OR[(household_demographics.hd_dep_count = 6),(household_demographics.hd_vehicle_count = 0)]) --------------------------------PhysicalOlapScan[household_demographics] ------------------------PhysicalProject --------------------------filter(s_city IN ('Centerville', 'Fairview', 'Five Points', 'Liberty', 'Oak Grove')) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query47.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query47.out index 7a668b06c433e5..ecc4777c22f664 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query47.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query47.out @@ -21,7 +21,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------------------------PhysicalProject --------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 ------------------------------------PhysicalProject ---------------------------------------filter((((date_dim.d_year = 2001) OR ((date_dim.d_year = 2000) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2002) AND (date_dim.d_moy = 1))) and d_year IN (2000, 2001, 2002)) +--------------------------------------filter(OR[(date_dim.d_year = 2001),AND[(date_dim.d_year = 2000),(date_dim.d_moy = 12)],AND[(date_dim.d_year = 2002),(date_dim.d_moy = 1)]] and d_year IN (2000, 2001, 2002)) ----------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query48.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query48.out index c81868cacca7a6..f8e2288f356010 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query48.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query48.out @@ -9,14 +9,14 @@ PhysicalResultSink ------------PhysicalProject --------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=((((ca_state IN ('IA', 'MD', 'MN') AND ((store_sales.ss_net_profit >= 0.00) AND (store_sales.ss_net_profit <= 2000.00))) OR (ca_state IN ('IL', 'TX', 'VA') AND ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 3000.00)))) OR (ca_state IN ('IN', 'MI', 'WI') AND ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 25000.00))))) build RFs:RF1 ca_address_sk->[ss_addr_sk] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=(OR[AND[ca_state IN ('IA', 'MD', 'MN'),(store_sales.ss_net_profit >= 0.00),(store_sales.ss_net_profit <= 2000.00)],AND[ca_state IN ('IL', 'TX', 'VA'),(store_sales.ss_net_profit >= 150.00),(store_sales.ss_net_profit <= 3000.00)],AND[ca_state IN ('IN', 'MI', 'WI'),(store_sales.ss_net_profit >= 50.00),(store_sales.ss_net_profit <= 25000.00)]]) build RFs:RF1 ca_address_sk->[ss_addr_sk] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)) otherCondition=((((((customer_demographics.cd_marital_status = 'U') AND (customer_demographics.cd_education_status = 'Primary')) AND ((store_sales.ss_sales_price >= 100.00) AND (store_sales.ss_sales_price <= 150.00))) OR (((customer_demographics.cd_marital_status = 'W') AND (customer_demographics.cd_education_status = 'College')) AND ((store_sales.ss_sales_price >= 50.00) AND (store_sales.ss_sales_price <= 100.00)))) OR (((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = '2 yr Degree')) AND ((store_sales.ss_sales_price >= 150.00) AND (store_sales.ss_sales_price <= 200.00))))) build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)) otherCondition=(OR[AND[(customer_demographics.cd_marital_status = 'U'),(customer_demographics.cd_education_status = 'Primary'),(store_sales.ss_sales_price >= 100.00),(store_sales.ss_sales_price <= 150.00)],AND[(customer_demographics.cd_marital_status = 'W'),(customer_demographics.cd_education_status = 'College'),(store_sales.ss_sales_price >= 50.00),(store_sales.ss_sales_price <= 100.00)],AND[(customer_demographics.cd_marital_status = 'D'),(customer_demographics.cd_education_status = '2 yr Degree'),(store_sales.ss_sales_price >= 150.00),(store_sales.ss_sales_price <= 200.00)]]) build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] ------------------------PhysicalProject --------------------------filter((store_sales.ss_net_profit <= 25000.00) and (store_sales.ss_net_profit >= 0.00) and (store_sales.ss_sales_price <= 200.00) and (store_sales.ss_sales_price >= 50.00)) ----------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 ------------------------PhysicalProject ---------------------------filter(((((customer_demographics.cd_marital_status = 'U') AND (customer_demographics.cd_education_status = 'Primary')) OR ((customer_demographics.cd_marital_status = 'W') AND (customer_demographics.cd_education_status = 'College'))) OR ((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = '2 yr Degree'))) and cd_education_status IN ('2 yr Degree', 'College', 'Primary') and cd_marital_status IN ('D', 'U', 'W')) +--------------------------filter(OR[AND[(customer_demographics.cd_marital_status = 'U'),(customer_demographics.cd_education_status = 'Primary')],AND[(customer_demographics.cd_marital_status = 'W'),(customer_demographics.cd_education_status = 'College')],AND[(customer_demographics.cd_marital_status = 'D'),(customer_demographics.cd_education_status = '2 yr Degree')]] and cd_education_status IN ('2 yr Degree', 'College', 'Primary') and cd_marital_status IN ('D', 'U', 'W')) ----------------------------PhysicalOlapScan[customer_demographics] --------------------PhysicalProject ----------------------filter((customer_address.ca_country = 'United States') and ca_state IN ('IA', 'IL', 'IN', 'MD', 'MI', 'MN', 'TX', 'VA', 'WI')) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query49.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query49.out index e34962e0847b0d..7d0f6b1ce22d0f 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query49.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query49.out @@ -16,7 +16,7 @@ PhysicalResultSink --------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) ----------------------------------PhysicalWindow ------------------------------------PhysicalQuickSort[LOCAL_SORT] --------------------------------------PhysicalWindow @@ -48,7 +48,7 @@ PhysicalResultSink --------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) ----------------------------------PhysicalWindow ------------------------------------PhysicalQuickSort[LOCAL_SORT] --------------------------------------PhysicalWindow @@ -80,7 +80,7 @@ PhysicalResultSink --------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) ----------------------------------PhysicalWindow ------------------------------------PhysicalQuickSort[LOCAL_SORT] --------------------------------------PhysicalWindow diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query53.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query53.out index 81e6091b663bf7..04920e65ac6894 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query53.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query53.out @@ -22,7 +22,7 @@ PhysicalResultSink --------------------------------------PhysicalProject ----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 --------------------------------------PhysicalProject -----------------------------------------filter((((i_category IN ('Books', 'Children', 'Electronics') AND i_class IN ('personal', 'portable', 'reference', 'self-help')) AND i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')) OR ((i_category IN ('Men', 'Music', 'Women') AND i_class IN ('accessories', 'classical', 'fragrances', 'pants')) AND i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1'))) and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) +----------------------------------------filter(OR[AND[i_category IN ('Books', 'Children', 'Electronics'),i_class IN ('personal', 'portable', 'reference', 'self-help'),i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')],AND[i_category IN ('Men', 'Music', 'Women'),i_class IN ('accessories', 'classical', 'fragrances', 'pants'),i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')]] and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) ------------------------------------------PhysicalOlapScan[item] ----------------------------------PhysicalProject ------------------------------------filter(d_month_seq IN (1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211)) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query57.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query57.out index a0157318139d88..2cab4f33e1358d 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query57.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query57.out @@ -21,7 +21,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------------------------PhysicalProject --------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 ------------------------------------PhysicalProject ---------------------------------------filter((((date_dim.d_year = 1999) OR ((date_dim.d_year = 1998) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2000) AND (date_dim.d_moy = 1))) and d_year IN (1998, 1999, 2000)) +--------------------------------------filter(OR[(date_dim.d_year = 1999),AND[(date_dim.d_year = 1998),(date_dim.d_moy = 12)],AND[(date_dim.d_year = 2000),(date_dim.d_moy = 1)]] and d_year IN (1998, 1999, 2000)) ----------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query61.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query61.out index d034c5b43231d7..faf30604b86926 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query61.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query61.out @@ -29,7 +29,7 @@ PhysicalResultSink ------------------------------PhysicalProject --------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_promo_sk = promotion.p_promo_sk)) otherCondition=() build RFs:RF6 ss_promo_sk->[p_promo_sk] ----------------------------------PhysicalProject -------------------------------------filter((((promotion.p_channel_dmail = 'Y') OR (promotion.p_channel_email = 'Y')) OR (promotion.p_channel_tv = 'Y'))) +------------------------------------filter(OR[(promotion.p_channel_dmail = 'Y'),(promotion.p_channel_email = 'Y'),(promotion.p_channel_tv = 'Y')]) --------------------------------------PhysicalOlapScan[promotion] apply RFs: RF6 ----------------------------------PhysicalProject ------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF5 s_store_sk->[ss_store_sk] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query63.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query63.out index 2cd49bd8c40786..d4fb4990da98b8 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query63.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query63.out @@ -22,7 +22,7 @@ PhysicalResultSink --------------------------------------PhysicalProject ----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 --------------------------------------PhysicalProject -----------------------------------------filter((((i_category IN ('Books', 'Children', 'Electronics') AND i_class IN ('personal', 'portable', 'reference', 'self-help')) AND i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')) OR ((i_category IN ('Men', 'Music', 'Women') AND i_class IN ('accessories', 'classical', 'fragrances', 'pants')) AND i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1'))) and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) +----------------------------------------filter(OR[AND[i_category IN ('Books', 'Children', 'Electronics'),i_class IN ('personal', 'portable', 'reference', 'self-help'),i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')],AND[i_category IN ('Men', 'Music', 'Women'),i_class IN ('accessories', 'classical', 'fragrances', 'pants'),i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')]] and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) ------------------------------------------PhysicalOlapScan[item] ----------------------------------PhysicalProject ------------------------------------filter(d_month_seq IN (1181, 1182, 1183, 1184, 1185, 1186, 1187, 1188, 1189, 1190, 1191, 1192)) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query68.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query68.out index 8975dc78ab76e5..aa07d1b2a42d9b 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query68.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query68.out @@ -33,6 +33,6 @@ PhysicalResultSink ----------------------------------filter(s_city IN ('Five Points', 'Pleasant Hill')) ------------------------------------PhysicalOlapScan[store] ----------------------------PhysicalProject -------------------------------filter(((household_demographics.hd_dep_count = 8) OR (household_demographics.hd_vehicle_count = -1))) +------------------------------filter(OR[(household_demographics.hd_dep_count = 8),(household_demographics.hd_vehicle_count = -1)]) --------------------------------PhysicalOlapScan[household_demographics] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query7.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query7.out index 516105a1c2b73f..e3c9dc2bfd0e67 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query7.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query7.out @@ -24,7 +24,7 @@ PhysicalResultSink ----------------------------filter((date_dim.d_year = 2001)) ------------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalProject -------------------------filter(((promotion.p_channel_email = 'N') OR (promotion.p_channel_event = 'N'))) +------------------------filter(OR[(promotion.p_channel_email = 'N'),(promotion.p_channel_event = 'N')]) --------------------------PhysicalOlapScan[promotion] ------------------PhysicalProject --------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query78.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query78.out index 30527883a49018..a6034ca86ac5c5 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query78.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query78.out @@ -5,7 +5,7 @@ PhysicalResultSink ----PhysicalDistribute[DistributionSpecGather] ------PhysicalTopN[LOCAL_SORT] --------PhysicalProject -----------filter(((coalesce(ws_qty, 0) > 0) OR (coalesce(cs_qty, 0) > 0))) +----------filter(OR[(coalesce(ws_qty, 0) > 0),(coalesce(cs_qty, 0) > 0)]) ------------hashJoin[LEFT_OUTER_JOIN colocated] hashCondition=((cs.cs_customer_sk = ss.ss_customer_sk) and (cs.cs_item_sk = ss.ss_item_sk) and (cs.cs_sold_year = ss.ss_sold_year)) otherCondition=() --------------PhysicalProject ----------------hashJoin[LEFT_OUTER_JOIN colocated] hashCondition=((ws.ws_customer_sk = ss.ss_customer_sk) and (ws.ws_item_sk = ss.ss_item_sk) and (ws.ws_sold_year = ss.ss_sold_year)) otherCondition=() diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query79.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query79.out index 4f8ac0ce4d7dea..974fb1da39b734 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query79.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query79.out @@ -22,7 +22,7 @@ PhysicalResultSink ----------------------------------filter((date_dim.d_dow = 1) and d_year IN (1998, 1999, 2000)) ------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalProject -------------------------------filter(((household_demographics.hd_dep_count = 5) OR (household_demographics.hd_vehicle_count > 4))) +------------------------------filter(OR[(household_demographics.hd_dep_count = 5),(household_demographics.hd_vehicle_count > 4)]) --------------------------------PhysicalOlapScan[household_demographics] ------------------------PhysicalProject --------------------------filter((store.s_number_employees <= 295) and (store.s_number_employees >= 200)) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query85.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query85.out index 8e543215ca9ce5..6818ffdb89889a 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query85.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query85.out @@ -18,12 +18,12 @@ PhysicalResultSink ------------------------------filter(cd_education_status IN ('4 yr Degree', 'Advanced Degree', 'Secondary') and cd_marital_status IN ('M', 'S', 'W')) --------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF5 RF6 RF7 ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cd1.cd_demo_sk = web_returns.wr_refunded_cdemo_sk)) otherCondition=((((((cd1.cd_marital_status = 'M') AND (cd1.cd_education_status = '4 yr Degree')) AND ((web_sales.ws_sales_price >= 100.00) AND (web_sales.ws_sales_price <= 150.00))) OR (((cd1.cd_marital_status = 'S') AND (cd1.cd_education_status = 'Secondary')) AND ((web_sales.ws_sales_price >= 50.00) AND (web_sales.ws_sales_price <= 100.00)))) OR (((cd1.cd_marital_status = 'W') AND (cd1.cd_education_status = 'Advanced Degree')) AND ((web_sales.ws_sales_price >= 150.00) AND (web_sales.ws_sales_price <= 200.00))))) build RFs:RF4 wr_refunded_cdemo_sk->[cd_demo_sk] +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cd1.cd_demo_sk = web_returns.wr_refunded_cdemo_sk)) otherCondition=(OR[AND[(cd1.cd_marital_status = 'M'),(cd1.cd_education_status = '4 yr Degree'),(web_sales.ws_sales_price >= 100.00),(web_sales.ws_sales_price <= 150.00)],AND[(cd1.cd_marital_status = 'S'),(cd1.cd_education_status = 'Secondary'),(web_sales.ws_sales_price >= 50.00),(web_sales.ws_sales_price <= 100.00)],AND[(cd1.cd_marital_status = 'W'),(cd1.cd_education_status = 'Advanced Degree'),(web_sales.ws_sales_price >= 150.00),(web_sales.ws_sales_price <= 200.00)]]) build RFs:RF4 wr_refunded_cdemo_sk->[cd_demo_sk] --------------------------------PhysicalProject -----------------------------------filter(((((cd1.cd_marital_status = 'M') AND (cd1.cd_education_status = '4 yr Degree')) OR ((cd1.cd_marital_status = 'S') AND (cd1.cd_education_status = 'Secondary'))) OR ((cd1.cd_marital_status = 'W') AND (cd1.cd_education_status = 'Advanced Degree'))) and cd_education_status IN ('4 yr Degree', 'Advanced Degree', 'Secondary') and cd_marital_status IN ('M', 'S', 'W')) +----------------------------------filter(OR[AND[(cd1.cd_marital_status = 'M'),(cd1.cd_education_status = '4 yr Degree')],AND[(cd1.cd_marital_status = 'S'),(cd1.cd_education_status = 'Secondary')],AND[(cd1.cd_marital_status = 'W'),(cd1.cd_education_status = 'Advanced Degree')]] and cd_education_status IN ('4 yr Degree', 'Advanced Degree', 'Secondary') and cd_marital_status IN ('M', 'S', 'W')) ------------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF4 --------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((customer_address.ca_address_sk = web_returns.wr_refunded_addr_sk)) otherCondition=((((ca_state IN ('DE', 'FL', 'TX') AND ((web_sales.ws_net_profit >= 100.00) AND (web_sales.ws_net_profit <= 200.00))) OR (ca_state IN ('ID', 'IN', 'ND') AND ((web_sales.ws_net_profit >= 150.00) AND (web_sales.ws_net_profit <= 300.00)))) OR (ca_state IN ('IL', 'MT', 'OH') AND ((web_sales.ws_net_profit >= 50.00) AND (web_sales.ws_net_profit <= 250.00))))) build RFs:RF3 wr_refunded_addr_sk->[ca_address_sk] +----------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((customer_address.ca_address_sk = web_returns.wr_refunded_addr_sk)) otherCondition=(OR[AND[ca_state IN ('DE', 'FL', 'TX'),(web_sales.ws_net_profit >= 100.00),(web_sales.ws_net_profit <= 200.00)],AND[ca_state IN ('ID', 'IN', 'ND'),(web_sales.ws_net_profit >= 150.00),(web_sales.ws_net_profit <= 300.00)],AND[ca_state IN ('IL', 'MT', 'OH'),(web_sales.ws_net_profit >= 50.00),(web_sales.ws_net_profit <= 250.00)]]) build RFs:RF3 wr_refunded_addr_sk->[ca_address_sk] ------------------------------------PhysicalProject --------------------------------------filter((customer_address.ca_country = 'United States') and ca_state IN ('DE', 'FL', 'ID', 'IL', 'IN', 'MT', 'ND', 'OH', 'TX')) ----------------------------------------PhysicalOlapScan[customer_address] apply RFs: RF3 diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query88.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query88.out index bca8b0d069014b..09c630cef443ef 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query88.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query88.out @@ -23,7 +23,7 @@ PhysicalResultSink ------------------------------------filter((time_dim.t_hour = 8) and (time_dim.t_minute >= 30)) --------------------------------------PhysicalOlapScan[time_dim] ------------------------------PhysicalProject ---------------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +--------------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) ----------------------------------PhysicalOlapScan[household_demographics] --------------------------PhysicalProject ----------------------------filter((store.s_store_name = 'ese')) @@ -43,7 +43,7 @@ PhysicalResultSink ------------------------------------filter((time_dim.t_hour = 9) and (time_dim.t_minute < 30)) --------------------------------------PhysicalOlapScan[time_dim] ------------------------------PhysicalProject ---------------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +--------------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) ----------------------------------PhysicalOlapScan[household_demographics] --------------------------PhysicalProject ----------------------------filter((store.s_store_name = 'ese')) @@ -63,7 +63,7 @@ PhysicalResultSink ----------------------------------filter((time_dim.t_hour = 9) and (time_dim.t_minute >= 30)) ------------------------------------PhysicalOlapScan[time_dim] ----------------------------PhysicalProject -------------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +------------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) --------------------------------PhysicalOlapScan[household_demographics] ------------------------PhysicalProject --------------------------filter((store.s_store_name = 'ese')) @@ -83,7 +83,7 @@ PhysicalResultSink --------------------------------filter((time_dim.t_hour = 10) and (time_dim.t_minute < 30)) ----------------------------------PhysicalOlapScan[time_dim] --------------------------PhysicalProject -----------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +----------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) ------------------------------PhysicalOlapScan[household_demographics] ----------------------PhysicalProject ------------------------filter((store.s_store_name = 'ese')) @@ -103,7 +103,7 @@ PhysicalResultSink ------------------------------filter((time_dim.t_hour = 10) and (time_dim.t_minute >= 30)) --------------------------------PhysicalOlapScan[time_dim] ------------------------PhysicalProject ---------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +--------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) ----------------------------PhysicalOlapScan[household_demographics] --------------------PhysicalProject ----------------------filter((store.s_store_name = 'ese')) @@ -123,7 +123,7 @@ PhysicalResultSink ----------------------------filter((time_dim.t_hour = 11) and (time_dim.t_minute < 30)) ------------------------------PhysicalOlapScan[time_dim] ----------------------PhysicalProject -------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) --------------------------PhysicalOlapScan[household_demographics] ------------------PhysicalProject --------------------filter((store.s_store_name = 'ese')) @@ -143,7 +143,7 @@ PhysicalResultSink --------------------------filter((time_dim.t_hour = 11) and (time_dim.t_minute >= 30)) ----------------------------PhysicalOlapScan[time_dim] --------------------PhysicalProject -----------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +----------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) ------------------------PhysicalOlapScan[household_demographics] ----------------PhysicalProject ------------------filter((store.s_store_name = 'ese')) @@ -163,7 +163,7 @@ PhysicalResultSink ------------------------filter((time_dim.t_hour = 12) and (time_dim.t_minute < 30)) --------------------------PhysicalOlapScan[time_dim] ------------------PhysicalProject ---------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +--------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) ----------------------PhysicalOlapScan[household_demographics] --------------PhysicalProject ----------------filter((store.s_store_name = 'ese')) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query89.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query89.out index 661b47d0ea980a..e4d2ae3435f174 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query89.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query89.out @@ -23,7 +23,7 @@ PhysicalResultSink ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 ----------------------------------------PhysicalProject -------------------------------------------filter(((i_category IN ('Electronics', 'Jewelry', 'Shoes') AND i_class IN ('athletic', 'portable', 'semi-precious')) OR (i_category IN ('Men', 'Music', 'Women') AND i_class IN ('accessories', 'maternity', 'rock'))) and i_category IN ('Electronics', 'Jewelry', 'Men', 'Music', 'Shoes', 'Women') and i_class IN ('accessories', 'athletic', 'maternity', 'portable', 'rock', 'semi-precious')) +------------------------------------------filter(OR[AND[i_category IN ('Electronics', 'Jewelry', 'Shoes'),i_class IN ('athletic', 'portable', 'semi-precious')],AND[i_category IN ('Men', 'Music', 'Women'),i_class IN ('accessories', 'maternity', 'rock')]] and i_category IN ('Electronics', 'Jewelry', 'Men', 'Music', 'Shoes', 'Women') and i_class IN ('accessories', 'athletic', 'maternity', 'portable', 'rock', 'semi-precious')) --------------------------------------------PhysicalOlapScan[item] ------------------------------------PhysicalProject --------------------------------------filter((date_dim.d_year = 1999)) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query91.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query91.out index 6f3970cbb29d50..6af6c014d0c298 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query91.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query91.out @@ -28,7 +28,7 @@ PhysicalResultSink ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 RF1 ----------------------------------------PhysicalProject -------------------------------------------filter((((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = 'Unknown')) OR ((customer_demographics.cd_marital_status = 'W') AND (customer_demographics.cd_education_status = 'Advanced Degree'))) and cd_education_status IN ('Advanced Degree', 'Unknown') and cd_marital_status IN ('M', 'W')) +------------------------------------------filter(OR[AND[(customer_demographics.cd_marital_status = 'M'),(customer_demographics.cd_education_status = 'Unknown')],AND[(customer_demographics.cd_marital_status = 'W'),(customer_demographics.cd_education_status = 'Advanced Degree')]] and cd_education_status IN ('Advanced Degree', 'Unknown') and cd_marital_status IN ('M', 'W')) --------------------------------------------PhysicalOlapScan[customer_demographics] ------------------------------------PhysicalProject --------------------------------------filter((hd_buy_potential like '1001-5000%')) diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query10.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query10.out index e1cace0be0ccec..43aef07e0be2c0 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query10.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query10.out @@ -9,7 +9,7 @@ PhysicalResultSink ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) +------------------filter(OR[ifnull($c$1, FALSE),ifnull($c$2, FALSE)]) --------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() ----------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() ------------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query13.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query13.out index 169daff229f9e5..862b1f5d64b5f6 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query13.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query13.out @@ -7,9 +7,9 @@ PhysicalResultSink --------PhysicalProject ----------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] ------------PhysicalProject ---------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=((((ca_state IN ('MI', 'OK', 'TX') AND ((store_sales.ss_net_profit >= 100.00) AND (store_sales.ss_net_profit <= 200.00))) OR (ca_state IN ('NC', 'OH', 'WA') AND ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 300.00)))) OR (ca_state IN ('FL', 'GA', 'MT') AND ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 250.00))))) build RFs:RF3 ca_address_sk->[ss_addr_sk] +--------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=(OR[AND[ca_state IN ('MI', 'OK', 'TX'),(store_sales.ss_net_profit >= 100.00),(store_sales.ss_net_profit <= 200.00)],AND[ca_state IN ('NC', 'OH', 'WA'),(store_sales.ss_net_profit >= 150.00),(store_sales.ss_net_profit <= 300.00)],AND[ca_state IN ('FL', 'GA', 'MT'),(store_sales.ss_net_profit >= 50.00),(store_sales.ss_net_profit <= 250.00)]]) build RFs:RF3 ca_address_sk->[ss_addr_sk] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=((((((household_demographics.hd_dep_count = 1) AND cd_marital_status IN ('D', 'W')) AND cd_education_status IN ('College', 'Primary')) AND ((((customer_demographics.cd_marital_status = 'W') AND (customer_demographics.cd_education_status = 'College')) AND ((store_sales.ss_sales_price >= 50.00) AND (store_sales.ss_sales_price <= 100.00))) OR (((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = 'Primary')) AND ((store_sales.ss_sales_price >= 150.00) AND (store_sales.ss_sales_price <= 200.00))))) OR ((((customer_demographics.cd_marital_status = 'U') AND (customer_demographics.cd_education_status = 'Secondary')) AND ((store_sales.ss_sales_price >= 100.00) AND (store_sales.ss_sales_price <= 150.00))) AND (household_demographics.hd_dep_count = 3)))) build RFs:RF2 hd_demo_sk->[ss_hdemo_sk] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=(OR[AND[(household_demographics.hd_dep_count = 1),cd_marital_status IN ('D', 'W'),cd_education_status IN ('College', 'Primary'),OR[AND[(customer_demographics.cd_marital_status = 'W'),(customer_demographics.cd_education_status = 'College'),(store_sales.ss_sales_price >= 50.00),(store_sales.ss_sales_price <= 100.00)],AND[(customer_demographics.cd_marital_status = 'D'),(customer_demographics.cd_education_status = 'Primary'),(store_sales.ss_sales_price >= 150.00),(store_sales.ss_sales_price <= 200.00)]]],AND[(customer_demographics.cd_marital_status = 'U'),(customer_demographics.cd_education_status = 'Secondary'),(store_sales.ss_sales_price >= 100.00),(store_sales.ss_sales_price <= 150.00),(household_demographics.hd_dep_count = 3)]]) build RFs:RF2 hd_demo_sk->[ss_hdemo_sk] --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)) otherCondition=() build RFs:RF1 cd_demo_sk->[ss_cdemo_sk] ------------------------PhysicalProject @@ -20,7 +20,7 @@ PhysicalResultSink ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[store] ------------------------PhysicalProject ---------------------------filter(((((customer_demographics.cd_marital_status = 'U') AND (customer_demographics.cd_education_status = 'Secondary')) OR ((customer_demographics.cd_marital_status = 'W') AND (customer_demographics.cd_education_status = 'College'))) OR ((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = 'Primary'))) and cd_education_status IN ('College', 'Primary', 'Secondary') and cd_marital_status IN ('D', 'U', 'W')) +--------------------------filter(OR[AND[(customer_demographics.cd_marital_status = 'U'),(customer_demographics.cd_education_status = 'Secondary')],AND[(customer_demographics.cd_marital_status = 'W'),(customer_demographics.cd_education_status = 'College')],AND[(customer_demographics.cd_marital_status = 'D'),(customer_demographics.cd_education_status = 'Primary')]] and cd_education_status IN ('College', 'Primary', 'Secondary') and cd_marital_status IN ('D', 'U', 'W')) ----------------------------PhysicalOlapScan[customer_demographics] --------------------PhysicalProject ----------------------filter(hd_dep_count IN (1, 3)) diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query15.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query15.out index 31789e71f5b27a..828af0129d7187 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query15.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query15.out @@ -10,7 +10,7 @@ PhysicalResultSink --------------PhysicalProject ----------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=(((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR ca_state IN ('CA', 'GA', 'WA')) OR (catalog_sales.cs_sales_price > 500.00))) build RFs:RF1 ca_address_sk->[c_current_addr_sk] +--------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),ca_state IN ('CA', 'GA', 'WA'),(catalog_sales.cs_sales_price > 500.00)]) build RFs:RF1 ca_address_sk->[c_current_addr_sk] ----------------------PhysicalProject ------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF0 c_customer_sk->[cs_bill_customer_sk] --------------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query26.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query26.out index 68a8944b8ce179..2b5d23c4aac26d 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query26.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query26.out @@ -26,6 +26,6 @@ PhysicalResultSink ------------------------filter((date_dim.d_year = 1998)) --------------------------PhysicalOlapScan[date_dim] ------------------PhysicalProject ---------------------filter(((promotion.p_channel_email = 'N') OR (promotion.p_channel_event = 'N'))) +--------------------filter(OR[(promotion.p_channel_email = 'N'),(promotion.p_channel_event = 'N')]) ----------------------PhysicalOlapScan[promotion] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query28.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query28.out index eb43c81cde41c2..fd2a30be06e564 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query28.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query28.out @@ -17,41 +17,41 @@ PhysicalResultSink ----------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------hashAgg[LOCAL] --------------------------------PhysicalProject -----------------------------------filter(((((store_sales.ss_list_price >= 73.00) AND (store_sales.ss_list_price <= 83.00)) OR ((store_sales.ss_coupon_amt >= 7826.00) AND (store_sales.ss_coupon_amt <= 8826.00))) OR ((store_sales.ss_wholesale_cost >= 70.00) AND (store_sales.ss_wholesale_cost <= 90.00))) and (store_sales.ss_quantity <= 5) and (store_sales.ss_quantity >= 0)) +----------------------------------filter((store_sales.ss_quantity <= 5) and (store_sales.ss_quantity >= 0) and OR[AND[(store_sales.ss_list_price >= 73.00),(store_sales.ss_list_price <= 83.00)],AND[(store_sales.ss_coupon_amt >= 7826.00),(store_sales.ss_coupon_amt <= 8826.00)],AND[(store_sales.ss_wholesale_cost >= 70.00),(store_sales.ss_wholesale_cost <= 90.00)]]) ------------------------------------PhysicalOlapScan[store_sales] ------------------------PhysicalLimit[LOCAL] --------------------------hashAgg[GLOBAL] ----------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------hashAgg[LOCAL] --------------------------------PhysicalProject -----------------------------------filter(((((store_sales.ss_list_price >= 152.00) AND (store_sales.ss_list_price <= 162.00)) OR ((store_sales.ss_coupon_amt >= 2196.00) AND (store_sales.ss_coupon_amt <= 3196.00))) OR ((store_sales.ss_wholesale_cost >= 56.00) AND (store_sales.ss_wholesale_cost <= 76.00))) and (store_sales.ss_quantity <= 10) and (store_sales.ss_quantity >= 6)) +----------------------------------filter((store_sales.ss_quantity <= 10) and (store_sales.ss_quantity >= 6) and OR[AND[(store_sales.ss_list_price >= 152.00),(store_sales.ss_list_price <= 162.00)],AND[(store_sales.ss_coupon_amt >= 2196.00),(store_sales.ss_coupon_amt <= 3196.00)],AND[(store_sales.ss_wholesale_cost >= 56.00),(store_sales.ss_wholesale_cost <= 76.00)]]) ------------------------------------PhysicalOlapScan[store_sales] --------------------PhysicalLimit[LOCAL] ----------------------hashAgg[GLOBAL] ------------------------PhysicalDistribute[DistributionSpecGather] --------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------filter(((((store_sales.ss_list_price >= 53.00) AND (store_sales.ss_list_price <= 63.00)) OR ((store_sales.ss_coupon_amt >= 3430.00) AND (store_sales.ss_coupon_amt <= 4430.00))) OR ((store_sales.ss_wholesale_cost >= 13.00) AND (store_sales.ss_wholesale_cost <= 33.00))) and (store_sales.ss_quantity <= 15) and (store_sales.ss_quantity >= 11)) +------------------------------filter((store_sales.ss_quantity <= 15) and (store_sales.ss_quantity >= 11) and OR[AND[(store_sales.ss_list_price >= 53.00),(store_sales.ss_list_price <= 63.00)],AND[(store_sales.ss_coupon_amt >= 3430.00),(store_sales.ss_coupon_amt <= 4430.00)],AND[(store_sales.ss_wholesale_cost >= 13.00),(store_sales.ss_wholesale_cost <= 33.00)]]) --------------------------------PhysicalOlapScan[store_sales] ----------------PhysicalLimit[LOCAL] ------------------hashAgg[GLOBAL] --------------------PhysicalDistribute[DistributionSpecGather] ----------------------hashAgg[LOCAL] ------------------------PhysicalProject ---------------------------filter(((((store_sales.ss_list_price >= 182.00) AND (store_sales.ss_list_price <= 192.00)) OR ((store_sales.ss_coupon_amt >= 3262.00) AND (store_sales.ss_coupon_amt <= 4262.00))) OR ((store_sales.ss_wholesale_cost >= 20.00) AND (store_sales.ss_wholesale_cost <= 40.00))) and (store_sales.ss_quantity <= 20) and (store_sales.ss_quantity >= 16)) +--------------------------filter((store_sales.ss_quantity <= 20) and (store_sales.ss_quantity >= 16) and OR[AND[(store_sales.ss_list_price >= 182.00),(store_sales.ss_list_price <= 192.00)],AND[(store_sales.ss_coupon_amt >= 3262.00),(store_sales.ss_coupon_amt <= 4262.00)],AND[(store_sales.ss_wholesale_cost >= 20.00),(store_sales.ss_wholesale_cost <= 40.00)]]) ----------------------------PhysicalOlapScan[store_sales] ------------PhysicalLimit[LOCAL] --------------hashAgg[GLOBAL] ----------------PhysicalDistribute[DistributionSpecGather] ------------------hashAgg[LOCAL] --------------------PhysicalProject -----------------------filter(((((store_sales.ss_list_price >= 85.00) AND (store_sales.ss_list_price <= 95.00)) OR ((store_sales.ss_coupon_amt >= 3310.00) AND (store_sales.ss_coupon_amt <= 4310.00))) OR ((store_sales.ss_wholesale_cost >= 37.00) AND (store_sales.ss_wholesale_cost <= 57.00))) and (store_sales.ss_quantity <= 25) and (store_sales.ss_quantity >= 21)) +----------------------filter((store_sales.ss_quantity <= 25) and (store_sales.ss_quantity >= 21) and OR[AND[(store_sales.ss_list_price >= 85.00),(store_sales.ss_list_price <= 95.00)],AND[(store_sales.ss_coupon_amt >= 3310.00),(store_sales.ss_coupon_amt <= 4310.00)],AND[(store_sales.ss_wholesale_cost >= 37.00),(store_sales.ss_wholesale_cost <= 57.00)]]) ------------------------PhysicalOlapScan[store_sales] --------PhysicalLimit[LOCAL] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecGather] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------filter(((((store_sales.ss_list_price >= 180.00) AND (store_sales.ss_list_price <= 190.00)) OR ((store_sales.ss_coupon_amt >= 12592.00) AND (store_sales.ss_coupon_amt <= 13592.00))) OR ((store_sales.ss_wholesale_cost >= 22.00) AND (store_sales.ss_wholesale_cost <= 42.00))) and (store_sales.ss_quantity <= 30) and (store_sales.ss_quantity >= 26)) +------------------filter((store_sales.ss_quantity <= 30) and (store_sales.ss_quantity >= 26) and OR[AND[(store_sales.ss_list_price >= 180.00),(store_sales.ss_list_price <= 190.00)],AND[(store_sales.ss_coupon_amt >= 12592.00),(store_sales.ss_coupon_amt <= 13592.00)],AND[(store_sales.ss_wholesale_cost >= 22.00),(store_sales.ss_wholesale_cost <= 42.00)]]) --------------------PhysicalOlapScan[store_sales] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query34.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query34.out index a883135293b760..5aa611041037f9 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query34.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query34.out @@ -19,7 +19,7 @@ PhysicalResultSink --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 --------------------------------PhysicalProject -----------------------------------filter((((date_dim.d_dom >= 1) AND (date_dim.d_dom <= 3)) OR ((date_dim.d_dom >= 25) AND (date_dim.d_dom <= 28))) and d_year IN (2000, 2001, 2002)) +----------------------------------filter(OR[AND[(date_dim.d_dom >= 1),(date_dim.d_dom <= 3)],AND[(date_dim.d_dom >= 25),(date_dim.d_dom <= 28)]] and d_year IN (2000, 2001, 2002)) ------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalProject ------------------------------filter(s_county IN ('Arthur County', 'Halifax County', 'Lunenburg County', 'Oglethorpe County', 'Perry County', 'Salem County', 'Sumner County', 'Terrell County')) diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query35.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query35.out index b14e619cc70b34..beedc2a19350ab 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query35.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query35.out @@ -9,7 +9,7 @@ PhysicalResultSink ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) +------------------filter(OR[ifnull($c$1, FALSE),ifnull($c$2, FALSE)]) --------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() ----------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() ------------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query41.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query41.out index 16a3a620b77c8e..c99d2b2031c4ea 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query41.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query41.out @@ -18,6 +18,6 @@ PhysicalResultSink ------------------------PhysicalDistribute[DistributionSpecHash] --------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------filter((((i_color IN ('black', 'chocolate', 'cornflower', 'firebrick', 'frosted', 'magenta', 'rose', 'slate') AND i_units IN ('Box', 'Bundle', 'Carton', 'Dram', 'Gross', 'Lb', 'Oz', 'Pound')) AND (((((((item.i_category = 'Women') AND i_color IN ('frosted', 'rose')) AND i_units IN ('Gross', 'Lb')) AND i_size IN ('large', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('black', 'chocolate')) AND i_units IN ('Box', 'Dram')) AND i_size IN ('economy', 'petite'))) OR ((((item.i_category = 'Men') AND i_color IN ('magenta', 'slate')) AND i_units IN ('Bundle', 'Carton')) AND i_size IN ('N/A', 'small'))) OR ((((item.i_category = 'Men') AND i_color IN ('cornflower', 'firebrick')) AND i_units IN ('Oz', 'Pound')) AND i_size IN ('large', 'medium')))) OR ((i_color IN ('almond', 'aquamarine', 'cyan', 'lavender', 'maroon', 'papaya', 'purple', 'steel') AND i_units IN ('Bunch', 'Case', 'Cup', 'Each', 'Gram', 'N/A', 'Pallet', 'Tsp')) AND (((((((item.i_category = 'Women') AND i_color IN ('almond', 'steel')) AND i_units IN ('Case', 'Tsp')) AND i_size IN ('large', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('aquamarine', 'purple')) AND i_units IN ('Bunch', 'Gram')) AND i_size IN ('economy', 'petite'))) OR ((((item.i_category = 'Men') AND i_color IN ('lavender', 'papaya')) AND i_units IN ('Cup', 'Pallet')) AND i_size IN ('N/A', 'small'))) OR ((((item.i_category = 'Men') AND i_color IN ('cyan', 'maroon')) AND i_units IN ('Each', 'N/A')) AND i_size IN ('large', 'medium'))))) and i_category IN ('Men', 'Women') and i_size IN ('N/A', 'economy', 'large', 'medium', 'petite', 'small')) +------------------------------filter(OR[AND[i_color IN ('black', 'chocolate', 'cornflower', 'firebrick', 'frosted', 'magenta', 'rose', 'slate'),i_units IN ('Box', 'Bundle', 'Carton', 'Dram', 'Gross', 'Lb', 'Oz', 'Pound'),OR[AND[(item.i_category = 'Women'),i_color IN ('frosted', 'rose'),i_units IN ('Gross', 'Lb'),i_size IN ('large', 'medium')],AND[(item.i_category = 'Women'),i_color IN ('black', 'chocolate'),i_units IN ('Box', 'Dram'),i_size IN ('economy', 'petite')],AND[(item.i_category = 'Men'),i_color IN ('magenta', 'slate'),i_units IN ('Bundle', 'Carton'),i_size IN ('N/A', 'small')],AND[(item.i_category = 'Men'),i_color IN ('cornflower', 'firebrick'),i_units IN ('Oz', 'Pound'),i_size IN ('large', 'medium')]]],AND[i_color IN ('almond', 'aquamarine', 'cyan', 'lavender', 'maroon', 'papaya', 'purple', 'steel'),i_units IN ('Bunch', 'Case', 'Cup', 'Each', 'Gram', 'N/A', 'Pallet', 'Tsp'),OR[AND[(item.i_category = 'Women'),i_color IN ('almond', 'steel'),i_units IN ('Case', 'Tsp'),i_size IN ('large', 'medium')],AND[(item.i_category = 'Women'),i_color IN ('aquamarine', 'purple'),i_units IN ('Bunch', 'Gram'),i_size IN ('economy', 'petite')],AND[(item.i_category = 'Men'),i_color IN ('lavender', 'papaya'),i_units IN ('Cup', 'Pallet'),i_size IN ('N/A', 'small')],AND[(item.i_category = 'Men'),i_color IN ('cyan', 'maroon'),i_units IN ('Each', 'N/A'),i_size IN ('large', 'medium')]]]] and i_category IN ('Men', 'Women') and i_size IN ('N/A', 'economy', 'large', 'medium', 'petite', 'small')) --------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query45.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query45.out index 98618affdbd763..9f0c50ae0a648a 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query45.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query45.out @@ -8,7 +8,7 @@ PhysicalResultSink ----------PhysicalDistribute[DistributionSpecHash] ------------hashAgg[LOCAL] --------------PhysicalProject -----------------filter((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR $c$1)) +----------------filter(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),$c$1]) ------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query46.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query46.out index 2831d1a7f8f00a..effe4533d7debc 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query46.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query46.out @@ -27,7 +27,7 @@ PhysicalResultSink ----------------------------------filter(s_city IN ('Fairview', 'Farmington', 'Five Forks', 'Oakland', 'Winchester')) ------------------------------------PhysicalOlapScan[store] ----------------------------PhysicalProject -------------------------------filter(((household_demographics.hd_dep_count = 0) OR (household_demographics.hd_vehicle_count = 1))) +------------------------------filter(OR[(household_demographics.hd_dep_count = 0),(household_demographics.hd_vehicle_count = 1)]) --------------------------------PhysicalOlapScan[household_demographics] ------------------------PhysicalProject --------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query47.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query47.out index a3205932929569..52c9d9bc883f66 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query47.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query47.out @@ -23,7 +23,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------------------------PhysicalProject --------------------------------------PhysicalOlapScan[item] --------------------------------PhysicalProject -----------------------------------filter((((date_dim.d_year = 1999) OR ((date_dim.d_year = 1998) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2000) AND (date_dim.d_moy = 1))) and d_year IN (1998, 1999, 2000)) +----------------------------------filter(OR[(date_dim.d_year = 1999),AND[(date_dim.d_year = 1998),(date_dim.d_moy = 12)],AND[(date_dim.d_year = 2000),(date_dim.d_moy = 1)]] and d_year IN (1998, 1999, 2000)) ------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query48.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query48.out index cb48bf987d7340..2cac05386db12c 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query48.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query48.out @@ -7,9 +7,9 @@ PhysicalResultSink --------PhysicalProject ----------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ss_sold_date_sk] ------------PhysicalProject ---------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=((((ca_state IN ('GA', 'MI', 'NH') AND ((store_sales.ss_net_profit >= 0.00) AND (store_sales.ss_net_profit <= 2000.00))) OR (ca_state IN ('KY', 'SD', 'TX') AND ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 3000.00)))) OR (ca_state IN ('FL', 'NY', 'OH') AND ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 25000.00))))) build RFs:RF2 ca_address_sk->[ss_addr_sk] +--------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=(OR[AND[ca_state IN ('GA', 'MI', 'NH'),(store_sales.ss_net_profit >= 0.00),(store_sales.ss_net_profit <= 2000.00)],AND[ca_state IN ('KY', 'SD', 'TX'),(store_sales.ss_net_profit >= 150.00),(store_sales.ss_net_profit <= 3000.00)],AND[ca_state IN ('FL', 'NY', 'OH'),(store_sales.ss_net_profit >= 50.00),(store_sales.ss_net_profit <= 25000.00)]]) build RFs:RF2 ca_address_sk->[ss_addr_sk] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)) otherCondition=((((((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = 'Unknown')) AND ((store_sales.ss_sales_price >= 100.00) AND (store_sales.ss_sales_price <= 150.00))) OR (((customer_demographics.cd_marital_status = 'W') AND (customer_demographics.cd_education_status = 'College')) AND ((store_sales.ss_sales_price >= 50.00) AND (store_sales.ss_sales_price <= 100.00)))) OR (((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = 'Primary')) AND ((store_sales.ss_sales_price >= 150.00) AND (store_sales.ss_sales_price <= 200.00))))) build RFs:RF1 cd_demo_sk->[ss_cdemo_sk] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)) otherCondition=(OR[AND[(customer_demographics.cd_marital_status = 'M'),(customer_demographics.cd_education_status = 'Unknown'),(store_sales.ss_sales_price >= 100.00),(store_sales.ss_sales_price <= 150.00)],AND[(customer_demographics.cd_marital_status = 'W'),(customer_demographics.cd_education_status = 'College'),(store_sales.ss_sales_price >= 50.00),(store_sales.ss_sales_price <= 100.00)],AND[(customer_demographics.cd_marital_status = 'D'),(customer_demographics.cd_education_status = 'Primary'),(store_sales.ss_sales_price >= 150.00),(store_sales.ss_sales_price <= 200.00)]]) build RFs:RF1 cd_demo_sk->[ss_cdemo_sk] --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF0 s_store_sk->[ss_store_sk] ------------------------PhysicalProject @@ -18,7 +18,7 @@ PhysicalResultSink ------------------------PhysicalProject --------------------------PhysicalOlapScan[store] --------------------PhysicalProject -----------------------filter(((((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = 'Unknown')) OR ((customer_demographics.cd_marital_status = 'W') AND (customer_demographics.cd_education_status = 'College'))) OR ((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = 'Primary'))) and cd_education_status IN ('College', 'Primary', 'Unknown') and cd_marital_status IN ('D', 'M', 'W')) +----------------------filter(OR[AND[(customer_demographics.cd_marital_status = 'M'),(customer_demographics.cd_education_status = 'Unknown')],AND[(customer_demographics.cd_marital_status = 'W'),(customer_demographics.cd_education_status = 'College')],AND[(customer_demographics.cd_marital_status = 'D'),(customer_demographics.cd_education_status = 'Primary')]] and cd_education_status IN ('College', 'Primary', 'Unknown') and cd_marital_status IN ('D', 'M', 'W')) ------------------------PhysicalOlapScan[customer_demographics] ----------------PhysicalProject ------------------filter((customer_address.ca_country = 'United States') and ca_state IN ('FL', 'GA', 'KY', 'MI', 'NH', 'NY', 'OH', 'SD', 'TX')) diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query49.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query49.out index 398dcef280ccea..04db2001d08cf7 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query49.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query49.out @@ -16,7 +16,7 @@ PhysicalResultSink --------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) ----------------------------------PhysicalWindow ------------------------------------PhysicalQuickSort[LOCAL_SORT] --------------------------------------PhysicalWindow @@ -48,7 +48,7 @@ PhysicalResultSink --------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) ----------------------------------PhysicalWindow ------------------------------------PhysicalQuickSort[LOCAL_SORT] --------------------------------------PhysicalWindow @@ -80,7 +80,7 @@ PhysicalResultSink --------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) ----------------------------------PhysicalWindow ------------------------------------PhysicalQuickSort[LOCAL_SORT] --------------------------------------PhysicalWindow diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query53.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query53.out index 0ef41f5f16c373..382c6a18e33765 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query53.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query53.out @@ -14,19 +14,19 @@ PhysicalResultSink ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------hashAgg[LOCAL] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF2 s_store_sk->[ss_store_sk] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 i_item_sk->[ss_item_sk] ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF1 s_store_sk->[ss_store_sk] ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] --------------------------------------PhysicalProject ----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 --------------------------------------PhysicalProject -----------------------------------------filter((((i_category IN ('Books', 'Children', 'Electronics') AND i_class IN ('personal', 'portable', 'reference', 'self-help')) AND i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')) OR ((i_category IN ('Men', 'Music', 'Women') AND i_class IN ('accessories', 'classical', 'fragrances', 'pants')) AND i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1'))) and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) -------------------------------------------PhysicalOlapScan[item] +----------------------------------------filter(d_month_seq IN (1212, 1213, 1214, 1215, 1216, 1217, 1218, 1219, 1220, 1221, 1222, 1223)) +------------------------------------------PhysicalOlapScan[date_dim] ----------------------------------PhysicalProject -------------------------------------filter(d_month_seq IN (1212, 1213, 1214, 1215, 1216, 1217, 1218, 1219, 1220, 1221, 1222, 1223)) ---------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalOlapScan[store] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store] +--------------------------------filter(OR[AND[i_category IN ('Books', 'Children', 'Electronics'),i_class IN ('personal', 'portable', 'reference', 'self-help'),i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')],AND[i_category IN ('Men', 'Music', 'Women'),i_class IN ('accessories', 'classical', 'fragrances', 'pants'),i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')]] and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) +----------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query57.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query57.out index e640ec24d7b3e3..88777bc1ff548d 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query57.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query57.out @@ -23,7 +23,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------------------------PhysicalProject --------------------------------------PhysicalOlapScan[item] --------------------------------PhysicalProject -----------------------------------filter((((date_dim.d_year = 1999) OR ((date_dim.d_year = 1998) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2000) AND (date_dim.d_moy = 1))) and d_year IN (1998, 1999, 2000)) +----------------------------------filter(OR[(date_dim.d_year = 1999),AND[(date_dim.d_year = 1998),(date_dim.d_moy = 12)],AND[(date_dim.d_year = 2000),(date_dim.d_moy = 1)]] and d_year IN (1998, 1999, 2000)) ------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[call_center] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query61.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query61.out index a45d788af7c5a8..98825fe6cf8c4a 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query61.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query61.out @@ -22,7 +22,7 @@ PhysicalResultSink --------------------------------------PhysicalProject ----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF5 RF6 RF7 RF8 RF10 --------------------------------------PhysicalProject -----------------------------------------filter((((promotion.p_channel_dmail = 'Y') OR (promotion.p_channel_email = 'Y')) OR (promotion.p_channel_tv = 'Y'))) +----------------------------------------filter(OR[(promotion.p_channel_dmail = 'Y'),(promotion.p_channel_email = 'Y'),(promotion.p_channel_tv = 'Y')]) ------------------------------------------PhysicalOlapScan[promotion] ----------------------------------PhysicalProject ------------------------------------PhysicalOlapScan[customer] apply RFs: RF9 diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query63.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query63.out index 5d47dbe07f57cb..1dfce555f674d1 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query63.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query63.out @@ -14,19 +14,19 @@ PhysicalResultSink ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------hashAgg[LOCAL] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF2 s_store_sk->[ss_store_sk] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 i_item_sk->[ss_item_sk] ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF1 s_store_sk->[ss_store_sk] ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] --------------------------------------PhysicalProject ----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 --------------------------------------PhysicalProject -----------------------------------------filter((((i_category IN ('Books', 'Children', 'Electronics') AND i_class IN ('personal', 'portable', 'reference', 'self-help')) AND i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')) OR ((i_category IN ('Men', 'Music', 'Women') AND i_class IN ('accessories', 'classical', 'fragrances', 'pants')) AND i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1'))) and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) -------------------------------------------PhysicalOlapScan[item] +----------------------------------------filter(d_month_seq IN (1211, 1212, 1213, 1214, 1215, 1216, 1217, 1218, 1219, 1220, 1221, 1222)) +------------------------------------------PhysicalOlapScan[date_dim] ----------------------------------PhysicalProject -------------------------------------filter(d_month_seq IN (1211, 1212, 1213, 1214, 1215, 1216, 1217, 1218, 1219, 1220, 1221, 1222)) ---------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalOlapScan[store] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store] +--------------------------------filter(OR[AND[i_category IN ('Books', 'Children', 'Electronics'),i_class IN ('personal', 'portable', 'reference', 'self-help'),i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')],AND[i_category IN ('Men', 'Music', 'Women'),i_class IN ('accessories', 'classical', 'fragrances', 'pants'),i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')]] and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) +----------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query68.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query68.out index a96932c173fb69..2339b9d070f674 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query68.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query68.out @@ -27,7 +27,7 @@ PhysicalResultSink ----------------------------------filter(s_city IN ('Bethel', 'Pleasant Hill')) ------------------------------------PhysicalOlapScan[store] ----------------------------PhysicalProject -------------------------------filter(((household_demographics.hd_dep_count = 4) OR (household_demographics.hd_vehicle_count = 0))) +------------------------------filter(OR[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count = 0)]) --------------------------------PhysicalOlapScan[household_demographics] ------------------------PhysicalProject --------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query7.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query7.out index 9ebde668ccdb31..82a5672ef981f2 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query7.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query7.out @@ -26,6 +26,6 @@ PhysicalResultSink ------------------------filter((date_dim.d_year = 2001)) --------------------------PhysicalOlapScan[date_dim] ------------------PhysicalProject ---------------------filter(((promotion.p_channel_email = 'N') OR (promotion.p_channel_event = 'N'))) +--------------------filter(OR[(promotion.p_channel_email = 'N'),(promotion.p_channel_event = 'N')]) ----------------------PhysicalOlapScan[promotion] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query78.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query78.out index 8662cb5a1ad007..81f28190a1b678 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query78.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query78.out @@ -5,7 +5,7 @@ PhysicalResultSink ----PhysicalDistribute[DistributionSpecGather] ------PhysicalTopN[LOCAL_SORT] --------PhysicalProject -----------filter(((coalesce(ws_qty, 0) > 0) OR (coalesce(cs_qty, 0) > 0))) +----------filter(OR[(coalesce(ws_qty, 0) > 0),(coalesce(cs_qty, 0) > 0)]) ------------hashJoin[LEFT_OUTER_JOIN colocated] hashCondition=((cs.cs_customer_sk = ss.ss_customer_sk) and (cs.cs_item_sk = ss.ss_item_sk) and (cs.cs_sold_year = ss.ss_sold_year)) otherCondition=() --------------PhysicalProject ----------------hashJoin[LEFT_OUTER_JOIN colocated] hashCondition=((ws.ws_customer_sk = ss.ss_customer_sk) and (ws.ws_item_sk = ss.ss_item_sk) and (ws.ws_sold_year = ss.ss_sold_year)) otherCondition=() diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query79.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query79.out index 8684d60ad233dd..0239a35e7676cd 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query79.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query79.out @@ -25,7 +25,7 @@ PhysicalResultSink ------------------------------filter((store.s_number_employees <= 295) and (store.s_number_employees >= 200)) --------------------------------PhysicalOlapScan[store] ------------------------PhysicalProject ---------------------------filter(((household_demographics.hd_dep_count = 0) OR (household_demographics.hd_vehicle_count > 3))) +--------------------------filter(OR[(household_demographics.hd_dep_count = 0),(household_demographics.hd_vehicle_count > 3)]) ----------------------------PhysicalOlapScan[household_demographics] ------------PhysicalProject --------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query85.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query85.out index 74754368bc4f59..106cf30dfcdd28 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query85.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query85.out @@ -13,11 +13,11 @@ PhysicalResultSink --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF8 d_date_sk->[ws_sold_date_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = web_returns.wr_refunded_addr_sk)) otherCondition=((((ca_state IN ('CA', 'TX', 'VA') AND ((web_sales.ws_net_profit >= 100.00) AND (web_sales.ws_net_profit <= 200.00))) OR (ca_state IN ('AR', 'MO', 'NE') AND ((web_sales.ws_net_profit >= 150.00) AND (web_sales.ws_net_profit <= 300.00)))) OR (ca_state IN ('IA', 'MS', 'WA') AND ((web_sales.ws_net_profit >= 50.00) AND (web_sales.ws_net_profit <= 250.00))))) build RFs:RF7 ca_address_sk->[wr_refunded_addr_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = web_returns.wr_refunded_addr_sk)) otherCondition=(OR[AND[ca_state IN ('CA', 'TX', 'VA'),(web_sales.ws_net_profit >= 100.00),(web_sales.ws_net_profit <= 200.00)],AND[ca_state IN ('AR', 'MO', 'NE'),(web_sales.ws_net_profit >= 150.00),(web_sales.ws_net_profit <= 300.00)],AND[ca_state IN ('IA', 'MS', 'WA'),(web_sales.ws_net_profit >= 50.00),(web_sales.ws_net_profit <= 250.00)]]) build RFs:RF7 ca_address_sk->[wr_refunded_addr_sk] ----------------------------PhysicalProject ------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cd1.cd_education_status = cd2.cd_education_status) and (cd1.cd_marital_status = cd2.cd_marital_status) and (cd2.cd_demo_sk = web_returns.wr_returning_cdemo_sk)) otherCondition=() build RFs:RF4 cd_demo_sk->[wr_returning_cdemo_sk];RF5 cd_marital_status->[cd_marital_status];RF6 cd_education_status->[cd_education_status] --------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cd1.cd_demo_sk = web_returns.wr_refunded_cdemo_sk)) otherCondition=((((((cd1.cd_marital_status = 'M') AND (cd1.cd_education_status = '4 yr Degree')) AND ((web_sales.ws_sales_price >= 100.00) AND (web_sales.ws_sales_price <= 150.00))) OR (((cd1.cd_marital_status = 'S') AND (cd1.cd_education_status = 'College')) AND ((web_sales.ws_sales_price >= 50.00) AND (web_sales.ws_sales_price <= 100.00)))) OR (((cd1.cd_marital_status = 'D') AND (cd1.cd_education_status = 'Secondary')) AND ((web_sales.ws_sales_price >= 150.00) AND (web_sales.ws_sales_price <= 200.00))))) build RFs:RF3 cd_demo_sk->[wr_refunded_cdemo_sk] +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cd1.cd_demo_sk = web_returns.wr_refunded_cdemo_sk)) otherCondition=(OR[AND[(cd1.cd_marital_status = 'M'),(cd1.cd_education_status = '4 yr Degree'),(web_sales.ws_sales_price >= 100.00),(web_sales.ws_sales_price <= 150.00)],AND[(cd1.cd_marital_status = 'S'),(cd1.cd_education_status = 'College'),(web_sales.ws_sales_price >= 50.00),(web_sales.ws_sales_price <= 100.00)],AND[(cd1.cd_marital_status = 'D'),(cd1.cd_education_status = 'Secondary'),(web_sales.ws_sales_price >= 150.00),(web_sales.ws_sales_price <= 200.00)]]) build RFs:RF3 cd_demo_sk->[wr_refunded_cdemo_sk] ------------------------------------PhysicalProject --------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_web_page_sk = web_page.wp_web_page_sk)) otherCondition=() build RFs:RF2 wp_web_page_sk->[ws_web_page_sk] ----------------------------------------PhysicalProject @@ -30,7 +30,7 @@ PhysicalResultSink ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[web_page] ------------------------------------PhysicalProject ---------------------------------------filter(((((cd1.cd_marital_status = 'M') AND (cd1.cd_education_status = '4 yr Degree')) OR ((cd1.cd_marital_status = 'S') AND (cd1.cd_education_status = 'College'))) OR ((cd1.cd_marital_status = 'D') AND (cd1.cd_education_status = 'Secondary'))) and cd_education_status IN ('4 yr Degree', 'College', 'Secondary') and cd_marital_status IN ('D', 'M', 'S')) +--------------------------------------filter(OR[AND[(cd1.cd_marital_status = 'M'),(cd1.cd_education_status = '4 yr Degree')],AND[(cd1.cd_marital_status = 'S'),(cd1.cd_education_status = 'College')],AND[(cd1.cd_marital_status = 'D'),(cd1.cd_education_status = 'Secondary')]] and cd_education_status IN ('4 yr Degree', 'College', 'Secondary') and cd_marital_status IN ('D', 'M', 'S')) ----------------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF5 RF6 --------------------------------PhysicalProject ----------------------------------filter(cd_education_status IN ('4 yr Degree', 'College', 'Secondary') and cd_marital_status IN ('D', 'M', 'S')) diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query88.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query88.out index fa1d26b12e7fa8..3d5752485f6b0a 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query88.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query88.out @@ -20,7 +20,7 @@ PhysicalResultSink ----------------------------------PhysicalProject ------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF21 RF22 RF23 ----------------------------------PhysicalProject -------------------------------------filter(((((household_demographics.hd_dep_count = 2) AND (household_demographics.hd_vehicle_count <= 4)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (2, 3, 4)) +------------------------------------filter(OR[AND[(household_demographics.hd_dep_count = 2),(household_demographics.hd_vehicle_count <= 4)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (2, 3, 4)) --------------------------------------PhysicalOlapScan[household_demographics] ------------------------------PhysicalProject --------------------------------filter((time_dim.t_hour = 8) and (time_dim.t_minute >= 30)) @@ -40,7 +40,7 @@ PhysicalResultSink ----------------------------------PhysicalProject ------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF18 RF19 RF20 ----------------------------------PhysicalProject -------------------------------------filter(((((household_demographics.hd_dep_count = 2) AND (household_demographics.hd_vehicle_count <= 4)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (2, 3, 4)) +------------------------------------filter(OR[AND[(household_demographics.hd_dep_count = 2),(household_demographics.hd_vehicle_count <= 4)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (2, 3, 4)) --------------------------------------PhysicalOlapScan[household_demographics] ------------------------------PhysicalProject --------------------------------filter((time_dim.t_hour = 9) and (time_dim.t_minute < 30)) @@ -60,7 +60,7 @@ PhysicalResultSink --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF15 RF16 RF17 --------------------------------PhysicalProject -----------------------------------filter(((((household_demographics.hd_dep_count = 2) AND (household_demographics.hd_vehicle_count <= 4)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (2, 3, 4)) +----------------------------------filter(OR[AND[(household_demographics.hd_dep_count = 2),(household_demographics.hd_vehicle_count <= 4)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (2, 3, 4)) ------------------------------------PhysicalOlapScan[household_demographics] ----------------------------PhysicalProject ------------------------------filter((time_dim.t_hour = 9) and (time_dim.t_minute >= 30)) @@ -80,7 +80,7 @@ PhysicalResultSink ------------------------------PhysicalProject --------------------------------PhysicalOlapScan[store_sales] apply RFs: RF12 RF13 RF14 ------------------------------PhysicalProject ---------------------------------filter(((((household_demographics.hd_dep_count = 2) AND (household_demographics.hd_vehicle_count <= 4)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (2, 3, 4)) +--------------------------------filter(OR[AND[(household_demographics.hd_dep_count = 2),(household_demographics.hd_vehicle_count <= 4)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (2, 3, 4)) ----------------------------------PhysicalOlapScan[household_demographics] --------------------------PhysicalProject ----------------------------filter((time_dim.t_hour = 10) and (time_dim.t_minute < 30)) @@ -100,7 +100,7 @@ PhysicalResultSink ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[store_sales] apply RFs: RF9 RF10 RF11 ----------------------------PhysicalProject -------------------------------filter(((((household_demographics.hd_dep_count = 2) AND (household_demographics.hd_vehicle_count <= 4)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (2, 3, 4)) +------------------------------filter(OR[AND[(household_demographics.hd_dep_count = 2),(household_demographics.hd_vehicle_count <= 4)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (2, 3, 4)) --------------------------------PhysicalOlapScan[household_demographics] ------------------------PhysicalProject --------------------------filter((time_dim.t_hour = 10) and (time_dim.t_minute >= 30)) @@ -120,7 +120,7 @@ PhysicalResultSink --------------------------PhysicalProject ----------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 RF7 RF8 --------------------------PhysicalProject -----------------------------filter(((((household_demographics.hd_dep_count = 2) AND (household_demographics.hd_vehicle_count <= 4)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (2, 3, 4)) +----------------------------filter(OR[AND[(household_demographics.hd_dep_count = 2),(household_demographics.hd_vehicle_count <= 4)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (2, 3, 4)) ------------------------------PhysicalOlapScan[household_demographics] ----------------------PhysicalProject ------------------------filter((time_dim.t_hour = 11) and (time_dim.t_minute < 30)) @@ -140,7 +140,7 @@ PhysicalResultSink ------------------------PhysicalProject --------------------------PhysicalOlapScan[store_sales] apply RFs: RF3 RF4 RF5 ------------------------PhysicalProject ---------------------------filter(((((household_demographics.hd_dep_count = 2) AND (household_demographics.hd_vehicle_count <= 4)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (2, 3, 4)) +--------------------------filter(OR[AND[(household_demographics.hd_dep_count = 2),(household_demographics.hd_vehicle_count <= 4)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (2, 3, 4)) ----------------------------PhysicalOlapScan[household_demographics] --------------------PhysicalProject ----------------------filter((time_dim.t_hour = 11) and (time_dim.t_minute >= 30)) @@ -160,7 +160,7 @@ PhysicalResultSink ----------------------PhysicalProject ------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 ----------------------PhysicalProject -------------------------filter(((((household_demographics.hd_dep_count = 2) AND (household_demographics.hd_vehicle_count <= 4)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (2, 3, 4)) +------------------------filter(OR[AND[(household_demographics.hd_dep_count = 2),(household_demographics.hd_vehicle_count <= 4)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (2, 3, 4)) --------------------------PhysicalOlapScan[household_demographics] ------------------PhysicalProject --------------------filter((time_dim.t_hour = 12) and (time_dim.t_minute < 30)) diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query89.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query89.out index a57dfec0fc83fc..a24231e1dd660d 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query89.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query89.out @@ -23,7 +23,7 @@ PhysicalResultSink ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 ----------------------------------------PhysicalProject -------------------------------------------filter(((i_category IN ('Books', 'Home', 'Music') AND i_class IN ('classical', 'fiction', 'glassware')) OR (i_category IN ('Jewelry', 'Sports', 'Women') AND i_class IN ('baseball', 'dresses', 'semi-precious'))) and i_category IN ('Books', 'Home', 'Jewelry', 'Music', 'Sports', 'Women') and i_class IN ('baseball', 'classical', 'dresses', 'fiction', 'glassware', 'semi-precious')) +------------------------------------------filter(OR[AND[i_category IN ('Books', 'Home', 'Music'),i_class IN ('classical', 'fiction', 'glassware')],AND[i_category IN ('Jewelry', 'Sports', 'Women'),i_class IN ('baseball', 'dresses', 'semi-precious')]] and i_category IN ('Books', 'Home', 'Jewelry', 'Music', 'Sports', 'Women') and i_class IN ('baseball', 'classical', 'dresses', 'fiction', 'glassware', 'semi-precious')) --------------------------------------------PhysicalOlapScan[item] ------------------------------------PhysicalProject --------------------------------------filter((date_dim.d_year = 2000)) diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query91.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query91.out index 48587871410146..15a09077f4ccd5 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query91.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query91.out @@ -33,7 +33,7 @@ PhysicalResultSink ------------------------------filter((customer_address.ca_gmt_offset = -7.00)) --------------------------------PhysicalOlapScan[customer_address] ------------------------PhysicalProject ---------------------------filter((((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = 'Unknown')) OR ((customer_demographics.cd_marital_status = 'W') AND (customer_demographics.cd_education_status = 'Advanced Degree'))) and cd_education_status IN ('Advanced Degree', 'Unknown') and cd_marital_status IN ('M', 'W')) +--------------------------filter(OR[AND[(customer_demographics.cd_marital_status = 'M'),(customer_demographics.cd_education_status = 'Unknown')],AND[(customer_demographics.cd_marital_status = 'W'),(customer_demographics.cd_education_status = 'Advanced Degree')]] and cd_education_status IN ('Advanced Degree', 'Unknown') and cd_marital_status IN ('M', 'W')) ----------------------------PhysicalOlapScan[customer_demographics] --------------------PhysicalProject ----------------------filter((hd_buy_potential like 'Unknown%')) diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/nostats_rf_prune/q19.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/nostats_rf_prune/q19.out index bf86e68a7f9837..ad82594176a219 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/nostats_rf_prune/q19.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/nostats_rf_prune/q19.out @@ -5,11 +5,11 @@ PhysicalResultSink ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] --------PhysicalProject -----------hashJoin[INNER_JOIN broadcast] hashCondition=((part.p_partkey = lineitem.l_partkey)) otherCondition=(((((((part.p_brand = 'Brand#12') AND p_container IN ('SM BOX', 'SM CASE', 'SM PACK', 'SM PKG')) AND ((lineitem.l_quantity >= 1.00) AND (lineitem.l_quantity <= 11.00))) AND (part.p_size <= 5)) OR ((((part.p_brand = 'Brand#23') AND p_container IN ('MED BAG', 'MED BOX', 'MED PACK', 'MED PKG')) AND ((lineitem.l_quantity >= 10.00) AND (lineitem.l_quantity <= 20.00))) AND (part.p_size <= 10))) OR ((((part.p_brand = 'Brand#34') AND p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG')) AND ((lineitem.l_quantity >= 20.00) AND (lineitem.l_quantity <= 30.00))) AND (part.p_size <= 15)))) build RFs:RF0 p_partkey->[l_partkey] +----------hashJoin[INNER_JOIN broadcast] hashCondition=((part.p_partkey = lineitem.l_partkey)) otherCondition=(OR[AND[(part.p_brand = 'Brand#12'),p_container IN ('SM BOX', 'SM CASE', 'SM PACK', 'SM PKG'),(lineitem.l_quantity >= 1.00),(lineitem.l_quantity <= 11.00),(part.p_size <= 5)],AND[(part.p_brand = 'Brand#23'),p_container IN ('MED BAG', 'MED BOX', 'MED PACK', 'MED PKG'),(lineitem.l_quantity >= 10.00),(lineitem.l_quantity <= 20.00),(part.p_size <= 10)],AND[(part.p_brand = 'Brand#34'),p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG'),(lineitem.l_quantity >= 20.00),(lineitem.l_quantity <= 30.00),(part.p_size <= 15)]]) build RFs:RF0 p_partkey->[l_partkey] ------------PhysicalProject --------------filter((lineitem.l_quantity <= 30.00) and (lineitem.l_quantity >= 1.00) and (lineitem.l_shipinstruct = 'DELIVER IN PERSON') and l_shipmode IN ('AIR REG', 'AIR')) ----------------PhysicalOlapScan[lineitem] apply RFs: RF0 ------------PhysicalProject ---------------filter((((((part.p_brand = 'Brand#12') AND p_container IN ('SM BOX', 'SM CASE', 'SM PACK', 'SM PKG')) AND (part.p_size <= 5)) OR (((part.p_brand = 'Brand#23') AND p_container IN ('MED BAG', 'MED BOX', 'MED PACK', 'MED PKG')) AND (part.p_size <= 10))) OR (((part.p_brand = 'Brand#34') AND p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG')) AND (part.p_size <= 15))) and (part.p_size >= 1) and p_brand IN ('Brand#12', 'Brand#23', 'Brand#34') and p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG', 'MED BAG', 'MED BOX', 'MED PACK', 'MED PKG', 'SM BOX', 'SM CASE', 'SM PACK', 'SM PKG')) +--------------filter((part.p_size >= 1) and OR[AND[(part.p_brand = 'Brand#12'),p_container IN ('SM BOX', 'SM CASE', 'SM PACK', 'SM PKG'),(part.p_size <= 5)],AND[(part.p_brand = 'Brand#23'),p_container IN ('MED BAG', 'MED BOX', 'MED PACK', 'MED PKG'),(part.p_size <= 10)],AND[(part.p_brand = 'Brand#34'),p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG'),(part.p_size <= 15)]] and p_brand IN ('Brand#12', 'Brand#23', 'Brand#34') and p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG', 'MED BAG', 'MED BOX', 'MED PACK', 'MED PKG', 'SM BOX', 'SM CASE', 'SM PACK', 'SM PKG')) ----------------PhysicalOlapScan[part] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/nostats_rf_prune/q7.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/nostats_rf_prune/q7.out index 9c2ccaeb016726..e953eb4cbcee33 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/nostats_rf_prune/q7.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/nostats_rf_prune/q7.out @@ -24,7 +24,7 @@ PhysicalResultSink --------------------------------PhysicalOlapScan[supplier] apply RFs: RF4 ----------------------PhysicalProject ------------------------PhysicalOlapScan[customer] apply RFs: RF3 -------------------NestedLoopJoin[INNER_JOIN](((n1.n_name = 'FRANCE') AND (n2.n_name = 'GERMANY')) OR ((n1.n_name = 'GERMANY') AND (n2.n_name = 'FRANCE'))) +------------------NestedLoopJoin[INNER_JOIN]OR[AND[(n1.n_name = 'FRANCE'),(n2.n_name = 'GERMANY')],AND[(n1.n_name = 'GERMANY'),(n2.n_name = 'FRANCE')]] --------------------PhysicalProject ----------------------filter(n_name IN ('FRANCE', 'GERMANY')) ------------------------PhysicalOlapScan[nation] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q19.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q19.out index bf86e68a7f9837..ad82594176a219 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q19.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q19.out @@ -5,11 +5,11 @@ PhysicalResultSink ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] --------PhysicalProject -----------hashJoin[INNER_JOIN broadcast] hashCondition=((part.p_partkey = lineitem.l_partkey)) otherCondition=(((((((part.p_brand = 'Brand#12') AND p_container IN ('SM BOX', 'SM CASE', 'SM PACK', 'SM PKG')) AND ((lineitem.l_quantity >= 1.00) AND (lineitem.l_quantity <= 11.00))) AND (part.p_size <= 5)) OR ((((part.p_brand = 'Brand#23') AND p_container IN ('MED BAG', 'MED BOX', 'MED PACK', 'MED PKG')) AND ((lineitem.l_quantity >= 10.00) AND (lineitem.l_quantity <= 20.00))) AND (part.p_size <= 10))) OR ((((part.p_brand = 'Brand#34') AND p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG')) AND ((lineitem.l_quantity >= 20.00) AND (lineitem.l_quantity <= 30.00))) AND (part.p_size <= 15)))) build RFs:RF0 p_partkey->[l_partkey] +----------hashJoin[INNER_JOIN broadcast] hashCondition=((part.p_partkey = lineitem.l_partkey)) otherCondition=(OR[AND[(part.p_brand = 'Brand#12'),p_container IN ('SM BOX', 'SM CASE', 'SM PACK', 'SM PKG'),(lineitem.l_quantity >= 1.00),(lineitem.l_quantity <= 11.00),(part.p_size <= 5)],AND[(part.p_brand = 'Brand#23'),p_container IN ('MED BAG', 'MED BOX', 'MED PACK', 'MED PKG'),(lineitem.l_quantity >= 10.00),(lineitem.l_quantity <= 20.00),(part.p_size <= 10)],AND[(part.p_brand = 'Brand#34'),p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG'),(lineitem.l_quantity >= 20.00),(lineitem.l_quantity <= 30.00),(part.p_size <= 15)]]) build RFs:RF0 p_partkey->[l_partkey] ------------PhysicalProject --------------filter((lineitem.l_quantity <= 30.00) and (lineitem.l_quantity >= 1.00) and (lineitem.l_shipinstruct = 'DELIVER IN PERSON') and l_shipmode IN ('AIR REG', 'AIR')) ----------------PhysicalOlapScan[lineitem] apply RFs: RF0 ------------PhysicalProject ---------------filter((((((part.p_brand = 'Brand#12') AND p_container IN ('SM BOX', 'SM CASE', 'SM PACK', 'SM PKG')) AND (part.p_size <= 5)) OR (((part.p_brand = 'Brand#23') AND p_container IN ('MED BAG', 'MED BOX', 'MED PACK', 'MED PKG')) AND (part.p_size <= 10))) OR (((part.p_brand = 'Brand#34') AND p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG')) AND (part.p_size <= 15))) and (part.p_size >= 1) and p_brand IN ('Brand#12', 'Brand#23', 'Brand#34') and p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG', 'MED BAG', 'MED BOX', 'MED PACK', 'MED PKG', 'SM BOX', 'SM CASE', 'SM PACK', 'SM PKG')) +--------------filter((part.p_size >= 1) and OR[AND[(part.p_brand = 'Brand#12'),p_container IN ('SM BOX', 'SM CASE', 'SM PACK', 'SM PKG'),(part.p_size <= 5)],AND[(part.p_brand = 'Brand#23'),p_container IN ('MED BAG', 'MED BOX', 'MED PACK', 'MED PKG'),(part.p_size <= 10)],AND[(part.p_brand = 'Brand#34'),p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG'),(part.p_size <= 15)]] and p_brand IN ('Brand#12', 'Brand#23', 'Brand#34') and p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG', 'MED BAG', 'MED BOX', 'MED PACK', 'MED PKG', 'SM BOX', 'SM CASE', 'SM PACK', 'SM PKG')) ----------------PhysicalOlapScan[part] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q7.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q7.out index f1424cc0f730c4..957b17a7402749 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q7.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q7.out @@ -8,7 +8,7 @@ PhysicalResultSink ----------PhysicalDistribute[DistributionSpecHash] ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_custkey = orders.o_custkey)) otherCondition=((((n1.n_name = 'FRANCE') AND (n2.n_name = 'GERMANY')) OR ((n1.n_name = 'GERMANY') AND (n2.n_name = 'FRANCE')))) build RFs:RF4 c_custkey->[o_custkey] +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_custkey = orders.o_custkey)) otherCondition=(OR[AND[(n1.n_name = 'FRANCE'),(n2.n_name = 'GERMANY')],AND[(n1.n_name = 'GERMANY'),(n2.n_name = 'FRANCE')]]) build RFs:RF4 c_custkey->[o_custkey] ------------------PhysicalProject --------------------hashJoin[INNER_JOIN colocated] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=() build RFs:RF3 l_orderkey->[o_orderkey] ----------------------PhysicalProject diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q19.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q19.out index bf86e68a7f9837..ad82594176a219 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q19.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q19.out @@ -5,11 +5,11 @@ PhysicalResultSink ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] --------PhysicalProject -----------hashJoin[INNER_JOIN broadcast] hashCondition=((part.p_partkey = lineitem.l_partkey)) otherCondition=(((((((part.p_brand = 'Brand#12') AND p_container IN ('SM BOX', 'SM CASE', 'SM PACK', 'SM PKG')) AND ((lineitem.l_quantity >= 1.00) AND (lineitem.l_quantity <= 11.00))) AND (part.p_size <= 5)) OR ((((part.p_brand = 'Brand#23') AND p_container IN ('MED BAG', 'MED BOX', 'MED PACK', 'MED PKG')) AND ((lineitem.l_quantity >= 10.00) AND (lineitem.l_quantity <= 20.00))) AND (part.p_size <= 10))) OR ((((part.p_brand = 'Brand#34') AND p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG')) AND ((lineitem.l_quantity >= 20.00) AND (lineitem.l_quantity <= 30.00))) AND (part.p_size <= 15)))) build RFs:RF0 p_partkey->[l_partkey] +----------hashJoin[INNER_JOIN broadcast] hashCondition=((part.p_partkey = lineitem.l_partkey)) otherCondition=(OR[AND[(part.p_brand = 'Brand#12'),p_container IN ('SM BOX', 'SM CASE', 'SM PACK', 'SM PKG'),(lineitem.l_quantity >= 1.00),(lineitem.l_quantity <= 11.00),(part.p_size <= 5)],AND[(part.p_brand = 'Brand#23'),p_container IN ('MED BAG', 'MED BOX', 'MED PACK', 'MED PKG'),(lineitem.l_quantity >= 10.00),(lineitem.l_quantity <= 20.00),(part.p_size <= 10)],AND[(part.p_brand = 'Brand#34'),p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG'),(lineitem.l_quantity >= 20.00),(lineitem.l_quantity <= 30.00),(part.p_size <= 15)]]) build RFs:RF0 p_partkey->[l_partkey] ------------PhysicalProject --------------filter((lineitem.l_quantity <= 30.00) and (lineitem.l_quantity >= 1.00) and (lineitem.l_shipinstruct = 'DELIVER IN PERSON') and l_shipmode IN ('AIR REG', 'AIR')) ----------------PhysicalOlapScan[lineitem] apply RFs: RF0 ------------PhysicalProject ---------------filter((((((part.p_brand = 'Brand#12') AND p_container IN ('SM BOX', 'SM CASE', 'SM PACK', 'SM PKG')) AND (part.p_size <= 5)) OR (((part.p_brand = 'Brand#23') AND p_container IN ('MED BAG', 'MED BOX', 'MED PACK', 'MED PKG')) AND (part.p_size <= 10))) OR (((part.p_brand = 'Brand#34') AND p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG')) AND (part.p_size <= 15))) and (part.p_size >= 1) and p_brand IN ('Brand#12', 'Brand#23', 'Brand#34') and p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG', 'MED BAG', 'MED BOX', 'MED PACK', 'MED PKG', 'SM BOX', 'SM CASE', 'SM PACK', 'SM PKG')) +--------------filter((part.p_size >= 1) and OR[AND[(part.p_brand = 'Brand#12'),p_container IN ('SM BOX', 'SM CASE', 'SM PACK', 'SM PKG'),(part.p_size <= 5)],AND[(part.p_brand = 'Brand#23'),p_container IN ('MED BAG', 'MED BOX', 'MED PACK', 'MED PKG'),(part.p_size <= 10)],AND[(part.p_brand = 'Brand#34'),p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG'),(part.p_size <= 15)]] and p_brand IN ('Brand#12', 'Brand#23', 'Brand#34') and p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG', 'MED BAG', 'MED BOX', 'MED PACK', 'MED PKG', 'SM BOX', 'SM CASE', 'SM PACK', 'SM PKG')) ----------------PhysicalOlapScan[part] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q7.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q7.out index f1424cc0f730c4..957b17a7402749 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q7.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q7.out @@ -8,7 +8,7 @@ PhysicalResultSink ----------PhysicalDistribute[DistributionSpecHash] ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_custkey = orders.o_custkey)) otherCondition=((((n1.n_name = 'FRANCE') AND (n2.n_name = 'GERMANY')) OR ((n1.n_name = 'GERMANY') AND (n2.n_name = 'FRANCE')))) build RFs:RF4 c_custkey->[o_custkey] +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_custkey = orders.o_custkey)) otherCondition=(OR[AND[(n1.n_name = 'FRANCE'),(n2.n_name = 'GERMANY')],AND[(n1.n_name = 'GERMANY'),(n2.n_name = 'FRANCE')]]) build RFs:RF4 c_custkey->[o_custkey] ------------------PhysicalProject --------------------hashJoin[INNER_JOIN colocated] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=() build RFs:RF3 l_orderkey->[o_orderkey] ----------------------PhysicalProject diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q19.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q19.out index bf86e68a7f9837..ad82594176a219 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q19.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q19.out @@ -5,11 +5,11 @@ PhysicalResultSink ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] --------PhysicalProject -----------hashJoin[INNER_JOIN broadcast] hashCondition=((part.p_partkey = lineitem.l_partkey)) otherCondition=(((((((part.p_brand = 'Brand#12') AND p_container IN ('SM BOX', 'SM CASE', 'SM PACK', 'SM PKG')) AND ((lineitem.l_quantity >= 1.00) AND (lineitem.l_quantity <= 11.00))) AND (part.p_size <= 5)) OR ((((part.p_brand = 'Brand#23') AND p_container IN ('MED BAG', 'MED BOX', 'MED PACK', 'MED PKG')) AND ((lineitem.l_quantity >= 10.00) AND (lineitem.l_quantity <= 20.00))) AND (part.p_size <= 10))) OR ((((part.p_brand = 'Brand#34') AND p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG')) AND ((lineitem.l_quantity >= 20.00) AND (lineitem.l_quantity <= 30.00))) AND (part.p_size <= 15)))) build RFs:RF0 p_partkey->[l_partkey] +----------hashJoin[INNER_JOIN broadcast] hashCondition=((part.p_partkey = lineitem.l_partkey)) otherCondition=(OR[AND[(part.p_brand = 'Brand#12'),p_container IN ('SM BOX', 'SM CASE', 'SM PACK', 'SM PKG'),(lineitem.l_quantity >= 1.00),(lineitem.l_quantity <= 11.00),(part.p_size <= 5)],AND[(part.p_brand = 'Brand#23'),p_container IN ('MED BAG', 'MED BOX', 'MED PACK', 'MED PKG'),(lineitem.l_quantity >= 10.00),(lineitem.l_quantity <= 20.00),(part.p_size <= 10)],AND[(part.p_brand = 'Brand#34'),p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG'),(lineitem.l_quantity >= 20.00),(lineitem.l_quantity <= 30.00),(part.p_size <= 15)]]) build RFs:RF0 p_partkey->[l_partkey] ------------PhysicalProject --------------filter((lineitem.l_quantity <= 30.00) and (lineitem.l_quantity >= 1.00) and (lineitem.l_shipinstruct = 'DELIVER IN PERSON') and l_shipmode IN ('AIR REG', 'AIR')) ----------------PhysicalOlapScan[lineitem] apply RFs: RF0 ------------PhysicalProject ---------------filter((((((part.p_brand = 'Brand#12') AND p_container IN ('SM BOX', 'SM CASE', 'SM PACK', 'SM PKG')) AND (part.p_size <= 5)) OR (((part.p_brand = 'Brand#23') AND p_container IN ('MED BAG', 'MED BOX', 'MED PACK', 'MED PKG')) AND (part.p_size <= 10))) OR (((part.p_brand = 'Brand#34') AND p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG')) AND (part.p_size <= 15))) and (part.p_size >= 1) and p_brand IN ('Brand#12', 'Brand#23', 'Brand#34') and p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG', 'MED BAG', 'MED BOX', 'MED PACK', 'MED PKG', 'SM BOX', 'SM CASE', 'SM PACK', 'SM PKG')) +--------------filter((part.p_size >= 1) and OR[AND[(part.p_brand = 'Brand#12'),p_container IN ('SM BOX', 'SM CASE', 'SM PACK', 'SM PKG'),(part.p_size <= 5)],AND[(part.p_brand = 'Brand#23'),p_container IN ('MED BAG', 'MED BOX', 'MED PACK', 'MED PKG'),(part.p_size <= 10)],AND[(part.p_brand = 'Brand#34'),p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG'),(part.p_size <= 15)]] and p_brand IN ('Brand#12', 'Brand#23', 'Brand#34') and p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG', 'MED BAG', 'MED BOX', 'MED PACK', 'MED PKG', 'SM BOX', 'SM CASE', 'SM PACK', 'SM PKG')) ----------------PhysicalOlapScan[part] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q7.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q7.out index 9c2ccaeb016726..e953eb4cbcee33 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q7.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q7.out @@ -24,7 +24,7 @@ PhysicalResultSink --------------------------------PhysicalOlapScan[supplier] apply RFs: RF4 ----------------------PhysicalProject ------------------------PhysicalOlapScan[customer] apply RFs: RF3 -------------------NestedLoopJoin[INNER_JOIN](((n1.n_name = 'FRANCE') AND (n2.n_name = 'GERMANY')) OR ((n1.n_name = 'GERMANY') AND (n2.n_name = 'FRANCE'))) +------------------NestedLoopJoin[INNER_JOIN]OR[AND[(n1.n_name = 'FRANCE'),(n2.n_name = 'GERMANY')],AND[(n1.n_name = 'GERMANY'),(n2.n_name = 'FRANCE')]] --------------------PhysicalProject ----------------------filter(n_name IN ('FRANCE', 'GERMANY')) ------------------------PhysicalOlapScan[nation] diff --git a/regression-test/data/new_shapes_p0/hint_tpcds/shape/query78.out b/regression-test/data/new_shapes_p0/hint_tpcds/shape/query78.out index 52ab844c3f1bbc..e57834b15ff42a 100644 --- a/regression-test/data/new_shapes_p0/hint_tpcds/shape/query78.out +++ b/regression-test/data/new_shapes_p0/hint_tpcds/shape/query78.out @@ -5,7 +5,7 @@ PhysicalResultSink ----PhysicalDistribute[DistributionSpecGather] ------PhysicalTopN[LOCAL_SORT] --------PhysicalProject -----------filter(((coalesce(ws_qty, 0) > 0) OR (coalesce(cs_qty, 0) > 0))) +----------filter(OR[(coalesce(ws_qty, 0) > 0),(coalesce(cs_qty, 0) > 0)]) ------------hashJoin[LEFT_OUTER_JOIN colocated] hashCondition=((cs.cs_customer_sk = ss.ss_customer_sk) and (cs.cs_item_sk = ss.ss_item_sk) and (cs.cs_sold_year = ss.ss_sold_year)) otherCondition=() --------------PhysicalProject ----------------hashJoin[LEFT_OUTER_JOIN colocated] hashCondition=((ws.ws_customer_sk = ss.ss_customer_sk) and (ws.ws_item_sk = ss.ss_item_sk) and (ws.ws_sold_year = ss.ss_sold_year)) otherCondition=() diff --git a/regression-test/data/new_shapes_p0/hint_tpch/shape/q19.out b/regression-test/data/new_shapes_p0/hint_tpch/shape/q19.out index 940b16c90e40d5..5cabdf9f163c56 100644 --- a/regression-test/data/new_shapes_p0/hint_tpch/shape/q19.out +++ b/regression-test/data/new_shapes_p0/hint_tpch/shape/q19.out @@ -5,12 +5,12 @@ PhysicalResultSink ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] --------PhysicalProject -----------hashJoin[INNER_JOIN broadcast] hashCondition=((part.p_partkey = lineitem.l_partkey)) otherCondition=(((((((part.p_brand = 'Brand#12') AND p_container IN ('SM BOX', 'SM CASE', 'SM PACK', 'SM PKG')) AND ((lineitem.l_quantity >= 1.00) AND (lineitem.l_quantity <= 11.00))) AND (part.p_size <= 5)) OR ((((part.p_brand = 'Brand#23') AND p_container IN ('MED BAG', 'MED BOX', 'MED PACK', 'MED PKG')) AND ((lineitem.l_quantity >= 10.00) AND (lineitem.l_quantity <= 20.00))) AND (part.p_size <= 10))) OR ((((part.p_brand = 'Brand#34') AND p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG')) AND ((lineitem.l_quantity >= 20.00) AND (lineitem.l_quantity <= 30.00))) AND (part.p_size <= 15)))) +----------hashJoin[INNER_JOIN broadcast] hashCondition=((part.p_partkey = lineitem.l_partkey)) otherCondition=(OR[AND[(part.p_brand = 'Brand#12'),p_container IN ('SM BOX', 'SM CASE', 'SM PACK', 'SM PKG'),(lineitem.l_quantity >= 1.00),(lineitem.l_quantity <= 11.00),(part.p_size <= 5)],AND[(part.p_brand = 'Brand#23'),p_container IN ('MED BAG', 'MED BOX', 'MED PACK', 'MED PKG'),(lineitem.l_quantity >= 10.00),(lineitem.l_quantity <= 20.00),(part.p_size <= 10)],AND[(part.p_brand = 'Brand#34'),p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG'),(lineitem.l_quantity >= 20.00),(lineitem.l_quantity <= 30.00),(part.p_size <= 15)]]) ------------PhysicalProject --------------filter((lineitem.l_quantity <= 30.00) and (lineitem.l_quantity >= 1.00) and (lineitem.l_shipinstruct = 'DELIVER IN PERSON') and l_shipmode IN ('AIR REG', 'AIR')) ----------------PhysicalOlapScan[lineitem] ------------PhysicalProject ---------------filter((((((part.p_brand = 'Brand#12') AND p_container IN ('SM BOX', 'SM CASE', 'SM PACK', 'SM PKG')) AND (part.p_size <= 5)) OR (((part.p_brand = 'Brand#23') AND p_container IN ('MED BAG', 'MED BOX', 'MED PACK', 'MED PKG')) AND (part.p_size <= 10))) OR (((part.p_brand = 'Brand#34') AND p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG')) AND (part.p_size <= 15))) and (part.p_size >= 1) and p_brand IN ('Brand#12', 'Brand#23', 'Brand#34') and p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG', 'MED BAG', 'MED BOX', 'MED PACK', 'MED PKG', 'SM BOX', 'SM CASE', 'SM PACK', 'SM PKG')) +--------------filter((part.p_size >= 1) and OR[AND[(part.p_brand = 'Brand#12'),p_container IN ('SM BOX', 'SM CASE', 'SM PACK', 'SM PKG'),(part.p_size <= 5)],AND[(part.p_brand = 'Brand#23'),p_container IN ('MED BAG', 'MED BOX', 'MED PACK', 'MED PKG'),(part.p_size <= 10)],AND[(part.p_brand = 'Brand#34'),p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG'),(part.p_size <= 15)]] and p_brand IN ('Brand#12', 'Brand#23', 'Brand#34') and p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG', 'MED BAG', 'MED BOX', 'MED PACK', 'MED PKG', 'SM BOX', 'SM CASE', 'SM PACK', 'SM PKG')) ----------------PhysicalOlapScan[part] Hint log: diff --git a/regression-test/data/new_shapes_p0/hint_tpch/shape/q7.out b/regression-test/data/new_shapes_p0/hint_tpch/shape/q7.out index 74cca15a749172..919b8547bb8a69 100644 --- a/regression-test/data/new_shapes_p0/hint_tpch/shape/q7.out +++ b/regression-test/data/new_shapes_p0/hint_tpch/shape/q7.out @@ -8,7 +8,7 @@ PhysicalResultSink ----------PhysicalDistribute[DistributionSpecHash] ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=((((n1.n_name = 'FRANCE') AND (n2.n_name = 'GERMANY')) OR ((n1.n_name = 'GERMANY') AND (n2.n_name = 'FRANCE')))) +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=(OR[AND[(n1.n_name = 'FRANCE'),(n2.n_name = 'GERMANY')],AND[(n1.n_name = 'GERMANY'),(n2.n_name = 'FRANCE')]]) ------------------PhysicalProject --------------------hashJoin[INNER_JOIN broadcast] hashCondition=((supplier.s_suppkey = lineitem.l_suppkey)) otherCondition=() ----------------------PhysicalProject diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query10.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query10.out index bc11b07f3a3809..5fb40519b131d5 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query10.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query10.out @@ -9,7 +9,7 @@ PhysicalResultSink ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) +------------------filter(OR[ifnull($c$1, FALSE),ifnull($c$2, FALSE)]) --------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() ----------------------PhysicalProject ------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF5 ca_address_sk->[c_current_addr_sk] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query13.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query13.out index 6df93fe3046d5c..9c7d9382777bee 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query13.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query13.out @@ -7,9 +7,9 @@ PhysicalResultSink --------PhysicalProject ----------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] ------------PhysicalProject ---------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=((((ca_state IN ('KS', 'MI', 'SD') AND ((store_sales.ss_net_profit >= 100.00) AND (store_sales.ss_net_profit <= 200.00))) OR (ca_state IN ('CO', 'MO', 'ND') AND ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 300.00)))) OR (ca_state IN ('NH', 'OH', 'TX') AND ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 250.00))))) build RFs:RF3 ca_address_sk->[ss_addr_sk] +--------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=(OR[AND[ca_state IN ('KS', 'MI', 'SD'),(store_sales.ss_net_profit >= 100.00),(store_sales.ss_net_profit <= 200.00)],AND[ca_state IN ('CO', 'MO', 'ND'),(store_sales.ss_net_profit >= 150.00),(store_sales.ss_net_profit <= 300.00)],AND[ca_state IN ('NH', 'OH', 'TX'),(store_sales.ss_net_profit >= 50.00),(store_sales.ss_net_profit <= 250.00)]]) build RFs:RF3 ca_address_sk->[ss_addr_sk] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=((((((household_demographics.hd_dep_count = 1) AND cd_marital_status IN ('M', 'S')) AND cd_education_status IN ('4 yr Degree', 'College')) AND ((((customer_demographics.cd_marital_status = 'S') AND (customer_demographics.cd_education_status = 'College')) AND ((store_sales.ss_sales_price >= 50.00) AND (store_sales.ss_sales_price <= 100.00))) OR (((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = '4 yr Degree')) AND ((store_sales.ss_sales_price >= 150.00) AND (store_sales.ss_sales_price <= 200.00))))) OR ((((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = 'Unknown')) AND ((store_sales.ss_sales_price >= 100.00) AND (store_sales.ss_sales_price <= 150.00))) AND (household_demographics.hd_dep_count = 3)))) build RFs:RF2 hd_demo_sk->[ss_hdemo_sk] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=(OR[AND[(household_demographics.hd_dep_count = 1),cd_marital_status IN ('M', 'S'),cd_education_status IN ('4 yr Degree', 'College'),OR[AND[(customer_demographics.cd_marital_status = 'S'),(customer_demographics.cd_education_status = 'College'),(store_sales.ss_sales_price >= 50.00),(store_sales.ss_sales_price <= 100.00)],AND[(customer_demographics.cd_marital_status = 'M'),(customer_demographics.cd_education_status = '4 yr Degree'),(store_sales.ss_sales_price >= 150.00),(store_sales.ss_sales_price <= 200.00)]]],AND[(customer_demographics.cd_marital_status = 'D'),(customer_demographics.cd_education_status = 'Unknown'),(store_sales.ss_sales_price >= 100.00),(store_sales.ss_sales_price <= 150.00),(household_demographics.hd_dep_count = 3)]]) build RFs:RF2 hd_demo_sk->[ss_hdemo_sk] --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)) otherCondition=() build RFs:RF1 cd_demo_sk->[ss_cdemo_sk] ------------------------PhysicalProject @@ -20,7 +20,7 @@ PhysicalResultSink ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[store] ------------------------PhysicalProject ---------------------------filter(((((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = 'Unknown')) OR ((customer_demographics.cd_marital_status = 'S') AND (customer_demographics.cd_education_status = 'College'))) OR ((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = '4 yr Degree'))) and cd_education_status IN ('4 yr Degree', 'College', 'Unknown') and cd_marital_status IN ('D', 'M', 'S')) +--------------------------filter(OR[AND[(customer_demographics.cd_marital_status = 'D'),(customer_demographics.cd_education_status = 'Unknown')],AND[(customer_demographics.cd_marital_status = 'S'),(customer_demographics.cd_education_status = 'College')],AND[(customer_demographics.cd_marital_status = 'M'),(customer_demographics.cd_education_status = '4 yr Degree')]] and cd_education_status IN ('4 yr Degree', 'College', 'Unknown') and cd_marital_status IN ('D', 'M', 'S')) ----------------------------PhysicalOlapScan[customer_demographics] --------------------PhysicalProject ----------------------filter(hd_dep_count IN (1, 3)) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query15.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query15.out index 7b0fd9b10fd861..5825559155b8e7 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query15.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query15.out @@ -10,7 +10,7 @@ PhysicalResultSink --------------PhysicalProject ----------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=(((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR ca_state IN ('CA', 'GA', 'WA')) OR (catalog_sales.cs_sales_price > 500.00))) +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),ca_state IN ('CA', 'GA', 'WA'),(catalog_sales.cs_sales_price > 500.00)]) ----------------------PhysicalProject ------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() --------------------------PhysicalProject diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query26.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query26.out index 3762fabaeead07..52f628f8b600a2 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query26.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query26.out @@ -26,6 +26,6 @@ PhysicalResultSink ------------------------filter((date_dim.d_year = 2001)) --------------------------PhysicalOlapScan[date_dim] ------------------PhysicalProject ---------------------filter(((promotion.p_channel_email = 'N') OR (promotion.p_channel_event = 'N'))) +--------------------filter(OR[(promotion.p_channel_email = 'N'),(promotion.p_channel_event = 'N')]) ----------------------PhysicalOlapScan[promotion] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query28.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query28.out index dae7cb1c23a402..7a6bdd8868ef00 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query28.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query28.out @@ -17,41 +17,41 @@ PhysicalResultSink ----------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------hashAgg[LOCAL] --------------------------------PhysicalProject -----------------------------------filter(((((store_sales.ss_list_price >= 131.00) AND (store_sales.ss_list_price <= 141.00)) OR ((store_sales.ss_coupon_amt >= 16798.00) AND (store_sales.ss_coupon_amt <= 17798.00))) OR ((store_sales.ss_wholesale_cost >= 25.00) AND (store_sales.ss_wholesale_cost <= 45.00))) and (store_sales.ss_quantity <= 5) and (store_sales.ss_quantity >= 0)) +----------------------------------filter((store_sales.ss_quantity <= 5) and (store_sales.ss_quantity >= 0) and OR[AND[(store_sales.ss_list_price >= 131.00),(store_sales.ss_list_price <= 141.00)],AND[(store_sales.ss_coupon_amt >= 16798.00),(store_sales.ss_coupon_amt <= 17798.00)],AND[(store_sales.ss_wholesale_cost >= 25.00),(store_sales.ss_wholesale_cost <= 45.00)]]) ------------------------------------PhysicalOlapScan[store_sales] ------------------------PhysicalLimit[LOCAL] --------------------------hashAgg[GLOBAL] ----------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------hashAgg[LOCAL] --------------------------------PhysicalProject -----------------------------------filter(((((store_sales.ss_list_price >= 145.00) AND (store_sales.ss_list_price <= 155.00)) OR ((store_sales.ss_coupon_amt >= 14792.00) AND (store_sales.ss_coupon_amt <= 15792.00))) OR ((store_sales.ss_wholesale_cost >= 46.00) AND (store_sales.ss_wholesale_cost <= 66.00))) and (store_sales.ss_quantity <= 10) and (store_sales.ss_quantity >= 6)) +----------------------------------filter((store_sales.ss_quantity <= 10) and (store_sales.ss_quantity >= 6) and OR[AND[(store_sales.ss_list_price >= 145.00),(store_sales.ss_list_price <= 155.00)],AND[(store_sales.ss_coupon_amt >= 14792.00),(store_sales.ss_coupon_amt <= 15792.00)],AND[(store_sales.ss_wholesale_cost >= 46.00),(store_sales.ss_wholesale_cost <= 66.00)]]) ------------------------------------PhysicalOlapScan[store_sales] --------------------PhysicalLimit[LOCAL] ----------------------hashAgg[GLOBAL] ------------------------PhysicalDistribute[DistributionSpecGather] --------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------filter(((((store_sales.ss_list_price >= 150.00) AND (store_sales.ss_list_price <= 160.00)) OR ((store_sales.ss_coupon_amt >= 6600.00) AND (store_sales.ss_coupon_amt <= 7600.00))) OR ((store_sales.ss_wholesale_cost >= 9.00) AND (store_sales.ss_wholesale_cost <= 29.00))) and (store_sales.ss_quantity <= 15) and (store_sales.ss_quantity >= 11)) +------------------------------filter((store_sales.ss_quantity <= 15) and (store_sales.ss_quantity >= 11) and OR[AND[(store_sales.ss_list_price >= 150.00),(store_sales.ss_list_price <= 160.00)],AND[(store_sales.ss_coupon_amt >= 6600.00),(store_sales.ss_coupon_amt <= 7600.00)],AND[(store_sales.ss_wholesale_cost >= 9.00),(store_sales.ss_wholesale_cost <= 29.00)]]) --------------------------------PhysicalOlapScan[store_sales] ----------------PhysicalLimit[LOCAL] ------------------hashAgg[GLOBAL] --------------------PhysicalDistribute[DistributionSpecGather] ----------------------hashAgg[LOCAL] ------------------------PhysicalProject ---------------------------filter(((((store_sales.ss_list_price >= 91.00) AND (store_sales.ss_list_price <= 101.00)) OR ((store_sales.ss_coupon_amt >= 13493.00) AND (store_sales.ss_coupon_amt <= 14493.00))) OR ((store_sales.ss_wholesale_cost >= 36.00) AND (store_sales.ss_wholesale_cost <= 56.00))) and (store_sales.ss_quantity <= 20) and (store_sales.ss_quantity >= 16)) +--------------------------filter((store_sales.ss_quantity <= 20) and (store_sales.ss_quantity >= 16) and OR[AND[(store_sales.ss_list_price >= 91.00),(store_sales.ss_list_price <= 101.00)],AND[(store_sales.ss_coupon_amt >= 13493.00),(store_sales.ss_coupon_amt <= 14493.00)],AND[(store_sales.ss_wholesale_cost >= 36.00),(store_sales.ss_wholesale_cost <= 56.00)]]) ----------------------------PhysicalOlapScan[store_sales] ------------PhysicalLimit[LOCAL] --------------hashAgg[GLOBAL] ----------------PhysicalDistribute[DistributionSpecGather] ------------------hashAgg[LOCAL] --------------------PhysicalProject -----------------------filter(((((store_sales.ss_list_price >= 0.00) AND (store_sales.ss_list_price <= 10.00)) OR ((store_sales.ss_coupon_amt >= 7629.00) AND (store_sales.ss_coupon_amt <= 8629.00))) OR ((store_sales.ss_wholesale_cost >= 6.00) AND (store_sales.ss_wholesale_cost <= 26.00))) and (store_sales.ss_quantity <= 25) and (store_sales.ss_quantity >= 21)) +----------------------filter((store_sales.ss_quantity <= 25) and (store_sales.ss_quantity >= 21) and OR[AND[(store_sales.ss_list_price >= 0.00),(store_sales.ss_list_price <= 10.00)],AND[(store_sales.ss_coupon_amt >= 7629.00),(store_sales.ss_coupon_amt <= 8629.00)],AND[(store_sales.ss_wholesale_cost >= 6.00),(store_sales.ss_wholesale_cost <= 26.00)]]) ------------------------PhysicalOlapScan[store_sales] --------PhysicalLimit[LOCAL] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecGather] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------filter(((((store_sales.ss_list_price >= 89.00) AND (store_sales.ss_list_price <= 99.00)) OR ((store_sales.ss_coupon_amt >= 15257.00) AND (store_sales.ss_coupon_amt <= 16257.00))) OR ((store_sales.ss_wholesale_cost >= 31.00) AND (store_sales.ss_wholesale_cost <= 51.00))) and (store_sales.ss_quantity <= 30) and (store_sales.ss_quantity >= 26)) +------------------filter((store_sales.ss_quantity <= 30) and (store_sales.ss_quantity >= 26) and OR[AND[(store_sales.ss_list_price >= 89.00),(store_sales.ss_list_price <= 99.00)],AND[(store_sales.ss_coupon_amt >= 15257.00),(store_sales.ss_coupon_amt <= 16257.00)],AND[(store_sales.ss_wholesale_cost >= 31.00),(store_sales.ss_wholesale_cost <= 51.00)]]) --------------------PhysicalOlapScan[store_sales] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query34.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query34.out index dad99a6b6465dd..6c90e3a48d7cba 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query34.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query34.out @@ -19,7 +19,7 @@ PhysicalResultSink --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 --------------------------------PhysicalProject -----------------------------------filter((((date_dim.d_dom >= 1) AND (date_dim.d_dom <= 3)) OR ((date_dim.d_dom >= 25) AND (date_dim.d_dom <= 28))) and d_year IN (1998, 1999, 2000)) +----------------------------------filter(OR[AND[(date_dim.d_dom >= 1),(date_dim.d_dom <= 3)],AND[(date_dim.d_dom >= 25),(date_dim.d_dom <= 28)]] and d_year IN (1998, 1999, 2000)) ------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalProject ------------------------------filter(s_county IN ('Barrow County', 'Daviess County', 'Franklin Parish', 'Luce County', 'Richland County', 'Walker County', 'Williamson County', 'Ziebach County')) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query35.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query35.out index a3f752e2ce0e3d..6def6ef536b340 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query35.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query35.out @@ -9,7 +9,7 @@ PhysicalResultSink ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) +------------------filter(OR[ifnull($c$1, FALSE),ifnull($c$2, FALSE)]) --------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() ----------------------PhysicalProject ------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query41.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query41.out index c9c1ffff499741..3034a77fe0897a 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query41.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query41.out @@ -18,6 +18,6 @@ PhysicalResultSink ------------------------PhysicalDistribute[DistributionSpecHash] --------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------filter((((i_color IN ('aquamarine', 'blue', 'chartreuse', 'chiffon', 'dodger', 'gainsboro', 'tan', 'violet') AND i_units IN ('Bunch', 'Dozen', 'Each', 'Ounce', 'Oz', 'Pound', 'Ton', 'Tsp')) AND (((((((item.i_category = 'Women') AND i_color IN ('aquamarine', 'gainsboro')) AND i_units IN ('Dozen', 'Ounce')) AND i_size IN ('economy', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('chiffon', 'violet')) AND i_units IN ('Pound', 'Ton')) AND i_size IN ('extra large', 'small'))) OR ((((item.i_category = 'Men') AND i_color IN ('blue', 'chartreuse')) AND i_units IN ('Each', 'Oz')) AND i_size IN ('N/A', 'large'))) OR ((((item.i_category = 'Men') AND i_color IN ('dodger', 'tan')) AND i_units IN ('Bunch', 'Tsp')) AND i_size IN ('economy', 'medium')))) OR ((i_color IN ('almond', 'blanched', 'indian', 'lime', 'peru', 'saddle', 'spring', 'tomato') AND i_units IN ('Box', 'Carton', 'Case', 'Dram', 'Gram', 'Pallet', 'Tbl', 'Unknown')) AND (((((((item.i_category = 'Women') AND i_color IN ('blanched', 'tomato')) AND i_units IN ('Case', 'Tbl')) AND i_size IN ('economy', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('almond', 'lime')) AND i_units IN ('Box', 'Dram')) AND i_size IN ('extra large', 'small'))) OR ((((item.i_category = 'Men') AND i_color IN ('peru', 'saddle')) AND i_units IN ('Gram', 'Pallet')) AND i_size IN ('N/A', 'large'))) OR ((((item.i_category = 'Men') AND i_color IN ('indian', 'spring')) AND i_units IN ('Carton', 'Unknown')) AND i_size IN ('economy', 'medium'))))) and i_category IN ('Men', 'Women') and i_size IN ('N/A', 'economy', 'extra large', 'large', 'medium', 'small')) +------------------------------filter(OR[AND[i_color IN ('aquamarine', 'blue', 'chartreuse', 'chiffon', 'dodger', 'gainsboro', 'tan', 'violet'),i_units IN ('Bunch', 'Dozen', 'Each', 'Ounce', 'Oz', 'Pound', 'Ton', 'Tsp'),OR[AND[(item.i_category = 'Women'),i_color IN ('aquamarine', 'gainsboro'),i_units IN ('Dozen', 'Ounce'),i_size IN ('economy', 'medium')],AND[(item.i_category = 'Women'),i_color IN ('chiffon', 'violet'),i_units IN ('Pound', 'Ton'),i_size IN ('extra large', 'small')],AND[(item.i_category = 'Men'),i_color IN ('blue', 'chartreuse'),i_units IN ('Each', 'Oz'),i_size IN ('N/A', 'large')],AND[(item.i_category = 'Men'),i_color IN ('dodger', 'tan'),i_units IN ('Bunch', 'Tsp'),i_size IN ('economy', 'medium')]]],AND[i_color IN ('almond', 'blanched', 'indian', 'lime', 'peru', 'saddle', 'spring', 'tomato'),i_units IN ('Box', 'Carton', 'Case', 'Dram', 'Gram', 'Pallet', 'Tbl', 'Unknown'),OR[AND[(item.i_category = 'Women'),i_color IN ('blanched', 'tomato'),i_units IN ('Case', 'Tbl'),i_size IN ('economy', 'medium')],AND[(item.i_category = 'Women'),i_color IN ('almond', 'lime'),i_units IN ('Box', 'Dram'),i_size IN ('extra large', 'small')],AND[(item.i_category = 'Men'),i_color IN ('peru', 'saddle'),i_units IN ('Gram', 'Pallet'),i_size IN ('N/A', 'large')],AND[(item.i_category = 'Men'),i_color IN ('indian', 'spring'),i_units IN ('Carton', 'Unknown'),i_size IN ('economy', 'medium')]]]] and i_category IN ('Men', 'Women') and i_size IN ('N/A', 'economy', 'extra large', 'large', 'medium', 'small')) --------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query45.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query45.out index c6b4daafdce0df..7270fe9092a53b 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query45.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query45.out @@ -8,7 +8,7 @@ PhysicalResultSink ----------PhysicalDistribute[DistributionSpecHash] ------------hashAgg[LOCAL] --------------PhysicalProject -----------------filter((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR $c$1)) +----------------filter(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),$c$1]) ------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query46.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query46.out index 495863c5dc9ef9..37d045d1ebc4a1 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query46.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query46.out @@ -27,7 +27,7 @@ PhysicalResultSink ----------------------------------filter(s_city IN ('Centerville', 'Fairview', 'Five Points', 'Liberty', 'Oak Grove')) ------------------------------------PhysicalOlapScan[store] ----------------------------PhysicalProject -------------------------------filter(((household_demographics.hd_dep_count = 6) OR (household_demographics.hd_vehicle_count = 0))) +------------------------------filter(OR[(household_demographics.hd_dep_count = 6),(household_demographics.hd_vehicle_count = 0)]) --------------------------------PhysicalOlapScan[household_demographics] ------------------------PhysicalProject --------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query47.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query47.out index 0182bbf46eb817..77dcc9357f98b8 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query47.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query47.out @@ -23,7 +23,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------------------------PhysicalProject --------------------------------------PhysicalOlapScan[item] --------------------------------PhysicalProject -----------------------------------filter((((date_dim.d_year = 2001) OR ((date_dim.d_year = 2000) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2002) AND (date_dim.d_moy = 1))) and d_year IN (2000, 2001, 2002)) +----------------------------------filter(OR[(date_dim.d_year = 2001),AND[(date_dim.d_year = 2000),(date_dim.d_moy = 12)],AND[(date_dim.d_year = 2002),(date_dim.d_moy = 1)]] and d_year IN (2000, 2001, 2002)) ------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query48.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query48.out index 338a70100eb21b..57618c21f263d3 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query48.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query48.out @@ -7,9 +7,9 @@ PhysicalResultSink --------PhysicalProject ----------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ss_sold_date_sk] ------------PhysicalProject ---------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=((((ca_state IN ('IA', 'MD', 'MN') AND ((store_sales.ss_net_profit >= 0.00) AND (store_sales.ss_net_profit <= 2000.00))) OR (ca_state IN ('IL', 'TX', 'VA') AND ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 3000.00)))) OR (ca_state IN ('IN', 'MI', 'WI') AND ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 25000.00))))) build RFs:RF2 ca_address_sk->[ss_addr_sk] +--------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=(OR[AND[ca_state IN ('IA', 'MD', 'MN'),(store_sales.ss_net_profit >= 0.00),(store_sales.ss_net_profit <= 2000.00)],AND[ca_state IN ('IL', 'TX', 'VA'),(store_sales.ss_net_profit >= 150.00),(store_sales.ss_net_profit <= 3000.00)],AND[ca_state IN ('IN', 'MI', 'WI'),(store_sales.ss_net_profit >= 50.00),(store_sales.ss_net_profit <= 25000.00)]]) build RFs:RF2 ca_address_sk->[ss_addr_sk] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)) otherCondition=((((((customer_demographics.cd_marital_status = 'U') AND (customer_demographics.cd_education_status = 'Primary')) AND ((store_sales.ss_sales_price >= 100.00) AND (store_sales.ss_sales_price <= 150.00))) OR (((customer_demographics.cd_marital_status = 'W') AND (customer_demographics.cd_education_status = 'College')) AND ((store_sales.ss_sales_price >= 50.00) AND (store_sales.ss_sales_price <= 100.00)))) OR (((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = '2 yr Degree')) AND ((store_sales.ss_sales_price >= 150.00) AND (store_sales.ss_sales_price <= 200.00))))) build RFs:RF1 cd_demo_sk->[ss_cdemo_sk] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)) otherCondition=(OR[AND[(customer_demographics.cd_marital_status = 'U'),(customer_demographics.cd_education_status = 'Primary'),(store_sales.ss_sales_price >= 100.00),(store_sales.ss_sales_price <= 150.00)],AND[(customer_demographics.cd_marital_status = 'W'),(customer_demographics.cd_education_status = 'College'),(store_sales.ss_sales_price >= 50.00),(store_sales.ss_sales_price <= 100.00)],AND[(customer_demographics.cd_marital_status = 'D'),(customer_demographics.cd_education_status = '2 yr Degree'),(store_sales.ss_sales_price >= 150.00),(store_sales.ss_sales_price <= 200.00)]]) build RFs:RF1 cd_demo_sk->[ss_cdemo_sk] --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() ------------------------PhysicalProject @@ -18,7 +18,7 @@ PhysicalResultSink ------------------------PhysicalProject --------------------------PhysicalOlapScan[store] --------------------PhysicalProject -----------------------filter(((((customer_demographics.cd_marital_status = 'U') AND (customer_demographics.cd_education_status = 'Primary')) OR ((customer_demographics.cd_marital_status = 'W') AND (customer_demographics.cd_education_status = 'College'))) OR ((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = '2 yr Degree'))) and cd_education_status IN ('2 yr Degree', 'College', 'Primary') and cd_marital_status IN ('D', 'U', 'W')) +----------------------filter(OR[AND[(customer_demographics.cd_marital_status = 'U'),(customer_demographics.cd_education_status = 'Primary')],AND[(customer_demographics.cd_marital_status = 'W'),(customer_demographics.cd_education_status = 'College')],AND[(customer_demographics.cd_marital_status = 'D'),(customer_demographics.cd_education_status = '2 yr Degree')]] and cd_education_status IN ('2 yr Degree', 'College', 'Primary') and cd_marital_status IN ('D', 'U', 'W')) ------------------------PhysicalOlapScan[customer_demographics] ----------------PhysicalProject ------------------filter((customer_address.ca_country = 'United States') and ca_state IN ('IA', 'IL', 'IN', 'MD', 'MI', 'MN', 'TX', 'VA', 'WI')) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query49.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query49.out index dc56d7f9e81411..0db3fa841189b2 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query49.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query49.out @@ -16,7 +16,7 @@ PhysicalResultSink --------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) ----------------------------------PhysicalWindow ------------------------------------PhysicalQuickSort[LOCAL_SORT] --------------------------------------PhysicalWindow @@ -48,7 +48,7 @@ PhysicalResultSink --------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) ----------------------------------PhysicalWindow ------------------------------------PhysicalQuickSort[LOCAL_SORT] --------------------------------------PhysicalWindow @@ -80,7 +80,7 @@ PhysicalResultSink --------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) ----------------------------------PhysicalWindow ------------------------------------PhysicalQuickSort[LOCAL_SORT] --------------------------------------PhysicalWindow diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query53.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query53.out index 93289fca48e4ef..89dc632eb527c4 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query53.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query53.out @@ -22,7 +22,7 @@ PhysicalResultSink --------------------------------------PhysicalProject ----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 --------------------------------------PhysicalProject -----------------------------------------filter((((i_category IN ('Books', 'Children', 'Electronics') AND i_class IN ('personal', 'portable', 'reference', 'self-help')) AND i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')) OR ((i_category IN ('Men', 'Music', 'Women') AND i_class IN ('accessories', 'classical', 'fragrances', 'pants')) AND i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1'))) and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) +----------------------------------------filter(OR[AND[i_category IN ('Books', 'Children', 'Electronics'),i_class IN ('personal', 'portable', 'reference', 'self-help'),i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')],AND[i_category IN ('Men', 'Music', 'Women'),i_class IN ('accessories', 'classical', 'fragrances', 'pants'),i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')]] and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) ------------------------------------------PhysicalOlapScan[item] ----------------------------------PhysicalProject ------------------------------------filter(d_month_seq IN (1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211)) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query57.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query57.out index 842b18a4eac9e6..96d8f68090e5de 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query57.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query57.out @@ -23,7 +23,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------------------------PhysicalProject --------------------------------------PhysicalOlapScan[item] --------------------------------PhysicalProject -----------------------------------filter((((date_dim.d_year = 1999) OR ((date_dim.d_year = 1998) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2000) AND (date_dim.d_moy = 1))) and d_year IN (1998, 1999, 2000)) +----------------------------------filter(OR[(date_dim.d_year = 1999),AND[(date_dim.d_year = 1998),(date_dim.d_moy = 12)],AND[(date_dim.d_year = 2000),(date_dim.d_moy = 1)]] and d_year IN (1998, 1999, 2000)) ------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[call_center] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query61.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query61.out index f9dcb3c9460ca4..62da8c9cb21a0f 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query61.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query61.out @@ -27,7 +27,7 @@ PhysicalResultSink ------------------------------------filter((store.s_gmt_offset = -7.00)) --------------------------------------PhysicalOlapScan[store] ------------------------------PhysicalProject ---------------------------------filter((((promotion.p_channel_dmail = 'Y') OR (promotion.p_channel_email = 'Y')) OR (promotion.p_channel_tv = 'Y'))) +--------------------------------filter(OR[(promotion.p_channel_dmail = 'Y'),(promotion.p_channel_email = 'Y'),(promotion.p_channel_tv = 'Y')]) ----------------------------------PhysicalOlapScan[promotion] --------------------------PhysicalProject ----------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 1999)) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query63.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query63.out index 3bd844ce72c196..9653f6c52199aa 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query63.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query63.out @@ -22,7 +22,7 @@ PhysicalResultSink --------------------------------------PhysicalProject ----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 --------------------------------------PhysicalProject -----------------------------------------filter((((i_category IN ('Books', 'Children', 'Electronics') AND i_class IN ('personal', 'portable', 'reference', 'self-help')) AND i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')) OR ((i_category IN ('Men', 'Music', 'Women') AND i_class IN ('accessories', 'classical', 'fragrances', 'pants')) AND i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1'))) and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) +----------------------------------------filter(OR[AND[i_category IN ('Books', 'Children', 'Electronics'),i_class IN ('personal', 'portable', 'reference', 'self-help'),i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')],AND[i_category IN ('Men', 'Music', 'Women'),i_class IN ('accessories', 'classical', 'fragrances', 'pants'),i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')]] and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) ------------------------------------------PhysicalOlapScan[item] ----------------------------------PhysicalProject ------------------------------------filter(d_month_seq IN (1181, 1182, 1183, 1184, 1185, 1186, 1187, 1188, 1189, 1190, 1191, 1192)) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query68.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query68.out index 046c5866f5196c..d645b44a1efaa7 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query68.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query68.out @@ -27,7 +27,7 @@ PhysicalResultSink ----------------------------------filter(s_city IN ('Five Points', 'Pleasant Hill')) ------------------------------------PhysicalOlapScan[store] ----------------------------PhysicalProject -------------------------------filter(((household_demographics.hd_dep_count = 8) OR (household_demographics.hd_vehicle_count = -1))) +------------------------------filter(OR[(household_demographics.hd_dep_count = 8),(household_demographics.hd_vehicle_count = -1)]) --------------------------------PhysicalOlapScan[household_demographics] ------------------------PhysicalProject --------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query7.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query7.out index ff218ca1256a94..f47da720468166 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query7.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query7.out @@ -26,6 +26,6 @@ PhysicalResultSink ------------------------filter((date_dim.d_year = 2001)) --------------------------PhysicalOlapScan[date_dim] ------------------PhysicalProject ---------------------filter(((promotion.p_channel_email = 'N') OR (promotion.p_channel_event = 'N'))) +--------------------filter(OR[(promotion.p_channel_email = 'N'),(promotion.p_channel_event = 'N')]) ----------------------PhysicalOlapScan[promotion] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query78.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query78.out index 30527883a49018..a6034ca86ac5c5 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query78.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query78.out @@ -5,7 +5,7 @@ PhysicalResultSink ----PhysicalDistribute[DistributionSpecGather] ------PhysicalTopN[LOCAL_SORT] --------PhysicalProject -----------filter(((coalesce(ws_qty, 0) > 0) OR (coalesce(cs_qty, 0) > 0))) +----------filter(OR[(coalesce(ws_qty, 0) > 0),(coalesce(cs_qty, 0) > 0)]) ------------hashJoin[LEFT_OUTER_JOIN colocated] hashCondition=((cs.cs_customer_sk = ss.ss_customer_sk) and (cs.cs_item_sk = ss.ss_item_sk) and (cs.cs_sold_year = ss.ss_sold_year)) otherCondition=() --------------PhysicalProject ----------------hashJoin[LEFT_OUTER_JOIN colocated] hashCondition=((ws.ws_customer_sk = ss.ss_customer_sk) and (ws.ws_item_sk = ss.ss_item_sk) and (ws.ws_sold_year = ss.ss_sold_year)) otherCondition=() diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query79.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query79.out index 77168450e857f5..8f9b721f08ee59 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query79.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query79.out @@ -25,7 +25,7 @@ PhysicalResultSink ------------------------------filter((store.s_number_employees <= 295) and (store.s_number_employees >= 200)) --------------------------------PhysicalOlapScan[store] ------------------------PhysicalProject ---------------------------filter(((household_demographics.hd_dep_count = 5) OR (household_demographics.hd_vehicle_count > 4))) +--------------------------filter(OR[(household_demographics.hd_dep_count = 5),(household_demographics.hd_vehicle_count > 4)]) ----------------------------PhysicalOlapScan[household_demographics] ------------PhysicalProject --------------PhysicalOlapScan[customer] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query85.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query85.out index 590589c61a42a0..9f12763cec5114 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query85.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query85.out @@ -13,11 +13,11 @@ PhysicalResultSink --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF8 d_date_sk->[ws_sold_date_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = web_returns.wr_refunded_addr_sk)) otherCondition=((((ca_state IN ('DE', 'FL', 'TX') AND ((web_sales.ws_net_profit >= 100.00) AND (web_sales.ws_net_profit <= 200.00))) OR (ca_state IN ('ID', 'IN', 'ND') AND ((web_sales.ws_net_profit >= 150.00) AND (web_sales.ws_net_profit <= 300.00)))) OR (ca_state IN ('IL', 'MT', 'OH') AND ((web_sales.ws_net_profit >= 50.00) AND (web_sales.ws_net_profit <= 250.00))))) build RFs:RF7 ca_address_sk->[wr_refunded_addr_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = web_returns.wr_refunded_addr_sk)) otherCondition=(OR[AND[ca_state IN ('DE', 'FL', 'TX'),(web_sales.ws_net_profit >= 100.00),(web_sales.ws_net_profit <= 200.00)],AND[ca_state IN ('ID', 'IN', 'ND'),(web_sales.ws_net_profit >= 150.00),(web_sales.ws_net_profit <= 300.00)],AND[ca_state IN ('IL', 'MT', 'OH'),(web_sales.ws_net_profit >= 50.00),(web_sales.ws_net_profit <= 250.00)]]) build RFs:RF7 ca_address_sk->[wr_refunded_addr_sk] ----------------------------PhysicalProject ------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cd1.cd_education_status = cd2.cd_education_status) and (cd1.cd_marital_status = cd2.cd_marital_status) and (cd2.cd_demo_sk = web_returns.wr_returning_cdemo_sk)) otherCondition=() build RFs:RF4 cd_demo_sk->[wr_returning_cdemo_sk];RF5 cd_marital_status->[cd_marital_status];RF6 cd_education_status->[cd_education_status] --------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cd1.cd_demo_sk = web_returns.wr_refunded_cdemo_sk)) otherCondition=((((((cd1.cd_marital_status = 'M') AND (cd1.cd_education_status = '4 yr Degree')) AND ((web_sales.ws_sales_price >= 100.00) AND (web_sales.ws_sales_price <= 150.00))) OR (((cd1.cd_marital_status = 'S') AND (cd1.cd_education_status = 'Secondary')) AND ((web_sales.ws_sales_price >= 50.00) AND (web_sales.ws_sales_price <= 100.00)))) OR (((cd1.cd_marital_status = 'W') AND (cd1.cd_education_status = 'Advanced Degree')) AND ((web_sales.ws_sales_price >= 150.00) AND (web_sales.ws_sales_price <= 200.00))))) build RFs:RF3 cd_demo_sk->[wr_refunded_cdemo_sk] +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cd1.cd_demo_sk = web_returns.wr_refunded_cdemo_sk)) otherCondition=(OR[AND[(cd1.cd_marital_status = 'M'),(cd1.cd_education_status = '4 yr Degree'),(web_sales.ws_sales_price >= 100.00),(web_sales.ws_sales_price <= 150.00)],AND[(cd1.cd_marital_status = 'S'),(cd1.cd_education_status = 'Secondary'),(web_sales.ws_sales_price >= 50.00),(web_sales.ws_sales_price <= 100.00)],AND[(cd1.cd_marital_status = 'W'),(cd1.cd_education_status = 'Advanced Degree'),(web_sales.ws_sales_price >= 150.00),(web_sales.ws_sales_price <= 200.00)]]) build RFs:RF3 cd_demo_sk->[wr_refunded_cdemo_sk] ------------------------------------PhysicalProject --------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_web_page_sk = web_page.wp_web_page_sk)) otherCondition=() ----------------------------------------PhysicalProject @@ -30,7 +30,7 @@ PhysicalResultSink ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[web_page] ------------------------------------PhysicalProject ---------------------------------------filter(((((cd1.cd_marital_status = 'M') AND (cd1.cd_education_status = '4 yr Degree')) OR ((cd1.cd_marital_status = 'S') AND (cd1.cd_education_status = 'Secondary'))) OR ((cd1.cd_marital_status = 'W') AND (cd1.cd_education_status = 'Advanced Degree'))) and cd_education_status IN ('4 yr Degree', 'Advanced Degree', 'Secondary') and cd_marital_status IN ('M', 'S', 'W')) +--------------------------------------filter(OR[AND[(cd1.cd_marital_status = 'M'),(cd1.cd_education_status = '4 yr Degree')],AND[(cd1.cd_marital_status = 'S'),(cd1.cd_education_status = 'Secondary')],AND[(cd1.cd_marital_status = 'W'),(cd1.cd_education_status = 'Advanced Degree')]] and cd_education_status IN ('4 yr Degree', 'Advanced Degree', 'Secondary') and cd_marital_status IN ('M', 'S', 'W')) ----------------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF5 RF6 --------------------------------PhysicalProject ----------------------------------filter(cd_education_status IN ('4 yr Degree', 'Advanced Degree', 'Secondary') and cd_marital_status IN ('M', 'S', 'W')) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query88.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query88.out index dc879d6daa5a4b..a20acf339431ea 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query88.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query88.out @@ -20,7 +20,7 @@ PhysicalResultSink ----------------------------------PhysicalProject ------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF21 RF22 RF23 ----------------------------------PhysicalProject -------------------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +------------------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) --------------------------------------PhysicalOlapScan[household_demographics] ------------------------------PhysicalProject --------------------------------filter((time_dim.t_hour = 8) and (time_dim.t_minute >= 30)) @@ -40,7 +40,7 @@ PhysicalResultSink ----------------------------------PhysicalProject ------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF18 RF19 RF20 ----------------------------------PhysicalProject -------------------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +------------------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) --------------------------------------PhysicalOlapScan[household_demographics] ------------------------------PhysicalProject --------------------------------filter((time_dim.t_hour = 9) and (time_dim.t_minute < 30)) @@ -60,7 +60,7 @@ PhysicalResultSink --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF15 RF16 RF17 --------------------------------PhysicalProject -----------------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +----------------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) ------------------------------------PhysicalOlapScan[household_demographics] ----------------------------PhysicalProject ------------------------------filter((time_dim.t_hour = 9) and (time_dim.t_minute >= 30)) @@ -80,7 +80,7 @@ PhysicalResultSink ------------------------------PhysicalProject --------------------------------PhysicalOlapScan[store_sales] apply RFs: RF12 RF13 RF14 ------------------------------PhysicalProject ---------------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +--------------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) ----------------------------------PhysicalOlapScan[household_demographics] --------------------------PhysicalProject ----------------------------filter((time_dim.t_hour = 10) and (time_dim.t_minute < 30)) @@ -100,7 +100,7 @@ PhysicalResultSink ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[store_sales] apply RFs: RF9 RF10 RF11 ----------------------------PhysicalProject -------------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +------------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) --------------------------------PhysicalOlapScan[household_demographics] ------------------------PhysicalProject --------------------------filter((time_dim.t_hour = 10) and (time_dim.t_minute >= 30)) @@ -120,7 +120,7 @@ PhysicalResultSink --------------------------PhysicalProject ----------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 RF7 RF8 --------------------------PhysicalProject -----------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +----------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) ------------------------------PhysicalOlapScan[household_demographics] ----------------------PhysicalProject ------------------------filter((time_dim.t_hour = 11) and (time_dim.t_minute < 30)) @@ -140,7 +140,7 @@ PhysicalResultSink ------------------------PhysicalProject --------------------------PhysicalOlapScan[store_sales] apply RFs: RF3 RF4 RF5 ------------------------PhysicalProject ---------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +--------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) ----------------------------PhysicalOlapScan[household_demographics] --------------------PhysicalProject ----------------------filter((time_dim.t_hour = 11) and (time_dim.t_minute >= 30)) @@ -160,7 +160,7 @@ PhysicalResultSink ----------------------PhysicalProject ------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 ----------------------PhysicalProject -------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) --------------------------PhysicalOlapScan[household_demographics] ------------------PhysicalProject --------------------filter((time_dim.t_hour = 12) and (time_dim.t_minute < 30)) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query89.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query89.out index c709c4df5fe097..2e9294bce91fd2 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query89.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query89.out @@ -23,7 +23,7 @@ PhysicalResultSink ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 ----------------------------------------PhysicalProject -------------------------------------------filter(((i_category IN ('Electronics', 'Jewelry', 'Shoes') AND i_class IN ('athletic', 'portable', 'semi-precious')) OR (i_category IN ('Men', 'Music', 'Women') AND i_class IN ('accessories', 'maternity', 'rock'))) and i_category IN ('Electronics', 'Jewelry', 'Men', 'Music', 'Shoes', 'Women') and i_class IN ('accessories', 'athletic', 'maternity', 'portable', 'rock', 'semi-precious')) +------------------------------------------filter(OR[AND[i_category IN ('Electronics', 'Jewelry', 'Shoes'),i_class IN ('athletic', 'portable', 'semi-precious')],AND[i_category IN ('Men', 'Music', 'Women'),i_class IN ('accessories', 'maternity', 'rock')]] and i_category IN ('Electronics', 'Jewelry', 'Men', 'Music', 'Shoes', 'Women') and i_class IN ('accessories', 'athletic', 'maternity', 'portable', 'rock', 'semi-precious')) --------------------------------------------PhysicalOlapScan[item] ------------------------------------PhysicalProject --------------------------------------filter((date_dim.d_year = 1999)) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query91.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query91.out index aa211a9f494891..6a4b369a5fe1ed 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query91.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query91.out @@ -33,7 +33,7 @@ PhysicalResultSink ------------------------------filter((customer_address.ca_gmt_offset = -6.00)) --------------------------------PhysicalOlapScan[customer_address] ------------------------PhysicalProject ---------------------------filter((((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = 'Unknown')) OR ((customer_demographics.cd_marital_status = 'W') AND (customer_demographics.cd_education_status = 'Advanced Degree'))) and cd_education_status IN ('Advanced Degree', 'Unknown') and cd_marital_status IN ('M', 'W')) +--------------------------filter(OR[AND[(customer_demographics.cd_marital_status = 'M'),(customer_demographics.cd_education_status = 'Unknown')],AND[(customer_demographics.cd_marital_status = 'W'),(customer_demographics.cd_education_status = 'Advanced Degree')]] and cd_education_status IN ('Advanced Degree', 'Unknown') and cd_marital_status IN ('M', 'W')) ----------------------------PhysicalOlapScan[customer_demographics] --------------------PhysicalProject ----------------------filter((hd_buy_potential like '1001-5000%')) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query10.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query10.out index 36a2c69f6b9516..4fdff8b37961c5 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query10.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query10.out @@ -9,7 +9,7 @@ PhysicalResultSink ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) +------------------filter(OR[ifnull($c$1, FALSE),ifnull($c$2, FALSE)]) --------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() ----------------------PhysicalProject ------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF5 ca_address_sk->[c_current_addr_sk] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query13.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query13.out index 7f109c5dece1a7..a16b428cbc5159 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query13.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query13.out @@ -7,9 +7,9 @@ PhysicalResultSink --------PhysicalProject ----------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] ------------PhysicalProject ---------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=((((ca_state IN ('KS', 'MI', 'SD') AND ((store_sales.ss_net_profit >= 100.00) AND (store_sales.ss_net_profit <= 200.00))) OR (ca_state IN ('CO', 'MO', 'ND') AND ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 300.00)))) OR (ca_state IN ('NH', 'OH', 'TX') AND ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 250.00))))) build RFs:RF3 ca_address_sk->[ss_addr_sk] +--------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=(OR[AND[ca_state IN ('KS', 'MI', 'SD'),(store_sales.ss_net_profit >= 100.00),(store_sales.ss_net_profit <= 200.00)],AND[ca_state IN ('CO', 'MO', 'ND'),(store_sales.ss_net_profit >= 150.00),(store_sales.ss_net_profit <= 300.00)],AND[ca_state IN ('NH', 'OH', 'TX'),(store_sales.ss_net_profit >= 50.00),(store_sales.ss_net_profit <= 250.00)]]) build RFs:RF3 ca_address_sk->[ss_addr_sk] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=((((((household_demographics.hd_dep_count = 1) AND cd_marital_status IN ('M', 'S')) AND cd_education_status IN ('4 yr Degree', 'College')) AND ((((customer_demographics.cd_marital_status = 'S') AND (customer_demographics.cd_education_status = 'College')) AND ((store_sales.ss_sales_price >= 50.00) AND (store_sales.ss_sales_price <= 100.00))) OR (((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = '4 yr Degree')) AND ((store_sales.ss_sales_price >= 150.00) AND (store_sales.ss_sales_price <= 200.00))))) OR ((((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = 'Unknown')) AND ((store_sales.ss_sales_price >= 100.00) AND (store_sales.ss_sales_price <= 150.00))) AND (household_demographics.hd_dep_count = 3)))) build RFs:RF2 hd_demo_sk->[ss_hdemo_sk] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=(OR[AND[(household_demographics.hd_dep_count = 1),cd_marital_status IN ('M', 'S'),cd_education_status IN ('4 yr Degree', 'College'),OR[AND[(customer_demographics.cd_marital_status = 'S'),(customer_demographics.cd_education_status = 'College'),(store_sales.ss_sales_price >= 50.00),(store_sales.ss_sales_price <= 100.00)],AND[(customer_demographics.cd_marital_status = 'M'),(customer_demographics.cd_education_status = '4 yr Degree'),(store_sales.ss_sales_price >= 150.00),(store_sales.ss_sales_price <= 200.00)]]],AND[(customer_demographics.cd_marital_status = 'D'),(customer_demographics.cd_education_status = 'Unknown'),(store_sales.ss_sales_price >= 100.00),(store_sales.ss_sales_price <= 150.00),(household_demographics.hd_dep_count = 3)]]) build RFs:RF2 hd_demo_sk->[ss_hdemo_sk] --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)) otherCondition=() build RFs:RF1 cd_demo_sk->[ss_cdemo_sk] ------------------------PhysicalProject @@ -20,7 +20,7 @@ PhysicalResultSink ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[store] ------------------------PhysicalProject ---------------------------filter(((((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = 'Unknown')) OR ((customer_demographics.cd_marital_status = 'S') AND (customer_demographics.cd_education_status = 'College'))) OR ((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = '4 yr Degree'))) and cd_education_status IN ('4 yr Degree', 'College', 'Unknown') and cd_marital_status IN ('D', 'M', 'S')) +--------------------------filter(OR[AND[(customer_demographics.cd_marital_status = 'D'),(customer_demographics.cd_education_status = 'Unknown')],AND[(customer_demographics.cd_marital_status = 'S'),(customer_demographics.cd_education_status = 'College')],AND[(customer_demographics.cd_marital_status = 'M'),(customer_demographics.cd_education_status = '4 yr Degree')]] and cd_education_status IN ('4 yr Degree', 'College', 'Unknown') and cd_marital_status IN ('D', 'M', 'S')) ----------------------------PhysicalOlapScan[customer_demographics] --------------------PhysicalProject ----------------------filter(hd_dep_count IN (1, 3)) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query15.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query15.out index 1fba646d62b40a..81b0bae51498c1 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query15.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query15.out @@ -10,7 +10,7 @@ PhysicalResultSink --------------PhysicalProject ----------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=(((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR ca_state IN ('CA', 'GA', 'WA')) OR (catalog_sales.cs_sales_price > 500.00))) build RFs:RF1 ca_address_sk->[c_current_addr_sk] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),ca_state IN ('CA', 'GA', 'WA'),(catalog_sales.cs_sales_price > 500.00)]) build RFs:RF1 ca_address_sk->[c_current_addr_sk] ----------------------PhysicalProject ------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF0 c_customer_sk->[cs_bill_customer_sk] --------------------------PhysicalProject diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query26.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query26.out index 4561d649bd82c2..edbed407b77921 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query26.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query26.out @@ -26,6 +26,6 @@ PhysicalResultSink ------------------------filter((date_dim.d_year = 2001)) --------------------------PhysicalOlapScan[date_dim] ------------------PhysicalProject ---------------------filter(((promotion.p_channel_email = 'N') OR (promotion.p_channel_event = 'N'))) +--------------------filter(OR[(promotion.p_channel_email = 'N'),(promotion.p_channel_event = 'N')]) ----------------------PhysicalOlapScan[promotion] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query28.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query28.out index dae7cb1c23a402..7a6bdd8868ef00 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query28.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query28.out @@ -17,41 +17,41 @@ PhysicalResultSink ----------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------hashAgg[LOCAL] --------------------------------PhysicalProject -----------------------------------filter(((((store_sales.ss_list_price >= 131.00) AND (store_sales.ss_list_price <= 141.00)) OR ((store_sales.ss_coupon_amt >= 16798.00) AND (store_sales.ss_coupon_amt <= 17798.00))) OR ((store_sales.ss_wholesale_cost >= 25.00) AND (store_sales.ss_wholesale_cost <= 45.00))) and (store_sales.ss_quantity <= 5) and (store_sales.ss_quantity >= 0)) +----------------------------------filter((store_sales.ss_quantity <= 5) and (store_sales.ss_quantity >= 0) and OR[AND[(store_sales.ss_list_price >= 131.00),(store_sales.ss_list_price <= 141.00)],AND[(store_sales.ss_coupon_amt >= 16798.00),(store_sales.ss_coupon_amt <= 17798.00)],AND[(store_sales.ss_wholesale_cost >= 25.00),(store_sales.ss_wholesale_cost <= 45.00)]]) ------------------------------------PhysicalOlapScan[store_sales] ------------------------PhysicalLimit[LOCAL] --------------------------hashAgg[GLOBAL] ----------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------hashAgg[LOCAL] --------------------------------PhysicalProject -----------------------------------filter(((((store_sales.ss_list_price >= 145.00) AND (store_sales.ss_list_price <= 155.00)) OR ((store_sales.ss_coupon_amt >= 14792.00) AND (store_sales.ss_coupon_amt <= 15792.00))) OR ((store_sales.ss_wholesale_cost >= 46.00) AND (store_sales.ss_wholesale_cost <= 66.00))) and (store_sales.ss_quantity <= 10) and (store_sales.ss_quantity >= 6)) +----------------------------------filter((store_sales.ss_quantity <= 10) and (store_sales.ss_quantity >= 6) and OR[AND[(store_sales.ss_list_price >= 145.00),(store_sales.ss_list_price <= 155.00)],AND[(store_sales.ss_coupon_amt >= 14792.00),(store_sales.ss_coupon_amt <= 15792.00)],AND[(store_sales.ss_wholesale_cost >= 46.00),(store_sales.ss_wholesale_cost <= 66.00)]]) ------------------------------------PhysicalOlapScan[store_sales] --------------------PhysicalLimit[LOCAL] ----------------------hashAgg[GLOBAL] ------------------------PhysicalDistribute[DistributionSpecGather] --------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------filter(((((store_sales.ss_list_price >= 150.00) AND (store_sales.ss_list_price <= 160.00)) OR ((store_sales.ss_coupon_amt >= 6600.00) AND (store_sales.ss_coupon_amt <= 7600.00))) OR ((store_sales.ss_wholesale_cost >= 9.00) AND (store_sales.ss_wholesale_cost <= 29.00))) and (store_sales.ss_quantity <= 15) and (store_sales.ss_quantity >= 11)) +------------------------------filter((store_sales.ss_quantity <= 15) and (store_sales.ss_quantity >= 11) and OR[AND[(store_sales.ss_list_price >= 150.00),(store_sales.ss_list_price <= 160.00)],AND[(store_sales.ss_coupon_amt >= 6600.00),(store_sales.ss_coupon_amt <= 7600.00)],AND[(store_sales.ss_wholesale_cost >= 9.00),(store_sales.ss_wholesale_cost <= 29.00)]]) --------------------------------PhysicalOlapScan[store_sales] ----------------PhysicalLimit[LOCAL] ------------------hashAgg[GLOBAL] --------------------PhysicalDistribute[DistributionSpecGather] ----------------------hashAgg[LOCAL] ------------------------PhysicalProject ---------------------------filter(((((store_sales.ss_list_price >= 91.00) AND (store_sales.ss_list_price <= 101.00)) OR ((store_sales.ss_coupon_amt >= 13493.00) AND (store_sales.ss_coupon_amt <= 14493.00))) OR ((store_sales.ss_wholesale_cost >= 36.00) AND (store_sales.ss_wholesale_cost <= 56.00))) and (store_sales.ss_quantity <= 20) and (store_sales.ss_quantity >= 16)) +--------------------------filter((store_sales.ss_quantity <= 20) and (store_sales.ss_quantity >= 16) and OR[AND[(store_sales.ss_list_price >= 91.00),(store_sales.ss_list_price <= 101.00)],AND[(store_sales.ss_coupon_amt >= 13493.00),(store_sales.ss_coupon_amt <= 14493.00)],AND[(store_sales.ss_wholesale_cost >= 36.00),(store_sales.ss_wholesale_cost <= 56.00)]]) ----------------------------PhysicalOlapScan[store_sales] ------------PhysicalLimit[LOCAL] --------------hashAgg[GLOBAL] ----------------PhysicalDistribute[DistributionSpecGather] ------------------hashAgg[LOCAL] --------------------PhysicalProject -----------------------filter(((((store_sales.ss_list_price >= 0.00) AND (store_sales.ss_list_price <= 10.00)) OR ((store_sales.ss_coupon_amt >= 7629.00) AND (store_sales.ss_coupon_amt <= 8629.00))) OR ((store_sales.ss_wholesale_cost >= 6.00) AND (store_sales.ss_wholesale_cost <= 26.00))) and (store_sales.ss_quantity <= 25) and (store_sales.ss_quantity >= 21)) +----------------------filter((store_sales.ss_quantity <= 25) and (store_sales.ss_quantity >= 21) and OR[AND[(store_sales.ss_list_price >= 0.00),(store_sales.ss_list_price <= 10.00)],AND[(store_sales.ss_coupon_amt >= 7629.00),(store_sales.ss_coupon_amt <= 8629.00)],AND[(store_sales.ss_wholesale_cost >= 6.00),(store_sales.ss_wholesale_cost <= 26.00)]]) ------------------------PhysicalOlapScan[store_sales] --------PhysicalLimit[LOCAL] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecGather] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------filter(((((store_sales.ss_list_price >= 89.00) AND (store_sales.ss_list_price <= 99.00)) OR ((store_sales.ss_coupon_amt >= 15257.00) AND (store_sales.ss_coupon_amt <= 16257.00))) OR ((store_sales.ss_wholesale_cost >= 31.00) AND (store_sales.ss_wholesale_cost <= 51.00))) and (store_sales.ss_quantity <= 30) and (store_sales.ss_quantity >= 26)) +------------------filter((store_sales.ss_quantity <= 30) and (store_sales.ss_quantity >= 26) and OR[AND[(store_sales.ss_list_price >= 89.00),(store_sales.ss_list_price <= 99.00)],AND[(store_sales.ss_coupon_amt >= 15257.00),(store_sales.ss_coupon_amt <= 16257.00)],AND[(store_sales.ss_wholesale_cost >= 31.00),(store_sales.ss_wholesale_cost <= 51.00)]]) --------------------PhysicalOlapScan[store_sales] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query34.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query34.out index 25429183c8fe4a..9a53073479fb8c 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query34.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query34.out @@ -19,7 +19,7 @@ PhysicalResultSink --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 --------------------------------PhysicalProject -----------------------------------filter((((date_dim.d_dom >= 1) AND (date_dim.d_dom <= 3)) OR ((date_dim.d_dom >= 25) AND (date_dim.d_dom <= 28))) and d_year IN (1998, 1999, 2000)) +----------------------------------filter(OR[AND[(date_dim.d_dom >= 1),(date_dim.d_dom <= 3)],AND[(date_dim.d_dom >= 25),(date_dim.d_dom <= 28)]] and d_year IN (1998, 1999, 2000)) ------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalProject ------------------------------filter(s_county IN ('Barrow County', 'Daviess County', 'Franklin Parish', 'Luce County', 'Richland County', 'Walker County', 'Williamson County', 'Ziebach County')) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query35.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query35.out index 14b71f576d97e0..83807f4b912bfe 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query35.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query35.out @@ -9,7 +9,7 @@ PhysicalResultSink ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) +------------------filter(OR[ifnull($c$1, FALSE),ifnull($c$2, FALSE)]) --------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() ----------------------PhysicalProject ------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF5 cd_demo_sk->[c_current_cdemo_sk] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query41.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query41.out index c9c1ffff499741..3034a77fe0897a 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query41.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query41.out @@ -18,6 +18,6 @@ PhysicalResultSink ------------------------PhysicalDistribute[DistributionSpecHash] --------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------filter((((i_color IN ('aquamarine', 'blue', 'chartreuse', 'chiffon', 'dodger', 'gainsboro', 'tan', 'violet') AND i_units IN ('Bunch', 'Dozen', 'Each', 'Ounce', 'Oz', 'Pound', 'Ton', 'Tsp')) AND (((((((item.i_category = 'Women') AND i_color IN ('aquamarine', 'gainsboro')) AND i_units IN ('Dozen', 'Ounce')) AND i_size IN ('economy', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('chiffon', 'violet')) AND i_units IN ('Pound', 'Ton')) AND i_size IN ('extra large', 'small'))) OR ((((item.i_category = 'Men') AND i_color IN ('blue', 'chartreuse')) AND i_units IN ('Each', 'Oz')) AND i_size IN ('N/A', 'large'))) OR ((((item.i_category = 'Men') AND i_color IN ('dodger', 'tan')) AND i_units IN ('Bunch', 'Tsp')) AND i_size IN ('economy', 'medium')))) OR ((i_color IN ('almond', 'blanched', 'indian', 'lime', 'peru', 'saddle', 'spring', 'tomato') AND i_units IN ('Box', 'Carton', 'Case', 'Dram', 'Gram', 'Pallet', 'Tbl', 'Unknown')) AND (((((((item.i_category = 'Women') AND i_color IN ('blanched', 'tomato')) AND i_units IN ('Case', 'Tbl')) AND i_size IN ('economy', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('almond', 'lime')) AND i_units IN ('Box', 'Dram')) AND i_size IN ('extra large', 'small'))) OR ((((item.i_category = 'Men') AND i_color IN ('peru', 'saddle')) AND i_units IN ('Gram', 'Pallet')) AND i_size IN ('N/A', 'large'))) OR ((((item.i_category = 'Men') AND i_color IN ('indian', 'spring')) AND i_units IN ('Carton', 'Unknown')) AND i_size IN ('economy', 'medium'))))) and i_category IN ('Men', 'Women') and i_size IN ('N/A', 'economy', 'extra large', 'large', 'medium', 'small')) +------------------------------filter(OR[AND[i_color IN ('aquamarine', 'blue', 'chartreuse', 'chiffon', 'dodger', 'gainsboro', 'tan', 'violet'),i_units IN ('Bunch', 'Dozen', 'Each', 'Ounce', 'Oz', 'Pound', 'Ton', 'Tsp'),OR[AND[(item.i_category = 'Women'),i_color IN ('aquamarine', 'gainsboro'),i_units IN ('Dozen', 'Ounce'),i_size IN ('economy', 'medium')],AND[(item.i_category = 'Women'),i_color IN ('chiffon', 'violet'),i_units IN ('Pound', 'Ton'),i_size IN ('extra large', 'small')],AND[(item.i_category = 'Men'),i_color IN ('blue', 'chartreuse'),i_units IN ('Each', 'Oz'),i_size IN ('N/A', 'large')],AND[(item.i_category = 'Men'),i_color IN ('dodger', 'tan'),i_units IN ('Bunch', 'Tsp'),i_size IN ('economy', 'medium')]]],AND[i_color IN ('almond', 'blanched', 'indian', 'lime', 'peru', 'saddle', 'spring', 'tomato'),i_units IN ('Box', 'Carton', 'Case', 'Dram', 'Gram', 'Pallet', 'Tbl', 'Unknown'),OR[AND[(item.i_category = 'Women'),i_color IN ('blanched', 'tomato'),i_units IN ('Case', 'Tbl'),i_size IN ('economy', 'medium')],AND[(item.i_category = 'Women'),i_color IN ('almond', 'lime'),i_units IN ('Box', 'Dram'),i_size IN ('extra large', 'small')],AND[(item.i_category = 'Men'),i_color IN ('peru', 'saddle'),i_units IN ('Gram', 'Pallet'),i_size IN ('N/A', 'large')],AND[(item.i_category = 'Men'),i_color IN ('indian', 'spring'),i_units IN ('Carton', 'Unknown'),i_size IN ('economy', 'medium')]]]] and i_category IN ('Men', 'Women') and i_size IN ('N/A', 'economy', 'extra large', 'large', 'medium', 'small')) --------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query45.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query45.out index 038ece83d7c044..68d1ef7855a7fd 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query45.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query45.out @@ -8,7 +8,7 @@ PhysicalResultSink ----------PhysicalDistribute[DistributionSpecHash] ------------hashAgg[LOCAL] --------------PhysicalProject -----------------filter((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR $c$1)) +----------------filter(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),$c$1]) ------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query46.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query46.out index ae7ea5a5a310fe..4f754d410d4b6d 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query46.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query46.out @@ -27,7 +27,7 @@ PhysicalResultSink ----------------------------------filter(s_city IN ('Centerville', 'Fairview', 'Five Points', 'Liberty', 'Oak Grove')) ------------------------------------PhysicalOlapScan[store] ----------------------------PhysicalProject -------------------------------filter(((household_demographics.hd_dep_count = 6) OR (household_demographics.hd_vehicle_count = 0))) +------------------------------filter(OR[(household_demographics.hd_dep_count = 6),(household_demographics.hd_vehicle_count = 0)]) --------------------------------PhysicalOlapScan[household_demographics] ------------------------PhysicalProject --------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query47.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query47.out index b6f046eb01fe81..29aa8e6ae22e6c 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query47.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query47.out @@ -23,7 +23,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------------------------PhysicalProject --------------------------------------PhysicalOlapScan[item] --------------------------------PhysicalProject -----------------------------------filter((((date_dim.d_year = 2001) OR ((date_dim.d_year = 2000) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2002) AND (date_dim.d_moy = 1))) and d_year IN (2000, 2001, 2002)) +----------------------------------filter(OR[(date_dim.d_year = 2001),AND[(date_dim.d_year = 2000),(date_dim.d_moy = 12)],AND[(date_dim.d_year = 2002),(date_dim.d_moy = 1)]] and d_year IN (2000, 2001, 2002)) ------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query48.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query48.out index ed7ed3b4bdf669..fa9006ce24f673 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query48.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query48.out @@ -7,9 +7,9 @@ PhysicalResultSink --------PhysicalProject ----------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ss_sold_date_sk] ------------PhysicalProject ---------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=((((ca_state IN ('IA', 'MD', 'MN') AND ((store_sales.ss_net_profit >= 0.00) AND (store_sales.ss_net_profit <= 2000.00))) OR (ca_state IN ('IL', 'TX', 'VA') AND ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 3000.00)))) OR (ca_state IN ('IN', 'MI', 'WI') AND ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 25000.00))))) build RFs:RF2 ca_address_sk->[ss_addr_sk] +--------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=(OR[AND[ca_state IN ('IA', 'MD', 'MN'),(store_sales.ss_net_profit >= 0.00),(store_sales.ss_net_profit <= 2000.00)],AND[ca_state IN ('IL', 'TX', 'VA'),(store_sales.ss_net_profit >= 150.00),(store_sales.ss_net_profit <= 3000.00)],AND[ca_state IN ('IN', 'MI', 'WI'),(store_sales.ss_net_profit >= 50.00),(store_sales.ss_net_profit <= 25000.00)]]) build RFs:RF2 ca_address_sk->[ss_addr_sk] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)) otherCondition=((((((customer_demographics.cd_marital_status = 'U') AND (customer_demographics.cd_education_status = 'Primary')) AND ((store_sales.ss_sales_price >= 100.00) AND (store_sales.ss_sales_price <= 150.00))) OR (((customer_demographics.cd_marital_status = 'W') AND (customer_demographics.cd_education_status = 'College')) AND ((store_sales.ss_sales_price >= 50.00) AND (store_sales.ss_sales_price <= 100.00)))) OR (((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = '2 yr Degree')) AND ((store_sales.ss_sales_price >= 150.00) AND (store_sales.ss_sales_price <= 200.00))))) build RFs:RF1 cd_demo_sk->[ss_cdemo_sk] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)) otherCondition=(OR[AND[(customer_demographics.cd_marital_status = 'U'),(customer_demographics.cd_education_status = 'Primary'),(store_sales.ss_sales_price >= 100.00),(store_sales.ss_sales_price <= 150.00)],AND[(customer_demographics.cd_marital_status = 'W'),(customer_demographics.cd_education_status = 'College'),(store_sales.ss_sales_price >= 50.00),(store_sales.ss_sales_price <= 100.00)],AND[(customer_demographics.cd_marital_status = 'D'),(customer_demographics.cd_education_status = '2 yr Degree'),(store_sales.ss_sales_price >= 150.00),(store_sales.ss_sales_price <= 200.00)]]) build RFs:RF1 cd_demo_sk->[ss_cdemo_sk] --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF0 s_store_sk->[ss_store_sk] ------------------------PhysicalProject @@ -18,7 +18,7 @@ PhysicalResultSink ------------------------PhysicalProject --------------------------PhysicalOlapScan[store] --------------------PhysicalProject -----------------------filter(((((customer_demographics.cd_marital_status = 'U') AND (customer_demographics.cd_education_status = 'Primary')) OR ((customer_demographics.cd_marital_status = 'W') AND (customer_demographics.cd_education_status = 'College'))) OR ((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = '2 yr Degree'))) and cd_education_status IN ('2 yr Degree', 'College', 'Primary') and cd_marital_status IN ('D', 'U', 'W')) +----------------------filter(OR[AND[(customer_demographics.cd_marital_status = 'U'),(customer_demographics.cd_education_status = 'Primary')],AND[(customer_demographics.cd_marital_status = 'W'),(customer_demographics.cd_education_status = 'College')],AND[(customer_demographics.cd_marital_status = 'D'),(customer_demographics.cd_education_status = '2 yr Degree')]] and cd_education_status IN ('2 yr Degree', 'College', 'Primary') and cd_marital_status IN ('D', 'U', 'W')) ------------------------PhysicalOlapScan[customer_demographics] ----------------PhysicalProject ------------------filter((customer_address.ca_country = 'United States') and ca_state IN ('IA', 'IL', 'IN', 'MD', 'MI', 'MN', 'TX', 'VA', 'WI')) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query49.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query49.out index dc56d7f9e81411..0db3fa841189b2 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query49.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query49.out @@ -16,7 +16,7 @@ PhysicalResultSink --------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) ----------------------------------PhysicalWindow ------------------------------------PhysicalQuickSort[LOCAL_SORT] --------------------------------------PhysicalWindow @@ -48,7 +48,7 @@ PhysicalResultSink --------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) ----------------------------------PhysicalWindow ------------------------------------PhysicalQuickSort[LOCAL_SORT] --------------------------------------PhysicalWindow @@ -80,7 +80,7 @@ PhysicalResultSink --------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) ----------------------------------PhysicalWindow ------------------------------------PhysicalQuickSort[LOCAL_SORT] --------------------------------------PhysicalWindow diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query53.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query53.out index 81e6091b663bf7..04920e65ac6894 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query53.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query53.out @@ -22,7 +22,7 @@ PhysicalResultSink --------------------------------------PhysicalProject ----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 --------------------------------------PhysicalProject -----------------------------------------filter((((i_category IN ('Books', 'Children', 'Electronics') AND i_class IN ('personal', 'portable', 'reference', 'self-help')) AND i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')) OR ((i_category IN ('Men', 'Music', 'Women') AND i_class IN ('accessories', 'classical', 'fragrances', 'pants')) AND i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1'))) and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) +----------------------------------------filter(OR[AND[i_category IN ('Books', 'Children', 'Electronics'),i_class IN ('personal', 'portable', 'reference', 'self-help'),i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')],AND[i_category IN ('Men', 'Music', 'Women'),i_class IN ('accessories', 'classical', 'fragrances', 'pants'),i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')]] and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) ------------------------------------------PhysicalOlapScan[item] ----------------------------------PhysicalProject ------------------------------------filter(d_month_seq IN (1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211)) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query57.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query57.out index e640ec24d7b3e3..88777bc1ff548d 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query57.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query57.out @@ -23,7 +23,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------------------------PhysicalProject --------------------------------------PhysicalOlapScan[item] --------------------------------PhysicalProject -----------------------------------filter((((date_dim.d_year = 1999) OR ((date_dim.d_year = 1998) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2000) AND (date_dim.d_moy = 1))) and d_year IN (1998, 1999, 2000)) +----------------------------------filter(OR[(date_dim.d_year = 1999),AND[(date_dim.d_year = 1998),(date_dim.d_moy = 12)],AND[(date_dim.d_year = 2000),(date_dim.d_moy = 1)]] and d_year IN (1998, 1999, 2000)) ------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[call_center] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query61.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query61.out index f9dcb3c9460ca4..62da8c9cb21a0f 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query61.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query61.out @@ -27,7 +27,7 @@ PhysicalResultSink ------------------------------------filter((store.s_gmt_offset = -7.00)) --------------------------------------PhysicalOlapScan[store] ------------------------------PhysicalProject ---------------------------------filter((((promotion.p_channel_dmail = 'Y') OR (promotion.p_channel_email = 'Y')) OR (promotion.p_channel_tv = 'Y'))) +--------------------------------filter(OR[(promotion.p_channel_dmail = 'Y'),(promotion.p_channel_email = 'Y'),(promotion.p_channel_tv = 'Y')]) ----------------------------------PhysicalOlapScan[promotion] --------------------------PhysicalProject ----------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 1999)) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query63.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query63.out index 2cd49bd8c40786..d4fb4990da98b8 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query63.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query63.out @@ -22,7 +22,7 @@ PhysicalResultSink --------------------------------------PhysicalProject ----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 --------------------------------------PhysicalProject -----------------------------------------filter((((i_category IN ('Books', 'Children', 'Electronics') AND i_class IN ('personal', 'portable', 'reference', 'self-help')) AND i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')) OR ((i_category IN ('Men', 'Music', 'Women') AND i_class IN ('accessories', 'classical', 'fragrances', 'pants')) AND i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1'))) and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) +----------------------------------------filter(OR[AND[i_category IN ('Books', 'Children', 'Electronics'),i_class IN ('personal', 'portable', 'reference', 'self-help'),i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')],AND[i_category IN ('Men', 'Music', 'Women'),i_class IN ('accessories', 'classical', 'fragrances', 'pants'),i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')]] and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) ------------------------------------------PhysicalOlapScan[item] ----------------------------------PhysicalProject ------------------------------------filter(d_month_seq IN (1181, 1182, 1183, 1184, 1185, 1186, 1187, 1188, 1189, 1190, 1191, 1192)) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query68.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query68.out index 70945beb33b729..82e85b4fc4697a 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query68.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query68.out @@ -27,7 +27,7 @@ PhysicalResultSink ----------------------------------filter(s_city IN ('Five Points', 'Pleasant Hill')) ------------------------------------PhysicalOlapScan[store] ----------------------------PhysicalProject -------------------------------filter(((household_demographics.hd_dep_count = 8) OR (household_demographics.hd_vehicle_count = -1))) +------------------------------filter(OR[(household_demographics.hd_dep_count = 8),(household_demographics.hd_vehicle_count = -1)]) --------------------------------PhysicalOlapScan[household_demographics] ------------------------PhysicalProject --------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query7.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query7.out index 12c9bd7d3a0b25..2b6615e0b93b84 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query7.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query7.out @@ -26,6 +26,6 @@ PhysicalResultSink ------------------------filter((date_dim.d_year = 2001)) --------------------------PhysicalOlapScan[date_dim] ------------------PhysicalProject ---------------------filter(((promotion.p_channel_email = 'N') OR (promotion.p_channel_event = 'N'))) +--------------------filter(OR[(promotion.p_channel_email = 'N'),(promotion.p_channel_event = 'N')]) ----------------------PhysicalOlapScan[promotion] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query78.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query78.out index 30527883a49018..a6034ca86ac5c5 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query78.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query78.out @@ -5,7 +5,7 @@ PhysicalResultSink ----PhysicalDistribute[DistributionSpecGather] ------PhysicalTopN[LOCAL_SORT] --------PhysicalProject -----------filter(((coalesce(ws_qty, 0) > 0) OR (coalesce(cs_qty, 0) > 0))) +----------filter(OR[(coalesce(ws_qty, 0) > 0),(coalesce(cs_qty, 0) > 0)]) ------------hashJoin[LEFT_OUTER_JOIN colocated] hashCondition=((cs.cs_customer_sk = ss.ss_customer_sk) and (cs.cs_item_sk = ss.ss_item_sk) and (cs.cs_sold_year = ss.ss_sold_year)) otherCondition=() --------------PhysicalProject ----------------hashJoin[LEFT_OUTER_JOIN colocated] hashCondition=((ws.ws_customer_sk = ss.ss_customer_sk) and (ws.ws_item_sk = ss.ss_item_sk) and (ws.ws_sold_year = ss.ss_sold_year)) otherCondition=() diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query79.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query79.out index 5d935352c491c6..1a19308d991441 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query79.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query79.out @@ -25,7 +25,7 @@ PhysicalResultSink ------------------------------filter((store.s_number_employees <= 295) and (store.s_number_employees >= 200)) --------------------------------PhysicalOlapScan[store] ------------------------PhysicalProject ---------------------------filter(((household_demographics.hd_dep_count = 5) OR (household_demographics.hd_vehicle_count > 4))) +--------------------------filter(OR[(household_demographics.hd_dep_count = 5),(household_demographics.hd_vehicle_count > 4)]) ----------------------------PhysicalOlapScan[household_demographics] ------------PhysicalProject --------------PhysicalOlapScan[customer] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query85.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query85.out index 08e59c1e36507e..424102e34021a1 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query85.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query85.out @@ -13,11 +13,11 @@ PhysicalResultSink --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF8 d_date_sk->[ws_sold_date_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = web_returns.wr_refunded_addr_sk)) otherCondition=((((ca_state IN ('DE', 'FL', 'TX') AND ((web_sales.ws_net_profit >= 100.00) AND (web_sales.ws_net_profit <= 200.00))) OR (ca_state IN ('ID', 'IN', 'ND') AND ((web_sales.ws_net_profit >= 150.00) AND (web_sales.ws_net_profit <= 300.00)))) OR (ca_state IN ('IL', 'MT', 'OH') AND ((web_sales.ws_net_profit >= 50.00) AND (web_sales.ws_net_profit <= 250.00))))) build RFs:RF7 ca_address_sk->[wr_refunded_addr_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = web_returns.wr_refunded_addr_sk)) otherCondition=(OR[AND[ca_state IN ('DE', 'FL', 'TX'),(web_sales.ws_net_profit >= 100.00),(web_sales.ws_net_profit <= 200.00)],AND[ca_state IN ('ID', 'IN', 'ND'),(web_sales.ws_net_profit >= 150.00),(web_sales.ws_net_profit <= 300.00)],AND[ca_state IN ('IL', 'MT', 'OH'),(web_sales.ws_net_profit >= 50.00),(web_sales.ws_net_profit <= 250.00)]]) build RFs:RF7 ca_address_sk->[wr_refunded_addr_sk] ----------------------------PhysicalProject ------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cd1.cd_education_status = cd2.cd_education_status) and (cd1.cd_marital_status = cd2.cd_marital_status) and (cd2.cd_demo_sk = web_returns.wr_returning_cdemo_sk)) otherCondition=() build RFs:RF4 cd_demo_sk->[wr_returning_cdemo_sk];RF5 cd_marital_status->[cd_marital_status];RF6 cd_education_status->[cd_education_status] --------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cd1.cd_demo_sk = web_returns.wr_refunded_cdemo_sk)) otherCondition=((((((cd1.cd_marital_status = 'M') AND (cd1.cd_education_status = '4 yr Degree')) AND ((web_sales.ws_sales_price >= 100.00) AND (web_sales.ws_sales_price <= 150.00))) OR (((cd1.cd_marital_status = 'S') AND (cd1.cd_education_status = 'Secondary')) AND ((web_sales.ws_sales_price >= 50.00) AND (web_sales.ws_sales_price <= 100.00)))) OR (((cd1.cd_marital_status = 'W') AND (cd1.cd_education_status = 'Advanced Degree')) AND ((web_sales.ws_sales_price >= 150.00) AND (web_sales.ws_sales_price <= 200.00))))) build RFs:RF3 cd_demo_sk->[wr_refunded_cdemo_sk] +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cd1.cd_demo_sk = web_returns.wr_refunded_cdemo_sk)) otherCondition=(OR[AND[(cd1.cd_marital_status = 'M'),(cd1.cd_education_status = '4 yr Degree'),(web_sales.ws_sales_price >= 100.00),(web_sales.ws_sales_price <= 150.00)],AND[(cd1.cd_marital_status = 'S'),(cd1.cd_education_status = 'Secondary'),(web_sales.ws_sales_price >= 50.00),(web_sales.ws_sales_price <= 100.00)],AND[(cd1.cd_marital_status = 'W'),(cd1.cd_education_status = 'Advanced Degree'),(web_sales.ws_sales_price >= 150.00),(web_sales.ws_sales_price <= 200.00)]]) build RFs:RF3 cd_demo_sk->[wr_refunded_cdemo_sk] ------------------------------------PhysicalProject --------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_web_page_sk = web_page.wp_web_page_sk)) otherCondition=() build RFs:RF2 wp_web_page_sk->[ws_web_page_sk] ----------------------------------------PhysicalProject @@ -30,7 +30,7 @@ PhysicalResultSink ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[web_page] ------------------------------------PhysicalProject ---------------------------------------filter(((((cd1.cd_marital_status = 'M') AND (cd1.cd_education_status = '4 yr Degree')) OR ((cd1.cd_marital_status = 'S') AND (cd1.cd_education_status = 'Secondary'))) OR ((cd1.cd_marital_status = 'W') AND (cd1.cd_education_status = 'Advanced Degree'))) and cd_education_status IN ('4 yr Degree', 'Advanced Degree', 'Secondary') and cd_marital_status IN ('M', 'S', 'W')) +--------------------------------------filter(OR[AND[(cd1.cd_marital_status = 'M'),(cd1.cd_education_status = '4 yr Degree')],AND[(cd1.cd_marital_status = 'S'),(cd1.cd_education_status = 'Secondary')],AND[(cd1.cd_marital_status = 'W'),(cd1.cd_education_status = 'Advanced Degree')]] and cd_education_status IN ('4 yr Degree', 'Advanced Degree', 'Secondary') and cd_marital_status IN ('M', 'S', 'W')) ----------------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF5 RF6 --------------------------------PhysicalProject ----------------------------------filter(cd_education_status IN ('4 yr Degree', 'Advanced Degree', 'Secondary') and cd_marital_status IN ('M', 'S', 'W')) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query88.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query88.out index dc879d6daa5a4b..a20acf339431ea 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query88.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query88.out @@ -20,7 +20,7 @@ PhysicalResultSink ----------------------------------PhysicalProject ------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF21 RF22 RF23 ----------------------------------PhysicalProject -------------------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +------------------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) --------------------------------------PhysicalOlapScan[household_demographics] ------------------------------PhysicalProject --------------------------------filter((time_dim.t_hour = 8) and (time_dim.t_minute >= 30)) @@ -40,7 +40,7 @@ PhysicalResultSink ----------------------------------PhysicalProject ------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF18 RF19 RF20 ----------------------------------PhysicalProject -------------------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +------------------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) --------------------------------------PhysicalOlapScan[household_demographics] ------------------------------PhysicalProject --------------------------------filter((time_dim.t_hour = 9) and (time_dim.t_minute < 30)) @@ -60,7 +60,7 @@ PhysicalResultSink --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF15 RF16 RF17 --------------------------------PhysicalProject -----------------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +----------------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) ------------------------------------PhysicalOlapScan[household_demographics] ----------------------------PhysicalProject ------------------------------filter((time_dim.t_hour = 9) and (time_dim.t_minute >= 30)) @@ -80,7 +80,7 @@ PhysicalResultSink ------------------------------PhysicalProject --------------------------------PhysicalOlapScan[store_sales] apply RFs: RF12 RF13 RF14 ------------------------------PhysicalProject ---------------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +--------------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) ----------------------------------PhysicalOlapScan[household_demographics] --------------------------PhysicalProject ----------------------------filter((time_dim.t_hour = 10) and (time_dim.t_minute < 30)) @@ -100,7 +100,7 @@ PhysicalResultSink ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[store_sales] apply RFs: RF9 RF10 RF11 ----------------------------PhysicalProject -------------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +------------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) --------------------------------PhysicalOlapScan[household_demographics] ------------------------PhysicalProject --------------------------filter((time_dim.t_hour = 10) and (time_dim.t_minute >= 30)) @@ -120,7 +120,7 @@ PhysicalResultSink --------------------------PhysicalProject ----------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 RF7 RF8 --------------------------PhysicalProject -----------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +----------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) ------------------------------PhysicalOlapScan[household_demographics] ----------------------PhysicalProject ------------------------filter((time_dim.t_hour = 11) and (time_dim.t_minute < 30)) @@ -140,7 +140,7 @@ PhysicalResultSink ------------------------PhysicalProject --------------------------PhysicalOlapScan[store_sales] apply RFs: RF3 RF4 RF5 ------------------------PhysicalProject ---------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +--------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) ----------------------------PhysicalOlapScan[household_demographics] --------------------PhysicalProject ----------------------filter((time_dim.t_hour = 11) and (time_dim.t_minute >= 30)) @@ -160,7 +160,7 @@ PhysicalResultSink ----------------------PhysicalProject ------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 ----------------------PhysicalProject -------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) --------------------------PhysicalOlapScan[household_demographics] ------------------PhysicalProject --------------------filter((time_dim.t_hour = 12) and (time_dim.t_minute < 30)) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query89.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query89.out index 661b47d0ea980a..e4d2ae3435f174 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query89.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query89.out @@ -23,7 +23,7 @@ PhysicalResultSink ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 ----------------------------------------PhysicalProject -------------------------------------------filter(((i_category IN ('Electronics', 'Jewelry', 'Shoes') AND i_class IN ('athletic', 'portable', 'semi-precious')) OR (i_category IN ('Men', 'Music', 'Women') AND i_class IN ('accessories', 'maternity', 'rock'))) and i_category IN ('Electronics', 'Jewelry', 'Men', 'Music', 'Shoes', 'Women') and i_class IN ('accessories', 'athletic', 'maternity', 'portable', 'rock', 'semi-precious')) +------------------------------------------filter(OR[AND[i_category IN ('Electronics', 'Jewelry', 'Shoes'),i_class IN ('athletic', 'portable', 'semi-precious')],AND[i_category IN ('Men', 'Music', 'Women'),i_class IN ('accessories', 'maternity', 'rock')]] and i_category IN ('Electronics', 'Jewelry', 'Men', 'Music', 'Shoes', 'Women') and i_class IN ('accessories', 'athletic', 'maternity', 'portable', 'rock', 'semi-precious')) --------------------------------------------PhysicalOlapScan[item] ------------------------------------PhysicalProject --------------------------------------filter((date_dim.d_year = 1999)) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query91.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query91.out index c19b43be01a9df..a2e5d4a7660114 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query91.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query91.out @@ -33,7 +33,7 @@ PhysicalResultSink ------------------------------filter((customer_address.ca_gmt_offset = -6.00)) --------------------------------PhysicalOlapScan[customer_address] ------------------------PhysicalProject ---------------------------filter((((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = 'Unknown')) OR ((customer_demographics.cd_marital_status = 'W') AND (customer_demographics.cd_education_status = 'Advanced Degree'))) and cd_education_status IN ('Advanced Degree', 'Unknown') and cd_marital_status IN ('M', 'W')) +--------------------------filter(OR[AND[(customer_demographics.cd_marital_status = 'M'),(customer_demographics.cd_education_status = 'Unknown')],AND[(customer_demographics.cd_marital_status = 'W'),(customer_demographics.cd_education_status = 'Advanced Degree')]] and cd_education_status IN ('Advanced Degree', 'Unknown') and cd_marital_status IN ('M', 'W')) ----------------------------PhysicalOlapScan[customer_demographics] --------------------PhysicalProject ----------------------filter((hd_buy_potential like '1001-5000%')) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query10.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query10.out index 6027b8b2684b4b..4dfc2de4cf3fe5 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query10.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query10.out @@ -9,7 +9,7 @@ PhysicalResultSink ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) +------------------filter(OR[ifnull($c$1, FALSE),ifnull($c$2, FALSE)]) --------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() ----------------------PhysicalProject ------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query13.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query13.out index e92ddac6d97742..4b72ea24b2c114 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query13.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query13.out @@ -7,16 +7,16 @@ PhysicalResultSink --------PhysicalProject ----------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() ------------PhysicalProject ---------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)) otherCondition=((((((household_demographics.hd_dep_count = 1) AND cd_marital_status IN ('M', 'S')) AND cd_education_status IN ('4 yr Degree', 'College')) AND ((((customer_demographics.cd_marital_status = 'S') AND (customer_demographics.cd_education_status = 'College')) AND ((store_sales.ss_sales_price >= 50.00) AND (store_sales.ss_sales_price <= 100.00))) OR (((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = '4 yr Degree')) AND ((store_sales.ss_sales_price >= 150.00) AND (store_sales.ss_sales_price <= 200.00))))) OR ((((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = 'Unknown')) AND ((store_sales.ss_sales_price >= 100.00) AND (store_sales.ss_sales_price <= 150.00))) AND (household_demographics.hd_dep_count = 3)))) build RFs:RF3 ss_cdemo_sk->[cd_demo_sk] +--------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)) otherCondition=(OR[AND[(household_demographics.hd_dep_count = 1),cd_marital_status IN ('M', 'S'),cd_education_status IN ('4 yr Degree', 'College'),OR[AND[(customer_demographics.cd_marital_status = 'S'),(customer_demographics.cd_education_status = 'College'),(store_sales.ss_sales_price >= 50.00),(store_sales.ss_sales_price <= 100.00)],AND[(customer_demographics.cd_marital_status = 'M'),(customer_demographics.cd_education_status = '4 yr Degree'),(store_sales.ss_sales_price >= 150.00),(store_sales.ss_sales_price <= 200.00)]]],AND[(customer_demographics.cd_marital_status = 'D'),(customer_demographics.cd_education_status = 'Unknown'),(store_sales.ss_sales_price >= 100.00),(store_sales.ss_sales_price <= 150.00),(household_demographics.hd_dep_count = 3)]]) build RFs:RF3 ss_cdemo_sk->[cd_demo_sk] ----------------PhysicalProject -------------------filter(((((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = 'Unknown')) OR ((customer_demographics.cd_marital_status = 'S') AND (customer_demographics.cd_education_status = 'College'))) OR ((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = '4 yr Degree'))) and cd_education_status IN ('4 yr Degree', 'College', 'Unknown') and cd_marital_status IN ('D', 'M', 'S')) +------------------filter(OR[AND[(customer_demographics.cd_marital_status = 'D'),(customer_demographics.cd_education_status = 'Unknown')],AND[(customer_demographics.cd_marital_status = 'S'),(customer_demographics.cd_education_status = 'College')],AND[(customer_demographics.cd_marital_status = 'M'),(customer_demographics.cd_education_status = '4 yr Degree')]] and cd_education_status IN ('4 yr Degree', 'College', 'Unknown') and cd_marital_status IN ('D', 'M', 'S')) --------------------PhysicalOlapScan[customer_demographics] apply RFs: RF3 ----------------PhysicalProject ------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF2 hd_demo_sk->[ss_hdemo_sk] --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=((((ca_state IN ('KS', 'MI', 'SD') AND ((store_sales.ss_net_profit >= 100.00) AND (store_sales.ss_net_profit <= 200.00))) OR (ca_state IN ('CO', 'MO', 'ND') AND ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 300.00)))) OR (ca_state IN ('NH', 'OH', 'TX') AND ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 250.00))))) build RFs:RF0 ca_address_sk->[ss_addr_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=(OR[AND[ca_state IN ('KS', 'MI', 'SD'),(store_sales.ss_net_profit >= 100.00),(store_sales.ss_net_profit <= 200.00)],AND[ca_state IN ('CO', 'MO', 'ND'),(store_sales.ss_net_profit >= 150.00),(store_sales.ss_net_profit <= 300.00)],AND[ca_state IN ('NH', 'OH', 'TX'),(store_sales.ss_net_profit >= 50.00),(store_sales.ss_net_profit <= 250.00)]]) build RFs:RF0 ca_address_sk->[ss_addr_sk] ----------------------------PhysicalProject ------------------------------filter((store_sales.ss_net_profit <= 300.00) and (store_sales.ss_net_profit >= 50.00) and (store_sales.ss_sales_price <= 200.00) and (store_sales.ss_sales_price >= 50.00)) --------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query15.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query15.out index f15477f65e12fb..c070b7d34c00d4 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query15.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query15.out @@ -8,7 +8,7 @@ PhysicalResultSink ----------PhysicalDistribute[DistributionSpecHash] ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=(((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR ca_state IN ('CA', 'GA', 'WA')) OR (catalog_sales.cs_sales_price > 500.00))) +----------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),ca_state IN ('CA', 'GA', 'WA'),(catalog_sales.cs_sales_price > 500.00)]) ------------------PhysicalProject --------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] ----------------------PhysicalProject diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query26.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query26.out index c1cc05ec8ee03f..37ede3b355320f 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query26.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query26.out @@ -24,7 +24,7 @@ PhysicalResultSink ----------------------------filter((date_dim.d_year = 2001)) ------------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalProject -------------------------filter(((promotion.p_channel_email = 'N') OR (promotion.p_channel_event = 'N'))) +------------------------filter(OR[(promotion.p_channel_email = 'N'),(promotion.p_channel_event = 'N')]) --------------------------PhysicalOlapScan[promotion] ------------------PhysicalProject --------------------PhysicalOlapScan[item] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query28.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query28.out index dae7cb1c23a402..7a6bdd8868ef00 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query28.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query28.out @@ -17,41 +17,41 @@ PhysicalResultSink ----------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------hashAgg[LOCAL] --------------------------------PhysicalProject -----------------------------------filter(((((store_sales.ss_list_price >= 131.00) AND (store_sales.ss_list_price <= 141.00)) OR ((store_sales.ss_coupon_amt >= 16798.00) AND (store_sales.ss_coupon_amt <= 17798.00))) OR ((store_sales.ss_wholesale_cost >= 25.00) AND (store_sales.ss_wholesale_cost <= 45.00))) and (store_sales.ss_quantity <= 5) and (store_sales.ss_quantity >= 0)) +----------------------------------filter((store_sales.ss_quantity <= 5) and (store_sales.ss_quantity >= 0) and OR[AND[(store_sales.ss_list_price >= 131.00),(store_sales.ss_list_price <= 141.00)],AND[(store_sales.ss_coupon_amt >= 16798.00),(store_sales.ss_coupon_amt <= 17798.00)],AND[(store_sales.ss_wholesale_cost >= 25.00),(store_sales.ss_wholesale_cost <= 45.00)]]) ------------------------------------PhysicalOlapScan[store_sales] ------------------------PhysicalLimit[LOCAL] --------------------------hashAgg[GLOBAL] ----------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------hashAgg[LOCAL] --------------------------------PhysicalProject -----------------------------------filter(((((store_sales.ss_list_price >= 145.00) AND (store_sales.ss_list_price <= 155.00)) OR ((store_sales.ss_coupon_amt >= 14792.00) AND (store_sales.ss_coupon_amt <= 15792.00))) OR ((store_sales.ss_wholesale_cost >= 46.00) AND (store_sales.ss_wholesale_cost <= 66.00))) and (store_sales.ss_quantity <= 10) and (store_sales.ss_quantity >= 6)) +----------------------------------filter((store_sales.ss_quantity <= 10) and (store_sales.ss_quantity >= 6) and OR[AND[(store_sales.ss_list_price >= 145.00),(store_sales.ss_list_price <= 155.00)],AND[(store_sales.ss_coupon_amt >= 14792.00),(store_sales.ss_coupon_amt <= 15792.00)],AND[(store_sales.ss_wholesale_cost >= 46.00),(store_sales.ss_wholesale_cost <= 66.00)]]) ------------------------------------PhysicalOlapScan[store_sales] --------------------PhysicalLimit[LOCAL] ----------------------hashAgg[GLOBAL] ------------------------PhysicalDistribute[DistributionSpecGather] --------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------filter(((((store_sales.ss_list_price >= 150.00) AND (store_sales.ss_list_price <= 160.00)) OR ((store_sales.ss_coupon_amt >= 6600.00) AND (store_sales.ss_coupon_amt <= 7600.00))) OR ((store_sales.ss_wholesale_cost >= 9.00) AND (store_sales.ss_wholesale_cost <= 29.00))) and (store_sales.ss_quantity <= 15) and (store_sales.ss_quantity >= 11)) +------------------------------filter((store_sales.ss_quantity <= 15) and (store_sales.ss_quantity >= 11) and OR[AND[(store_sales.ss_list_price >= 150.00),(store_sales.ss_list_price <= 160.00)],AND[(store_sales.ss_coupon_amt >= 6600.00),(store_sales.ss_coupon_amt <= 7600.00)],AND[(store_sales.ss_wholesale_cost >= 9.00),(store_sales.ss_wholesale_cost <= 29.00)]]) --------------------------------PhysicalOlapScan[store_sales] ----------------PhysicalLimit[LOCAL] ------------------hashAgg[GLOBAL] --------------------PhysicalDistribute[DistributionSpecGather] ----------------------hashAgg[LOCAL] ------------------------PhysicalProject ---------------------------filter(((((store_sales.ss_list_price >= 91.00) AND (store_sales.ss_list_price <= 101.00)) OR ((store_sales.ss_coupon_amt >= 13493.00) AND (store_sales.ss_coupon_amt <= 14493.00))) OR ((store_sales.ss_wholesale_cost >= 36.00) AND (store_sales.ss_wholesale_cost <= 56.00))) and (store_sales.ss_quantity <= 20) and (store_sales.ss_quantity >= 16)) +--------------------------filter((store_sales.ss_quantity <= 20) and (store_sales.ss_quantity >= 16) and OR[AND[(store_sales.ss_list_price >= 91.00),(store_sales.ss_list_price <= 101.00)],AND[(store_sales.ss_coupon_amt >= 13493.00),(store_sales.ss_coupon_amt <= 14493.00)],AND[(store_sales.ss_wholesale_cost >= 36.00),(store_sales.ss_wholesale_cost <= 56.00)]]) ----------------------------PhysicalOlapScan[store_sales] ------------PhysicalLimit[LOCAL] --------------hashAgg[GLOBAL] ----------------PhysicalDistribute[DistributionSpecGather] ------------------hashAgg[LOCAL] --------------------PhysicalProject -----------------------filter(((((store_sales.ss_list_price >= 0.00) AND (store_sales.ss_list_price <= 10.00)) OR ((store_sales.ss_coupon_amt >= 7629.00) AND (store_sales.ss_coupon_amt <= 8629.00))) OR ((store_sales.ss_wholesale_cost >= 6.00) AND (store_sales.ss_wholesale_cost <= 26.00))) and (store_sales.ss_quantity <= 25) and (store_sales.ss_quantity >= 21)) +----------------------filter((store_sales.ss_quantity <= 25) and (store_sales.ss_quantity >= 21) and OR[AND[(store_sales.ss_list_price >= 0.00),(store_sales.ss_list_price <= 10.00)],AND[(store_sales.ss_coupon_amt >= 7629.00),(store_sales.ss_coupon_amt <= 8629.00)],AND[(store_sales.ss_wholesale_cost >= 6.00),(store_sales.ss_wholesale_cost <= 26.00)]]) ------------------------PhysicalOlapScan[store_sales] --------PhysicalLimit[LOCAL] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecGather] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------filter(((((store_sales.ss_list_price >= 89.00) AND (store_sales.ss_list_price <= 99.00)) OR ((store_sales.ss_coupon_amt >= 15257.00) AND (store_sales.ss_coupon_amt <= 16257.00))) OR ((store_sales.ss_wholesale_cost >= 31.00) AND (store_sales.ss_wholesale_cost <= 51.00))) and (store_sales.ss_quantity <= 30) and (store_sales.ss_quantity >= 26)) +------------------filter((store_sales.ss_quantity <= 30) and (store_sales.ss_quantity >= 26) and OR[AND[(store_sales.ss_list_price >= 89.00),(store_sales.ss_list_price <= 99.00)],AND[(store_sales.ss_coupon_amt >= 15257.00),(store_sales.ss_coupon_amt <= 16257.00)],AND[(store_sales.ss_wholesale_cost >= 31.00),(store_sales.ss_wholesale_cost <= 51.00)]]) --------------------PhysicalOlapScan[store_sales] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query34.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query34.out index bb286ee190e816..489abcd4c2e33c 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query34.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query34.out @@ -21,7 +21,7 @@ PhysicalResultSink --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 --------------------------------PhysicalProject -----------------------------------filter((((date_dim.d_dom >= 1) AND (date_dim.d_dom <= 3)) OR ((date_dim.d_dom >= 25) AND (date_dim.d_dom <= 28))) and d_year IN (1998, 1999, 2000)) +----------------------------------filter(OR[AND[(date_dim.d_dom >= 1),(date_dim.d_dom <= 3)],AND[(date_dim.d_dom >= 25),(date_dim.d_dom <= 28)]] and d_year IN (1998, 1999, 2000)) ------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalProject ------------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('0-500', '1001-5000')) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query35.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query35.out index 4da981f140aa0e..1e865046f6cf27 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query35.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query35.out @@ -9,7 +9,7 @@ PhysicalResultSink ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) +------------------filter(OR[ifnull($c$1, FALSE),ifnull($c$2, FALSE)]) --------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() ----------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() ------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF5 ss_customer_sk->[c_customer_sk] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query41.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query41.out index c9c1ffff499741..3034a77fe0897a 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query41.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query41.out @@ -18,6 +18,6 @@ PhysicalResultSink ------------------------PhysicalDistribute[DistributionSpecHash] --------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------filter((((i_color IN ('aquamarine', 'blue', 'chartreuse', 'chiffon', 'dodger', 'gainsboro', 'tan', 'violet') AND i_units IN ('Bunch', 'Dozen', 'Each', 'Ounce', 'Oz', 'Pound', 'Ton', 'Tsp')) AND (((((((item.i_category = 'Women') AND i_color IN ('aquamarine', 'gainsboro')) AND i_units IN ('Dozen', 'Ounce')) AND i_size IN ('economy', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('chiffon', 'violet')) AND i_units IN ('Pound', 'Ton')) AND i_size IN ('extra large', 'small'))) OR ((((item.i_category = 'Men') AND i_color IN ('blue', 'chartreuse')) AND i_units IN ('Each', 'Oz')) AND i_size IN ('N/A', 'large'))) OR ((((item.i_category = 'Men') AND i_color IN ('dodger', 'tan')) AND i_units IN ('Bunch', 'Tsp')) AND i_size IN ('economy', 'medium')))) OR ((i_color IN ('almond', 'blanched', 'indian', 'lime', 'peru', 'saddle', 'spring', 'tomato') AND i_units IN ('Box', 'Carton', 'Case', 'Dram', 'Gram', 'Pallet', 'Tbl', 'Unknown')) AND (((((((item.i_category = 'Women') AND i_color IN ('blanched', 'tomato')) AND i_units IN ('Case', 'Tbl')) AND i_size IN ('economy', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('almond', 'lime')) AND i_units IN ('Box', 'Dram')) AND i_size IN ('extra large', 'small'))) OR ((((item.i_category = 'Men') AND i_color IN ('peru', 'saddle')) AND i_units IN ('Gram', 'Pallet')) AND i_size IN ('N/A', 'large'))) OR ((((item.i_category = 'Men') AND i_color IN ('indian', 'spring')) AND i_units IN ('Carton', 'Unknown')) AND i_size IN ('economy', 'medium'))))) and i_category IN ('Men', 'Women') and i_size IN ('N/A', 'economy', 'extra large', 'large', 'medium', 'small')) +------------------------------filter(OR[AND[i_color IN ('aquamarine', 'blue', 'chartreuse', 'chiffon', 'dodger', 'gainsboro', 'tan', 'violet'),i_units IN ('Bunch', 'Dozen', 'Each', 'Ounce', 'Oz', 'Pound', 'Ton', 'Tsp'),OR[AND[(item.i_category = 'Women'),i_color IN ('aquamarine', 'gainsboro'),i_units IN ('Dozen', 'Ounce'),i_size IN ('economy', 'medium')],AND[(item.i_category = 'Women'),i_color IN ('chiffon', 'violet'),i_units IN ('Pound', 'Ton'),i_size IN ('extra large', 'small')],AND[(item.i_category = 'Men'),i_color IN ('blue', 'chartreuse'),i_units IN ('Each', 'Oz'),i_size IN ('N/A', 'large')],AND[(item.i_category = 'Men'),i_color IN ('dodger', 'tan'),i_units IN ('Bunch', 'Tsp'),i_size IN ('economy', 'medium')]]],AND[i_color IN ('almond', 'blanched', 'indian', 'lime', 'peru', 'saddle', 'spring', 'tomato'),i_units IN ('Box', 'Carton', 'Case', 'Dram', 'Gram', 'Pallet', 'Tbl', 'Unknown'),OR[AND[(item.i_category = 'Women'),i_color IN ('blanched', 'tomato'),i_units IN ('Case', 'Tbl'),i_size IN ('economy', 'medium')],AND[(item.i_category = 'Women'),i_color IN ('almond', 'lime'),i_units IN ('Box', 'Dram'),i_size IN ('extra large', 'small')],AND[(item.i_category = 'Men'),i_color IN ('peru', 'saddle'),i_units IN ('Gram', 'Pallet'),i_size IN ('N/A', 'large')],AND[(item.i_category = 'Men'),i_color IN ('indian', 'spring'),i_units IN ('Carton', 'Unknown'),i_size IN ('economy', 'medium')]]]] and i_category IN ('Men', 'Women') and i_size IN ('N/A', 'economy', 'extra large', 'large', 'medium', 'small')) --------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query45.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query45.out index 83f6b9ca5df1f0..40b25ae51ad929 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query45.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query45.out @@ -8,7 +8,7 @@ PhysicalResultSink ----------PhysicalDistribute[DistributionSpecHash] ------------hashAgg[LOCAL] --------------PhysicalProject -----------------filter((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR $c$1)) +----------------filter(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),$c$1]) ------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query46.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query46.out index 2895fa954ff4eb..75c9af2b354fae 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query46.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query46.out @@ -22,7 +22,7 @@ PhysicalResultSink ----------------------------------filter(d_dow IN (0, 6) and d_year IN (1999, 2000, 2001)) ------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalProject -------------------------------filter(((household_demographics.hd_dep_count = 6) OR (household_demographics.hd_vehicle_count = 0))) +------------------------------filter(OR[(household_demographics.hd_dep_count = 6),(household_demographics.hd_vehicle_count = 0)]) --------------------------------PhysicalOlapScan[household_demographics] ------------------------PhysicalProject --------------------------filter(s_city IN ('Centerville', 'Fairview', 'Five Points', 'Liberty', 'Oak Grove')) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query47.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query47.out index e115c8751281e0..048f93392595ec 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query47.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query47.out @@ -21,7 +21,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------------------------PhysicalProject --------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 ------------------------------------PhysicalProject ---------------------------------------filter((((date_dim.d_year = 2001) OR ((date_dim.d_year = 2000) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2002) AND (date_dim.d_moy = 1))) and d_year IN (2000, 2001, 2002)) +--------------------------------------filter(OR[(date_dim.d_year = 2001),AND[(date_dim.d_year = 2000),(date_dim.d_moy = 12)],AND[(date_dim.d_year = 2002),(date_dim.d_moy = 1)]] and d_year IN (2000, 2001, 2002)) ----------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query48.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query48.out index 476aa9dd8793af..3f16a98d674c76 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query48.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query48.out @@ -7,23 +7,23 @@ PhysicalResultSink --------PhysicalProject ----------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() ------------PhysicalProject ---------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +--------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=(OR[AND[ca_state IN ('IA', 'MD', 'MN'),(store_sales.ss_net_profit >= 0.00),(store_sales.ss_net_profit <= 2000.00)],AND[ca_state IN ('IL', 'TX', 'VA'),(store_sales.ss_net_profit >= 150.00),(store_sales.ss_net_profit <= 3000.00)],AND[ca_state IN ('IN', 'MI', 'WI'),(store_sales.ss_net_profit >= 50.00),(store_sales.ss_net_profit <= 25000.00)]]) build RFs:RF2 ca_address_sk->[ss_addr_sk] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=((((ca_state IN ('IA', 'MD', 'MN') AND ((store_sales.ss_net_profit >= 0.00) AND (store_sales.ss_net_profit <= 2000.00))) OR (ca_state IN ('IL', 'TX', 'VA') AND ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 3000.00)))) OR (ca_state IN ('IN', 'MI', 'WI') AND ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 25000.00))))) build RFs:RF1 ca_address_sk->[ss_addr_sk] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)) otherCondition=((((((customer_demographics.cd_marital_status = 'U') AND (customer_demographics.cd_education_status = 'Primary')) AND ((store_sales.ss_sales_price >= 100.00) AND (store_sales.ss_sales_price <= 150.00))) OR (((customer_demographics.cd_marital_status = 'W') AND (customer_demographics.cd_education_status = 'College')) AND ((store_sales.ss_sales_price >= 50.00) AND (store_sales.ss_sales_price <= 100.00)))) OR (((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = '2 yr Degree')) AND ((store_sales.ss_sales_price >= 150.00) AND (store_sales.ss_sales_price <= 200.00))))) build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)) otherCondition=(OR[AND[(customer_demographics.cd_marital_status = 'U'),(customer_demographics.cd_education_status = 'Primary'),(store_sales.ss_sales_price >= 100.00),(store_sales.ss_sales_price <= 150.00)],AND[(customer_demographics.cd_marital_status = 'W'),(customer_demographics.cd_education_status = 'College'),(store_sales.ss_sales_price >= 50.00),(store_sales.ss_sales_price <= 100.00)],AND[(customer_demographics.cd_marital_status = 'D'),(customer_demographics.cd_education_status = '2 yr Degree'),(store_sales.ss_sales_price >= 150.00),(store_sales.ss_sales_price <= 200.00)]]) build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] ------------------------PhysicalProject --------------------------filter((store_sales.ss_net_profit <= 25000.00) and (store_sales.ss_net_profit >= 0.00) and (store_sales.ss_sales_price <= 200.00) and (store_sales.ss_sales_price >= 50.00)) ----------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 ------------------------PhysicalProject ---------------------------filter(((((customer_demographics.cd_marital_status = 'U') AND (customer_demographics.cd_education_status = 'Primary')) OR ((customer_demographics.cd_marital_status = 'W') AND (customer_demographics.cd_education_status = 'College'))) OR ((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = '2 yr Degree'))) and cd_education_status IN ('2 yr Degree', 'College', 'Primary') and cd_marital_status IN ('D', 'U', 'W')) +--------------------------filter(OR[AND[(customer_demographics.cd_marital_status = 'U'),(customer_demographics.cd_education_status = 'Primary')],AND[(customer_demographics.cd_marital_status = 'W'),(customer_demographics.cd_education_status = 'College')],AND[(customer_demographics.cd_marital_status = 'D'),(customer_demographics.cd_education_status = '2 yr Degree')]] and cd_education_status IN ('2 yr Degree', 'College', 'Primary') and cd_marital_status IN ('D', 'U', 'W')) ----------------------------PhysicalOlapScan[customer_demographics] --------------------PhysicalProject -----------------------filter((customer_address.ca_country = 'United States') and ca_state IN ('IA', 'IL', 'IN', 'MD', 'MI', 'MN', 'TX', 'VA', 'WI')) -------------------------PhysicalOlapScan[customer_address] +----------------------filter((date_dim.d_year = 1999)) +------------------------PhysicalOlapScan[date_dim] ----------------PhysicalProject -------------------filter((date_dim.d_year = 1999)) ---------------------PhysicalOlapScan[date_dim] +------------------filter((customer_address.ca_country = 'United States') and ca_state IN ('IA', 'IL', 'IN', 'MD', 'MI', 'MN', 'TX', 'VA', 'WI')) +--------------------PhysicalOlapScan[customer_address] ------------PhysicalProject --------------PhysicalOlapScan[store] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query49.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query49.out index e34962e0847b0d..7d0f6b1ce22d0f 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query49.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query49.out @@ -16,7 +16,7 @@ PhysicalResultSink --------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) ----------------------------------PhysicalWindow ------------------------------------PhysicalQuickSort[LOCAL_SORT] --------------------------------------PhysicalWindow @@ -48,7 +48,7 @@ PhysicalResultSink --------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) ----------------------------------PhysicalWindow ------------------------------------PhysicalQuickSort[LOCAL_SORT] --------------------------------------PhysicalWindow @@ -80,7 +80,7 @@ PhysicalResultSink --------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) ----------------------------------PhysicalWindow ------------------------------------PhysicalQuickSort[LOCAL_SORT] --------------------------------------PhysicalWindow diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query53.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query53.out index 93289fca48e4ef..89dc632eb527c4 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query53.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query53.out @@ -22,7 +22,7 @@ PhysicalResultSink --------------------------------------PhysicalProject ----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 --------------------------------------PhysicalProject -----------------------------------------filter((((i_category IN ('Books', 'Children', 'Electronics') AND i_class IN ('personal', 'portable', 'reference', 'self-help')) AND i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')) OR ((i_category IN ('Men', 'Music', 'Women') AND i_class IN ('accessories', 'classical', 'fragrances', 'pants')) AND i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1'))) and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) +----------------------------------------filter(OR[AND[i_category IN ('Books', 'Children', 'Electronics'),i_class IN ('personal', 'portable', 'reference', 'self-help'),i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')],AND[i_category IN ('Men', 'Music', 'Women'),i_class IN ('accessories', 'classical', 'fragrances', 'pants'),i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')]] and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) ------------------------------------------PhysicalOlapScan[item] ----------------------------------PhysicalProject ------------------------------------filter(d_month_seq IN (1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211)) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query57.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query57.out index 4a7de98eb56b50..4f23fac89cf958 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query57.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query57.out @@ -21,7 +21,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------------------------PhysicalProject --------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 ------------------------------------PhysicalProject ---------------------------------------filter((((date_dim.d_year = 1999) OR ((date_dim.d_year = 1998) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2000) AND (date_dim.d_moy = 1))) and d_year IN (1998, 1999, 2000)) +--------------------------------------filter(OR[(date_dim.d_year = 1999),AND[(date_dim.d_year = 1998),(date_dim.d_moy = 12)],AND[(date_dim.d_year = 2000),(date_dim.d_moy = 1)]] and d_year IN (1998, 1999, 2000)) ----------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query61.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query61.out index d034c5b43231d7..faf30604b86926 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query61.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query61.out @@ -29,7 +29,7 @@ PhysicalResultSink ------------------------------PhysicalProject --------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_promo_sk = promotion.p_promo_sk)) otherCondition=() build RFs:RF6 ss_promo_sk->[p_promo_sk] ----------------------------------PhysicalProject -------------------------------------filter((((promotion.p_channel_dmail = 'Y') OR (promotion.p_channel_email = 'Y')) OR (promotion.p_channel_tv = 'Y'))) +------------------------------------filter(OR[(promotion.p_channel_dmail = 'Y'),(promotion.p_channel_email = 'Y'),(promotion.p_channel_tv = 'Y')]) --------------------------------------PhysicalOlapScan[promotion] apply RFs: RF6 ----------------------------------PhysicalProject ------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF5 s_store_sk->[ss_store_sk] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query63.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query63.out index 3bd844ce72c196..9653f6c52199aa 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query63.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query63.out @@ -22,7 +22,7 @@ PhysicalResultSink --------------------------------------PhysicalProject ----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 --------------------------------------PhysicalProject -----------------------------------------filter((((i_category IN ('Books', 'Children', 'Electronics') AND i_class IN ('personal', 'portable', 'reference', 'self-help')) AND i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')) OR ((i_category IN ('Men', 'Music', 'Women') AND i_class IN ('accessories', 'classical', 'fragrances', 'pants')) AND i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1'))) and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) +----------------------------------------filter(OR[AND[i_category IN ('Books', 'Children', 'Electronics'),i_class IN ('personal', 'portable', 'reference', 'self-help'),i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')],AND[i_category IN ('Men', 'Music', 'Women'),i_class IN ('accessories', 'classical', 'fragrances', 'pants'),i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')]] and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) ------------------------------------------PhysicalOlapScan[item] ----------------------------------PhysicalProject ------------------------------------filter(d_month_seq IN (1181, 1182, 1183, 1184, 1185, 1186, 1187, 1188, 1189, 1190, 1191, 1192)) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query68.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query68.out index 8975dc78ab76e5..aa07d1b2a42d9b 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query68.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query68.out @@ -33,6 +33,6 @@ PhysicalResultSink ----------------------------------filter(s_city IN ('Five Points', 'Pleasant Hill')) ------------------------------------PhysicalOlapScan[store] ----------------------------PhysicalProject -------------------------------filter(((household_demographics.hd_dep_count = 8) OR (household_demographics.hd_vehicle_count = -1))) +------------------------------filter(OR[(household_demographics.hd_dep_count = 8),(household_demographics.hd_vehicle_count = -1)]) --------------------------------PhysicalOlapScan[household_demographics] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query7.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query7.out index 3d8133e9159fea..18425373b08452 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query7.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query7.out @@ -24,7 +24,7 @@ PhysicalResultSink ----------------------------filter((date_dim.d_year = 2001)) ------------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalProject -------------------------filter(((promotion.p_channel_email = 'N') OR (promotion.p_channel_event = 'N'))) +------------------------filter(OR[(promotion.p_channel_email = 'N'),(promotion.p_channel_event = 'N')]) --------------------------PhysicalOlapScan[promotion] ------------------PhysicalProject --------------------PhysicalOlapScan[item] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query78.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query78.out index 30527883a49018..a6034ca86ac5c5 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query78.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query78.out @@ -5,7 +5,7 @@ PhysicalResultSink ----PhysicalDistribute[DistributionSpecGather] ------PhysicalTopN[LOCAL_SORT] --------PhysicalProject -----------filter(((coalesce(ws_qty, 0) > 0) OR (coalesce(cs_qty, 0) > 0))) +----------filter(OR[(coalesce(ws_qty, 0) > 0),(coalesce(cs_qty, 0) > 0)]) ------------hashJoin[LEFT_OUTER_JOIN colocated] hashCondition=((cs.cs_customer_sk = ss.ss_customer_sk) and (cs.cs_item_sk = ss.ss_item_sk) and (cs.cs_sold_year = ss.ss_sold_year)) otherCondition=() --------------PhysicalProject ----------------hashJoin[LEFT_OUTER_JOIN colocated] hashCondition=((ws.ws_customer_sk = ss.ss_customer_sk) and (ws.ws_item_sk = ss.ss_item_sk) and (ws.ws_sold_year = ss.ss_sold_year)) otherCondition=() diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query79.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query79.out index 0ceac85291e12d..f57418546e7fb9 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query79.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query79.out @@ -22,7 +22,7 @@ PhysicalResultSink ----------------------------------filter((date_dim.d_dow = 1) and d_year IN (1998, 1999, 2000)) ------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalProject -------------------------------filter(((household_demographics.hd_dep_count = 5) OR (household_demographics.hd_vehicle_count > 4))) +------------------------------filter(OR[(household_demographics.hd_dep_count = 5),(household_demographics.hd_vehicle_count > 4)]) --------------------------------PhysicalOlapScan[household_demographics] ------------------------PhysicalProject --------------------------filter((store.s_number_employees <= 295) and (store.s_number_employees >= 200)) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query85.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query85.out index c428ce03bde80e..d9d74a4dd87f7e 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query85.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query85.out @@ -18,12 +18,12 @@ PhysicalResultSink ------------------------------filter(cd_education_status IN ('4 yr Degree', 'Advanced Degree', 'Secondary') and cd_marital_status IN ('M', 'S', 'W')) --------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF5 RF6 RF7 ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cd1.cd_demo_sk = web_returns.wr_refunded_cdemo_sk)) otherCondition=((((((cd1.cd_marital_status = 'M') AND (cd1.cd_education_status = '4 yr Degree')) AND ((web_sales.ws_sales_price >= 100.00) AND (web_sales.ws_sales_price <= 150.00))) OR (((cd1.cd_marital_status = 'S') AND (cd1.cd_education_status = 'Secondary')) AND ((web_sales.ws_sales_price >= 50.00) AND (web_sales.ws_sales_price <= 100.00)))) OR (((cd1.cd_marital_status = 'W') AND (cd1.cd_education_status = 'Advanced Degree')) AND ((web_sales.ws_sales_price >= 150.00) AND (web_sales.ws_sales_price <= 200.00))))) build RFs:RF4 wr_refunded_cdemo_sk->[cd_demo_sk] +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cd1.cd_demo_sk = web_returns.wr_refunded_cdemo_sk)) otherCondition=(OR[AND[(cd1.cd_marital_status = 'M'),(cd1.cd_education_status = '4 yr Degree'),(web_sales.ws_sales_price >= 100.00),(web_sales.ws_sales_price <= 150.00)],AND[(cd1.cd_marital_status = 'S'),(cd1.cd_education_status = 'Secondary'),(web_sales.ws_sales_price >= 50.00),(web_sales.ws_sales_price <= 100.00)],AND[(cd1.cd_marital_status = 'W'),(cd1.cd_education_status = 'Advanced Degree'),(web_sales.ws_sales_price >= 150.00),(web_sales.ws_sales_price <= 200.00)]]) build RFs:RF4 wr_refunded_cdemo_sk->[cd_demo_sk] --------------------------------PhysicalProject -----------------------------------filter(((((cd1.cd_marital_status = 'M') AND (cd1.cd_education_status = '4 yr Degree')) OR ((cd1.cd_marital_status = 'S') AND (cd1.cd_education_status = 'Secondary'))) OR ((cd1.cd_marital_status = 'W') AND (cd1.cd_education_status = 'Advanced Degree'))) and cd_education_status IN ('4 yr Degree', 'Advanced Degree', 'Secondary') and cd_marital_status IN ('M', 'S', 'W')) +----------------------------------filter(OR[AND[(cd1.cd_marital_status = 'M'),(cd1.cd_education_status = '4 yr Degree')],AND[(cd1.cd_marital_status = 'S'),(cd1.cd_education_status = 'Secondary')],AND[(cd1.cd_marital_status = 'W'),(cd1.cd_education_status = 'Advanced Degree')]] and cd_education_status IN ('4 yr Degree', 'Advanced Degree', 'Secondary') and cd_marital_status IN ('M', 'S', 'W')) ------------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF4 --------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((customer_address.ca_address_sk = web_returns.wr_refunded_addr_sk)) otherCondition=((((ca_state IN ('DE', 'FL', 'TX') AND ((web_sales.ws_net_profit >= 100.00) AND (web_sales.ws_net_profit <= 200.00))) OR (ca_state IN ('ID', 'IN', 'ND') AND ((web_sales.ws_net_profit >= 150.00) AND (web_sales.ws_net_profit <= 300.00)))) OR (ca_state IN ('IL', 'MT', 'OH') AND ((web_sales.ws_net_profit >= 50.00) AND (web_sales.ws_net_profit <= 250.00))))) build RFs:RF3 ca_address_sk->[wr_refunded_addr_sk] +----------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((customer_address.ca_address_sk = web_returns.wr_refunded_addr_sk)) otherCondition=(OR[AND[ca_state IN ('DE', 'FL', 'TX'),(web_sales.ws_net_profit >= 100.00),(web_sales.ws_net_profit <= 200.00)],AND[ca_state IN ('ID', 'IN', 'ND'),(web_sales.ws_net_profit >= 150.00),(web_sales.ws_net_profit <= 300.00)],AND[ca_state IN ('IL', 'MT', 'OH'),(web_sales.ws_net_profit >= 50.00),(web_sales.ws_net_profit <= 250.00)]]) build RFs:RF3 ca_address_sk->[wr_refunded_addr_sk] ------------------------------------PhysicalProject --------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((web_sales.ws_item_sk = web_returns.wr_item_sk) and (web_sales.ws_order_number = web_returns.wr_order_number)) otherCondition=() build RFs:RF1 ws_item_sk->[wr_item_sk];RF2 ws_order_number->[wr_order_number] ----------------------------------------PhysicalProject diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query88.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query88.out index bca8b0d069014b..09c630cef443ef 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query88.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query88.out @@ -23,7 +23,7 @@ PhysicalResultSink ------------------------------------filter((time_dim.t_hour = 8) and (time_dim.t_minute >= 30)) --------------------------------------PhysicalOlapScan[time_dim] ------------------------------PhysicalProject ---------------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +--------------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) ----------------------------------PhysicalOlapScan[household_demographics] --------------------------PhysicalProject ----------------------------filter((store.s_store_name = 'ese')) @@ -43,7 +43,7 @@ PhysicalResultSink ------------------------------------filter((time_dim.t_hour = 9) and (time_dim.t_minute < 30)) --------------------------------------PhysicalOlapScan[time_dim] ------------------------------PhysicalProject ---------------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +--------------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) ----------------------------------PhysicalOlapScan[household_demographics] --------------------------PhysicalProject ----------------------------filter((store.s_store_name = 'ese')) @@ -63,7 +63,7 @@ PhysicalResultSink ----------------------------------filter((time_dim.t_hour = 9) and (time_dim.t_minute >= 30)) ------------------------------------PhysicalOlapScan[time_dim] ----------------------------PhysicalProject -------------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +------------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) --------------------------------PhysicalOlapScan[household_demographics] ------------------------PhysicalProject --------------------------filter((store.s_store_name = 'ese')) @@ -83,7 +83,7 @@ PhysicalResultSink --------------------------------filter((time_dim.t_hour = 10) and (time_dim.t_minute < 30)) ----------------------------------PhysicalOlapScan[time_dim] --------------------------PhysicalProject -----------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +----------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) ------------------------------PhysicalOlapScan[household_demographics] ----------------------PhysicalProject ------------------------filter((store.s_store_name = 'ese')) @@ -103,7 +103,7 @@ PhysicalResultSink ------------------------------filter((time_dim.t_hour = 10) and (time_dim.t_minute >= 30)) --------------------------------PhysicalOlapScan[time_dim] ------------------------PhysicalProject ---------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +--------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) ----------------------------PhysicalOlapScan[household_demographics] --------------------PhysicalProject ----------------------filter((store.s_store_name = 'ese')) @@ -123,7 +123,7 @@ PhysicalResultSink ----------------------------filter((time_dim.t_hour = 11) and (time_dim.t_minute < 30)) ------------------------------PhysicalOlapScan[time_dim] ----------------------PhysicalProject -------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) --------------------------PhysicalOlapScan[household_demographics] ------------------PhysicalProject --------------------filter((store.s_store_name = 'ese')) @@ -143,7 +143,7 @@ PhysicalResultSink --------------------------filter((time_dim.t_hour = 11) and (time_dim.t_minute >= 30)) ----------------------------PhysicalOlapScan[time_dim] --------------------PhysicalProject -----------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +----------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) ------------------------PhysicalOlapScan[household_demographics] ----------------PhysicalProject ------------------filter((store.s_store_name = 'ese')) @@ -163,7 +163,7 @@ PhysicalResultSink ------------------------filter((time_dim.t_hour = 12) and (time_dim.t_minute < 30)) --------------------------PhysicalOlapScan[time_dim] ------------------PhysicalProject ---------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +--------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) ----------------------PhysicalOlapScan[household_demographics] --------------PhysicalProject ----------------filter((store.s_store_name = 'ese')) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query89.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query89.out index c709c4df5fe097..2e9294bce91fd2 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query89.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query89.out @@ -23,7 +23,7 @@ PhysicalResultSink ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 ----------------------------------------PhysicalProject -------------------------------------------filter(((i_category IN ('Electronics', 'Jewelry', 'Shoes') AND i_class IN ('athletic', 'portable', 'semi-precious')) OR (i_category IN ('Men', 'Music', 'Women') AND i_class IN ('accessories', 'maternity', 'rock'))) and i_category IN ('Electronics', 'Jewelry', 'Men', 'Music', 'Shoes', 'Women') and i_class IN ('accessories', 'athletic', 'maternity', 'portable', 'rock', 'semi-precious')) +------------------------------------------filter(OR[AND[i_category IN ('Electronics', 'Jewelry', 'Shoes'),i_class IN ('athletic', 'portable', 'semi-precious')],AND[i_category IN ('Men', 'Music', 'Women'),i_class IN ('accessories', 'maternity', 'rock')]] and i_category IN ('Electronics', 'Jewelry', 'Men', 'Music', 'Shoes', 'Women') and i_class IN ('accessories', 'athletic', 'maternity', 'portable', 'rock', 'semi-precious')) --------------------------------------------PhysicalOlapScan[item] ------------------------------------PhysicalProject --------------------------------------filter((date_dim.d_year = 1999)) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query91.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query91.out index e2a4ed89af442d..6593bb83f07994 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query91.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query91.out @@ -28,7 +28,7 @@ PhysicalResultSink ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 RF1 ----------------------------------------PhysicalProject -------------------------------------------filter((((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = 'Unknown')) OR ((customer_demographics.cd_marital_status = 'W') AND (customer_demographics.cd_education_status = 'Advanced Degree'))) and cd_education_status IN ('Advanced Degree', 'Unknown') and cd_marital_status IN ('M', 'W')) +------------------------------------------filter(OR[AND[(customer_demographics.cd_marital_status = 'M'),(customer_demographics.cd_education_status = 'Unknown')],AND[(customer_demographics.cd_marital_status = 'W'),(customer_demographics.cd_education_status = 'Advanced Degree')]] and cd_education_status IN ('Advanced Degree', 'Unknown') and cd_marital_status IN ('M', 'W')) --------------------------------------------PhysicalOlapScan[customer_demographics] ------------------------------------PhysicalProject --------------------------------------filter((hd_buy_potential like '1001-5000%')) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query10.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query10.out index 6027b8b2684b4b..4dfc2de4cf3fe5 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query10.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query10.out @@ -9,7 +9,7 @@ PhysicalResultSink ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) +------------------filter(OR[ifnull($c$1, FALSE),ifnull($c$2, FALSE)]) --------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() ----------------------PhysicalProject ------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query13.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query13.out index 7528312c6dae6b..e8a4d92c4f7bf1 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query13.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query13.out @@ -7,16 +7,16 @@ PhysicalResultSink --------PhysicalProject ----------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF4 s_store_sk->[ss_store_sk] ------------PhysicalProject ---------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)) otherCondition=((((((household_demographics.hd_dep_count = 1) AND cd_marital_status IN ('M', 'S')) AND cd_education_status IN ('4 yr Degree', 'College')) AND ((((customer_demographics.cd_marital_status = 'S') AND (customer_demographics.cd_education_status = 'College')) AND ((store_sales.ss_sales_price >= 50.00) AND (store_sales.ss_sales_price <= 100.00))) OR (((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = '4 yr Degree')) AND ((store_sales.ss_sales_price >= 150.00) AND (store_sales.ss_sales_price <= 200.00))))) OR ((((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = 'Unknown')) AND ((store_sales.ss_sales_price >= 100.00) AND (store_sales.ss_sales_price <= 150.00))) AND (household_demographics.hd_dep_count = 3)))) build RFs:RF3 ss_cdemo_sk->[cd_demo_sk] +--------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)) otherCondition=(OR[AND[(household_demographics.hd_dep_count = 1),cd_marital_status IN ('M', 'S'),cd_education_status IN ('4 yr Degree', 'College'),OR[AND[(customer_demographics.cd_marital_status = 'S'),(customer_demographics.cd_education_status = 'College'),(store_sales.ss_sales_price >= 50.00),(store_sales.ss_sales_price <= 100.00)],AND[(customer_demographics.cd_marital_status = 'M'),(customer_demographics.cd_education_status = '4 yr Degree'),(store_sales.ss_sales_price >= 150.00),(store_sales.ss_sales_price <= 200.00)]]],AND[(customer_demographics.cd_marital_status = 'D'),(customer_demographics.cd_education_status = 'Unknown'),(store_sales.ss_sales_price >= 100.00),(store_sales.ss_sales_price <= 150.00),(household_demographics.hd_dep_count = 3)]]) build RFs:RF3 ss_cdemo_sk->[cd_demo_sk] ----------------PhysicalProject -------------------filter(((((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = 'Unknown')) OR ((customer_demographics.cd_marital_status = 'S') AND (customer_demographics.cd_education_status = 'College'))) OR ((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = '4 yr Degree'))) and cd_education_status IN ('4 yr Degree', 'College', 'Unknown') and cd_marital_status IN ('D', 'M', 'S')) +------------------filter(OR[AND[(customer_demographics.cd_marital_status = 'D'),(customer_demographics.cd_education_status = 'Unknown')],AND[(customer_demographics.cd_marital_status = 'S'),(customer_demographics.cd_education_status = 'College')],AND[(customer_demographics.cd_marital_status = 'M'),(customer_demographics.cd_education_status = '4 yr Degree')]] and cd_education_status IN ('4 yr Degree', 'College', 'Unknown') and cd_marital_status IN ('D', 'M', 'S')) --------------------PhysicalOlapScan[customer_demographics] apply RFs: RF3 ----------------PhysicalProject ------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF2 hd_demo_sk->[ss_hdemo_sk] --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=((((ca_state IN ('KS', 'MI', 'SD') AND ((store_sales.ss_net_profit >= 100.00) AND (store_sales.ss_net_profit <= 200.00))) OR (ca_state IN ('CO', 'MO', 'ND') AND ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 300.00)))) OR (ca_state IN ('NH', 'OH', 'TX') AND ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 250.00))))) build RFs:RF0 ca_address_sk->[ss_addr_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=(OR[AND[ca_state IN ('KS', 'MI', 'SD'),(store_sales.ss_net_profit >= 100.00),(store_sales.ss_net_profit <= 200.00)],AND[ca_state IN ('CO', 'MO', 'ND'),(store_sales.ss_net_profit >= 150.00),(store_sales.ss_net_profit <= 300.00)],AND[ca_state IN ('NH', 'OH', 'TX'),(store_sales.ss_net_profit >= 50.00),(store_sales.ss_net_profit <= 250.00)]]) build RFs:RF0 ca_address_sk->[ss_addr_sk] ----------------------------PhysicalProject ------------------------------filter((store_sales.ss_net_profit <= 300.00) and (store_sales.ss_net_profit >= 50.00) and (store_sales.ss_sales_price <= 200.00) and (store_sales.ss_sales_price >= 50.00)) --------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF4 diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query15.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query15.out index b80017b7ebd0f9..b93c82158d296a 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query15.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query15.out @@ -8,7 +8,7 @@ PhysicalResultSink ----------PhysicalDistribute[DistributionSpecHash] ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=(((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR ca_state IN ('CA', 'GA', 'WA')) OR (catalog_sales.cs_sales_price > 500.00))) build RFs:RF2 c_customer_sk->[cs_bill_customer_sk] +----------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),ca_state IN ('CA', 'GA', 'WA'),(catalog_sales.cs_sales_price > 500.00)]) build RFs:RF2 c_customer_sk->[cs_bill_customer_sk] ------------------PhysicalProject --------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] ----------------------PhysicalProject diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query26.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query26.out index d5fbe1d1762fb9..0c3a3f432aeb48 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query26.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query26.out @@ -24,7 +24,7 @@ PhysicalResultSink ----------------------------filter((date_dim.d_year = 2001)) ------------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalProject -------------------------filter(((promotion.p_channel_email = 'N') OR (promotion.p_channel_event = 'N'))) +------------------------filter(OR[(promotion.p_channel_email = 'N'),(promotion.p_channel_event = 'N')]) --------------------------PhysicalOlapScan[promotion] ------------------PhysicalProject --------------------PhysicalOlapScan[item] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query28.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query28.out index dae7cb1c23a402..7a6bdd8868ef00 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query28.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query28.out @@ -17,41 +17,41 @@ PhysicalResultSink ----------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------hashAgg[LOCAL] --------------------------------PhysicalProject -----------------------------------filter(((((store_sales.ss_list_price >= 131.00) AND (store_sales.ss_list_price <= 141.00)) OR ((store_sales.ss_coupon_amt >= 16798.00) AND (store_sales.ss_coupon_amt <= 17798.00))) OR ((store_sales.ss_wholesale_cost >= 25.00) AND (store_sales.ss_wholesale_cost <= 45.00))) and (store_sales.ss_quantity <= 5) and (store_sales.ss_quantity >= 0)) +----------------------------------filter((store_sales.ss_quantity <= 5) and (store_sales.ss_quantity >= 0) and OR[AND[(store_sales.ss_list_price >= 131.00),(store_sales.ss_list_price <= 141.00)],AND[(store_sales.ss_coupon_amt >= 16798.00),(store_sales.ss_coupon_amt <= 17798.00)],AND[(store_sales.ss_wholesale_cost >= 25.00),(store_sales.ss_wholesale_cost <= 45.00)]]) ------------------------------------PhysicalOlapScan[store_sales] ------------------------PhysicalLimit[LOCAL] --------------------------hashAgg[GLOBAL] ----------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------hashAgg[LOCAL] --------------------------------PhysicalProject -----------------------------------filter(((((store_sales.ss_list_price >= 145.00) AND (store_sales.ss_list_price <= 155.00)) OR ((store_sales.ss_coupon_amt >= 14792.00) AND (store_sales.ss_coupon_amt <= 15792.00))) OR ((store_sales.ss_wholesale_cost >= 46.00) AND (store_sales.ss_wholesale_cost <= 66.00))) and (store_sales.ss_quantity <= 10) and (store_sales.ss_quantity >= 6)) +----------------------------------filter((store_sales.ss_quantity <= 10) and (store_sales.ss_quantity >= 6) and OR[AND[(store_sales.ss_list_price >= 145.00),(store_sales.ss_list_price <= 155.00)],AND[(store_sales.ss_coupon_amt >= 14792.00),(store_sales.ss_coupon_amt <= 15792.00)],AND[(store_sales.ss_wholesale_cost >= 46.00),(store_sales.ss_wholesale_cost <= 66.00)]]) ------------------------------------PhysicalOlapScan[store_sales] --------------------PhysicalLimit[LOCAL] ----------------------hashAgg[GLOBAL] ------------------------PhysicalDistribute[DistributionSpecGather] --------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------filter(((((store_sales.ss_list_price >= 150.00) AND (store_sales.ss_list_price <= 160.00)) OR ((store_sales.ss_coupon_amt >= 6600.00) AND (store_sales.ss_coupon_amt <= 7600.00))) OR ((store_sales.ss_wholesale_cost >= 9.00) AND (store_sales.ss_wholesale_cost <= 29.00))) and (store_sales.ss_quantity <= 15) and (store_sales.ss_quantity >= 11)) +------------------------------filter((store_sales.ss_quantity <= 15) and (store_sales.ss_quantity >= 11) and OR[AND[(store_sales.ss_list_price >= 150.00),(store_sales.ss_list_price <= 160.00)],AND[(store_sales.ss_coupon_amt >= 6600.00),(store_sales.ss_coupon_amt <= 7600.00)],AND[(store_sales.ss_wholesale_cost >= 9.00),(store_sales.ss_wholesale_cost <= 29.00)]]) --------------------------------PhysicalOlapScan[store_sales] ----------------PhysicalLimit[LOCAL] ------------------hashAgg[GLOBAL] --------------------PhysicalDistribute[DistributionSpecGather] ----------------------hashAgg[LOCAL] ------------------------PhysicalProject ---------------------------filter(((((store_sales.ss_list_price >= 91.00) AND (store_sales.ss_list_price <= 101.00)) OR ((store_sales.ss_coupon_amt >= 13493.00) AND (store_sales.ss_coupon_amt <= 14493.00))) OR ((store_sales.ss_wholesale_cost >= 36.00) AND (store_sales.ss_wholesale_cost <= 56.00))) and (store_sales.ss_quantity <= 20) and (store_sales.ss_quantity >= 16)) +--------------------------filter((store_sales.ss_quantity <= 20) and (store_sales.ss_quantity >= 16) and OR[AND[(store_sales.ss_list_price >= 91.00),(store_sales.ss_list_price <= 101.00)],AND[(store_sales.ss_coupon_amt >= 13493.00),(store_sales.ss_coupon_amt <= 14493.00)],AND[(store_sales.ss_wholesale_cost >= 36.00),(store_sales.ss_wholesale_cost <= 56.00)]]) ----------------------------PhysicalOlapScan[store_sales] ------------PhysicalLimit[LOCAL] --------------hashAgg[GLOBAL] ----------------PhysicalDistribute[DistributionSpecGather] ------------------hashAgg[LOCAL] --------------------PhysicalProject -----------------------filter(((((store_sales.ss_list_price >= 0.00) AND (store_sales.ss_list_price <= 10.00)) OR ((store_sales.ss_coupon_amt >= 7629.00) AND (store_sales.ss_coupon_amt <= 8629.00))) OR ((store_sales.ss_wholesale_cost >= 6.00) AND (store_sales.ss_wholesale_cost <= 26.00))) and (store_sales.ss_quantity <= 25) and (store_sales.ss_quantity >= 21)) +----------------------filter((store_sales.ss_quantity <= 25) and (store_sales.ss_quantity >= 21) and OR[AND[(store_sales.ss_list_price >= 0.00),(store_sales.ss_list_price <= 10.00)],AND[(store_sales.ss_coupon_amt >= 7629.00),(store_sales.ss_coupon_amt <= 8629.00)],AND[(store_sales.ss_wholesale_cost >= 6.00),(store_sales.ss_wholesale_cost <= 26.00)]]) ------------------------PhysicalOlapScan[store_sales] --------PhysicalLimit[LOCAL] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecGather] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------filter(((((store_sales.ss_list_price >= 89.00) AND (store_sales.ss_list_price <= 99.00)) OR ((store_sales.ss_coupon_amt >= 15257.00) AND (store_sales.ss_coupon_amt <= 16257.00))) OR ((store_sales.ss_wholesale_cost >= 31.00) AND (store_sales.ss_wholesale_cost <= 51.00))) and (store_sales.ss_quantity <= 30) and (store_sales.ss_quantity >= 26)) +------------------filter((store_sales.ss_quantity <= 30) and (store_sales.ss_quantity >= 26) and OR[AND[(store_sales.ss_list_price >= 89.00),(store_sales.ss_list_price <= 99.00)],AND[(store_sales.ss_coupon_amt >= 15257.00),(store_sales.ss_coupon_amt <= 16257.00)],AND[(store_sales.ss_wholesale_cost >= 31.00),(store_sales.ss_wholesale_cost <= 51.00)]]) --------------------PhysicalOlapScan[store_sales] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query34.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query34.out index bb286ee190e816..489abcd4c2e33c 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query34.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query34.out @@ -21,7 +21,7 @@ PhysicalResultSink --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 --------------------------------PhysicalProject -----------------------------------filter((((date_dim.d_dom >= 1) AND (date_dim.d_dom <= 3)) OR ((date_dim.d_dom >= 25) AND (date_dim.d_dom <= 28))) and d_year IN (1998, 1999, 2000)) +----------------------------------filter(OR[AND[(date_dim.d_dom >= 1),(date_dim.d_dom <= 3)],AND[(date_dim.d_dom >= 25),(date_dim.d_dom <= 28)]] and d_year IN (1998, 1999, 2000)) ------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalProject ------------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('0-500', '1001-5000')) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query35.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query35.out index 93fdd630b3352d..dc926eb4b522f0 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query35.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query35.out @@ -9,7 +9,7 @@ PhysicalResultSink ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) +------------------filter(OR[ifnull($c$1, FALSE),ifnull($c$2, FALSE)]) --------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() ----------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() ------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF5 ss_customer_sk->[c_customer_sk] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query41.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query41.out index c9c1ffff499741..3034a77fe0897a 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query41.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query41.out @@ -18,6 +18,6 @@ PhysicalResultSink ------------------------PhysicalDistribute[DistributionSpecHash] --------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------filter((((i_color IN ('aquamarine', 'blue', 'chartreuse', 'chiffon', 'dodger', 'gainsboro', 'tan', 'violet') AND i_units IN ('Bunch', 'Dozen', 'Each', 'Ounce', 'Oz', 'Pound', 'Ton', 'Tsp')) AND (((((((item.i_category = 'Women') AND i_color IN ('aquamarine', 'gainsboro')) AND i_units IN ('Dozen', 'Ounce')) AND i_size IN ('economy', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('chiffon', 'violet')) AND i_units IN ('Pound', 'Ton')) AND i_size IN ('extra large', 'small'))) OR ((((item.i_category = 'Men') AND i_color IN ('blue', 'chartreuse')) AND i_units IN ('Each', 'Oz')) AND i_size IN ('N/A', 'large'))) OR ((((item.i_category = 'Men') AND i_color IN ('dodger', 'tan')) AND i_units IN ('Bunch', 'Tsp')) AND i_size IN ('economy', 'medium')))) OR ((i_color IN ('almond', 'blanched', 'indian', 'lime', 'peru', 'saddle', 'spring', 'tomato') AND i_units IN ('Box', 'Carton', 'Case', 'Dram', 'Gram', 'Pallet', 'Tbl', 'Unknown')) AND (((((((item.i_category = 'Women') AND i_color IN ('blanched', 'tomato')) AND i_units IN ('Case', 'Tbl')) AND i_size IN ('economy', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('almond', 'lime')) AND i_units IN ('Box', 'Dram')) AND i_size IN ('extra large', 'small'))) OR ((((item.i_category = 'Men') AND i_color IN ('peru', 'saddle')) AND i_units IN ('Gram', 'Pallet')) AND i_size IN ('N/A', 'large'))) OR ((((item.i_category = 'Men') AND i_color IN ('indian', 'spring')) AND i_units IN ('Carton', 'Unknown')) AND i_size IN ('economy', 'medium'))))) and i_category IN ('Men', 'Women') and i_size IN ('N/A', 'economy', 'extra large', 'large', 'medium', 'small')) +------------------------------filter(OR[AND[i_color IN ('aquamarine', 'blue', 'chartreuse', 'chiffon', 'dodger', 'gainsboro', 'tan', 'violet'),i_units IN ('Bunch', 'Dozen', 'Each', 'Ounce', 'Oz', 'Pound', 'Ton', 'Tsp'),OR[AND[(item.i_category = 'Women'),i_color IN ('aquamarine', 'gainsboro'),i_units IN ('Dozen', 'Ounce'),i_size IN ('economy', 'medium')],AND[(item.i_category = 'Women'),i_color IN ('chiffon', 'violet'),i_units IN ('Pound', 'Ton'),i_size IN ('extra large', 'small')],AND[(item.i_category = 'Men'),i_color IN ('blue', 'chartreuse'),i_units IN ('Each', 'Oz'),i_size IN ('N/A', 'large')],AND[(item.i_category = 'Men'),i_color IN ('dodger', 'tan'),i_units IN ('Bunch', 'Tsp'),i_size IN ('economy', 'medium')]]],AND[i_color IN ('almond', 'blanched', 'indian', 'lime', 'peru', 'saddle', 'spring', 'tomato'),i_units IN ('Box', 'Carton', 'Case', 'Dram', 'Gram', 'Pallet', 'Tbl', 'Unknown'),OR[AND[(item.i_category = 'Women'),i_color IN ('blanched', 'tomato'),i_units IN ('Case', 'Tbl'),i_size IN ('economy', 'medium')],AND[(item.i_category = 'Women'),i_color IN ('almond', 'lime'),i_units IN ('Box', 'Dram'),i_size IN ('extra large', 'small')],AND[(item.i_category = 'Men'),i_color IN ('peru', 'saddle'),i_units IN ('Gram', 'Pallet'),i_size IN ('N/A', 'large')],AND[(item.i_category = 'Men'),i_color IN ('indian', 'spring'),i_units IN ('Carton', 'Unknown'),i_size IN ('economy', 'medium')]]]] and i_category IN ('Men', 'Women') and i_size IN ('N/A', 'economy', 'extra large', 'large', 'medium', 'small')) --------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query45.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query45.out index e05c3f0537a669..6bbc52ecb1a343 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query45.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query45.out @@ -8,7 +8,7 @@ PhysicalResultSink ----------PhysicalDistribute[DistributionSpecHash] ------------hashAgg[LOCAL] --------------PhysicalProject -----------------filter((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR $c$1)) +----------------filter(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),$c$1]) ------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ws_bill_customer_sk] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query46.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query46.out index 0a6a5e279d3f80..c1ebbb551053cb 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query46.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query46.out @@ -22,7 +22,7 @@ PhysicalResultSink ----------------------------------filter(d_dow IN (0, 6) and d_year IN (1999, 2000, 2001)) ------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalProject -------------------------------filter(((household_demographics.hd_dep_count = 6) OR (household_demographics.hd_vehicle_count = 0))) +------------------------------filter(OR[(household_demographics.hd_dep_count = 6),(household_demographics.hd_vehicle_count = 0)]) --------------------------------PhysicalOlapScan[household_demographics] ------------------------PhysicalProject --------------------------filter(s_city IN ('Centerville', 'Fairview', 'Five Points', 'Liberty', 'Oak Grove')) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query47.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query47.out index 7a668b06c433e5..ecc4777c22f664 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query47.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query47.out @@ -21,7 +21,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------------------------PhysicalProject --------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 ------------------------------------PhysicalProject ---------------------------------------filter((((date_dim.d_year = 2001) OR ((date_dim.d_year = 2000) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2002) AND (date_dim.d_moy = 1))) and d_year IN (2000, 2001, 2002)) +--------------------------------------filter(OR[(date_dim.d_year = 2001),AND[(date_dim.d_year = 2000),(date_dim.d_moy = 12)],AND[(date_dim.d_year = 2002),(date_dim.d_moy = 1)]] and d_year IN (2000, 2001, 2002)) ----------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query48.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query48.out index c81868cacca7a6..1bf363b1426199 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query48.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query48.out @@ -7,23 +7,23 @@ PhysicalResultSink --------PhysicalProject ----------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] ------------PhysicalProject ---------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +--------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=(OR[AND[ca_state IN ('IA', 'MD', 'MN'),(store_sales.ss_net_profit >= 0.00),(store_sales.ss_net_profit <= 2000.00)],AND[ca_state IN ('IL', 'TX', 'VA'),(store_sales.ss_net_profit >= 150.00),(store_sales.ss_net_profit <= 3000.00)],AND[ca_state IN ('IN', 'MI', 'WI'),(store_sales.ss_net_profit >= 50.00),(store_sales.ss_net_profit <= 25000.00)]]) build RFs:RF2 ca_address_sk->[ss_addr_sk] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=((((ca_state IN ('IA', 'MD', 'MN') AND ((store_sales.ss_net_profit >= 0.00) AND (store_sales.ss_net_profit <= 2000.00))) OR (ca_state IN ('IL', 'TX', 'VA') AND ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 3000.00)))) OR (ca_state IN ('IN', 'MI', 'WI') AND ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 25000.00))))) build RFs:RF1 ca_address_sk->[ss_addr_sk] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)) otherCondition=((((((customer_demographics.cd_marital_status = 'U') AND (customer_demographics.cd_education_status = 'Primary')) AND ((store_sales.ss_sales_price >= 100.00) AND (store_sales.ss_sales_price <= 150.00))) OR (((customer_demographics.cd_marital_status = 'W') AND (customer_demographics.cd_education_status = 'College')) AND ((store_sales.ss_sales_price >= 50.00) AND (store_sales.ss_sales_price <= 100.00)))) OR (((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = '2 yr Degree')) AND ((store_sales.ss_sales_price >= 150.00) AND (store_sales.ss_sales_price <= 200.00))))) build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)) otherCondition=(OR[AND[(customer_demographics.cd_marital_status = 'U'),(customer_demographics.cd_education_status = 'Primary'),(store_sales.ss_sales_price >= 100.00),(store_sales.ss_sales_price <= 150.00)],AND[(customer_demographics.cd_marital_status = 'W'),(customer_demographics.cd_education_status = 'College'),(store_sales.ss_sales_price >= 50.00),(store_sales.ss_sales_price <= 100.00)],AND[(customer_demographics.cd_marital_status = 'D'),(customer_demographics.cd_education_status = '2 yr Degree'),(store_sales.ss_sales_price >= 150.00),(store_sales.ss_sales_price <= 200.00)]]) build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] ------------------------PhysicalProject --------------------------filter((store_sales.ss_net_profit <= 25000.00) and (store_sales.ss_net_profit >= 0.00) and (store_sales.ss_sales_price <= 200.00) and (store_sales.ss_sales_price >= 50.00)) ----------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 ------------------------PhysicalProject ---------------------------filter(((((customer_demographics.cd_marital_status = 'U') AND (customer_demographics.cd_education_status = 'Primary')) OR ((customer_demographics.cd_marital_status = 'W') AND (customer_demographics.cd_education_status = 'College'))) OR ((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = '2 yr Degree'))) and cd_education_status IN ('2 yr Degree', 'College', 'Primary') and cd_marital_status IN ('D', 'U', 'W')) +--------------------------filter(OR[AND[(customer_demographics.cd_marital_status = 'U'),(customer_demographics.cd_education_status = 'Primary')],AND[(customer_demographics.cd_marital_status = 'W'),(customer_demographics.cd_education_status = 'College')],AND[(customer_demographics.cd_marital_status = 'D'),(customer_demographics.cd_education_status = '2 yr Degree')]] and cd_education_status IN ('2 yr Degree', 'College', 'Primary') and cd_marital_status IN ('D', 'U', 'W')) ----------------------------PhysicalOlapScan[customer_demographics] --------------------PhysicalProject -----------------------filter((customer_address.ca_country = 'United States') and ca_state IN ('IA', 'IL', 'IN', 'MD', 'MI', 'MN', 'TX', 'VA', 'WI')) -------------------------PhysicalOlapScan[customer_address] +----------------------filter((date_dim.d_year = 1999)) +------------------------PhysicalOlapScan[date_dim] ----------------PhysicalProject -------------------filter((date_dim.d_year = 1999)) ---------------------PhysicalOlapScan[date_dim] +------------------filter((customer_address.ca_country = 'United States') and ca_state IN ('IA', 'IL', 'IN', 'MD', 'MI', 'MN', 'TX', 'VA', 'WI')) +--------------------PhysicalOlapScan[customer_address] ------------PhysicalProject --------------PhysicalOlapScan[store] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query49.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query49.out index e34962e0847b0d..7d0f6b1ce22d0f 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query49.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query49.out @@ -16,7 +16,7 @@ PhysicalResultSink --------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) ----------------------------------PhysicalWindow ------------------------------------PhysicalQuickSort[LOCAL_SORT] --------------------------------------PhysicalWindow @@ -48,7 +48,7 @@ PhysicalResultSink --------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) ----------------------------------PhysicalWindow ------------------------------------PhysicalQuickSort[LOCAL_SORT] --------------------------------------PhysicalWindow @@ -80,7 +80,7 @@ PhysicalResultSink --------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) ----------------------------------PhysicalWindow ------------------------------------PhysicalQuickSort[LOCAL_SORT] --------------------------------------PhysicalWindow diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query53.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query53.out index 81e6091b663bf7..04920e65ac6894 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query53.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query53.out @@ -22,7 +22,7 @@ PhysicalResultSink --------------------------------------PhysicalProject ----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 --------------------------------------PhysicalProject -----------------------------------------filter((((i_category IN ('Books', 'Children', 'Electronics') AND i_class IN ('personal', 'portable', 'reference', 'self-help')) AND i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')) OR ((i_category IN ('Men', 'Music', 'Women') AND i_class IN ('accessories', 'classical', 'fragrances', 'pants')) AND i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1'))) and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) +----------------------------------------filter(OR[AND[i_category IN ('Books', 'Children', 'Electronics'),i_class IN ('personal', 'portable', 'reference', 'self-help'),i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')],AND[i_category IN ('Men', 'Music', 'Women'),i_class IN ('accessories', 'classical', 'fragrances', 'pants'),i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')]] and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) ------------------------------------------PhysicalOlapScan[item] ----------------------------------PhysicalProject ------------------------------------filter(d_month_seq IN (1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211)) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query57.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query57.out index a0157318139d88..2cab4f33e1358d 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query57.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query57.out @@ -21,7 +21,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------------------------PhysicalProject --------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 ------------------------------------PhysicalProject ---------------------------------------filter((((date_dim.d_year = 1999) OR ((date_dim.d_year = 1998) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2000) AND (date_dim.d_moy = 1))) and d_year IN (1998, 1999, 2000)) +--------------------------------------filter(OR[(date_dim.d_year = 1999),AND[(date_dim.d_year = 1998),(date_dim.d_moy = 12)],AND[(date_dim.d_year = 2000),(date_dim.d_moy = 1)]] and d_year IN (1998, 1999, 2000)) ----------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query61.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query61.out index d034c5b43231d7..faf30604b86926 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query61.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query61.out @@ -29,7 +29,7 @@ PhysicalResultSink ------------------------------PhysicalProject --------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_promo_sk = promotion.p_promo_sk)) otherCondition=() build RFs:RF6 ss_promo_sk->[p_promo_sk] ----------------------------------PhysicalProject -------------------------------------filter((((promotion.p_channel_dmail = 'Y') OR (promotion.p_channel_email = 'Y')) OR (promotion.p_channel_tv = 'Y'))) +------------------------------------filter(OR[(promotion.p_channel_dmail = 'Y'),(promotion.p_channel_email = 'Y'),(promotion.p_channel_tv = 'Y')]) --------------------------------------PhysicalOlapScan[promotion] apply RFs: RF6 ----------------------------------PhysicalProject ------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF5 s_store_sk->[ss_store_sk] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query63.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query63.out index 2cd49bd8c40786..d4fb4990da98b8 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query63.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query63.out @@ -22,7 +22,7 @@ PhysicalResultSink --------------------------------------PhysicalProject ----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 --------------------------------------PhysicalProject -----------------------------------------filter((((i_category IN ('Books', 'Children', 'Electronics') AND i_class IN ('personal', 'portable', 'reference', 'self-help')) AND i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')) OR ((i_category IN ('Men', 'Music', 'Women') AND i_class IN ('accessories', 'classical', 'fragrances', 'pants')) AND i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1'))) and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) +----------------------------------------filter(OR[AND[i_category IN ('Books', 'Children', 'Electronics'),i_class IN ('personal', 'portable', 'reference', 'self-help'),i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')],AND[i_category IN ('Men', 'Music', 'Women'),i_class IN ('accessories', 'classical', 'fragrances', 'pants'),i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')]] and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) ------------------------------------------PhysicalOlapScan[item] ----------------------------------PhysicalProject ------------------------------------filter(d_month_seq IN (1181, 1182, 1183, 1184, 1185, 1186, 1187, 1188, 1189, 1190, 1191, 1192)) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query68.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query68.out index 8975dc78ab76e5..aa07d1b2a42d9b 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query68.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query68.out @@ -33,6 +33,6 @@ PhysicalResultSink ----------------------------------filter(s_city IN ('Five Points', 'Pleasant Hill')) ------------------------------------PhysicalOlapScan[store] ----------------------------PhysicalProject -------------------------------filter(((household_demographics.hd_dep_count = 8) OR (household_demographics.hd_vehicle_count = -1))) +------------------------------filter(OR[(household_demographics.hd_dep_count = 8),(household_demographics.hd_vehicle_count = -1)]) --------------------------------PhysicalOlapScan[household_demographics] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query7.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query7.out index 03f95005ce9cbc..49acfd90e56654 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query7.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query7.out @@ -24,7 +24,7 @@ PhysicalResultSink ----------------------------filter((date_dim.d_year = 2001)) ------------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalProject -------------------------filter(((promotion.p_channel_email = 'N') OR (promotion.p_channel_event = 'N'))) +------------------------filter(OR[(promotion.p_channel_email = 'N'),(promotion.p_channel_event = 'N')]) --------------------------PhysicalOlapScan[promotion] ------------------PhysicalProject --------------------PhysicalOlapScan[item] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query78.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query78.out index 30527883a49018..a6034ca86ac5c5 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query78.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query78.out @@ -5,7 +5,7 @@ PhysicalResultSink ----PhysicalDistribute[DistributionSpecGather] ------PhysicalTopN[LOCAL_SORT] --------PhysicalProject -----------filter(((coalesce(ws_qty, 0) > 0) OR (coalesce(cs_qty, 0) > 0))) +----------filter(OR[(coalesce(ws_qty, 0) > 0),(coalesce(cs_qty, 0) > 0)]) ------------hashJoin[LEFT_OUTER_JOIN colocated] hashCondition=((cs.cs_customer_sk = ss.ss_customer_sk) and (cs.cs_item_sk = ss.ss_item_sk) and (cs.cs_sold_year = ss.ss_sold_year)) otherCondition=() --------------PhysicalProject ----------------hashJoin[LEFT_OUTER_JOIN colocated] hashCondition=((ws.ws_customer_sk = ss.ss_customer_sk) and (ws.ws_item_sk = ss.ss_item_sk) and (ws.ws_sold_year = ss.ss_sold_year)) otherCondition=() diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query79.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query79.out index 4f8ac0ce4d7dea..974fb1da39b734 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query79.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query79.out @@ -22,7 +22,7 @@ PhysicalResultSink ----------------------------------filter((date_dim.d_dow = 1) and d_year IN (1998, 1999, 2000)) ------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalProject -------------------------------filter(((household_demographics.hd_dep_count = 5) OR (household_demographics.hd_vehicle_count > 4))) +------------------------------filter(OR[(household_demographics.hd_dep_count = 5),(household_demographics.hd_vehicle_count > 4)]) --------------------------------PhysicalOlapScan[household_demographics] ------------------------PhysicalProject --------------------------filter((store.s_number_employees <= 295) and (store.s_number_employees >= 200)) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query85.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query85.out index c69ca3b6c0ed48..332573e1b96303 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query85.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query85.out @@ -18,12 +18,12 @@ PhysicalResultSink ------------------------------filter(cd_education_status IN ('4 yr Degree', 'Advanced Degree', 'Secondary') and cd_marital_status IN ('M', 'S', 'W')) --------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF5 RF6 RF7 ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cd1.cd_demo_sk = web_returns.wr_refunded_cdemo_sk)) otherCondition=((((((cd1.cd_marital_status = 'M') AND (cd1.cd_education_status = '4 yr Degree')) AND ((web_sales.ws_sales_price >= 100.00) AND (web_sales.ws_sales_price <= 150.00))) OR (((cd1.cd_marital_status = 'S') AND (cd1.cd_education_status = 'Secondary')) AND ((web_sales.ws_sales_price >= 50.00) AND (web_sales.ws_sales_price <= 100.00)))) OR (((cd1.cd_marital_status = 'W') AND (cd1.cd_education_status = 'Advanced Degree')) AND ((web_sales.ws_sales_price >= 150.00) AND (web_sales.ws_sales_price <= 200.00))))) build RFs:RF4 wr_refunded_cdemo_sk->[cd_demo_sk] +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cd1.cd_demo_sk = web_returns.wr_refunded_cdemo_sk)) otherCondition=(OR[AND[(cd1.cd_marital_status = 'M'),(cd1.cd_education_status = '4 yr Degree'),(web_sales.ws_sales_price >= 100.00),(web_sales.ws_sales_price <= 150.00)],AND[(cd1.cd_marital_status = 'S'),(cd1.cd_education_status = 'Secondary'),(web_sales.ws_sales_price >= 50.00),(web_sales.ws_sales_price <= 100.00)],AND[(cd1.cd_marital_status = 'W'),(cd1.cd_education_status = 'Advanced Degree'),(web_sales.ws_sales_price >= 150.00),(web_sales.ws_sales_price <= 200.00)]]) build RFs:RF4 wr_refunded_cdemo_sk->[cd_demo_sk] --------------------------------PhysicalProject -----------------------------------filter(((((cd1.cd_marital_status = 'M') AND (cd1.cd_education_status = '4 yr Degree')) OR ((cd1.cd_marital_status = 'S') AND (cd1.cd_education_status = 'Secondary'))) OR ((cd1.cd_marital_status = 'W') AND (cd1.cd_education_status = 'Advanced Degree'))) and cd_education_status IN ('4 yr Degree', 'Advanced Degree', 'Secondary') and cd_marital_status IN ('M', 'S', 'W')) +----------------------------------filter(OR[AND[(cd1.cd_marital_status = 'M'),(cd1.cd_education_status = '4 yr Degree')],AND[(cd1.cd_marital_status = 'S'),(cd1.cd_education_status = 'Secondary')],AND[(cd1.cd_marital_status = 'W'),(cd1.cd_education_status = 'Advanced Degree')]] and cd_education_status IN ('4 yr Degree', 'Advanced Degree', 'Secondary') and cd_marital_status IN ('M', 'S', 'W')) ------------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF4 --------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((customer_address.ca_address_sk = web_returns.wr_refunded_addr_sk)) otherCondition=((((ca_state IN ('DE', 'FL', 'TX') AND ((web_sales.ws_net_profit >= 100.00) AND (web_sales.ws_net_profit <= 200.00))) OR (ca_state IN ('ID', 'IN', 'ND') AND ((web_sales.ws_net_profit >= 150.00) AND (web_sales.ws_net_profit <= 300.00)))) OR (ca_state IN ('IL', 'MT', 'OH') AND ((web_sales.ws_net_profit >= 50.00) AND (web_sales.ws_net_profit <= 250.00))))) build RFs:RF3 ca_address_sk->[wr_refunded_addr_sk] +----------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((customer_address.ca_address_sk = web_returns.wr_refunded_addr_sk)) otherCondition=(OR[AND[ca_state IN ('DE', 'FL', 'TX'),(web_sales.ws_net_profit >= 100.00),(web_sales.ws_net_profit <= 200.00)],AND[ca_state IN ('ID', 'IN', 'ND'),(web_sales.ws_net_profit >= 150.00),(web_sales.ws_net_profit <= 300.00)],AND[ca_state IN ('IL', 'MT', 'OH'),(web_sales.ws_net_profit >= 50.00),(web_sales.ws_net_profit <= 250.00)]]) build RFs:RF3 ca_address_sk->[wr_refunded_addr_sk] ------------------------------------PhysicalProject --------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((web_sales.ws_item_sk = web_returns.wr_item_sk) and (web_sales.ws_order_number = web_returns.wr_order_number)) otherCondition=() build RFs:RF1 ws_item_sk->[wr_item_sk];RF2 ws_order_number->[wr_order_number] ----------------------------------------PhysicalProject diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query88.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query88.out index bca8b0d069014b..09c630cef443ef 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query88.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query88.out @@ -23,7 +23,7 @@ PhysicalResultSink ------------------------------------filter((time_dim.t_hour = 8) and (time_dim.t_minute >= 30)) --------------------------------------PhysicalOlapScan[time_dim] ------------------------------PhysicalProject ---------------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +--------------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) ----------------------------------PhysicalOlapScan[household_demographics] --------------------------PhysicalProject ----------------------------filter((store.s_store_name = 'ese')) @@ -43,7 +43,7 @@ PhysicalResultSink ------------------------------------filter((time_dim.t_hour = 9) and (time_dim.t_minute < 30)) --------------------------------------PhysicalOlapScan[time_dim] ------------------------------PhysicalProject ---------------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +--------------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) ----------------------------------PhysicalOlapScan[household_demographics] --------------------------PhysicalProject ----------------------------filter((store.s_store_name = 'ese')) @@ -63,7 +63,7 @@ PhysicalResultSink ----------------------------------filter((time_dim.t_hour = 9) and (time_dim.t_minute >= 30)) ------------------------------------PhysicalOlapScan[time_dim] ----------------------------PhysicalProject -------------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +------------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) --------------------------------PhysicalOlapScan[household_demographics] ------------------------PhysicalProject --------------------------filter((store.s_store_name = 'ese')) @@ -83,7 +83,7 @@ PhysicalResultSink --------------------------------filter((time_dim.t_hour = 10) and (time_dim.t_minute < 30)) ----------------------------------PhysicalOlapScan[time_dim] --------------------------PhysicalProject -----------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +----------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) ------------------------------PhysicalOlapScan[household_demographics] ----------------------PhysicalProject ------------------------filter((store.s_store_name = 'ese')) @@ -103,7 +103,7 @@ PhysicalResultSink ------------------------------filter((time_dim.t_hour = 10) and (time_dim.t_minute >= 30)) --------------------------------PhysicalOlapScan[time_dim] ------------------------PhysicalProject ---------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +--------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) ----------------------------PhysicalOlapScan[household_demographics] --------------------PhysicalProject ----------------------filter((store.s_store_name = 'ese')) @@ -123,7 +123,7 @@ PhysicalResultSink ----------------------------filter((time_dim.t_hour = 11) and (time_dim.t_minute < 30)) ------------------------------PhysicalOlapScan[time_dim] ----------------------PhysicalProject -------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +------------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) --------------------------PhysicalOlapScan[household_demographics] ------------------PhysicalProject --------------------filter((store.s_store_name = 'ese')) @@ -143,7 +143,7 @@ PhysicalResultSink --------------------------filter((time_dim.t_hour = 11) and (time_dim.t_minute >= 30)) ----------------------------PhysicalOlapScan[time_dim] --------------------PhysicalProject -----------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +----------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) ------------------------PhysicalOlapScan[household_demographics] ----------------PhysicalProject ------------------filter((store.s_store_name = 'ese')) @@ -163,7 +163,7 @@ PhysicalResultSink ------------------------filter((time_dim.t_hour = 12) and (time_dim.t_minute < 30)) --------------------------PhysicalOlapScan[time_dim] ------------------PhysicalProject ---------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 3, 4)) +--------------------filter(OR[AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 4),(household_demographics.hd_vehicle_count <= 6)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 3, 4)) ----------------------PhysicalOlapScan[household_demographics] --------------PhysicalProject ----------------filter((store.s_store_name = 'ese')) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query89.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query89.out index 661b47d0ea980a..e4d2ae3435f174 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query89.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query89.out @@ -23,7 +23,7 @@ PhysicalResultSink ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 ----------------------------------------PhysicalProject -------------------------------------------filter(((i_category IN ('Electronics', 'Jewelry', 'Shoes') AND i_class IN ('athletic', 'portable', 'semi-precious')) OR (i_category IN ('Men', 'Music', 'Women') AND i_class IN ('accessories', 'maternity', 'rock'))) and i_category IN ('Electronics', 'Jewelry', 'Men', 'Music', 'Shoes', 'Women') and i_class IN ('accessories', 'athletic', 'maternity', 'portable', 'rock', 'semi-precious')) +------------------------------------------filter(OR[AND[i_category IN ('Electronics', 'Jewelry', 'Shoes'),i_class IN ('athletic', 'portable', 'semi-precious')],AND[i_category IN ('Men', 'Music', 'Women'),i_class IN ('accessories', 'maternity', 'rock')]] and i_category IN ('Electronics', 'Jewelry', 'Men', 'Music', 'Shoes', 'Women') and i_class IN ('accessories', 'athletic', 'maternity', 'portable', 'rock', 'semi-precious')) --------------------------------------------PhysicalOlapScan[item] ------------------------------------PhysicalProject --------------------------------------filter((date_dim.d_year = 1999)) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query91.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query91.out index 6f3970cbb29d50..6af6c014d0c298 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query91.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query91.out @@ -28,7 +28,7 @@ PhysicalResultSink ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 RF1 ----------------------------------------PhysicalProject -------------------------------------------filter((((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = 'Unknown')) OR ((customer_demographics.cd_marital_status = 'W') AND (customer_demographics.cd_education_status = 'Advanced Degree'))) and cd_education_status IN ('Advanced Degree', 'Unknown') and cd_marital_status IN ('M', 'W')) +------------------------------------------filter(OR[AND[(customer_demographics.cd_marital_status = 'M'),(customer_demographics.cd_education_status = 'Unknown')],AND[(customer_demographics.cd_marital_status = 'W'),(customer_demographics.cd_education_status = 'Advanced Degree')]] and cd_education_status IN ('Advanced Degree', 'Unknown') and cd_marital_status IN ('M', 'W')) --------------------------------------------PhysicalOlapScan[customer_demographics] ------------------------------------PhysicalProject --------------------------------------filter((hd_buy_potential like '1001-5000%')) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/bs_downgrade_shape/query13.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/bs_downgrade_shape/query13.out index 461f54e0e1767c..8138a8e20ed7e5 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/bs_downgrade_shape/query13.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/bs_downgrade_shape/query13.out @@ -7,21 +7,21 @@ PhysicalResultSink --------PhysicalProject ----------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF4 s_store_sk->[ss_store_sk] ------------PhysicalProject ---------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=((((ca_state IN ('IL', 'TN', 'TX') AND ((store_sales.ss_net_profit >= 100.00) AND (store_sales.ss_net_profit <= 200.00))) OR (ca_state IN ('ID', 'OH', 'WY') AND ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 300.00)))) OR (ca_state IN ('IA', 'MS', 'SC') AND ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 250.00))))) build RFs:RF3 ss_addr_sk->[ca_address_sk] +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=(OR[AND[ca_state IN ('IL', 'TN', 'TX'),(store_sales.ss_net_profit >= 100.00),(store_sales.ss_net_profit <= 200.00)],AND[ca_state IN ('ID', 'OH', 'WY'),(store_sales.ss_net_profit >= 150.00),(store_sales.ss_net_profit <= 300.00)],AND[ca_state IN ('IA', 'MS', 'SC'),(store_sales.ss_net_profit >= 50.00),(store_sales.ss_net_profit <= 250.00)]]) build RFs:RF3 ss_addr_sk->[ca_address_sk] ----------------PhysicalProject ------------------filter((customer_address.ca_country = 'United States') and ca_state IN ('IA', 'ID', 'IL', 'MS', 'OH', 'SC', 'TN', 'TX', 'WY')) --------------------PhysicalOlapScan[customer_address] apply RFs: RF3 ----------------PhysicalProject ------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=((((((household_demographics.hd_dep_count = 1) AND cd_marital_status IN ('D', 'W')) AND cd_education_status IN ('2 yr Degree', 'Primary')) AND ((((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = 'Primary')) AND ((store_sales.ss_sales_price >= 50.00) AND (store_sales.ss_sales_price <= 100.00))) OR (((customer_demographics.cd_marital_status = 'W') AND (customer_demographics.cd_education_status = '2 yr Degree')) AND ((store_sales.ss_sales_price >= 150.00) AND (store_sales.ss_sales_price <= 200.00))))) OR ((((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = 'College')) AND ((store_sales.ss_sales_price >= 100.00) AND (store_sales.ss_sales_price <= 150.00))) AND (household_demographics.hd_dep_count = 3)))) build RFs:RF1 hd_demo_sk->[ss_hdemo_sk] +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=(OR[AND[(household_demographics.hd_dep_count = 1),cd_marital_status IN ('D', 'W'),cd_education_status IN ('2 yr Degree', 'Primary'),OR[AND[(customer_demographics.cd_marital_status = 'D'),(customer_demographics.cd_education_status = 'Primary'),(store_sales.ss_sales_price >= 50.00),(store_sales.ss_sales_price <= 100.00)],AND[(customer_demographics.cd_marital_status = 'W'),(customer_demographics.cd_education_status = '2 yr Degree'),(store_sales.ss_sales_price >= 150.00),(store_sales.ss_sales_price <= 200.00)]]],AND[(customer_demographics.cd_marital_status = 'M'),(customer_demographics.cd_education_status = 'College'),(store_sales.ss_sales_price >= 100.00),(store_sales.ss_sales_price <= 150.00),(household_demographics.hd_dep_count = 3)]]) build RFs:RF1 hd_demo_sk->[ss_hdemo_sk] ------------------------PhysicalProject --------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] ----------------------------PhysicalProject ------------------------------filter((store_sales.ss_net_profit <= 300.00) and (store_sales.ss_net_profit >= 50.00) and (store_sales.ss_sales_price <= 200.00) and (store_sales.ss_sales_price >= 50.00)) --------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF4 ----------------------------PhysicalProject -------------------------------filter(((((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = 'College')) OR ((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = 'Primary'))) OR ((customer_demographics.cd_marital_status = 'W') AND (customer_demographics.cd_education_status = '2 yr Degree'))) and cd_education_status IN ('2 yr Degree', 'College', 'Primary') and cd_marital_status IN ('D', 'M', 'W')) +------------------------------filter(OR[AND[(customer_demographics.cd_marital_status = 'M'),(customer_demographics.cd_education_status = 'College')],AND[(customer_demographics.cd_marital_status = 'D'),(customer_demographics.cd_education_status = 'Primary')],AND[(customer_demographics.cd_marital_status = 'W'),(customer_demographics.cd_education_status = '2 yr Degree')]] and cd_education_status IN ('2 yr Degree', 'College', 'Primary') and cd_marital_status IN ('D', 'M', 'W')) --------------------------------PhysicalOlapScan[customer_demographics] ------------------------PhysicalProject --------------------------filter(hd_dep_count IN (1, 3)) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/bs_downgrade_shape/query45.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/bs_downgrade_shape/query45.out index 6ac3b85090aceb..72acf1dc6b71ff 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/bs_downgrade_shape/query45.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/bs_downgrade_shape/query45.out @@ -8,7 +8,7 @@ PhysicalResultSink ----------PhysicalDistribute[DistributionSpecHash] ------------hashAgg[LOCAL] --------------PhysicalProject -----------------filter((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR $c$1)) +----------------filter(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),$c$1]) ------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ws_bill_customer_sk] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/bs_downgrade_shape/query61.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/bs_downgrade_shape/query61.out index 7453f3ef1a7080..e768a09ec1494d 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/bs_downgrade_shape/query61.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/bs_downgrade_shape/query61.out @@ -33,7 +33,7 @@ PhysicalResultSink ------------------------------------filter((item.i_category = 'Home')) --------------------------------------PhysicalOlapScan[item] ------------------------------PhysicalProject ---------------------------------filter((((promotion.p_channel_dmail = 'Y') OR (promotion.p_channel_email = 'Y')) OR (promotion.p_channel_tv = 'Y'))) +--------------------------------filter(OR[(promotion.p_channel_dmail = 'Y'),(promotion.p_channel_email = 'Y'),(promotion.p_channel_tv = 'Y')]) ----------------------------------PhysicalOlapScan[promotion] --------------------------PhysicalProject ----------------------------filter((store.s_gmt_offset = -7.00)) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/bs_downgrade_shape/query68.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/bs_downgrade_shape/query68.out index f4503a600d6272..2f4fbe401f1315 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/bs_downgrade_shape/query68.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/bs_downgrade_shape/query68.out @@ -33,6 +33,6 @@ PhysicalResultSink ----------------------------------filter(s_city IN ('Fairview', 'Midway')) ------------------------------------PhysicalOlapScan[store] ----------------------------PhysicalProject -------------------------------filter(((household_demographics.hd_dep_count = 3) OR (household_demographics.hd_vehicle_count = 4))) +------------------------------filter(OR[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count = 4)]) --------------------------------PhysicalOlapScan[household_demographics] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/bs_downgrade_shape/query91.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/bs_downgrade_shape/query91.out index 624685e9dab961..9d3c77acb23ca8 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/bs_downgrade_shape/query91.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/bs_downgrade_shape/query91.out @@ -28,7 +28,7 @@ PhysicalResultSink ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 RF1 ----------------------------------------PhysicalProject -------------------------------------------filter((((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = 'Unknown')) OR ((customer_demographics.cd_marital_status = 'W') AND (customer_demographics.cd_education_status = 'Advanced Degree'))) and cd_education_status IN ('Advanced Degree', 'Unknown') and cd_marital_status IN ('M', 'W')) +------------------------------------------filter(OR[AND[(customer_demographics.cd_marital_status = 'M'),(customer_demographics.cd_education_status = 'Unknown')],AND[(customer_demographics.cd_marital_status = 'W'),(customer_demographics.cd_education_status = 'Advanced Degree')]] and cd_education_status IN ('Advanced Degree', 'Unknown') and cd_marital_status IN ('M', 'W')) --------------------------------------------PhysicalOlapScan[customer_demographics] ------------------------------------PhysicalProject --------------------------------------filter((hd_buy_potential like 'Unknown%')) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/eliminate_empty/query10_empty.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/eliminate_empty/query10_empty.out index f68d46db52fbb8..78fd7c847c29ed 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/eliminate_empty/query10_empty.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/eliminate_empty/query10_empty.out @@ -9,7 +9,7 @@ PhysicalResultSink ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) +------------------filter(OR[ifnull($c$1, FALSE),ifnull($c$2, FALSE)]) --------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() ----------------------PhysicalProject ------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query10.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query10.out index f68d46db52fbb8..78fd7c847c29ed 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query10.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query10.out @@ -9,7 +9,7 @@ PhysicalResultSink ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) +------------------filter(OR[ifnull($c$1, FALSE),ifnull($c$2, FALSE)]) --------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() ----------------------PhysicalProject ------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query13.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query13.out index 461f54e0e1767c..8138a8e20ed7e5 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query13.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query13.out @@ -7,21 +7,21 @@ PhysicalResultSink --------PhysicalProject ----------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF4 s_store_sk->[ss_store_sk] ------------PhysicalProject ---------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=((((ca_state IN ('IL', 'TN', 'TX') AND ((store_sales.ss_net_profit >= 100.00) AND (store_sales.ss_net_profit <= 200.00))) OR (ca_state IN ('ID', 'OH', 'WY') AND ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 300.00)))) OR (ca_state IN ('IA', 'MS', 'SC') AND ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 250.00))))) build RFs:RF3 ss_addr_sk->[ca_address_sk] +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=(OR[AND[ca_state IN ('IL', 'TN', 'TX'),(store_sales.ss_net_profit >= 100.00),(store_sales.ss_net_profit <= 200.00)],AND[ca_state IN ('ID', 'OH', 'WY'),(store_sales.ss_net_profit >= 150.00),(store_sales.ss_net_profit <= 300.00)],AND[ca_state IN ('IA', 'MS', 'SC'),(store_sales.ss_net_profit >= 50.00),(store_sales.ss_net_profit <= 250.00)]]) build RFs:RF3 ss_addr_sk->[ca_address_sk] ----------------PhysicalProject ------------------filter((customer_address.ca_country = 'United States') and ca_state IN ('IA', 'ID', 'IL', 'MS', 'OH', 'SC', 'TN', 'TX', 'WY')) --------------------PhysicalOlapScan[customer_address] apply RFs: RF3 ----------------PhysicalProject ------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=((((((household_demographics.hd_dep_count = 1) AND cd_marital_status IN ('D', 'W')) AND cd_education_status IN ('2 yr Degree', 'Primary')) AND ((((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = 'Primary')) AND ((store_sales.ss_sales_price >= 50.00) AND (store_sales.ss_sales_price <= 100.00))) OR (((customer_demographics.cd_marital_status = 'W') AND (customer_demographics.cd_education_status = '2 yr Degree')) AND ((store_sales.ss_sales_price >= 150.00) AND (store_sales.ss_sales_price <= 200.00))))) OR ((((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = 'College')) AND ((store_sales.ss_sales_price >= 100.00) AND (store_sales.ss_sales_price <= 150.00))) AND (household_demographics.hd_dep_count = 3)))) build RFs:RF1 hd_demo_sk->[ss_hdemo_sk] +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=(OR[AND[(household_demographics.hd_dep_count = 1),cd_marital_status IN ('D', 'W'),cd_education_status IN ('2 yr Degree', 'Primary'),OR[AND[(customer_demographics.cd_marital_status = 'D'),(customer_demographics.cd_education_status = 'Primary'),(store_sales.ss_sales_price >= 50.00),(store_sales.ss_sales_price <= 100.00)],AND[(customer_demographics.cd_marital_status = 'W'),(customer_demographics.cd_education_status = '2 yr Degree'),(store_sales.ss_sales_price >= 150.00),(store_sales.ss_sales_price <= 200.00)]]],AND[(customer_demographics.cd_marital_status = 'M'),(customer_demographics.cd_education_status = 'College'),(store_sales.ss_sales_price >= 100.00),(store_sales.ss_sales_price <= 150.00),(household_demographics.hd_dep_count = 3)]]) build RFs:RF1 hd_demo_sk->[ss_hdemo_sk] ------------------------PhysicalProject --------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] ----------------------------PhysicalProject ------------------------------filter((store_sales.ss_net_profit <= 300.00) and (store_sales.ss_net_profit >= 50.00) and (store_sales.ss_sales_price <= 200.00) and (store_sales.ss_sales_price >= 50.00)) --------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF4 ----------------------------PhysicalProject -------------------------------filter(((((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = 'College')) OR ((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = 'Primary'))) OR ((customer_demographics.cd_marital_status = 'W') AND (customer_demographics.cd_education_status = '2 yr Degree'))) and cd_education_status IN ('2 yr Degree', 'College', 'Primary') and cd_marital_status IN ('D', 'M', 'W')) +------------------------------filter(OR[AND[(customer_demographics.cd_marital_status = 'M'),(customer_demographics.cd_education_status = 'College')],AND[(customer_demographics.cd_marital_status = 'D'),(customer_demographics.cd_education_status = 'Primary')],AND[(customer_demographics.cd_marital_status = 'W'),(customer_demographics.cd_education_status = '2 yr Degree')]] and cd_education_status IN ('2 yr Degree', 'College', 'Primary') and cd_marital_status IN ('D', 'M', 'W')) --------------------------------PhysicalOlapScan[customer_demographics] ------------------------PhysicalProject --------------------------filter(hd_dep_count IN (1, 3)) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query15.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query15.out index 6f522756f18098..fe0fd80c2f8b05 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query15.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query15.out @@ -8,7 +8,7 @@ PhysicalResultSink ----------PhysicalDistribute[DistributionSpecHash] ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=(((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR ca_state IN ('CA', 'GA', 'WA')) OR (catalog_sales.cs_sales_price > 500.00))) build RFs:RF2 c_customer_sk->[cs_bill_customer_sk] +----------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),ca_state IN ('CA', 'GA', 'WA'),(catalog_sales.cs_sales_price > 500.00)]) build RFs:RF2 c_customer_sk->[cs_bill_customer_sk] ------------------PhysicalProject --------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] ----------------------PhysicalProject diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query26.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query26.out index dc5aee071856c1..383242890f9dd4 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query26.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query26.out @@ -24,7 +24,7 @@ PhysicalResultSink ----------------------------filter((date_dim.d_year = 2002)) ------------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalProject -------------------------filter(((promotion.p_channel_email = 'N') OR (promotion.p_channel_event = 'N'))) +------------------------filter(OR[(promotion.p_channel_email = 'N'),(promotion.p_channel_event = 'N')]) --------------------------PhysicalOlapScan[promotion] ------------------PhysicalProject --------------------PhysicalOlapScan[item] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query28.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query28.out index 2fdcc86b103f71..36ec7305d96abb 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query28.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query28.out @@ -17,41 +17,41 @@ PhysicalResultSink ----------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------hashAgg[LOCAL] --------------------------------PhysicalProject -----------------------------------filter(((((store_sales.ss_list_price >= 107.00) AND (store_sales.ss_list_price <= 117.00)) OR ((store_sales.ss_coupon_amt >= 1319.00) AND (store_sales.ss_coupon_amt <= 2319.00))) OR ((store_sales.ss_wholesale_cost >= 60.00) AND (store_sales.ss_wholesale_cost <= 80.00))) and (store_sales.ss_quantity <= 5) and (store_sales.ss_quantity >= 0)) +----------------------------------filter((store_sales.ss_quantity <= 5) and (store_sales.ss_quantity >= 0) and OR[AND[(store_sales.ss_list_price >= 107.00),(store_sales.ss_list_price <= 117.00)],AND[(store_sales.ss_coupon_amt >= 1319.00),(store_sales.ss_coupon_amt <= 2319.00)],AND[(store_sales.ss_wholesale_cost >= 60.00),(store_sales.ss_wholesale_cost <= 80.00)]]) ------------------------------------PhysicalOlapScan[store_sales] ------------------------PhysicalLimit[LOCAL] --------------------------hashAgg[GLOBAL] ----------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------hashAgg[LOCAL] --------------------------------PhysicalProject -----------------------------------filter(((((store_sales.ss_list_price >= 23.00) AND (store_sales.ss_list_price <= 33.00)) OR ((store_sales.ss_coupon_amt >= 825.00) AND (store_sales.ss_coupon_amt <= 1825.00))) OR ((store_sales.ss_wholesale_cost >= 43.00) AND (store_sales.ss_wholesale_cost <= 63.00))) and (store_sales.ss_quantity <= 10) and (store_sales.ss_quantity >= 6)) +----------------------------------filter((store_sales.ss_quantity <= 10) and (store_sales.ss_quantity >= 6) and OR[AND[(store_sales.ss_list_price >= 23.00),(store_sales.ss_list_price <= 33.00)],AND[(store_sales.ss_coupon_amt >= 825.00),(store_sales.ss_coupon_amt <= 1825.00)],AND[(store_sales.ss_wholesale_cost >= 43.00),(store_sales.ss_wholesale_cost <= 63.00)]]) ------------------------------------PhysicalOlapScan[store_sales] --------------------PhysicalLimit[LOCAL] ----------------------hashAgg[GLOBAL] ------------------------PhysicalDistribute[DistributionSpecGather] --------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------filter(((((store_sales.ss_list_price >= 74.00) AND (store_sales.ss_list_price <= 84.00)) OR ((store_sales.ss_coupon_amt >= 4381.00) AND (store_sales.ss_coupon_amt <= 5381.00))) OR ((store_sales.ss_wholesale_cost >= 57.00) AND (store_sales.ss_wholesale_cost <= 77.00))) and (store_sales.ss_quantity <= 15) and (store_sales.ss_quantity >= 11)) +------------------------------filter((store_sales.ss_quantity <= 15) and (store_sales.ss_quantity >= 11) and OR[AND[(store_sales.ss_list_price >= 74.00),(store_sales.ss_list_price <= 84.00)],AND[(store_sales.ss_coupon_amt >= 4381.00),(store_sales.ss_coupon_amt <= 5381.00)],AND[(store_sales.ss_wholesale_cost >= 57.00),(store_sales.ss_wholesale_cost <= 77.00)]]) --------------------------------PhysicalOlapScan[store_sales] ----------------PhysicalLimit[LOCAL] ------------------hashAgg[GLOBAL] --------------------PhysicalDistribute[DistributionSpecGather] ----------------------hashAgg[LOCAL] ------------------------PhysicalProject ---------------------------filter(((((store_sales.ss_list_price >= 89.00) AND (store_sales.ss_list_price <= 99.00)) OR ((store_sales.ss_coupon_amt >= 3117.00) AND (store_sales.ss_coupon_amt <= 4117.00))) OR ((store_sales.ss_wholesale_cost >= 68.00) AND (store_sales.ss_wholesale_cost <= 88.00))) and (store_sales.ss_quantity <= 20) and (store_sales.ss_quantity >= 16)) +--------------------------filter((store_sales.ss_quantity <= 20) and (store_sales.ss_quantity >= 16) and OR[AND[(store_sales.ss_list_price >= 89.00),(store_sales.ss_list_price <= 99.00)],AND[(store_sales.ss_coupon_amt >= 3117.00),(store_sales.ss_coupon_amt <= 4117.00)],AND[(store_sales.ss_wholesale_cost >= 68.00),(store_sales.ss_wholesale_cost <= 88.00)]]) ----------------------------PhysicalOlapScan[store_sales] ------------PhysicalLimit[LOCAL] --------------hashAgg[GLOBAL] ----------------PhysicalDistribute[DistributionSpecGather] ------------------hashAgg[LOCAL] --------------------PhysicalProject -----------------------filter(((((store_sales.ss_list_price >= 58.00) AND (store_sales.ss_list_price <= 68.00)) OR ((store_sales.ss_coupon_amt >= 9402.00) AND (store_sales.ss_coupon_amt <= 10402.00))) OR ((store_sales.ss_wholesale_cost >= 38.00) AND (store_sales.ss_wholesale_cost <= 58.00))) and (store_sales.ss_quantity <= 25) and (store_sales.ss_quantity >= 21)) +----------------------filter((store_sales.ss_quantity <= 25) and (store_sales.ss_quantity >= 21) and OR[AND[(store_sales.ss_list_price >= 58.00),(store_sales.ss_list_price <= 68.00)],AND[(store_sales.ss_coupon_amt >= 9402.00),(store_sales.ss_coupon_amt <= 10402.00)],AND[(store_sales.ss_wholesale_cost >= 38.00),(store_sales.ss_wholesale_cost <= 58.00)]]) ------------------------PhysicalOlapScan[store_sales] --------PhysicalLimit[LOCAL] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecGather] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------filter(((((store_sales.ss_list_price >= 64.00) AND (store_sales.ss_list_price <= 74.00)) OR ((store_sales.ss_coupon_amt >= 5792.00) AND (store_sales.ss_coupon_amt <= 6792.00))) OR ((store_sales.ss_wholesale_cost >= 73.00) AND (store_sales.ss_wholesale_cost <= 93.00))) and (store_sales.ss_quantity <= 30) and (store_sales.ss_quantity >= 26)) +------------------filter((store_sales.ss_quantity <= 30) and (store_sales.ss_quantity >= 26) and OR[AND[(store_sales.ss_list_price >= 64.00),(store_sales.ss_list_price <= 74.00)],AND[(store_sales.ss_coupon_amt >= 5792.00),(store_sales.ss_coupon_amt <= 6792.00)],AND[(store_sales.ss_wholesale_cost >= 73.00),(store_sales.ss_wholesale_cost <= 93.00)]]) --------------------PhysicalOlapScan[store_sales] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query34.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query34.out index 135b461a144762..ad28c2d46104d0 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query34.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query34.out @@ -24,7 +24,7 @@ PhysicalResultSink ----------------------------------filter((store.s_county = 'Williamson County')) ------------------------------------PhysicalOlapScan[store] ----------------------------PhysicalProject -------------------------------filter((((date_dim.d_dom >= 1) AND (date_dim.d_dom <= 3)) OR ((date_dim.d_dom >= 25) AND (date_dim.d_dom <= 28))) and d_year IN (2000, 2001, 2002)) +------------------------------filter(OR[AND[(date_dim.d_dom >= 1),(date_dim.d_dom <= 3)],AND[(date_dim.d_dom >= 25),(date_dim.d_dom <= 28)]] and d_year IN (2000, 2001, 2002)) --------------------------------PhysicalOlapScan[date_dim] ------------------------PhysicalProject --------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('0-500', '1001-5000')) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query35.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query35.out index a7a8d54bd3e135..9012700621a358 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query35.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query35.out @@ -9,7 +9,7 @@ PhysicalResultSink ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) +------------------filter(OR[ifnull($c$1, FALSE),ifnull($c$2, FALSE)]) --------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() ----------------------PhysicalProject ------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF5 cd_demo_sk->[c_current_cdemo_sk] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query41.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query41.out index ea5acccd883550..0bba60d4cdac39 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query41.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query41.out @@ -18,6 +18,6 @@ PhysicalResultSink ------------------------PhysicalDistribute[DistributionSpecHash] --------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------filter((((i_color IN ('forest', 'lime', 'maroon', 'navy', 'powder', 'sky', 'slate', 'smoke') AND i_units IN ('Bunch', 'Case', 'Dozen', 'Gross', 'Lb', 'Ounce', 'Pallet', 'Pound')) AND (((((((item.i_category = 'Women') AND i_color IN ('forest', 'lime')) AND i_units IN ('Pallet', 'Pound')) AND i_size IN ('economy', 'small')) OR ((((item.i_category = 'Women') AND i_color IN ('navy', 'slate')) AND i_units IN ('Bunch', 'Gross')) AND i_size IN ('extra large', 'petite'))) OR ((((item.i_category = 'Men') AND i_color IN ('powder', 'sky')) AND i_units IN ('Dozen', 'Lb')) AND i_size IN ('N/A', 'large'))) OR ((((item.i_category = 'Men') AND i_color IN ('maroon', 'smoke')) AND i_units IN ('Case', 'Ounce')) AND i_size IN ('economy', 'small')))) OR ((i_color IN ('aquamarine', 'dark', 'firebrick', 'frosted', 'papaya', 'peach', 'plum', 'sienna') AND i_units IN ('Box', 'Bundle', 'Carton', 'Cup', 'Dram', 'Each', 'Tbl', 'Ton')) AND (((((((item.i_category = 'Women') AND i_color IN ('aquamarine', 'dark')) AND i_units IN ('Tbl', 'Ton')) AND i_size IN ('economy', 'small')) OR ((((item.i_category = 'Women') AND i_color IN ('frosted', 'plum')) AND i_units IN ('Box', 'Dram')) AND i_size IN ('extra large', 'petite'))) OR ((((item.i_category = 'Men') AND i_color IN ('papaya', 'peach')) AND i_units IN ('Bundle', 'Carton')) AND i_size IN ('N/A', 'large'))) OR ((((item.i_category = 'Men') AND i_color IN ('firebrick', 'sienna')) AND i_units IN ('Cup', 'Each')) AND i_size IN ('economy', 'small'))))) and i_category IN ('Men', 'Women') and i_size IN ('N/A', 'economy', 'extra large', 'large', 'petite', 'small')) +------------------------------filter(OR[AND[i_color IN ('forest', 'lime', 'maroon', 'navy', 'powder', 'sky', 'slate', 'smoke'),i_units IN ('Bunch', 'Case', 'Dozen', 'Gross', 'Lb', 'Ounce', 'Pallet', 'Pound'),OR[AND[(item.i_category = 'Women'),i_color IN ('forest', 'lime'),i_units IN ('Pallet', 'Pound'),i_size IN ('economy', 'small')],AND[(item.i_category = 'Women'),i_color IN ('navy', 'slate'),i_units IN ('Bunch', 'Gross'),i_size IN ('extra large', 'petite')],AND[(item.i_category = 'Men'),i_color IN ('powder', 'sky'),i_units IN ('Dozen', 'Lb'),i_size IN ('N/A', 'large')],AND[(item.i_category = 'Men'),i_color IN ('maroon', 'smoke'),i_units IN ('Case', 'Ounce'),i_size IN ('economy', 'small')]]],AND[i_color IN ('aquamarine', 'dark', 'firebrick', 'frosted', 'papaya', 'peach', 'plum', 'sienna'),i_units IN ('Box', 'Bundle', 'Carton', 'Cup', 'Dram', 'Each', 'Tbl', 'Ton'),OR[AND[(item.i_category = 'Women'),i_color IN ('aquamarine', 'dark'),i_units IN ('Tbl', 'Ton'),i_size IN ('economy', 'small')],AND[(item.i_category = 'Women'),i_color IN ('frosted', 'plum'),i_units IN ('Box', 'Dram'),i_size IN ('extra large', 'petite')],AND[(item.i_category = 'Men'),i_color IN ('papaya', 'peach'),i_units IN ('Bundle', 'Carton'),i_size IN ('N/A', 'large')],AND[(item.i_category = 'Men'),i_color IN ('firebrick', 'sienna'),i_units IN ('Cup', 'Each'),i_size IN ('economy', 'small')]]]] and i_category IN ('Men', 'Women') and i_size IN ('N/A', 'economy', 'extra large', 'large', 'petite', 'small')) --------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query45.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query45.out index 6ac3b85090aceb..72acf1dc6b71ff 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query45.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query45.out @@ -8,7 +8,7 @@ PhysicalResultSink ----------PhysicalDistribute[DistributionSpecHash] ------------hashAgg[LOCAL] --------------PhysicalProject -----------------filter((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR $c$1)) +----------------filter(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),$c$1]) ------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ws_bill_customer_sk] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query46.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query46.out index b80a9affecc0d6..e7aa0e014dcdd9 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query46.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query46.out @@ -29,7 +29,7 @@ PhysicalResultSink ----------------------------------filter(d_dow IN (0, 6) and d_year IN (2000, 2001, 2002)) ------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalProject -------------------------------filter(((household_demographics.hd_dep_count = 8) OR (household_demographics.hd_vehicle_count = 0))) +------------------------------filter(OR[(household_demographics.hd_dep_count = 8),(household_demographics.hd_vehicle_count = 0)]) --------------------------------PhysicalOlapScan[household_demographics] ------------------------PhysicalProject --------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query47.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query47.out index d51d48c5ab5677..0e9f713243773a 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query47.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query47.out @@ -21,7 +21,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------------------------PhysicalProject --------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 ------------------------------------PhysicalProject ---------------------------------------filter((((date_dim.d_year = 2000) OR ((date_dim.d_year = 1999) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2001) AND (date_dim.d_moy = 1))) and d_year IN (1999, 2000, 2001)) +--------------------------------------filter(OR[(date_dim.d_year = 2000),AND[(date_dim.d_year = 1999),(date_dim.d_moy = 12)],AND[(date_dim.d_year = 2001),(date_dim.d_moy = 1)]] and d_year IN (1999, 2000, 2001)) ----------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query48.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query48.out index 3fcfef0d8f68ba..6ed75b1e21eca8 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query48.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query48.out @@ -9,14 +9,14 @@ PhysicalResultSink ------------PhysicalProject --------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=((((ca_state IN ('ND', 'NY', 'SD') AND ((store_sales.ss_net_profit >= 0.00) AND (store_sales.ss_net_profit <= 2000.00))) OR (ca_state IN ('GA', 'KS', 'MD') AND ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 3000.00)))) OR (ca_state IN ('CO', 'MN', 'NC') AND ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 25000.00))))) build RFs:RF1 ca_address_sk->[ss_addr_sk] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=(OR[AND[ca_state IN ('ND', 'NY', 'SD'),(store_sales.ss_net_profit >= 0.00),(store_sales.ss_net_profit <= 2000.00)],AND[ca_state IN ('GA', 'KS', 'MD'),(store_sales.ss_net_profit >= 150.00),(store_sales.ss_net_profit <= 3000.00)],AND[ca_state IN ('CO', 'MN', 'NC'),(store_sales.ss_net_profit >= 50.00),(store_sales.ss_net_profit <= 25000.00)]]) build RFs:RF1 ca_address_sk->[ss_addr_sk] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)) otherCondition=((((((customer_demographics.cd_marital_status = 'S') AND (customer_demographics.cd_education_status = 'Secondary')) AND ((store_sales.ss_sales_price >= 100.00) AND (store_sales.ss_sales_price <= 150.00))) OR (((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = '2 yr Degree')) AND ((store_sales.ss_sales_price >= 50.00) AND (store_sales.ss_sales_price <= 100.00)))) OR (((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = 'Advanced Degree')) AND ((store_sales.ss_sales_price >= 150.00) AND (store_sales.ss_sales_price <= 200.00))))) build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)) otherCondition=(OR[AND[(customer_demographics.cd_marital_status = 'S'),(customer_demographics.cd_education_status = 'Secondary'),(store_sales.ss_sales_price >= 100.00),(store_sales.ss_sales_price <= 150.00)],AND[(customer_demographics.cd_marital_status = 'M'),(customer_demographics.cd_education_status = '2 yr Degree'),(store_sales.ss_sales_price >= 50.00),(store_sales.ss_sales_price <= 100.00)],AND[(customer_demographics.cd_marital_status = 'D'),(customer_demographics.cd_education_status = 'Advanced Degree'),(store_sales.ss_sales_price >= 150.00),(store_sales.ss_sales_price <= 200.00)]]) build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] ------------------------PhysicalProject --------------------------filter((store_sales.ss_net_profit <= 25000.00) and (store_sales.ss_net_profit >= 0.00) and (store_sales.ss_sales_price <= 200.00) and (store_sales.ss_sales_price >= 50.00)) ----------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 ------------------------PhysicalProject ---------------------------filter(((((customer_demographics.cd_marital_status = 'S') AND (customer_demographics.cd_education_status = 'Secondary')) OR ((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = '2 yr Degree'))) OR ((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = 'Advanced Degree'))) and cd_education_status IN ('2 yr Degree', 'Advanced Degree', 'Secondary') and cd_marital_status IN ('D', 'M', 'S')) +--------------------------filter(OR[AND[(customer_demographics.cd_marital_status = 'S'),(customer_demographics.cd_education_status = 'Secondary')],AND[(customer_demographics.cd_marital_status = 'M'),(customer_demographics.cd_education_status = '2 yr Degree')],AND[(customer_demographics.cd_marital_status = 'D'),(customer_demographics.cd_education_status = 'Advanced Degree')]] and cd_education_status IN ('2 yr Degree', 'Advanced Degree', 'Secondary') and cd_marital_status IN ('D', 'M', 'S')) ----------------------------PhysicalOlapScan[customer_demographics] --------------------PhysicalProject ----------------------filter((customer_address.ca_country = 'United States') and ca_state IN ('CO', 'GA', 'KS', 'MD', 'MN', 'NC', 'ND', 'NY', 'SD')) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query49.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query49.out index 889d5069ff9751..8b807baf5409e4 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query49.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query49.out @@ -16,7 +16,7 @@ PhysicalResultSink --------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) ----------------------------------PhysicalWindow ------------------------------------PhysicalQuickSort[LOCAL_SORT] --------------------------------------PhysicalWindow @@ -48,7 +48,7 @@ PhysicalResultSink --------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) ----------------------------------PhysicalWindow ------------------------------------PhysicalQuickSort[LOCAL_SORT] --------------------------------------PhysicalWindow @@ -80,7 +80,7 @@ PhysicalResultSink --------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) ----------------------------------PhysicalWindow ------------------------------------PhysicalQuickSort[LOCAL_SORT] --------------------------------------PhysicalWindow diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query53.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query53.out index 07d64a35b071ca..d2467a65e93e09 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query53.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query53.out @@ -22,7 +22,7 @@ PhysicalResultSink --------------------------------------PhysicalProject ----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 --------------------------------------PhysicalProject -----------------------------------------filter((((i_category IN ('Books', 'Children', 'Electronics') AND i_class IN ('personal', 'portable', 'reference', 'self-help')) AND i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')) OR ((i_category IN ('Men', 'Music', 'Women') AND i_class IN ('accessories', 'classical', 'fragrances', 'pants')) AND i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1'))) and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) +----------------------------------------filter(OR[AND[i_category IN ('Books', 'Children', 'Electronics'),i_class IN ('personal', 'portable', 'reference', 'self-help'),i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')],AND[i_category IN ('Men', 'Music', 'Women'),i_class IN ('accessories', 'classical', 'fragrances', 'pants'),i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')]] and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) ------------------------------------------PhysicalOlapScan[item] ----------------------------------PhysicalProject ------------------------------------filter(d_month_seq IN (1186, 1187, 1188, 1189, 1190, 1191, 1192, 1193, 1194, 1195, 1196, 1197)) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query57.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query57.out index c1b802ebfc4498..2e8174812f69ea 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query57.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query57.out @@ -21,7 +21,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------------------------PhysicalProject --------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 ------------------------------------PhysicalProject ---------------------------------------filter((((date_dim.d_year = 2001) OR ((date_dim.d_year = 2000) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2002) AND (date_dim.d_moy = 1))) and d_year IN (2000, 2001, 2002)) +--------------------------------------filter(OR[(date_dim.d_year = 2001),AND[(date_dim.d_year = 2000),(date_dim.d_moy = 12)],AND[(date_dim.d_year = 2002),(date_dim.d_moy = 1)]] and d_year IN (2000, 2001, 2002)) ----------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[call_center] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query61.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query61.out index 7453f3ef1a7080..e768a09ec1494d 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query61.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query61.out @@ -33,7 +33,7 @@ PhysicalResultSink ------------------------------------filter((item.i_category = 'Home')) --------------------------------------PhysicalOlapScan[item] ------------------------------PhysicalProject ---------------------------------filter((((promotion.p_channel_dmail = 'Y') OR (promotion.p_channel_email = 'Y')) OR (promotion.p_channel_tv = 'Y'))) +--------------------------------filter(OR[(promotion.p_channel_dmail = 'Y'),(promotion.p_channel_email = 'Y'),(promotion.p_channel_tv = 'Y')]) ----------------------------------PhysicalOlapScan[promotion] --------------------------PhysicalProject ----------------------------filter((store.s_gmt_offset = -7.00)) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query63.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query63.out index d1807a29db2f0d..bbbb80bc4b68e0 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query63.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query63.out @@ -22,7 +22,7 @@ PhysicalResultSink --------------------------------------PhysicalProject ----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 --------------------------------------PhysicalProject -----------------------------------------filter((((i_category IN ('Books', 'Children', 'Electronics') AND i_class IN ('personal', 'portable', 'reference', 'self-help')) AND i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')) OR ((i_category IN ('Men', 'Music', 'Women') AND i_class IN ('accessories', 'classical', 'fragrances', 'pants')) AND i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1'))) and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) +----------------------------------------filter(OR[AND[i_category IN ('Books', 'Children', 'Electronics'),i_class IN ('personal', 'portable', 'reference', 'self-help'),i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')],AND[i_category IN ('Men', 'Music', 'Women'),i_class IN ('accessories', 'classical', 'fragrances', 'pants'),i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')]] and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) ------------------------------------------PhysicalOlapScan[item] ----------------------------------PhysicalProject ------------------------------------filter(d_month_seq IN (1222, 1223, 1224, 1225, 1226, 1227, 1228, 1229, 1230, 1231, 1232, 1233)) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query68.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query68.out index f4503a600d6272..2f4fbe401f1315 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query68.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query68.out @@ -33,6 +33,6 @@ PhysicalResultSink ----------------------------------filter(s_city IN ('Fairview', 'Midway')) ------------------------------------PhysicalOlapScan[store] ----------------------------PhysicalProject -------------------------------filter(((household_demographics.hd_dep_count = 3) OR (household_demographics.hd_vehicle_count = 4))) +------------------------------filter(OR[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count = 4)]) --------------------------------PhysicalOlapScan[household_demographics] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query7.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query7.out index aadb31a78e012f..2d63af9e61b19e 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query7.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query7.out @@ -24,7 +24,7 @@ PhysicalResultSink ----------------------------filter((date_dim.d_year = 2001)) ------------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalProject -------------------------filter(((promotion.p_channel_email = 'N') OR (promotion.p_channel_event = 'N'))) +------------------------filter(OR[(promotion.p_channel_email = 'N'),(promotion.p_channel_event = 'N')]) --------------------------PhysicalOlapScan[promotion] ------------------PhysicalProject --------------------PhysicalOlapScan[item] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query78.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query78.out index 3b2d980a6ac2c7..0663ee2198a5fe 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query78.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query78.out @@ -5,7 +5,7 @@ PhysicalResultSink ----PhysicalDistribute[DistributionSpecGather] ------PhysicalTopN[LOCAL_SORT] --------PhysicalProject -----------filter(((coalesce(ws_qty, 0) > 0) OR (coalesce(cs_qty, 0) > 0))) +----------filter(OR[(coalesce(ws_qty, 0) > 0),(coalesce(cs_qty, 0) > 0)]) ------------hashJoin[LEFT_OUTER_JOIN colocated] hashCondition=((cs.cs_customer_sk = ss.ss_customer_sk) and (cs.cs_item_sk = ss.ss_item_sk) and (cs.cs_sold_year = ss.ss_sold_year)) otherCondition=() --------------PhysicalProject ----------------hashJoin[LEFT_OUTER_JOIN colocated] hashCondition=((ws.ws_customer_sk = ss.ss_customer_sk) and (ws.ws_item_sk = ss.ss_item_sk) and (ws.ws_sold_year = ss.ss_sold_year)) otherCondition=() diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query79.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query79.out index b8b18b2657766b..e3d8f3af326d91 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query79.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query79.out @@ -22,7 +22,7 @@ PhysicalResultSink ----------------------------------filter((date_dim.d_dow = 1) and d_year IN (2000, 2001, 2002)) ------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalProject -------------------------------filter(((household_demographics.hd_dep_count = 7) OR (household_demographics.hd_vehicle_count > -1))) +------------------------------filter(OR[(household_demographics.hd_dep_count = 7),(household_demographics.hd_vehicle_count > -1)]) --------------------------------PhysicalOlapScan[household_demographics] ------------------------PhysicalProject --------------------------filter((store.s_number_employees <= 295) and (store.s_number_employees >= 200)) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query85.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query85.out index 1781cd24d5b9f5..c734d7945f3a56 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query85.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query85.out @@ -18,12 +18,12 @@ PhysicalResultSink ------------------------PhysicalProject --------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_web_page_sk = web_page.wp_web_page_sk)) otherCondition=() build RFs:RF5 wp_web_page_sk->[ws_web_page_sk] ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((customer_address.ca_address_sk = web_returns.wr_refunded_addr_sk)) otherCondition=((((ca_state IN ('IA', 'NC', 'TX') AND ((web_sales.ws_net_profit >= 100.00) AND (web_sales.ws_net_profit <= 200.00))) OR (ca_state IN ('GA', 'WI', 'WV') AND ((web_sales.ws_net_profit >= 150.00) AND (web_sales.ws_net_profit <= 300.00)))) OR (ca_state IN ('KY', 'OK', 'VA') AND ((web_sales.ws_net_profit >= 50.00) AND (web_sales.ws_net_profit <= 250.00))))) build RFs:RF4 wr_refunded_addr_sk->[ca_address_sk] +------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((customer_address.ca_address_sk = web_returns.wr_refunded_addr_sk)) otherCondition=(OR[AND[ca_state IN ('IA', 'NC', 'TX'),(web_sales.ws_net_profit >= 100.00),(web_sales.ws_net_profit <= 200.00)],AND[ca_state IN ('GA', 'WI', 'WV'),(web_sales.ws_net_profit >= 150.00),(web_sales.ws_net_profit <= 300.00)],AND[ca_state IN ('KY', 'OK', 'VA'),(web_sales.ws_net_profit >= 50.00),(web_sales.ws_net_profit <= 250.00)]]) build RFs:RF4 wr_refunded_addr_sk->[ca_address_sk] --------------------------------PhysicalProject ----------------------------------filter((customer_address.ca_country = 'United States') and ca_state IN ('GA', 'IA', 'KY', 'NC', 'OK', 'TX', 'VA', 'WI', 'WV')) ------------------------------------PhysicalOlapScan[customer_address] apply RFs: RF4 --------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cd1.cd_demo_sk = web_returns.wr_refunded_cdemo_sk)) otherCondition=((((((cd1.cd_marital_status = 'D') AND (cd1.cd_education_status = 'Primary')) AND ((web_sales.ws_sales_price >= 100.00) AND (web_sales.ws_sales_price <= 150.00))) OR (((cd1.cd_marital_status = 'S') AND (cd1.cd_education_status = 'College')) AND ((web_sales.ws_sales_price >= 50.00) AND (web_sales.ws_sales_price <= 100.00)))) OR (((cd1.cd_marital_status = 'U') AND (cd1.cd_education_status = 'Advanced Degree')) AND ((web_sales.ws_sales_price >= 150.00) AND (web_sales.ws_sales_price <= 200.00))))) build RFs:RF3 cd_demo_sk->[wr_refunded_cdemo_sk] +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cd1.cd_demo_sk = web_returns.wr_refunded_cdemo_sk)) otherCondition=(OR[AND[(cd1.cd_marital_status = 'D'),(cd1.cd_education_status = 'Primary'),(web_sales.ws_sales_price >= 100.00),(web_sales.ws_sales_price <= 150.00)],AND[(cd1.cd_marital_status = 'S'),(cd1.cd_education_status = 'College'),(web_sales.ws_sales_price >= 50.00),(web_sales.ws_sales_price <= 100.00)],AND[(cd1.cd_marital_status = 'U'),(cd1.cd_education_status = 'Advanced Degree'),(web_sales.ws_sales_price >= 150.00),(web_sales.ws_sales_price <= 200.00)]]) build RFs:RF3 cd_demo_sk->[wr_refunded_cdemo_sk] ------------------------------------PhysicalProject --------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((web_sales.ws_item_sk = web_returns.wr_item_sk) and (web_sales.ws_order_number = web_returns.wr_order_number)) otherCondition=() build RFs:RF1 ws_item_sk->[wr_item_sk];RF2 ws_order_number->[wr_order_number] ----------------------------------------PhysicalProject @@ -37,7 +37,7 @@ PhysicalResultSink ----------------------------------------------filter((date_dim.d_year = 1998)) ------------------------------------------------PhysicalOlapScan[date_dim] ------------------------------------PhysicalProject ---------------------------------------filter(((((cd1.cd_marital_status = 'D') AND (cd1.cd_education_status = 'Primary')) OR ((cd1.cd_marital_status = 'S') AND (cd1.cd_education_status = 'College'))) OR ((cd1.cd_marital_status = 'U') AND (cd1.cd_education_status = 'Advanced Degree'))) and cd_education_status IN ('Advanced Degree', 'College', 'Primary') and cd_marital_status IN ('D', 'S', 'U')) +--------------------------------------filter(OR[AND[(cd1.cd_marital_status = 'D'),(cd1.cd_education_status = 'Primary')],AND[(cd1.cd_marital_status = 'S'),(cd1.cd_education_status = 'College')],AND[(cd1.cd_marital_status = 'U'),(cd1.cd_education_status = 'Advanced Degree')]] and cd_education_status IN ('Advanced Degree', 'College', 'Primary') and cd_marital_status IN ('D', 'S', 'U')) ----------------------------------------PhysicalOlapScan[customer_demographics] ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[web_page] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query88.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query88.out index 3c8f0335f1f415..d016dca48db832 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query88.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query88.out @@ -23,7 +23,7 @@ PhysicalResultSink ------------------------------------filter((time_dim.t_hour = 8) and (time_dim.t_minute >= 30)) --------------------------------------PhysicalOlapScan[time_dim] ------------------------------PhysicalProject ---------------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3)) +--------------------------------filter(OR[AND[(household_demographics.hd_dep_count = 0),(household_demographics.hd_vehicle_count <= 2)],AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 0, 3)) ----------------------------------PhysicalOlapScan[household_demographics] --------------------------PhysicalProject ----------------------------filter((store.s_store_name = 'ese')) @@ -43,7 +43,7 @@ PhysicalResultSink ------------------------------------filter((time_dim.t_hour = 9) and (time_dim.t_minute < 30)) --------------------------------------PhysicalOlapScan[time_dim] ------------------------------PhysicalProject ---------------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3)) +--------------------------------filter(OR[AND[(household_demographics.hd_dep_count = 0),(household_demographics.hd_vehicle_count <= 2)],AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 0, 3)) ----------------------------------PhysicalOlapScan[household_demographics] --------------------------PhysicalProject ----------------------------filter((store.s_store_name = 'ese')) @@ -63,7 +63,7 @@ PhysicalResultSink ----------------------------------filter((time_dim.t_hour = 9) and (time_dim.t_minute >= 30)) ------------------------------------PhysicalOlapScan[time_dim] ----------------------------PhysicalProject -------------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3)) +------------------------------filter(OR[AND[(household_demographics.hd_dep_count = 0),(household_demographics.hd_vehicle_count <= 2)],AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 0, 3)) --------------------------------PhysicalOlapScan[household_demographics] ------------------------PhysicalProject --------------------------filter((store.s_store_name = 'ese')) @@ -83,7 +83,7 @@ PhysicalResultSink --------------------------------filter((time_dim.t_hour = 10) and (time_dim.t_minute < 30)) ----------------------------------PhysicalOlapScan[time_dim] --------------------------PhysicalProject -----------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3)) +----------------------------filter(OR[AND[(household_demographics.hd_dep_count = 0),(household_demographics.hd_vehicle_count <= 2)],AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 0, 3)) ------------------------------PhysicalOlapScan[household_demographics] ----------------------PhysicalProject ------------------------filter((store.s_store_name = 'ese')) @@ -103,7 +103,7 @@ PhysicalResultSink ------------------------------filter((time_dim.t_hour = 10) and (time_dim.t_minute >= 30)) --------------------------------PhysicalOlapScan[time_dim] ------------------------PhysicalProject ---------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3)) +--------------------------filter(OR[AND[(household_demographics.hd_dep_count = 0),(household_demographics.hd_vehicle_count <= 2)],AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 0, 3)) ----------------------------PhysicalOlapScan[household_demographics] --------------------PhysicalProject ----------------------filter((store.s_store_name = 'ese')) @@ -123,7 +123,7 @@ PhysicalResultSink ----------------------------filter((time_dim.t_hour = 11) and (time_dim.t_minute < 30)) ------------------------------PhysicalOlapScan[time_dim] ----------------------PhysicalProject -------------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3)) +------------------------filter(OR[AND[(household_demographics.hd_dep_count = 0),(household_demographics.hd_vehicle_count <= 2)],AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 0, 3)) --------------------------PhysicalOlapScan[household_demographics] ------------------PhysicalProject --------------------filter((store.s_store_name = 'ese')) @@ -143,7 +143,7 @@ PhysicalResultSink --------------------------filter((time_dim.t_hour = 11) and (time_dim.t_minute >= 30)) ----------------------------PhysicalOlapScan[time_dim] --------------------PhysicalProject -----------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3)) +----------------------filter(OR[AND[(household_demographics.hd_dep_count = 0),(household_demographics.hd_vehicle_count <= 2)],AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 0, 3)) ------------------------PhysicalOlapScan[household_demographics] ----------------PhysicalProject ------------------filter((store.s_store_name = 'ese')) @@ -163,7 +163,7 @@ PhysicalResultSink ------------------------filter((time_dim.t_hour = 12) and (time_dim.t_minute < 30)) --------------------------PhysicalOlapScan[time_dim] ------------------PhysicalProject ---------------------filter(((((household_demographics.hd_dep_count = 0) AND (household_demographics.hd_vehicle_count <= 2)) OR ((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5))) and hd_dep_count IN (-1, 0, 3)) +--------------------filter(OR[AND[(household_demographics.hd_dep_count = 0),(household_demographics.hd_vehicle_count <= 2)],AND[(household_demographics.hd_dep_count = -1),(household_demographics.hd_vehicle_count <= 1)],AND[(household_demographics.hd_dep_count = 3),(household_demographics.hd_vehicle_count <= 5)]] and hd_dep_count IN (-1, 0, 3)) ----------------------PhysicalOlapScan[household_demographics] --------------PhysicalProject ----------------filter((store.s_store_name = 'ese')) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query89.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query89.out index 786de940dad5d8..b8751687a0ff29 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query89.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query89.out @@ -23,7 +23,7 @@ PhysicalResultSink ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 ----------------------------------------PhysicalProject -------------------------------------------filter(((i_category IN ('Books', 'Children', 'Electronics') AND i_class IN ('audio', 'history', 'school-uniforms')) OR (i_category IN ('Men', 'Shoes', 'Sports') AND i_class IN ('pants', 'tennis', 'womens'))) and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Shoes', 'Sports') and i_class IN ('audio', 'history', 'pants', 'school-uniforms', 'tennis', 'womens')) +------------------------------------------filter(OR[AND[i_category IN ('Books', 'Children', 'Electronics'),i_class IN ('audio', 'history', 'school-uniforms')],AND[i_category IN ('Men', 'Shoes', 'Sports'),i_class IN ('pants', 'tennis', 'womens')]] and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Shoes', 'Sports') and i_class IN ('audio', 'history', 'pants', 'school-uniforms', 'tennis', 'womens')) --------------------------------------------PhysicalOlapScan[item] ------------------------------------PhysicalProject --------------------------------------filter((date_dim.d_year = 2001)) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query91.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query91.out index 624685e9dab961..9d3c77acb23ca8 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query91.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query91.out @@ -28,7 +28,7 @@ PhysicalResultSink ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 RF1 ----------------------------------------PhysicalProject -------------------------------------------filter((((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = 'Unknown')) OR ((customer_demographics.cd_marital_status = 'W') AND (customer_demographics.cd_education_status = 'Advanced Degree'))) and cd_education_status IN ('Advanced Degree', 'Unknown') and cd_marital_status IN ('M', 'W')) +------------------------------------------filter(OR[AND[(customer_demographics.cd_marital_status = 'M'),(customer_demographics.cd_education_status = 'Unknown')],AND[(customer_demographics.cd_marital_status = 'W'),(customer_demographics.cd_education_status = 'Advanced Degree')]] and cd_education_status IN ('Advanced Degree', 'Unknown') and cd_marital_status IN ('M', 'W')) --------------------------------------------PhysicalOlapScan[customer_demographics] ------------------------------------PhysicalProject --------------------------------------filter((hd_buy_potential like 'Unknown%')) diff --git a/regression-test/data/new_shapes_p0/tpch_sf1000/nostats_rf_prune/q19.out b/regression-test/data/new_shapes_p0/tpch_sf1000/nostats_rf_prune/q19.out index bf86e68a7f9837..ad82594176a219 100644 --- a/regression-test/data/new_shapes_p0/tpch_sf1000/nostats_rf_prune/q19.out +++ b/regression-test/data/new_shapes_p0/tpch_sf1000/nostats_rf_prune/q19.out @@ -5,11 +5,11 @@ PhysicalResultSink ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] --------PhysicalProject -----------hashJoin[INNER_JOIN broadcast] hashCondition=((part.p_partkey = lineitem.l_partkey)) otherCondition=(((((((part.p_brand = 'Brand#12') AND p_container IN ('SM BOX', 'SM CASE', 'SM PACK', 'SM PKG')) AND ((lineitem.l_quantity >= 1.00) AND (lineitem.l_quantity <= 11.00))) AND (part.p_size <= 5)) OR ((((part.p_brand = 'Brand#23') AND p_container IN ('MED BAG', 'MED BOX', 'MED PACK', 'MED PKG')) AND ((lineitem.l_quantity >= 10.00) AND (lineitem.l_quantity <= 20.00))) AND (part.p_size <= 10))) OR ((((part.p_brand = 'Brand#34') AND p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG')) AND ((lineitem.l_quantity >= 20.00) AND (lineitem.l_quantity <= 30.00))) AND (part.p_size <= 15)))) build RFs:RF0 p_partkey->[l_partkey] +----------hashJoin[INNER_JOIN broadcast] hashCondition=((part.p_partkey = lineitem.l_partkey)) otherCondition=(OR[AND[(part.p_brand = 'Brand#12'),p_container IN ('SM BOX', 'SM CASE', 'SM PACK', 'SM PKG'),(lineitem.l_quantity >= 1.00),(lineitem.l_quantity <= 11.00),(part.p_size <= 5)],AND[(part.p_brand = 'Brand#23'),p_container IN ('MED BAG', 'MED BOX', 'MED PACK', 'MED PKG'),(lineitem.l_quantity >= 10.00),(lineitem.l_quantity <= 20.00),(part.p_size <= 10)],AND[(part.p_brand = 'Brand#34'),p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG'),(lineitem.l_quantity >= 20.00),(lineitem.l_quantity <= 30.00),(part.p_size <= 15)]]) build RFs:RF0 p_partkey->[l_partkey] ------------PhysicalProject --------------filter((lineitem.l_quantity <= 30.00) and (lineitem.l_quantity >= 1.00) and (lineitem.l_shipinstruct = 'DELIVER IN PERSON') and l_shipmode IN ('AIR REG', 'AIR')) ----------------PhysicalOlapScan[lineitem] apply RFs: RF0 ------------PhysicalProject ---------------filter((((((part.p_brand = 'Brand#12') AND p_container IN ('SM BOX', 'SM CASE', 'SM PACK', 'SM PKG')) AND (part.p_size <= 5)) OR (((part.p_brand = 'Brand#23') AND p_container IN ('MED BAG', 'MED BOX', 'MED PACK', 'MED PKG')) AND (part.p_size <= 10))) OR (((part.p_brand = 'Brand#34') AND p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG')) AND (part.p_size <= 15))) and (part.p_size >= 1) and p_brand IN ('Brand#12', 'Brand#23', 'Brand#34') and p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG', 'MED BAG', 'MED BOX', 'MED PACK', 'MED PKG', 'SM BOX', 'SM CASE', 'SM PACK', 'SM PKG')) +--------------filter((part.p_size >= 1) and OR[AND[(part.p_brand = 'Brand#12'),p_container IN ('SM BOX', 'SM CASE', 'SM PACK', 'SM PKG'),(part.p_size <= 5)],AND[(part.p_brand = 'Brand#23'),p_container IN ('MED BAG', 'MED BOX', 'MED PACK', 'MED PKG'),(part.p_size <= 10)],AND[(part.p_brand = 'Brand#34'),p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG'),(part.p_size <= 15)]] and p_brand IN ('Brand#12', 'Brand#23', 'Brand#34') and p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG', 'MED BAG', 'MED BOX', 'MED PACK', 'MED PKG', 'SM BOX', 'SM CASE', 'SM PACK', 'SM PKG')) ----------------PhysicalOlapScan[part] diff --git a/regression-test/data/new_shapes_p0/tpch_sf1000/nostats_rf_prune/q7.out b/regression-test/data/new_shapes_p0/tpch_sf1000/nostats_rf_prune/q7.out index f434b57fc9f511..b98149f8668a1c 100644 --- a/regression-test/data/new_shapes_p0/tpch_sf1000/nostats_rf_prune/q7.out +++ b/regression-test/data/new_shapes_p0/tpch_sf1000/nostats_rf_prune/q7.out @@ -24,7 +24,7 @@ PhysicalResultSink --------------------------------PhysicalOlapScan[supplier] apply RFs: RF4 ----------------------PhysicalProject ------------------------PhysicalOlapScan[customer] apply RFs: RF3 -------------------NestedLoopJoin[INNER_JOIN](((n1.n_name = 'FRANCE') AND (n2.n_name = 'GERMANY')) OR ((n1.n_name = 'GERMANY') AND (n2.n_name = 'FRANCE'))) +------------------NestedLoopJoin[INNER_JOIN]OR[AND[(n1.n_name = 'FRANCE'),(n2.n_name = 'GERMANY')],AND[(n1.n_name = 'GERMANY'),(n2.n_name = 'FRANCE')]] --------------------PhysicalProject ----------------------filter(n_name IN ('FRANCE', 'GERMANY')) ------------------------PhysicalOlapScan[nation] diff --git a/regression-test/data/new_shapes_p0/tpch_sf1000/rf_prune/q19.out b/regression-test/data/new_shapes_p0/tpch_sf1000/rf_prune/q19.out index bf86e68a7f9837..ad82594176a219 100644 --- a/regression-test/data/new_shapes_p0/tpch_sf1000/rf_prune/q19.out +++ b/regression-test/data/new_shapes_p0/tpch_sf1000/rf_prune/q19.out @@ -5,11 +5,11 @@ PhysicalResultSink ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] --------PhysicalProject -----------hashJoin[INNER_JOIN broadcast] hashCondition=((part.p_partkey = lineitem.l_partkey)) otherCondition=(((((((part.p_brand = 'Brand#12') AND p_container IN ('SM BOX', 'SM CASE', 'SM PACK', 'SM PKG')) AND ((lineitem.l_quantity >= 1.00) AND (lineitem.l_quantity <= 11.00))) AND (part.p_size <= 5)) OR ((((part.p_brand = 'Brand#23') AND p_container IN ('MED BAG', 'MED BOX', 'MED PACK', 'MED PKG')) AND ((lineitem.l_quantity >= 10.00) AND (lineitem.l_quantity <= 20.00))) AND (part.p_size <= 10))) OR ((((part.p_brand = 'Brand#34') AND p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG')) AND ((lineitem.l_quantity >= 20.00) AND (lineitem.l_quantity <= 30.00))) AND (part.p_size <= 15)))) build RFs:RF0 p_partkey->[l_partkey] +----------hashJoin[INNER_JOIN broadcast] hashCondition=((part.p_partkey = lineitem.l_partkey)) otherCondition=(OR[AND[(part.p_brand = 'Brand#12'),p_container IN ('SM BOX', 'SM CASE', 'SM PACK', 'SM PKG'),(lineitem.l_quantity >= 1.00),(lineitem.l_quantity <= 11.00),(part.p_size <= 5)],AND[(part.p_brand = 'Brand#23'),p_container IN ('MED BAG', 'MED BOX', 'MED PACK', 'MED PKG'),(lineitem.l_quantity >= 10.00),(lineitem.l_quantity <= 20.00),(part.p_size <= 10)],AND[(part.p_brand = 'Brand#34'),p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG'),(lineitem.l_quantity >= 20.00),(lineitem.l_quantity <= 30.00),(part.p_size <= 15)]]) build RFs:RF0 p_partkey->[l_partkey] ------------PhysicalProject --------------filter((lineitem.l_quantity <= 30.00) and (lineitem.l_quantity >= 1.00) and (lineitem.l_shipinstruct = 'DELIVER IN PERSON') and l_shipmode IN ('AIR REG', 'AIR')) ----------------PhysicalOlapScan[lineitem] apply RFs: RF0 ------------PhysicalProject ---------------filter((((((part.p_brand = 'Brand#12') AND p_container IN ('SM BOX', 'SM CASE', 'SM PACK', 'SM PKG')) AND (part.p_size <= 5)) OR (((part.p_brand = 'Brand#23') AND p_container IN ('MED BAG', 'MED BOX', 'MED PACK', 'MED PKG')) AND (part.p_size <= 10))) OR (((part.p_brand = 'Brand#34') AND p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG')) AND (part.p_size <= 15))) and (part.p_size >= 1) and p_brand IN ('Brand#12', 'Brand#23', 'Brand#34') and p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG', 'MED BAG', 'MED BOX', 'MED PACK', 'MED PKG', 'SM BOX', 'SM CASE', 'SM PACK', 'SM PKG')) +--------------filter((part.p_size >= 1) and OR[AND[(part.p_brand = 'Brand#12'),p_container IN ('SM BOX', 'SM CASE', 'SM PACK', 'SM PKG'),(part.p_size <= 5)],AND[(part.p_brand = 'Brand#23'),p_container IN ('MED BAG', 'MED BOX', 'MED PACK', 'MED PKG'),(part.p_size <= 10)],AND[(part.p_brand = 'Brand#34'),p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG'),(part.p_size <= 15)]] and p_brand IN ('Brand#12', 'Brand#23', 'Brand#34') and p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG', 'MED BAG', 'MED BOX', 'MED PACK', 'MED PKG', 'SM BOX', 'SM CASE', 'SM PACK', 'SM PKG')) ----------------PhysicalOlapScan[part] diff --git a/regression-test/data/new_shapes_p0/tpch_sf1000/rf_prune/q7.out b/regression-test/data/new_shapes_p0/tpch_sf1000/rf_prune/q7.out index f1424cc0f730c4..957b17a7402749 100644 --- a/regression-test/data/new_shapes_p0/tpch_sf1000/rf_prune/q7.out +++ b/regression-test/data/new_shapes_p0/tpch_sf1000/rf_prune/q7.out @@ -8,7 +8,7 @@ PhysicalResultSink ----------PhysicalDistribute[DistributionSpecHash] ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_custkey = orders.o_custkey)) otherCondition=((((n1.n_name = 'FRANCE') AND (n2.n_name = 'GERMANY')) OR ((n1.n_name = 'GERMANY') AND (n2.n_name = 'FRANCE')))) build RFs:RF4 c_custkey->[o_custkey] +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_custkey = orders.o_custkey)) otherCondition=(OR[AND[(n1.n_name = 'FRANCE'),(n2.n_name = 'GERMANY')],AND[(n1.n_name = 'GERMANY'),(n2.n_name = 'FRANCE')]]) build RFs:RF4 c_custkey->[o_custkey] ------------------PhysicalProject --------------------hashJoin[INNER_JOIN colocated] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=() build RFs:RF3 l_orderkey->[o_orderkey] ----------------------PhysicalProject diff --git a/regression-test/data/new_shapes_p0/tpch_sf1000/shape/q19.out b/regression-test/data/new_shapes_p0/tpch_sf1000/shape/q19.out index bf86e68a7f9837..ad82594176a219 100644 --- a/regression-test/data/new_shapes_p0/tpch_sf1000/shape/q19.out +++ b/regression-test/data/new_shapes_p0/tpch_sf1000/shape/q19.out @@ -5,11 +5,11 @@ PhysicalResultSink ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] --------PhysicalProject -----------hashJoin[INNER_JOIN broadcast] hashCondition=((part.p_partkey = lineitem.l_partkey)) otherCondition=(((((((part.p_brand = 'Brand#12') AND p_container IN ('SM BOX', 'SM CASE', 'SM PACK', 'SM PKG')) AND ((lineitem.l_quantity >= 1.00) AND (lineitem.l_quantity <= 11.00))) AND (part.p_size <= 5)) OR ((((part.p_brand = 'Brand#23') AND p_container IN ('MED BAG', 'MED BOX', 'MED PACK', 'MED PKG')) AND ((lineitem.l_quantity >= 10.00) AND (lineitem.l_quantity <= 20.00))) AND (part.p_size <= 10))) OR ((((part.p_brand = 'Brand#34') AND p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG')) AND ((lineitem.l_quantity >= 20.00) AND (lineitem.l_quantity <= 30.00))) AND (part.p_size <= 15)))) build RFs:RF0 p_partkey->[l_partkey] +----------hashJoin[INNER_JOIN broadcast] hashCondition=((part.p_partkey = lineitem.l_partkey)) otherCondition=(OR[AND[(part.p_brand = 'Brand#12'),p_container IN ('SM BOX', 'SM CASE', 'SM PACK', 'SM PKG'),(lineitem.l_quantity >= 1.00),(lineitem.l_quantity <= 11.00),(part.p_size <= 5)],AND[(part.p_brand = 'Brand#23'),p_container IN ('MED BAG', 'MED BOX', 'MED PACK', 'MED PKG'),(lineitem.l_quantity >= 10.00),(lineitem.l_quantity <= 20.00),(part.p_size <= 10)],AND[(part.p_brand = 'Brand#34'),p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG'),(lineitem.l_quantity >= 20.00),(lineitem.l_quantity <= 30.00),(part.p_size <= 15)]]) build RFs:RF0 p_partkey->[l_partkey] ------------PhysicalProject --------------filter((lineitem.l_quantity <= 30.00) and (lineitem.l_quantity >= 1.00) and (lineitem.l_shipinstruct = 'DELIVER IN PERSON') and l_shipmode IN ('AIR REG', 'AIR')) ----------------PhysicalOlapScan[lineitem] apply RFs: RF0 ------------PhysicalProject ---------------filter((((((part.p_brand = 'Brand#12') AND p_container IN ('SM BOX', 'SM CASE', 'SM PACK', 'SM PKG')) AND (part.p_size <= 5)) OR (((part.p_brand = 'Brand#23') AND p_container IN ('MED BAG', 'MED BOX', 'MED PACK', 'MED PKG')) AND (part.p_size <= 10))) OR (((part.p_brand = 'Brand#34') AND p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG')) AND (part.p_size <= 15))) and (part.p_size >= 1) and p_brand IN ('Brand#12', 'Brand#23', 'Brand#34') and p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG', 'MED BAG', 'MED BOX', 'MED PACK', 'MED PKG', 'SM BOX', 'SM CASE', 'SM PACK', 'SM PKG')) +--------------filter((part.p_size >= 1) and OR[AND[(part.p_brand = 'Brand#12'),p_container IN ('SM BOX', 'SM CASE', 'SM PACK', 'SM PKG'),(part.p_size <= 5)],AND[(part.p_brand = 'Brand#23'),p_container IN ('MED BAG', 'MED BOX', 'MED PACK', 'MED PKG'),(part.p_size <= 10)],AND[(part.p_brand = 'Brand#34'),p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG'),(part.p_size <= 15)]] and p_brand IN ('Brand#12', 'Brand#23', 'Brand#34') and p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG', 'MED BAG', 'MED BOX', 'MED PACK', 'MED PKG', 'SM BOX', 'SM CASE', 'SM PACK', 'SM PKG')) ----------------PhysicalOlapScan[part] diff --git a/regression-test/data/new_shapes_p0/tpch_sf1000/shape/q7.out b/regression-test/data/new_shapes_p0/tpch_sf1000/shape/q7.out index f1424cc0f730c4..957b17a7402749 100644 --- a/regression-test/data/new_shapes_p0/tpch_sf1000/shape/q7.out +++ b/regression-test/data/new_shapes_p0/tpch_sf1000/shape/q7.out @@ -8,7 +8,7 @@ PhysicalResultSink ----------PhysicalDistribute[DistributionSpecHash] ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_custkey = orders.o_custkey)) otherCondition=((((n1.n_name = 'FRANCE') AND (n2.n_name = 'GERMANY')) OR ((n1.n_name = 'GERMANY') AND (n2.n_name = 'FRANCE')))) build RFs:RF4 c_custkey->[o_custkey] +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_custkey = orders.o_custkey)) otherCondition=(OR[AND[(n1.n_name = 'FRANCE'),(n2.n_name = 'GERMANY')],AND[(n1.n_name = 'GERMANY'),(n2.n_name = 'FRANCE')]]) build RFs:RF4 c_custkey->[o_custkey] ------------------PhysicalProject --------------------hashJoin[INNER_JOIN colocated] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=() build RFs:RF3 l_orderkey->[o_orderkey] ----------------------PhysicalProject diff --git a/regression-test/data/new_shapes_p0/tpch_sf1000/shape_no_stats/q19.out b/regression-test/data/new_shapes_p0/tpch_sf1000/shape_no_stats/q19.out index bf86e68a7f9837..ad82594176a219 100644 --- a/regression-test/data/new_shapes_p0/tpch_sf1000/shape_no_stats/q19.out +++ b/regression-test/data/new_shapes_p0/tpch_sf1000/shape_no_stats/q19.out @@ -5,11 +5,11 @@ PhysicalResultSink ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] --------PhysicalProject -----------hashJoin[INNER_JOIN broadcast] hashCondition=((part.p_partkey = lineitem.l_partkey)) otherCondition=(((((((part.p_brand = 'Brand#12') AND p_container IN ('SM BOX', 'SM CASE', 'SM PACK', 'SM PKG')) AND ((lineitem.l_quantity >= 1.00) AND (lineitem.l_quantity <= 11.00))) AND (part.p_size <= 5)) OR ((((part.p_brand = 'Brand#23') AND p_container IN ('MED BAG', 'MED BOX', 'MED PACK', 'MED PKG')) AND ((lineitem.l_quantity >= 10.00) AND (lineitem.l_quantity <= 20.00))) AND (part.p_size <= 10))) OR ((((part.p_brand = 'Brand#34') AND p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG')) AND ((lineitem.l_quantity >= 20.00) AND (lineitem.l_quantity <= 30.00))) AND (part.p_size <= 15)))) build RFs:RF0 p_partkey->[l_partkey] +----------hashJoin[INNER_JOIN broadcast] hashCondition=((part.p_partkey = lineitem.l_partkey)) otherCondition=(OR[AND[(part.p_brand = 'Brand#12'),p_container IN ('SM BOX', 'SM CASE', 'SM PACK', 'SM PKG'),(lineitem.l_quantity >= 1.00),(lineitem.l_quantity <= 11.00),(part.p_size <= 5)],AND[(part.p_brand = 'Brand#23'),p_container IN ('MED BAG', 'MED BOX', 'MED PACK', 'MED PKG'),(lineitem.l_quantity >= 10.00),(lineitem.l_quantity <= 20.00),(part.p_size <= 10)],AND[(part.p_brand = 'Brand#34'),p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG'),(lineitem.l_quantity >= 20.00),(lineitem.l_quantity <= 30.00),(part.p_size <= 15)]]) build RFs:RF0 p_partkey->[l_partkey] ------------PhysicalProject --------------filter((lineitem.l_quantity <= 30.00) and (lineitem.l_quantity >= 1.00) and (lineitem.l_shipinstruct = 'DELIVER IN PERSON') and l_shipmode IN ('AIR REG', 'AIR')) ----------------PhysicalOlapScan[lineitem] apply RFs: RF0 ------------PhysicalProject ---------------filter((((((part.p_brand = 'Brand#12') AND p_container IN ('SM BOX', 'SM CASE', 'SM PACK', 'SM PKG')) AND (part.p_size <= 5)) OR (((part.p_brand = 'Brand#23') AND p_container IN ('MED BAG', 'MED BOX', 'MED PACK', 'MED PKG')) AND (part.p_size <= 10))) OR (((part.p_brand = 'Brand#34') AND p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG')) AND (part.p_size <= 15))) and (part.p_size >= 1) and p_brand IN ('Brand#12', 'Brand#23', 'Brand#34') and p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG', 'MED BAG', 'MED BOX', 'MED PACK', 'MED PKG', 'SM BOX', 'SM CASE', 'SM PACK', 'SM PKG')) +--------------filter((part.p_size >= 1) and OR[AND[(part.p_brand = 'Brand#12'),p_container IN ('SM BOX', 'SM CASE', 'SM PACK', 'SM PKG'),(part.p_size <= 5)],AND[(part.p_brand = 'Brand#23'),p_container IN ('MED BAG', 'MED BOX', 'MED PACK', 'MED PKG'),(part.p_size <= 10)],AND[(part.p_brand = 'Brand#34'),p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG'),(part.p_size <= 15)]] and p_brand IN ('Brand#12', 'Brand#23', 'Brand#34') and p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG', 'MED BAG', 'MED BOX', 'MED PACK', 'MED PKG', 'SM BOX', 'SM CASE', 'SM PACK', 'SM PKG')) ----------------PhysicalOlapScan[part] diff --git a/regression-test/data/new_shapes_p0/tpch_sf1000/shape_no_stats/q7.out b/regression-test/data/new_shapes_p0/tpch_sf1000/shape_no_stats/q7.out index f434b57fc9f511..b98149f8668a1c 100644 --- a/regression-test/data/new_shapes_p0/tpch_sf1000/shape_no_stats/q7.out +++ b/regression-test/data/new_shapes_p0/tpch_sf1000/shape_no_stats/q7.out @@ -24,7 +24,7 @@ PhysicalResultSink --------------------------------PhysicalOlapScan[supplier] apply RFs: RF4 ----------------------PhysicalProject ------------------------PhysicalOlapScan[customer] apply RFs: RF3 -------------------NestedLoopJoin[INNER_JOIN](((n1.n_name = 'FRANCE') AND (n2.n_name = 'GERMANY')) OR ((n1.n_name = 'GERMANY') AND (n2.n_name = 'FRANCE'))) +------------------NestedLoopJoin[INNER_JOIN]OR[AND[(n1.n_name = 'FRANCE'),(n2.n_name = 'GERMANY')],AND[(n1.n_name = 'GERMANY'),(n2.n_name = 'FRANCE')]] --------------------PhysicalProject ----------------------filter(n_name IN ('FRANCE', 'GERMANY')) ------------------------PhysicalOlapScan[nation] diff --git a/regression-test/suites/nereids_p0/huge_compound/huge_compound.groovy b/regression-test/suites/nereids_p0/huge_compound/huge_compound.groovy new file mode 100644 index 00000000000000..3eb38185aef06e --- /dev/null +++ b/regression-test/suites/nereids_p0/huge_compound/huge_compound.groovy @@ -0,0 +1,69 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("huge_compound") { + sql """ + drop table if exists tyt_old_order; + CREATE TABLE `tyt_old_order` ( + `n` varchar(65533) NULL, + `c` varchar(65533) NULL + ) ENGINE=OLAP + DUPLICATE KEY(`n`) + DISTRIBUTED BY HASH(`n`) BUCKETS AUTO + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + insert into tyt_old_order values("72", "93"), ("62", "xx") + """ + + qt_1 """ + SELECT * + FROM tyt_old_order + where(n='72'AND c='93')OR(n='04'AND c='85')OR(n='43'AND c='37')OR(n='92'AND c='09')OR(n='85'AND c='69')OR(n='16'AND c='88')OR(n='87'AND c='22')OR(n='38'AND c='44')OR(n='65'AND c='22')OR(n='83'AND c='12')OR(n='14'AND c='44')OR(n='64'AND c='33')OR(n='44'AND c='58')OR(n='37'AND c='27')OR(n='59'AND c='36')OR(n='15'AND c='66')OR(n='44'AND c='66')OR(n='59'AND c='23')OR(n='08'AND c='87')OR(n='03'AND c='09')OR(n='85'AND c='26')OR(n='61'AND c='08')OR(n='23'AND c='13')OR(n='96'AND c='97')OR(n='15'AND c='44')OR(n='33'AND c='33')OR(n='11'AND c='87')OR(n='49'AND c='55')OR(n='70'AND c='29')OR(n='47'AND c='85')OR(n='14'AND c='18')OR(n='77'AND c='17')OR(n='66'AND c='07')OR(n='59'AND c='16')OR(n='48'AND c='11')OR(n='66'AND c='85')OR(n='46'AND c='65')OR(n='26'AND c='68')OR(n='63'AND c='22')OR(n='24'AND c='96')OR(n='17'AND c='00')OR(n='35'AND c='00')OR(n='06'AND c='26')OR(n='55'AND c='65')OR(n='16'AND c='17')OR(n='89'AND c='19')OR(n='57'AND c='30')OR(n='87'AND c='99')OR(n='27'AND c='89')OR(n='72'AND c='91')OR(n='47'AND c='51')OR(n='39'AND c='29')OR(n='87'AND c='45')OR(n='47'AND c='85')OR(n='99'AND c='40')OR(n='04'AND c='67')OR(n='65'AND c='50')OR(n='23'AND c='13')OR(n='54'AND c='44')OR(n='33'AND c='45')OR(n='48'AND c='86')OR(n='82'AND c='33')OR(n='15'AND c='85')OR(n='11'AND c='85')OR(n='33'AND c='93')OR(n='74'AND c='88')OR(n='07'AND c='58')OR(n='34'AND c='35')OR(n='90'AND c='18')OR(n='60'AND c='34')OR(n='78'AND c='77')OR(n='59'AND c='15')OR(n='89'AND c='79') + OR(n='69'AND c='69')OR(n='91'AND c='84')OR(n='28'AND c='76')OR(n='71'AND c='77')OR(n='12'AND c='65')OR(n='08'AND c='36')OR(n='23'AND c='16')OR(n='51'AND c='11')OR(n='48'AND c='19')OR(n='58'AND c='16')OR(n='77'AND c='15')OR(n='80'AND c='15')OR(n='75'AND c='15')OR(n='78'AND c='15')OR(n='61'AND c='15')OR(n='59'AND c='15')OR(n='07'AND c='15')OR(n='18'AND c='15')OR(n='59'AND c='15')OR(n='94'AND c='15')OR(n='55'AND c='15')OR(n='71'AND c='15')OR(n='11'AND c='65')OR(n='42'AND c='48')OR(n='91'AND c='00')OR(n='74'AND c='77')OR(n='20'AND c='80')OR(n='95'AND c='65')OR(n='12'AND c='00')OR(n='26'AND c='77')OR(n='10'AND c='48')OR(n='65'AND c='80')OR(n='20'AND c='48')OR(n='86'AND c='00')OR(n='22'AND c='77')OR(n='16'AND c='65')OR(n='56'AND c='80')OR(n='95'AND c='74')OR(n='61'AND c='68')OR(n='08'AND c='65')OR(n='05'AND c='48')OR(n='90'AND c='00')OR(n='96'AND c='77')OR(n='77'AND c='80')OR(n='46'AND c='65')OR(n='11'AND c='48')OR(n='79'AND c='00')OR(n='54'AND c='77')OR(n='25'AND c='80')OR(n='12'AND c='65')OR(n='39'AND c='48')OR(n='39'AND c='00')OR(n='24'AND c='77')OR(n='78'AND c='80')OR(n='21'AND c='02')OR(n='52'AND c='68')OR(n='12'AND c='66')OR(n='39'AND c='82')OR(n='36'AND c='19')OR(n='08'AND c='19')OR(n='29'AND c='19')OR(n='81'AND c='19')OR(n='25'AND c='19')OR(n='96'AND c='08')OR(n='98'AND c='19')OR(n='40'AND c='19')OR(n='03'AND c='19')OR(n='17'AND c='19')OR(n='97'AND c='19')OR(n='91'AND c='19')OR(n='21'AND c='19')OR(n='46'AND c='19')OR(n='69'AND c='19')OR(n='75'AND c='19')OR(n='68'AND c='19')OR(n='64'AND c='19')OR(n='31'AND c='19') + OR(n='89'AND c='19')OR(n='99'AND c='19')OR(n='60'AND c='19')OR(n='89'AND c='48')OR(n='40'AND c='00')OR(n='04'AND c='77')OR(n='40'AND c='65')OR(n='97'AND c='80')OR(n='59'AND c='48')OR(n='49'AND c='00')OR(n='38'AND c='77')OR(n='89'AND c='65')OR(n='25'AND c='80')OR(n='23'AND c='48')OR(n='88'AND c='00')OR(n='25'AND c='77')OR(n='82'AND c='65')OR(n='94'AND c='80')OR(n='26'AND c='65')OR(n='58'AND c='48')OR(n='95'AND c='00')OR(n='85'AND c='77')OR(n='46'AND c='80')OR(n='65'AND c='65')OR(n='40'AND c='48')OR(n='14'AND c='00')OR(n='93'AND c='77')OR(n='90'AND c='80')OR(n='51'AND c='65')OR(n='40'AND c='48')OR(n='74'AND c='00')OR(n='24'AND c='77')OR(n='56'AND c='80')OR(n='01'AND c='00')OR(n='79'AND c='77')OR(n='24'AND c='80')OR(n='48'AND c='65')OR(n='29'AND c='48')OR(n='09'AND c='00')OR(n='16'AND c='77')OR(n='61'AND c='80')OR(n='64'AND c='23')OR(n='52'AND c='65')OR(n='86'AND c='48')OR(n='62'AND c='00')OR(n='73'AND c='77')OR(n='36'AND c='80')OR(n='11'AND c='65')OR(n='13'AND c='48')OR(n='53'AND c='00')OR(n='39'AND c='77')OR(n='06'AND c='80')OR(n='21'AND c='76')OR(n='53'AND c='88')OR(n='37'AND c='53')OR(n='50'AND c='00')OR(n='42'AND c='00')OR(n='03'AND c='15')OR(n='82'AND c='71')OR(n='15'AND c='77')OR(n='79'AND c='18')OR(n='09'AND c='96')OR(n='15'AND c='98')OR(n='87'AND c='33')OR(n='04'AND c='80')OR(n='64'AND c='80')OR(n='88'AND c='05')OR(n='75'AND c='66')OR(n='85'AND c='80')OR(n='18'AND c='80')OR(n='12'AND c='58')OR(n='28'AND c='60')OR(n='61'AND c='55')OR(n='78'AND c='05')OR(n='39'AND c='77')OR(n='78'AND c='77')OR(n='02'AND c='65')OR(n='29'AND c='11')OR(n='49'AND c='87')OR(n='49'AND c='19')OR(n='43'AND c='19')OR(n='25'AND c='36')OR(n='90'AND c='84')OR(n='52'AND c='87')OR(n='46'AND c='77') + OR(n='99'AND c='89')OR(n='83'AND c='53')OR(n='73'AND c='69')OR(n='38'AND c='18')OR(n='06'AND c='02')OR(n='21'AND c='51')OR(n='27'AND c='88')OR(n='77'AND c='72')OR(n='28'AND c='92')OR(n='90'AND c='55')OR(n='75'AND c='05')OR(n='57'AND c='99')OR(n='45'AND c='07')OR(n='92'AND c='99')OR(n='51'AND c='55')OR(n='29'AND c='77')OR(n='82'AND c='30')OR(n='83'AND c='40')OR(n='79'AND c='89')OR(n='27'AND c='99')OR(n='25'AND c='06')OR(n='71'AND c='51')OR(n='57'AND c='39')OR(n='96'AND c='18')OR(n='18'AND c='02')OR(n='87'AND c='69')OR(n='53'AND c='01')OR(n='14'AND c='50')OR(n='19'AND c='33')OR(n='12'AND c='33')OR(n='99'AND c='86')OR(n='78'AND c='99')OR(n='98'AND c='66')OR(n='94'AND c='52')OR(n='96'AND c='50')OR(n='36'AND c='16')OR(n='58'AND c='18')OR(n='02'AND c='96')OR(n='84'AND c='11')OR(n='94'AND c='73')OR(n='97'AND c='08')OR(n='19'AND c='33')OR(n='41'AND c='06')OR(n='05'AND c='58')OR(n='12'AND c='30')OR(n='58'AND c='22')OR(n='41'AND c='66')OR(n='20'AND c='71')OR(n='41'AND c='71')OR(n='95'AND c='44')OR(n='90'AND c='58')OR(n='96'AND c='06')OR(n='18'AND c='00')OR(n='35'AND c='38')OR(n='45'AND c='02')OR(n='04'AND c='00')OR(n='87'AND c='99')OR(n='51'AND c='38')OR(n='42'AND c='61')OR(n='09'AND c='14')OR(n='48'AND c='70')OR(n='70'AND c='04')OR(n='39'AND c='97')OR(n='72'AND c='32')OR(n='46'AND c='72')OR(n='19'AND c='13')OR(n='98'AND c='37')OR(n='34'AND c='18')OR(n='64'AND c='71')OR(n='73'AND c='87')OR(n='82'AND c='19')OR(n='14'AND c='43')OR(n='36'AND c='22')OR(n='06'AND c='71')OR(n='08'AND c='71')OR(n='61'AND c='71')OR(n='71'AND c='88')OR(n='74'AND c='71') + OR(n='64'AND c='89')OR(n='13'AND c='71')OR(n='77'AND c='35')OR(n='66'AND c='11')OR(n='59'AND c='85')OR(n='29'AND c='66')OR(n='85'AND c='50')OR(n='18'AND c='66')OR(n='10'AND c='92')OR(n='76'AND c='66')OR(n='31'AND c='99')OR(n='86'AND c='91')OR(n='92'AND c='88')OR(n='40'AND c='18')OR(n='18'AND c='77')OR(n='72'AND c='63')OR(n='29'AND c='77')OR(n='24'AND c='87')OR(n='03'AND c='51')OR(n='81'AND c='01')OR(n='50'AND c='72')OR(n='25'AND c='33')OR(n='99'AND c='68')OR(n='51'AND c='13')OR(n='59'AND c='88')OR(n='31'AND c='97')OR(n='71'AND c='10')OR(n='15'AND c='91')OR(n='31'AND c='88')OR(n='92'AND c='00')OR(n='11'AND c='75')OR(n='06'AND c='99')OR(n='46'AND c='32')OR(n='13'AND c='09')OR(n='18'AND c='33')OR(n='88'AND c='58')OR(n='90'AND c='77')OR(n='22'AND c='10')OR(n='37'AND c='19')OR(n='52'AND c='10')OR(n='30'AND c='22')OR(n='98'AND c='66')OR(n='65'AND c='66')OR(n='22'AND c='98')OR(n='99'AND c='77')OR(n='84'AND c='32')OR(n='73'AND c='03')OR(n='99'AND c='58')OR(n='63'AND c='77')OR(n='50'AND c='99')OR(n='23'AND c='35')OR(n='40'AND c='88')OR(n='76'AND c='88')OR(n='10'AND c='88')OR(n='37'AND c='86')OR(n='94'AND c='08')OR(n='84'AND c='30')OR(n='95'AND c='88')OR(n='81'AND c='25')OR(n='34'AND c='25')OR(n='58'AND c='66')OR(n='40'AND c='21')OR(n='02'AND c='88')OR(n='44'AND c='53')OR(n='42'AND c='99')OR(n='56'AND c='16')OR(n='54'AND c='37')OR(n='89'AND c='97')OR(n='49'AND c='22')OR(n='97'AND c='81')OR(n='56'AND c='85')OR(n='13'AND c='93')OR(n='06'AND c='93')OR(n='40'AND c='44')OR(n='84'AND c='09')OR(n='67'AND c='83')OR(n='56'AND c='88')OR(n='72'AND c='58')OR(n='04'AND c='68') + OR(n='96'AND c='68')OR(n='74'AND c='39')OR(n='16'AND c='14')OR(n='28'AND c='00')OR(n='83'AND c='60')OR(n='61'AND c='88')OR(n='13'AND c='70')OR(n='79'AND c='36')OR(n='26'AND c='91')OR(n='41'AND c='91')OR(n='98'AND c='91')OR(n='29'AND c='50')OR(n='02'AND c='45')OR(n='51'AND c='77')OR(n='53'AND c='05')OR(n='02'AND c='14')OR(n='88'AND c='50')OR(n='93'AND c='90')OR(n='32'AND c='09')OR(n='53'AND c='56')OR(n='27'AND c='16')OR(n='30'AND c='68')OR(n='62'AND c='26')OR(n='33'AND c='26')OR(n='04'AND c='74')OR(n='79'AND c='08')OR(n='47'AND c='28')OR(n='93'AND c='44')OR(n='28'AND c='34')OR(n='89'AND c='79')OR(n='63'AND c='88')OR(n='51'AND c='33')OR(n='72'AND c='66')OR(n='22'AND c='79')OR(n='97'AND c='82')OR(n='45'AND c='61')OR(n='02'AND c='33')OR(n='83'AND c='08')OR(n='46'AND c='11')OR(n='09'AND c='33')OR(n='94'AND c='72')OR(n='82'AND c='09')OR(n='15'AND c='92')OR(n='54'AND c='50')OR(n='17'AND c='17')OR(n='89'AND c='26')OR(n='79'AND c='76')OR(n='08'AND c='76')OR(n='46'AND c='94')OR(n='73'AND c='78')OR(n='61'AND c='16')OR(n='42'AND c='43')OR(n='79'AND c='97')OR(n='43'AND c='02')OR(n='50'AND c='15')OR(n='26'AND c='21')OR(n='60'AND c='56')OR(n='08'AND c='51')OR(n='46'AND c='21')OR(n='29'AND c='37')OR(n='03'AND c='58')OR(n='84'AND c='79')OR(n='51'AND c='09')OR(n='15'AND c='09')OR(n='68'AND c='77')OR(n='45'AND c='77')OR(n='65'AND c='17')OR(n='76'AND c='16')OR(n='07'AND c='10')OR(n='13'AND c='56')OR(n='75'AND c='93')OR(n='17'AND c='51')OR(n='86'AND c='68')OR(n='73'AND c='68')OR(n='24'AND c='99')OR(n='88'AND c='99')OR(n='15'AND c='33')OR(n='24'AND c='95')OR(n='36'AND c='83') + OR(n='12'AND c='83')OR(n='32'AND c='83')OR(n='62'AND c='83')OR(n='35'AND c='56')OR(n='01'AND c='67')OR(n='37'AND c='85')OR(n='89'AND c='08')OR(n='57'AND c='39')OR(n='95'AND c='18')OR(n='31'AND c='08')OR(n='06'AND c='08')OR(n='69'AND c='79')OR(n='69'AND c='70')OR(n='43'AND c='06')OR(n='55'AND c='53')OR(n='75'AND c='91')OR(n='26'AND c='66')OR(n='14'AND c='22')OR(n='23'AND c='38')OR(n='07'AND c='18')OR(n='22'AND c='29')OR(n='39'AND c='53')OR(n='60'AND c='28')OR(n='12'AND c='12')OR(n='45'AND c='03')OR(n='54'AND c='60')OR(n='53'AND c='98')OR(n='11'AND c='78')OR(n='43'AND c='53')OR(n='80'AND c='98')OR(n='04'AND c='69')OR(n='75'AND c='68')OR(n='91'AND c='26')OR(n='55'AND c='93')OR(n='12'AND c='88')OR(n='10'AND c='03')OR(n='27'AND c='93')OR(n='96'AND c='89')OR(n='83'AND c='66')OR(n='09'AND c='23')OR(n='27'AND c='68')OR(n='71'AND c='33')OR(n='21'AND c='42')OR(n='49'AND c='56')OR(n='71'AND c='81')OR(n='83'AND c='82')OR(n='91'AND c='82')OR(n='82'AND c='92')OR(n='09'AND c='50')OR(n='99'AND c='45')OR(n='52'AND c='45')OR(n='47'AND c='70')OR(n='98'AND c='88')OR(n='01'AND c='35')OR(n='81'AND c='85')OR(n='02'AND c='53')OR(n='13'AND c='91')OR(n='75'AND c='88')OR(n='30'AND c='46')OR(n='56'AND c='90')OR(n='64'AND c='54')OR(n='37'AND c='95')OR(n='38'AND c='88')OR(n='89'AND c='05')OR(n='73'AND c='88')OR(n='60'AND c='51')OR(n='48'AND c='60')OR(n='25'AND c='62')OR(n='61'AND c='88')OR(n='74'AND c='19')OR(n='24'AND c='62')OR(n='44'AND c='68')OR(n='15'AND c='60')OR(n='89'AND c='69')OR(n='06'AND c='57')OR(n='83'AND c='49')OR(n='66'AND c='99')OR(n='31'AND c='11')OR(n='68'AND c='96') + OR(n='49'AND c='48')OR(n='59'AND c='95')OR(n='30'AND c='05')OR(n='63'AND c='20')OR(n='68'AND c='85')OR(n='75'AND c='99')OR(n='46'AND c='29')OR(n='93'AND c='99')OR(n='48'AND c='99')OR(n='24'AND c='88')OR(n='47'AND c='15')OR(n='52'AND c='99')OR(n='17'AND c='91')OR(n='12'AND c='53')OR(n='38'AND c='16')OR(n='82'AND c='82')OR(n='79'AND c='11')OR(n='99'AND c='01')OR(n='40'AND c='68')OR(n='33'AND c='79')OR(n='32'AND c='07')OR(n='39'AND c='26')OR(n='31'AND c='81')OR(n='16'AND c='66')OR(n='75'AND c='59')OR(n='76'AND c='77')OR(n='22'AND c='82')OR(n='49'AND c='08')OR(n='09'AND c='18')OR(n='79'AND c='87')OR(n='88'AND c='32')OR(n='14'AND c='16')OR(n='47'AND c='37')OR(n='24'AND c='33')OR(n='03'AND c='53')OR(n='96'AND c='88')OR(n='66'AND c='88')OR(n='82'AND c='19')OR(n='29'AND c='26')OR(n='33'AND c='48')OR(n='75'AND c='19')OR(n='76'AND c='39')OR(n='39'AND c='11')OR(n='41'AND c='22')OR(n='96'AND c='76')OR(n='63'AND c='33')OR(n='13'AND c='00')OR(n='08'AND c='28')OR(n='56'AND c='20')OR(n='28'AND c='20')OR(n='54'AND c='88')OR(n='53'AND c='78')OR(n='95'AND c='79')OR(n='08'AND c='75')OR(n='44'AND c='31')OR(n='45'AND c='47')OR(n='47'AND c='96')OR(n='84'AND c='33')OR(n='73'AND c='55')OR(n='74'AND c='77')OR(n='66'AND c='19')OR(n='20'AND c='19')OR(n='42'AND c='19')OR(n='16'AND c='19')OR(n='10'AND c='19')OR(n='37'AND c='65')OR(n='75'AND c='77')OR(n='64'AND c='48')OR(n='19'AND c='00')OR(n='93'AND c='80')OR(n='83'AND c='65')OR(n='45'AND c='77')OR(n='91'AND c='48')OR(n='81'AND c='00')OR(n='74'AND c='80') + OR(n='88'AND c='68')OR(n='21'AND c='34')OR(n='17'AND c='17')OR(n='40'AND c='79')OR(n='90'AND c='71')OR(n='80'AND c='22')OR(n='27'AND c='07')OR(n='12'AND c='88')OR(n='24'AND c='85')OR(n='16'AND c='38')OR(n='67'AND c='54')OR(n='06'AND c='20')OR(n='97'AND c='20')OR(n='41'AND c='66')OR(n='99'AND c='51')OR(n='88'AND c='55')OR(n='44'AND c='70')OR(n='23'AND c='91')OR(n='94'AND c='78')OR(n='64'AND c='00')OR(n='66'AND c='83')OR(n='83'AND c='27')OR(n='06'AND c='91')OR(n='28'AND c='23')OR(n='89'AND c='20')OR(n='79'AND c='13')OR(n='25'AND c='60')OR(n='92'AND c='28')OR(n='28'AND c='66')OR(n='84'AND c='76')OR(n='25'AND c='76')OR(n='59'AND c='66')OR(n='18'AND c='43')OR(n='70'AND c='58')OR(n='11'AND c='77')OR(n='22'AND c='83')OR(n='28'AND c='00')OR(n='05'AND c='00')OR(n='16'AND c='76')OR(n='74'AND c='98')OR(n='41'AND c='06')OR(n='99'AND c='06')OR(n='58'AND c='76')OR(n='24'AND c='22')OR(n='85'AND c='86')OR(n='30'AND c='67')OR(n='02'AND c='76')OR(n='40'AND c='00')OR(n='96'AND c='43')OR(n='51'AND c='39')OR(n='73'AND c='77')OR(n='74'AND c='75')OR(n='85'AND c='85')OR(n='97'AND c='93')OR(n='28'AND c='80')OR(n='29'AND c='80')OR(n='84'AND c='41')OR(n='05'AND c='86')OR(n='50'AND c='86')OR(n='89'AND c='00')OR(n='64'AND c='48')OR(n='85'AND c='65')OR(n='33'AND c='77')OR(n='85'AND c='80')OR(n='70'AND c='99')OR(n='63'AND c='69')OR(n='16'AND c='86')OR(n='42'AND c='86')OR(n='77'AND c='48')OR(n='10'AND c='65')OR(n='95'AND c='77')OR(n='81'AND c='00')OR(n='94'AND c='08')OR(n='09'AND c='44')OR(n='70'AND c='22')OR(n='32'AND c='51')OR(n='16'AND c='51')OR(n='47'AND c='99')OR(n='30'AND c='99')OR(n='32'AND c='99') + OR(n='20'AND c='86')OR(n='36'AND c='86')OR(n='50'AND c='86')OR(n='77'AND c='86')OR(n='62'AND c='86')OR(n='92'AND c='86')OR(n='43'AND c='86')OR(n='40'AND c='86')OR(n='26'AND c='22')OR(n='99'AND c='35')OR(n='06'AND c='48')OR(n='77'AND c='65')OR(n='50'AND c='77')OR(n='14'AND c='00')OR(n='85'AND c='95')OR(n='21'AND c='48')OR(n='05'AND c='65')OR(n='19'AND c='77')OR(n='03'AND c='00')OR(n='65'AND c='19')OR(n='30'AND c='48')OR(n='93'AND c='65')OR(n='48'AND c='77')OR(n='74'AND c='00')OR(n='23'AND c='68')OR(n='73'AND c='22')OR(n='10'AND c='00')OR(n='55'AND c='48')OR(n='01'AND c='65')OR(n='17'AND c='77')OR(n='16'AND c='80')OR(n='44'AND c='88')OR(n='85'AND c='00')OR(n='30'AND c='48')OR(n='03'AND c='65')OR(n='91'AND c='77')OR(n='94'AND c='80')OR(n='05'AND c='00')OR(n='52'AND c='48')OR(n='00'AND c='65')OR(n='77'AND c='77')OR(n='77'AND c='80')OR(n='12'AND c='48')OR(n='39'AND c='65')OR(n='60'AND c='77')OR(n='53'AND c='00')OR(n='06'AND c='80')OR(n='22'AND c='97')OR(n='47'AND c='15')OR(n='06'AND c='15')OR(n='58'AND c='48')OR(n='99'AND c='65')OR(n='96'AND c='77')OR(n='40'AND c='00')OR(n='31'AND c='80')OR(n='45'AND c='00')OR(n='67'AND c='48')OR(n='63'AND c='65')OR(n='91'AND c='77')OR(n='41'AND c='80')OR(n='81'AND c='99')OR(n='87'AND c='99')OR(n='67'AND c='99')OR(n='46'AND c='00')OR(n='69'AND c='48')OR(n='62'AND c='65')OR(n='80'AND c='77')OR(n='92'AND c='80')OR(n='44'AND c='00')OR(n='07'AND c='48')OR(n='74'AND c='65')OR(n='92'AND c='77')OR(n='79'AND c='80')OR(n='72'AND c='12')OR(n='42'AND c='00')OR(n='23'AND c='48')OR(n='23'AND c='65')OR(n='88'AND c='77') + OR(n='41'AND c='80')OR(n='26'AND c='71')OR(n='95'AND c='85')OR(n='59'AND c='15')OR(n='63'AND c='70')OR(n='02'AND c='10')OR(n='26'AND c='33')OR(n='94'AND c='16')OR(n='99'AND c='60')OR(n='83'AND c='38')OR(n='40'AND c='85')OR(n='56'AND c='73')OR(n='58'AND c='85')OR(n='16'AND c='67')OR(n='36'AND c='36')OR(n='86'AND c='05')OR(n='73'AND c='82')OR(n='22'AND c='44')OR(n='04'AND c='86')OR(n='74'AND c='95')OR(n='83'AND c='56')OR(n='29'AND c='07')OR(n='74'AND c='71')OR(n='19'AND c='00')OR(n='79'AND c='22')OR(n='07'AND c='91')OR(n='48'AND c='00')OR(n='88'AND c='78')OR(n='05'AND c='88')OR(n='81'AND c='52')OR(n='20'AND c='52')OR(n='30'AND c='51')OR(n='77'AND c='32')OR(n='62'AND c='81')OR(n='78'AND c='97')OR(n='83'AND c='88')OR(n='12'AND c='96')OR(n='58'AND c='13')OR(n='69'AND c='10')OR(n='90'AND c='13')OR(n='60'AND c='13')OR(n='28'AND c='99')OR(n='83'AND c='45')OR(n='67'AND c='01')OR(n='05'AND c='88')OR(n='55'AND c='30')OR(n='74'AND c='40')OR(n='77'AND c='05')OR(n='65'AND c='44')OR(n='62'AND c='93')OR(n='70'AND c='88')OR(n='03'AND c='71')OR(n='35'AND c='86')OR(n='03'AND c='33')OR(n='64'AND c='62')OR(n='86'AND c='55')OR(n='52'AND c='68')OR(n='98'AND c='22')OR(n='81'AND c='96')OR(n='73'AND c='73')OR(n='26'AND c='33')OR(n='77'AND c='99')OR(n='61'AND c='30')OR(n='28'AND c='92')OR(n='28'AND c='66')OR(n='17'AND c='16')OR(n='27'AND c='62')OR(n='43'AND c='22')OR(n='97'AND c='00')OR(n='03'AND c='48')OR(n='65'AND c='65')OR(n='87'AND c='77')OR(n='93'AND c='80')OR(n='75'AND c='62')OR(n='29'AND c='19')OR(n='47'AND c='56')OR(n='42'AND c='33')OR(n='24'AND c='38')OR(n='08'AND c='52')OR(n='19'AND c='77')OR(n='30'AND c='26')OR(n='86'AND c='44')OR(n='94'AND c='95')OR(n='41'AND c='23')OR(n='44'AND c='51')OR(n='10'AND c='66')OR(n='59'AND c='54')OR(n='91'AND c='31')OR(n='95'AND c='99') + OR(n='73'AND c='20')OR(n='57'AND c='22')OR(n='81'AND c='88')OR(n='95'AND c='00')OR(n='96'AND c='88')OR(n='80'AND c='05')OR(n='43'AND c='22')OR(n='70'AND c='99')OR(n='99'AND c='78')OR(n='47'AND c='78')OR(n='85'AND c='77')OR(n='77'AND c='62')OR(n='64'AND c='67')OR(n='40'AND c='97')OR(n='48'AND c='68')OR(n='42'AND c='40')OR(n='54'AND c='02')OR(n='47'AND c='22')OR(n='77'AND c='62')OR(n='96'AND c='02')OR(n='65'AND c='87')OR(n='94'AND c='95')OR(n='67'AND c='52')OR(n='91'AND c='41')OR(n='87'AND c='06')OR(n='55'AND c='09')OR(n='20'AND c='89')OR(n='23'AND c='19')OR(n='59'AND c='27')OR(n='78'AND c='57')OR(n='67'AND c='97')OR(n='36'AND c='33')OR(n='71'AND c='27')OR(n='60'AND c='85')OR(n='06'AND c='69')OR(n='46'AND c='12')OR(n='71'AND c='00')OR(n='33'AND c='87')OR(n='26'AND c='31')OR(n='68'AND c='43')OR(n='47'AND c='41')OR(n='25'AND c='11')OR(n='67'AND c='56')OR(n='62'AND c='21')OR(n='92'AND c='88')OR(n='35'AND c='62')OR(n='34'AND c='99')OR(n='39'AND c='09')OR(n='42'AND c='22')OR(n='67'AND c='88')OR(n='79'AND c='65')OR(n='72'AND c='77')OR(n='21'AND c='38')OR(n='81'AND c='99')OR(n='06'AND c='40')OR(n='06'AND c='00')OR(n='99'AND c='99')OR(n='11'AND c='88')OR(n='38'AND c='55')OR(n='02'AND c='55')OR(n='44'AND c='99')OR(n='47'AND c='09')OR(n='02'AND c='33')OR(n='67'AND c='85')OR(n='00'AND c='53')OR(n='16'AND c='96')OR(n='97'AND c='20')OR(n='40'AND c='56')OR(n='59'AND c='73')OR(n='22'AND c='77')OR(n='88'AND c='55')OR(n='06'AND c='25')OR(n='17'AND c='19')OR(n='51'AND c='03')OR(n='30'AND c='22')OR(n='15'AND c='08')OR(n='95'AND c='00')OR(n='25'AND c='88')OR(n='54'AND c='88')OR(n='81'AND c='09')OR(n='06'AND c='33')OR(n='51'AND c='32')OR(n='79'AND c='32')OR(n='13'AND c='32')OR(n='54'AND c='39')OR(n='55'AND c='99')OR(n='47'AND c='30') + OR(n='03'AND c='66')OR(n='47'AND c='45')OR(n='21'AND c='08')OR(n='01'AND c='18')OR(n='71'AND c='65')OR(n='37'AND c='65')OR(n='66'AND c='99')OR(n='93'AND c='73')OR(n='46'AND c='33')OR(n='45'AND c='18')OR(n='10'AND c='27')OR(n='25'AND c='66')OR(n='84'AND c='71')OR(n='91'AND c='71')OR(n='02'AND c='71')OR(n='81'AND c='33')OR(n='49'AND c='44')OR(n='80'AND c='55')OR(n='22'AND c='05')OR(n='75'AND c='62')OR(n='98'AND c='67')OR(n='75'AND c='18')OR(n='02'AND c='00')OR(n='83'AND c='80')OR(n='36'AND c='00')OR(n='50'AND c='99')OR(n='36'AND c='37')OR(n='41'AND c='37')OR(n='83'AND c='00')OR(n='80'AND c='48')OR(n='71'AND c='77')OR(n='57'AND c='65')OR(n='77'AND c='80')OR(n='96'AND c='99')OR(n='70'AND c='00')OR(n='37'AND c='48')OR(n='09'AND c='77')OR(n='47'AND c='65')OR(n='23'AND c='80')OR(n='10'AND c='77')OR(n='19'AND c='88')OR(n='56'AND c='88')OR(n='11'AND c='93')OR(n='41'AND c='15')OR(n='00'AND c='15')OR(n='70'AND c='19')OR(n='35'AND c='15')OR(n='26'AND c='15')OR(n='56'AND c='00')OR(n='53'AND c='48')OR(n='18'AND c='77')OR(n='73'AND c='65')OR(n='82'AND c='80')OR(n='67'AND c='19')OR(n='69'AND c='48')OR(n='07'AND c='77')OR(n='44'AND c='00')OR(n='54'AND c='65')OR(n='20'AND c='80')OR(n='62'AND c='19')OR(n='91'AND c='19')OR(n='13'AND c='48')OR(n='05'AND c='77')OR(n='20'AND c='00')OR(n='54'AND c='65')OR(n='43'AND c='80')OR(n='53'AND c='80')OR(n='53'AND c='48')OR(n='02'AND c='77')OR(n='01'AND c='00')OR(n='82'AND c='65')OR(n='22'AND c='80')OR(n='23'AND c='36')OR(n='60'AND c='53')OR(n='68'AND c='53')OR(n='77'AND c='83')OR(n='87'AND c='35')OR(n='51'AND c='32')OR(n='44'AND c='76')OR(n='44'AND c='16')OR(n='06'AND c='89')OR(n='17'AND c='57')OR(n='61'AND c='57')OR(n='06'AND c='18')OR(n='18'AND c='89') + OR(n='17'AND c='18')OR(n='85'AND c='20')OR(n='98'AND c='15')OR(n='22'AND c='27')OR(n='00'AND c='27')OR(n='70'AND c='68')OR(n='23'AND c='08')OR(n='43'AND c='99')OR(n='43'AND c='88')OR(n='30'AND c='44')OR(n='69'AND c='78')OR(n='85'AND c='22')OR(n='00'AND c='62')OR(n='39'AND c='96')OR(n='25'AND c='77')OR(n='92'AND c='29')OR(n='77'AND c='77')OR(n='30'AND c='26')OR(n='16'AND c='88')OR(n='68'AND c='95')OR(n='21'AND c='10')OR(n='52'AND c='18')OR(n='07'AND c='19')OR(n='95'AND c='68')OR(n='01'AND c='37')OR(n='08'AND c='66')OR(n='69'AND c='23')OR(n='68'AND c='38')OR(n='60'AND c='25')OR(n='05'AND c='90')OR(n='02'AND c='87')OR(n='81'AND c='41')OR(n='33'AND c='57')OR(n='65'AND c='52')OR(n='03'AND c='28')OR(n='56'AND c='82')OR(n='49'AND c='79')OR(n='84'AND c='94')OR(n='86'AND c='56')OR(n='58'AND c='08')OR(n='42'AND c='50')OR(n='51'AND c='00')OR(n='55'AND c='55')OR(n='27'AND c='69')OR(n='23'AND c='69')OR(n='47'AND c='82')OR(n='82'AND c='18')OR(n='43'AND c='25')OR(n='84'AND c='87')OR(n='04'AND c='31')OR(n='20'AND c='54')OR(n='04'AND c='76')OR(n='31'AND c='99')OR(n='01'AND c='77')OR(n='30'AND c='77')OR(n='01'AND c='88')OR(n='33'AND c='97')OR(n='59'AND c='43')OR(n='49'AND c='77')OR(n='20'AND c='99')OR(n='99'AND c='82')OR(n='92'AND c='88')OR(n='54'AND c='87')OR(n='71'AND c='76')OR(n='02'AND c='22')OR(n='99'AND c='89')OR(n='18'AND c='84')OR(n='52'AND c='73')OR(n='05'AND c='26')OR(n='01'AND c='88')OR(n='48'AND c='78')OR(n='80'AND c='78')OR(n='29'AND c='86')OR(n='88'AND c='11')OR(n='60'AND c='22')OR(n='59'AND c='88')OR(n='33'AND c='99')OR(n='26'AND c='50')OR(n='62'AND c='59')OR(n='76'AND c='23')OR(n='17'AND c='81')OR(n='44'AND c='99')OR(n='57'AND c='00')OR(n='57'AND c='07')OR(n='22'AND c='78')OR(n='36'AND c='52')OR(n='08'AND c='12')OR(n='87'AND c='34')OR(n='94'AND c='77')OR(n='72'AND c='33')OR(n='56'AND c='89')OR(n='36'AND c='37')OR(n='38'AND c='99')OR(n='02'AND c='01')OR(n='89'AND c='97')OR(n='25'AND c='91') + OR(n='79'AND c='67')OR(n='61'AND c='77')OR(n='50'AND c='75')OR(n='05'AND c='07')OR(n='15'AND c='86')OR(n='60'AND c='11')OR(n='54'AND c='19')OR(n='19'AND c='14')OR(n='29'AND c='92') + OR(n='26'AND c='06')OR(n='63'AND c='78')OR(n='24'AND c='67')OR(n='56'AND c='58')OR(n='33'AND c='58')OR(n='79'AND c='53')OR(n='89'AND c='44')OR(n='52'AND c='14')OR(n='28'AND c='94')OR(n='32'AND c='76')OR(n='99'AND c='87')OR(n='73'AND c='87')OR(n='44'AND c='43')OR(n='33'AND c='33')OR(n='72'AND c='14')OR(n='16'AND c='29')OR(n='26'AND c='86')OR(n='24'AND c='99')OR(n='68'AND c='26')OR(n='99'AND c='24')OR(n='32'AND c='09')OR(n='22'AND c='34')OR(n='36'AND c='93')OR(n='78'AND c='57')OR(n='40'AND c='55')OR(n='81'AND c='01')OR(n='39'AND c='19')OR(n='39'AND c='88')OR(n='52'AND c='11')OR(n='61'AND c='58')OR(n='84'AND c='32')OR(n='42'AND c='83')OR(n='48'AND c='00')OR(n='63'AND c='51')OR(n='28'AND c='18')OR(n='18'AND c='60')OR(n='64'AND c='05')OR(n='50'AND c='18')OR(n='10'AND c='88')OR(n='68'AND c='88')OR(n='14'AND c='08')OR(n='62'AND c='88')OR(n='95'AND c='26')OR(n='78'AND c='76')OR(n='73'AND c='91')OR(n='74'AND c='89')OR(n='60'AND c='55')OR(n='46'AND c='90')OR(n='69'AND c='89')OR(n='52'AND c='98')OR(n='38'AND c='65')OR(n='99'AND c='75')OR(n='53'AND c='88')OR(n='35'AND c='77')OR(n='84'AND c='50')OR(n='50'AND c='77')OR(n='67'AND c='06')OR(n='28'AND c='49')OR(n='43'AND c='55')OR(n='79'AND c='31')OR(n='06'AND c='88')OR(n='41'AND c='93')OR(n='20'AND c='11')OR(n='23'AND c='50')OR(n='08'AND c='55')OR(n='15'AND c='00')OR(n='91'AND c='46')OR(n='03'AND c='88')OR(n='81'AND c='48')OR(n='09'AND c='11')OR(n='00'AND c='88')OR(n='49'AND c='00')OR(n='76'AND c='32')OR(n='58'AND c='06')OR(n='18'AND c='99')OR(n='76'AND c='39')OR(n='14'AND c='33')OR(n='66'AND c='05')OR(n='28'AND c='99')OR(n='20'AND c='90')OR(n='07'AND c='11')OR(n='45'AND c='19')OR(n='44'AND c='19')OR(n='51'AND c='80') + OR(n='64'AND c='81')OR(n='36'AND c='19')OR(n='35'AND c='67')OR(n='35'AND c='19')OR(n='21'AND c='11')OR(n='07'AND c='19')OR(n='54'AND c='19')OR(n='42'AND c='55')OR(n='77'AND c='96')OR(n='39'AND c='33')OR(n='04'AND c='01')OR(n='45'AND c='50')OR(n='69'AND c='88')OR(n='87'AND c='53')OR(n='96'AND c='98')OR(n='28'AND c='11')OR(n='55'AND c='71')OR(n='78'AND c='11')OR(n='50'AND c='77')OR(n='47'AND c='11')OR(n='49'AND c='48')OR(n='56'AND c='77')OR(n='09'AND c='77')OR(n='17'AND c='65')OR(n='51'AND c='80')OR(n='30'AND c='22')OR(n='37'AND c='44')OR(n='88'AND c='62')OR(n='17'AND c='88')OR(n='12'AND c='13')OR(n='24'AND c='49')OR(n='36'AND c='11')OR(n='00'AND c='55')OR(n='61'AND c='04')OR(n='88'AND c='32')OR(n='82'AND c='58')OR(n='42'AND c='60')OR(n='76'AND c='99')OR(n='41'AND c='39')OR(n='91'AND c='57')OR(n='28'AND c='18')OR(n='86'AND c='71')OR(n='18'AND c='71')OR(n='26'AND c='98')OR(n='53'AND c='27')OR(n='05'AND c='67')OR(n='32'AND c='82')OR(n='33'AND c='69')OR(n='27'AND c='68')OR(n='64'AND c='89')OR(n='48'AND c='56')OR(n='80'AND c='91')OR(n='02'AND c='39')OR(n='45'AND c='26')OR(n='67'AND c='88')OR(n='52'AND c='56')OR(n='22'AND c='12')OR(n='31'AND c='93')OR(n='36'AND c='50')OR(n='50'AND c='72')OR(n='34'AND c='44')OR(n='10'AND c='26')OR(n='98'AND c='40')OR(n='16'AND c='88')OR(n='89'AND c='66')OR(n='84'AND c='76')OR(n='55'AND c='32')OR(n='06'AND c='89')OR(n='94'AND c='33')OR(n='40'AND c='76')OR(n='01'AND c='00')OR(n='53'AND c='44')OR(n='36'AND c='78')OR(n='96'AND c='89')OR(n='18'AND c='77')OR(n='03'AND c='66')OR(n='61'AND c='83')OR(n='21'AND c='83')OR(n='56'AND c='32')OR(n='58'AND c='74')OR(n='71'AND c='98')OR(n='66'AND c='26')OR(n='75'AND c='30')OR(n='14'AND c='21')OR(n='58'AND c='81')OR(n='39'AND c='07')OR(n='26'AND c='88')OR(n='94'AND c='40')OR(n='02'AND c='29')OR(n='41'AND c='37')OR(n='20'AND c='92')OR(n='49'AND c='27')OR(n='29'AND c='89')OR(n='39'AND c='98')OR(n='40'AND c='98') + OR(n='95'AND c='33')OR(n='58'AND c='98')OR(n='85'AND c='88')OR(n='61'AND c='40')OR(n='85'AND c='77')OR(n='90'AND c='77')OR(n='52'AND c='77')OR(n='54'AND c='66')OR(n='06'AND c='92')OR(n='89'AND c='14')OR(n='37'AND c='69')OR(n='49'AND c='30')OR(n='64'AND c='20')OR(n='77'AND c='40')OR(n='22'AND c='98')OR(n='21'AND c='88')OR(n='98'AND c='66')OR(n='08'AND c='86')OR(n='65'AND c='18')OR(n='08'AND c='08')OR(n='29'AND c='92')OR(n='33'AND c='77')OR(n='12'AND c='89')OR(n='76'AND c='31')OR(n='12'AND c='31')OR(n='13'AND c='44')OR(n='04'AND c='17')OR(n='79'AND c='88')OR(n='87'AND c='83')OR(n='40'AND c='22')OR(n='74'AND c='19')OR(n='55'AND c='88')OR(n='19'AND c='96')OR(n='45'AND c='88')OR(n='55'AND c='88')OR(n='32'AND c='45')OR(n='10'AND c='37')OR(n='16'AND c='12')OR(n='02'AND c='28')OR(n='24'AND c='37')OR(n='34'AND c='56')OR(n='35'AND c='05')OR(n='48'AND c='50')OR(n='80'AND c='51')OR(n='86'AND c='92')OR(n='73'AND c='72')OR(n='37'AND c='37')OR(n='20'AND c='12')OR(n='24'AND c='28')OR(n='53'AND c='37')OR(n='75'AND c='56')OR(n='65'AND c='05')OR(n='52'AND c='50')OR(n='66'AND c='51')OR(n='81'AND c='92')OR(n='58'AND c='72')OR(n='40'AND c='77')OR(n='93'AND c='77')OR(n='05'AND c='60')OR(n='37'AND c='33')OR(n='06'AND c='71')OR(n='40'AND c='68')OR(n='54'AND c='15')OR(n='03'AND c='11')OR(n='15'AND c='77')OR(n='70'AND c='62')OR(n='42'AND c='28')OR(n='57'AND c='11')OR(n='46'AND c='33')OR(n='83'AND c='17')OR(n='12'AND c='81')OR(n='69'AND c='11')OR(n='76'AND c='05')OR(n='57'AND c='19')OR(n='75'AND c='19')OR(n='06'AND c='19')OR(n='09'AND c='77')OR(n='20'AND c='48')OR(n='45'AND c='80')OR(n='38'AND c='65')OR(n='13'AND c='19')OR(n='42'AND c='19')OR(n='55'AND c='11')OR(n='94'AND c='80')OR(n='59'AND c='77')OR(n='50'AND c='48')OR(n='39'AND c='80')OR(n='24'AND c='65')OR(n='21'AND c='77')OR(n='27'AND c='48')OR(n='92'AND c='80')OR(n='11'AND c='65')OR(n='59'AND c='19')OR(n='18'AND c='19') + OR(n='26'AND c='19')OR(n='15'AND c='48')OR(n='21'AND c='80')OR(n='54'AND c='77')OR(n='74'AND c='44')OR(n='00'AND c='48')OR(n='97'AND c='80')OR(n='72'AND c='77')OR(n='67'AND c='65')OR(n='33'AND c='48')OR(n='03'AND c='80')OR(n='37'AND c='77')OR(n='24'AND c='65')OR(n='05'AND c='15')OR(n='81'AND c='22')OR(n='47'AND c='71')OR(n='14'AND c='78')OR(n='11'AND c='99')OR(n='74'AND c='86')OR(n='37'AND c='95')OR(n='18'AND c='56')OR(n='30'AND c='90')OR(n='09'AND c='03')OR(n='55'AND c='53')OR(n='17'AND c='99')OR(n='83'AND c='81')OR(n='47'AND c='52')OR(n='99'AND c='97')OR(n='79'AND c='34')OR(n='02'AND c='22')OR(n='87'AND c='19')OR(n='03'AND c='98')OR(n='32'AND c='61')OR(n='36'AND c='71')OR(n='85'AND c='71')OR(n='54'AND c='71')OR(n='06'AND c='22')OR(n='43'AND c='30')OR(n='90'AND c='22')OR(n='15'AND c='05')OR(n='37'AND c='56')OR(n='24'AND c='00')OR(n='88'AND c='27')OR(n='37'AND c='27')OR(n='70'AND c='85')OR(n='04'AND c='85')OR(n='31'AND c='85')OR(n='25'AND c='85')OR(n='91'AND c='51')OR(n='57'AND c='85')OR(n='25'AND c='77')OR(n='67'AND c='87')OR(n='30'AND c='55')OR(n='46'AND c='48')OR(n='47'AND c='77')OR(n='34'AND c='15')OR(n='08'AND c='55')OR(n='05'AND c='77')OR(n='04'AND c='35')OR(n='64'AND c='55')OR(n='16'AND c='35')OR(n='84'AND c='73')OR(n='90'AND c='17')OR(n='68'AND c='08')OR(n='32'AND c='98')OR(n='82'AND c='27')OR(n='82'AND c='78')OR(n='65'AND c='66')OR(n='35'AND c='30')OR(n='60'AND c='45')OR(n='56'AND c='94')OR(n='42'AND c='02')OR(n='84'AND c='14')OR(n='01'AND c='80')OR(n='15'AND c='97')OR(n='18'AND c='85')OR(n='12'AND c='78')OR(n='50'AND c='85')OR(n='10'AND c='18')OR(n='11'AND c='96')OR(n='62'AND c='18')OR(n='49'AND c='85')OR(n='41'AND c='00')OR(n='33'AND c='82')OR(n='46'AND c='73')OR(n='95'AND c='71')OR(n='48'AND c='88')OR(n='82'AND c='73')OR(n='23'AND c='36')OR(n='43'AND c='67')OR(n='36'AND c='88')OR(n='20'AND c='78')OR(n='99'AND c='43')OR(n='66'AND c='68') + OR(n='82'AND c='09')OR(n='37'AND c='58')OR(n='24'AND c='13')OR(n='03'AND c='52')OR(n='47'AND c='22')OR(n='03'AND c='37')OR(n='46'AND c='76')OR(n='00'AND c='77')OR(n='04'AND c='19')OR(n='63'AND c='08')OR(n='15'AND c='90')OR(n='52'AND c='45')OR(n='18'AND c='05')OR(n='48'AND c='28')OR(n='84'AND c='00')OR(n='22'AND c='19')OR(n='32'AND c='89')OR(n='97'AND c='44')OR(n='74'AND c='89')OR(n='36'AND c='89')OR(n='21'AND c='55')OR(n='88'AND c='97')OR(n='34'AND c='86')OR(n='39'AND c='15')OR(n='00'AND c='19')OR(n='10'AND c='55')OR(n='21'AND c='70')OR(n='96'AND c='06')OR(n='88'AND c='58')OR(n='10'AND c='81')OR(n='39'AND c='99')OR(n='10'AND c='68')OR(n='85'AND c='28')OR(n='08'AND c='65')OR(n='97'AND c='65')OR(n='15'AND c='33')OR(n='85'AND c='55')OR(n='98'AND c='00')OR(n='67'AND c='78')OR(n='15'AND c='08')OR(n='94'AND c='96')OR(n='79'AND c='81')OR(n='36'AND c='10')OR(n='50'AND c='53')OR(n='61'AND c='98')OR(n='74'AND c='56')OR(n='38'AND c='44')OR(n='39'AND c='99')OR(n='78'AND c='92')OR(n='79'AND c='33')OR(n='07'AND c='75')OR(n='05'AND c='38')OR(n='04'AND c='31')OR(n='72'AND c='88')OR(n='22'AND c='36')OR(n='27'AND c='99')OR(n='59'AND c='99')OR(n='44'AND c='87')OR(n='42'AND c='03')OR(n='52'AND c='87')OR(n='42'AND c='91')OR(n='38'AND c='02')OR(n='19'AND c='91')OR(n='43'AND c='04')OR(n='30'AND c='83')OR(n='36'AND c='91')OR(n='09'AND c='95')OR(n='44'AND c='45')OR(n='32'AND c='05')OR(n='06'AND c='38')OR(n='40'AND c='37')OR(n='59'AND c='38')OR(n='39'AND c='37')OR(n='05'AND c='38')OR(n='09'AND c='38')OR(n='94'AND c='37')OR(n='88'AND c='38')OR(n='60'AND c='29')OR(n='52'AND c='38')OR(n='87'AND c='38')OR(n='56'AND c='37')OR(n='50'AND c='38')OR(n='30'AND c='37')OR(n='38'AND c='38')OR(n='24'AND c='29')OR(n='17'AND c='38')OR(n='91'AND c='26')OR(n='80'AND c='38')OR(n='34'AND c='29') + OR(n='68'AND c='38')OR(n='91'AND c='29')OR(n='23'AND c='29')OR(n='20'AND c='38')OR(n='83'AND c='29')OR(n='77'AND c='38')OR(n='61'AND c='38')OR(n='55'AND c='37')OR(n='39'AND c='38')OR(n='19'AND c='37')OR(n='41'AND c='38')OR(n='02'AND c='37')OR(n='40'AND c='38')OR(n='12'AND c='29')OR(n='01'AND c='38')OR(n='59'AND c='38')OR(n='76'AND c='37')OR(n='81'AND c='38')OR(n='54'AND c='37')OR(n='42'AND c='07')OR(n='38'AND c='03')OR(n='99'AND c='72')OR(n='79'AND c='19')OR(n='27'AND c='19')OR(n='31'AND c='19')OR(n='81'AND c='19')OR(n='66'AND c='19')OR(n='80'AND c='82')OR(n='69'AND c='27')OR(n='09'AND c='06')OR(n='97'AND c='87')OR(n='15'AND c='15')OR(n='06'AND c='15')OR(n='27'AND c='15')OR(n='04'AND c='15')OR(n='64'AND c='06')OR(n='14'AND c='15')OR(n='64'AND c='76')OR(n='39'AND c='59')OR(n='07'AND c='40')OR(n='51'AND c='68')OR(n='39'AND c='00')OR(n='35'AND c='33')OR(n='89'AND c='19')OR(n='16'AND c='00')OR(n='44'AND c='58')OR(n='76'AND c='18')OR(n='11'AND c='66')OR(n='27'AND c='87')OR(n='38'AND c='38')OR(n='57'AND c='55')OR(n='88'AND c='23')OR(n='24'AND c='29')OR(n='60'AND c='75')OR(n='81'AND c='27')OR(n='21'AND c='34')OR(n='11'AND c='96')OR(n='53'AND c='33')OR(n='26'AND c='89')OR(n='41'AND c='98')OR(n='59'AND c='98')OR(n='31'AND c='08')OR(n='29'AND c='70')OR(n='86'AND c='22')OR(n='18'AND c='96')OR(n='78'AND c='19')OR(n='32'AND c='81')OR(n='05'AND c='93')OR(n='59'AND c='65')OR(n='51'AND c='76')OR(n='87'AND c='15')OR(n='86'AND c='88')OR(n='18'AND c='15')OR(n='91'AND c='15')OR(n='43'AND c='48')OR(n='02'AND c='65')OR(n='21'AND c='14')OR(n='63'AND c='85')OR(n='50'AND c='48')OR(n='54'AND c='65')OR(n='84'AND c='15')OR(n='89'AND c='78')OR(n='95'AND c='60')OR(n='10'AND c='78')OR(n='42'AND c='87')OR(n='02'AND c='50')OR(n='82'AND c='78')OR(n='72'AND c='85')OR(n='24'AND c='77') + OR(n='43'AND c='98')OR(n='57'AND c='00')OR(n='23'AND c='00')OR(n='64'AND c='23')OR(n='15'AND c='83')OR(n='60'AND c='66')OR(n='40'AND c='88')OR(n='60'AND c='78')OR(n='92'AND c='51')OR(n='08'AND c='77')OR(n='13'AND c='99')OR(n='20'AND c='00')OR(n='23'AND c='89')OR(n='32'AND c='44')OR(n='71'AND c='67')OR(n='95'AND c='67')OR(n='28'AND c='31')OR(n='56'AND c='35')OR(n='02'AND c='36')OR(n='59'AND c='93')OR(n='31'AND c='92')OR(n='70'AND c='09')OR(n='37'AND c='44')OR(n='42'AND c='20')OR(n='16'AND c='13')OR(n='91'AND c='82')OR(n='20'AND c='37')OR(n='39'AND c='51')OR(n='04'AND c='85')OR(n='20'AND c='48')OR(n='65'AND c='11')OR(n='71'AND c='02')OR(n='82'AND c='52')OR(n='55'AND c='24')OR(n='74'AND c='55')OR(n='79'AND c='80')OR(n='31'AND c='33')OR(n='09'AND c='99')OR(n='89'AND c='98')OR(n='79'AND c='98')OR(n='54'AND c='37')OR(n='43'AND c='61')OR(n='13'AND c='53')OR(n='95'AND c='90')OR(n='17'AND c='70')OR(n='94'AND c='33')OR(n='25'AND c='26')OR(n='22'AND c='21')OR(n='06'AND c='03')OR(n='64'AND c='30')OR(n='37'AND c='00')OR(n='61'AND c='45')OR(n='80'AND c='38')OR(n='52'AND c='34')OR(n='84'AND c='29')OR(n='35'AND c='99')OR(n='30'AND c='99')OR(n='60'AND c='99')OR(n='38'AND c='89')OR(n='50'AND c='99')OR(n='16'AND c='18')OR(n='99'AND c='99')OR(n='14'AND c='88')OR(n='55'AND c='56')OR(n='11'AND c='82')OR(n='13'AND c='82')OR(n='74'AND c='78')OR(n='47'AND c='37')OR(n='19'AND c='48')OR(n='83'AND c='65')OR(n='09'AND c='99')OR(n='63'AND c='99')OR(n='64'AND c='99')OR(n='27'AND c='99')OR(n='59'AND c='99')OR(n='02'AND c='48')OR(n='29'AND c='65')OR(n='68'AND c='22')OR(n='34'AND c='48')OR(n='62'AND c='65')OR(n='52'AND c='48')OR(n='90'AND c='65')OR(n='22'AND c='48')OR(n='49'AND c='65')OR(n='97'AND c='48')OR(n='94'AND c='65')OR(n='67'AND c='91')OR(n='15'AND c='97')OR(n='96'AND c='94') + OR(n='66'AND c='15')OR(n='58'AND c='15')OR(n='27'AND c='11')OR(n='17'AND c='88')OR(n='81'AND c='27')OR(n='31'AND c='16')OR(n='00'AND c='19')OR(n='93'AND c='48')OR(n='43'AND c='65')OR(n='72'AND c='97')OR(n='09'AND c='48')OR(n='38'AND c='65')OR(n='52'AND c='11')OR(n='89'AND c='89')OR(n='24'AND c='53')OR(n='14'AND c='18')OR(n='72'AND c='80')OR(n='52'AND c='28')OR(n='01'AND c='26')OR(n='97'AND c='66')OR(n='34'AND c='00')OR(n='29'AND c='88')OR(n='19'AND c='90')OR(n='73'AND c='65')OR(n='67'AND c='71')OR(n='50'AND c='19')OR(n='92'AND c='67')OR(n='60'AND c='48')OR(n='36'AND c='65')OR(n='01'AND c='41')OR(n='19'AND c='78')OR(n='39'AND c='55')OR(n='26'AND c='66')OR(n='18'AND c='78')OR(n='30'AND c='71')OR(n='62'AND c='54')OR(n='41'AND c='66')OR(n='45'AND c='83')OR(n='62'AND c='72')OR(n='08'AND c='06')OR(n='09'AND c='88')OR(n='23'AND c='19')OR(n='90'AND c='46')OR(n='63'AND c='15')OR(n='70'AND c='89')OR(n='60'AND c='50')OR(n='68'AND c='28')OR(n='66'AND c='99')OR(n='56'AND c='93')OR(n='18'AND c='71')OR(n='94'AND c='29')OR(n='90'AND c='54')OR(n='94'AND c='71')OR(n='47'AND c='29')OR(n='96'AND c='71')OR(n='11'AND c='71')OR(n='87'AND c='37')OR(n='03'AND c='71') + OR(n='97'AND c='71')OR(n='64'AND c='71')OR(n='10'AND c='66')OR(n='24'AND c='71')OR(n='25'AND c='71')OR(n='69'AND c='71')OR(n='57'AND c='22')OR(n='12'AND c='71')OR(n='27'AND c='71')OR(n='41'AND c='71')OR(n='12'AND c='71')OR(n='15'AND c='29')OR(n='46'AND c='71')OR(n='00'AND c='85')OR(n='96'AND c='71')OR(n='42'AND c='82')OR(n='05'AND c='71')OR(n='45'AND c='71')OR(n='19'AND c='37')OR(n='55'AND c='71')OR(n='32'AND c='29')OR(n='70'AND c='71')OR(n='56'AND c='71')OR(n='38'AND c='71')OR(n='47'AND c='71')OR(n='24'AND c='37')OR(n='30'AND c='71')OR(n='86'AND c='28')OR(n='28'AND c='58')OR(n='09'AND c='33')OR(n='55'AND c='12')OR(n='37'AND c='32')OR(n='48'AND c='29')OR(n='08'AND c='38')OR(n='42'AND c='11')OR(n='65'AND c='57')OR(n='18'AND c='57')OR(n='42'AND c='08')OR(n='35'AND c='11')OR(n='19'AND c='00')OR(n='84'AND c='52')OR(n='20'AND c='59')OR(n='84'AND c='65')OR(n='01'AND c='59')OR(n='87'AND c='65')OR(n='98'AND c='65')OR(n='87'AND c='59')OR(n='58'AND c='24')OR(n='10'AND c='99')OR(n='50'AND c='57')OR(n='92'AND c='88')OR(n='69'AND c='20')OR(n='47'AND c='44')OR(n='71'AND c='44')OR(n='28'AND c='18')OR(n='12'AND c='18')OR(n='24'AND c='88')OR(n='84'AND c='16')OR(n='19'AND c='37')OR(n='01'AND c='34')OR(n='07'AND c='73')OR(n='48'AND c='02')OR(n='64'AND c='73')OR(n='23'AND c='73')OR(n='95'AND c='05')OR(n='42'AND c='05')OR(n='67'AND c='12')OR(n='02'AND c='13')OR(n='49'AND c='87')OR(n='23'AND c='53')OR(n='97'AND c='24')OR(n='47'AND c='98')OR(n='96'AND c='10')OR(n='17'AND c='78')OR(n='02'AND c='99')OR(n='21'AND c='69')OR(n='76'AND c='33')OR(n='41'AND c='18')OR(n='97'AND c='32')OR(n='40'AND c='97')OR(n='48'AND c='95')OR(n='26'AND c='95')OR(n='28'AND c='88')OR(n='51'AND c='08')OR(n='13'AND c='37')OR(n='43'AND c='83')OR(n='54'AND c='20')OR(n='98'AND c='34')OR(n='32'AND c='93')OR(n='63'AND c='96')OR(n='91'AND c='16')OR(n='24'AND c='16')OR(n='68'AND c='10')OR(n='76'AND c='88')OR(n='62'AND c='68')OR(n='49'AND c='76')OR(n='10'AND c='76')OR(n='95'AND c='62')OR(n='42'AND c='91')OR(n='98'AND c='22')OR(n='55'AND c='19')OR(n='45'AND c='66')OR(n='19'AND c='56')OR(n='54'AND c='98')OR(n='86'AND c='76')OR(n='92'AND c='63')OR(n='89'AND c='11')OR(n='63'AND c='90')OR(n='85'AND c='40')OR(n='01'AND c='67')OR(n='88'AND c='19')OR(n='26'AND c='11')OR(n='62'AND c='79')OR(n='74'AND c='81') + OR(n='47'AND c='55')OR(n='70'AND c='05')OR(n='82'AND c='20')OR(n='10'AND c='47')OR(n='29'AND c='59')OR(n='93'AND c='89')OR(n='06'AND c='88')OR(n='34'AND c='30')OR(n='23'AND c='55')OR(n='57'AND c='57')OR(n='43'AND c='86')OR(n='06'AND c='86')OR(n='28'AND c='86')OR(n='44'AND c='86')OR(n='20'AND c='86')OR(n='96'AND c='48')OR(n='19'AND c='48')OR(n='06'AND c='44')OR(n='82'AND c='68')OR(n='74'AND c='23')OR(n='60'AND c='86')OR(n='05'AND c='10')OR(n='61'AND c='93')OR(n='22'AND c='29')OR(n='22'AND c='92')OR(n='63'AND c='20')OR(n='47'AND c='19')OR(n='16'AND c='07')OR(n='02'AND c='18')OR(n='13'AND c='77')OR(n='35'AND c='11')OR(n='76'AND c='17')OR(n='87'AND c='11')OR(n='93'AND c='90')OR(n='14'AND c='71')OR(n='35'AND c='30')OR(n='37'AND c='44')OR(n='33'AND c='61')OR(n='05'AND c='78')OR(n='25'AND c='88')OR(n='55'AND c='18')OR(n='67'AND c='66')OR(n='55'AND c='68')OR(n='92'AND c='78')OR(n='30'AND c='15')OR(n='14'AND c='15')OR(n='60'AND c='15')OR(n='18'AND c='15')OR(n='64'AND c='89')OR(n='56'AND c='50')OR(n='34'AND c='86')OR(n='47'AND c='22')OR(n='42'AND c='44')OR(n='05'AND c='56')OR(n='67'AND c='82')OR(n='90'AND c='55')OR(n='58'AND c='55')OR(n='01'AND c='55')OR(n='18'AND c='41')OR(n='66'AND c='37')OR(n='89'AND c='97')OR(n='38'AND c='66')OR(n='16'AND c='15')OR(n='29'AND c='15')OR(n='29'AND c='15')OR(n='26'AND c='15')OR(n='04'AND c='19')OR(n='96'AND c='96')OR(n='56'AND c='66')OR(n='08'AND c='48')OR(n='13'AND c='48')OR(n='88'AND c='48')OR(n='67'AND c='69')OR(n='49'AND c='26')OR(n='14'AND c='64')OR(n='39'AND c='48')OR(n='18'AND c='53')OR(n='34'AND c='80')OR(n='43'AND c='17')OR(n='22'AND c='26')OR(n='03'AND c='65')OR(n='50'AND c='85')OR(n='98'AND c='11')OR(n='65'AND c='90')OR(n='54'AND c='97')OR(n='13'AND c='77')OR(n='59'AND c='82')OR(n='82'AND c='98')OR(n='98'AND c='60')OR(n='22'AND c='26')OR(n='83'AND c='11')OR(n='51'AND c='57')OR(n='82'AND c='33')OR(n='08'AND c='32')OR(n='94'AND c='13')OR(n='89'AND c='45')OR(n='32'AND c='75')OR(n='40'AND c='32')OR(n='82'AND c='67')OR(n='13'AND c='98')OR(n='61'AND c='30')OR(n='50'AND c='00')OR(n='78'AND c='60')OR(n='35'AND c='77')OR(n='03'AND c='26')OR(n='84'AND c='45')OR(n='13'AND c='19')OR(n='27'AND c='45')OR(n='01'AND c='19')OR(n='66'AND c='45')OR(n='33'AND c='45')OR(n='51'AND c='19')OR(n='70'AND c='45')OR(n='65'AND c='18')OR(n='68'AND c='19')OR(n='69'AND c='19')OR(n='21'AND c='23')OR(n='95'AND c='45')OR(n='78'AND c='19')OR(n='37'AND c='45')OR(n='16'AND c='50')OR(n='75'AND c='19')OR(n='54'AND c='23')OR(n='56'AND c='22')OR(n='88'AND c='28')OR(n='21'AND c='53')OR(n='32'AND c='23')OR(n='97'AND c='71') + OR(n='76'AND c='69')OR(n='73'AND c='92')OR(n='50'AND c='65')OR(n='34'AND c='88')OR(n='91'AND c='99')OR(n='51'AND c='90')OR(n='97'AND c='93')OR(n='84'AND c='48')OR(n='50'AND c='88')OR(n='65'AND c='35')OR(n='87'AND c='68')OR(n='19'AND c='16')OR(n='99'AND c='71')OR(n='98'AND c='00')OR(n='73'AND c='52')OR(n='66'AND c='16')OR(n='32'AND c='10')OR(n='71'AND c='44')OR(n='01'AND c='55')OR(n='60'AND c='00')OR(n='10'AND c='16')OR(n='03'AND c='88')OR(n='04'AND c='52')OR(n='59'AND c='57')OR(n='00'AND c='99')OR(n='63'AND c='23')OR(n='78'AND c='07')OR(n='90'AND c='87')OR(n='95'AND c='96')OR(n='52'AND c='18')OR(n='61'AND c='99')OR(n='65'AND c='66')OR(n='66'AND c='87')OR(n='24'AND c='80')OR(n='88'AND c='77')OR(n='05'AND c='01')OR(n='35'AND c='11')OR(n='59'AND c='80')OR(n='04'AND c='62')OR(n='53'AND c='88')OR(n='78'AND c='07')OR(n='94'AND c='59')OR(n='12'AND c='79')OR(n='00'AND c='88')OR(n='50'AND c='85')OR(n='66'AND c='55')OR(n='15'AND c='18')OR(n='89'AND c='28')OR(n='77'AND c='69')OR(n='70'AND c='87')OR(n='48'AND c='26')OR(n='55'AND c='03')OR(n='87'AND c='67')OR(n='05'AND c='82')OR(n='64'AND c='15')OR(n='05'AND c='49')OR(n='80'AND c='66')OR(n='60'AND c='78')OR(n='93'AND c='02')OR(n='25'AND c='91')OR(n='65'AND c='00')OR(n='80'AND c='91')OR(n='45'AND c='30')OR(n='68'AND c='98')OR(n='40'AND c='88')OR(n='37'AND c='19')OR(n='24'AND c='62')OR(n='71'AND c='48')OR(n='27'AND c='78')OR(n='56'AND c='21')OR(n='49'AND c='00')OR(n='05'AND c='77')OR(n='41'AND c='57')OR(n='67'AND c='99')OR(n='15'AND c='41')OR(n='68'AND c='70')OR(n='98'AND c='49')OR(n='90'AND c='03')OR(n='92'AND c='41')OR(n='59'AND c='00')OR(n='16'AND c='89')OR(n='63'AND c='76')OR(n='23'AND c='90')OR(n='76'AND c='91')OR(n='41'AND c='06')OR(n='08'AND c='44')OR(n='11'AND c='44')OR(n='46'AND c='91')OR(n='17'AND c='91')OR(n='17'AND c='91')OR(n='74'AND c='72')OR(n='05'AND c='72')OR(n='03'AND c='52')OR(n='52'AND c='81')OR(n='42'AND c='91')OR(n='17'AND c='91')OR(n='54'AND c='60')OR(n='02'AND c='67')OR(n='72'AND c='32')OR(n='71'AND c='91')OR(n='34'AND c='91')OR(n='34'AND c='88')OR(n='42'AND c='60')OR(n='58'AND c='21')OR(n='16'AND c='70')OR(n='18'AND c='38')OR(n='84'AND c='05')OR(n='10'AND c='11')OR(n='85'AND c='85')OR(n='90'AND c='09')OR(n='06'AND c='45')OR(n='01'AND c='21')OR(n='38'AND c='76')OR(n='92'AND c='02')OR(n='01'AND c='66')OR(n='42'AND c='26')OR(n='17'AND c='58')OR(n='07'AND c='12')OR(n='31'AND c='85')OR(n='34'AND c='87')OR(n='24'AND c='19')OR(n='94'AND c='19')OR(n='44'AND c='55')OR(n='97'AND c='47')OR(n='93'AND c='55')OR(n='78'AND c='62')OR(n='36'AND c='95')OR(n='25'AND c='89')OR(n='38'AND c='27')OR(n='72'AND c='32')OR(n='42'AND c='77')OR(n='41'AND c='09')OR(n='25'AND c='27')OR(n='20'AND c='99')OR(n='66'AND c='33')OR(n='13'AND c='65')OR(n='68'AND c='85')OR(n='79'AND c='87')OR(n='01'AND c='08')OR(n='20'AND c='19')OR(n='34'AND c='19')OR(n='24'AND c='19')OR(n='61'AND c='19')OR(n='71'AND c='19')OR(n='86'AND c='19')OR(n='93'AND c='19')OR(n='65'AND c='78')OR(n='25'AND c='55')OR(n='71'AND c='16')OR(n='10'AND c='68')OR(n='44'AND c='98')OR(n='35'AND c='86')OR(n='54'AND c='48')OR(n='11'AND c='82')OR(n='11'AND c='68')OR(n='71'AND c='98')OR(n='44'AND c='55')OR(n='06'AND c='16')OR(n='18'AND c='86')OR(n='31'AND c='48')OR(n='47'AND c='55')OR(n='18'AND c='16')OR(n='89'AND c='98')OR(n='19'AND c='68')OR(n='31'AND c='86')OR(n='65'AND c='48')OR(n='15'AND c='55')OR(n='14'AND c='16')OR(n='39'AND c='98')OR(n='64'AND c='68')OR(n='39'AND c='86')OR(n='34'AND c='48')OR(n='09'AND c='19')OR(n='60'AND c='19')OR(n='94'AND c='19')OR(n='20'AND c='55')OR(n='59'AND c='98')OR(n='82'AND c='16')OR(n='63'AND c='68')OR(n='75'AND c='86')OR(n='41'AND c='48')OR(n='59'AND c='39')OR(n='47'AND c='74')OR(n='60'AND c='87')OR(n='51'AND c='99')OR(n='56'AND c='00')OR(n='35'AND c='32')OR(n='25'AND c='19')OR(n='16'AND c='19')OR(n='20'AND c='38')OR(n='02'AND c='86')OR(n='18'AND c='82')OR(n='16'AND c='68')OR(n='99'AND c='11')OR(n='64'AND c='22')OR(n='84'AND c='66')OR(n='74'AND c='81')OR(n='18'AND c='88')OR(n='69'AND c='19')OR(n='00'AND c='19')OR(n='74'AND c='19')OR(n='05'AND c='88')OR(n='55'AND c='19')OR(n='63'AND c='19')OR(n='45'AND c='19')OR(n='71'AND c='99')OR(n='08'AND c='19')OR(n='82'AND c='19')OR(n='21'AND c='88')OR(n='17'AND c='19')OR(n='33'AND c='19')OR(n='54'AND c='77')OR(n='11'AND c='06')OR(n='32'AND c='77')OR(n='14'AND c='76')OR(n='68'AND c='84')OR(n='63'AND c='60')OR(n='22'AND c='11')OR(n='02'AND c='68') + OR(n='37'AND c='86')OR(n='97'AND c='56')OR(n='14'AND c='78')OR(n='54'AND c='53')OR(n='37'AND c='88')OR(n='60'AND c='01')OR(n='85'AND c='66')OR(n='03'AND c='11')OR(n='41'AND c='66')OR(n='53'AND c='26')OR(n='86'AND c='92')OR(n='23'AND c='82')OR(n='20'AND c='45')OR(n='40'AND c='68')OR(n='64'AND c='28')OR(n='46'AND c='66')OR(n='59'AND c='73')OR(n='74'AND c='68')OR(n='15'AND c='55')OR(n='82'AND c='92')OR(n='58'AND c='70')OR(n='37'AND c='88')OR(n='57'AND c='84')OR(n='89'AND c='89')OR(n='14'AND c='58')OR(n='86'AND c='08')OR(n='23'AND c='98')OR(n='90'AND c='22')OR(n='56'AND c='88')OR(n='93'AND c='12')OR(n='03'AND c='85')OR(n='17'AND c='44')OR(n='38'AND c='89')OR(n='44'AND c='88')OR(n='89'AND c='36')OR(n='52'AND c='21')OR(n='58'AND c='15')OR(n='07'AND c='98')OR(n='41'AND c='70')OR(n='01'AND c='99')OR(n='38'AND c='45')OR(n='00'AND c='02')OR(n='32'AND c='03')OR(n='45'AND c='89')OR(n='49'AND c='86')OR(n='26'AND c='78')OR(n='72'AND c='39')OR(n='81'AND c='91')OR(n='04'AND c='78')OR(n='53'AND c='88')OR(n='01'AND c='48')OR(n='21'AND c='62')OR(n='94'AND c='08')OR(n='06'AND c='70')OR(n='37'AND c='70')OR(n='31'AND c='06')OR(n='87'AND c='32')OR(n='02'AND c='99')OR(n='84'AND c='15')OR(n='83'AND c='15')OR(n='87'AND c='66')OR(n='17'AND c='69')OR(n='62'AND c='85')OR(n='70'AND c='43')OR(n='61'AND c='22')OR(n='17'AND c='82')OR(n='04'AND c='77')OR(n='35'AND c='61')OR(n='42'AND c='08')OR(n='46'AND c='32')OR(n='20'AND c='22')OR(n='17'AND c='13')OR(n='47'AND c='85')OR(n='11'AND c='43')OR(n='63'AND c='75')OR(n='54'AND c='60')OR(n='01'AND c='97')OR(n='71'AND c='45')OR(n='68'AND c='99')OR(n='70'AND c='88')OR(n='64'AND c='19')OR(n='99'AND c='19')OR(n='78'AND c='19')OR(n='41'AND c='19')OR(n='67'AND c='48')OR(n='21'AND c='04')OR(n='49'AND c='15')OR(n='28'AND c='19')OR(n='91'AND c='48')OR(n='56'AND c='04')OR(n='36'AND c='15')OR(n='61'AND c='19')OR(n='45'AND c='48')OR(n='08'AND c='04')OR(n='10'AND c='15')OR(n='93'AND c='19')OR(n='74'AND c='48')OR(n='23'AND c='04')OR(n='80'AND c='15')OR(n='13'AND c='19')OR(n='35'AND c='48')OR(n='36'AND c='04')OR(n='51'AND c='15')OR(n='20'AND c='82')OR(n='32'AND c='19')OR(n='34'AND c='48')OR(n='11'AND c='04')OR(n='20'AND c='15')OR(n='40'AND c='82')OR(n='89'AND c='19')OR(n='51'AND c='69')OR(n='91'AND c='69')OR(n='72'AND c='17')OR(n='20'AND c='98')OR(n='76'AND c='80')OR(n='20'AND c='91')OR(n='99'AND c='69')OR(n='13'AND c='15')OR(n='88'AND c='67')OR(n='53'AND c='98')OR(n='88'AND c='08')OR(n='48'AND c='58')OR(n='94'AND c='70')OR(n='57'AND c='40')OR(n='32'AND c='40')OR(n='92'AND c='40')OR(n='22'AND c='16')OR(n='38'AND c='75')OR(n='32'AND c='99')OR(n='10'AND c='16')OR(n='86'AND c='75')OR(n='19'AND c='99')OR(n='52'AND c='40')OR(n='11'AND c='65')OR(n='53'AND c='65')OR(n='04'AND c='99')OR(n='70'AND c='44')OR(n='37'AND c='87')OR(n='27'AND c='87')OR(n='28'AND c='30')OR(n='34'AND c='34')OR(n='69'AND c='00')OR(n='56'AND c='18')OR(n='28'AND c='34')OR(n='14'AND c='98')OR(n='51'AND c='02')OR(n='39'AND c='51')OR(n='66'AND c='19')OR(n='70'AND c='69')OR(n='64'AND c='78')OR(n='10'AND c='45')OR(n='15'AND c='80')OR(n='78'AND c='78')OR(n='51'AND c='30')OR(n='11'AND c='90')OR(n='87'AND c='20')OR(n='97'AND c='28')OR(n='78'AND c='69')OR(n='58'AND c='88')OR(n='74'AND c='79')OR(n='14'AND c='99')OR(n='63'AND c='99')OR(n='41'AND c='93')OR(n='83'AND c='72')OR(n='52'AND c='77')OR(n='45'AND c='66')OR(n='75'AND c='31')OR(n='81'AND c='27')OR(n='24'AND c='10')OR(n='88'AND c='26')OR(n='25'AND c='68')OR(n='60'AND c='60')OR(n='18'AND c='65')OR(n='55'AND c='16')OR(n='32'AND c='88')OR(n='36'AND c='39')OR(n='27'AND c='01')OR(n='07'AND c='67')OR(n='42'AND c='66')OR(n='03'AND c='65')OR(n='12'AND c='11')OR(n='42'AND c='91')OR(n='77'AND c='13')OR(n='99'AND c='38') + OR(n='33'AND c='93')OR(n='46'AND c='92')OR(n='77'AND c='53')OR(n='20'AND c='00')OR(n='58'AND c='97')OR(n='04'AND c='09')OR(n='42'AND c='51')OR(n='93'AND c='56')OR(n='58'AND c='06')OR(n='38'AND c='08')OR(n='46'AND c='12')OR(n='23'AND c='77')OR(n='17'AND c='33')OR(n='28'AND c='90')OR(n='49'AND c='16')OR(n='47'AND c='21')OR(n='04'AND c='25')OR(n='09'AND c='16')OR(n='38'AND c='95')OR(n='13'AND c='88')OR(n='67'AND c='88')OR(n='31'AND c='30')OR(n='58'AND c='23')OR(n='16'AND c='78')OR(n='94'AND c='99')OR(n='88'AND c='99')OR(n='89'AND c='39')OR(n='42'AND c='87')OR(n='42'AND c='99')OR(n='06'AND c='39')OR(n='26'AND c='20')OR(n='49'AND c='56')OR(n='22'AND c='79')OR(n='21'AND c='05')OR(n='08'AND c='09')OR(n='15'AND c='50')OR(n='99'AND c='60')OR(n='90'AND c='22')OR(n='64'AND c='34')OR(n='35'AND c='27')OR(n='95'AND c='91')OR(n='22'AND c='83')OR(n='23'AND c='45')OR(n='79'AND c='07')OR(n='30'AND c='69')OR(n='59'AND c='11')OR(n='46'AND c='44')OR(n='05'AND c='18')OR(n='42'AND c='81')OR(n='95'AND c='43')OR(n='64'AND c='30')OR(n='69'AND c='25')OR(n='43'AND c='61')OR(n='38'AND c='88')OR(n='98'AND c='91')OR(n='37'AND c='22')OR(n='13'AND c='06')OR(n='02'AND c='02')OR(n='39'AND c='50')OR(n='20'AND c='10')OR(n='69'AND c='26')OR(n='10'AND c='88')OR(n='25'AND c='19')OR(n='60'AND c='66')OR(n='59'AND c='11')OR(n='24'AND c='88')OR(n='89'AND c='99')OR(n='99'AND c='12')OR(n='16'AND c='08')OR(n='73'AND c='99')OR(n='34'AND c='88')OR(n='61'AND c='18')OR(n='37'AND c='50')OR(n='53'AND c='79')OR(n='30'AND c='30')OR(n='87'AND c='37')OR(n='12'AND c='26')OR(n='93'AND c='03')OR(n='25'AND c='88')OR(n='64'AND c='83')OR(n='26'AND c='13')OR(n='83'AND c='00')OR(n='37'AND c='70')OR(n='46'AND c='09')OR(n='05'AND c='10')OR(n='88'AND c='08')OR(n='45'AND c='11')OR(n='73'AND c='11')OR(n='14'AND c='66')OR(n='08'AND c='47')OR(n='50'AND c='44')OR(n='71'AND c='36')OR(n='47'AND c='88')OR(n='72'AND c='59')OR(n='00'AND c='67')OR(n='54'AND c='18')OR(n='64'AND c='58')OR(n='05'AND c='51')OR(n='77'AND c='39')OR(n='88'AND c='66')OR(n='62'AND c='77')OR(n='32'AND c='88')OR(n='19'AND c='38')OR(n='32'AND c='85')OR(n='97'AND c='17')OR(n='06'AND c='01')OR(n='53'AND c='03')OR(n='27'AND c='28')OR(n='55'AND c='51')OR(n='20'AND c='77')OR(n='41'AND c='97')OR(n='54'AND c='77')OR(n='18'AND c='23')OR(n='09'AND c='75')OR(n='63'AND c='89')OR(n='59'AND c='97')OR(n='01'AND c='66')OR(n='47'AND c='27')OR(n='32'AND c='50')OR(n='97'AND c='39')OR(n='44'AND c='98')OR(n='06'AND c='96')OR(n='32'AND c='79')OR(n='95'AND c='00')OR(n='52'AND c='57')OR(n='22'AND c='23')OR(n='00'AND c='19')OR(n='55'AND c='19')OR(n='65'AND c='19')OR(n='93'AND c='19')OR(n='94'AND c='19')OR(n='54'AND c='19')OR(n='91'AND c='20')OR(n='92'AND c='19')OR(n='12'AND c='19') + OR(n='49'AND c='19')OR(n='16'AND c='69')OR(n='97'AND c='29')OR(n='48'AND c='19')OR(n='89'AND c='19')OR(n='75'AND c='19')OR(n='97'AND c='19')OR(n='07'AND c='19')OR(n='67'AND c='19')OR(n='62'AND c='19')OR(n='73'AND c='19')OR(n='70'AND c='19')OR(n='66'AND c='48')OR(n='13'AND c='04')OR(n='13'AND c='15')OR(n='43'AND c='19')OR(n='93'AND c='82')OR(n='10'AND c='62')OR(n='69'AND c='30')OR(n='23'AND c='15')OR(n='33'AND c='89')OR(n='29'AND c='73')OR(n='35'AND c='30')OR(n='16'AND c='77')OR(n='28'AND c='19')OR(n='05'AND c='19')OR(n='68'AND c='19')OR(n='23'AND c='19')OR(n='33'AND c='19')OR(n='77'AND c='19')OR(n='79'AND c='19')OR(n='80'AND c='19')OR(n='22'AND c='19')OR(n='69'AND c='19')OR(n='23'AND c='19')OR(n='29'AND c='19')OR(n='69'AND c='19')OR(n='02'AND c='19')OR(n='72'AND c='19')OR(n='42'AND c='19')OR(n='95'AND c='65')OR(n='53'AND c='77')OR(n='65'AND c='77')OR(n='75'AND c='77')OR(n='67'AND c='15')OR(n='29'AND c='77')OR(n='98'AND c='77')OR(n='78'AND c='77')OR(n='85'AND c='77')OR(n='94'AND c='77')OR(n='58'AND c='77')OR(n='90'AND c='77')OR(n='88'AND c='77')OR(n='83'AND c='77')OR(n='28'AND c='77')OR(n='99'AND c='77')OR(n='37'AND c='77')OR(n='14'AND c='30')OR(n='34'AND c='91')OR(n='20'AND c='99')OR(n='98'AND c='54')OR(n='54'AND c='98')OR(n='88'AND c='98')OR(n='29'AND c='98')OR(n='31'AND c='98')OR(n='37'AND c='98')OR(n='25'AND c='60')OR(n='69'AND c='66')OR(n='98'AND c='89')OR(n='75'AND c='20')OR(n='83'AND c='60')OR(n='61'AND c='48')OR(n='00'AND c='04')OR(n='33'AND c='15')OR(n='36'AND c='19')OR(n='79'AND c='65')OR(n='51'AND c='99')OR(n='70'AND c='34')OR(n='97'AND c='88')OR(n='89'AND c='39')OR(n='73'AND c='11')OR(n='72'AND c='39')OR(n='06'AND c='22')OR(n='48'AND c='39')OR(n='59'AND c='81')OR(n='85'AND c='02')OR(n='40'AND c='39')OR(n='01'AND c='73')OR(n='44'AND c='57')OR(n='01'AND c='33')OR(n='51'AND c='36')OR(n='45'AND c='11')OR(n='31'AND c='88')OR(n='58'AND c='11')OR(n='71'AND c='11')OR(n='77'AND c='52')OR(n='88'AND c='33')OR(n='66'AND c='86')OR(n='14'AND c='40')OR(n='23'AND c='33')OR(n='79'AND c='19')OR(n='15'AND c='05')OR(n='23'AND c='76')OR(n='05'AND c='53')OR(n='69'AND c='76')OR(n='86'AND c='85')OR(n='17'AND c='44')OR(n='32'AND c='77')OR(n='17'AND c='95')OR(n='83'AND c='62')OR(n='59'AND c='28')OR(n='52'AND c='26')OR(n='48'AND c='26')OR(n='65'AND c='77')OR(n='62'AND c='19')OR(n='27'AND c='90')OR(n='10'AND c='88')OR(n='82'AND c='65')OR(n='38'AND c='44')OR(n='65'AND c='91')OR(n='98'AND c='12')OR(n='84'AND c='91')OR(n='94'AND c='85')OR(n='27'AND c='86')OR(n='18'AND c='78')OR(n='90'AND c='83')OR(n='85'AND c='80')OR(n='93'AND c='05') + OR(n='97'AND c='58')OR(n='60'AND c='71')OR(n='96'AND c='88')OR(n='31'AND c='86')OR(n='13'AND c='91')OR(n='33'AND c='91')OR(n='24'AND c='23')OR(n='74'AND c='80')OR(n='52'AND c='56')OR(n='16'AND c='85')OR(n='22'AND c='97')OR(n='74'AND c='30')OR(n='11'AND c='55')OR(n='68'AND c='36')OR(n='61'AND c='79')OR(n='86'AND c='03')OR(n='81'AND c='76')OR(n='77'AND c='82')OR(n='49'AND c='65')OR(n='68'AND c='76')OR(n='70'AND c='82')OR(n='13'AND c='18')OR(n='14'AND c='06')OR(n='52'AND c='88')OR(n='12'AND c='37')OR(n='57'AND c='11')OR(n='61'AND c='08')OR(n='04'AND c='44')OR(n='43'AND c='98')OR(n='40'AND c='00')OR(n='93'AND c='76')OR(n='62'AND c='82')OR(n='57'AND c='83')OR(n='57'AND c='19')OR(n='00'AND c='82')OR(n='28'AND c='48')OR(n='57'AND c='04')OR(n='06'AND c='19')OR(n='13'AND c='15')OR(n='81'AND c='88')OR(n='26'AND c='48')OR(n='45'AND c='04')OR(n='98'AND c='19')OR(n='08'AND c='15')OR(n='70'AND c='86')OR(n='84'AND c='77')OR(n='57'AND c='34')OR(n='99'AND c='90')OR(n='54'AND c='65')OR(n='19'AND c='36')OR(n='70'AND c='11')OR(n='68'AND c='11')OR(n='52'AND c='11')OR(n='11'AND c='11')OR(n='48'AND c='11')OR(n='33'AND c='11')OR(n='60'AND c='26')OR(n='36'AND c='40')OR(n='99'AND c='01')OR(n='52'AND c='99')OR(n='74'AND c='38')OR(n='45'AND c='52')OR(n='53'AND c='83')OR(n='74'AND c='96')OR(n='11'AND c='94')OR(n='43'AND c='22')OR(n='46'AND c='88')OR(n='01'AND c='93')OR(n='87'AND c='52')OR(n='82'AND c='98')OR(n='70'AND c='58')OR(n='43'AND c='57')OR(n='43'AND c='61')OR(n='30'AND c='68')OR(n='82'AND c='89')OR(n='55'AND c='06')OR(n='16'AND c='89')OR(n='57'AND c='89')OR(n='93'AND c='66')OR(n='77'AND c='22')OR(n='99'AND c='88')OR(n='00'AND c='50')OR(n='23'AND c='99')OR(n='17'AND c='22')OR(n='22'AND c='56')OR(n='88'AND c='27')OR(n='47'AND c='26')OR(n='81'AND c='91')OR(n='32'AND c='69')OR(n='68'AND c='58')OR(n='74'AND c='68')OR(n='38'AND c='60')OR(n='36'AND c='51')OR(n='85'AND c='22')OR(n='62'AND c='86')OR(n='54'AND c='82')OR(n='32'AND c='99')OR(n='38'AND c='67')OR(n='65'AND c='29')OR(n='35'AND c='00')OR(n='62'AND c='35')OR(n='24'AND c='73')OR(n='40'AND c='67')OR(n='38'AND c='01')OR(n='92'AND c='15')OR(n='46'AND c='78')OR(n='65'AND c='18')OR(n='31'AND c='44')OR(n='74'AND c='00')OR(n='91'AND c='86')OR(n='23'AND c='78')OR(n='92'AND c='33')OR(n='62'AND c='26')OR(n='08'AND c='58')OR(n='12'AND c='55')OR(n='97'AND c='85')OR(n='21'AND c='88')OR(n='07'AND c='33')OR(n='44'AND c='72')OR(n='32'AND c='29')OR(n='87'AND c='18')OR(n='97'AND c='40')OR(n='42'AND c='98')OR(n='72'AND c='83')OR(n='37'AND c='50')OR(n='33'AND c='63')OR(n='67'AND c='18')OR(n='55'AND c='11')OR(n='67'AND c='59')OR(n='84'AND c='88')OR(n='16'AND c='58')OR(n='92'AND c='55')OR(n='99'AND c='20')OR(n='31'AND c='15')OR(n='56'AND c='83') + OR(n='85'AND c='77')OR(n='70'AND c='89')OR(n='78'AND c='91')OR(n='21'AND c='43')OR(n='20'AND c='72')OR(n='27'AND c='86')OR(n='84'AND c='33')OR(n='46'AND c='30')OR(n='34'AND c='36')OR(n='40'AND c='67')OR(n='48'AND c='25')OR(n='53'AND c='18')OR(n='30'AND c='96')OR(n='16'AND c='95')OR(n='66'AND c='69')OR(n='02'AND c='43')OR(n='50'AND c='11')OR(n='92'AND c='30')OR(n='14'AND c='35')OR(n='34'AND c='45')OR(n='92'AND c='45')OR(n='84'AND c='37')OR(n='81'AND c='29')OR(n='96'AND c='19')OR(n='15'AND c='97')OR(n='89'AND c='32')OR(n='28'AND c='45')OR(n='89'AND c='71')OR(n='98'AND c='45')OR(n='11'AND c='22')OR(n='83'AND c='29')OR(n='71'AND c='19')OR(n='77'AND c='97')OR(n='19'AND c='45')OR(n='26'AND c='71')OR(n='36'AND c='45')OR(n='61'AND c='37')OR(n='67'AND c='29')OR(n='73'AND c='19')OR(n='70'AND c='97')OR(n='24'AND c='32')OR(n='99'AND c='45')OR(n='61'AND c='71')OR(n='50'AND c='11')OR(n='48'AND c='76')OR(n='22'AND c='81')OR(n='62'AND c='15')OR(n='67'AND c='77')OR(n='69'AND c='50')OR(n='97'AND c='34')OR(n='93'AND c='34')OR(n='04'AND c='55')OR(n='67'AND c='88')OR(n='98'AND c='93')OR(n='93'AND c='82')OR(n='63'AND c='77')OR(n='95'AND c='11')OR(n='91'AND c='25')OR(n='59'AND c='08')OR(n='48'AND c='97')OR(n='06'AND c='91')OR(n='02'AND c='22')OR(n='10'AND c='69')OR(n='32'AND c='28')OR(n='30'AND c='18')OR(n='70'AND c='68')OR(n='43'AND c='86')OR(n='76'AND c='95')OR(n='65'AND c='77')OR(n='64'AND c='29')OR(n='66'AND c='69')OR(n='13'AND c='93')OR(n='08'AND c='50')OR(n='65'AND c='66')OR(n='35'AND c='98')OR(n='46'AND c='49')OR(n='85'AND c='22')OR(n='90'AND c='77')OR(n='03'AND c='66')OR(n='33'AND c='99')OR(n='60'AND c='44')OR(n='71'AND c='86')OR(n='11'AND c='86')OR(n='48'AND c='86')OR(n='28'AND c='86')OR(n='40'AND c='86')OR(n='70'AND c='86')OR(n='99'AND c='82')OR(n='83'AND c='59')OR(n='85'AND c='88')OR(n='55'AND c='74')OR(n='97'AND c='96')OR(n='39'AND c='56')OR(n='59'AND c='48')OR(n='42'AND c='04')OR(n='96'AND c='15')OR(n='34'AND c='19')OR(n='26'AND c='15')OR(n='74'AND c='15')OR(n='76'AND c='25')OR(n='24'AND c='58')OR(n='67'AND c='81')OR(n='25'AND c='13')OR(n='98'AND c='75')OR(n='71'AND c='21')OR(n='81'AND c='25')OR(n='85'AND c='99')OR(n='34'AND c='99')OR(n='48'AND c='99')OR(n='53'AND c='79')OR(n='70'AND c='86')OR(n='02'AND c='86')OR(n='13'AND c='86')OR(n='25'AND c='11')OR(n='85'AND c='18')OR(n='50'AND c='78')OR(n='04'AND c='11')OR(n='54'AND c='86')OR(n='00'AND c='86')OR(n='28'AND c='86')OR(n='40'AND c='99')OR(n='97'AND c='99')OR(n='84'AND c='99')OR(n='47'AND c='48')OR(n='41'AND c='04')OR(n='56'AND c='15')OR(n='97'AND c='19')OR(n='91'AND c='48')OR(n='00'AND c='04')OR(n='97'AND c='15')OR(n='05'AND c='19')OR(n='47'AND c='48')OR(n='25'AND c='04')OR(n='72'AND c='15')OR(n='93'AND c='19')OR(n='68'AND c='48')OR(n='41'AND c='04')OR(n='17'AND c='15')OR(n='17'AND c='19')OR(n='70'AND c='48')OR(n='96'AND c='04')OR(n='65'AND c='15')OR(n='50'AND c='19')OR(n='31'AND c='48')OR(n='79'AND c='04')OR(n='01'AND c='15')OR(n='03'AND c='19')OR(n='12'AND c='48')OR(n='14'AND c='04')OR(n='70'AND c='15')OR(n='51'AND c='19')OR(n='30'AND c='48')OR(n='49'AND c='04')OR(n='41'AND c='15')OR(n='32'AND c='19')OR(n='34'AND c='48')OR(n='20'AND c='04')OR(n='13'AND c='15')OR(n='05'AND c='19')OR(n='29'AND c='48')OR(n='21'AND c='04')OR(n='48'AND c='15')OR(n='18'AND c='19')OR(n='17'AND c='99')OR(n='53'AND c='22')OR(n='96'AND c='17')OR(n='98'AND c='79')OR(n='86'AND c='25')OR(n='67'AND c='98')OR(n='46'AND c='12')OR(n='84'AND c='55')OR(n='67'AND c='83')OR(n='86'AND c='28')OR(n='65'AND c='36')OR(n='00'AND c='07')OR(n='78'AND c='60')OR(n='58'AND c='89')OR(n='60'AND c='25')OR(n='82'AND c='29')OR(n='19'AND c='77')OR(n='90'AND c='66')OR(n='82'AND c='95')OR(n='91'AND c='80')OR(n='59'AND c='11')OR(n='66'AND c='38')OR(n='93'AND c='78')OR(n='71'AND c='72')OR(n='95'AND c='32')OR(n='36'AND c='32')OR(n='09'AND c='32')OR(n='39'AND c='15')OR(n='61'AND c='48')OR(n='71'AND c='11')OR(n='30'AND c='22')OR(n='51'AND c='26')OR(n='82'AND c='37')OR(n='35'AND c='79')OR(n='43'AND c='44')OR(n='61'AND c='56')OR(n='39'AND c='08')OR(n='83'AND c='56')OR(n='68'AND c='99')OR(n='51'AND c='99')OR(n='31'AND c='78')OR(n='94'AND c='99')OR(n='01'AND c='99')OR(n='31'AND c='80')OR(n='78'AND c='78')OR(n='16'AND c='99')OR(n='65'AND c='55')OR(n='49'AND c='11')OR(n='35'AND c='94')OR(n='78'AND c='99')OR(n='49'AND c='49')OR(n='58'AND c='27')OR(n='75'AND c='66')OR(n='97'AND c='99')OR(n='55'AND c='23')OR(n='02'AND c='77')OR(n='44'AND c='98')OR(n='58'AND c='18')OR(n='04'AND c='11')OR(n='83'AND c='10')OR(n='49'AND c='93')OR(n='22'AND c='99')OR(n='65'AND c='30')OR(n='01'AND c='87')OR(n='03'AND c='22')OR(n='73'AND c='06')OR(n='41'AND c='76')OR(n='13'AND c='26')OR(n='92'AND c='69')OR(n='88'AND c='37')OR(n='62'AND c='06')OR(n='85'AND c='00')OR(n='28'AND c='77')OR(n='06'AND c='06')OR(n='47'AND c='26') + OR(n='16'AND c='98')OR(n='33'AND c='83')OR(n='14'AND c='56')OR(n='57'AND c='71')OR(n='01'AND c='71')OR(n='12'AND c='45')OR(n='50'AND c='68')OR(n='06'AND c='88')OR(n='07'AND c='68')OR(n='34'AND c='28')OR(n='44'AND c='60')OR(n='48'AND c='96')OR(n='42'AND c='24')OR(n='93'AND c='82')OR(n='33'AND c='67')OR(n='59'AND c='96')OR(n='62'AND c='30')OR(n='03'AND c='18')OR(n='37'AND c='11')OR(n='73'AND c='33')OR(n='84'AND c='18')OR(n='58'AND c='66')OR(n='17'AND c='02')OR(n='92'AND c='56')OR(n='39'AND c='38')OR(n='32'AND c='44')OR(n='62'AND c='77')OR(n='70'AND c='68')OR(n='50'AND c='26')OR(n='17'AND c='48')OR(n='91'AND c='48')OR(n='41'AND c='66')OR(n='11'AND c='60')OR(n='45'AND c='66')OR(n='89'AND c='71')OR(n='97'AND c='75')OR(n='19'AND c='88')OR(n='56'AND c='70')OR(n='28'AND c='08')OR(n='55'AND c='55')OR(n='48'AND c='88')OR(n='92'AND c='82')OR(n='41'AND c='30')OR(n='98'AND c='30')OR(n='27'AND c='66')OR(n='68'AND c='09')OR(n='02'AND c='82')OR(n='65'AND c='88')OR(n='76'AND c='03')OR(n='15'AND c='20')OR(n='39'AND c='56')OR(n='29'AND c='56')OR(n='55'AND c='51')OR(n='20'AND c='22')OR(n='27'AND c='00')OR(n='66'AND c='37')OR(n='75'AND c='27')OR(n='79'AND c='35')OR(n='21'AND c='54')OR(n='13'AND c='98')OR(n='21'AND c='83')OR(n='17'AND c='11')OR(n='95'AND c='26')OR(n='76'AND c='91')OR(n='74'AND c='68')OR(n='28'AND c='71')OR(n='83'AND c='56')OR(n='04'AND c='99')OR(n='17'AND c='60')OR(n='18'AND c='75')OR(n='56'AND c='94')OR(n='00'AND c='73')OR(n='42'AND c='88')OR(n='40'AND c='90')OR(n='63'AND c='87')OR(n='25'AND c='56')OR(n='77'AND c='03')OR(n='08'AND c='96')OR(n='91'AND c='66')OR(n='06'AND c='72')OR(n='94'AND c='86')OR(n='80'AND c='77')OR(n='34'AND c='07')OR(n='75'AND c='72')OR(n='05'AND c='50')OR(n='32'AND c='91')OR(n='94'AND c='71')OR(n='24'AND c='91')OR(n='95'AND c='06')OR(n='94'AND c='91')OR(n='71'AND c='86')OR(n='88'AND c='96')OR(n='36'AND c='88')OR(n='37'AND c='22')OR(n='65'AND c='66')OR(n='04'AND c='71')OR(n='41'AND c='27')OR(n='47'AND c='22')OR(n='32'AND c='97')OR(n='20'AND c='36')OR(n='63'AND c='66')OR(n='43'AND c='20')OR(n='98'AND c='97')OR(n='06'AND c='86')OR(n='45'AND c='58')OR(n='42'AND c='95')OR(n='20'AND c='65')OR(n='44'AND c='77')OR(n='76'AND c='88')OR(n='33'AND c='68')OR(n='48'AND c='83')OR(n='47'AND c='99') + """ +} diff --git a/regression-test/suites/nereids_rules_p0/infer_predicate/pull_up_predicate_set_op.groovy b/regression-test/suites/nereids_rules_p0/infer_predicate/pull_up_predicate_set_op.groovy index 14e3733adab177..1ed15dbfc17883 100644 --- a/regression-test/suites/nereids_rules_p0/infer_predicate/pull_up_predicate_set_op.groovy +++ b/regression-test/suites/nereids_rules_p0/infer_predicate/pull_up_predicate_set_op.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_pull_up_predicate_set_op") { +suite("pull_up_predicate_set_op") { sql "set enable_nereids_planner=true" sql "set enable_fallback_to_original_planner=false" sql """SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'""" diff --git a/regression-test/suites/query_p0/explain/test_compoundpredicate_explain.groovy b/regression-test/suites/query_p0/explain/test_compoundpredicate_explain.groovy index fccdd1b2e2fefa..26d91edbb12dea 100644 --- a/regression-test/suites/query_p0/explain/test_compoundpredicate_explain.groovy +++ b/regression-test/suites/query_p0/explain/test_compoundpredicate_explain.groovy @@ -23,49 +23,74 @@ suite("test_compoundpredicate_explain") { sql """INSERT INTO test_compoundpredicate_explain (k1, k2) VALUES (500, 450), (1100, 400), (300, 600), (700, 650), (800, 800), (1500, 300);""" - def testQueries = [ - "select * from test_compoundpredicate_explain where k1 > 500 and k2 < 700 or k1 < 3000", - "select * from test_compoundpredicate_explain where k1 > 500 or k2 < 700 and k1 < 3000", - "select * from test_compoundpredicate_explain where not (k1 > 500 and k2 < 700) or k1 < 3000", - "select * from test_compoundpredicate_explain where k1 > 500 and (k2 < 700 or k1 < 3000)", - "select * from test_compoundpredicate_explain where not (k1 > 500 or k2 < 700) and k1 < 3000", - "select * from test_compoundpredicate_explain where (k1 > 500 and not k2 < 700) or k1 < 3000", - "select * from test_compoundpredicate_explain where (k1 > 500 and k2 < 700) and (k1 < 3000 or k2 > 400)", - "select * from test_compoundpredicate_explain where not (k1 > 500 or (k2 < 700 and k1 < 3000))", - "select * from test_compoundpredicate_explain where k1 > 500 or not (k2 < 700 and k1 < 3000)", - "select * from test_compoundpredicate_explain where k1 < 1000 and (k2 < 700 or k1 > 500) and not (k2 > 300)", - "select * from test_compoundpredicate_explain where not ((k1 > 500 and k2 < 700) or k1 < 3000)", - "select * from test_compoundpredicate_explain where k1 > 500 and not (k2 < 700 or k1 < 3000)", - "select * from test_compoundpredicate_explain where (k1 > 500 or k2 < 700) and (k1 < 3000 and k2 > 200)", - "select * from test_compoundpredicate_explain where (k1 > 500 and k2 < 700) or not (k1 < 3000 and k2 > 200)" - ] - - testQueries.each { query -> - def explainResult1 = sql "explain all plan ${query}" - def explainResult2 = sql "explain ${query}" - - def predicates2Line = explainResult2.find { line -> - line[0].toString().trim().startsWith("PREDICATES:") - } - - if (predicates2Line != null) { - def predicates2 = predicates2Line[0].split("PREDICATES:").last().trim() - - predicates2 = predicates2?.replaceAll(/\[\#(\d+)\]/) { match, group1 -> "#" + group1 } - - def isMatch = explainResult1.any { line -> - line.toString().contains(predicates2) - } - - log.info("Testing query: " + query) - log.info("Standardized Predicates from PREDICATES: " + predicates2) - log.info("Match found in OPTIMIZED PLAN: " + isMatch) - - assert isMatch : "Predicates are not equal for query: ${query}" - } else { - logger.error("PREDICATES: not found in explain result for query: ${query}") - assert false : "PREDICATES: not found in explain result" - } + explain { + sql "select * from test_compoundpredicate_explain where k1 > 500 and k2 < 700 or k1 < 3000" + contains "(((k1[#0] > 500) AND (k2[#1] < 700)) OR (k1[#0] < 3000))" + } + + explain { + sql "select * from test_compoundpredicate_explain where k1 > 500 or k2 < 700 and k1 < 3000" + contains "((k1[#0] > 500) OR ((k2[#1] < 700) AND (k1[#0] < 3000)))" + } + + explain { + sql "select * from test_compoundpredicate_explain where not (k1 > 500 and k2 < 700) or k1 < 3000" + contains "((k1[#0] < 3000) OR (k2[#1] >= 700))" + } + + explain { + sql "select * from test_compoundpredicate_explain where k1 > 500 and (k2 < 700 or k1 < 3000)" + contains "PREDICATES: ((k1[#0] > 500) AND ((k2[#1] < 700) OR (k1[#0] < 3000)))" + } + + explain { + sql "select * from test_compoundpredicate_explain where not (k1 > 500 or k2 < 700) and k1 < 3000" + contains "PREDICATES: ((k1[#0] <= 500) AND (k2[#1] >= 700))" + } + + explain { + sql "select * from test_compoundpredicate_explain where (k1 > 500 and not k2 < 700) or k1 < 3000" + contains "PREDICATES: (((k1[#0] > 500) AND (k2[#1] >= 700)) OR (k1[#0] < 3000))" + } + + explain { + sql "select * from test_compoundpredicate_explain where (k1 > 500 and k2 < 700) and (k1 < 3000 or k2 > 400)" + contains "PREDICATES: (((k1[#0] > 500) AND (k2[#1] < 700)) AND ((k1[#0] < 3000) OR (k2[#1] > 400)))" + } + + explain { + sql "select * from test_compoundpredicate_explain where not (k1 > 500 or (k2 < 700 and k1 < 3000))" + contains "PREDICATES: ((k1[#0] <= 500) AND ((k2[#1] >= 700) OR (k1[#0] >= 3000)))" + } + + explain { + sql "select * from test_compoundpredicate_explain where k1 > 500 or not (k2 < 700 and k1 < 3000)" + contains "PREDICATES: ((k1[#0] > 500) OR (k2[#1] >= 700))" + } + + explain { + sql "select * from test_compoundpredicate_explain where k1 < 1000 and (k2 < 700 or k1 > 500) and not (k2 > 300)" + contains "PREDICATES: (((k1[#0] < 1000) AND ((k2[#1] < 700) OR (k1[#0] > 500))) AND (k2[#1] <= 300))" + } + + explain { + sql "select * from test_compoundpredicate_explain where not ((k1 > 500 and k2 < 700) or k1 < 3000)" + contains "PREDICATES: (((k1[#0] <= 500) OR (k2[#1] >= 700)) AND (k1[#0] >= 3000))" + } + + explain { + sql "select * from test_compoundpredicate_explain where k1 > 500 and not (k2 < 700 or k1 < 3000)" + contains "PREDICATES: ((k1[#0] >= 3000) AND (k2[#1] >= 700))" + } + + explain { + sql "select * from test_compoundpredicate_explain where (k1 > 500 or k2 < 700) and (k1 < 3000 and k2 > 200)" + contains "PREDICATES: ((((k1[#0] > 500) OR (k2[#1] < 700)) AND (k1[#0] < 3000)) AND (k2[#1] > 200))" + } + + explain { + sql "select * from test_compoundpredicate_explain where (k1 > 500 and k2 < 700) or not (k1 < 3000 and k2 > 200)" + contains "PREDICATES: ((((k1[#0] > 500) AND (k2[#1] < 700)) OR (k1[#0] >= 3000)) OR (k2[#1] <= 200))" } sql "drop table if exists test_compoundpredicate_explain" From ea2a21dc7519142ffcb4af893c88e1dfd4215556 Mon Sep 17 00:00:00 2001 From: htyoung Date: Sat, 30 Nov 2024 22:48:56 +0800 Subject: [PATCH 090/399] [fix](cache) fix same sql return wrong result when switch database with `use db` and enable sql cache (#44782) ### What problem does this PR solve? It will return wrong result when running same sql in different db with `enable_sql_cache=true` for example, the `db1` and `db2` has the same table `tbl` but the data are not equals, if execute the below sql in `db1` and cache the result, then execute it in `db2`, it will return the wrong result ```sql select count(*) from tbl ``` Co-authored-by: tongyang.han --- .../doris/common/NereidsSqlCacheManager.java | 49 +++++++----- .../cache/parse_sql_from_sql_cache.groovy | 74 +++++++++++++++++++ 2 files changed, 105 insertions(+), 18 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/NereidsSqlCacheManager.java b/fe/fe-core/src/main/java/org/apache/doris/common/NereidsSqlCacheManager.java index 1317fdaefc766e..cd32b52034a5d4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/NereidsSqlCacheManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/NereidsSqlCacheManager.java @@ -74,9 +74,11 @@ import java.util.Optional; import java.util.Set; -/** NereidsSqlCacheManager */ +/** + * NereidsSqlCacheManager + */ public class NereidsSqlCacheManager { - // key: : + // key: :: // value: SqlCacheContext private volatile Cache sqlCaches; @@ -110,7 +112,7 @@ private static Cache buildSqlCaches(int sqlCacheNum, lo // auto evict cache when jvm memory too low .softValues(); if (sqlCacheNum > 0) { - cacheBuilder = cacheBuilder.maximumSize(sqlCacheNum); + cacheBuilder.maximumSize(sqlCacheNum); } if (expireAfterAccessSeconds > 0) { cacheBuilder = cacheBuilder.expireAfterAccess(Duration.ofSeconds(expireAfterAccessSeconds)); @@ -119,7 +121,9 @@ private static Cache buildSqlCaches(int sqlCacheNum, lo return cacheBuilder.build(); } - /** tryAddFeCache */ + /** + * tryAddFeCache + */ public void tryAddFeSqlCache(ConnectContext connectContext, String sql) { Optional sqlCacheContextOpt = connectContext.getStatementContext().getSqlCacheContext(); if (!sqlCacheContextOpt.isPresent()) { @@ -127,17 +131,18 @@ public void tryAddFeSqlCache(ConnectContext connectContext, String sql) { } SqlCacheContext sqlCacheContext = sqlCacheContextOpt.get(); - UserIdentity currentUserIdentity = connectContext.getCurrentUserIdentity(); String key = sqlCacheContext.getCacheKeyType() == CacheKeyType.SQL - ? currentUserIdentity.toString() + ":" + normalizeSql(sql.trim()) - : currentUserIdentity.toString() + ":" + DebugUtil.printId(sqlCacheContext.getOrComputeCacheKeyMd5()); + ? generateCacheKey(connectContext, normalizeSql(sql)) + : generateCacheKey(connectContext, DebugUtil.printId(sqlCacheContext.getOrComputeCacheKeyMd5())); if (sqlCaches.getIfPresent(key) == null && sqlCacheContext.getOrComputeCacheKeyMd5() != null && sqlCacheContext.getResultSetInFe().isPresent()) { sqlCaches.put(key, sqlCacheContext); } } - /** tryAddBeCache */ + /** + * tryAddBeCache + */ public void tryAddBeCache(ConnectContext connectContext, String sql, CacheAnalyzer analyzer) { Optional sqlCacheContextOpt = connectContext.getStatementContext().getSqlCacheContext(); if (!sqlCacheContextOpt.isPresent()) { @@ -147,10 +152,9 @@ public void tryAddBeCache(ConnectContext connectContext, String sql, CacheAnalyz return; } SqlCacheContext sqlCacheContext = sqlCacheContextOpt.get(); - UserIdentity currentUserIdentity = connectContext.getCurrentUserIdentity(); String key = sqlCacheContext.getCacheKeyType() == CacheKeyType.SQL - ? currentUserIdentity.toString() + ":" + normalizeSql(sql.trim()) - : currentUserIdentity.toString() + ":" + DebugUtil.printId(sqlCacheContext.getOrComputeCacheKeyMd5()); + ? generateCacheKey(connectContext, normalizeSql(sql)) + : generateCacheKey(connectContext, DebugUtil.printId(sqlCacheContext.getOrComputeCacheKeyMd5())); if (sqlCaches.getIfPresent(key) == null && sqlCacheContext.getOrComputeCacheKeyMd5() != null) { SqlCache cache = (SqlCache) analyzer.getCache(); sqlCacheContext.setSumOfPartitionNum(cache.getSumOfPartitionNum()); @@ -167,23 +171,23 @@ public void tryAddBeCache(ConnectContext connectContext, String sql, CacheAnalyz } } - /** tryParseSql */ + /** + * tryParseSql + */ public Optional tryParseSql(ConnectContext connectContext, String sql) { - UserIdentity currentUserIdentity = connectContext.getCurrentUserIdentity(); - String key = currentUserIdentity + ":" + normalizeSql(sql.trim()); + String key = generateCacheKey(connectContext, normalizeSql(sql.trim())); SqlCacheContext sqlCacheContext = sqlCaches.getIfPresent(key); if (sqlCacheContext == null) { return Optional.empty(); } // LOG.info("Total size: " + GraphLayout.parseInstance(sqlCacheContext).totalSize()); - + UserIdentity currentUserIdentity = connectContext.getCurrentUserIdentity(); List currentVariables = resolveUserVariables(sqlCacheContext); if (usedVariablesChanged(currentVariables, sqlCacheContext)) { String md5 = DebugUtil.printId( sqlCacheContext.doComputeCacheKeyMd5(Utils.fastToImmutableSet(currentVariables))); - - String md5CacheKey = currentUserIdentity + ":" + md5; + String md5CacheKey = generateCacheKey(connectContext, md5); SqlCacheContext sqlCacheContextWithVariable = sqlCaches.getIfPresent(md5CacheKey); // already exist cache in the fe, but the variable is different to this query, @@ -203,6 +207,15 @@ public Optional tryParseSql(ConnectContext connectContext, Stri } } + private String generateCacheKey(ConnectContext connectContext, String sqlOrMd5) { + CatalogIf currentCatalog = connectContext.getCurrentCatalog(); + String currentCatalogName = currentCatalog != null ? currentCatalog.getName() : ""; + String currentDatabase = connectContext.getDatabase(); + String currentDatabaseName = currentDatabase != null ? currentDatabase : ""; + return currentCatalogName + "." + currentDatabaseName + ":" + connectContext.getCurrentUserIdentity().toString() + + ":" + sqlOrMd5; + } + private String normalizeSql(String sql) { return NereidsParser.removeCommentAndTrimBlank(sql); } @@ -402,7 +415,7 @@ private boolean usedVariablesChanged(List currentVariables, SqlCacheCo Variable cachedVariable = cachedUsedVariables.get(i); if (!Objects.equals(currentVariable, cachedVariable) || cachedVariable.getRealExpression().anyMatch( - expr -> !((ExpressionTrait) expr).isDeterministic())) { + expr -> !((ExpressionTrait) expr).isDeterministic())) { return true; } } diff --git a/regression-test/suites/nereids_p0/cache/parse_sql_from_sql_cache.groovy b/regression-test/suites/nereids_p0/cache/parse_sql_from_sql_cache.groovy index 54ab702888888d..3635936e8be083 100644 --- a/regression-test/suites/nereids_p0/cache/parse_sql_from_sql_cache.groovy +++ b/regression-test/suites/nereids_p0/cache/parse_sql_from_sql_cache.groovy @@ -827,6 +827,80 @@ suite("parse_sql_from_sql_cache") { def result = sql "select FROM_UNIXTIME(UNIX_TIMESTAMP(), 'yyyy-MM-dd HH:mm:ss')" assertNotEquals("yyyy-MM-dd HH:mm:ss", result[0][0]) } + }), + extraThread("test_same_sql_with_different_db", { + def dbName1 = "test_db1" + def dbName2 = "test_db2" + def tableName = "test_cache_table" + + sql "CREATE DATABASE IF NOT EXISTS ${dbName1}" + sql "DROP TABLE IF EXISTS ${dbName1}.${tableName}" + sql """ + CREATE TABLE IF NOT EXISTS ${dbName1}.${tableName} ( + `k1` date NOT NULL COMMENT "", + `k2` int(11) NOT NULL COMMENT "" + ) ENGINE=OLAP + DUPLICATE KEY(`k1`, `k2`) + COMMENT "OLAP" + PARTITION BY RANGE(`k1`) + (PARTITION p202411 VALUES [('2024-11-01'), ('2024-12-01'))) + DISTRIBUTED BY HASH(`k1`, `k2`) BUCKETS 1 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1", + "in_memory" = "false", + "storage_format" = "V2" + ) + """ + sql "CREATE DATABASE IF NOT EXISTS ${dbName2}" + sql "DROP TABLE IF EXISTS ${dbName2}.${tableName}" + sql """ + CREATE TABLE IF NOT EXISTS ${dbName2}.${tableName} ( + `k1` date NOT NULL COMMENT "", + `k2` int(11) NOT NULL COMMENT "" + ) ENGINE=OLAP + DUPLICATE KEY(`k1`, `k2`) + COMMENT "OLAP" + PARTITION BY RANGE(`k1`) + (PARTITION p202411 VALUES [('2024-11-01'), ('2024-12-01'))) + DISTRIBUTED BY HASH(`k1`, `k2`) BUCKETS 1 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1", + "in_memory" = "false", + "storage_format" = "V2" + ) + """ + + sql """ + INSERT INTO ${dbName1}.${tableName} VALUES + ("2024-11-29",0), + ("2024-11-30",0) + """ + // after partition changed 10s, the sql cache can be used + sleep(10000) + sql """ + INSERT INTO ${dbName2}.${tableName} VALUES + ("2024-11-29",0) + """ + // after partition changed 10s, the sql cache can be used + sleep(10000) + + sql "set enable_sql_cache=true" + sql "use ${dbName1}" + List> result1 = sql """ + SELECT COUNT(*) FROM ${tableName} + """ + assertEquals(result1[0][0],2) + + sql "use ${dbName2}" + List> result2 = sql """ + SELECT COUNT(*) FROM ${tableName} + """ + assertEquals(result2[0][0],1) + + sql "DROP TABLE IF EXISTS ${dbName1}.${tableName}" + sql "DROP TABLE IF EXISTS ${dbName2}.${tableName}" + sql "DROP DATABASE IF EXISTS ${dbName1}" + sql "DROP DATABASE IF EXISTS ${dbName2}" }) ).get() } From 97adac19f8c02600e1306a39e61b8237bf5f856b Mon Sep 17 00:00:00 2001 From: daidai Date: Sun, 1 Dec 2024 20:17:20 +0800 Subject: [PATCH 091/399] [fix](maxcompute)add mc catalog read partition table partition prune (#44508) ### What problem does this PR solve? Based on the `requiredPartitions` API interface of the maxcompute API, we implemented partition pruning when reading partitioned tables to avoid queries that scan the entire table. --- .../doris/datasource/ExternalTable.java | 9 +- .../datasource/hive/HMSExternalTable.java | 4 +- .../datasource/hive/source/HiveScanNode.java | 2 +- .../maxcompute/MaxComputeExternalTable.java | 33 +- .../maxcompute/source/MaxComputeScanNode.java | 50 +++- .../translator/PhysicalPlanTranslator.java | 3 +- .../expression/rules/PartitionPruner.java | 2 +- .../rules/rewrite/PruneFileScanPartition.java | 4 +- .../doris/planner/SingleNodePlanner.java | 5 +- .../test_max_compute_partition_prune.out | 125 ++++++++ .../test_max_compute_partition_prune.groovy | 282 ++++++++++++++++++ 11 files changed, 497 insertions(+), 22 deletions(-) create mode 100644 regression-test/data/external_table_p2/maxcompute/test_max_compute_partition_prune.out create mode 100644 regression-test/suites/external_table_p2/maxcompute/test_max_compute_partition_prune.groovy diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalTable.java index d82959954f2607..6c72d0a67d4732 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalTable.java @@ -377,7 +377,7 @@ public TableIndexes getTableIndexes() { * @return */ public SelectedPartitions initSelectedPartitions(Optional snapshot) { - if (!supportPartitionPruned()) { + if (!supportInternalPartitionPruned()) { return SelectedPartitions.NOT_PRUNED; } if (CollectionUtils.isEmpty(this.getPartitionColumns(snapshot))) { @@ -394,7 +394,7 @@ public SelectedPartitions initSelectedPartitions(Optional snapshot * @param snapshot if not support mvcc, ignore this * @return partitionName ==> PartitionItem */ - public Map getNameToPartitionItems(Optional snapshot) { + protected Map getNameToPartitionItems(Optional snapshot) { return Collections.emptyMap(); } @@ -410,11 +410,12 @@ public List getPartitionColumns(Optional snapshot) { } /** - * Does it support partition cpruned, If so, this method needs to be overridden in subclasses + * Does it support Internal partition pruned, If so, this method needs to be overridden in subclasses + * Internal partition pruned : Implement partition pruning logic without relying on external APIs. * * @return */ - public boolean supportPartitionPruned() { + public boolean supportInternalPartitionPruned() { return false; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalTable.java index 6d65f8bcdbccb7..134ad362fa1eed 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalTable.java @@ -301,12 +301,12 @@ public List getPartitionColumns(Optional snapshot) { } @Override - public boolean supportPartitionPruned() { + public boolean supportInternalPartitionPruned() { return getDlaType() == DLAType.HIVE; } @Override - public Map getNameToPartitionItems(Optional snapshot) { + protected Map getNameToPartitionItems(Optional snapshot) { return getNameToPartitionItems(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/source/HiveScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/source/HiveScanNode.java index 3a2a4d3eb5c6ae..99d3cd1cd21622 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/source/HiveScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/source/HiveScanNode.java @@ -82,7 +82,7 @@ public class HiveScanNode extends FileQueryScanNode { // will only be set in Nereids, for lagency planner, it should be null @Setter - private SelectedPartitions selectedPartitions = null; + protected SelectedPartitions selectedPartitions = null; private boolean partitionInit = false; private final AtomicReference batchException = new AtomicReference<>(null); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeExternalTable.java index dc3232f79f5f71..0f748f59e927bc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeExternalTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeExternalTable.java @@ -21,6 +21,7 @@ import org.apache.doris.catalog.Column; import org.apache.doris.catalog.Env; import org.apache.doris.catalog.MapType; +import org.apache.doris.catalog.PartitionItem; import org.apache.doris.catalog.ScalarType; import org.apache.doris.catalog.StructField; import org.apache.doris.catalog.StructType; @@ -28,6 +29,7 @@ import org.apache.doris.datasource.ExternalTable; import org.apache.doris.datasource.SchemaCacheValue; import org.apache.doris.datasource.TablePartitionValues; +import org.apache.doris.datasource.mvcc.MvccSnapshot; import org.apache.doris.thrift.TMCTable; import org.apache.doris.thrift.TTableDescriptor; import org.apache.doris.thrift.TTableType; @@ -50,6 +52,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Map.Entry; import java.util.Optional; import java.util.stream.Collectors; @@ -71,6 +74,15 @@ protected synchronized void makeSureInitialized() { } } + @Override + public boolean supportInternalPartitionPruned() { + return true; + } + + @Override + public List getPartitionColumns(Optional snapshot) { + return getPartitionColumns(); + } public List getPartitionColumns() { makeSureInitialized(); @@ -79,7 +91,24 @@ public List getPartitionColumns() { .orElse(Collections.emptyList()); } - public TablePartitionValues getPartitionValues() { + @Override + protected Map getNameToPartitionItems(Optional snapshot) { + if (getPartitionColumns().isEmpty()) { + return Collections.emptyMap(); + } + + TablePartitionValues tablePartitionValues = getPartitionValues(); + Map idToPartitionItem = tablePartitionValues.getIdToPartitionItem(); + Map idToNameMap = tablePartitionValues.getPartitionIdToNameMap(); + + Map nameToPartitionItem = Maps.newHashMapWithExpectedSize(idToPartitionItem.size()); + for (Entry entry : idToPartitionItem.entrySet()) { + nameToPartitionItem.put(idToNameMap.get(entry.getKey()), entry.getValue()); + } + return nameToPartitionItem; + } + + private TablePartitionValues getPartitionValues() { makeSureInitialized(); Optional schemaCacheValue = getSchemaCacheValue(); if (!schemaCacheValue.isPresent()) { @@ -110,6 +139,8 @@ private TablePartitionValues loadPartitionValues(MaxComputeSchemaCacheValue sche /** * parse all values from partitionPath to a single list. + * In MaxCompute : Support special characters : _$#.!@ + * Ref : MaxCompute Error Code: ODPS-0130071 Invalid partition value. * * @param partitionColumns partitionColumns can contain the part1,part2,part3... * @param partitionPath partitionPath format is like the 'part1=123/part2=abc/part3=1bc' diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/source/MaxComputeScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/source/MaxComputeScanNode.java index e0b84b0860e551..e177e9d8b7c88c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/source/MaxComputeScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/source/MaxComputeScanNode.java @@ -40,6 +40,7 @@ import org.apache.doris.datasource.maxcompute.MaxComputeExternalTable; import org.apache.doris.datasource.maxcompute.source.MaxComputeSplit.SplitType; import org.apache.doris.datasource.property.constants.MCProperties; +import org.apache.doris.nereids.trees.plans.logical.LogicalFileScan.SelectedPartitions; import org.apache.doris.nereids.util.DateUtils; import org.apache.doris.planner.PlanNodeId; import org.apache.doris.spi.Split; @@ -50,6 +51,7 @@ import org.apache.doris.thrift.TTableFormatFileDesc; import com.aliyun.odps.OdpsType; +import com.aliyun.odps.PartitionSpec; import com.aliyun.odps.table.TableIdentifier; import com.aliyun.odps.table.configuration.ArrowOptions; import com.aliyun.odps.table.configuration.ArrowOptions.TimestampUnit; @@ -60,6 +62,7 @@ import com.aliyun.odps.table.read.split.impl.IndexedInputSplit; import com.google.common.collect.Maps; import jline.internal.Log; +import lombok.Setter; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -86,14 +89,28 @@ public class MaxComputeScanNode extends FileQueryScanNode { private static final LocationPath ROW_OFFSET_PATH = new LocationPath("/row_offset", Maps.newHashMap()); private static final LocationPath BYTE_SIZE_PATH = new LocationPath("/byte_size", Maps.newHashMap()); + @Setter + private SelectedPartitions selectedPartitions = null; + + // For new planner + public MaxComputeScanNode(PlanNodeId id, TupleDescriptor desc, + SelectedPartitions selectedPartitions, boolean needCheckColumnPriv) { + this(id, desc, "MCScanNode", StatisticalType.MAX_COMPUTE_SCAN_NODE, + selectedPartitions, needCheckColumnPriv); + } + + // For old planner public MaxComputeScanNode(PlanNodeId id, TupleDescriptor desc, boolean needCheckColumnPriv) { - this(id, desc, "MCScanNode", StatisticalType.MAX_COMPUTE_SCAN_NODE, needCheckColumnPriv); + this(id, desc, "MCScanNode", StatisticalType.MAX_COMPUTE_SCAN_NODE, + SelectedPartitions.NOT_PRUNED, needCheckColumnPriv); } - public MaxComputeScanNode(PlanNodeId id, TupleDescriptor desc, String planNodeName, - StatisticalType statisticalType, boolean needCheckColumnPriv) { + private MaxComputeScanNode(PlanNodeId id, TupleDescriptor desc, String planNodeName, + StatisticalType statisticalType, SelectedPartitions selectedPartitions, + boolean needCheckColumnPriv) { super(id, desc, planNodeName, statisticalType, needCheckColumnPriv); table = (MaxComputeExternalTable) desc.getTable(); + this.selectedPartitions = selectedPartitions; } @Override @@ -117,10 +134,27 @@ private void setScanParams(TFileRangeDesc rangeDesc, MaxComputeSplit maxComputeS rangeDesc.setSize(maxComputeSplit.getLength()); } - void createTableBatchReadSession() throws UserException { + // Return false if no need to read any partition data. + // Return true if need to read partition data. + boolean createTableBatchReadSession() throws UserException { List requiredPartitionColumns = new ArrayList<>(); List orderedRequiredDataColumns = new ArrayList<>(); + List requiredPartitionSpecs = new ArrayList<>(); + //if requiredPartitionSpecs is empty, get all partition data. + if (!table.getPartitionColumns().isEmpty() && selectedPartitions != SelectedPartitions.NOT_PRUNED) { + this.totalPartitionNum = selectedPartitions.totalPartitionNum; + this.selectedPartitionNum = selectedPartitions.selectedPartitions.size(); + + if (selectedPartitions.selectedPartitions.isEmpty()) { + //no need read any partition data. + return false; + } + selectedPartitions.selectedPartitions.forEach( + (key, value) -> requiredPartitionSpecs.add(new PartitionSpec(key)) + ); + } + Set requiredSlots = desc.getSlots().stream().map(e -> e.getColumn().getName()).collect(Collectors.toSet()); @@ -150,6 +184,7 @@ void createTableBatchReadSession() throws UserException { .withSettings(mcCatalog.getSettings()) .withSplitOptions(mcCatalog.getSplitOption()) .requiredPartitionColumns(requiredPartitionColumns) + .requiredPartitions(requiredPartitionSpecs) .requiredDataColumns(orderedRequiredDataColumns) .withArrowOptions( ArrowOptions.newBuilder() @@ -162,7 +197,7 @@ void createTableBatchReadSession() throws UserException { } catch (java.io.IOException e) { throw new RuntimeException(e); } - + return true; } @Override @@ -430,7 +465,10 @@ public List getSplits() throws UserException { if (desc.getSlots().isEmpty() || odpsTable.getFileNum() <= 0) { return result; } - createTableBatchReadSession(); + + if (!createTableBatchReadSession()) { + return result; + } try { String scanSessionSerialize = serializeSession(tableBatchReadSession); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java index 56ae65ec722941..c474e6bd56e1d7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java @@ -579,7 +579,8 @@ public PlanFragment visitPhysicalFileScan(PhysicalFileScan fileScan, PlanTransla } else if (table instanceof TrinoConnectorExternalTable) { scanNode = new TrinoConnectorScanNode(context.nextPlanNodeId(), tupleDescriptor, false); } else if (table instanceof MaxComputeExternalTable) { - scanNode = new MaxComputeScanNode(context.nextPlanNodeId(), tupleDescriptor, false); + scanNode = new MaxComputeScanNode(context.nextPlanNodeId(), tupleDescriptor, + fileScan.getSelectedPartitions(), false); } else if (table instanceof LakeSoulExternalTable) { scanNode = new LakeSoulScanNode(context.nextPlanNodeId(), tupleDescriptor, false); } else { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/PartitionPruner.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/PartitionPruner.java index fac1a7f82d2cfb..ed783aa3d5a9b6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/PartitionPruner.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/PartitionPruner.java @@ -55,7 +55,7 @@ public class PartitionPruner extends DefaultExpressionRewriter { /** Different type of table may have different partition prune behavior. */ public enum PartitionTableType { OLAP, - HIVE + EXTERNAL } private PartitionPruner(List partitions, Expression partitionPredicate) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneFileScanPartition.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneFileScanPartition.java index 4bbb0a8aa76270..ba8b270d1f397d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneFileScanPartition.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneFileScanPartition.java @@ -59,7 +59,7 @@ public Rule build() { ExternalTable tbl = scan.getTable(); SelectedPartitions selectedPartitions; - if (tbl.supportPartitionPruned()) { + if (tbl.supportInternalPartitionPruned()) { selectedPartitions = pruneExternalPartitions(tbl, filter, scan, ctx.cascadesContext); } else { // set isPruned so that it won't go pass the partition prune again @@ -91,7 +91,7 @@ private SelectedPartitions pruneExternalPartitions(ExternalTable externalTable, Map nameToPartitionItem = scan.getSelectedPartitions().selectedPartitions; List prunedPartitions = new ArrayList<>(PartitionPruner.prune( - partitionSlots, filter.getPredicate(), nameToPartitionItem, ctx, PartitionTableType.HIVE)); + partitionSlots, filter.getPredicate(), nameToPartitionItem, ctx, PartitionTableType.EXTERNAL)); for (String name : prunedPartitions) { selectedPartitionItems.put(name, nameToPartitionItem.get(name)); diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/SingleNodePlanner.java b/fe/fe-core/src/main/java/org/apache/doris/planner/SingleNodePlanner.java index d94ad0a2552240..4091640066c1d7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/SingleNodePlanner.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/SingleNodePlanner.java @@ -76,7 +76,6 @@ import org.apache.doris.datasource.trinoconnector.source.TrinoConnectorScanNode; import org.apache.doris.qe.ConnectContext; import org.apache.doris.rewrite.mvrewrite.MVSelectFailedException; -import org.apache.doris.statistics.StatisticalType; import org.apache.doris.thrift.TPushAggOp; import com.google.common.base.Preconditions; @@ -1993,9 +1992,7 @@ private PlanNode createScanNode(Analyzer analyzer, TableRef tblRef, SelectStmt s scanNode = new TrinoConnectorScanNode(ctx.getNextNodeId(), tblRef.getDesc(), true); break; case MAX_COMPUTE_EXTERNAL_TABLE: - // TODO: support max compute scan node - scanNode = new MaxComputeScanNode(ctx.getNextNodeId(), tblRef.getDesc(), "MCScanNode", - StatisticalType.MAX_COMPUTE_SCAN_NODE, true); + scanNode = new MaxComputeScanNode(ctx.getNextNodeId(), tblRef.getDesc(), true); break; case ES_EXTERNAL_TABLE: scanNode = new EsScanNode(ctx.getNextNodeId(), tblRef.getDesc(), true); diff --git a/regression-test/data/external_table_p2/maxcompute/test_max_compute_partition_prune.out b/regression-test/data/external_table_p2/maxcompute/test_max_compute_partition_prune.out new file mode 100644 index 00000000000000..8f443829c4c394 --- /dev/null +++ b/regression-test/data/external_table_p2/maxcompute/test_max_compute_partition_prune.out @@ -0,0 +1,125 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !one_partition_1_1 -- +1 Alice 2024 +2 Bob 2024 +3 Charlie 2024 + +-- !one_partition_2_1 -- +4 David 2025 +5 Eva 2025 + +-- !one_partition_3_all -- +1 Alice 2024 +2 Bob 2024 +3 Charlie 2024 +4 David 2025 +5 Eva 2025 + +-- !one_partition_4_all -- +5 Eva 2025 + +-- !one_partition_5_1 -- +3 Charlie 2024 + +-- !two_partition_1_1 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 + +-- !two_partition_2_1 -- +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !two_partition_3_2 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 + +-- !two_partition_4_all -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 +6 Frank EU 1 +7 Grace EU 1 +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !two_partition_5_1 -- + +-- !two_partition_6_1 -- +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !three_partition_1_1 -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 + +-- !three_partition_2_1 -- +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 + +-- !three_partition_3_3 -- +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_4_2 -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 +6 Frank US 2025 Q1 + +-- !three_partition_5_all -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 +4 David US 2024 Q2 +5 Eva US 2024 Q2 +6 Frank US 2025 Q1 +7 Grace US 2025 Q2 +8 Hannah EU 2024 Q1 +9 Ivy EU 2024 Q1 +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 +12 Mia EU 2025 Q3 +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_6_1 -- +8 Hannah EU 2024 Q1 +9 Ivy EU 2024 Q1 + +-- !three_partition_7_7 -- +6 Frank US 2025 Q1 +7 Grace US 2025 Q2 +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 +12 Mia EU 2025 Q3 +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_8_2 -- +7 Grace US 2025 Q2 + +-- !one_partition_6_0 -- + +-- !two_partition_7_0 -- + +-- !two_partition_8_0 -- + +-- !three_partition_9_0 -- + +-- !three_partition_10_0 -- + +-- !three_partition_11_0 -- + diff --git a/regression-test/suites/external_table_p2/maxcompute/test_max_compute_partition_prune.groovy b/regression-test/suites/external_table_p2/maxcompute/test_max_compute_partition_prune.groovy new file mode 100644 index 00000000000000..e34569117a167f --- /dev/null +++ b/regression-test/suites/external_table_p2/maxcompute/test_max_compute_partition_prune.groovy @@ -0,0 +1,282 @@ +// 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. + + + +/* +CREATE TABLE one_partition_tb ( + id INT, + name string +) +PARTITIONED BY (part1 INT); +INSERT INTO one_partition_tb PARTITION (part1=2024) VALUES (1, 'Alice'); +INSERT INTO one_partition_tb PARTITION (part1=2024) VALUES (2, 'Bob'); +INSERT INTO one_partition_tb PARTITION (part1=2024) VALUES (3, 'Charlie'); +INSERT INTO one_partition_tb PARTITION (part1=2025) VALUES (4, 'David'); +INSERT INTO one_partition_tb PARTITION (part1=2025) VALUES (5, 'Eva'); +CREATE TABLE two_partition_tb ( + id INT, + name string +) +PARTITIONED BY (part1 STRING, part2 int); +INSERT INTO two_partition_tb PARTITION (part1='US', part2=1) VALUES (1, 'Alice'); +INSERT INTO two_partition_tb PARTITION (part1='US', part2=1) VALUES (2, 'Bob'); +INSERT INTO two_partition_tb PARTITION (part1='US', part2=1) VALUES (3, 'Charlie'); +INSERT INTO two_partition_tb PARTITION (part1='US', part2=2) VALUES (4, 'David'); +INSERT INTO two_partition_tb PARTITION (part1='US', part2=2) VALUES (5, 'Eva'); +INSERT INTO two_partition_tb PARTITION (part1='EU', part2=1) VALUES (6, 'Frank'); +INSERT INTO two_partition_tb PARTITION (part1='EU', part2=1) VALUES (7, 'Grace'); +INSERT INTO two_partition_tb PARTITION (part1='EU', part2=2) VALUES (8, 'Hannah'); +INSERT INTO two_partition_tb PARTITION (part1='EU', part2=2) VALUES (9, 'Ivy'); +INSERT INTO two_partition_tb PARTITION (part1='EU', part2=2) VALUES (10, 'Jack'); +CREATE TABLE three_partition_tb ( + id INT, + name string +) +PARTITIONED BY (part1 STRING, part2 INT, part3 STRING); +INSERT INTO three_partition_tb PARTITION (part1='US', part2=2024, part3='Q1') VALUES (1, 'Alice'); +INSERT INTO three_partition_tb PARTITION (part1='US', part2=2024, part3='Q1') VALUES (2, 'Bob'); +INSERT INTO three_partition_tb PARTITION (part1='US', part2=2024, part3='Q1') VALUES (3, 'Charlie'); +INSERT INTO three_partition_tb PARTITION (part1='US', part2=2024, part3='Q2') VALUES (4, 'David'); +INSERT INTO three_partition_tb PARTITION (part1='US', part2=2024, part3='Q2') VALUES (5, 'Eva'); +INSERT INTO three_partition_tb PARTITION (part1='US', part2=2025, part3='Q1') VALUES (6, 'Frank'); +INSERT INTO three_partition_tb PARTITION (part1='US', part2=2025, part3='Q2') VALUES (7, 'Grace'); +INSERT INTO three_partition_tb PARTITION (part1='EU', part2=2024, part3='Q1') VALUES (8, 'Hannah'); +INSERT INTO three_partition_tb PARTITION (part1='EU', part2=2024, part3='Q1') VALUES (9, 'Ivy'); +INSERT INTO three_partition_tb PARTITION (part1='EU', part2=2025, part3='Q2') VALUES (10, 'Jack'); +INSERT INTO three_partition_tb PARTITION (part1='EU', part2=2025, part3='Q2') VALUES (11, 'Leo'); +INSERT INTO three_partition_tb PARTITION (part1='EU', part2=2025, part3='Q3') VALUES (12, 'Mia'); +INSERT INTO three_partition_tb PARTITION (part1='AS', part2=2025, part3='Q1') VALUES (13, 'Nina'); +INSERT INTO three_partition_tb PARTITION (part1='AS', part2=2025, part3='Q2') VALUES (14, 'Oscar'); +INSERT INTO three_partition_tb PARTITION (part1='AS', part2=2025, part3='Q3') VALUES (15, 'Paul'); +select * from one_partition_tb; +select * from two_partition_tb; +select * from three_partition_tb; +show partitions one_partition_tb; +show partitions two_partition_tb; +show partitions three_partition_tb; +*/ + +suite("test_max_compute_partition_prune", "p2,external,maxcompute,external_remote,external_remote_maxcompute") { + + + def one_partition_1_1 = """SELECT * FROM one_partition_tb WHERE part1 = 2024 ORDER BY id;""" + def one_partition_2_1 = """SELECT * FROM one_partition_tb WHERE part1 = 2025 ORDER BY id;""" + def one_partition_3_all = """SELECT * FROM one_partition_tb ORDER BY id;""" + def one_partition_4_all = """SELECT * FROM one_partition_tb WHERE id = 5 ORDER BY id;""" + def one_partition_5_1 = """SELECT * FROM one_partition_tb WHERE part1 = 2024 AND id >= 3 ORDER BY id;""" + + def two_partition_1_1 = """SELECT * FROM two_partition_tb WHERE part1 = 'US' AND part2 = 1 ORDER BY id;""" + def two_partition_2_1 = """SELECT * FROM two_partition_tb WHERE part1 = 'EU' AND part2 = 2 ORDER BY id;""" + def two_partition_3_2 = """SELECT * FROM two_partition_tb WHERE part1 = 'US' ORDER BY id;""" + def two_partition_4_all = """SELECT * FROM two_partition_tb ORDER BY id;""" + def two_partition_5_1 = """SELECT * FROM two_partition_tb WHERE part1 = 'US' AND part2 = 2 AND id > 5 ORDER BY id;""" + def two_partition_6_1 = """SELECT * FROM two_partition_tb WHERE part1 = 'EU' AND part2 = 2 ORDER BY id;""" + + def three_partition_1_1 = """SELECT * FROM three_partition_tb WHERE part1 = 'US' AND part2 = 2024 AND part3 = 'Q1' ORDER BY id;""" + def three_partition_2_1 = """SELECT * FROM three_partition_tb WHERE part1 = 'EU' AND part2 = 2025 AND part3 = 'Q2' ORDER BY id;""" + def three_partition_3_3 = """SELECT * FROM three_partition_tb WHERE part1 = 'AS' AND part2 = 2025 ORDER BY id;""" + def three_partition_4_2 = """SELECT * FROM three_partition_tb WHERE part1 = 'US' AND part3 = 'Q1' ORDER BY id;""" + def three_partition_5_all = """SELECT * FROM three_partition_tb ORDER BY id;""" + def three_partition_6_1 = """SELECT * FROM three_partition_tb WHERE part1 = 'EU' AND part2 = 2024 AND part3 = 'Q1' ORDER BY id;""" + def three_partition_7_7 = """SELECT * FROM three_partition_tb WHERE part2 = 2025 ORDER BY id;""" + def three_partition_8_2 = """SELECT * FROM three_partition_tb WHERE part1 = 'US' AND part3 = 'Q2' AND id BETWEEN 6 AND 10 ORDER BY id;""" + + + String enabled = context.config.otherConfigs.get("enableMaxComputeTest") + if (enabled != null && enabled.equalsIgnoreCase("true")) { + String ak = context.config.otherConfigs.get("ak") + String sk = context.config.otherConfigs.get("sk"); + String mc_db = "mc_datalake" + String mc_catalog_name = "test_max_compute_partition_prune" + + sql """drop catalog if exists ${mc_catalog_name};""" + sql """ + create catalog if not exists ${mc_catalog_name} properties ( + "type" = "max_compute", + "mc.default.project" = "${mc_db}", + "mc.access_key" = "${ak}", + "mc.secret_key" = "${sk}", + "mc.endpoint" = "http://service.cn-beijing-vpc.maxcompute.aliyun-inc.com/api" + ); + """ + sql """ switch ${mc_catalog_name} """ + sql """ use ${mc_db}""" + + qt_one_partition_1_1 one_partition_1_1 + explain { + sql("${one_partition_1_1}") + contains "partition=1/2" + } + + qt_one_partition_2_1 one_partition_2_1 + explain { + sql("${one_partition_2_1}") + contains "partition=1/2" + } + + qt_one_partition_3_all one_partition_3_all + explain { + sql("${one_partition_3_all}") + contains "partition=2/2" + } + + qt_one_partition_4_all one_partition_4_all + explain { + sql("${one_partition_4_all}") + contains "partition=2/2" + } + + qt_one_partition_5_1 one_partition_5_1 + explain { + sql("${one_partition_5_1}") + contains "partition=1/2" + } + + + qt_two_partition_1_1 two_partition_1_1 + explain { + sql("${two_partition_1_1}") + contains "partition=1/4" + } + + qt_two_partition_2_1 two_partition_2_1 + explain { + sql("${two_partition_2_1}") + contains "partition=1/4" + } + + qt_two_partition_3_2 two_partition_3_2 + explain { + sql("${two_partition_3_2}") + contains "partition=2/4" + } + + qt_two_partition_4_all two_partition_4_all + explain { + sql("${two_partition_4_all}") + contains "partition=4/4" + } + + qt_two_partition_5_1 two_partition_5_1 + explain { + sql("${two_partition_5_1}") + contains "partition=1/4" + } + + qt_two_partition_6_1 two_partition_6_1 + explain { + sql("${two_partition_6_1}") + contains "partition=1/4" + } + + + + qt_three_partition_1_1 three_partition_1_1 + explain { + sql("${three_partition_1_1}") + contains "partition=1/10" + } + + qt_three_partition_2_1 three_partition_2_1 + explain { + sql("${three_partition_2_1}") + contains "partition=1/10" + } + + qt_three_partition_3_3 three_partition_3_3 + explain { + sql("${three_partition_3_3}") + contains "partition=3/10" + } + + qt_three_partition_4_2 three_partition_4_2 + explain { + sql("${three_partition_4_2}") + contains "partition=2/10" + } + + qt_three_partition_5_all three_partition_5_all + explain { + sql("${three_partition_5_all}") + contains "partition=10/10" + } + + qt_three_partition_6_1 three_partition_6_1 + explain { + sql("${three_partition_6_1}") + contains "partition=1/10" + } + + qt_three_partition_7_7 three_partition_7_7 + explain { + sql("${three_partition_7_7}") + contains "partition=7/10" + } + + qt_three_partition_8_2 three_partition_8_2 + explain { + sql("${three_partition_8_2}") + contains "partition=2/10" + } + + + // 0 partitions + def one_partition_6_0 = """SELECT * FROM one_partition_tb WHERE part1 = 2023 ORDER BY id;""" + qt_one_partition_6_0 one_partition_6_0 + explain { + sql("${one_partition_6_0}") + contains "partition=0/2" + } + + def two_partition_7_0 = """SELECT * FROM two_partition_tb WHERE part1 = 'CN' AND part2 = 1 ORDER BY id;""" + qt_two_partition_7_0 two_partition_7_0 + explain { + sql("${two_partition_7_0}") + contains "partition=0/4" + } + + def two_partition_8_0 = """SELECT * FROM two_partition_tb WHERE part1 = 'US' AND part2 = 3 ORDER BY id;""" + qt_two_partition_8_0 two_partition_8_0 + explain { + sql("${two_partition_8_0}") + contains "partition=0/4" + } + + def three_partition_9_0 = """SELECT * FROM three_partition_tb WHERE part1 = 'US' AND part2 = 2023 AND part3 = 'Q1' ORDER BY id;""" + qt_three_partition_9_0 three_partition_9_0 + explain { + sql("${three_partition_9_0}") + contains "partition=0/10" + } + + def three_partition_10_0 = """SELECT * FROM three_partition_tb WHERE part1 = 'EU' AND part2 = 2024 AND part3 = 'Q4' ORDER BY id;""" + qt_three_partition_10_0 three_partition_10_0 + explain { + sql("${three_partition_10_0}") + contains "partition=0/10" + } + + def three_partition_11_0 = """SELECT * FROM three_partition_tb WHERE part1 = 'AS' AND part2 = 2025 AND part3 = 'Q4' ORDER BY id;""" + qt_three_partition_11_0 three_partition_11_0 + explain { + sql("${three_partition_11_0}") + contains "partition=0/10" + } + + } +} \ No newline at end of file From 9ca8048d4d3e60f431b8c46212bdc2d29d844f0b Mon Sep 17 00:00:00 2001 From: wuwenchi Date: Sun, 1 Dec 2024 22:18:42 +0800 Subject: [PATCH 092/399] [fix](iceberg)Bring field_id with parquet files And fix map type's key optional (#44470) ### What problem does this PR solve? 1. Column IDs are required to be stored as [field IDs](http://github.com/apache/parquet-format/blob/40699d05bd24181de6b1457babbee2c16dce3803/src/main/thrift/parquet.thrift#L459) on the parquet schema. ref: https://iceberg.apache.org/spec/?h=field+id#parquet So, we should add field ids. 2. For `MapType`, its key is always required. --- .../table/iceberg/arrow_schema_util.cpp | 134 ++++++++ .../format/table/iceberg/arrow_schema_util.h | 45 +++ .../vec/exec/format/table/iceberg/types.cpp | 3 +- be/src/vec/exec/format/table/iceberg/types.h | 4 + be/src/vec/runtime/vparquet_transformer.cpp | 51 +-- be/src/vec/runtime/vparquet_transformer.h | 5 +- .../iceberg/viceberg_partition_writer.cpp | 2 +- .../table/iceberg/arrow_schema_util_test.cpp | 304 ++++++++++++++++++ .../table/iceberg/schema_parser_test.cpp | 24 ++ 9 files changed, 547 insertions(+), 25 deletions(-) create mode 100644 be/src/vec/exec/format/table/iceberg/arrow_schema_util.cpp create mode 100644 be/src/vec/exec/format/table/iceberg/arrow_schema_util.h create mode 100644 be/test/vec/exec/format/table/iceberg/arrow_schema_util_test.cpp diff --git a/be/src/vec/exec/format/table/iceberg/arrow_schema_util.cpp b/be/src/vec/exec/format/table/iceberg/arrow_schema_util.cpp new file mode 100644 index 00000000000000..35a4d51b7f1959 --- /dev/null +++ b/be/src/vec/exec/format/table/iceberg/arrow_schema_util.cpp @@ -0,0 +1,134 @@ +// 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/exec/format/table/iceberg/arrow_schema_util.h" + +#include +#include + +namespace doris { +namespace iceberg { + +const char* ArrowSchemaUtil::PARQUET_FIELD_ID = "PARQUET:field_id"; +const char* ArrowSchemaUtil::ORIGINAL_TYPE = "originalType"; +const char* ArrowSchemaUtil::MAP_TYPE_VALUE = "mapType"; + +Status ArrowSchemaUtil::convert(const Schema* schema, const std::string& timezone, + std::vector>& fields) { + for (const auto& column : schema->columns()) { + std::shared_ptr arrow_field; + RETURN_IF_ERROR(convert_to(column, &arrow_field, timezone)); + fields.push_back(arrow_field); + } + return Status::OK(); +} + +Status ArrowSchemaUtil::convert_to(const iceberg::NestedField& field, + std::shared_ptr* arrow_field, + const std::string& timezone) { + std::shared_ptr arrow_type; + std::unordered_map metadata; + metadata[PARQUET_FIELD_ID] = std::to_string(field.field_id()); + + switch (field.field_type()->type_id()) { + case iceberg::TypeID::BOOLEAN: + arrow_type = arrow::boolean(); + break; + + case iceberg::TypeID::INTEGER: + arrow_type = arrow::int32(); + break; + + case iceberg::TypeID::LONG: + arrow_type = arrow::int64(); + break; + + case iceberg::TypeID::FLOAT: + arrow_type = arrow::float32(); + break; + + case iceberg::TypeID::DOUBLE: + arrow_type = arrow::float64(); + break; + + case iceberg::TypeID::DATE: + arrow_type = arrow::date32(); + break; + + case iceberg::TypeID::TIMESTAMP: { + arrow_type = std::make_shared(arrow::TimeUnit::MICRO, timezone); + break; + } + + case iceberg::TypeID::BINARY: + case iceberg::TypeID::STRING: + case iceberg::TypeID::UUID: + case iceberg::TypeID::FIXED: + arrow_type = arrow::utf8(); + break; + + case iceberg::TypeID::DECIMAL: { + auto dt = dynamic_cast(field.field_type()); + arrow_type = arrow::decimal(dt->get_precision(), dt->get_scale()); + break; + } + + case iceberg::TypeID::STRUCT: { + std::vector> element_fields; + StructType* st = field.field_type()->as_struct_type(); + for (const auto& column : st->fields()) { + std::shared_ptr element_field; + RETURN_IF_ERROR(convert_to(column, &element_field, timezone)); + element_fields.push_back(element_field); + } + arrow_type = arrow::struct_(element_fields); + break; + } + + case iceberg::TypeID::LIST: { + std::shared_ptr item_field; + ListType* list_type = field.field_type()->as_list_type(); + RETURN_IF_ERROR(convert_to(list_type->element_field(), &item_field, timezone)); + arrow_type = arrow::list(item_field); + break; + } + + case iceberg::TypeID::MAP: { + std::shared_ptr key_field; + std::shared_ptr value_field; + MapType* map_type = field.field_type()->as_map_type(); + RETURN_IF_ERROR(convert_to(map_type->key_field(), &key_field, timezone)); + RETURN_IF_ERROR(convert_to(map_type->value_field(), &value_field, timezone)); + metadata[ORIGINAL_TYPE] = MAP_TYPE_VALUE; + arrow_type = std::make_shared(key_field, value_field); + break; + } + + case iceberg::TypeID::TIME: + default: + return Status::InternalError("Unsupported field type:" + field.field_type()->to_string()); + } + + std::shared_ptr schema_metadata = + std::make_shared(metadata); + *arrow_field = + arrow::field(field.field_name(), arrow_type, field.is_optional(), schema_metadata); + return Status::OK(); +} + +} // namespace iceberg +} // namespace doris \ No newline at end of file diff --git a/be/src/vec/exec/format/table/iceberg/arrow_schema_util.h b/be/src/vec/exec/format/table/iceberg/arrow_schema_util.h new file mode 100644 index 00000000000000..20b7dbc627cc44 --- /dev/null +++ b/be/src/vec/exec/format/table/iceberg/arrow_schema_util.h @@ -0,0 +1,45 @@ +// 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 "vec/exec/format/table/iceberg/schema.h" + +namespace doris { +namespace iceberg { + +class ArrowSchemaUtil { +public: + static Status convert(const Schema* schema, const std::string& timezone, + std::vector>& fields); + +private: + static const char* PARQUET_FIELD_ID; + static const char* ORIGINAL_TYPE; + static const char* MAP_TYPE_VALUE; + + static Status convert_to(const iceberg::NestedField& field, + std::shared_ptr* arrow_field, + const std::string& timezone); +}; + +} // namespace iceberg +} // namespace doris diff --git a/be/src/vec/exec/format/table/iceberg/types.cpp b/be/src/vec/exec/format/table/iceberg/types.cpp index b56a231979ace1..bf643655ab8810 100644 --- a/be/src/vec/exec/format/table/iceberg/types.cpp +++ b/be/src/vec/exec/format/table/iceberg/types.cpp @@ -25,8 +25,9 @@ namespace iceberg { std::unique_ptr MapType::of_optional(int key_id, int value_id, std::unique_ptr key_type, std::unique_ptr value_type) { + // key is always required auto key_field = - std::make_unique(true, key_id, "key", std::move(key_type), std::nullopt); + std::make_unique(false, key_id, "key", std::move(key_type), std::nullopt); auto value_field = std::make_unique(true, value_id, "value", std::move(value_type), std::nullopt); return std::unique_ptr(new MapType(std::move(key_field), std::move(value_field))); diff --git a/be/src/vec/exec/format/table/iceberg/types.h b/be/src/vec/exec/format/table/iceberg/types.h index f5262b36f55cd3..91a2f705df0d0b 100644 --- a/be/src/vec/exec/format/table/iceberg/types.h +++ b/be/src/vec/exec/format/table/iceberg/types.h @@ -265,6 +265,10 @@ class DecimalType : public PrimitiveType { ss << "decimal(" << precision << ", " << scale << ")"; return ss.str(); } + + int get_precision() const { return precision; } + + int get_scale() const { return scale; } }; class BinaryType : public PrimitiveType { diff --git a/be/src/vec/runtime/vparquet_transformer.cpp b/be/src/vec/runtime/vparquet_transformer.cpp index ab288537313973..5409dc0abf3466 100644 --- a/be/src/vec/runtime/vparquet_transformer.cpp +++ b/be/src/vec/runtime/vparquet_transformer.cpp @@ -65,6 +65,7 @@ #include "vec/core/types.h" #include "vec/data_types/data_type_decimal.h" #include "vec/data_types/data_type_nullable.h" +#include "vec/exec/format/table/iceberg/arrow_schema_util.h" #include "vec/exprs/vexpr.h" #include "vec/exprs/vexpr_context.h" #include "vec/functions/function_helpers.h" @@ -202,21 +203,20 @@ void ParquetBuildHelper::build_version(parquet::WriterProperties::Builder& build } } -VParquetTransformer::VParquetTransformer(RuntimeState* state, doris::io::FileWriter* file_writer, - const VExprContextSPtrs& output_vexpr_ctxs, - std::vector column_names, - TParquetCompressionType::type compression_type, - bool parquet_disable_dictionary, - TParquetVersion::type parquet_version, - bool output_object_data, - const std::string* iceberg_schema_json) +VParquetTransformer::VParquetTransformer( + RuntimeState* state, doris::io::FileWriter* file_writer, + const VExprContextSPtrs& output_vexpr_ctxs, std::vector column_names, + TParquetCompressionType::type compression_type, bool parquet_disable_dictionary, + TParquetVersion::type parquet_version, bool output_object_data, + const std::string* iceberg_schema_json, const iceberg::Schema* iceberg_schema) : VFileFormatTransformer(state, output_vexpr_ctxs, output_object_data), _column_names(std::move(column_names)), _parquet_schemas(nullptr), _compression_type(compression_type), _parquet_disable_dictionary(parquet_disable_dictionary), _parquet_version(parquet_version), - _iceberg_schema_json(iceberg_schema_json) { + _iceberg_schema_json(iceberg_schema_json), + _iceberg_schema(iceberg_schema) { _outstream = std::shared_ptr(new ParquetOutputStream(file_writer)); } @@ -234,6 +234,7 @@ VParquetTransformer::VParquetTransformer(RuntimeState* state, doris::io::FileWri _parquet_disable_dictionary(parquet_disable_dictionary), _parquet_version(parquet_version), _iceberg_schema_json(iceberg_schema_json) { + _iceberg_schema = nullptr; _outstream = std::shared_ptr(new ParquetOutputStream(file_writer)); } @@ -265,21 +266,27 @@ Status VParquetTransformer::_parse_properties() { Status VParquetTransformer::_parse_schema() { std::vector> fields; - for (size_t i = 0; i < _output_vexpr_ctxs.size(); i++) { - std::shared_ptr type; - RETURN_IF_ERROR(convert_to_arrow_type(_output_vexpr_ctxs[i]->root()->type(), &type, - _state->timezone())); - if (_parquet_schemas != nullptr) { - std::shared_ptr field = - arrow::field(_parquet_schemas->operator[](i).schema_column_name, type, - _output_vexpr_ctxs[i]->root()->is_nullable()); - fields.emplace_back(field); - } else { - std::shared_ptr field = arrow::field( - _column_names[i], type, _output_vexpr_ctxs[i]->root()->is_nullable()); - fields.emplace_back(field); + if (_iceberg_schema != nullptr) { + RETURN_IF_ERROR( + iceberg::ArrowSchemaUtil::convert(_iceberg_schema, _state->timezone(), fields)); + } else { + for (size_t i = 0; i < _output_vexpr_ctxs.size(); i++) { + std::shared_ptr type; + RETURN_IF_ERROR(convert_to_arrow_type(_output_vexpr_ctxs[i]->root()->type(), &type, + _state->timezone())); + if (_parquet_schemas != nullptr) { + std::shared_ptr field = + arrow::field(_parquet_schemas->operator[](i).schema_column_name, type, + _output_vexpr_ctxs[i]->root()->is_nullable()); + fields.emplace_back(field); + } else { + std::shared_ptr field = arrow::field( + _column_names[i], type, _output_vexpr_ctxs[i]->root()->is_nullable()); + fields.emplace_back(field); + } } } + if (_iceberg_schema_json != nullptr) { std::shared_ptr schema_metadata = arrow::KeyValueMetadata::Make({"iceberg.schema"}, {*_iceberg_schema_json}); diff --git a/be/src/vec/runtime/vparquet_transformer.h b/be/src/vec/runtime/vparquet_transformer.h index 9fdbb271373212..ecc4a8ddeac4bc 100644 --- a/be/src/vec/runtime/vparquet_transformer.h +++ b/be/src/vec/runtime/vparquet_transformer.h @@ -27,6 +27,7 @@ #include #include +#include "vec/exec/format/table/iceberg/schema.h" #include "vfile_format_transformer.h" namespace doris { @@ -95,7 +96,8 @@ class VParquetTransformer final : public VFileFormatTransformer { std::vector column_names, TParquetCompressionType::type compression_type, bool parquet_disable_dictionary, TParquetVersion::type parquet_version, - bool output_object_data, const std::string* iceberg_schema_json = nullptr); + bool output_object_data, const std::string* iceberg_schema_json = nullptr, + const iceberg::Schema* iceberg_schema = nullptr); VParquetTransformer(RuntimeState* state, doris::io::FileWriter* file_writer, const VExprContextSPtrs& output_vexpr_ctxs, @@ -132,6 +134,7 @@ class VParquetTransformer final : public VFileFormatTransformer { const TParquetVersion::type _parquet_version; const std::string* _iceberg_schema_json; uint64_t _write_size = 0; + const iceberg::Schema* _iceberg_schema; }; } // namespace doris::vectorized diff --git a/be/src/vec/sink/writer/iceberg/viceberg_partition_writer.cpp b/be/src/vec/sink/writer/iceberg/viceberg_partition_writer.cpp index 924adf68145a7a..23ee389a8b7d10 100644 --- a/be/src/vec/sink/writer/iceberg/viceberg_partition_writer.cpp +++ b/be/src/vec/sink/writer/iceberg/viceberg_partition_writer.cpp @@ -84,7 +84,7 @@ Status VIcebergPartitionWriter::open(RuntimeState* state, RuntimeProfile* profil _file_format_transformer.reset(new VParquetTransformer( state, _file_writer.get(), _write_output_expr_ctxs, _write_column_names, parquet_compression_type, parquet_disable_dictionary, TParquetVersion::PARQUET_1_0, - false, _iceberg_schema_json)); + false, _iceberg_schema_json, &_schema)); return _file_format_transformer->open(); } case TFileFormatType::FORMAT_ORC: { diff --git a/be/test/vec/exec/format/table/iceberg/arrow_schema_util_test.cpp b/be/test/vec/exec/format/table/iceberg/arrow_schema_util_test.cpp new file mode 100644 index 00000000000000..b5f61c9d2e3cdf --- /dev/null +++ b/be/test/vec/exec/format/table/iceberg/arrow_schema_util_test.cpp @@ -0,0 +1,304 @@ +// 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/exec/format/table/iceberg/arrow_schema_util.h" + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include "io/fs/local_file_system.h" +#include "vec/exec/format/table/iceberg/schema.h" +#include "vec/exec/format/table/iceberg/schema_parser.h" + +namespace doris { +namespace iceberg { + +class ArrowSchemaUtilTest : public testing::Test { +public: + ArrowSchemaUtilTest() = default; + virtual ~ArrowSchemaUtilTest() = default; +}; + +const std::string_view pfid = "PARQUET:field_id"; + +TEST(ArrowSchemaUtilTest, test_simple_field) { + std::vector nested_fields; + nested_fields.reserve(2); + NestedField field1(false, 1, "field1", std::make_unique(), std::nullopt); + NestedField field2(false, 2, "field2", std::make_unique(), std::nullopt); + nested_fields.emplace_back(std::move(field1)); + nested_fields.emplace_back(std::move(field2)); + + Schema schema(1, std::move(nested_fields)); + + std::vector> fields; + Status st; + st = ArrowSchemaUtil::convert(&schema, "utc", fields); + EXPECT_TRUE(st.ok()); + EXPECT_EQ(2, fields.size()); + EXPECT_EQ("field1", fields[0]->name()); + EXPECT_EQ("field2", fields[1]->name()); + EXPECT_TRUE(fields[0]->HasMetadata()); + EXPECT_TRUE(fields[1]->HasMetadata()); + EXPECT_EQ("1", fields[0]->metadata()->Get(pfid).ValueUnsafe()); + EXPECT_EQ("2", fields[1]->metadata()->Get(pfid).ValueUnsafe()); +} + +TEST(ArrowSchemaUtilTest, test_stuct_field) { + // struct_json comes from : + // Schema schema = new Schema( + // Types.NestedField.optional( + // 21, "st_col", Types.StructType.of( + // Types.NestedField.optional(32, "st_col_c1", Types.IntegerType.get()), + // Types.NestedField.optional(43, "st_col_c2", Types.StringType.get()) + // ) + // ) + // ); + // StringWriter writer = new StringWriter(); + // JsonGenerator generator = JsonUtil.factory().createGenerator(writer); + // SchemaParser.toJson(schema.asStruct(), generator); + // generator.flush(); + // System.out.println(writer.toString()); + + const std::string struct_json = R"({ + "type": "struct", + "fields": [ + { + "id": 21, + "name": "st_col", + "required": false, + "type": { + "type": "struct", + "fields": [ + { + "id": 32, + "name": "st_col_c1", + "required": false, + "type": "int" + }, + { + "id": 43, + "name": "st_col_c2", + "required": false, + "type": "string" + } + ] + } + } + ] + })"; + std::unique_ptr schema = SchemaParser::from_json(struct_json); + + std::vector> fields; + Status st; + st = ArrowSchemaUtil::convert(schema.get(), "utc", fields); + EXPECT_TRUE(st.ok()); + EXPECT_EQ(1, fields.size()); + EXPECT_EQ("st_col", fields[0]->name()); + EXPECT_EQ("21", fields[0]->metadata()->Get(pfid).ValueUnsafe()); + + arrow::StructType* arrow_struct = dynamic_cast(fields[0]->type().get()); + auto map_fields = arrow_struct->fields(); + EXPECT_EQ(2, arrow_struct->fields().size()); + EXPECT_EQ("st_col_c1", map_fields.at(0).get()->name()); + EXPECT_EQ("st_col_c2", map_fields.at(1).get()->name()); + EXPECT_EQ("32", map_fields.at(0).get()->metadata()->Get(pfid).ValueUnsafe()); + EXPECT_EQ("43", map_fields.at(1).get()->metadata()->Get(pfid).ValueUnsafe()); +} + +TEST(ArrowSchemaUtilTest, test_map_field) { + // map_json comes from : + // Schema schema = new Schema( + // Types.NestedField.optional( + // 21, "map_col", Types.MapType.ofOptional( + // 32, 43, Types.IntegerType.get(), Types.StringType.get() + // ) + // ) + // ); + // StringWriter writer = new StringWriter(); + // JsonGenerator generator = JsonUtil.factory().createGenerator(writer); + // SchemaParser.toJson(schema.asStruct(), generator); + // generator.flush(); + // System.out.println(writer.toString()); + + const std::string map_json = R"({ + "type": "struct", + "fields": [ + { + "id": 21, + "name": "map_col", + "required": false, + "type": { + "type": "map", + "key-id": 32, + "key": "int", + "value-id": 43, + "value": "string", + "value-required": false + } + } + ] + })"; + std::unique_ptr schema = SchemaParser::from_json(map_json); + + std::vector> fields; + Status st; + st = ArrowSchemaUtil::convert(schema.get(), "utc", fields); + EXPECT_TRUE(st.ok()); + EXPECT_EQ(1, fields.size()); + EXPECT_EQ("map_col", fields[0]->name()); + EXPECT_EQ("21", fields[0]->metadata()->Get(pfid).ValueUnsafe()); + + arrow::MapType* arrow_map = dynamic_cast(fields[0]->type().get()); + auto map_fields = arrow_map->fields(); + EXPECT_EQ(1, arrow_map->fields().size()); + EXPECT_EQ("key", arrow_map->key_field()->name()); + EXPECT_EQ("value", arrow_map->item_field()->name()); + EXPECT_EQ("32", arrow_map->key_field()->metadata()->Get(pfid).ValueUnsafe()); + EXPECT_EQ("43", arrow_map->item_field()->metadata()->Get(pfid).ValueUnsafe()); +} + +TEST(ArrowSchemaUtilTest, test_list_field) { + // list_json comes from : + // Schema schema = new Schema( + // Types.NestedField.optional( + // 21, "list_col", Types.ListType.ofOptional( + // 32, Types.IntegerType.get()))); + // StringWriter writer = new StringWriter(); + // JsonGenerator generator = JsonUtil.factory().createGenerator(writer); + // SchemaParser.toJson(schema.asStruct(), generator); + // generator.flush(); + // System.out.println(writer.toString()); + + const std::string list_json = R"({ + "type": "struct", + "fields": [ + { + "id": 21, + "name": "list_col", + "required": false, + "type": { + "type": "list", + "element-id": 32, + "element": "int", + "element-required": false + } + } + ] + })"; + std::unique_ptr schema = SchemaParser::from_json(list_json); + + std::vector> fields; + Status st; + st = ArrowSchemaUtil::convert(schema.get(), "utc", fields); + EXPECT_TRUE(st.ok()); + EXPECT_EQ(1, fields.size()); + EXPECT_EQ("list_col", fields[0]->name()); + EXPECT_EQ("21", fields[0]->metadata()->Get(pfid).ValueUnsafe()); + + arrow::ListType* arrow_list = dynamic_cast(fields[0]->type().get()); + auto map_fields = arrow_list->fields(); + EXPECT_EQ(1, arrow_list->fields().size()); + EXPECT_EQ("element", arrow_list->value_field()->name()); + EXPECT_EQ("32", arrow_list->value_field()->metadata()->Get(pfid).ValueUnsafe()); +} + +TEST(ArrowSchemaUtilTest, test_parquet_filed_id) { + std::string test_dir = "ut_dir/test_parquet_filed_id"; + Status st; + st = io::global_local_filesystem()->delete_directory(test_dir); + ASSERT_TRUE(st.ok()) << st; + st = io::global_local_filesystem()->create_directory(test_dir); + ASSERT_TRUE(st.ok()) << st; + + std::shared_ptr id_array; + std::shared_ptr name_array; + + arrow::Int32Builder id_builder; + ASSERT_TRUE(id_builder.Append(1).ok()); + ASSERT_TRUE(id_builder.Append(2).ok()); + ASSERT_TRUE(id_builder.Append(3).ok()); + auto&& result_id = id_builder.Finish(); + ASSERT_TRUE(result_id.ok()); + id_array = std::move(result_id).ValueUnsafe(); + + arrow::StringBuilder name_builder; + ASSERT_TRUE(name_builder.Append("Alice").ok()); + ASSERT_TRUE(name_builder.Append("Bob").ok()); + ASSERT_TRUE(name_builder.Append("Charlie").ok()); + auto&& result_name = name_builder.Finish(); + ASSERT_TRUE(result_name.ok()); + name_array = std::move(result_name).ValueUnsafe(); + + // 定义表的 Schema + std::vector nested_fields; + nested_fields.reserve(2); + NestedField field1(false, 17, "field_1", std::make_unique(), std::nullopt); + NestedField field2(false, 36, "field_2", std::make_unique(), std::nullopt); + nested_fields.emplace_back(std::move(field1)); + nested_fields.emplace_back(std::move(field2)); + + Schema schema(1, std::move(nested_fields)); + + std::vector> fields; + st = ArrowSchemaUtil::convert(&schema, "utc", fields); + auto arrow_schema = arrow::schema(fields); + + // create arrow table + auto table = arrow::Table::Make(arrow_schema, {id_array, name_array}); + + std::string file_path = test_dir + "/f1.parquet"; + std::shared_ptr outfile; + auto&& result_file = arrow::io::FileOutputStream::Open(file_path); + ASSERT_TRUE(result_file.ok()); + outfile = std::move(result_file).ValueUnsafe(); + + // arrow table to parquet file + PARQUET_THROW_NOT_OK( + parquet::arrow::WriteTable(*table, arrow::default_memory_pool(), outfile, 1024)); + + // open parquet with parquet's API + std::unique_ptr parquet_reader = + parquet::ParquetFileReader::OpenFile(file_path, false); + + // get MessageType + std::shared_ptr file_metadata = parquet_reader->metadata(); + auto schema_descriptor = file_metadata->schema(); + const parquet::schema::Node& root = *schema_descriptor->group_node(); + const auto& group_node = static_cast(root); + + EXPECT_EQ(2, group_node.field_count()); + auto filed1 = group_node.field(0); + auto filed2 = group_node.field(1); + EXPECT_EQ("field_1", filed1->name()); + EXPECT_EQ(17, filed1->field_id()); + EXPECT_EQ("field_2", filed2->name()); + EXPECT_EQ(36, filed2->field_id()); + + st = io::global_local_filesystem()->delete_directory(test_dir); + EXPECT_TRUE(st.ok()) << st; +} + +} // namespace iceberg +} // namespace doris diff --git a/be/test/vec/exec/format/table/iceberg/schema_parser_test.cpp b/be/test/vec/exec/format/table/iceberg/schema_parser_test.cpp index 4c3f58cdd10491..f464525a7f99bc 100644 --- a/be/test/vec/exec/format/table/iceberg/schema_parser_test.cpp +++ b/be/test/vec/exec/format/table/iceberg/schema_parser_test.cpp @@ -78,6 +78,15 @@ const std::string valid_map_json = R"({ "value-required": true })"; +const std::string valid_map_json2 = R"({ + "type": "map", + "key-id": 4, + "key": "string", + "value-id": 5, + "value": "int", + "value-required": false +})"; + const std::string nested_list_json = R"({ "type": "list", "element-id": 6, @@ -209,6 +218,21 @@ TEST(SchemaParserTest, parse_valid_map) { SchemaParser::_type_from_json(rapidjson::Document().Parse(valid_map_json.c_str())); ASSERT_NE(type, nullptr); EXPECT_EQ(type->to_string(), "map"); + EXPECT_TRUE(type->is_map_type()); + MapType* mt = type->as_map_type(); + EXPECT_TRUE(mt->field(4)->is_required()); + EXPECT_TRUE(mt->field(5)->is_required()); +} + +TEST(SchemaParserTest, parse_valid_map2) { + std::unique_ptr type = + SchemaParser::_type_from_json(rapidjson::Document().Parse(valid_map_json2.c_str())); + ASSERT_NE(type, nullptr); + EXPECT_EQ(type->to_string(), "map"); + EXPECT_TRUE(type->is_map_type()); + MapType* mt = type->as_map_type(); + EXPECT_TRUE(mt->field(4)->is_required()); + EXPECT_TRUE(mt->field(5)->is_optional()); } TEST(SchemaParserTest, parse_nested_list) { From 456a08ba746900ad1167a7c5397775195ade8195 Mon Sep 17 00:00:00 2001 From: starocean999 Date: Mon, 2 Dec 2024 09:41:10 +0800 Subject: [PATCH 093/399] [refactor](nereids)add NotNullableAggregateFunction as base class for all non-nullable agg functions (#44671) AlwaysNotNullable agg functions must return a non-null value if it's a scalar agg. So a new base class NotNullableAggregateFunction is introduced, all sub classes must implement resultForEmptyInput method to get the non-null value correctly --- .../rules/analysis/SubqueryToApply.java | 8 ++-- .../functions/AlwaysNotNullable.java | 9 ---- .../expressions/functions/agg/ArrayAgg.java | 5 +- .../expressions/functions/agg/BitmapAgg.java | 5 +- .../functions/agg/BitmapIntersect.java | 5 +- .../functions/agg/BitmapUnion.java | 5 +- .../functions/agg/BitmapUnionCount.java | 5 +- .../functions/agg/BitmapUnionInt.java | 5 +- .../functions/agg/CollectList.java | 5 +- .../expressions/functions/agg/CollectSet.java | 5 +- .../expressions/functions/agg/Count.java | 5 +- .../functions/agg/CountByEnum.java | 3 +- .../functions/agg/GroupArrayIntersect.java | 5 +- .../expressions/functions/agg/Histogram.java | 5 +- .../expressions/functions/agg/HllUnion.java | 5 +- .../functions/agg/HllUnionAgg.java | 5 +- .../functions/agg/IntersectCount.java | 5 +- .../functions/agg/LinearHistogram.java | 9 +++- .../expressions/functions/agg/MapAgg.java | 5 +- .../functions/agg/MultiDistinctCount.java | 5 +- .../functions/agg/MultiDistinctSum0.java | 5 +- .../trees/expressions/functions/agg/Ndv.java | 5 +- .../agg/NotNullableAggregateFunction.java | 48 +++++++++++++++++++ .../agg/OrthogonalBitmapIntersect.java | 11 +++-- .../agg/OrthogonalBitmapIntersectCount.java | 13 +++-- .../agg/OrthogonalBitmapUnionCount.java | 11 +++-- .../functions/agg/PercentileArray.java | 5 +- .../functions/agg/QuantileUnion.java | 5 +- .../functions/agg/SequenceCount.java | 5 +- .../trees/expressions/functions/agg/Sum0.java | 5 +- 30 files changed, 129 insertions(+), 93 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/NotNullableAggregateFunction.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/SubqueryToApply.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/SubqueryToApply.java index 14700b030d68b7..4bb55eab9d4378 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/SubqueryToApply.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/SubqueryToApply.java @@ -42,9 +42,9 @@ import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.SlotReference; import org.apache.doris.nereids.trees.expressions.SubqueryExpr; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.agg.AnyValue; import org.apache.doris.nereids.trees.expressions.functions.agg.Count; +import org.apache.doris.nereids.trees.expressions.functions.agg.NotNullableAggregateFunction; import org.apache.doris.nereids.trees.expressions.functions.scalar.AssertTrue; import org.apache.doris.nereids.trees.expressions.functions.scalar.Nvl; import org.apache.doris.nereids.trees.expressions.literal.BooleanLiteral; @@ -424,9 +424,9 @@ private Pair> addApply(SubqueryExpr subquery, Map replaceMap = new HashMap<>(); NamedExpression agg = ((ScalarSubquery) subquery).getTopLevelScalarAggFunction().get(); if (agg instanceof Alias) { - if (((Alias) agg).child() instanceof AlwaysNotNullable) { - AlwaysNotNullable notNullableAggFunc = - (AlwaysNotNullable) ((Alias) agg).child(); + if (((Alias) agg).child() instanceof NotNullableAggregateFunction) { + NotNullableAggregateFunction notNullableAggFunc = + (NotNullableAggregateFunction) ((Alias) agg).child(); if (subquery.getQueryPlan() instanceof LogicalProject) { LogicalProject logicalProject = (LogicalProject) subquery.getQueryPlan(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/AlwaysNotNullable.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/AlwaysNotNullable.java index 6b12f9cd642992..8fda4d4b020f2b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/AlwaysNotNullable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/AlwaysNotNullable.java @@ -17,9 +17,6 @@ package org.apache.doris.nereids.trees.expressions.functions; -import org.apache.doris.nereids.exceptions.AnalysisException; -import org.apache.doris.nereids.trees.expressions.Expression; - /** * nullable is always false. * @@ -30,10 +27,4 @@ public interface AlwaysNotNullable extends ComputeNullable { default boolean nullable() { return false; } - - // return value of this function if the input data is empty. - // for example, count(*) of empty table is 0; - default Expression resultForEmptyInput() { - throw new AnalysisException("should implement resultForEmptyInput() for " + this.getClass()); - } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/ArrayAgg.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/ArrayAgg.java index bc91207e31f3a6..6218569261a775 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/ArrayAgg.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/ArrayAgg.java @@ -19,7 +19,6 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.literal.ArrayLiteral; import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression; @@ -37,8 +36,8 @@ /** * AggregateFunction 'array_agg'. */ -public class ArrayAgg extends AggregateFunction - implements UnaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable { +public class ArrayAgg extends NotNullableAggregateFunction + implements UnaryExpression, ExplicitlyCastableSignature { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(ArrayType.of(new FollowToAnyDataType(0))).args(new AnyDataType(0)) ); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapAgg.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapAgg.java index eaf766b908d7af..a0eaf88efaf13b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapAgg.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapAgg.java @@ -19,7 +19,6 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.functions.scalar.BitmapEmpty; import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression; @@ -37,8 +36,8 @@ /** * AggregateFunction 'bitmap_agg'. */ -public class BitmapAgg extends AggregateFunction - implements UnaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable { +public class BitmapAgg extends NotNullableAggregateFunction + implements UnaryExpression, ExplicitlyCastableSignature { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(BitmapType.INSTANCE).args(BigIntType.INSTANCE), FunctionSignature.ret(BitmapType.INSTANCE).args(IntegerType.INSTANCE), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapIntersect.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapIntersect.java index 1b7d2d3c3cddec..80a9887d325634 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapIntersect.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapIntersect.java @@ -19,7 +19,6 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.functions.scalar.BitmapEmpty; import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression; @@ -35,8 +34,8 @@ /** * AggregateFunction 'bitmap_intersect'. This class is generated by GenerateFunction. */ -public class BitmapIntersect extends AggregateFunction - implements UnaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable, BitmapFunction { +public class BitmapIntersect extends NotNullableAggregateFunction + implements UnaryExpression, ExplicitlyCastableSignature, BitmapFunction { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(BitmapType.INSTANCE).args(BitmapType.INSTANCE) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapUnion.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapUnion.java index cd0756a1c9361f..fce2f4fa70c483 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapUnion.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapUnion.java @@ -19,7 +19,6 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.functions.Function; import org.apache.doris.nereids.trees.expressions.functions.scalar.BitmapEmpty; @@ -36,8 +35,8 @@ /** * AggregateFunction 'bitmap_union'. This class is generated by GenerateFunction. */ -public class BitmapUnion extends AggregateFunction - implements UnaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable, BitmapFunction, RollUpTrait { +public class BitmapUnion extends NotNullableAggregateFunction + implements UnaryExpression, ExplicitlyCastableSignature, BitmapFunction, RollUpTrait { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(BitmapType.INSTANCE).args(BitmapType.INSTANCE) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapUnionCount.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapUnionCount.java index 593c814f22d159..d1e2d6010a27a7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapUnionCount.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapUnionCount.java @@ -19,7 +19,6 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.functions.Function; import org.apache.doris.nereids.trees.expressions.literal.BigIntLiteral; @@ -37,8 +36,8 @@ /** * AggregateFunction 'bitmap_union_count'. This class is generated by GenerateFunction. */ -public class BitmapUnionCount extends AggregateFunction - implements UnaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable, BitmapFunction, RollUpTrait { +public class BitmapUnionCount extends NotNullableAggregateFunction + implements UnaryExpression, ExplicitlyCastableSignature, BitmapFunction, RollUpTrait { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(BigIntType.INSTANCE).args(BitmapType.INSTANCE) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapUnionInt.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapUnionInt.java index 2efe1631176c86..af569982c4e62e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapUnionInt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapUnionInt.java @@ -20,7 +20,6 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.literal.BigIntLiteral; import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression; @@ -41,8 +40,8 @@ /** * AggregateFunction 'bitmap_union_int'. This class is generated by GenerateFunction. */ -public class BitmapUnionInt extends AggregateFunction - implements UnaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable { +public class BitmapUnionInt extends NotNullableAggregateFunction + implements UnaryExpression, ExplicitlyCastableSignature { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(BigIntType.INSTANCE).args(SmallIntType.INSTANCE), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/CollectList.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/CollectList.java index d6cca2d0b90b6d..ed0dd2406211ce 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/CollectList.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/CollectList.java @@ -19,7 +19,6 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.literal.ArrayLiteral; import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression; @@ -38,8 +37,8 @@ /** * AggregateFunction 'collect_list'. This class is generated by GenerateFunction. */ -public class CollectList extends AggregateFunction - implements UnaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable { +public class CollectList extends NotNullableAggregateFunction + implements UnaryExpression, ExplicitlyCastableSignature { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(ArrayType.of(new FollowToAnyDataType(0))).args(new AnyDataType(0)), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/CollectSet.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/CollectSet.java index d9e7e7227c6f86..2aba485fc7daaa 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/CollectSet.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/CollectSet.java @@ -19,7 +19,6 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.literal.ArrayLiteral; import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression; @@ -38,8 +37,8 @@ /** * AggregateFunction 'collect_set'. This class is generated by GenerateFunction. */ -public class CollectSet extends AggregateFunction - implements UnaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable { +public class CollectSet extends NotNullableAggregateFunction + implements UnaryExpression, ExplicitlyCastableSignature { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(ArrayType.of(new FollowToAnyDataType(0))).args(new AnyDataType(0)), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Count.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Count.java index 10874d47ee3df9..e86e90974da1bd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Count.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Count.java @@ -20,7 +20,6 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.functions.Function; import org.apache.doris.nereids.trees.expressions.functions.window.SupportWindowAnalytic; @@ -37,8 +36,8 @@ import java.util.List; /** count agg function. */ -public class Count extends AggregateFunction - implements ExplicitlyCastableSignature, AlwaysNotNullable, SupportWindowAnalytic, RollUpTrait { +public class Count extends NotNullableAggregateFunction + implements ExplicitlyCastableSignature, SupportWindowAnalytic, RollUpTrait { public static final List SIGNATURES = ImmutableList.of( // count(*) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/CountByEnum.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/CountByEnum.java index 2a4ee7be3f49c2..f956854d6871bd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/CountByEnum.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/CountByEnum.java @@ -19,7 +19,6 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.literal.StringLiteral; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; @@ -31,7 +30,7 @@ import java.util.List; /** count_by_enum agg function. */ -public class CountByEnum extends AggregateFunction implements ExplicitlyCastableSignature, AlwaysNotNullable { +public class CountByEnum extends NotNullableAggregateFunction implements ExplicitlyCastableSignature { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(StringType.INSTANCE).varArgs(StringType.INSTANCE) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/GroupArrayIntersect.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/GroupArrayIntersect.java index 0720d6838bb1b6..1cc0db1bd5ed5a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/GroupArrayIntersect.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/GroupArrayIntersect.java @@ -19,7 +19,6 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.literal.ArrayLiteral; import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression; @@ -36,8 +35,8 @@ /** * AggregateFunction 'group_array_intersect'. */ -public class GroupArrayIntersect extends AggregateFunction - implements UnaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable { +public class GroupArrayIntersect extends NotNullableAggregateFunction + implements UnaryExpression, ExplicitlyCastableSignature { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.retArgType(0) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Histogram.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Histogram.java index 6b0a2759823f3a..827c57facd7498 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Histogram.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Histogram.java @@ -19,7 +19,6 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.functions.SearchSignature; import org.apache.doris.nereids.trees.expressions.literal.VarcharLiteral; @@ -37,8 +36,8 @@ /** * AggregateFunction 'histogram'. This class is generated by GenerateFunction. */ -public class Histogram extends AggregateFunction - implements ExplicitlyCastableSignature, AlwaysNotNullable { +public class Histogram extends NotNullableAggregateFunction + implements ExplicitlyCastableSignature { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(VarcharType.SYSTEM_DEFAULT) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/HllUnion.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/HllUnion.java index b81fad270b0cda..8f6224f66b0043 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/HllUnion.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/HllUnion.java @@ -19,7 +19,6 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.functions.Function; import org.apache.doris.nereids.trees.expressions.functions.scalar.HllEmpty; @@ -36,8 +35,8 @@ /** * AggregateFunction 'hll_union'. This class is generated by GenerateFunction. */ -public class HllUnion extends AggregateFunction - implements UnaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable, HllFunction, RollUpTrait { +public class HllUnion extends NotNullableAggregateFunction + implements UnaryExpression, ExplicitlyCastableSignature, HllFunction, RollUpTrait { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(HllType.INSTANCE).args(HllType.INSTANCE) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/HllUnionAgg.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/HllUnionAgg.java index b14b61b5be07e9..b2c15b402aa2c4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/HllUnionAgg.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/HllUnionAgg.java @@ -19,7 +19,6 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.functions.Function; import org.apache.doris.nereids.trees.expressions.literal.BigIntLiteral; @@ -37,8 +36,8 @@ /** * AggregateFunction 'hll_union_agg'. This class is generated by GenerateFunction. */ -public class HllUnionAgg extends AggregateFunction - implements UnaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable, HllFunction, RollUpTrait { +public class HllUnionAgg extends NotNullableAggregateFunction + implements UnaryExpression, ExplicitlyCastableSignature, HllFunction, RollUpTrait { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(BigIntType.INSTANCE).args(HllType.INSTANCE) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/IntersectCount.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/IntersectCount.java index c013b2e8b4c31d..3eed9b9a6e4ddb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/IntersectCount.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/IntersectCount.java @@ -19,7 +19,6 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.BitmapIntersectFunction; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.literal.BigIntLiteral; @@ -37,8 +36,8 @@ /** * AggregateFunction 'intersect_count'. This class is generated by GenerateFunction. */ -public class IntersectCount extends AggregateFunction - implements ExplicitlyCastableSignature, AlwaysNotNullable, BitmapIntersectFunction { +public class IntersectCount extends NotNullableAggregateFunction + implements ExplicitlyCastableSignature, BitmapIntersectFunction { public static final List SIGNATURES = DataType.trivialTypes().stream() .map(type -> FunctionSignature.ret(BigIntType.INSTANCE).varArgs(BitmapType.INSTANCE, type, type)) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/LinearHistogram.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/LinearHistogram.java index ac172fc7881c7c..ca8cf3a2f71560 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/LinearHistogram.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/LinearHistogram.java @@ -20,9 +20,9 @@ import org.apache.doris.catalog.FunctionSet; import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.functions.SearchSignature; +import org.apache.doris.nereids.trees.expressions.literal.VarcharLiteral; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; import org.apache.doris.nereids.types.DoubleType; import org.apache.doris.nereids.types.VarcharType; @@ -36,7 +36,7 @@ /** * AggregateFunction 'linear_histogram'. */ -public class LinearHistogram extends AggregateFunction implements ExplicitlyCastableSignature, AlwaysNotNullable { +public class LinearHistogram extends NotNullableAggregateFunction implements ExplicitlyCastableSignature { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(VarcharType.SYSTEM_DEFAULT) @@ -86,4 +86,9 @@ public R accept(ExpressionVisitor visitor, C context) { public List getSignatures() { return SIGNATURES; } + + @Override + public Expression resultForEmptyInput() { + return new VarcharLiteral("{\"num_buckets\":0,\"buckets\":[]}"); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/MapAgg.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/MapAgg.java index 744d4a23a667e5..798b3677ff127d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/MapAgg.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/MapAgg.java @@ -19,7 +19,6 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.literal.MapLiteral; import org.apache.doris.nereids.trees.expressions.shape.BinaryExpression; @@ -36,8 +35,8 @@ /** * AggregateFunction 'map_agg'. This class is generated by GenerateFunction. */ -public class MapAgg extends AggregateFunction - implements BinaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable { +public class MapAgg extends NotNullableAggregateFunction + implements BinaryExpression, ExplicitlyCastableSignature { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(MapType.of(new FollowToAnyDataType(0), new FollowToAnyDataType(1))) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/MultiDistinctCount.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/MultiDistinctCount.java index 68d31e3e7bd19d..81768ef48dd960 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/MultiDistinctCount.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/MultiDistinctCount.java @@ -21,7 +21,6 @@ import org.apache.doris.nereids.analyzer.Unbound; import org.apache.doris.nereids.trees.expressions.Cast; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.literal.BigIntLiteral; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; @@ -36,8 +35,8 @@ import java.util.List; /** MultiDistinctCount */ -public class MultiDistinctCount extends AggregateFunction - implements AlwaysNotNullable, ExplicitlyCastableSignature, MultiDistinction { +public class MultiDistinctCount extends NotNullableAggregateFunction + implements ExplicitlyCastableSignature, MultiDistinction { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(BigIntType.INSTANCE).varArgs(AnyDataType.INSTANCE_WITHOUT_INDEX) ); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/MultiDistinctSum0.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/MultiDistinctSum0.java index 2b0eda06b4264a..abd5292e31a233 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/MultiDistinctSum0.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/MultiDistinctSum0.java @@ -20,7 +20,6 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ComputePrecisionForSum; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.literal.BigIntLiteral; @@ -39,8 +38,8 @@ import java.util.List; /** MultiDistinctSum0 */ -public class MultiDistinctSum0 extends AggregateFunction implements UnaryExpression, - ExplicitlyCastableSignature, ComputePrecisionForSum, MultiDistinction, AlwaysNotNullable { +public class MultiDistinctSum0 extends NotNullableAggregateFunction implements UnaryExpression, + ExplicitlyCastableSignature, ComputePrecisionForSum, MultiDistinction { private final boolean mustUseMultiDistinctAgg; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Ndv.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Ndv.java index ea90bc587911be..c2a04b4fd9931b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Ndv.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Ndv.java @@ -21,7 +21,6 @@ import org.apache.doris.catalog.Type; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.functions.Function; import org.apache.doris.nereids.trees.expressions.literal.BigIntLiteral; @@ -38,8 +37,8 @@ /** * AggregateFunction 'ndv'. This class is generated by GenserateFunction. */ -public class Ndv extends AggregateFunction - implements UnaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable, RollUpTrait { +public class Ndv extends NotNullableAggregateFunction + implements UnaryExpression, ExplicitlyCastableSignature, RollUpTrait { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(BigIntType.INSTANCE).args(AnyDataType.INSTANCE_WITHOUT_INDEX) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/NotNullableAggregateFunction.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/NotNullableAggregateFunction.java new file mode 100644 index 00000000000000..f38decf439c2c6 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/NotNullableAggregateFunction.java @@ -0,0 +1,48 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.expressions.functions.agg; + +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; + +import java.util.List; + +/** + * base class for AlwaysNotNullable aggregate function + */ +public abstract class NotNullableAggregateFunction extends AggregateFunction implements AlwaysNotNullable { + protected NotNullableAggregateFunction(String name, Expression ...expressions) { + super(name, false, expressions); + } + + protected NotNullableAggregateFunction(String name, List expressions) { + super(name, false, expressions); + } + + protected NotNullableAggregateFunction(String name, boolean distinct, Expression ...expressions) { + super(name, distinct, expressions); + } + + protected NotNullableAggregateFunction(String name, boolean distinct, List expressions) { + super(name, distinct, expressions); + } + + // return value of this function if the input data is empty. + // for example, count(*) of empty table is 0; + public abstract Expression resultForEmptyInput(); +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/OrthogonalBitmapIntersect.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/OrthogonalBitmapIntersect.java index 956f585f0a1e4c..7837ea1bec7569 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/OrthogonalBitmapIntersect.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/OrthogonalBitmapIntersect.java @@ -19,8 +19,8 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.BitmapIntersectFunction; +import org.apache.doris.nereids.trees.expressions.functions.scalar.BitmapEmpty; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; import org.apache.doris.nereids.types.BitmapType; import org.apache.doris.nereids.util.ExpressionUtils; @@ -33,8 +33,8 @@ /** * AggregateFunction 'orthogonal_bitmap_intersect'. This class is generated by GenerateFunction. */ -public class OrthogonalBitmapIntersect extends AggregateFunction - implements AlwaysNotNullable, OrthogonalBitmapFunction, BitmapIntersectFunction { +public class OrthogonalBitmapIntersect extends NotNullableAggregateFunction + implements OrthogonalBitmapFunction, BitmapIntersectFunction { static final List FUNCTION_SIGNATURES = SUPPORTED_TYPES.stream() .map(type -> FunctionSignature.ret(BitmapType.INSTANCE).varArgs(BitmapType.INSTANCE, type, type)) @@ -75,4 +75,9 @@ public OrthogonalBitmapIntersect withDistinctAndChildren(boolean distinct, List< public R accept(ExpressionVisitor visitor, C context) { return visitor.visitOrthogonalBitmapIntersect(this, context); } + + @Override + public Expression resultForEmptyInput() { + return new BitmapEmpty(); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/OrthogonalBitmapIntersectCount.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/OrthogonalBitmapIntersectCount.java index 82bfa911c26faf..fd282341249e0a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/OrthogonalBitmapIntersectCount.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/OrthogonalBitmapIntersectCount.java @@ -18,8 +18,8 @@ package org.apache.doris.nereids.trees.expressions.functions.agg; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.BitmapIntersectFunction; +import org.apache.doris.nereids.trees.expressions.literal.BigIntLiteral; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; import org.apache.doris.nereids.util.ExpressionUtils; @@ -30,8 +30,8 @@ /** * AggregateFunction 'orthogonal_bitmap_intersect_count'. This class is generated by GenerateFunction. */ -public class OrthogonalBitmapIntersectCount extends AggregateFunction - implements AlwaysNotNullable, OrthogonalBitmapFunction, BitmapIntersectFunction { +public class OrthogonalBitmapIntersectCount extends NotNullableAggregateFunction + implements OrthogonalBitmapFunction, BitmapIntersectFunction { /** * constructor with 3 or more arguments. @@ -44,7 +44,7 @@ public OrthogonalBitmapIntersectCount(Expression arg0, Expression arg1, Expressi * constructor with 3 or more arguments. */ public OrthogonalBitmapIntersectCount(boolean distinct, Expression arg0, Expression arg1, - Expression arg2, Expression... varArgs) { + Expression arg2, Expression... varArgs) { super("orthogonal_bitmap_intersect_count", distinct, ExpressionUtils.mergeArguments(arg0, arg1, arg2, varArgs)); } @@ -63,4 +63,9 @@ public OrthogonalBitmapIntersectCount withDistinctAndChildren(boolean distinct, public R accept(ExpressionVisitor visitor, C context) { return visitor.visitOrthogonalBitmapIntersectCount(this, context); } + + @Override + public Expression resultForEmptyInput() { + return new BigIntLiteral(0); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/OrthogonalBitmapUnionCount.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/OrthogonalBitmapUnionCount.java index c08bb08aeecf64..081e5d7f6c4250 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/OrthogonalBitmapUnionCount.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/OrthogonalBitmapUnionCount.java @@ -19,8 +19,8 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; +import org.apache.doris.nereids.trees.expressions.literal.BigIntLiteral; import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; import org.apache.doris.nereids.types.BigIntType; @@ -34,8 +34,8 @@ /** * AggregateFunction 'orthogonal_bitmap_union_count'. This class is generated by GenerateFunction. */ -public class OrthogonalBitmapUnionCount extends AggregateFunction - implements UnaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable { +public class OrthogonalBitmapUnionCount extends NotNullableAggregateFunction + implements UnaryExpression, ExplicitlyCastableSignature { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(BigIntType.INSTANCE).args(BitmapType.INSTANCE) @@ -73,4 +73,9 @@ public R accept(ExpressionVisitor visitor, C context) { public List getSignatures() { return SIGNATURES; } + + @Override + public Expression resultForEmptyInput() { + return new BigIntLiteral(0); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/PercentileArray.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/PercentileArray.java index bf189d67e89b0b..1abbe4d5450531 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/PercentileArray.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/PercentileArray.java @@ -19,7 +19,6 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.literal.ArrayLiteral; import org.apache.doris.nereids.trees.expressions.shape.BinaryExpression; @@ -42,8 +41,8 @@ /** * AggregateFunction 'percentile_array'. This class is generated by GenerateFunction. */ -public class PercentileArray extends AggregateFunction - implements BinaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable { +public class PercentileArray extends NotNullableAggregateFunction + implements BinaryExpression, ExplicitlyCastableSignature { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(ArrayType.of(DoubleType.INSTANCE)) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/QuantileUnion.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/QuantileUnion.java index 3d0729775a5cec..3d107ec282b606 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/QuantileUnion.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/QuantileUnion.java @@ -20,7 +20,6 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.functions.scalar.QuantileStateEmpty; import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression; @@ -36,8 +35,8 @@ /** * AggregateFunction 'quantile_union'. This class is generated by GenerateFunction. */ -public class QuantileUnion extends AggregateFunction - implements UnaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable { +public class QuantileUnion extends NotNullableAggregateFunction + implements UnaryExpression, ExplicitlyCastableSignature { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(QuantileStateType.INSTANCE).args(QuantileStateType.INSTANCE) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/SequenceCount.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/SequenceCount.java index 7af112ef8e800f..5ac114f00e83ad 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/SequenceCount.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/SequenceCount.java @@ -19,7 +19,6 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.literal.BigIntLiteral; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; @@ -39,8 +38,8 @@ /** * AggregateFunction 'sequence_count'. This class is generated by GenerateFunction. */ -public class SequenceCount extends AggregateFunction - implements ExplicitlyCastableSignature, AlwaysNotNullable, SequenceFunction { +public class SequenceCount extends NotNullableAggregateFunction + implements ExplicitlyCastableSignature, SequenceFunction { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(BigIntType.INSTANCE) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Sum0.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Sum0.java index fd052a69c0e90e..5a1f0f9fb93d34 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Sum0.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Sum0.java @@ -20,7 +20,6 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ComputePrecisionForSum; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.functions.Function; @@ -53,8 +52,8 @@ * AggregateFunction 'sum0'. sum0 returns the sum of the values which go into it like sum. * It differs in that when no non null values are applied zero is returned instead of null. */ -public class Sum0 extends AggregateFunction - implements UnaryExpression, AlwaysNotNullable, ExplicitlyCastableSignature, ComputePrecisionForSum, +public class Sum0 extends NotNullableAggregateFunction + implements UnaryExpression, ExplicitlyCastableSignature, ComputePrecisionForSum, SupportWindowAnalytic, RollUpTrait { public static final List SIGNATURES = ImmutableList.of( From c0b8478f0a659c0e93dae9b9a86e529085944d99 Mon Sep 17 00:00:00 2001 From: Calvin Kirs Date: Mon, 2 Dec 2024 09:59:52 +0800 Subject: [PATCH 094/399] [Fix](http)Enhanced Security Checks for Audit Log File Names (#44612) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## Purpose: To improve the security of audit log files, a new method checkAuditLogFileName has been added to validate the file name and path to ensure they meet security requirements. This method is designed to prevent invalid file names and path traversal attacks, ensuring that only files within the designated directory can be accessed.↳ ### Changes: #### File Name Validation: A regular expression check has been added to validate the file name: ^[a-zA-Z0-9._-]+$, restricting the file name to letters, numbers, dots, underscores, and hyphens. If the file name contains invalid characters (e.g., spaces, path traversal characters), a SecurityException is thrown with the message “Invalid file name.” Path Validation: The file name is resolved into a normalized path, and it is checked to ensure that it is within the allowed directory. The path is constructed using Paths.get(Config.audit_log_dir).resolve(logFile).normalize(). If the path does not start with the specified audit log directory (Config.audit_log_dir), indicating an attempt to access outside the permitted directory (e.g., a path traversal attack), a SecurityException is thrown with the message “Invalid file path: Access outside of permitted directory.” --- .../doris/httpv2/rest/GetLogFileAction.java | 38 +++++++++++- .../doris/httpv2/GetLogFileActionTest.java | 60 +++++++++++++++++++ 2 files changed, 97 insertions(+), 1 deletion(-) create mode 100644 fe/fe-core/src/test/java/org/apache/doris/httpv2/GetLogFileActionTest.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/GetLogFileAction.java b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/GetLogFileAction.java index 475ee5ace1ea0d..87c4c4cfa90a68 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/GetLogFileAction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/GetLogFileAction.java @@ -32,6 +32,8 @@ import java.io.File; import java.io.IOException; +import java.nio.file.Path; +import java.nio.file.Paths; import java.util.Map; import java.util.Set; import javax.servlet.http.HttpServletRequest; @@ -51,6 +53,23 @@ */ @RestController public class GetLogFileAction extends RestBaseController { + /** + * This method fetches internal logs via HTTP, which is no longer recommended and will + * be deprecated in future versions. + *

+ * Using HTTP to fetch logs introduces serious security and performance issues: + * - **Security Risks**: Log content may expose sensitive information, allowing attackers to exploit the exposed + * HTTP endpoints. + * - **Performance Problems**: Frequent HTTP requests can cause significant system load, affecting the + * responsiveness and stability of the application. + *

+ * It is strongly advised not to use this approach for accessing logs. Any new requirements should be + * handled using more secure, reliable, and efficient methods such as log aggregation tools (e.g., ELK, Splunk) + * or dedicated internal APIs. + *

+ * **Note**: No new HTTP endpoints or types for log access will be accepted. + * Any further attempts to extend this HTTP-based log retrieval method will not be supported. + */ private final Set logFileTypes = Sets.newHashSet("fe.audit.log"); @RequestMapping(path = "/api/get_log_file", method = {RequestMethod.GET, RequestMethod.HEAD}) @@ -79,7 +98,13 @@ public Object execute(HttpServletRequest request, HttpServletResponse response) String fileInfos = getFileInfos(logType); response.setHeader("file_infos", fileInfos); return ResponseEntityBuilder.ok(); - } else if (method.equals(RequestMethod.GET.name())) { + } + if (method.equals(RequestMethod.GET.name())) { + try { + checkAuditLogFileName(logFile); + } catch (SecurityException e) { + return ResponseEntityBuilder.internalError(e.getMessage()); + } File log = getLogFile(logType, logFile); if (!log.exists() || !log.isFile()) { return ResponseEntityBuilder.okWithCommonError("Log file not exist: " + log.getName()); @@ -97,6 +122,17 @@ public Object execute(HttpServletRequest request, HttpServletResponse response) return ResponseEntityBuilder.ok(); } + private void checkAuditLogFileName(String logFile) { + if (!logFile.matches("^[a-zA-Z0-9._-]+$")) { + throw new SecurityException("Invalid file name"); + } + Path normalizedPath = Paths.get(Config.audit_log_dir).resolve(logFile).normalize(); + // check path is valid or not + if (!normalizedPath.startsWith(Config.audit_log_dir)) { + throw new SecurityException("Invalid file path: Access outside of permitted directory"); + } + } + private String getFileInfos(String logType) { Map fileInfos = Maps.newTreeMap(); if (logType.equals("fe.audit.log")) { diff --git a/fe/fe-core/src/test/java/org/apache/doris/httpv2/GetLogFileActionTest.java b/fe/fe-core/src/test/java/org/apache/doris/httpv2/GetLogFileActionTest.java new file mode 100644 index 00000000000000..8d4cac9b6ad9f4 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/httpv2/GetLogFileActionTest.java @@ -0,0 +1,60 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.httpv2; + +import org.apache.doris.common.Config; +import org.apache.doris.httpv2.rest.GetLogFileAction; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import java.io.File; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; + +public class GetLogFileActionTest { + + @TempDir + public File tempDir; + + @BeforeAll + public static void before() { + File tempDir = new File("test/audit.log"); + tempDir.mkdir(); + Config.audit_log_dir = tempDir.getAbsolutePath(); + } + + @Test + public void testCheckAuditLogFileName() throws NoSuchMethodException, InvocationTargetException, IllegalAccessException { + //private method checkAuditLogFileName + GetLogFileAction action = new GetLogFileAction(); + Method method = GetLogFileAction.class.getDeclaredMethod("checkAuditLogFileName", String.class); + method.setAccessible(true); + method.invoke(action, "audit.log"); + method.invoke(action, "fe.audit.log.20241104-1"); + Assertions.assertThrows(InvocationTargetException.class, () -> method.invoke(action, "../etc/passwd")); + Assertions.assertThrows(InvocationTargetException.class, () -> method.invoke(action, + "fe.audit.log.20241104-1/../../etc/passwd")); + Assertions.assertThrows(InvocationTargetException.class, + () -> method.invoke(action, "fe.audit.log.20241104-1; rm -rf /")); + + + } +} From 2d9d3f79df01c475a1a0bbc1b68d2084323f04f5 Mon Sep 17 00:00:00 2001 From: Pxl Date: Mon, 2 Dec 2024 10:26:56 +0800 Subject: [PATCH 095/399] [Bug](runtime-filter) fix publish not inited rf when broadcast join meet wake up by downsteam (#44823) ### What problem does this PR solve? fix publish not inited rf when broadcast join meet wake up by downsteam related with #44408 #41751 --- be/src/exprs/runtime_filter.cpp | 2 +- be/src/pipeline/exec/hashjoin_build_sink.cpp | 38 +++++++++++--------- 2 files changed, 22 insertions(+), 18 deletions(-) diff --git a/be/src/exprs/runtime_filter.cpp b/be/src/exprs/runtime_filter.cpp index e26452c9ef69e5..c983af0fb3ea71 100644 --- a/be/src/exprs/runtime_filter.cpp +++ b/be/src/exprs/runtime_filter.cpp @@ -1542,7 +1542,7 @@ void IRuntimeFilter::update_runtime_filter_type_to_profile(uint64_t local_merge_ std::string IRuntimeFilter::debug_string() const { return fmt::format( - "RuntimeFilter: (id = {}, type = {}, is_broadcast: {}, ignored: {}" + "RuntimeFilter: (id = {}, type = {}, is_broadcast: {}, ignored: {}, " "build_bf_cardinality: {}, dependency: {}, synced_size: {}, has_local_target: {}, " "has_remote_target: {},error_msg: [{}]", _filter_id, to_string(_runtime_filter_type), _is_broadcast_join, diff --git a/be/src/pipeline/exec/hashjoin_build_sink.cpp b/be/src/pipeline/exec/hashjoin_build_sink.cpp index 0a71b86bed0e86..cec0c77da8a61d 100644 --- a/be/src/pipeline/exec/hashjoin_build_sink.cpp +++ b/be/src/pipeline/exec/hashjoin_build_sink.cpp @@ -139,28 +139,32 @@ Status HashJoinBuildSinkLocalState::close(RuntimeState* state, Status exec_statu return Base::close(state, exec_status); } - if (_should_build_hash_table) { - if (state->get_task()->wake_up_by_downstream()) { + if (state->get_task()->wake_up_by_downstream()) { + if (_should_build_hash_table) { + // partitial ignore rf to make global rf work RETURN_IF_ERROR(_runtime_filter_slots->send_filter_size(state, 0, _finish_dependency)); RETURN_IF_ERROR(_runtime_filter_slots->ignore_all_filters()); } else { - if (p._shared_hashtable_controller && - !p._shared_hash_table_context->complete_build_stage) { - return Status::InternalError("close before sink meet eos"); - } - auto* block = _shared_state->build_block.get(); - uint64_t hash_table_size = block ? block->rows() : 0; - { - SCOPED_TIMER(_runtime_filter_init_timer); - RETURN_IF_ERROR(_runtime_filter_slots->init_filters(state, hash_table_size)); - RETURN_IF_ERROR(_runtime_filter_slots->ignore_filters(state)); - } - if (hash_table_size > 1) { - SCOPED_TIMER(_runtime_filter_compute_timer); - _runtime_filter_slots->insert(block); - } + // do not publish filter coz local rf not inited and useless + return Base::close(state, exec_status); + } + } else if (_should_build_hash_table) { + if (p._shared_hashtable_controller && !p._shared_hash_table_context->complete_build_stage) { + return Status::InternalError("close before sink meet eos"); + } + auto* block = _shared_state->build_block.get(); + uint64_t hash_table_size = block ? block->rows() : 0; + { + SCOPED_TIMER(_runtime_filter_init_timer); + RETURN_IF_ERROR(_runtime_filter_slots->init_filters(state, hash_table_size)); + RETURN_IF_ERROR(_runtime_filter_slots->ignore_filters(state)); + } + if (hash_table_size > 1) { + SCOPED_TIMER(_runtime_filter_compute_timer); + _runtime_filter_slots->insert(block); } } + SCOPED_TIMER(_publish_runtime_filter_timer); RETURN_IF_ERROR_OR_CATCH_EXCEPTION( _runtime_filter_slots->publish(state, !_should_build_hash_table)); From 3ce581ec0cdad4f2e953b5fd554917e6c3761d7c Mon Sep 17 00:00:00 2001 From: Gabriel Date: Mon, 2 Dec 2024 10:33:31 +0800 Subject: [PATCH 096/399] [minor](scan) Delete unused function in ScanNode (#44733) --- .../src/main/java/org/apache/doris/planner/ScanNode.java | 9 --------- 1 file changed, 9 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/ScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/ScanNode.java index b4033a0535ef3d..2b10520e67d317 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/ScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/ScanNode.java @@ -727,15 +727,6 @@ public static TScanRangeLocations createSingleScanRangeLocations(FederationBacke return scanRangeLocation; } - public boolean ignoreStorageDataDistribution(ConnectContext context, int numBackends) { - return context != null - && context.getSessionVariable().isIgnoreStorageDataDistribution() - && !fragment.hasNullAwareLeftAntiJoin() - && getScanRangeNum() - < ConnectContext.get().getSessionVariable().getParallelExecInstanceNum() - * (numScanBackends() > 0 ? numScanBackends() : numBackends); - } - public int numScanBackends() { return scanBackendIds.size(); } From 59ae8003866a752044b8e450760f2b3077d08e58 Mon Sep 17 00:00:00 2001 From: walter Date: Mon, 2 Dec 2024 10:43:18 +0800 Subject: [PATCH 097/399] [chore](binlog) Save table type in CreateTableRecord (#44735) --- .../main/java/org/apache/doris/binlog/CreateTableRecord.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/fe/fe-core/src/main/java/org/apache/doris/binlog/CreateTableRecord.java b/fe/fe-core/src/main/java/org/apache/doris/binlog/CreateTableRecord.java index 89d11fc014e979..602afe88081d5b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/binlog/CreateTableRecord.java +++ b/fe/fe-core/src/main/java/org/apache/doris/binlog/CreateTableRecord.java @@ -20,6 +20,7 @@ import org.apache.doris.catalog.Database; import org.apache.doris.catalog.Env; import org.apache.doris.catalog.Table; +import org.apache.doris.catalog.TableIf.TableType; import org.apache.doris.persist.CreateTableInfo; import org.apache.doris.persist.gson.GsonUtils; @@ -43,6 +44,8 @@ public class CreateTableRecord { private long tableId; @SerializedName(value = "tableName") private String tableName; + @SerializedName(value = "tableType") + protected TableType type; @SerializedName(value = "sql") private String sql; @@ -56,6 +59,8 @@ public CreateTableRecord(long commitSeq, CreateTableInfo info) { String dbName = info.getDbName(); this.dbName = dbName; + this.type = table.getType(); + Database db = Env.getCurrentInternalCatalog().getDbNullable(dbName); if (db == null) { LOG.warn("db not found. dbId: {}", dbId); From 4bd1cc1a01b82ea7e74d99b94c5369ae77d08336 Mon Sep 17 00:00:00 2001 From: Sridhar R Manikarnike Date: Mon, 2 Dec 2024 08:26:11 +0530 Subject: [PATCH 098/399] [Enhancement] (nereids)implement showCreateDatabaseCommand in nereids (#43034) Issue Number: close #42739 --- .../org/apache/doris/nereids/DorisParser.g4 | 2 +- .../nereids/parser/LogicalPlanBuilder.java | 20 +++ .../doris/nereids/trees/plans/PlanType.java | 1 + .../commands/ShowCreateDatabaseCommand.java | 122 ++++++++++++++++++ .../trees/plans/visitor/CommandVisitor.java | 5 + .../show/test_show_create_db_nereids.out | 7 + .../show/test_show_create_db_nereids.groovy | 42 ++++++ 7 files changed, 198 insertions(+), 1 deletion(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowCreateDatabaseCommand.java create mode 100644 regression-test/data/nereids_p0/show/test_show_create_db_nereids.out create mode 100644 regression-test/suites/nereids_p0/show/test_show_create_db_nereids.groovy diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index 1b1e218a958146..f78a1c4b53179a 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -214,6 +214,7 @@ supportedDropStatement supportedShowStatement : SHOW (GLOBAL | SESSION | LOCAL)? VARIABLES wildWhere? #showVariables | SHOW AUTHORS #showAuthors + | SHOW CREATE (DATABASE | SCHEMA) name=multipartIdentifier #showCreateDatabase | SHOW BROKER #showBroker | SHOW DYNAMIC PARTITION TABLES ((FROM | IN) database=multipartIdentifier)? #showDynamicPartition | SHOW EVENTS ((FROM | IN) database=multipartIdentifier)? wildWhere? #showEvents @@ -295,7 +296,6 @@ unsupportedShowStatement | SHOW FULL? PROCESSLIST #showProcessList | SHOW (GLOBAL | SESSION | LOCAL)? STATUS wildWhere? #showStatus | SHOW CREATE MATERIALIZED VIEW name=multipartIdentifier #showMaterializedView - | SHOW CREATE (DATABASE | SCHEMA) name=multipartIdentifier #showCreateDatabase | SHOW CREATE (GLOBAL | SESSION | LOCAL)? FUNCTION functionIdentifier LEFT_PAREN functionArguments? RIGHT_PAREN ((FROM | IN) database=multipartIdentifier)? #showCreateFunction diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index e5f4322225de53..d52d2696a2951c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -20,6 +20,7 @@ import org.apache.doris.analysis.ArithmeticExpr.Operator; import org.apache.doris.analysis.BrokerDesc; import org.apache.doris.analysis.ColumnNullableType; +import org.apache.doris.analysis.DbName; import org.apache.doris.analysis.EncryptKeyName; import org.apache.doris.analysis.PassVar; import org.apache.doris.analysis.SetType; @@ -226,6 +227,7 @@ import org.apache.doris.nereids.DorisParser.ShowConfigContext; import org.apache.doris.nereids.DorisParser.ShowConstraintContext; import org.apache.doris.nereids.DorisParser.ShowCreateCatalogContext; +import org.apache.doris.nereids.DorisParser.ShowCreateDatabaseContext; import org.apache.doris.nereids.DorisParser.ShowCreateMTMVContext; import org.apache.doris.nereids.DorisParser.ShowCreateMaterializedViewContext; import org.apache.doris.nereids.DorisParser.ShowCreateProcedureContext; @@ -512,6 +514,7 @@ import org.apache.doris.nereids.trees.plans.commands.ShowConfigCommand; import org.apache.doris.nereids.trees.plans.commands.ShowConstraintsCommand; import org.apache.doris.nereids.trees.plans.commands.ShowCreateCatalogCommand; +import org.apache.doris.nereids.trees.plans.commands.ShowCreateDatabaseCommand; import org.apache.doris.nereids.trees.plans.commands.ShowCreateMTMVCommand; import org.apache.doris.nereids.trees.plans.commands.ShowCreateMaterializedViewCommand; import org.apache.doris.nereids.trees.plans.commands.ShowCreateProcedureCommand; @@ -4617,6 +4620,23 @@ public LogicalPlan visitShowFrontends(ShowFrontendsContext ctx) { return new ShowFrontendsCommand(detail); } + @Override + public LogicalPlan visitShowCreateDatabase(ShowCreateDatabaseContext ctx) { + List nameParts = visitMultipartIdentifier(ctx.name); + String databaseName = ""; + String catalogName = ""; + if (nameParts.size() == 2) { + // The identifier is in the form "internalcatalog.databasename" + catalogName = nameParts.get(0); + databaseName = nameParts.get(1); + } else if (nameParts.size() == 1) { + // The identifier is in the form "databasename" + databaseName = nameParts.get(0); + } + + return new ShowCreateDatabaseCommand(new DbName(catalogName, databaseName)); + } + @Override public LogicalPlan visitCleanAllProfile(CleanAllProfileContext ctx) { return new CleanAllProfileCommand(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index 8f474ea395f889..8625d103f3b453 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -198,6 +198,7 @@ public enum PlanType { SHOW_BROKER_COMMAND, SHOW_CONFIG_COMMAND, SHOW_CREATE_CATALOG_COMMAND, + SHOW_CREATE_DATABASE_COMMAND, SHOW_CREATE_MATERIALIZED_VIEW_COMMAND, SHOW_CREATE_TABLE_COMMAND, SHOW_CREATE_VIEW_COMMAND, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowCreateDatabaseCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowCreateDatabaseCommand.java new file mode 100644 index 00000000000000..29186e9e5cddee --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowCreateDatabaseCommand.java @@ -0,0 +1,122 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.commands; + +import org.apache.doris.analysis.DbName; +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.DatabaseIf; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.ScalarType; +import org.apache.doris.cluster.ClusterNamespace; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.common.util.PrintableMap; +import org.apache.doris.datasource.CatalogIf; +import org.apache.doris.datasource.hive.HMSExternalCatalog; +import org.apache.doris.datasource.iceberg.IcebergExternalCatalog; +import org.apache.doris.datasource.iceberg.IcebergExternalDatabase; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.ShowResultSet; +import org.apache.doris.qe.ShowResultSetMetaData; +import org.apache.doris.qe.StmtExecutor; + +import com.google.common.base.Strings; +import com.google.common.collect.Lists; + +import java.util.List; +import java.util.Objects; + +/** + * Represents the command for SHOW CREATE DATABASE. + */ +public class ShowCreateDatabaseCommand extends ShowCommand { + private static final ShowResultSetMetaData META_DATA = + ShowResultSetMetaData.builder() + .addColumn(new Column("Database", ScalarType.createVarchar(20))) + .addColumn(new Column("Create Database", ScalarType.createVarchar(30))) + .build(); + + private final String databaseName; + private final String catalogName; + + public ShowCreateDatabaseCommand(DbName dbName) { + super(PlanType.SHOW_CREATE_DATABASE_COMMAND); + this.databaseName = Objects.requireNonNull(dbName.getDb(), "Database name cannot be null"); + this.catalogName = dbName.getCtl(); + } + + @Override + public ShowResultSet doRun(ConnectContext ctx, StmtExecutor executor) throws Exception { + String ctlgName = catalogName; + if (Strings.isNullOrEmpty(catalogName)) { + ctlgName = Env.getCurrentEnv().getCurrentCatalog().getName(); + } + if (Strings.isNullOrEmpty(databaseName)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_WRONG_DB_NAME, databaseName); + } + + if (!Env.getCurrentEnv().getAccessManager().checkDbPriv(ConnectContext.get(), ctlgName, databaseName, + PrivPredicate.SHOW)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_DB_ACCESS_DENIED_ERROR, + PrivPredicate.SHOW.getPrivs().toString(), databaseName); + } + + List> rows = Lists.newArrayList(); + + StringBuilder sb = new StringBuilder(); + CatalogIf catalog = Env.getCurrentEnv().getCatalogMgr().getCatalogOrAnalysisException(ctlgName); + if (catalog instanceof HMSExternalCatalog) { + String simpleDBName = ClusterNamespace.getNameFromFullName(databaseName); + org.apache.hadoop.hive.metastore.api.Database db = ((HMSExternalCatalog) catalog).getClient() + .getDatabase(simpleDBName); + sb.append("CREATE DATABASE `").append(simpleDBName).append("`") + .append(" LOCATION '") + .append(db.getLocationUri()) + .append("'"); + } else if (catalog instanceof IcebergExternalCatalog) { + IcebergExternalDatabase db = (IcebergExternalDatabase) catalog.getDbOrAnalysisException(databaseName); + sb.append("CREATE DATABASE `").append(databaseName).append("`") + .append(" LOCATION '") + .append(db.getLocation()) + .append("'"); + } else { + DatabaseIf db = catalog.getDbOrAnalysisException(databaseName); + sb.append("CREATE DATABASE `").append(ClusterNamespace.getNameFromFullName(databaseName)).append("`"); + if (db.getDbProperties().getProperties().size() > 0) { + sb.append("\nPROPERTIES (\n"); + sb.append(new PrintableMap<>(db.getDbProperties().getProperties(), "=", true, true, false)); + sb.append("\n)"); + } + } + + rows.add(Lists.newArrayList(ClusterNamespace.getNameFromFullName(databaseName), sb.toString())); + return new ShowResultSet(this.getMetaData(), rows); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitShowCreateDatabaseCommand(this, context); + } + + public ShowResultSetMetaData getMetaData() { + return META_DATA; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index c00c0ef90ca018..49485cda51d0c2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -75,6 +75,7 @@ import org.apache.doris.nereids.trees.plans.commands.ShowConfigCommand; import org.apache.doris.nereids.trees.plans.commands.ShowConstraintsCommand; import org.apache.doris.nereids.trees.plans.commands.ShowCreateCatalogCommand; +import org.apache.doris.nereids.trees.plans.commands.ShowCreateDatabaseCommand; import org.apache.doris.nereids.trees.plans.commands.ShowCreateMTMVCommand; import org.apache.doris.nereids.trees.plans.commands.ShowCreateMaterializedViewCommand; import org.apache.doris.nereids.trees.plans.commands.ShowCreateProcedureCommand; @@ -398,6 +399,10 @@ default R visitShowCreateMaterializedViewCommand(ShowCreateMaterializedViewComma return visitCommand(showCreateMtlzViewCommand, context); } + default R visitShowCreateDatabaseCommand(ShowCreateDatabaseCommand showCreateDatabaseCommand, C context) { + return visitCommand(showCreateDatabaseCommand, context); + } + default R visitShowCreateViewCommand(ShowCreateViewCommand showCreateViewCommand, C context) { return visitCommand(showCreateViewCommand, context); } diff --git a/regression-test/data/nereids_p0/show/test_show_create_db_nereids.out b/regression-test/data/nereids_p0/show/test_show_create_db_nereids.out new file mode 100644 index 00000000000000..df88b46f8fbe77 --- /dev/null +++ b/regression-test/data/nereids_p0/show/test_show_create_db_nereids.out @@ -0,0 +1,7 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !cmd -- +db_test_show_create CREATE DATABASE `db_test_show_create` + +-- !cmd -- +db_test_show_create CREATE DATABASE `db_test_show_create`\nPROPERTIES (\n"property_key" = "property_value"\n) + diff --git a/regression-test/suites/nereids_p0/show/test_show_create_db_nereids.groovy b/regression-test/suites/nereids_p0/show/test_show_create_db_nereids.groovy new file mode 100644 index 00000000000000..bfc84c9a60f72b --- /dev/null +++ b/regression-test/suites/nereids_p0/show/test_show_create_db_nereids.groovy @@ -0,0 +1,42 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("test_show_create_db_nereids", "query,create_database") { + String dbName = "db_test_show_create"; + + try { + // Create a new database to test the SHOW CREATE DATABASE command + sql "CREATE DATABASE IF NOT EXISTS ${dbName}" + + // Run the SHOW CREATE DATABASE command and validate the output using checkNereidsExecute and qt_cmd + checkNereidsExecute("""SHOW CREATE DATABASE ${dbName}""") + qt_cmd("""SHOW CREATE DATABASE ${dbName}""") + + // Drop the database and verify that the command runs successfully + sql "DROP DATABASE IF EXISTS ${dbName}" + + // Re-create the database with additional properties + sql "CREATE DATABASE IF NOT EXISTS ${dbName} PROPERTIES ('property_key'='property_value')" + + // Verify the SHOW CREATE DATABASE command captures the properties using checkNereidsExecute and qt_cmd + checkNereidsExecute("""SHOW CREATE DATABASE ${dbName}""") + qt_cmd("""SHOW CREATE DATABASE ${dbName}""") + } finally { + // Clean up by dropping the database if it still exists + try_sql("DROP DATABASE IF EXISTS ${dbName}") + } +} From 70415afa864af41d5028605b897ae2ff67a62c66 Mon Sep 17 00:00:00 2001 From: echo-dundun <50286010+echo-hhj@users.noreply.github.com> Date: Mon, 2 Dec 2024 10:57:27 +0800 Subject: [PATCH 099/399] [Feat] (Nereids)support showTrash Command (#44684) Issue Number: close #42763 --- .../org/apache/doris/nereids/DorisParser.g4 | 2 +- .../nereids/parser/LogicalPlanBuilder.java | 13 +++ .../doris/nereids/trees/plans/PlanType.java | 1 + .../plans/commands/ShowTrashCommand.java | 109 ++++++++++++++++++ .../trees/plans/visitor/CommandVisitor.java | 5 + .../ddl/show_trash/test_nereids_trash.groovy | 22 ++++ 6 files changed, 151 insertions(+), 1 deletion(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowTrashCommand.java create mode 100644 regression-test/suites/nereids_p0/ddl/show_trash/test_nereids_trash.groovy diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index f78a1c4b53179a..e5c9a997073df8 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -246,6 +246,7 @@ supportedShowStatement | SHOW TABLET DIAGNOSIS tabletId=INTEGER_VALUE #showDiagnoseTablet | SHOW FRONTENDS name=identifier? #showFrontends | SHOW TABLE tableId=INTEGER_VALUE #showTableId + | SHOW TRASH (ON backend=STRING_LITERAL)? #showTrash | SHOW WHITELIST #showWhitelist | SHOW TABLETS BELONG tabletIds+=INTEGER_VALUE (COMMA tabletIds+=INTEGER_VALUE)* #showTabletsBelong @@ -332,7 +333,6 @@ unsupportedShowStatement | SHOW BRIEF? RESTORE ((FROM | IN) database=multipartIdentifier)? wildWhere? #showRestore | SHOW RESOURCES wildWhere? sortClause? limitClause? #showResources | SHOW WORKLOAD GROUPS wildWhere? #showWorkloadGroups - | SHOW TRASH (ON backend=STRING_LITERAL)? #showTrash | SHOW SNAPSHOT ON repo=identifier wildWhere? #showSnapshot | SHOW FULL? BUILTIN? FUNCTIONS ((FROM | IN) database=multipartIdentifier)? wildWhere? #showFunctions diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index d52d2696a2951c..5ab217a1d5eb2c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -255,6 +255,7 @@ import org.apache.doris.nereids.DorisParser.ShowStorageEnginesContext; import org.apache.doris.nereids.DorisParser.ShowTableIdContext; import org.apache.doris.nereids.DorisParser.ShowTabletsBelongContext; +import org.apache.doris.nereids.DorisParser.ShowTrashContext; import org.apache.doris.nereids.DorisParser.ShowTriggersContext; import org.apache.doris.nereids.DorisParser.ShowVariablesContext; import org.apache.doris.nereids.DorisParser.ShowViewContext; @@ -541,6 +542,7 @@ import org.apache.doris.nereids.trees.plans.commands.ShowStorageEnginesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowTableIdCommand; import org.apache.doris.nereids.trees.plans.commands.ShowTabletsBelongCommand; +import org.apache.doris.nereids.trees.plans.commands.ShowTrashCommand; import org.apache.doris.nereids.trees.plans.commands.ShowTriggersCommand; import org.apache.doris.nereids.trees.plans.commands.ShowVariablesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowViewCommand; @@ -4462,6 +4464,17 @@ public LogicalPlan visitShowTriggers(ShowTriggersContext ctx) { return new ShowTriggersCommand(); } + @Override + public LogicalPlan visitShowTrash(ShowTrashContext ctx) { + if (ctx.ON() != null) { + String backend = stripQuotes(ctx.STRING_LITERAL().getText()); + new ShowTrashCommand(backend); + } else { + return new ShowTrashCommand(); + } + return new ShowTrashCommand(); + } + @Override public LogicalPlan visitShowRepositories(ShowRepositoriesContext ctx) { return new ShowRepositoriesCommand(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index 8625d103f3b453..683af9fe2c4700 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -220,6 +220,7 @@ public enum PlanType { SHOW_SMALL_FILES_COMMAND, SHOW_STORAGE_ENGINES_COMMAND, SHOW_TABLE_ID_COMMAND, + SHOW_TRASH_COMMAND, SHOW_TRIGGERS_COMMAND, SHOW_VARIABLES_COMMAND, SHOW_AUTHORS_COMMAND, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowTrashCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowTrashCommand.java new file mode 100644 index 00000000000000..bcf30490cc991a --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowTrashCommand.java @@ -0,0 +1,109 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.commands; + +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.ScalarType; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.common.proc.TrashProcDir; +import org.apache.doris.common.util.NetUtils; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.ShowResultSet; +import org.apache.doris.qe.ShowResultSetMetaData; +import org.apache.doris.qe.StmtExecutor; +import org.apache.doris.system.Backend; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; + +import java.util.List; + +/** + * show trash command + */ +public class ShowTrashCommand extends ShowCommand { + private List backends = Lists.newArrayList(); + private String backendQuery; + + public ShowTrashCommand() { + super(PlanType.SHOW_TRASH_COMMAND); + } + + public ShowTrashCommand(String backendQuery) { + super(PlanType.SHOW_TRASH_COMMAND); + this.backendQuery = backendQuery; + } + + public List getBackends() { + return backends; + } + + public String getBackend() { + return backendQuery; + } + + public ShowResultSetMetaData getMetaData() { + ShowResultSetMetaData.Builder builder = ShowResultSetMetaData.builder(); + for (String title : TrashProcDir.TITLE_NAMES) { + builder.addColumn(new Column(title, ScalarType.createVarchar(30))); + } + return builder.build(); + } + + private ShowResultSet handleShowTrash(String backendQuery) throws Exception { + if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN) + && !Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), + PrivPredicate.OPERATOR)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, "ADMIN/OPERATOR"); + } + ImmutableMap backendsInfo = Env.getCurrentSystemInfo().getAllBackendsByAllCluster(); + if (backendQuery == null || backendQuery.isEmpty()) { + for (Backend backend : backendsInfo.values()) { + this.backends.add(backend); + } + } else { + for (Backend backend : backendsInfo.values()) { + String backendStr = NetUtils.getHostPortInAccessibleFormat(backend.getHost(), + backend.getHeartbeatPort()); + if (backendQuery.equals(backendStr)) { + this.backends.add(backend); + break; + } + } + } + List> infos = Lists.newArrayList(); + TrashProcDir.getTrashInfo(backends, infos); + return new ShowResultSet(getMetaData(), infos); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitShowTrashCommand(this, context); + } + + @Override + public ShowResultSet doRun(ConnectContext ctx, StmtExecutor executor) throws Exception { + return handleShowTrash(backendQuery); + } +} + diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index 49485cda51d0c2..98050c598e1c55 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -102,6 +102,7 @@ import org.apache.doris.nereids.trees.plans.commands.ShowStorageEnginesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowTableIdCommand; import org.apache.doris.nereids.trees.plans.commands.ShowTabletsBelongCommand; +import org.apache.doris.nereids.trees.plans.commands.ShowTrashCommand; import org.apache.doris.nereids.trees.plans.commands.ShowTriggersCommand; import org.apache.doris.nereids.trees.plans.commands.ShowVariablesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowViewCommand; @@ -370,6 +371,10 @@ default R visitShowPluginsCommand(ShowPluginsCommand showPluginsCommand, C conte return visitCommand(showPluginsCommand, context); } + default R visitShowTrashCommand(ShowTrashCommand showTrashCommand, C context) { + return visitCommand(showTrashCommand, context); + } + default R visitShowTriggersCommand(ShowTriggersCommand showTriggersCommand, C context) { return visitCommand(showTriggersCommand, context); } diff --git a/regression-test/suites/nereids_p0/ddl/show_trash/test_nereids_trash.groovy b/regression-test/suites/nereids_p0/ddl/show_trash/test_nereids_trash.groovy new file mode 100644 index 00000000000000..5d795e6fd119a1 --- /dev/null +++ b/regression-test/suites/nereids_p0/ddl/show_trash/test_nereids_trash.groovy @@ -0,0 +1,22 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("show_trash_nereids") { + // can not use qt command since the output change based on cluster and backend ip + checkNereidsExecute("""show trash;""") + checkNereidsExecute("""show trash on "127.0.0.1:9050";""") +} From b8555c4dfa9fbcf122c3df077d919991388dcb3b Mon Sep 17 00:00:00 2001 From: Sridhar R Manikarnike Date: Mon, 2 Dec 2024 08:27:58 +0530 Subject: [PATCH 100/399] [Enhancement] (nereids)implement showCollationCommand in nereids (#43157) Issue Number: close #42748 --- .../org/apache/doris/nereids/DorisParser.g4 | 2 +- .../nereids/parser/LogicalPlanBuilder.java | 15 ++++ .../doris/nereids/trees/plans/PlanType.java | 1 + .../plans/commands/ShowCollationCommand.java | 89 +++++++++++++++++++ .../trees/plans/visitor/CommandVisitor.java | 5 ++ .../nereids_p0/show/test_show_collation.out | 5 ++ .../show/test_show_collation.groovy | 29 ++++++ 7 files changed, 145 insertions(+), 1 deletion(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowCollationCommand.java create mode 100644 regression-test/data/nereids_p0/show/test_show_collation.out create mode 100644 regression-test/suites/nereids_p0/show/test_show_collation.groovy diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index e5c9a997073df8..8edcbb40c3c13b 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -236,6 +236,7 @@ supportedShowStatement | SHOW FILE ((FROM | IN) database=multipartIdentifier)? #showSmallFiles | SHOW STORAGE? ENGINES #showStorageEngines | SHOW CREATE CATALOG name=identifier #showCreateCatalog + | SHOW COLLATION wildWhere? #showCollation | SHOW SQL_BLOCK_RULE (FOR ruleName=identifier)? #showSqlBlockRule | SHOW CREATE VIEW name=multipartIdentifier #showCreateView | SHOW CREATE MATERIALIZED VIEW mvName=identifier @@ -307,7 +308,6 @@ unsupportedShowStatement | SHOW CATALOG name=identifier #showCatalog | SHOW FULL? (COLUMNS | FIELDS) (FROM | IN) tableName=multipartIdentifier ((FROM | IN) database=multipartIdentifier)? wildWhere? #showColumns - | SHOW COLLATION wildWhere? #showCollation | SHOW ((CHAR SET) | CHARSET) wildWhere? #showCharset | SHOW COUNT LEFT_PAREN ASTERISK RIGHT_PAREN (WARNINGS | ERRORS) #showWaringErrorCount | SHOW (WARNINGS | ERRORS) limitClause? #showWaringErrors diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index 5ab217a1d5eb2c..60eda00ce89620 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -224,6 +224,7 @@ import org.apache.doris.nereids.DorisParser.ShowAuthorsContext; import org.apache.doris.nereids.DorisParser.ShowBackendsContext; import org.apache.doris.nereids.DorisParser.ShowBrokerContext; +import org.apache.doris.nereids.DorisParser.ShowCollationContext; import org.apache.doris.nereids.DorisParser.ShowConfigContext; import org.apache.doris.nereids.DorisParser.ShowConstraintContext; import org.apache.doris.nereids.DorisParser.ShowCreateCatalogContext; @@ -512,6 +513,7 @@ import org.apache.doris.nereids.trees.plans.commands.ShowAuthorsCommand; import org.apache.doris.nereids.trees.plans.commands.ShowBackendsCommand; import org.apache.doris.nereids.trees.plans.commands.ShowBrokerCommand; +import org.apache.doris.nereids.trees.plans.commands.ShowCollationCommand; import org.apache.doris.nereids.trees.plans.commands.ShowConfigCommand; import org.apache.doris.nereids.trees.plans.commands.ShowConstraintsCommand; import org.apache.doris.nereids.trees.plans.commands.ShowCreateCatalogCommand; @@ -4782,4 +4784,17 @@ public LogicalPlan visitShowTabletsBelong(ShowTabletsBelongContext ctx) { }); return new ShowTabletsBelongCommand(tabletIdLists); } + + @Override + public LogicalPlan visitShowCollation(ShowCollationContext ctx) { + String wild = null; + if (ctx.wildWhere() != null) { + if (ctx.wildWhere().LIKE() != null) { + wild = stripQuotes(ctx.wildWhere().STRING_LITERAL().getText()); + } else if (ctx.wildWhere().WHERE() != null) { + wild = ctx.wildWhere().expression().getText(); + } + } + return new ShowCollationCommand(wild); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index 683af9fe2c4700..d07c90b7ccd5e7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -196,6 +196,7 @@ public enum PlanType { SHOW_BACKENDS_COMMAND, SHOW_BLOCK_RULE_COMMAND, SHOW_BROKER_COMMAND, + SHOW_COLLATION_COMMAND, SHOW_CONFIG_COMMAND, SHOW_CREATE_CATALOG_COMMAND, SHOW_CREATE_DATABASE_COMMAND, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowCollationCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowCollationCommand.java new file mode 100644 index 00000000000000..945db68bf5901a --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowCollationCommand.java @@ -0,0 +1,89 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.commands; + +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.ScalarType; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.ShowResultSet; +import org.apache.doris.qe.ShowResultSetMetaData; +import org.apache.doris.qe.StmtExecutor; + +import com.google.common.collect.Lists; + +import java.util.List; + +/** + * Represents the command for SHOW COLLATION + */ +public class ShowCollationCommand extends ShowCommand { + private static final ShowResultSetMetaData COLLATION_META_DATA = + ShowResultSetMetaData.builder() + .addColumn(new Column("Collation", ScalarType.createVarchar(20))) + .addColumn(new Column("Charset", ScalarType.createVarchar(20))) + .addColumn(new Column("Id", ScalarType.createVarchar(10))) + .addColumn(new Column("Default", ScalarType.createVarchar(10))) + .addColumn(new Column("Compiled", ScalarType.createVarchar(10))) + .addColumn(new Column("Sortlen", ScalarType.createVarchar(10))) + .build(); + + private final String wild; + + public ShowCollationCommand(String wild) { + super(PlanType.SHOW_COLLATION_COMMAND); + this.wild = wild; + } + + @Override + public ShowResultSet doRun(ConnectContext ctx, StmtExecutor executor) throws Exception { + List> rows = Lists.newArrayList(); + List utf8mb40900Bin = Lists.newArrayList(); + // | utf8mb4_0900_bin | utf8mb4 | 309 | Yes | Yes | 1 | + utf8mb40900Bin.add(ctx.getSessionVariable().getCollationConnection()); + utf8mb40900Bin.add(ctx.getSessionVariable().getCharsetServer()); + utf8mb40900Bin.add("309"); + utf8mb40900Bin.add("Yes"); + utf8mb40900Bin.add("Yes"); + utf8mb40900Bin.add("1"); + rows.add(utf8mb40900Bin); + // ATTN: we must have this collation for compatible with some bi tools + List utf8mb3GeneralCi = Lists.newArrayList(); + // | utf8mb3_general_ci | utf8mb3 | 33 | Yes | Yes | 1 | + utf8mb3GeneralCi.add("utf8mb3_general_ci"); + utf8mb3GeneralCi.add("utf8mb3"); + utf8mb3GeneralCi.add("33"); + utf8mb3GeneralCi.add("Yes"); + utf8mb3GeneralCi.add("Yes"); + utf8mb3GeneralCi.add("1"); + rows.add(utf8mb3GeneralCi); + // Set the result set and send it using the executor + return new ShowResultSet(COLLATION_META_DATA, rows); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitShowCollationCommand(this, context); + } + + @Override + public String toString() { + return "SHOW COLLATION" + (wild != null ? " LIKE '" + wild + "'" : ""); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index 98050c598e1c55..f3bf05b2377d1f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -72,6 +72,7 @@ import org.apache.doris.nereids.trees.plans.commands.ShowAuthorsCommand; import org.apache.doris.nereids.trees.plans.commands.ShowBackendsCommand; import org.apache.doris.nereids.trees.plans.commands.ShowBrokerCommand; +import org.apache.doris.nereids.trees.plans.commands.ShowCollationCommand; import org.apache.doris.nereids.trees.plans.commands.ShowConfigCommand; import org.apache.doris.nereids.trees.plans.commands.ShowConstraintsCommand; import org.apache.doris.nereids.trees.plans.commands.ShowCreateCatalogCommand; @@ -521,6 +522,10 @@ default R visitShowTabletsBelongCommand(ShowTabletsBelongCommand showTabletBelon return visitCommand(showTabletBelongCommand, context); } + default R visitShowCollationCommand(ShowCollationCommand showCollationCommand, C context) { + return visitCommand(showCollationCommand, context); + } + default R visitCreateRoutineLoadCommand(CreateRoutineLoadCommand createRoutineLoadCommand, C context) { return visitCommand(createRoutineLoadCommand, context); } diff --git a/regression-test/data/nereids_p0/show/test_show_collation.out b/regression-test/data/nereids_p0/show/test_show_collation.out new file mode 100644 index 00000000000000..60bac6866a761e --- /dev/null +++ b/regression-test/data/nereids_p0/show/test_show_collation.out @@ -0,0 +1,5 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !cmd -- +utf8mb4_0900_bin utf8mb4 309 Yes Yes 1 +utf8mb3_general_ci utf8mb3 33 Yes Yes 1 + diff --git a/regression-test/suites/nereids_p0/show/test_show_collation.groovy b/regression-test/suites/nereids_p0/show/test_show_collation.groovy new file mode 100644 index 00000000000000..00837d72161a97 --- /dev/null +++ b/regression-test/suites/nereids_p0/show/test_show_collation.groovy @@ -0,0 +1,29 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("test_show_collation", "query,collation") { + try { + // Execute the SHOW COLLATION command and verify the output + checkNereidsExecute("SHOW COLLATION") + qt_cmd("SHOW COLLATION") + } catch (Exception e) { + // Log any exceptions that occur during testing + log.error("Failed to execute SHOW COLLATION command", e) + throw e + } +} + From b49374d30f3b515a5e6f86837c7caec7633ebfc5 Mon Sep 17 00:00:00 2001 From: Sridhar R Manikarnike Date: Mon, 2 Dec 2024 08:28:19 +0530 Subject: [PATCH 101/399] [Enhancement] (nereids)implement showProcessListCommand in nereids (#43158) Issue Number: close #42729 --- .../org/apache/doris/nereids/DorisParser.g4 | 2 +- .../nereids/parser/LogicalPlanBuilder.java | 7 + .../doris/nereids/trees/plans/PlanType.java | 1 + .../commands/ShowProcessListCommand.java | 131 ++++++++++++++++++ .../trees/plans/visitor/CommandVisitor.java | 5 + .../test_show_process_list_command.groovy | 30 ++++ 6 files changed, 175 insertions(+), 1 deletion(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowProcessListCommand.java create mode 100644 regression-test/suites/nereids_p0/show/test_show_process_list_command.groovy diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index 8edcbb40c3c13b..d30ef144e52cd0 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -229,6 +229,7 @@ supportedShowStatement | SHOW PLUGINS #showPlugins | SHOW REPOSITORIES #showRepositories | SHOW BRIEF? CREATE TABLE name=multipartIdentifier #showCreateTable + | SHOW FULL? PROCESSLIST #showProcessList | SHOW ROLES #showRoles | SHOW PARTITION partitionId=INTEGER_VALUE #showPartitionId | SHOW PRIVILEGES #showPrivileges @@ -295,7 +296,6 @@ unsupportedShowStatement | SHOW TABLE STATUS ((FROM | IN) database=multipartIdentifier)? wildWhere? #showTableStatus | SHOW FULL? TABLES ((FROM | IN) database=multipartIdentifier)? wildWhere? #showTables | SHOW FULL? VIEWS ((FROM | IN) database=multipartIdentifier)? wildWhere? #showViews - | SHOW FULL? PROCESSLIST #showProcessList | SHOW (GLOBAL | SESSION | LOCAL)? STATUS wildWhere? #showStatus | SHOW CREATE MATERIALIZED VIEW name=multipartIdentifier #showMaterializedView | SHOW CREATE (GLOBAL | SESSION | LOCAL)? FUNCTION functionIdentifier diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index 60eda00ce89620..1422d807b71bd9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -248,6 +248,7 @@ import org.apache.doris.nereids.DorisParser.ShowPrivilegesContext; import org.apache.doris.nereids.DorisParser.ShowProcContext; import org.apache.doris.nereids.DorisParser.ShowProcedureStatusContext; +import org.apache.doris.nereids.DorisParser.ShowProcessListContext; import org.apache.doris.nereids.DorisParser.ShowReplicaDistributionContext; import org.apache.doris.nereids.DorisParser.ShowRepositoriesContext; import org.apache.doris.nereids.DorisParser.ShowRolesContext; @@ -536,6 +537,7 @@ import org.apache.doris.nereids.trees.plans.commands.ShowPrivilegesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowProcCommand; import org.apache.doris.nereids.trees.plans.commands.ShowProcedureStatusCommand; +import org.apache.doris.nereids.trees.plans.commands.ShowProcessListCommand; import org.apache.doris.nereids.trees.plans.commands.ShowReplicaDistributionCommand; import org.apache.doris.nereids.trees.plans.commands.ShowRepositoriesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowRolesCommand; @@ -4756,6 +4758,11 @@ public LogicalPlan visitShowTableId(ShowTableIdContext ctx) { return new ShowTableIdCommand(tableId); } + @Override + public LogicalPlan visitShowProcessList(ShowProcessListContext ctx) { + return new ShowProcessListCommand(ctx.FULL() != null); + } + @Override public LogicalPlan visitSync(SyncContext ctx) { return new SyncCommand(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index d07c90b7ccd5e7..177e6b4fcff157 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -212,6 +212,7 @@ public enum PlanType { SHOW_LAST_INSERT_COMMAND, SHOW_LOAD_PROFILE_COMMAND, SHOW_PARTITIONID_COMMAND, + SHOW_PROCESSLIST_COMMAND, SHOW_PROC_COMMAND, SHOW_PLUGINS_COMMAND, SHOW_PRIVILEGES_COMMAND, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowProcessListCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowProcessListCommand.java new file mode 100644 index 00000000000000..fe04d61eab689e --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowProcessListCommand.java @@ -0,0 +1,131 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.commands; + +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.PrimitiveType; +import org.apache.doris.catalog.ScalarType; +import org.apache.doris.common.ClientPool; +import org.apache.doris.common.Pair; +import org.apache.doris.common.proc.FrontendsProcNode; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.ShowResultSet; +import org.apache.doris.qe.ShowResultSetMetaData; +import org.apache.doris.qe.StmtExecutor; +import org.apache.doris.thrift.FrontendService; +import org.apache.doris.thrift.TNetworkAddress; +import org.apache.doris.thrift.TShowProcessListRequest; +import org.apache.doris.thrift.TShowProcessListResult; + +import com.google.common.collect.Lists; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.List; + +/** + * Represents the command for SHOW PROCESSLIST + */ +public class ShowProcessListCommand extends ShowCommand { + private static final Logger LOG = LogManager.getLogger(ShowProcessListCommand.class); + private static final ShowResultSetMetaData PROCESSLIST_META_DATA = ShowResultSetMetaData.builder() + .addColumn(new Column("CurrentConnected", ScalarType.createVarchar(16))) + .addColumn(new Column("Id", ScalarType.createType(PrimitiveType.BIGINT))) + .addColumn(new Column("User", ScalarType.createVarchar(16))) + .addColumn(new Column("Host", ScalarType.createVarchar(16))) + .addColumn(new Column("LoginTime", ScalarType.createVarchar(16))) + .addColumn(new Column("Catalog", ScalarType.createVarchar(16))) + .addColumn(new Column("Db", ScalarType.createVarchar(16))) + .addColumn(new Column("Command", ScalarType.createVarchar(16))) + .addColumn(new Column("Time", ScalarType.createType(PrimitiveType.INT))) + .addColumn(new Column("State", ScalarType.createVarchar(64))) + .addColumn(new Column("QueryId", ScalarType.createVarchar(64))) + .addColumn(new Column("Info", ScalarType.STRING)) + .addColumn(new Column("FE", ScalarType.createVarchar(16))) + .addColumn(new Column("CloudCluster", ScalarType.createVarchar(16))).build(); + + private final boolean isFull; + + public ShowProcessListCommand(boolean isFull) { + super(PlanType.SHOW_PROCESSLIST_COMMAND); + this.isFull = isFull; + } + + @Override + public ShowResultSet doRun(ConnectContext ctx, StmtExecutor executor) throws Exception { + boolean isShowFullSql = isFull; + boolean isShowAllFe = ConnectContext.get().getSessionVariable().getShowAllFeConnection(); + + List> rowSet = Lists.newArrayList(); + List threadInfos = ctx.getConnectScheduler() + .listConnection(ctx.getQualifiedUser(), isShowFullSql); + long nowMs = System.currentTimeMillis(); + for (ConnectContext.ThreadInfo info : threadInfos) { + rowSet.add(info.toRow(ctx.getConnectionId(), nowMs)); + } + + if (isShowAllFe) { + try { + TShowProcessListRequest request = new TShowProcessListRequest(); + request.setShowFullSql(isShowFullSql); + request.setCurrentUserIdent(ConnectContext.get().getCurrentUserIdentity().toThrift()); + List> frontends = FrontendsProcNode.getFrontendWithRpcPort(Env.getCurrentEnv(), + false); + FrontendService.Client client = null; + for (Pair fe : frontends) { + TNetworkAddress thriftAddress = new TNetworkAddress(fe.key(), fe.value()); + try { + client = ClientPool.frontendPool.borrowObject(thriftAddress, 3000); + } catch (Exception e) { + LOG.warn("Failed to get frontend {} client. exception: {}", fe.key(), e); + continue; + } + + boolean isReturnToPool = false; + try { + TShowProcessListResult result = client.showProcessList(request); + if (result.process_list != null && result.process_list.size() > 0) { + rowSet.addAll(result.process_list); + } + isReturnToPool = true; + } catch (Exception e) { + LOG.warn("Failed to request processlist to fe: {} . exception: {}", fe.key(), e); + } finally { + if (isReturnToPool) { + ClientPool.frontendPool.returnObject(thriftAddress, client); + } else { + ClientPool.frontendPool.invalidateObject(thriftAddress, client); + } + } + } + } catch (Throwable t) { + LOG.warn(" fetch process list from other fe failed, ", t); + } + } + + return new ShowResultSet(PROCESSLIST_META_DATA, rowSet); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitShowProcessListCommand(this, context); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index f3bf05b2377d1f..de5228f498178b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -95,6 +95,7 @@ import org.apache.doris.nereids.trees.plans.commands.ShowPrivilegesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowProcCommand; import org.apache.doris.nereids.trees.plans.commands.ShowProcedureStatusCommand; +import org.apache.doris.nereids.trees.plans.commands.ShowProcessListCommand; import org.apache.doris.nereids.trees.plans.commands.ShowReplicaDistributionCommand; import org.apache.doris.nereids.trees.plans.commands.ShowRepositoriesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowRolesCommand; @@ -529,4 +530,8 @@ default R visitShowCollationCommand(ShowCollationCommand showCollationCommand, C default R visitCreateRoutineLoadCommand(CreateRoutineLoadCommand createRoutineLoadCommand, C context) { return visitCommand(createRoutineLoadCommand, context); } + + default R visitShowProcessListCommand(ShowProcessListCommand showProcessListCommand, C context) { + return visitCommand(showProcessListCommand, context); + } } diff --git a/regression-test/suites/nereids_p0/show/test_show_process_list_command.groovy b/regression-test/suites/nereids_p0/show/test_show_process_list_command.groovy new file mode 100644 index 00000000000000..b67ab1148c0eac --- /dev/null +++ b/regression-test/suites/nereids_p0/show/test_show_process_list_command.groovy @@ -0,0 +1,30 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("test_show_process_list_command", "query,process_list") { + try { + // Execute the SHOW PROCESSLIST command and verify the output + checkNereidsExecute("SHOW PROCESSLIST") + + // Execute the SHOW FULL PROCESSLIST command and verify the output + checkNereidsExecute("SHOW FULL PROCESSLIST") + } catch (Exception e) { + // Log any exceptions that occur during testing + log.error("Failed to execute SHOW PROCESSLIST command", e) + throw e + } +} From 3e41c91246f0ed615c546e6d7f6dfc8b3e698a87 Mon Sep 17 00:00:00 2001 From: Petrichor <31833513+vinlee19@users.noreply.github.com> Date: Mon, 2 Dec 2024 10:58:37 +0800 Subject: [PATCH 102/399] [feat](Nereids) support refresh database command (#44298) Issue Number: close #https://github.com/apache/doris/issues/42579 --- .../org/apache/doris/nereids/DorisParser.g4 | 2 +- .../nereids/parser/LogicalPlanBuilder.java | 22 ++- .../doris/nereids/trees/plans/PlanType.java | 1 + .../refresh/RefreshDatabaseCommand.java | 149 ++++++++++++++++++ .../trees/plans/visitor/CommandVisitor.java | 5 + .../test_nereids_refresh_catalog.out | 24 ++- .../test_nereids_refresh_catalog.groovy | 28 +++- 7 files changed, 218 insertions(+), 13 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/refresh/RefreshDatabaseCommand.java diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index d30ef144e52cd0..fd5ae1bd5744f9 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -437,6 +437,7 @@ channelDescription supportedRefreshStatement : REFRESH CATALOG name=identifier propertyClause? #refreshCatalog + | REFRESH DATABASE name=multipartIdentifier propertyClause? #refreshDatabase ; supportedCleanStatement @@ -445,7 +446,6 @@ supportedCleanStatement unsupportedRefreshStatement : REFRESH TABLE name=multipartIdentifier #refreshTable - | REFRESH DATABASE name=multipartIdentifier propertyClause? #refreshDatabase | REFRESH LDAP (ALL | (FOR user=identifierOrText)) #refreshLdap ; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index 1422d807b71bd9..70c4f61e4033c4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -188,6 +188,7 @@ import org.apache.doris.nereids.DorisParser.RecoverPartitionContext; import org.apache.doris.nereids.DorisParser.RecoverTableContext; import org.apache.doris.nereids.DorisParser.RefreshCatalogContext; +import org.apache.doris.nereids.DorisParser.RefreshDatabaseContext; import org.apache.doris.nereids.DorisParser.RefreshMTMVContext; import org.apache.doris.nereids.DorisParser.RefreshMethodContext; import org.apache.doris.nereids.DorisParser.RefreshScheduleContext; @@ -618,6 +619,7 @@ import org.apache.doris.nereids.trees.plans.commands.load.LoadSequenceClause; import org.apache.doris.nereids.trees.plans.commands.load.LoadWhereClause; import org.apache.doris.nereids.trees.plans.commands.refresh.RefreshCatalogCommand; +import org.apache.doris.nereids.trees.plans.commands.refresh.RefreshDatabaseCommand; import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; import org.apache.doris.nereids.trees.plans.logical.LogicalCTE; import org.apache.doris.nereids.trees.plans.logical.LogicalExcept; @@ -4351,6 +4353,25 @@ public Object visitRefreshCatalog(RefreshCatalogContext ctx) { } @Override + public RefreshDatabaseCommand visitRefreshDatabase(RefreshDatabaseContext ctx) { + Map properties = visitPropertyClause(ctx.propertyClause()) == null ? Maps.newHashMap() + : visitPropertyClause(ctx.propertyClause()); + List parts = visitMultipartIdentifier(ctx.name); + int size = parts.size(); + if (size == 0) { + throw new ParseException("database name can't be empty"); + } + String dbName = parts.get(size - 1); + + // [db]. + if (size == 1) { + return new RefreshDatabaseCommand(dbName, properties); + } else if (parts.size() == 2) { // [ctl,db]. + return new RefreshDatabaseCommand(parts.get(0), dbName, properties); + } + throw new ParseException("Only one dot can be in the name: " + String.join(".", parts)); + } + public LogicalPlan visitShowLastInsert(ShowLastInsertContext ctx) { return new ShowLastInsertCommand(); } @@ -4379,7 +4400,6 @@ public LogicalPlan visitShowPartitionId(ShowPartitionIdContext ctx) { partitionId = Long.parseLong(ctx.partitionId.getText()); } return new ShowPartitionIdCommand(partitionId); - } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index 177e6b4fcff157..6a8fcadaf520c5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -187,6 +187,7 @@ public enum PlanType { SET_USER_PROPERTIES_COMMAND, SET_DEFAULT_STORAGE_VAULT_COMMAND, REFRESH_CATALOG_COMMAND, + REFRESH_DATABASE_COMMAND, PREPARED_COMMAND, EXECUTE_COMMAND, DROP_SQL_BLOCK_RULE_COMMAND, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/refresh/RefreshDatabaseCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/refresh/RefreshDatabaseCommand.java new file mode 100644 index 00000000000000..010adbb05d1cba --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/refresh/RefreshDatabaseCommand.java @@ -0,0 +1,149 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.commands.refresh; + +import org.apache.doris.analysis.StmtType; +import org.apache.doris.catalog.DatabaseIf; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.InfoSchemaDb; +import org.apache.doris.catalog.MysqlDb; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.DdlException; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.datasource.CatalogIf; +import org.apache.doris.datasource.ExternalCatalog; +import org.apache.doris.datasource.ExternalDatabase; +import org.apache.doris.datasource.ExternalObjectLog; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.commands.Command; +import org.apache.doris.nereids.trees.plans.commands.ForwardWithSync; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.StmtExecutor; + +import com.google.common.base.Strings; + +import java.util.Map; + +/** + * Refresh database. + */ +public class RefreshDatabaseCommand extends Command implements ForwardWithSync { + private static final String INVALID_CACHE = "invalid_cache"; + + private String catalogName; + private String dbName; + private Map properties; + private boolean invalidCache = false; + + public RefreshDatabaseCommand(String dbName, Map properties) { + super(PlanType.REFRESH_DATABASE_COMMAND); + this.dbName = dbName; + this.properties = properties; + } + + public RefreshDatabaseCommand(String catalogName, String dbName, Map properties) { + super(PlanType.REFRESH_DATABASE_COMMAND); + this.catalogName = catalogName; + this.dbName = dbName; + this.properties = properties; + } + + private void validate(ConnectContext ctx) throws AnalysisException { + if (Strings.isNullOrEmpty(catalogName)) { + catalogName = ConnectContext.get().getCurrentCatalog().getName(); + } + if (Strings.isNullOrEmpty(dbName)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_WRONG_DB_NAME, dbName); + } + + // Don't allow dropping 'information_schema' database + if (dbName.equalsIgnoreCase(InfoSchemaDb.DATABASE_NAME)) { + + ErrorReport.reportAnalysisException( + ErrorCode.ERR_DBACCESS_DENIED_ERROR, ctx.getQualifiedUser(), dbName); + } + // Don't allow dropping 'mysql' database + if (dbName.equalsIgnoreCase(MysqlDb.DATABASE_NAME)) { + ErrorReport.reportAnalysisException( + ErrorCode.ERR_DBACCESS_DENIED_ERROR, ctx.getQualifiedUser(), dbName); + } + // check access + if (!Env.getCurrentEnv().getAccessManager().checkDbPriv(ConnectContext.get(), catalogName, + dbName, PrivPredicate.SHOW)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_DB_ACCESS_DENIED_ERROR, + PrivPredicate.SHOW.getPrivs().toString(), dbName); + } + String invalidConfig = properties == null ? null : properties.get(INVALID_CACHE); + // Default is to invalid cache. + invalidCache = invalidConfig == null || invalidConfig.equalsIgnoreCase("true"); + } + + /** + * Refresh database + */ + public void handleRefreshDb() throws DdlException { + Env env = Env.getCurrentEnv(); + CatalogIf catalog = catalogName != null ? env.getCatalogMgr().getCatalog(catalogName) : env.getCurrentCatalog(); + if (catalog == null) { + throw new DdlException("Catalog " + catalogName + " doesn't exist."); + } + if (!(catalog instanceof ExternalCatalog)) { + throw new DdlException("Only support refresh database in external catalog"); + } + DatabaseIf db = catalog.getDbOrDdlException(dbName); + ((ExternalDatabase) db).setUnInitialized(invalidCache); + + ExternalObjectLog log = new ExternalObjectLog(); + log.setCatalogId(catalog.getId()); + log.setDbId(db.getId()); + log.setInvalidCache(invalidCache); + Env.getCurrentEnv().getEditLog().logRefreshExternalDb(log); + } + + @Override + public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { + validate(ctx); + handleRefreshDb(); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitRefreshDatabaseCommand(this, context); + } + + /** + * refresh database statement. + */ + public String toSql() { + StringBuilder sb = new StringBuilder(); + sb.append("REFRESH DATABASE "); + if (catalogName != null) { + sb.append("`").append(catalogName).append("`."); + } + sb.append("`").append(dbName).append("`"); + return sb.toString(); + } + + @Override + public StmtType stmtType() { + return StmtType.REFRESH; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index de5228f498178b..fec52fd395f9db 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -119,6 +119,7 @@ import org.apache.doris.nereids.trees.plans.commands.insert.InsertOverwriteTableCommand; import org.apache.doris.nereids.trees.plans.commands.load.CreateRoutineLoadCommand; import org.apache.doris.nereids.trees.plans.commands.refresh.RefreshCatalogCommand; +import org.apache.doris.nereids.trees.plans.commands.refresh.RefreshDatabaseCommand; /** CommandVisitor. */ public interface CommandVisitor { @@ -353,6 +354,10 @@ default R visitShowViewCommand(ShowViewCommand showViewCommand, C context) { return visitCommand(showViewCommand, context); } + default R visitRefreshDatabaseCommand(RefreshDatabaseCommand refreshDatabaseCommand, C context) { + return visitCommand(refreshDatabaseCommand, context); + } + default R visitShowBackendsCommand(ShowBackendsCommand showBackendsCommand, C context) { return visitCommand(showBackendsCommand, context); } diff --git a/regression-test/data/external_table_p0/nereids_commands/test_nereids_refresh_catalog.out b/regression-test/data/external_table_p0/nereids_commands/test_nereids_refresh_catalog.out index 89b47bbe4eb1e7..c3f57d7ba810b8 100644 --- a/regression-test/data/external_table_p0/nereids_commands/test_nereids_refresh_catalog.out +++ b/regression-test/data/external_table_p0/nereids_commands/test_nereids_refresh_catalog.out @@ -1,5 +1,5 @@ -- This file is automatically generated. You should know what you did if you want to edit this --- !sql -- +-- !database -- DORIS Doris doris @@ -19,7 +19,7 @@ show_test_do_not_modify 114 abf 115 abg --- !sql -- +-- !preceding_create_external_database -- DORIS Doris doris @@ -29,7 +29,7 @@ init_db mysql show_test_do_not_modify --- !sql -- +-- !subsequent_create_external_database -- DORIS Doris doris @@ -40,7 +40,21 @@ mysql new_mysql_db show_test_do_not_modify --- !sql -- +-- !sql_show_tables -- + +-- !preceding_refresh_database -- + +-- !subsequent_refresh_database -- +new_mysql_table1 + +-- !preceding_refresh_database -- +new_mysql_table1 + +-- !subsequent_refresh_database -- +new_mysql_table1 +new_mysql_table2 + +-- !preceding_drop_external_database -- DORIS Doris doris @@ -51,7 +65,7 @@ mysql new_mysql_db show_test_do_not_modify --- !sql -- +-- !subsequent_drop_external_database -- DORIS Doris doris diff --git a/regression-test/suites/external_table_p0/nereids_commands/test_nereids_refresh_catalog.groovy b/regression-test/suites/external_table_p0/nereids_commands/test_nereids_refresh_catalog.groovy index f74f78a3709dc2..87dacd4142501e 100644 --- a/regression-test/suites/external_table_p0/nereids_commands/test_nereids_refresh_catalog.groovy +++ b/regression-test/suites/external_table_p0/nereids_commands/test_nereids_refresh_catalog.groovy @@ -28,6 +28,8 @@ suite("test_nereids_refresh_catalog", "p0,external,mysql,external_docker,externa String mysql_port = context.config.otherConfigs.get("mysql_57_port"); String ex_tb0 = "ex_tb0"; String new_mysql_db = "new_mysql_db"; + String new_mysql_table1 = "new_mysql_table1"; + String new_mysql_table2 = "new_mysql_table2"; sql """drop catalog if exists ${catalog_name} """ @@ -43,27 +45,41 @@ suite("test_nereids_refresh_catalog", "p0,external,mysql,external_docker,externa "driver_class" = "com.mysql.cj.jdbc.Driver" );""" - sql """switch ${catalog_name}""" sql """CALL EXECUTE_STMT("${catalog_name}", "drop database if exists ${new_mysql_db}");""" + sql """switch ${catalog_name}""" - qt_sql """show databases;""" + qt_database """show databases;""" sql """ use ${ex_db_name}""" qt_ex_tb0_where """select id from ${ex_tb0} where id = 111;""" order_qt_ex_tb0 """ select id, name from ${ex_tb0} order by id; """ // create database in mysql sql """CALL EXECUTE_STMT("${catalog_name}", "create database ${new_mysql_db} ;");""" - qt_sql """show databases;""" + qt_preceding_create_external_database """show databases;""" checkNereidsExecute("refresh catalog ${catalog_name} ;") - qt_sql """show databases;""" + qt_subsequent_create_external_database """show databases;""" checkNereidsExecute("refresh catalog ${catalog_name} properties ('invalid_cache'='true');") + sql """use ${new_mysql_db}""" + qt_sql_show_tables """show tables;""" + + // create table in mysql external database + sql """CALL EXECUTE_STMT("${catalog_name}", "create table ${new_mysql_db}.${new_mysql_table1} (id int, name varchar(20));");""" + + qt_preceding_refresh_database """show tables;""" + checkNereidsExecute("refresh database ${new_mysql_db} ;") + qt_subsequent_refresh_database """show tables;""" + + sql """CALL EXECUTE_STMT("${catalog_name}", "create table ${new_mysql_db}.${new_mysql_table2} (id int, name varchar(20));");""" + qt_preceding_refresh_database """show tables;""" + checkNereidsExecute("refresh database ${catalog_name}.${new_mysql_db} ;") + qt_subsequent_refresh_database """show tables;""" sql """CALL EXECUTE_STMT("${catalog_name}", "drop database if exists ${new_mysql_db} ;");""" - qt_sql """show databases;""" + qt_preceding_drop_external_database """show databases;""" checkNereidsExecute("refresh catalog ${catalog_name} properties ('invalid_cache'='true');") - qt_sql """show databases;""" + qt_subsequent_drop_external_database """show databases;""" sql """ drop catalog if exists ${catalog_name} ;""" } From 2bd293c560afdb7665f05ae513f1354d6dfb6371 Mon Sep 17 00:00:00 2001 From: Vallish Pai Date: Mon, 2 Dec 2024 08:30:54 +0530 Subject: [PATCH 103/399] [Enhancement] (nereids)implement CreateSqlBlockCommand and AlterSqlBlockCommand in nereids (#44702) Issue Number: close #42592 #42796 --- .../org/apache/doris/nereids/DorisParser.g4 | 8 +- .../doris/blockrule/SqlBlockRuleMgr.java | 12 +- .../nereids/parser/LogicalPlanBuilder.java | 18 +++ .../doris/nereids/trees/plans/PlanType.java | 2 + .../commands/AlterSqlBlockRuleCommand.java | 87 +++++++++++++ .../commands/CreateSqlBlockRuleCommand.java | 99 +++++++++++++++ .../plans/commands/SqlBlockRuleCommand.java | 117 ++++++++++++++++++ .../trees/plans/visitor/CommandVisitor.java | 10 ++ .../test_sql_block_rule.groovy | 11 +- 9 files changed, 349 insertions(+), 15 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterSqlBlockRuleCommand.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateSqlBlockRuleCommand.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/SqlBlockRuleCommand.java diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index fd5ae1bd5744f9..6e2a518ec20982 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -185,7 +185,9 @@ supportedCreateStatement ON table=multipartIdentifier AS type=(RESTRICTIVE | PERMISSIVE) TO (user=userIdentify | ROLE roleName=identifier) - USING LEFT_PAREN booleanExpression RIGHT_PAREN #createRowPolicy + USING LEFT_PAREN booleanExpression RIGHT_PAREN #createRowPolicy + | CREATE SQL_BLOCK_RULE (IF NOT EXISTS)? + name=identifier properties=propertyClause? #createSqlBlockRule ; supportedAlterStatement @@ -197,6 +199,7 @@ supportedAlterStatement properties=propertyClause? #alterWorkloadGroup | ALTER WORKLOAD POLICY name=identifierOrText properties=propertyClause? #alterWorkloadPolicy + | ALTER SQL_BLOCK_RULE name=identifier properties=propertyClause? #alterSqlBlockRule ; supportedDropStatement @@ -573,7 +576,6 @@ unsupportedAlterStatement SET LEFT_PAREN propertyItemList RIGHT_PAREN #alterColocateGroup | ALTER ROUTINE LOAD FOR name=multipartIdentifier properties=propertyClause? (FROM type=identifier LEFT_PAREN propertyItemList RIGHT_PAREN)? #alterRoutineLoad - | ALTER SQL_BLOCK_RULE name=identifier properties=propertyClause? #alterSqlBlockRule | ALTER TABLE name=multipartIdentifier SET LEFT_PAREN propertyItemList RIGHT_PAREN #alterTableProperties | ALTER STORAGE POLICY name=identifierOrText @@ -765,8 +767,6 @@ unsupportedCreateStatement (ACTIONS LEFT_PAREN workloadPolicyActions RIGHT_PAREN)? properties=propertyClause? #createWorkloadPolicy | CREATE ENCRYPTKEY (IF NOT EXISTS)? multipartIdentifier AS STRING_LITERAL #createEncryptkey - | CREATE SQL_BLOCK_RULE (IF NOT EXISTS)? - name=identifier properties=propertyClause? #createSqlBlockRule | CREATE STORAGE POLICY (IF NOT EXISTS)? name=identifier properties=propertyClause? #createStoragePolicy | BUILD INDEX name=identifier ON tableName=multipartIdentifier diff --git a/fe/fe-core/src/main/java/org/apache/doris/blockrule/SqlBlockRuleMgr.java b/fe/fe-core/src/main/java/org/apache/doris/blockrule/SqlBlockRuleMgr.java index aa3b844f3e38ef..e4e288bc14b2bd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/blockrule/SqlBlockRuleMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/blockrule/SqlBlockRuleMgr.java @@ -116,12 +116,15 @@ private static void verifyLimitations(SqlBlockRule sqlBlockRule) throws DdlExcep * Create SqlBlockRule for create stmt. **/ public void createSqlBlockRule(CreateSqlBlockRuleStmt stmt) throws UserException { + createSqlBlockRule(SqlBlockRule.fromCreateStmt(stmt), stmt.isIfNotExists()); + } + + public void createSqlBlockRule(SqlBlockRule sqlBlockRule, boolean isIfNotExists) throws UserException { writeLock(); try { - SqlBlockRule sqlBlockRule = SqlBlockRule.fromCreateStmt(stmt); String ruleName = sqlBlockRule.getName(); if (existRule(ruleName)) { - if (stmt.isIfNotExists()) { + if (isIfNotExists) { return; } throw new DdlException("the sql block rule " + ruleName + " already create"); @@ -146,9 +149,12 @@ public void replayCreate(SqlBlockRule sqlBlockRule) { * Alter SqlBlockRule for alter stmt. **/ public void alterSqlBlockRule(AlterSqlBlockRuleStmt stmt) throws AnalysisException, DdlException { + alterSqlBlockRule(SqlBlockRule.fromAlterStmt(stmt)); + } + + public void alterSqlBlockRule(SqlBlockRule sqlBlockRule) throws AnalysisException, DdlException { writeLock(); try { - SqlBlockRule sqlBlockRule = SqlBlockRule.fromAlterStmt(stmt); String ruleName = sqlBlockRule.getName(); if (!existRule(ruleName)) { throw new DdlException("the sql block rule " + ruleName + " not exist"); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index 70c4f61e4033c4..dfe879e7d44dad 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -57,6 +57,7 @@ import org.apache.doris.nereids.DorisParser.AliasedQueryContext; import org.apache.doris.nereids.DorisParser.AlterMTMVContext; import org.apache.doris.nereids.DorisParser.AlterRoleContext; +import org.apache.doris.nereids.DorisParser.AlterSqlBlockRuleContext; import org.apache.doris.nereids.DorisParser.AlterStorageVaultContext; import org.apache.doris.nereids.DorisParser.AlterViewContext; import org.apache.doris.nereids.DorisParser.AlterWorkloadGroupContext; @@ -92,6 +93,7 @@ import org.apache.doris.nereids.DorisParser.CreateProcedureContext; import org.apache.doris.nereids.DorisParser.CreateRoutineLoadContext; import org.apache.doris.nereids.DorisParser.CreateRowPolicyContext; +import org.apache.doris.nereids.DorisParser.CreateSqlBlockRuleContext; import org.apache.doris.nereids.DorisParser.CreateTableContext; import org.apache.doris.nereids.DorisParser.CreateTableLikeContext; import org.apache.doris.nereids.DorisParser.CreateViewContext; @@ -460,6 +462,7 @@ import org.apache.doris.nereids.trees.plans.commands.AddConstraintCommand; import org.apache.doris.nereids.trees.plans.commands.AlterMTMVCommand; import org.apache.doris.nereids.trees.plans.commands.AlterRoleCommand; +import org.apache.doris.nereids.trees.plans.commands.AlterSqlBlockRuleCommand; import org.apache.doris.nereids.trees.plans.commands.AlterStorageVaultCommand; import org.apache.doris.nereids.trees.plans.commands.AlterViewCommand; import org.apache.doris.nereids.trees.plans.commands.AlterWorkloadGroupCommand; @@ -477,6 +480,7 @@ import org.apache.doris.nereids.trees.plans.commands.CreateMTMVCommand; import org.apache.doris.nereids.trees.plans.commands.CreatePolicyCommand; import org.apache.doris.nereids.trees.plans.commands.CreateProcedureCommand; +import org.apache.doris.nereids.trees.plans.commands.CreateSqlBlockRuleCommand; import org.apache.doris.nereids.trees.plans.commands.CreateTableCommand; import org.apache.doris.nereids.trees.plans.commands.CreateTableLikeCommand; import org.apache.doris.nereids.trees.plans.commands.CreateViewCommand; @@ -4154,6 +4158,20 @@ public LogicalPlan visitShowCreateProcedure(ShowCreateProcedureContext ctx) { return ParserUtils.withOrigin(ctx, () -> new ShowCreateProcedureCommand(procedureName)); } + @Override + public LogicalPlan visitCreateSqlBlockRule(CreateSqlBlockRuleContext ctx) { + Map properties = ctx.propertyClause() != null + ? Maps.newHashMap(visitPropertyClause(ctx.propertyClause())) : Maps.newHashMap(); + return new CreateSqlBlockRuleCommand(stripQuotes(ctx.name.getText()), ctx.EXISTS() != null, properties); + } + + @Override + public LogicalPlan visitAlterSqlBlockRule(AlterSqlBlockRuleContext ctx) { + Map properties = ctx.propertyClause() != null + ? Maps.newHashMap(visitPropertyClause(ctx.propertyClause())) : Maps.newHashMap(); + return new AlterSqlBlockRuleCommand(stripQuotes(ctx.name.getText()), properties); + } + @Override public LogicalPlan visitDropCatalogRecycleBin(DropCatalogRecycleBinContext ctx) { String idTypeStr = ctx.idType.getText().substring(1, ctx.idType.getText().length() - 1); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index 6a8fcadaf520c5..14a1b13a4667df 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -134,6 +134,7 @@ public enum PlanType { // commands CREATE_POLICY_COMMAND, CREATE_TABLE_COMMAND, + CREATE_SQL_BLOCK_RULE_COMMAND, DELETE_COMMAND, EXPLAIN_COMMAND, EXPORT_COMMAND, @@ -194,6 +195,7 @@ public enum PlanType { DROP_USER_COMMAND, DROP_WORKLOAD_GROUP_NAME, DROP_WORKLOAD_POLICY_COMMAND, + ALTER_SQL_BLOCK_RULE_COMMAND, SHOW_BACKENDS_COMMAND, SHOW_BLOCK_RULE_COMMAND, SHOW_BROKER_COMMAND, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterSqlBlockRuleCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterSqlBlockRuleCommand.java new file mode 100644 index 00000000000000..481eb64a58d6d2 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterSqlBlockRuleCommand.java @@ -0,0 +1,87 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.commands; + +import org.apache.doris.analysis.StmtType; +import org.apache.doris.blockrule.SqlBlockRule; +import org.apache.doris.catalog.Env; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.util.SqlBlockUtil; +import org.apache.doris.common.util.Util; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.StmtExecutor; + +import org.apache.commons.lang3.StringUtils; + +import java.util.Map; + +/** + * alter Sql block rule Commands. + */ +public class AlterSqlBlockRuleCommand extends SqlBlockRuleCommand { + + /** + * constructor + */ + public AlterSqlBlockRuleCommand(String ruleName, Map properties) { + super(ruleName, properties, PlanType.ALTER_SQL_BLOCK_RULE_COMMAND); + } + + @Override + public void doRun(ConnectContext ctx, StmtExecutor executor) throws Exception { + Env.getCurrentEnv().getSqlBlockRuleMgr().alterSqlBlockRule(new SqlBlockRule(ruleName, + sql, sqlHash, partitionNum, + tabletNum, cardinality, global, enable)); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitAlterSqlBlockRuleCommand(this, context); + } + + @Override + public StmtType stmtType() { + return StmtType.ALTER; + } + + @Override + public void setProperties(Map properties) throws AnalysisException { + this.sql = properties.getOrDefault(SQL_PROPERTY, SqlBlockRuleCommand.STRING_NOT_SET); + this.sqlHash = properties.getOrDefault(SQL_HASH_PROPERTY, SqlBlockRuleCommand.STRING_NOT_SET); + String partitionNumString = properties.get(SCANNED_PARTITION_NUM); + String tabletNumString = properties.get(SCANNED_TABLET_NUM); + String cardinalityString = properties.get(SCANNED_CARDINALITY); + + SqlBlockUtil.checkSqlAndSqlHashSetBoth(sql, sqlHash); + SqlBlockUtil.checkSqlAndLimitationsSetBoth(sql, sqlHash, + partitionNumString, tabletNumString, cardinalityString); + this.partitionNum = Util.getLongPropertyOrDefault(partitionNumString, SqlBlockRuleCommand.LONG_NOT_SET, null, + SCANNED_PARTITION_NUM + " should be a long"); + this.tabletNum = Util.getLongPropertyOrDefault(tabletNumString, SqlBlockRuleCommand.LONG_NOT_SET, null, + SCANNED_TABLET_NUM + " should be a long"); + this.cardinality = Util.getLongPropertyOrDefault(cardinalityString, SqlBlockRuleCommand.LONG_NOT_SET, null, + SCANNED_CARDINALITY + " should be a long"); + // allow null, represents no modification + String globalStr = properties.get(GLOBAL_PROPERTY); + this.global = StringUtils.isNotEmpty(globalStr) ? Boolean.parseBoolean(globalStr) : null; + String enableStr = properties.get(ENABLE_PROPERTY); + this.enable = StringUtils.isNotEmpty(enableStr) ? Boolean.parseBoolean(enableStr) : null; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateSqlBlockRuleCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateSqlBlockRuleCommand.java new file mode 100644 index 00000000000000..ab8e1a1b2689e3 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateSqlBlockRuleCommand.java @@ -0,0 +1,99 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.commands; + +import org.apache.doris.analysis.StmtType; +import org.apache.doris.blockrule.SqlBlockRule; +import org.apache.doris.catalog.Env; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.FeNameFormat; +import org.apache.doris.common.UserException; +import org.apache.doris.common.util.SqlBlockUtil; +import org.apache.doris.common.util.Util; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.StmtExecutor; + +import org.apache.commons.lang3.StringUtils; + +import java.util.Map; +import java.util.regex.Pattern; + +/** + * create Sql block rule Commands. + */ +public class CreateSqlBlockRuleCommand extends SqlBlockRuleCommand { + private static final String NAME_TYPE = "SQL BLOCK RULE NAME"; + private final boolean ifNotExists; + + /** + * constructor + */ + public CreateSqlBlockRuleCommand(String ruleName, boolean ifNotExists, Map properties) { + super(ruleName, properties, PlanType.CREATE_SQL_BLOCK_RULE_COMMAND); + this.ifNotExists = ifNotExists; + } + + @Override + public void doRun(ConnectContext ctx, StmtExecutor executor) throws Exception { + // check name + FeNameFormat.checkCommonName(NAME_TYPE, ruleName); + // avoid a rule block any ddl for itself + if (StringUtils.isNotEmpty(sql) && Pattern.compile(sql).matcher(this.ruleName).find()) { + throw new AnalysisException("sql of SQL_BLOCK_RULE should not match its name"); + } + Env.getCurrentEnv().getSqlBlockRuleMgr().createSqlBlockRule(new SqlBlockRule(ruleName, sql, + sqlHash, partitionNum, + tabletNum, cardinality, global, enable), ifNotExists); + } + + @Override + public void setProperties(Map properties) throws UserException { + this.sql = properties.getOrDefault(SQL_PROPERTY, SqlBlockRuleCommand.STRING_NOT_SET); + this.sqlHash = properties.getOrDefault(SQL_HASH_PROPERTY, SqlBlockRuleCommand.STRING_NOT_SET); + String partitionNumString = properties.get(SCANNED_PARTITION_NUM); + String tabletNumString = properties.get(SCANNED_TABLET_NUM); + String cardinalityString = properties.get(SCANNED_CARDINALITY); + + SqlBlockUtil.checkSqlAndSqlHashSetBoth(sql, sqlHash); + SqlBlockUtil.checkPropertiesValidate(sql, sqlHash, partitionNumString, tabletNumString, cardinalityString); + + this.partitionNum = Util.getLongPropertyOrDefault(partitionNumString, 0L, null, + SCANNED_PARTITION_NUM + " should be a long"); + this.tabletNum = Util.getLongPropertyOrDefault(tabletNumString, 0L, null, + SCANNED_TABLET_NUM + " should be a long"); + this.cardinality = Util.getLongPropertyOrDefault(cardinalityString, 0L, null, + SCANNED_CARDINALITY + " should be a long"); + + this.global = Util.getBooleanPropertyOrDefault(properties.get(GLOBAL_PROPERTY), false, + GLOBAL_PROPERTY + " should be a boolean"); + this.enable = Util.getBooleanPropertyOrDefault(properties.get(ENABLE_PROPERTY), true, + ENABLE_PROPERTY + " should be a boolean"); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitCreateSqlBlockRuleCommand(this, context); + } + + @Override + public StmtType stmtType() { + return StmtType.CREATE; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/SqlBlockRuleCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/SqlBlockRuleCommand.java new file mode 100644 index 00000000000000..785da1aeb5861e --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/SqlBlockRuleCommand.java @@ -0,0 +1,117 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.commands; + +import org.apache.doris.catalog.Env; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.common.UserException; +import org.apache.doris.common.util.SqlBlockUtil; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.StmtExecutor; + +import com.google.common.collect.ImmutableSet; + +import java.util.Map; +import java.util.Optional; + +/** + * Common class for SqlBlockRule Commands. + */ +public abstract class SqlBlockRuleCommand extends Command { + public static final String SQL_PROPERTY = "sql"; + + public static final String SQL_HASH_PROPERTY = "sqlHash"; + + public static final String SCANNED_PARTITION_NUM = "partition_num"; + + public static final String SCANNED_TABLET_NUM = "tablet_num"; + + public static final String SCANNED_CARDINALITY = "cardinality"; + + public static final String GLOBAL_PROPERTY = "global"; + + public static final String ENABLE_PROPERTY = "enable"; + + public static final Long LONG_NOT_SET = SqlBlockUtil.LONG_MINUS_ONE; + + public static final String STRING_NOT_SET = SqlBlockUtil.STRING_DEFAULT; + + private static final ImmutableSet PROPERTIES_SET = new ImmutableSet.Builder().add(SQL_PROPERTY) + .add(SQL_HASH_PROPERTY).add(GLOBAL_PROPERTY).add(ENABLE_PROPERTY).add(SCANNED_PARTITION_NUM) + .add(SCANNED_TABLET_NUM).add(SCANNED_CARDINALITY).build(); + + protected final String ruleName; + + protected String sql; + + protected String sqlHash; + + protected Long partitionNum; + + protected Long tabletNum; + + protected Long cardinality; + + // whether effective global, default is false + protected Boolean global; + + // whether to use the rule, default is true + protected Boolean enable; + + protected final Map properties; + + /** + * constructor + */ + public SqlBlockRuleCommand(String ruleName, Map properties, PlanType planType) { + super(planType); + this.ruleName = ruleName; + this.properties = properties; + } + + private static void checkCommonProperties(Map properties) throws UserException { + if (properties == null || properties.isEmpty()) { + throw new AnalysisException("Not set properties"); + } + Optional optional = properties.keySet().stream().filter(entity -> !PROPERTIES_SET.contains(entity)) + .findFirst(); + if (optional.isPresent()) { + throw new AnalysisException(optional.get() + " is invalid property"); + } + } + + @Override + public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { + if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, "ADMIN"); + } + // check properties + checkCommonProperties(properties); + setProperties(properties); + doRun(ctx, executor); + } + + public abstract void doRun(ConnectContext ctx, StmtExecutor executor) throws Exception; + + public abstract void setProperties(Map properties) throws UserException; +} + diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index fec52fd395f9db..d386d097a93b49 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -21,6 +21,7 @@ import org.apache.doris.nereids.trees.plans.commands.AlterJobStatusCommand; import org.apache.doris.nereids.trees.plans.commands.AlterMTMVCommand; import org.apache.doris.nereids.trees.plans.commands.AlterRoleCommand; +import org.apache.doris.nereids.trees.plans.commands.AlterSqlBlockRuleCommand; import org.apache.doris.nereids.trees.plans.commands.AlterViewCommand; import org.apache.doris.nereids.trees.plans.commands.AlterWorkloadGroupCommand; import org.apache.doris.nereids.trees.plans.commands.AlterWorkloadPolicyCommand; @@ -36,6 +37,7 @@ import org.apache.doris.nereids.trees.plans.commands.CreateMTMVCommand; import org.apache.doris.nereids.trees.plans.commands.CreatePolicyCommand; import org.apache.doris.nereids.trees.plans.commands.CreateProcedureCommand; +import org.apache.doris.nereids.trees.plans.commands.CreateSqlBlockRuleCommand; import org.apache.doris.nereids.trees.plans.commands.CreateTableCommand; import org.apache.doris.nereids.trees.plans.commands.CreateTableLikeCommand; import org.apache.doris.nereids.trees.plans.commands.CreateViewCommand; @@ -471,6 +473,14 @@ default R visitShowLoadProfileCommand(ShowLoadProfileCommand showLoadProfileComm return visitCommand(showLoadProfileCommand, context); } + default R visitAlterSqlBlockRuleCommand(AlterSqlBlockRuleCommand dropRoleCommand, C context) { + return visitCommand(dropRoleCommand, context); + } + + default R visitCreateSqlBlockRuleCommand(CreateSqlBlockRuleCommand dropRoleCommand, C context) { + return visitCommand(dropRoleCommand, context); + } + default R visitDropRoleCommand(DropRoleCommand dropRoleCommand, C context) { return visitCommand(dropRoleCommand, context); } diff --git a/regression-test/suites/sql_block_rule_p0/test_sql_block_rule.groovy b/regression-test/suites/sql_block_rule_p0/test_sql_block_rule.groovy index 001c64fbc0d0ad..f90b89dfb6e5a7 100644 --- a/regression-test/suites/sql_block_rule_p0/test_sql_block_rule.groovy +++ b/regression-test/suites/sql_block_rule_p0/test_sql_block_rule.groovy @@ -47,10 +47,7 @@ suite("test_sql_block_rule", "nonConcurrent") { """ sql """ INSERT INTO table_2 VALUES ('H220427011909850160918','2022-04-27 16:00:33'),('T220427400109910160949','2022-04-27 16:00:54'),('T220427400123770120058','2022-04-27 16:00:56'),('T220427400126530112854','2022-04-27 16:00:34'),('T220427400127160144672','2022-04-27 16:00:10'),('T220427400127900184511','2022-04-27 16:00:34'),('T220427400129940120380','2022-04-27 16:00:23'),('T220427400139720192986','2022-04-27 16:00:34'),('T220427400140260152375','2022-04-27 16:00:02'),('T220427400153170104281','2022-04-27 16:00:31'),('H220427011909800104411','2022-04-27 16:00:14'),('H220427011909870184823','2022-04-27 16:00:36'),('T220427400115770144416','2022-04-27 16:00:12'),('T220427400126390112736','2022-04-27 16:00:19'),('T220427400128350120717','2022-04-27 16:00:56'),('T220427400129680120838','2022-04-27 16:00:39'),('T220427400136970192083','2022-04-27 16:00:51'),('H220427011909770192580','2022-04-27 16:00:04'),('H220427011909820192943','2022-04-27 16:00:23'),('T220427400109110184990','2022-04-27 16:00:29'),('T220427400109930192249','2022-04-27 16:00:56'),('T220427400123050168464','2022-04-27 16:00:37'),('T220427400124330112931','2022-04-27 16:00:56'),('T220427400124430144718','2022-04-27 16:00:07'),('T220427400130570160488','2022-04-27 16:00:34'),('T220427400130610112671','2022-04-27 16:00:30'),('T220427400137600160704','2022-04-27 16:00:35'),('T220427400144590176969','2022-04-27 16:00:49'),('T220427400146320176530','2022-04-27 16:00:34'),('T220427601780480120027','2022-04-27 16:00:58');""" - sql """ - CREATE SQL_BLOCK_RULE if not exists test_rule_sql - PROPERTIES("sql"="SELECT \\\\* FROM table_2", "global"= "true", "enable"= "true") - """ + checkNereidsExecute("CREATE SQL_BLOCK_RULE if not exists test_rule_sql PROPERTIES(\"sql\"=\"SELECT \\\\* FROM table_2\", \"global\"= \"true\", \"enable\"= \"true\")") test { sql("SELECT * FROM table_2", false) @@ -66,10 +63,8 @@ suite("test_sql_block_rule", "nonConcurrent") { exception "sql match regex sql block rule: test_rule_sql" } - sql """ - ALTER SQL_BLOCK_RULE test_rule_sql PROPERTIES("enable"="false") - """ - + checkNereidsExecute("ALTER SQL_BLOCK_RULE test_rule_sql PROPERTIES(\"enable\"=\"false\")") + sql "SELECT * FROM table_2" sql """ From 7d7f7fec5b3ca0ae9d30e8fd798e11da96c077ee Mon Sep 17 00:00:00 2001 From: lihangyu Date: Mon, 2 Dec 2024 11:08:46 +0800 Subject: [PATCH 104/399] [Opt](SegmentIterator) clear and release iterators memory footprint in advance when EOF (#44768) Otherwise the memory will hold until BetaRowsetReader released --- be/src/olap/rowset/segment_v2/segment_iterator.cpp | 8 ++++++++ be/src/olap/rowset/segment_v2/segment_iterator.h | 2 ++ 2 files changed, 10 insertions(+) diff --git a/be/src/olap/rowset/segment_v2/segment_iterator.cpp b/be/src/olap/rowset/segment_v2/segment_iterator.cpp index 96b0bea2ae8212..4ee73547c117e9 100644 --- a/be/src/olap/rowset/segment_v2/segment_iterator.cpp +++ b/be/src/olap/rowset/segment_v2/segment_iterator.cpp @@ -1998,6 +1998,12 @@ Status SegmentIterator::copy_column_data_by_selector(vectorized::IColumn* input_ return input_col_ptr->filter_by_selector(sel_rowid_idx, select_size, output_col); } +void SegmentIterator::_clear_iterators() { + _column_iterators.clear(); + _bitmap_index_iterators.clear(); + _inverted_index_iterators.clear(); +} + Status SegmentIterator::_next_batch_internal(vectorized::Block* block) { bool is_mem_reuse = block->mem_reuse(); DCHECK(is_mem_reuse); @@ -2104,6 +2110,8 @@ Status SegmentIterator::_next_batch_internal(vectorized::Block* block) { } } block->clear_column_data(); + // clear and release iterators memory footprint in advance + _clear_iterators(); return Status::EndOfFile("no more data in segment"); } diff --git a/be/src/olap/rowset/segment_v2/segment_iterator.h b/be/src/olap/rowset/segment_v2/segment_iterator.h index 5588661302dd06..5b4c8f6d73d0cd 100644 --- a/be/src/olap/rowset/segment_v2/segment_iterator.h +++ b/be/src/olap/rowset/segment_v2/segment_iterator.h @@ -383,6 +383,8 @@ class SegmentIterator : public RowwiseIterator { void _calculate_expr_in_remaining_conjunct_root(); + void _clear_iterators(); + class BitmapRangeIterator; class BackwardBitmapRangeIterator; From 1abfd10c34934bd24b3b208975c3441751f0608d Mon Sep 17 00:00:00 2001 From: lihangyu Date: Mon, 2 Dec 2024 11:09:02 +0800 Subject: [PATCH 105/399] [Fix](Variant) fix some nested explode_variant_array bug and add more test (#44533) --- be/src/vec/exprs/table_function/vexplode.cpp | 40 +++++++++++--- be/src/vec/exprs/table_function/vexplode.h | 1 + .../array/function_array_contains_all.cpp | 5 +- .../functions/array/function_array_distance.h | 4 +- .../functions/array/function_array_utils.cpp | 13 ++++- .../functions/array/function_array_utils.h | 6 ++- regression-test/data/variant_p0/nested.out | 20 +++++++ .../suites/variant_p0/nested.groovy | 53 ++++++++++++++++++- 8 files changed, 128 insertions(+), 14 deletions(-) diff --git a/be/src/vec/exprs/table_function/vexplode.cpp b/be/src/vec/exprs/table_function/vexplode.cpp index feef58cd27726c..5fa378f63517ec 100644 --- a/be/src/vec/exprs/table_function/vexplode.cpp +++ b/be/src/vec/exprs/table_function/vexplode.cpp @@ -23,12 +23,17 @@ #include "common/status.h" #include "vec/columns/column.h" +#include "vec/columns/column_array.h" +#include "vec/columns/column_nothing.h" #include "vec/columns/column_object.h" #include "vec/core/block.h" #include "vec/core/column_with_type_and_name.h" #include "vec/data_types/data_type.h" +#include "vec/data_types/data_type_array.h" +#include "vec/data_types/data_type_nothing.h" #include "vec/exprs/vexpr.h" #include "vec/exprs/vexpr_context.h" +#include "vec/functions/function_helpers.h" namespace doris::vectorized { #include "common/compile_check_begin.h" @@ -37,6 +42,34 @@ VExplodeTableFunction::VExplodeTableFunction() { _fn_name = "vexplode"; } +Status VExplodeTableFunction::_process_init_variant(Block* block, int value_column_idx) { + // explode variant array + const auto& variant_column = check_and_get_column( + remove_nullable(block->get_by_position(value_column_idx) + .column->convert_to_full_column_if_const()) + .get()); + _detail.output_as_variant = true; + if (!variant_column->is_null_root()) { + _array_column = variant_column->get_root(); + // We need to wrap the output nested column within a variant column. + // Otherwise the type is missmatched + const auto* array_type = check_and_get_data_type( + remove_nullable(variant_column->get_root_type()).get()); + if (array_type == nullptr) { + return Status::NotSupported("explode not support none array type {}", + variant_column->get_root_type()->get_name()); + } + _detail.nested_type = array_type->get_nested_type(); + } else { + // null root, use nothing type + _array_column = ColumnNullable::create(ColumnArray::create(ColumnNothing::create(0)), + ColumnUInt8::create(0)); + _array_column->assume_mutable()->insert_many_defaults(variant_column->size()); + _detail.nested_type = std::make_shared(); + } + return Status::OK(); +} + Status VExplodeTableFunction::process_init(Block* block, RuntimeState* state) { CHECK(_expr_context->root()->children().size() == 1) << "VExplodeTableFunction only support 1 child but has " @@ -47,12 +80,7 @@ Status VExplodeTableFunction::process_init(Block* block, RuntimeState* state) { &value_column_idx)); if (WhichDataType(remove_nullable(block->get_by_position(value_column_idx).type)) .is_variant_type()) { - // explode variant array - const auto& variant_column = check_and_get_column( - remove_nullable(block->get_by_position(value_column_idx) - .column->convert_to_full_column_if_const()) - .get()); - _array_column = variant_column->get_root(); + RETURN_IF_ERROR(_process_init_variant(block, value_column_idx)); } else { _array_column = block->get_by_position(value_column_idx).column->convert_to_full_column_if_const(); diff --git a/be/src/vec/exprs/table_function/vexplode.h b/be/src/vec/exprs/table_function/vexplode.h index 17b67d07824fa0..7b53926ae2c31e 100644 --- a/be/src/vec/exprs/table_function/vexplode.h +++ b/be/src/vec/exprs/table_function/vexplode.h @@ -47,6 +47,7 @@ class VExplodeTableFunction : public TableFunction { int get_value(MutableColumnPtr& column, int max_step) override; private: + Status _process_init_variant(Block* block, int value_column_idx); ColumnPtr _array_column; ColumnArrayExecutionData _detail; size_t _array_offset; // start offset of array[row_idx] diff --git a/be/src/vec/functions/array/function_array_contains_all.cpp b/be/src/vec/functions/array/function_array_contains_all.cpp index 67279babd5f49b..c65ec57e3d6572 100644 --- a/be/src/vec/functions/array/function_array_contains_all.cpp +++ b/be/src/vec/functions/array/function_array_contains_all.cpp @@ -231,8 +231,9 @@ class FunctionArrayContainsAll : public IFunction { is_equal_value = false; } else { // all is not null, check the data is equal - const auto* left_column = assert_cast(left_data.nested_col); - const auto* right_column = assert_cast(right_data.nested_col); + const auto* left_column = assert_cast(left_data.nested_col.get()); + const auto* right_column = + assert_cast(right_data.nested_col.get()); auto res = left_column->compare_at(left_nested_loop_pos, right_pos, *right_column, -1); is_equal_value = (res == 0); diff --git a/be/src/vec/functions/array/function_array_distance.h b/be/src/vec/functions/array/function_array_distance.h index e03e52a0ce191f..0984479ce3404b 100644 --- a/be/src/vec/functions/array/function_array_distance.h +++ b/be/src/vec/functions/array/function_array_distance.h @@ -128,8 +128,8 @@ class FunctionArrayDistance : public IFunction { const auto& offsets1 = *arr1.offsets_ptr; const auto& offsets2 = *arr2.offsets_ptr; - const auto& nested_col1 = assert_cast(arr1.nested_col); - const auto& nested_col2 = assert_cast(arr2.nested_col); + const auto& nested_col1 = assert_cast(arr1.nested_col.get()); + const auto& nested_col2 = assert_cast(arr2.nested_col.get()); for (ssize_t row = 0; row < offsets1.size(); ++row) { if (arr1.array_nullmap_data && arr1.array_nullmap_data[row]) { dst_null_data[row] = true; diff --git a/be/src/vec/functions/array/function_array_utils.cpp b/be/src/vec/functions/array/function_array_utils.cpp index ab999aa21ccdd1..d25904baf931f4 100644 --- a/be/src/vec/functions/array/function_array_utils.cpp +++ b/be/src/vec/functions/array/function_array_utils.cpp @@ -24,7 +24,9 @@ #include "vec/columns/column.h" #include "vec/columns/column_nullable.h" +#include "vec/columns/column_object.h" #include "vec/columns/column_vector.h" +#include "vec/data_types/data_type.h" namespace doris::vectorized { @@ -45,12 +47,19 @@ bool extract_column_array_info(const IColumn& src, ColumnArrayExecutionData& dat // extract array offsets and nested column data.offsets_ptr = &data.array_col->get_offsets(); - data.nested_col = &data.array_col->get_data(); + data.nested_col = data.array_col->get_data_ptr(); // extract nested column is nullable if (data.nested_col->is_nullable()) { const auto& nested_null_col = reinterpret_cast(*data.nested_col); data.nested_nullmap_data = nested_null_col.get_null_map_data().data(); - data.nested_col = nested_null_col.get_nested_column_ptr().get(); + data.nested_col = nested_null_col.get_nested_column_ptr(); + } + if (data.output_as_variant && + !WhichDataType(remove_nullable(data.nested_type)).is_variant_type()) { + // set variant root column/type to from column/type + auto variant = ColumnObject::create(true /*always nullable*/); + variant->create_root(data.nested_type, make_nullable(data.nested_col)->assume_mutable()); + data.nested_col = variant->get_ptr(); } return true; } diff --git a/be/src/vec/functions/array/function_array_utils.h b/be/src/vec/functions/array/function_array_utils.h index dc522ed813760a..36217c02452214 100644 --- a/be/src/vec/functions/array/function_array_utils.h +++ b/be/src/vec/functions/array/function_array_utils.h @@ -16,6 +16,7 @@ // under the License. #pragma once +#include "vec/columns/column.h" #include "vec/columns/column_array.h" #include "vec/columns/column_nullable.h" #include "vec/columns/columns_number.h" @@ -55,7 +56,10 @@ struct ColumnArrayExecutionData { const ColumnArray* array_col = nullptr; const ColumnArray::Offsets64* offsets_ptr = nullptr; const UInt8* nested_nullmap_data = nullptr; - const IColumn* nested_col = nullptr; + ColumnPtr nested_col = nullptr; + DataTypePtr nested_type = nullptr; + // wrap the nested column as variant column + bool output_as_variant = false; ColumnArrayMutableData to_mutable_data() const { ColumnArrayMutableData dst; diff --git a/regression-test/data/variant_p0/nested.out b/regression-test/data/variant_p0/nested.out index 2c105a68778a0d..d0e0e9c822ddc2 100644 --- a/regression-test/data/variant_p0/nested.out +++ b/regression-test/data/variant_p0/nested.out @@ -174,3 +174,23 @@ v.xx tinyint Yes false \N NONE 1 {"callLimit":3,"number":"02124713252","type":"HOME"} 1 {"callLimit":5,"number":"5550219210","type":"GSM"} +-- !sql -- +2 {"nested":[{"ba":"11111"},{"a":"1111"},{"axxxb":100,"xxxy111":111},{"aaa":"11","ddsss":1024},{"xx":10}]} +4 {"nested":[{"baaa":"11111"},{"ax1111":"1111"},{"axxxb":100,"xxxy111":111},{"aaa":"11","ddsss":1024},{"xx":10}]} +5 {"nested":[{"ba":"11111"},{"a":"1111"},{"axxxb":100,"xxxy111":111},{"aaa":"11","ddsss":1024},{"xx":10}]} +6 {"nested":[{"mmm":"11111"},{"ax1111":"1111"},{"axxxb":100,"xxxy111":111},{"aaa":"11","ddsss":1024},{"xx":10}]} +7 {"nested":[{"ba":"11111"},{"a":"1111"},{"axxxb":100,"xxxy111":111},{"aaa":"11","ddsss":1024},{"xx":10}]} +8 {"nested":[{"yyy":"11111"},{"ax1111":"1111"},{"axxxb":100,"xxxy111":111},{"aaa":"11","ddsss":1024},{"xx":10}]} +9 {"nested":[{"yyy":"11111"},{"ax1111":"1111"},{"axxxb":100,"xxxy111":111},{"aaa":"11","ddsss":1024},{"xx":10}]} +11 {"nested":[{"yyy":"11111"},{"ax1111":"1111"},{"axxxb":100,"xxxy111":111},{"aaa":"11","ddsss":1024},{"xx":10}]} +12 {"nested":[{"yyy":"11111"},{"ax1111":"1111"},{"axxxb":100,"xxxy111":111},{"aaa":"11","ddsss":1024},{"xx":10}]} +13 {"nested":[{"yyy":"11111"},{"ax1111":"1111"},{"axxxb":100,"xxxy111":111},{"aaa":"11","ddsss":1024},{"xx":10}]} + +-- !explode_sql -- + +-- !explode_sql -- +19 10 + +-- !explode_sql -- +2 10 + diff --git a/regression-test/suites/variant_p0/nested.groovy b/regression-test/suites/variant_p0/nested.groovy index 90728df2532668..25bd682d43aca4 100644 --- a/regression-test/suites/variant_p0/nested.groovy +++ b/regression-test/suites/variant_p0/nested.groovy @@ -164,13 +164,64 @@ parallel_pipeline_task_num=7,parallel_fragment_exec_instance_num=4,profile_level sql """insert into var_nested2 select * from var_nested order by k limit 1024""" qt_sql """select /*+SET_VAR(batch_size=4064,broker_load_batch_size=16352,disable_streaming_preaggregations=true,enable_distinct_streaming_aggregation=true,parallel_fragment_exec_instance_num=5,parallel_pipeline_task_num=1,profile_level=1,enable_pipeline_engine=false,enable_parallel_scan=true,parallel_scan_max_scanners_count=48,parallel_scan_min_rows_per_scanner=16384,enable_fold_constant_by_be=true,enable_rewrite_element_at_to_slot=true,runtime_filter_type=12,enable_parallel_result_sink=false,enable_nereids_planner=true,rewrite_or_to_in_predicate_threshold=2,enable_function_pushdown=true,enable_common_expr_pushdown=false,enable_local_exchange=false,partitioned_hash_join_rows_threshold=1048576,partitioned_hash_agg_rows_threshold=8,partition_pruning_expand_threshold=10,enable_share_hash_table_for_broadcast_join=false,enable_two_phase_read_opt=true,enable_common_expr_pushdown_for_inverted_index=true,enable_delete_sub_predicate_v2=true,min_revocable_mem=33554432,fetch_remote_schema_timeout_seconds=120,max_fetch_remote_schema_tablet_count=512,enable_join_spill=false,enable_sort_spill=false,enable_agg_spill=false,enable_force_spill=false,data_queue_max_blocks=1,spill_streaming_agg_mem_limit=268435456,external_agg_partition_bits=5) */ * from var_nested2 order by k limit 10;""" qt_sql """select v['nested'] from var_nested2 where k < 10 order by k limit 10;""" - // explode variant array + // 0. nomal explode variant array order_qt_explode_sql """select count(),cast(vv['xx'] as int) from var_nested lateral view explode_variant_array(v['nested']) tmp as vv where vv['xx'] = 10 group by cast(vv['xx'] as int)""" sql """truncate table var_nested2""" sql """insert into var_nested2 values(1119111, '{"eventId":1,"firstName":"Name1","lastName":"Surname1","body":{"phoneNumbers":[{"number":"5550219210","type":"GSM","callLimit":5},{"number":"02124713252","type":"HOME","callLimit":3},{"number":"05550219211","callLimit":2,"type":"WORK"}]}} ')""" order_qt_explode_sql """select v['eventId'], phone_numbers from var_nested2 lateral view explode_variant_array(v['body']['phoneNumbers']) tmp1 as phone_numbers where phone_numbers['type'] = 'GSM' OR phone_numbers['type'] = 'HOME' and phone_numbers['callLimit'] > 2;""" + + // test array_function + sql "DROP TABLE IF EXISTS var_nested_array_agg" + sql """ + CREATE TABLE IF NOT EXISTS var_nested_array_agg( + k bigint, + v variant + ) + UNIQUE KEY(`k`) + DISTRIBUTED BY HASH(k) BUCKETS 1 + properties("replication_num" = "1", "disable_auto_compaction" = "false", "enable_unique_key_merge_on_write" = "true", "variant_enable_flatten_nested" = "true"); + """ + sql "insert into var_nested_array_agg select * from var_nested" + // 1. array_contains + qt_sql "select * from var_nested_array_agg where array_contains(cast(v['nested']['xx'] as array), 10) order by k limit 10" + // 2. array_agg scalar + sql "select k, array_agg(cast(v['nested'] as text)) from var_nested_array_agg group by k limit 10" + + // test explode_variant_array with abonomal case + sql "DROP TABLE IF EXISTS var_nested_explode_variant_with_abnomal" + sql """ + CREATE TABLE IF NOT EXISTS var_nested_explode_variant_with_abnomal( + k bigint, + v variant + ) + UNIQUE KEY(`k`) + DISTRIBUTED BY HASH(k) BUCKETS 1 + properties("replication_num" = "1", "disable_auto_compaction" = "false", "enable_unique_key_merge_on_write" = "true", "variant_enable_flatten_nested" = "true"); + """ + sql "insert into var_nested_explode_variant_with_abnomal select * from var_nested" + // 1. v['nested']['x'] is null root + order_qt_explode_sql """select count(),cast(vv as int) from var_nested_explode_variant_with_abnomal lateral view explode_variant_array(v['nested']['x']) tmp as vv where vv = 10 group by cast(vv as int)""" + // 2. v['nested']['xx'] is normal array + order_qt_explode_sql """select count(),cast(vv as int) from var_nested_explode_variant_with_abnomal lateral view explode_variant_array(v['nested']['xx']) tmp as vv where vv = 10 group by cast(vv as int)""" + // 3. v['xx'] is none array scalar type + test { + sql """select count(),cast(vv as int) from var_nested_explode_variant_with_abnomal lateral view explode_variant_array(v['xx']) tmp as vv where vv = 10 group by cast(vv as int)""" + exception("explode not support none array type") + } + // 4. v['k1'] is json scalar type + test { + sql """select count(),cast(vv as int) from var_nested_explode_variant_with_abnomal lateral view explode_variant_array(v['k1']) tmp as vv where vv = 10 group by cast(vv as int)""" + exception("explode not support none array type") + } + // 5. toplevel nested array + sql "truncate table var_nested_explode_variant_with_abnomal" + sql """insert into var_nested_explode_variant_with_abnomal values(1, '[{"a" : 10}, {"b" : "20", "c" :1024, "a" : 11}]')""" + sql """insert into var_nested_explode_variant_with_abnomal values(2, '[{"a" : 10}, {"b" : "20", "a" : 150}]')""" + order_qt_explode_sql """select count(),cast(vv as int) from var_nested_explode_variant_with_abnomal lateral view explode_variant_array(v['a']) tmp as vv where vv = 10 group by cast(vv as int)""" + // FIXME after refator + // order_qt_explode_sql """select count(),cast(vv as int) from var_nested_explode_variant_with_abnomal lateral view explode_variant_array(v) tmp as vv where vv['a'] = 10 group by cast(vv as int)""" } finally { // reset flags } From 7526b9c3a3e5f4e477cda5615d7943ed0d34f11a Mon Sep 17 00:00:00 2001 From: Sun Chenyang Date: Mon, 2 Dec 2024 11:27:17 +0800 Subject: [PATCH 106/399] [fix](inverted index) add a sequence number to an enum variable (#44605) Related PR: #36757 Problem Summary: add a sequence number to an enum variable --- gensrc/thrift/Types.thrift | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/gensrc/thrift/Types.thrift b/gensrc/thrift/Types.thrift index 235c1cb28378e2..1912f950587dbf 100644 --- a/gensrc/thrift/Types.thrift +++ b/gensrc/thrift/Types.thrift @@ -122,9 +122,9 @@ enum TStorageBackendType { // This enum is used to distinguish between different organizational methods // of inverted index data, affecting how the index is stored and accessed. enum TInvertedIndexFileStorageFormat { - DEFAULT, // Default format, unspecified storage method. - V1, // Index per idx: Each index is stored separately based on its identifier. - V2 // Segment id per idx: Indexes are organized based on segment identifiers, grouping indexes by their associated segment. + DEFAULT = 0, // Default format, unspecified storage method. + V1 = 1, // Index per idx: Each index is stored separately based on its identifier. + V2 = 2 // Segment id per idx: Indexes are organized based on segment identifiers, grouping indexes by their associated segment. } struct TScalarType { From ed4f7fb5995b9726bcf05430ecdabd77ff6cbd21 Mon Sep 17 00:00:00 2001 From: feiniaofeiafei Date: Mon, 2 Dec 2024 12:00:06 +0800 Subject: [PATCH 107/399] [feat](nereids) add rewrite rule :EliminateGroupByKeyByUniform (#43391) This PR introduces two main changes: 1. Adds an optional constant value to the uniform attribute in DataTrait. A slot with a constant value that is not null will be considered uniform and not null. 2. Introduces a new transform rule: EliminateGroupByKeyByUniform, which utilizes the newly added part of the uniform attribute. Following is example transformation: from +--aggregate(group by a,b output a,b,max(c)) (a is uniform and not null: e.g. a is projection 2 as a in logicalProject) to +--aggregate(group by b output b,any_value(a) as a,max(c)) --- .../doris/nereids/jobs/executor/Rewriter.java | 2 + .../doris/nereids/properties/DataTrait.java | 211 +++++++++++-- .../apache/doris/nereids/rules/RuleType.java | 7 + .../rules/expression/ExpressionRewrite.java | 2 +- .../rewrite/EliminateGroupByKeyByUniform.java | 148 +++++++++ .../nereids/rules/rewrite/ExprIdRewriter.java | 284 ++++++++++++++++++ .../plans/commands/info/CreateMTMVInfo.java | 3 +- .../trees/plans/logical/LogicalFilter.java | 7 +- .../trees/plans/logical/LogicalHaving.java | 7 +- .../trees/plans/logical/LogicalJoin.java | 34 ++- .../trees/plans/logical/LogicalProject.java | 14 +- .../trees/plans/logical/LogicalQualify.java | 7 +- .../doris/nereids/util/ExpressionUtils.java | 11 +- .../mv/MaterializedViewUtilsTest.java | 2 +- .../EliminateGroupByKeyByUniformTest.java | 250 +++++++++++++++ .../rewrite/EliminateGroupByKeyTest.java | 4 +- .../eliminate_group_by_key_by_uniform.out | 269 +++++++++++++++++ .../eliminate_group_by_key_by_uniform.groovy | 221 ++++++++++++++ .../aggregate_without_roll_up.groovy | 6 +- 19 files changed, 1428 insertions(+), 61 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateGroupByKeyByUniform.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/ExprIdRewriter.java create mode 100644 fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/EliminateGroupByKeyByUniformTest.java create mode 100644 regression-test/data/nereids_rules_p0/eliminate_gby_key/eliminate_group_by_key_by_uniform.out create mode 100644 regression-test/suites/nereids_rules_p0/eliminate_gby_key/eliminate_group_by_key_by_uniform.groovy diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java index f4ca9a972a6814..fa00a4e9343325 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java @@ -64,6 +64,7 @@ import org.apache.doris.nereids.rules.rewrite.EliminateFilter; import org.apache.doris.nereids.rules.rewrite.EliminateGroupBy; import org.apache.doris.nereids.rules.rewrite.EliminateGroupByKey; +import org.apache.doris.nereids.rules.rewrite.EliminateGroupByKeyByUniform; import org.apache.doris.nereids.rules.rewrite.EliminateJoinByFK; import org.apache.doris.nereids.rules.rewrite.EliminateJoinByUnique; import org.apache.doris.nereids.rules.rewrite.EliminateJoinCondition; @@ -356,6 +357,7 @@ public class Rewriter extends AbstractBatchJobExecutor { topDown(new EliminateJoinByUnique()) ), topic("eliminate Aggregate according to fd items", + custom(RuleType.ELIMINATE_GROUP_BY_KEY_BY_UNIFORM, EliminateGroupByKeyByUniform::new), topDown(new EliminateGroupByKey()), topDown(new PushDownAggThroughJoinOnPkFk()), topDown(new PullUpJoinFromUnionAll()) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DataTrait.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DataTrait.java index e97fad6f479047..ff4756979e450e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DataTrait.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DataTrait.java @@ -17,18 +17,23 @@ package org.apache.doris.nereids.properties; +import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.functions.ExpressionTrait; +import org.apache.doris.nereids.trees.expressions.literal.NullLiteral; import org.apache.doris.nereids.util.ImmutableEqualSet; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Sets; import java.util.ArrayList; import java.util.HashSet; import java.util.Iterator; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; @@ -46,16 +51,16 @@ public class DataTrait { public static final DataTrait EMPTY_TRAIT - = new DataTrait(new NestedSet().toImmutable(), - new NestedSet().toImmutable(), new ImmutableSet.Builder().build(), + = new DataTrait(new UniqueDescription().toImmutable(), + new UniformDescription().toImmutable(), new ImmutableSet.Builder().build(), ImmutableEqualSet.empty(), new FuncDepsDG.Builder().build()); - private final NestedSet uniqueSet; - private final NestedSet uniformSet; + private final UniqueDescription uniqueSet; + private final UniformDescription uniformSet; private final ImmutableSet fdItems; private final ImmutableEqualSet equalSet; private final FuncDepsDG fdDg; - private DataTrait(NestedSet uniqueSet, NestedSet uniformSet, ImmutableSet fdItems, + private DataTrait(UniqueDescription uniqueSet, UniformDescription uniformSet, ImmutableSet fdItems, ImmutableEqualSet equalSet, FuncDepsDG fdDg) { this.uniqueSet = uniqueSet; this.uniformSet = uniformSet; @@ -86,8 +91,7 @@ public boolean isUniform(Slot slot) { } public boolean isUniform(Set slotSet) { - return !slotSet.isEmpty() - && uniformSet.slots.containsAll(slotSet); + return uniformSet.contains(slotSet); } public boolean isUniqueAndNotNull(Slot slot) { @@ -102,11 +106,25 @@ public boolean isUniqueAndNotNull(Set slotSet) { } public boolean isUniformAndNotNull(Slot slot) { - return !slot.nullable() && isUniform(slot); + return uniformSet.isUniformAndNotNull(slot); } + /** isUniformAndNotNull for slot set */ public boolean isUniformAndNotNull(ImmutableSet slotSet) { - return slotSet.stream().noneMatch(Slot::nullable) && isUniform(slotSet); + for (Slot slot : slotSet) { + if (!uniformSet.isUniformAndNotNull(slot)) { + return false; + } + } + return true; + } + + public boolean isUniformAndHasConstValue(Slot slot) { + return uniformSet.isUniformAndHasConstValue(slot); + } + + public Optional getUniformValue(Slot slot) { + return uniformSet.slotUniformValue.get(slot); } public boolean isNullSafeEqual(Slot l, Slot r) { @@ -143,23 +161,23 @@ public String toString() { * Builder of trait */ public static class Builder { - private final NestedSet uniqueSet; - private final NestedSet uniformSet; + private final UniqueDescription uniqueSet; + private final UniformDescription uniformSet; private ImmutableSet fdItems; private final ImmutableEqualSet.Builder equalSetBuilder; private final FuncDepsDG.Builder fdDgBuilder; public Builder() { - uniqueSet = new NestedSet(); - uniformSet = new NestedSet(); + uniqueSet = new UniqueDescription(); + uniformSet = new UniformDescription(); fdItems = new ImmutableSet.Builder().build(); equalSetBuilder = new ImmutableEqualSet.Builder<>(); fdDgBuilder = new FuncDepsDG.Builder(); } public Builder(DataTrait other) { - this.uniformSet = new NestedSet(other.uniformSet); - this.uniqueSet = new NestedSet(other.uniqueSet); + this.uniformSet = new UniformDescription(other.uniformSet); + this.uniqueSet = new UniqueDescription(other.uniqueSet); this.fdItems = ImmutableSet.copyOf(other.fdItems); equalSetBuilder = new ImmutableEqualSet.Builder<>(other.equalSet); fdDgBuilder = new FuncDepsDG.Builder(other.fdDg); @@ -173,6 +191,14 @@ public void addUniformSlot(DataTrait dataTrait) { uniformSet.add(dataTrait.uniformSet); } + public void addUniformSlotForOuterJoinNullableSide(DataTrait dataTrait) { + uniformSet.addUniformSlotForOuterJoinNullableSide(dataTrait.uniformSet); + } + + public void addUniformSlotAndLiteral(Slot slot, Expression literal) { + uniformSet.add(slot, literal); + } + public void addUniqueSlot(Slot slot) { uniqueSet.add(slot); } @@ -261,8 +287,21 @@ public void addUniqueByEqualSet(Set equalSet) { * if there is a uniform slot in the equivalence set, then all slots of an equivalence set are uniform */ public void addUniformByEqualSet(Set equalSet) { - if (uniformSet.isIntersect(uniformSet.slots, equalSet)) { - uniformSet.slots.addAll(equalSet); + List intersectionList = uniformSet.slotUniformValue.keySet().stream() + .filter(equalSet::contains) + .collect(Collectors.toList()); + if (intersectionList.isEmpty()) { + return; + } + Expression expr = null; + for (Slot slot : intersectionList) { + if (uniformSet.slotUniformValue.get(slot).isPresent()) { + expr = uniformSet.slotUniformValue.get(slot).get(); + break; + } + } + for (Slot equal : equalSet) { + uniformSet.add(equal, expr); } } @@ -293,9 +332,11 @@ public List> getAllUniqueAndNotNull() { */ public List> getAllUniformAndNotNull() { List> res = new ArrayList<>(); - for (Slot s : uniformSet.slots) { - if (!s.nullable()) { - res.add(ImmutableSet.of(s)); + for (Map.Entry> entry : uniformSet.slotUniformValue.entrySet()) { + if (!entry.getKey().nullable()) { + res.add(ImmutableSet.of(entry.getKey())); + } else if (entry.getValue().isPresent() && !entry.getValue().get().nullable()) { + res.add(ImmutableSet.of(entry.getKey())); } } return res; @@ -338,21 +379,21 @@ public void replaceFuncDepsBy(Map replaceMap) { } } - static class NestedSet { + static class UniqueDescription { Set slots; Set> slotSets; - NestedSet() { + UniqueDescription() { slots = new HashSet<>(); slotSets = new HashSet<>(); } - NestedSet(NestedSet o) { + UniqueDescription(UniqueDescription o) { this.slots = new HashSet<>(o.slots); this.slotSets = new HashSet<>(o.slotSets); } - NestedSet(Set slots, Set> slotSets) { + UniqueDescription(Set slots, Set> slotSets) { this.slots = slots; this.slotSets = slotSets; } @@ -408,9 +449,9 @@ public void add(ImmutableSet slotSet) { slotSets.add(slotSet); } - public void add(NestedSet nestedSet) { - slots.addAll(nestedSet.slots); - slotSets.addAll(nestedSet.slotSets); + public void add(UniqueDescription uniqueDescription) { + slots.addAll(uniqueDescription.slots); + slotSets.addAll(uniqueDescription.slotSets); } public boolean isIntersect(Set set1, Set set2) { @@ -446,8 +487,120 @@ public void replace(Map replaceMap) { .collect(Collectors.toSet()); } - public NestedSet toImmutable() { - return new NestedSet(ImmutableSet.copyOf(slots), ImmutableSet.copyOf(slotSets)); + public UniqueDescription toImmutable() { + return new UniqueDescription(ImmutableSet.copyOf(slots), ImmutableSet.copyOf(slotSets)); + } + } + + static class UniformDescription { + // slot and its uniform expression(literal or const expression) + // some slot can get uniform values, others can not. + // e.g.select a from t where a=10 group by a, b; + // in LogicalAggregate, a UniformDescription with map {a : 10} can be obtained. + // which means a is uniform and the uniform value is 10. + Map> slotUniformValue; + + public UniformDescription() { + slotUniformValue = new LinkedHashMap<>(); + } + + public UniformDescription(UniformDescription ud) { + slotUniformValue = new LinkedHashMap<>(ud.slotUniformValue); + } + + public UniformDescription(Map> slotUniformValue) { + this.slotUniformValue = slotUniformValue; + } + + public UniformDescription toImmutable() { + return new UniformDescription(ImmutableMap.copyOf(slotUniformValue)); + } + + public boolean isEmpty() { + return slotUniformValue.isEmpty(); + } + + public boolean contains(Slot slot) { + return slotUniformValue.containsKey(slot); + } + + public boolean contains(Set slots) { + return !slots.isEmpty() && slotUniformValue.keySet().containsAll(slots); + } + + public void add(Slot slot) { + slotUniformValue.putIfAbsent(slot, Optional.empty()); + } + + public void add(Set slots) { + for (Slot s : slots) { + slotUniformValue.putIfAbsent(s, Optional.empty()); + } + } + + public void add(UniformDescription ud) { + slotUniformValue.putAll(ud.slotUniformValue); + for (Map.Entry> entry : ud.slotUniformValue.entrySet()) { + add(entry.getKey(), entry.getValue().orElse(null)); + } + } + + public void add(Slot slot, Expression literal) { + if (null == literal) { + slotUniformValue.putIfAbsent(slot, Optional.empty()); + } else { + slotUniformValue.put(slot, Optional.of(literal)); + } + } + + public void addUniformSlotForOuterJoinNullableSide(UniformDescription ud) { + for (Map.Entry> entry : ud.slotUniformValue.entrySet()) { + if ((!entry.getValue().isPresent() && entry.getKey().nullable()) + || (entry.getValue().isPresent() && entry.getValue().get() instanceof NullLiteral)) { + add(entry.getKey(), entry.getValue().orElse(null)); + } + } + } + + public void removeNotContain(Set slotSet) { + if (slotSet.isEmpty()) { + return; + } + Map> newSlotUniformValue = new LinkedHashMap<>(); + for (Map.Entry> entry : slotUniformValue.entrySet()) { + if (slotSet.contains(entry.getKey())) { + newSlotUniformValue.put(entry.getKey(), entry.getValue()); + } + } + this.slotUniformValue = newSlotUniformValue; + } + + public void replace(Map replaceMap) { + Map> newSlotUniformValue = new LinkedHashMap<>(); + for (Map.Entry> entry : slotUniformValue.entrySet()) { + Slot newKey = replaceMap.getOrDefault(entry.getKey(), entry.getKey()); + newSlotUniformValue.put(newKey, entry.getValue()); + } + slotUniformValue = newSlotUniformValue; + } + + // The current implementation logic is: if a slot key exists in map slotUniformValue, + // its value is present and is not nullable, + // or if a slot key exists in map slotUniformValue and the slot is not nullable + // it indicates that this slot is uniform and not null. + public boolean isUniformAndNotNull(Slot slot) { + return slotUniformValue.containsKey(slot) + && (!slot.nullable() || slotUniformValue.get(slot).isPresent() + && !slotUniformValue.get(slot).get().nullable()); + } + + public boolean isUniformAndHasConstValue(Slot slot) { + return slotUniformValue.containsKey(slot) && slotUniformValue.get(slot).isPresent(); + } + + @Override + public String toString() { + return "{" + slotUniformValue + "}"; } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java index beb8bd43655743..c81f56f85b6add 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java @@ -220,6 +220,12 @@ public enum RuleType { REWRITE_HAVING_EXPRESSION(RuleTypeClass.REWRITE), REWRITE_REPEAT_EXPRESSION(RuleTypeClass.REWRITE), REWRITE_OLAP_TABLE_SINK_EXPRESSION(RuleTypeClass.REWRITE), + REWRITE_SINK_EXPRESSION(RuleTypeClass.REWRITE), + REWRITE_WINDOW_EXPRESSION(RuleTypeClass.REWRITE), + REWRITE_SET_OPERATION_EXPRESSION(RuleTypeClass.REWRITE), + REWRITE_PARTITION_TOPN_EXPRESSION(RuleTypeClass.REWRITE), + REWRITE_QUALIFY_EXPRESSION(RuleTypeClass.REWRITE), + REWRITE_TOPN_EXPRESSION(RuleTypeClass.REWRITE), EXTRACT_FILTER_FROM_JOIN(RuleTypeClass.REWRITE), REORDER_JOIN(RuleTypeClass.REWRITE), MERGE_PERCENTILE_TO_ARRAY(RuleTypeClass.REWRITE), @@ -249,6 +255,7 @@ public enum RuleType { ELIMINATE_JOIN_BY_UK(RuleTypeClass.REWRITE), ELIMINATE_JOIN_BY_FK(RuleTypeClass.REWRITE), ELIMINATE_GROUP_BY_KEY(RuleTypeClass.REWRITE), + ELIMINATE_GROUP_BY_KEY_BY_UNIFORM(RuleTypeClass.REWRITE), ELIMINATE_FILTER_GROUP_BY_KEY(RuleTypeClass.REWRITE), ELIMINATE_DEDUP_JOIN_CONDITION(RuleTypeClass.REWRITE), ELIMINATE_NULL_AWARE_LEFT_ANTI_JOIN(RuleTypeClass.REWRITE), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRewrite.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRewrite.java index e79dd6705c0513..e5b74ee26bcb02 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRewrite.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRewrite.java @@ -56,7 +56,7 @@ * expression of plan rewrite rule. */ public class ExpressionRewrite implements RewriteRuleFactory { - private final ExpressionRuleExecutor rewriter; + protected final ExpressionRuleExecutor rewriter; public ExpressionRewrite(ExpressionRewriteRule... rules) { this.rewriter = new ExpressionRuleExecutor(ImmutableList.copyOf(rules)); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateGroupByKeyByUniform.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateGroupByKeyByUniform.java new file mode 100644 index 00000000000000..4cb39c2a9341ae --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateGroupByKeyByUniform.java @@ -0,0 +1,148 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.rules.rewrite; + +import org.apache.doris.nereids.jobs.JobContext; +import org.apache.doris.nereids.properties.DataTrait; +import org.apache.doris.nereids.trees.expressions.Alias; +import org.apache.doris.nereids.trees.expressions.CTEId; +import org.apache.doris.nereids.trees.expressions.ExprId; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction; +import org.apache.doris.nereids.trees.expressions.functions.agg.AnyValue; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; +import org.apache.doris.nereids.trees.plans.visitor.CustomRewriter; +import org.apache.doris.nereids.trees.plans.visitor.DefaultPlanRewriter; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +/** + * +--aggregate(group by a, b output a#0 ,b#1, max(c) as max(c)#2) + * (a is uniform and not null: e.g. a is projection 2 as a in logicalProject) + * -> + * +--aggregate(group by b output b#1, any_value(a#0) as a#3, max(c)#2) + * if output any_value(a#0) as a#0, the uniqueness of ExprId #0 is violated, because #0 is both any_value(a#0) and a#0 + * error will occurs in other module(e.g. mv rewrite). + * As a result, new aggregate outputs #3 instead of #0, but upper plan refer slot #0, + * therefore, all references to #0 in the upper plan need to be changed to #3. + * use ExprIdRewriter to do this ExprId rewrite, and use CustomRewriter to rewrite upward。 + * */ +public class EliminateGroupByKeyByUniform extends DefaultPlanRewriter> implements CustomRewriter { + private ExprIdRewriter exprIdReplacer; + + @Override + public Plan rewriteRoot(Plan plan, JobContext jobContext) { + Optional cteId = jobContext.getCascadesContext().getCurrentTree(); + if (cteId.isPresent()) { + return plan; + } + Map replaceMap = new HashMap<>(); + ExprIdRewriter.ReplaceRule replaceRule = new ExprIdRewriter.ReplaceRule(replaceMap); + exprIdReplacer = new ExprIdRewriter(replaceRule, jobContext); + return plan.accept(this, replaceMap); + } + + @Override + public Plan visit(Plan plan, Map replaceMap) { + plan = visitChildren(this, plan, replaceMap); + plan = exprIdReplacer.rewriteExpr(plan, replaceMap); + return plan; + } + + @Override + public Plan visitLogicalAggregate(LogicalAggregate aggregate, Map replaceMap) { + aggregate = visitChildren(this, aggregate, replaceMap); + aggregate = (LogicalAggregate) exprIdReplacer.rewriteExpr(aggregate, replaceMap); + + if (aggregate.getGroupByExpressions().isEmpty() || aggregate.getSourceRepeat().isPresent()) { + return aggregate; + } + DataTrait aggChildTrait = aggregate.child().getLogicalProperties().getTrait(); + // Get the Group by column of agg. If there is a uniform one, delete the group by key. + Set removedExpression = new LinkedHashSet<>(); + List newGroupBy = new ArrayList<>(); + for (Expression groupBy : aggregate.getGroupByExpressions()) { + if (!(groupBy instanceof Slot)) { + newGroupBy.add(groupBy); + continue; + } + if (aggChildTrait.isUniformAndNotNull((Slot) groupBy)) { + removedExpression.add(groupBy); + } else { + newGroupBy.add(groupBy); + } + } + if (removedExpression.isEmpty()) { + return aggregate; + } + // when newGroupBy is empty, need retain one expr in group by, otherwise the result may be wrong in empty table + if (newGroupBy.isEmpty()) { + Expression expr = removedExpression.iterator().next(); + newGroupBy.add(expr); + removedExpression.remove(expr); + } + if (removedExpression.isEmpty()) { + return aggregate; + } + List newOutputs = new ArrayList<>(); + // If this output appears in the removedExpression column, replace it with any_value + for (NamedExpression output : aggregate.getOutputExpressions()) { + if (output instanceof Slot) { + if (removedExpression.contains(output)) { + Alias alias = new Alias(new AnyValue(false, output), output.getName()); + newOutputs.add(alias); + replaceMap.put(output.getExprId(), alias.getExprId()); + } else { + newOutputs.add(output); + } + } else if (output instanceof Alias) { + if (removedExpression.contains(output.child(0))) { + newOutputs.add(new Alias( + new AnyValue(false, output.child(0)), output.getName())); + } else { + newOutputs.add(output); + } + } else { + newOutputs.add(output); + } + } + + // Adjust the order of this new output so that aggregate functions are placed at the back + // and non-aggregated functions are placed at the front. + List aggFuncs = new ArrayList<>(); + List orderOutput = new ArrayList<>(); + for (NamedExpression output : newOutputs) { + if (output.anyMatch(e -> e instanceof AggregateFunction)) { + aggFuncs.add(output); + } else { + orderOutput.add(output); + } + } + orderOutput.addAll(aggFuncs); + return aggregate.withGroupByAndOutput(newGroupBy, orderOutput); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/ExprIdRewriter.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/ExprIdRewriter.java new file mode 100644 index 00000000000000..60c9da4bc6eec5 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/ExprIdRewriter.java @@ -0,0 +1,284 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.rules.rewrite; + +import org.apache.doris.nereids.jobs.JobContext; +import org.apache.doris.nereids.pattern.MatchingContext; +import org.apache.doris.nereids.pattern.Pattern; +import org.apache.doris.nereids.properties.OrderKey; +import org.apache.doris.nereids.rules.Rule; +import org.apache.doris.nereids.rules.RuleType; +import org.apache.doris.nereids.rules.expression.ExpressionPatternMatcher; +import org.apache.doris.nereids.rules.expression.ExpressionPatternRuleFactory; +import org.apache.doris.nereids.rules.expression.ExpressionRewrite; +import org.apache.doris.nereids.rules.expression.ExpressionRewriteContext; +import org.apache.doris.nereids.rules.expression.ExpressionRuleExecutor; +import org.apache.doris.nereids.trees.expressions.ExprId; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.expressions.OrderExpression; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.logical.LogicalPartitionTopN; +import org.apache.doris.nereids.trees.plans.logical.LogicalSetOperation; +import org.apache.doris.nereids.trees.plans.logical.LogicalSink; +import org.apache.doris.nereids.trees.plans.logical.LogicalTopN; +import org.apache.doris.nereids.trees.plans.logical.LogicalWindow; + +import com.google.common.collect.ImmutableList; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +/** replace SlotReference ExprId in logical plans */ +public class ExprIdRewriter extends ExpressionRewrite { + private final List rules; + private final JobContext jobContext; + + public ExprIdRewriter(ReplaceRule replaceRule, JobContext jobContext) { + super(new ExpressionRuleExecutor(ImmutableList.of(bottomUp(replaceRule)))); + rules = buildRules(); + this.jobContext = jobContext; + } + + @Override + public List buildRules() { + ImmutableList.Builder builder = ImmutableList.builder(); + builder.addAll(super.buildRules()); + builder.addAll(ImmutableList.of( + new LogicalPartitionTopNExpressionRewrite().build(), + new LogicalTopNExpressionRewrite().build(), + new LogicalSetOperationRewrite().build(), + new LogicalWindowRewrite().build(), + new LogicalResultSinkRewrite().build(), + new LogicalFileSinkRewrite().build(), + new LogicalHiveTableSinkRewrite().build(), + new LogicalIcebergTableSinkRewrite().build(), + new LogicalJdbcTableSinkRewrite().build(), + new LogicalOlapTableSinkRewrite().build(), + new LogicalDeferMaterializeResultSinkRewrite().build() + )); + return builder.build(); + } + + /**rewriteExpr*/ + public Plan rewriteExpr(Plan plan, Map replaceMap) { + if (replaceMap.isEmpty()) { + return plan; + } + for (Rule rule : rules) { + Pattern pattern = (Pattern) rule.getPattern(); + if (pattern.matchPlanTree(plan)) { + List newPlans = rule.transform(plan, jobContext.getCascadesContext()); + Plan newPlan = newPlans.get(0); + if (!newPlan.deepEquals(plan)) { + return newPlan; + } + } + } + return plan; + } + + /** + * Iteratively rewrites IDs using the replaceMap: + * 1. For a given SlotReference with initial ID, retrieve the corresponding value ID from the replaceMap. + * 2. If the value ID exists within the replaceMap, continue the lookup process using the value ID + * until it no longer appears in the replaceMap. + * 3. return SlotReference final value ID as the result of the rewrite. + * e.g. replaceMap:{0:3, 1:6, 6:7} + * SlotReference:a#0 -> a#3, a#1 -> a#7 + * */ + public static class ReplaceRule implements ExpressionPatternRuleFactory { + private final Map replaceMap; + + public ReplaceRule(Map replaceMap) { + this.replaceMap = replaceMap; + } + + @Override + public List> buildRules() { + return ImmutableList.of( + matchesType(SlotReference.class).thenApply(ctx -> { + Slot slot = ctx.expr; + if (replaceMap.containsKey(slot.getExprId())) { + ExprId newId = replaceMap.get(slot.getExprId()); + while (replaceMap.containsKey(newId)) { + newId = replaceMap.get(newId); + } + return slot.withExprId(newId); + } + return slot; + }) + ); + } + } + + private class LogicalResultSinkRewrite extends OneRewriteRuleFactory { + @Override + public Rule build() { + return logicalResultSink().thenApply(ExprIdRewriter.this::applyRewrite) + .toRule(RuleType.REWRITE_SINK_EXPRESSION); + } + } + + private class LogicalFileSinkRewrite extends OneRewriteRuleFactory { + @Override + public Rule build() { + return logicalFileSink().thenApply(ExprIdRewriter.this::applyRewrite) + .toRule(RuleType.REWRITE_SINK_EXPRESSION); + } + } + + private class LogicalHiveTableSinkRewrite extends OneRewriteRuleFactory { + @Override + public Rule build() { + return logicalHiveTableSink().thenApply(ExprIdRewriter.this::applyRewrite) + .toRule(RuleType.REWRITE_SINK_EXPRESSION); + } + } + + private class LogicalIcebergTableSinkRewrite extends OneRewriteRuleFactory { + @Override + public Rule build() { + return logicalIcebergTableSink().thenApply(ExprIdRewriter.this::applyRewrite) + .toRule(RuleType.REWRITE_SINK_EXPRESSION); + } + } + + private class LogicalJdbcTableSinkRewrite extends OneRewriteRuleFactory { + @Override + public Rule build() { + return logicalJdbcTableSink().thenApply(ExprIdRewriter.this::applyRewrite) + .toRule(RuleType.REWRITE_SINK_EXPRESSION); + } + } + + private class LogicalOlapTableSinkRewrite extends OneRewriteRuleFactory { + @Override + public Rule build() { + return logicalOlapTableSink().thenApply(ExprIdRewriter.this::applyRewrite) + .toRule(RuleType.REWRITE_SINK_EXPRESSION); + } + } + + private class LogicalDeferMaterializeResultSinkRewrite extends OneRewriteRuleFactory { + @Override + public Rule build() { + return logicalDeferMaterializeResultSink().thenApply(ExprIdRewriter.this::applyRewrite) + .toRule(RuleType.REWRITE_SINK_EXPRESSION); + } + } + + private class LogicalSetOperationRewrite extends OneRewriteRuleFactory { + @Override + public Rule build() { + return logicalSetOperation().thenApply(ctx -> { + LogicalSetOperation setOperation = ctx.root; + List> slotsList = setOperation.getRegularChildrenOutputs(); + List> newSlotsList = new ArrayList<>(); + ExpressionRewriteContext context = new ExpressionRewriteContext(ctx.cascadesContext); + for (List slots : slotsList) { + List newSlots = rewriteAll(slots, rewriter, context); + newSlotsList.add(newSlots); + } + if (newSlotsList.equals(slotsList)) { + return setOperation; + } + return setOperation.withChildrenAndTheirOutputs(setOperation.children(), newSlotsList); + }) + .toRule(RuleType.REWRITE_SET_OPERATION_EXPRESSION); + } + } + + private class LogicalWindowRewrite extends OneRewriteRuleFactory { + @Override + public Rule build() { + return logicalWindow().thenApply(ctx -> { + LogicalWindow window = ctx.root; + List windowExpressions = window.getWindowExpressions(); + ExpressionRewriteContext context = new ExpressionRewriteContext(ctx.cascadesContext); + List newWindowExpressions = rewriteAll(windowExpressions, rewriter, context); + if (newWindowExpressions.equals(windowExpressions)) { + return window; + } + return window.withExpressionsAndChild(newWindowExpressions, window.child()); + }) + .toRule(RuleType.REWRITE_WINDOW_EXPRESSION); + } + } + + private class LogicalTopNExpressionRewrite extends OneRewriteRuleFactory { + @Override + public Rule build() { + return logicalTopN().thenApply(ctx -> { + LogicalTopN topN = ctx.root; + List orderKeys = topN.getOrderKeys(); + ImmutableList.Builder rewrittenOrderKeys + = ImmutableList.builderWithExpectedSize(orderKeys.size()); + ExpressionRewriteContext context = new ExpressionRewriteContext(ctx.cascadesContext); + boolean changed = false; + for (OrderKey k : orderKeys) { + Expression expression = rewriter.rewrite(k.getExpr(), context); + changed |= expression != k.getExpr(); + rewrittenOrderKeys.add(new OrderKey(expression, k.isAsc(), k.isNullFirst())); + } + return changed ? topN.withOrderKeys(rewrittenOrderKeys.build()) : topN; + }).toRule(RuleType.REWRITE_TOPN_EXPRESSION); + } + } + + private class LogicalPartitionTopNExpressionRewrite extends OneRewriteRuleFactory { + @Override + public Rule build() { + return logicalPartitionTopN().thenApply(ctx -> { + LogicalPartitionTopN partitionTopN = ctx.root; + ExpressionRewriteContext context = new ExpressionRewriteContext(ctx.cascadesContext); + List newOrderExpressions = new ArrayList<>(); + boolean changed = false; + for (OrderExpression orderExpression : partitionTopN.getOrderKeys()) { + OrderKey orderKey = orderExpression.getOrderKey(); + Expression expr = rewriter.rewrite(orderKey.getExpr(), context); + changed |= expr != orderKey.getExpr(); + OrderKey newOrderKey = new OrderKey(expr, orderKey.isAsc(), orderKey.isNullFirst()); + newOrderExpressions.add(new OrderExpression(newOrderKey)); + } + List newPartitionKeys = rewriteAll(partitionTopN.getPartitionKeys(), rewriter, context); + if (!newPartitionKeys.equals(partitionTopN.getPartitionKeys())) { + changed = true; + } + if (!changed) { + return partitionTopN; + } + return partitionTopN.withPartitionKeysAndOrderKeys(newPartitionKeys, newOrderExpressions); + }).toRule(RuleType.REWRITE_PARTITION_TOPN_EXPRESSION); + } + } + + private LogicalSink applyRewrite(MatchingContext> ctx) { + LogicalSink sink = ctx.root; + ExpressionRewriteContext context = new ExpressionRewriteContext(ctx.cascadesContext); + List outputExprs = sink.getOutputExprs(); + List newOutputExprs = rewriteAll(outputExprs, rewriter, context); + if (outputExprs.equals(newOutputExprs)) { + return sink; + } + return sink.withOutputExprs(newOutputExprs); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java index 8c44b42a5ccfbd..5c6c71c92e8111 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java @@ -103,7 +103,8 @@ */ public class CreateMTMVInfo { public static final Logger LOG = LogManager.getLogger(CreateMTMVInfo.class); - public static final String MTMV_PLANER_DISABLE_RULES = "OLAP_SCAN_PARTITION_PRUNE,PRUNE_EMPTY_PARTITION"; + public static final String MTMV_PLANER_DISABLE_RULES = "OLAP_SCAN_PARTITION_PRUNE,PRUNE_EMPTY_PARTITION," + + "ELIMINATE_GROUP_BY_KEY_BY_UNIFORM"; private final boolean ifNotExists; private final TableNameInfo mvName; private List keys; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalFilter.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalFilter.java index d23ea3d2395f05..efd7e90c13615e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalFilter.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalFilter.java @@ -37,6 +37,7 @@ import java.util.Collection; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.Optional; import java.util.Set; @@ -154,9 +155,9 @@ public void computeUnique(Builder builder) { @Override public void computeUniform(Builder builder) { for (Expression e : getConjuncts()) { - Set uniformSlots = ExpressionUtils.extractUniformSlot(e); - for (Slot slot : uniformSlots) { - builder.addUniformSlot(slot); + Map uniformSlots = ExpressionUtils.extractUniformSlot(e); + for (Map.Entry entry : uniformSlots.entrySet()) { + builder.addUniformSlotAndLiteral(entry.getKey(), entry.getValue()); } } builder.addUniformSlot(child(0).getLogicalProperties().getTrait()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalHaving.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalHaving.java index f4f2178840b6ab..680988b39f6bb1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalHaving.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalHaving.java @@ -35,6 +35,7 @@ import com.google.common.collect.ImmutableSet; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.Optional; import java.util.Set; @@ -125,9 +126,9 @@ public void computeUnique(Builder builder) { @Override public void computeUniform(Builder builder) { for (Expression e : getConjuncts()) { - Set uniformSlots = ExpressionUtils.extractUniformSlot(e); - for (Slot slot : uniformSlots) { - builder.addUniformSlot(slot); + Map uniformSlots = ExpressionUtils.extractUniformSlot(e); + for (Map.Entry entry : uniformSlots.entrySet()) { + builder.addUniformSlotAndLiteral(entry.getKey(), entry.getValue()); } } builder.addUniformSlot(child(0).getLogicalProperties().getTrait()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalJoin.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalJoin.java index f557b07d3b646e..c583360c3d8a76 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalJoin.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalJoin.java @@ -535,11 +535,35 @@ public void computeUniform(Builder builder) { // TODO disable function dependence calculation for mark join, but need re-think this in future. return; } - if (!joinType.isLeftSemiOrAntiJoin()) { - builder.addUniformSlot(right().getLogicalProperties().getTrait()); - } - if (!joinType.isRightSemiOrAntiJoin()) { - builder.addUniformSlot(left().getLogicalProperties().getTrait()); + switch (joinType) { + case INNER_JOIN: + case CROSS_JOIN: + builder.addUniformSlot(left().getLogicalProperties().getTrait()); + builder.addUniformSlot(right().getLogicalProperties().getTrait()); + break; + case LEFT_SEMI_JOIN: + case LEFT_ANTI_JOIN: + case NULL_AWARE_LEFT_ANTI_JOIN: + builder.addUniformSlot(left().getLogicalProperties().getTrait()); + break; + case RIGHT_SEMI_JOIN: + case RIGHT_ANTI_JOIN: + builder.addUniformSlot(right().getLogicalProperties().getTrait()); + break; + case LEFT_OUTER_JOIN: + builder.addUniformSlot(left().getLogicalProperties().getTrait()); + builder.addUniformSlotForOuterJoinNullableSide(right().getLogicalProperties().getTrait()); + break; + case RIGHT_OUTER_JOIN: + builder.addUniformSlot(right().getLogicalProperties().getTrait()); + builder.addUniformSlotForOuterJoinNullableSide(left().getLogicalProperties().getTrait()); + break; + case FULL_OUTER_JOIN: + builder.addUniformSlotForOuterJoinNullableSide(left().getLogicalProperties().getTrait()); + builder.addUniformSlotForOuterJoinNullableSide(right().getLogicalProperties().getTrait()); + break; + default: + break; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalProject.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalProject.java index 171a355d8452d8..612231340f16ed 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalProject.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalProject.java @@ -238,14 +238,18 @@ public void computeUnique(DataTrait.Builder builder) { public void computeUniform(DataTrait.Builder builder) { builder.addUniformSlot(child(0).getLogicalProperties().getTrait()); for (NamedExpression proj : getProjects()) { - if (proj.children().isEmpty()) { + if (!(proj instanceof Alias)) { continue; } if (proj.child(0).isConstant()) { - builder.addUniformSlot(proj.toSlot()); - } else if (ExpressionUtils.isInjective(proj.child(0))) { - ImmutableSet inputs = ImmutableSet.copyOf(proj.getInputSlots()); - if (child(0).getLogicalProperties().getTrait().isUniform(inputs)) { + builder.addUniformSlotAndLiteral(proj.toSlot(), proj.child(0)); + } else if (proj.child(0) instanceof Slot) { + Slot slot = (Slot) proj.child(0); + DataTrait childTrait = child(0).getLogicalProperties().getTrait(); + if (childTrait.isUniformAndHasConstValue(slot)) { + builder.addUniformSlotAndLiteral(proj.toSlot(), + child(0).getLogicalProperties().getTrait().getUniformValue(slot).get()); + } else if (childTrait.isUniform(slot)) { builder.addUniformSlot(proj.toSlot()); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalQualify.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalQualify.java index ced6730dfb54a2..904c66f6482ee5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalQualify.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalQualify.java @@ -35,6 +35,7 @@ import com.google.common.collect.ImmutableSet; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.Optional; import java.util.Set; @@ -130,9 +131,9 @@ public void computeUnique(Builder builder) { @Override public void computeUniform(Builder builder) { for (Expression e : getConjuncts()) { - Set uniformSlots = ExpressionUtils.extractUniformSlot(e); - for (Slot slot : uniformSlots) { - builder.addUniformSlot(slot); + Map uniformSlots = ExpressionUtils.extractUniformSlot(e); + for (Map.Entry entry : uniformSlots.entrySet()) { + builder.addUniformSlotAndLiteral(entry.getKey(), entry.getValue()); } } builder.addUniformSlot(child(0).getLogicalProperties().getTrait()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java index e3909b87bd3726..53ab8a50683ecb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java @@ -73,6 +73,7 @@ import com.google.common.base.Predicate; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList.Builder; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; import com.google.common.collect.Maps; @@ -744,15 +745,15 @@ public static List collectToList(Collection express /** * extract uniform slot for the given predicate, such as a = 1 and b = 2 */ - public static ImmutableSet extractUniformSlot(Expression expression) { - ImmutableSet.Builder builder = new ImmutableSet.Builder<>(); + public static ImmutableMap extractUniformSlot(Expression expression) { + ImmutableMap.Builder builder = new ImmutableMap.Builder<>(); if (expression instanceof And) { - builder.addAll(extractUniformSlot(expression.child(0))); - builder.addAll(extractUniformSlot(expression.child(1))); + builder.putAll(extractUniformSlot(expression.child(0))); + builder.putAll(extractUniformSlot(expression.child(1))); } if (expression instanceof EqualTo) { if (isInjective(expression.child(0)) && expression.child(1).isConstant()) { - builder.add((Slot) expression.child(0)); + builder.put((Slot) expression.child(0), expression.child(1)); } } return builder.build(); diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtilsTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtilsTest.java index f824a40eda6474..45e1190412d0a4 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtilsTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtilsTest.java @@ -248,7 +248,7 @@ protected void runBeforeAll() throws Exception { + "\"replication_allocation\" = \"tag.location.default: 1\"\n" + ");\n"); // Should not make scan to empty relation when the table used by materialized view has no data - connectContext.getSessionVariable().setDisableNereidsRules("OLAP_SCAN_PARTITION_PRUNE,PRUNE_EMPTY_PARTITION"); + connectContext.getSessionVariable().setDisableNereidsRules("OLAP_SCAN_PARTITION_PRUNE,PRUNE_EMPTY_PARTITION,ELIMINATE_GROUP_BY_KEY_BY_UNIFORM"); } // Test when join both side are all partition table and partition column name is same diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/EliminateGroupByKeyByUniformTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/EliminateGroupByKeyByUniformTest.java new file mode 100644 index 00000000000000..78d8034e3fdfed --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/EliminateGroupByKeyByUniformTest.java @@ -0,0 +1,250 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.rules.rewrite; + +import org.apache.doris.nereids.util.MemoPatternMatchSupported; +import org.apache.doris.nereids.util.PlanChecker; +import org.apache.doris.utframe.TestWithFeService; + +import org.junit.jupiter.api.Test; + +public class EliminateGroupByKeyByUniformTest extends TestWithFeService implements MemoPatternMatchSupported { + @Override + protected void runBeforeAll() throws Exception { + createDatabase("test"); + createTable("create table test.eli_gbk_by_uniform_t(a int null, b int not null," + + "c varchar(10) null, d date, dt datetime)\n" + + "distributed by hash(a) properties('replication_num' = '1');"); + connectContext.setDatabase("test"); + connectContext.getSessionVariable().setDisableNereidsRules("PRUNE_EMPTY_PARTITION"); + } + + @Test + void testEliminateByFilter() { + PlanChecker.from(connectContext) + .analyze("select a, min(a), sum(a),b from eli_gbk_by_uniform_t where a = 1 group by a,b") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> + agg.getGroupByExpressions().size() == 1 + && agg.getGroupByExpressions().get(0).toSql().equals("b"))); + + } + + @Test + void testNotEliminateWhenOnlyOneGbyKey() { + PlanChecker.from(connectContext) + .analyze("select a, min(a), sum(a) from eli_gbk_by_uniform_t where a = 1 group by a") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> + agg.getGroupByExpressions().size() == 1 + && agg.getGroupByExpressions().get(0).toSql().equals("a"))); + + } + + @Test + void testEliminateByProjectConst() { + PlanChecker.from(connectContext) + .analyze("select sum(c1), c2 from (select a c1,1 c2, d c3 from eli_gbk_by_uniform_t) t group by c2,c3 ") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> + agg.getGroupByExpressions().size() == 1 + && agg.getGroupByExpressions().get(0).toSql().equals("c3"))); + } + + @Test + void testEliminateByProjectUniformSlot() { + PlanChecker.from(connectContext) + .analyze("select max(c3), c1,c2,c3 from (select a c1,1 c2, d c3 from eli_gbk_by_uniform_t where a=1) t group by c1,c2,c3") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> + agg.getGroupByExpressions().size() == 1 + && agg.getGroupByExpressions().get(0).toSql().equals("c3"))); + } + + @Test + void testEliminateDate() { + PlanChecker.from(connectContext) + .analyze("select d, min(a), sum(a), count(a) from eli_gbk_by_uniform_t where d = '2023-01-06' group by d,a") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> + agg.getGroupByExpressions().size() == 1 + && agg.getGroupByExpressions().get(0).toSql().equals("a"))); + } + + @Test + void testSaveOneExpr() { + PlanChecker.from(connectContext) + .analyze("select a, min(a), sum(a), count(a) from eli_gbk_by_uniform_t where a = 1 and b=100 group by a, b,'abc'") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> + agg.getGroupByExpressions().size() == 1 + && agg.getGroupByExpressions().get(0).toSql().equals("a"))); + } + + @Test + void testSaveOneExprProjectConst() { + PlanChecker.from(connectContext) + .analyze("select c2 from (select a c1,1 c2, 3 c3 from eli_gbk_by_uniform_t) t group by c2,c3 order by 1;") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> + agg.getGroupByExpressions().size() == 1 + && agg.getGroupByExpressions().get(0).toSql().equals("c2"))); + } + + @Test + void testNotRewriteWhenHasRepeat() { + PlanChecker.from(connectContext) + .analyze("select c2 from (select a c1,1 c2, 3 c3 from eli_gbk_by_uniform_t) t group by grouping sets((c2),(c3)) order by 1;") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> agg.getGroupByExpressions().size() == 3)); + } + + @Test + void testInnerJoin() { + PlanChecker.from(connectContext) + .analyze("select t1.b,t2.b from eli_gbk_by_uniform_t t1 inner join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t1.b=100 group by t1.b,t2.b,t2.c;") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> agg.getGroupByExpressions().size() == 1)); + } + + @Test + void testLeftJoinOnConditionNotRewrite() { + PlanChecker.from(connectContext) + .analyze("select t1.b,t2.b from eli_gbk_by_uniform_t t1 left join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t1.b=100 group by t1.b,t2.b,t2.c;") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> agg.getGroupByExpressions().size() == 3)); + } + + @Test + void testLeftJoinWhereConditionRewrite() { + PlanChecker.from(connectContext) + .analyze("select t1.b,t2.b from eli_gbk_by_uniform_t t1 left join eli_gbk_by_uniform_t t2 on t1.b=t2.b where t1.b=100 group by t1.b,t2.b,t2.c;") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> agg.getGroupByExpressions().size() == 2)); + } + + @Test + void testRightJoinOnConditionNullableSideFilterNotRewrite() { + PlanChecker.from(connectContext) + .analyze("select t1.b,t2.b from eli_gbk_by_uniform_t t1 right join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t1.b=100 group by t1.b,t2.b,t2.c;") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> agg.getGroupByExpressions().size() == 3)); + } + + @Test + void testRightJoinOnConditionNonNullableSideFilterNotRewrite() { + PlanChecker.from(connectContext) + .analyze("select t1.b,t2.b from eli_gbk_by_uniform_t t1 right join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t2.b=100 group by t1.b,t2.b,t2.c;") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> agg.getGroupByExpressions().size() == 3)); + } + + @Test + void testRightJoinWhereConditionToInnerRewrite() { + PlanChecker.from(connectContext) + .analyze("select t1.b,t2.b from eli_gbk_by_uniform_t t1 right join eli_gbk_by_uniform_t t2 on t1.b=t2.b where t1.b=100 group by t1.b,t2.b,t2.c;") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> agg.getGroupByExpressions().size() == 1)); + } + + @Test + void testLeftSemiJoinWhereConditionRewrite() { + PlanChecker.from(connectContext) + .analyze("select t1.b from eli_gbk_by_uniform_t t1 left semi join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t2.b=100 group by t1.b,t1.a") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> agg.getGroupByExpressions().size() == 1)); + } + + @Test + void testLeftSemiJoinRetainOneSlotInGroupBy() { + PlanChecker.from(connectContext) + .analyze("select t1.b from eli_gbk_by_uniform_t t1 left semi join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t2.b=100 group by t1.b") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> agg.getGroupByExpressions().size() == 1)); + } + + @Test + void testRightSemiJoinWhereConditionRewrite() { + PlanChecker.from(connectContext) + .analyze("select t2.b from eli_gbk_by_uniform_t t1 right semi join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t2.b=100 group by t2.b,t2.a") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> agg.getGroupByExpressions().size() == 1)); + } + + @Test + void testRightSemiJoinRetainOneSlotInGroupBy() { + PlanChecker.from(connectContext) + .analyze("select t2.b from eli_gbk_by_uniform_t t1 right semi join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t2.b=100 group by t2.b") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> agg.getGroupByExpressions().size() == 1)); + } + + @Test + void testLeftAntiJoinOnConditionNotRewrite() { + PlanChecker.from(connectContext) + .analyze("select t1.b from eli_gbk_by_uniform_t t1 left anti join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t1.b=100 group by t1.b,t1.a") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> agg.getGroupByExpressions().size() == 2)); + } + + @Test + void testLeftAntiJoinWhereConditionRewrite() { + PlanChecker.from(connectContext) + .analyze("select t1.b from eli_gbk_by_uniform_t t1 left anti join eli_gbk_by_uniform_t t2 on t1.b=t2.b where t1.b=100 group by t1.b,t1.c") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> agg.getGroupByExpressions().size() == 1)); + } + + @Test + void testRightAntiJoinOnConditionNotRewrite() { + PlanChecker.from(connectContext) + .analyze("select t2.b from eli_gbk_by_uniform_t t1 right anti join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t1.b=100 group by t2.b,t2.a") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> agg.getGroupByExpressions().size() == 2)); + } + + @Test + void testRightAntiJoinWhereConditionRewrite() { + PlanChecker.from(connectContext) + .analyze("select t2.b from eli_gbk_by_uniform_t t1 right anti join eli_gbk_by_uniform_t t2 on t1.b=t2.b where t2.b=100 group by t2.b,t2.c") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> agg.getGroupByExpressions().size() == 1)); + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/EliminateGroupByKeyTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/EliminateGroupByKeyTest.java index 5a9e15cf4774d1..103e074c73bfd5 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/EliminateGroupByKeyTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/EliminateGroupByKeyTest.java @@ -111,7 +111,7 @@ void testProjectAlias() { .rewrite() .printlnTree() .matches(logicalAggregate().when(agg -> - agg.getGroupByExpressions().size() == 2)); + agg.getGroupByExpressions().size() == 1)); PlanChecker.from(connectContext) .analyze("select id as c, name as n from t1 group by name, id") .rewrite() @@ -123,7 +123,7 @@ void testProjectAlias() { .rewrite() .printlnTree() .matches(logicalAggregate().when(agg -> - agg.getGroupByExpressions().size() == 2)); + agg.getGroupByExpressions().size() == 1)); } @Test diff --git a/regression-test/data/nereids_rules_p0/eliminate_gby_key/eliminate_group_by_key_by_uniform.out b/regression-test/data/nereids_rules_p0/eliminate_gby_key/eliminate_group_by_key_by_uniform.out new file mode 100644 index 00000000000000..94cebafa4e9aad --- /dev/null +++ b/regression-test/data/nereids_rules_p0/eliminate_gby_key/eliminate_group_by_key_by_uniform.out @@ -0,0 +1,269 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !empty_tranform_not_to_scalar_agg -- + +-- !empty_tranform_multi_column -- + +-- !empty_tranform_multi_column -- +1 1 2 2 + +-- !tranform_to_scalar_agg_not_null_column -- + +-- !project_const -- +\N 1 +\N 1 +2 1 +2 1 +4 1 +6 1 +6 1 +10 1 + +-- !project_slot_uniform -- +2023-01-01 1 1 2023-01-01 + +-- !upper_refer -- + +-- !upper_refer_varchar_alias -- +cherry 3 + +-- !upper_refer_date -- +2023-01-06 + +-- !upper_refer_datetime_not_to_scalar_agg -- +2023-01-06T15:00 + +-- !upper_refer_datetime -- +2023-01-06T15:00 + +-- !project_no_other_agg_func -- +1 +1 +1 +1 +1 +1 +1 +1 + +-- !project_const_not_to_scalar_agg_multi -- +1 + +-- !not_to_scalar_agg_multi -- +1 1 2 2 + +-- !conflict_equal_value -- + +-- !project_slot_uniform_confict_value -- + +-- !inner_join_left_has_filter -- +100 100 + +-- !inner_join_right_has_filter -- +100 100 + +-- !left_join_right_has_filter -- +100 100 +101 \N +102 \N +103 \N +104 \N +105 \N +106 \N +107 \N + +-- !left_join_left_has_filter -- +100 100 +101 \N +102 \N +103 \N +104 \N +105 \N +106 \N +107 \N + +-- !right_join_right_has_filter -- +\N 101 +\N 102 +\N 103 +\N 104 +\N 105 +\N 106 +\N 107 +100 100 + +-- !right_join_left_has_filter -- +\N 101 +\N 102 +\N 103 +\N 104 +\N 105 +\N 106 +\N 107 +100 100 + +-- !left_semi_join_right_has_filter -- +100 + +-- !left_semi_join_left_has_filter -- +100 + +-- !left_anti_join_right_has_on_filter -- +101 +102 +103 +104 +105 +106 +107 + +-- !left_anti_join_left_has_on_filter -- +101 +102 +103 +104 +105 +106 +107 + +-- !left_anti_join_left_has_where_filter -- + +-- !right_semi_join_right_has_filter -- +100 + +-- !right_semi_join_left_has_filter -- +100 + +-- !right_anti_join_right_has_on_filter -- +101 +102 +103 +104 +105 +106 +107 + +-- !right_anti_join_left_has_on_filter -- +101 +102 +103 +104 +105 +106 +107 + +-- !right_anti_join_right_has_where_filter -- + +-- !cross_join_left_has_filter -- +100 100 +100 101 +100 102 +100 103 +100 104 +100 105 +100 106 +100 107 + +-- !cross_join_right_has_filter -- +100 100 +101 100 +102 100 +103 100 +104 100 +105 100 +106 100 +107 100 + +-- !union -- +1 100 +5 105 + +-- !union_all -- +1 100 +1 100 +5 105 + +-- !intersect -- + +-- !except -- + +-- !set_op_mixed -- +1 100 + +-- !window -- + +-- !partition_topn -- + +-- !partition_topn_qualifiy -- + +-- !cte_producer -- +1 1 100 + +-- !cte_multi_producer -- + +-- !cte_consumer -- + +-- !filter -- +1 100 + +-- !topn -- +1 100 + +-- !sink -- +\N 103 date 2023-01-04 2023-01-04T13:00 +\N 107 grape 2023-01-08 2023-01-08T17:00 +1 100 apple 2023-01-01 2023-01-01T10:00 +1 100 apple 2023-01-01 2023-01-01T10:00 +1 100 apple 2023-01-01 2023-01-01T10:00 +2 101 banana 2023-01-02 2023-01-02T11:00 +3 102 cherry 2023-01-03 2023-01-03T12:00 +3 102 cherry 2023-01-03 2023-01-03T12:00 +4 104 elderberry 2023-01-05 2023-01-05T14:00 +5 105 \N 2023-01-06 2023-01-06T15:00 +5 105 \N 2023-01-06 2023-01-06T15:00 +6 106 fig 2023-01-07 2023-01-07T16:00 + +-- !nest_exprid_replace -- +2023-10-17 2 2023-10-17 2 6 +2023-10-17 2 2023-10-18 2 6 +2023-10-17 2 2023-10-21 2 6 +2023-10-18 2 2023-10-17 2 6 +2023-10-18 2 2023-10-18 2 6 +2023-10-18 2 2023-10-21 2 6 + +-- !full_join_uniform_should_not_eliminate_group_by_key -- +\N 1 +105 1 + +-- !full2 -- +1 \N +1 105 + +-- !left_join_right_side_should_not_eliminate_group_by_key -- +\N 1 +105 1 + +-- !left_join_left_side_should_eliminate_group_by_key -- +\N 1 +105 1 + +-- !right_join_left_side_should_not_eliminate_group_by_key -- +1 \N +1 105 + +-- !right_join_right_side_should_eliminate_group_by_key -- +1 \N +1 105 + +-- !left_semi_left_side -- +1 +1 + +-- !left_anti_left_side -- +1 + +-- !right_semi_right_side -- +105 +105 + +-- !right_anti_right_side -- + diff --git a/regression-test/suites/nereids_rules_p0/eliminate_gby_key/eliminate_group_by_key_by_uniform.groovy b/regression-test/suites/nereids_rules_p0/eliminate_gby_key/eliminate_group_by_key_by_uniform.groovy new file mode 100644 index 00000000000000..4375aac316d5d1 --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/eliminate_gby_key/eliminate_group_by_key_by_uniform.groovy @@ -0,0 +1,221 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. +suite("eliminate_group_by_key_by_uniform") { + sql "set enable_nereids_rules = 'ELIMINATE_GROUP_BY_KEY_BY_UNIFORM'" + sql "drop table if exists eli_gbk_by_uniform_t" + sql """create table eli_gbk_by_uniform_t(a int null, b int not null, c varchar(10) null, d date, dt datetime) + distributed by hash(a) properties("replication_num"="1"); + """ + qt_empty_tranform_not_to_scalar_agg "select a, min(a), sum(a), count(a) from eli_gbk_by_uniform_t where a = 1 group by a" + qt_empty_tranform_multi_column "select a,b, min(a), sum(a), count(a) from eli_gbk_by_uniform_t where a = 1 and b=2 group by a,b" + + sql """ + INSERT INTO eli_gbk_by_uniform_t (a, b, c, d, dt) VALUES + (1, 100, 'apple', '2023-01-01', '2023-01-01 10:00:00'), + (1, 100, 'apple', '2023-01-01', '2023-01-01 10:00:00'), + (2, 101, 'banana', '2023-01-02', '2023-01-02 11:00:00'), + (3, 102, 'cherry', '2023-01-03', '2023-01-03 12:00:00'), + (3, 102, 'cherry', '2023-01-03', '2023-01-03 12:00:00'), + (NULL, 103, 'date', '2023-01-04', '2023-01-04 13:00:00'), + (4, 104, 'elderberry', '2023-01-05', '2023-01-05 14:00:00'), + (5, 105, NULL, '2023-01-06', '2023-01-06 15:00:00'), + (5, 105, NULL, '2023-01-06', '2023-01-06 15:00:00'), + (6, 106, 'fig', '2023-01-07', '2023-01-07 16:00:00'), + (NULL, 107, 'grape', '2023-01-08', '2023-01-08 17:00:00'); + """ + qt_empty_tranform_multi_column "select a, min(a), sum(a), count(a) from eli_gbk_by_uniform_t where a = 1 group by a, b,'abc' order by 1,2,3,4" + qt_tranform_to_scalar_agg_not_null_column "select b, min(a), sum(a), count(a) from eli_gbk_by_uniform_t where b = 1 group by a, b order by 1,2,3,4" + + qt_project_const "select sum(c1), c2 from (select a c1,1 c2, d c3 from eli_gbk_by_uniform_t) t group by c2,c3 order by 1,2;" + qt_project_slot_uniform "select max(c3), c1,c2,c3 from (select a c1,1 c2, d c3 from eli_gbk_by_uniform_t where a=1) t group by c1,c2,c3 order by 1,2,3,4;" + + qt_upper_refer "select b from (select b, min(a), sum(a), count(a) from eli_gbk_by_uniform_t where b = 1 group by a, b) t order by b" + qt_upper_refer_varchar_alias "select c1,c2 from (select c as c1, min(a) c2, sum(a), count(a) from eli_gbk_by_uniform_t where c = 'cherry' group by a, b,c) t order by c1,c2" + qt_upper_refer_date "select d from (select d, min(a), sum(a), count(a) from eli_gbk_by_uniform_t where d = '2023-01-06' group by d,a) t order by 1" + qt_upper_refer_datetime_not_to_scalar_agg "select dt from (select dt, min(a), sum(a), count(a) from eli_gbk_by_uniform_t where dt = '2023-01-06 15:00:00' group by dt) t order by 1" + qt_upper_refer_datetime "select dt from (select dt, min(a), sum(a), count(a) from eli_gbk_by_uniform_t where dt = '2023-01-06 15:00:00' group by dt, a) t order by 1" + + qt_project_no_other_agg_func "select c2 from (select a c1,1 c2, d c3 from eli_gbk_by_uniform_t) t group by c2,c3 order by 1;" + qt_project_const_not_to_scalar_agg_multi "select c2 from (select a c1,1 c2, 3 c3 from eli_gbk_by_uniform_t) t group by c2,c3 order by 1;" + qt_not_to_scalar_agg_multi "select a, min(a), sum(a), count(a) from eli_gbk_by_uniform_t where a = 1 and b=100 group by a, b,'abc' order by 1,2,3,4" + qt_conflict_equal_value "select a, min(a), sum(a), count(a) from eli_gbk_by_uniform_t where a = 1 and a=2 group by a, b,'abc' order by 1,2,3,4" + qt_project_slot_uniform_confict_value "select max(c3), c1,c2,c3 from (select a c1,1 c2, d c3 from eli_gbk_by_uniform_t where a=1) t where c2=2 group by c1,c2,c3 order by 1,2,3,4;" + + // test join + qt_inner_join_left_has_filter "select t1.b,t2.b from eli_gbk_by_uniform_t t1 inner join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t1.b=100 group by t1.b,t2.b,t2.c order by 1,2" + qt_inner_join_right_has_filter "select t1.b,t2.b from eli_gbk_by_uniform_t t1 inner join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t2.b=100 group by t1.b,t2.b,t2.c order by 1,2" + qt_left_join_right_has_filter "select t1.b,t2.b from eli_gbk_by_uniform_t t1 left join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t2.b=100 group by t1.b,t2.b,t2.c order by 1,2" + qt_left_join_left_has_filter "select t1.b,t2.b from eli_gbk_by_uniform_t t1 left join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t1.b=100 group by t1.b,t2.b,t2.c order by 1,2" + qt_right_join_right_has_filter "select t1.b,t2.b from eli_gbk_by_uniform_t t1 right join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t2.b=100 group by t1.b,t2.b,t2.c order by 1,2" + qt_right_join_left_has_filter "select t1.b,t2.b from eli_gbk_by_uniform_t t1 right join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t1.b=100 group by t1.b,t2.b,t2.c order by 1,2" + qt_left_semi_join_right_has_filter "select t1.b from eli_gbk_by_uniform_t t1 left semi join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t2.b=100 group by t1.b,t1.a order by 1" + qt_left_semi_join_left_has_filter "select t1.b from eli_gbk_by_uniform_t t1 left semi join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t1.b=100 group by t1.b,t1.a order by 1" + qt_left_anti_join_right_has_on_filter "select t1.b from eli_gbk_by_uniform_t t1 left anti join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t2.b=100 group by t1.b,t1.a order by 1" + qt_left_anti_join_left_has_on_filter "select t1.b from eli_gbk_by_uniform_t t1 left anti join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t1.b=100 group by t1.b,t1.a order by 1" + qt_left_anti_join_left_has_where_filter "select t1.b from eli_gbk_by_uniform_t t1 left anti join eli_gbk_by_uniform_t t2 on t1.b=t2.b where t1.b=100 group by t1.b,t1.a order by 1" + qt_right_semi_join_right_has_filter "select t2.b from eli_gbk_by_uniform_t t1 right semi join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t2.b=100 group by t2.b,t2.c order by 1" + qt_right_semi_join_left_has_filter "select t2.b from eli_gbk_by_uniform_t t1 right semi join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t1.b=100 group by t2.b,t2.c order by 1" + qt_right_anti_join_right_has_on_filter "select t2.b from eli_gbk_by_uniform_t t1 right anti join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t2.b=100 group by t2.b,t2.c order by 1" + qt_right_anti_join_left_has_on_filter "select t2.b from eli_gbk_by_uniform_t t1 right anti join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t1.b=100 group by t2.b,t2.c order by 1" + qt_right_anti_join_right_has_where_filter "select t2.b from eli_gbk_by_uniform_t t1 right anti join eli_gbk_by_uniform_t t2 on t1.b=t2.b where t2.b=100 group by t2.b,t2.c order by 1" + qt_cross_join_left_has_filter "select t1.b,t2.b from eli_gbk_by_uniform_t t1 cross join eli_gbk_by_uniform_t t2 where t1.b=100 group by t1.b,t2.b,t2.c order by 1,2" + qt_cross_join_right_has_filter "select t1.b,t2.b from eli_gbk_by_uniform_t t1 cross join eli_gbk_by_uniform_t t2 where t2.b=100 group by t1.b,t2.b,t2.c order by 1,2" + + //test union + qt_union "select * from (select a,b from eli_gbk_by_uniform_t where a=1 group by a,b union select a,b from eli_gbk_by_uniform_t where b=100 group by a,b union select a,b from eli_gbk_by_uniform_t where a=5 group by a,b) t order by 1,2,3,4,5" + qt_union_all "select * from (select a,b from eli_gbk_by_uniform_t where a=1 group by a,b union all select a,b from eli_gbk_by_uniform_t where b=100 group by a,b union all select a,b from eli_gbk_by_uniform_t where a=5 group by a,b) t order by 1,2,3,4,5" + qt_intersect "select * from (select a,b from eli_gbk_by_uniform_t where a=1 group by a,b intersect select a,b from eli_gbk_by_uniform_t where b=100 group by a,b intersect select a,b from eli_gbk_by_uniform_t where a=5 group by a,b) t order by 1,2,3,4,5" + qt_except "select * from (select a,b from eli_gbk_by_uniform_t where a=1 group by a,b except select a,b from eli_gbk_by_uniform_t where b=100 group by a,b except select a,b from eli_gbk_by_uniform_t where a=5 group by a,b) t order by 1,2,3,4,5" + qt_set_op_mixed "select * from (select a,b from eli_gbk_by_uniform_t where a=1 group by a,b union select a,b from eli_gbk_by_uniform_t where b=100 group by a,b except select a,b from eli_gbk_by_uniform_t where a=5 group by a,b) t order by 1,2,3,4,5" + + //test window + qt_window "select max(a) over(partition by a order by a) from eli_gbk_by_uniform_t where a=10 group by a,b order by 1" + //test partition topn + qt_partition_topn "select r from (select rank() over(partition by a order by a) r from eli_gbk_by_uniform_t where a=10 group by a,b) t where r<2 order by 1" + qt_partition_topn_qualifiy "select rank() over(partition by a order by a) r from eli_gbk_by_uniform_t where a=10 group by a,b qualify r<2 order by 1" + //test cte + qt_cte_producer "with t as (select a,b,count(*) from eli_gbk_by_uniform_t where a=1 group by a,b) select t1.a,t2.a,t2.b from t t1 inner join t t2 on t1.a=t2.a order by 1,2,3" + qt_cte_multi_producer "with t as (select a,b,count(*) from eli_gbk_by_uniform_t where a=1 group by a,b), tt as (select a,b,count(*) from eli_gbk_by_uniform_t where b=10 group by a,b) select t1.a,t2.a,t2.b from t t1 inner join tt t2 on t1.a=t2.a order by 1,2,3" + qt_cte_consumer "with t as (select * from eli_gbk_by_uniform_t) select t1.a,t2.b from t t1 inner join t t2 on t1.a=t2.a where t1.a=10 group by t1.a,t2.b order by 1,2 " + + //test filter + qt_filter "select * from (select a,b from eli_gbk_by_uniform_t where a=1 group by a,b) t where a>0 order by 1,2" + + //test topn + qt_topn "select a,b from eli_gbk_by_uniform_t where a=1 group by a,b order by a limit 10 offset 0" + + //olap table sink + sql "insert into eli_gbk_by_uniform_t select a,b,c,d,dt from eli_gbk_by_uniform_t where a = 1 group by a,b,c,d,dt" + qt_sink "select * from eli_gbk_by_uniform_t order by 1,2,3,4,5" + + sql """ + drop table if exists orders_inner_1 + """ + + sql """CREATE TABLE `orders_inner_1` ( + `o_orderkey` BIGINT not NULL, + `o_custkey` INT NULL, + `o_orderstatus` VARCHAR(1) NULL, + `o_totalprice` DECIMAL(15, 2) NULL, + `o_orderpriority` VARCHAR(15) NULL, + `o_clerk` VARCHAR(15) NULL, + `o_shippriority` INT NULL, + `o_comment` VARCHAR(79) NULL, + `o_orderdate` DATE NULL + ) ENGINE=OLAP + DUPLICATE KEY(`o_orderkey`, `o_custkey`) + COMMENT 'OLAP' + PARTITION BY list(o_orderkey) ( + PARTITION p1 VALUES in ('1'), + PARTITION p2 VALUES in ('2'), + PARTITION p3 VALUES in ('3'), + PARTITION p4 VALUES in ('4') + ) + DISTRIBUTED BY HASH(`o_orderkey`) BUCKETS 96 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + );""" + + sql """ + drop table if exists lineitem_inner_1 + """ + + sql """CREATE TABLE `lineitem_inner_1` ( + `l_orderkey` BIGINT not NULL, + `l_linenumber` INT NULL, + `l_partkey` INT NULL, + `l_suppkey` INT NULL, + `l_quantity` DECIMAL(15, 2) NULL, + `l_extendedprice` DECIMAL(15, 2) NULL, + `l_discount` DECIMAL(15, 2) NULL, + `l_tax` DECIMAL(15, 2) NULL, + `l_returnflag` VARCHAR(1) NULL, + `l_linestatus` VARCHAR(1) NULL, + `l_commitdate` DATE NULL, + `l_receiptdate` DATE NULL, + `l_shipinstruct` VARCHAR(25) NULL, + `l_shipmode` VARCHAR(10) NULL, + `l_comment` VARCHAR(44) NULL, + `l_shipdate` DATE NULL + ) ENGINE=OLAP + DUPLICATE KEY(l_orderkey, l_linenumber, l_partkey, l_suppkey ) + COMMENT 'OLAP' + PARTITION BY list(l_orderkey) ( + PARTITION p1 VALUES in ('1'), + PARTITION p2 VALUES in ('2'), + PARTITION p3 VALUES in ('3') + ) + DISTRIBUTED BY HASH(`l_orderkey`) BUCKETS 96 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + );""" + + sql """ + insert into orders_inner_1 values + (2, 1, 'o', 99.5, 'a', 'b', 1, 'yy', '2023-10-17'), + (1, null, 'k', 109.2, 'c','d',2, 'mm', '2023-10-17'), + (3, 3, null, 99.5, 'a', 'b', 1, 'yy', '2023-10-19'), + (1, 2, 'o', null, 'a', 'b', 1, 'yy', '2023-10-20'), + (2, 3, 'k', 109.2, null,'d',2, 'mm', '2023-10-21'), + (3, 1, 'o', 99.5, 'a', null, 1, 'yy', '2023-10-22'), + (1, 3, 'k', 99.5, 'a', 'b', null, 'yy', '2023-10-19'), + (2, 1, 'o', 109.2, 'c','d',2, null, '2023-10-18'), + (3, 2, 'k', 99.5, 'a', 'b', 1, 'yy', '2023-10-17'), + (4, 5, 'o', 99.5, 'a', 'b', 1, 'yy', '2023-10-19'); + """ + + sql """ + insert into lineitem_inner_1 values + (2, 1, 2, 3, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-17', '2023-10-17', 'a', 'b', 'yyyyyyyyy', '2023-10-17'), + (1, null, 3, 1, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-18', '2023-10-18', 'a', 'b', 'yyyyyyyyy', '2023-10-17'), + (3, 3, null, 2, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-10-19', '2023-10-19', 'c', 'd', 'xxxxxxxxx', '2023-10-19'), + (1, 2, 3, null, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-17', '2023-10-17', 'a', 'b', 'yyyyyyyyy', '2023-10-17'), + (2, 3, 2, 1, 5.5, 6.5, 7.5, 8.5, 'o', 'k', null, '2023-10-18', 'a', 'b', 'yyyyyyyyy', '2023-10-18'), + (3, 1, 1, 2, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-10-19', null, 'c', 'd', 'xxxxxxxxx', '2023-10-19'), + (1, 3, 2, 2, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-17', '2023-10-17', 'a', 'b', 'yyyyyyyyy', '2023-10-17'); + """ + + qt_nest_exprid_replace """ + select l_shipdate, l_orderkey, t.O_ORDERDATE, t.o_orderkey, + count(t.O_ORDERDATE) over (partition by lineitem_inner_1.l_orderkey order by lineitem_inner_1.l_orderkey) as window_count + from lineitem_inner_1 + inner join (select O_ORDERDATE, o_orderkey, count(O_ORDERDATE) over (partition by O_ORDERDATE order by o_orderkey ) from orders_inner_1 where o_orderkey=2 group by O_ORDERDATE, o_orderkey) as t + on lineitem_inner_1.l_orderkey = t.o_orderkey + where t.o_orderkey=2 + group by l_shipdate, l_orderkey, t.O_ORDERDATE, t.o_orderkey + order by 1,2,3,4,5 + """ + sql "drop table if exists test1" + sql "drop table if exists test2" + sql "create table test1(a int, b int) distributed by hash(a) properties('replication_num'='1');" + sql "insert into test1 values(1,1),(2,1),(3,1);" + sql "create table test2(a int, b int) distributed by hash(a) properties('replication_num'='1');" + sql "insert into test2 values(1,105),(2,105);" + qt_full_join_uniform_should_not_eliminate_group_by_key "select t2.b,t1.b from test1 t1 full join (select * from test2 where b=105) t2 on t1.a=t2.a group by t2.b,t1.b order by 1,2;" + qt_full2 "select t2.b,t1.b from (select * from test2 where b=105) t1 full join test1 t2 on t1.a=t2.a group by t2.b,t1.b order by 1,2;" + + qt_left_join_right_side_should_not_eliminate_group_by_key "select t2.b,t1.b from test1 t1 left join (select * from test2 where b=105) t2 on t1.a=t2.a group by t2.b,t1.b order by 1,2;" + qt_left_join_left_side_should_eliminate_group_by_key "select t2.b,t1.b from test1 t1 left join (select * from test2 where b=105) t2 on t1.a=t2.a where t1.b=1 group by t2.b,t1.b order by 1,2;" + + qt_right_join_left_side_should_not_eliminate_group_by_key "select t2.b,t1.b from (select * from test2 where b=105) t1 right join test1 t2 on t1.a=t2.a group by t2.b,t1.b order by 1,2;" + qt_right_join_right_side_should_eliminate_group_by_key "select t2.b,t1.b from (select * from test2 where b=105) t1 right join test1 t2 on t1.a=t2.a where t2.b=1 group by t2.b,t1.b order by 1,2;" + + qt_left_semi_left_side "select t1.b from test1 t1 left semi join (select * from test2 where b=105) t2 on t1.a=t2.a where t1.b=1 group by t1.b,t1.a order by 1;" + qt_left_anti_left_side "select t1.b from test1 t1 left anti join (select * from test2 where b=105) t2 on t1.a=t2.a where t1.b=1 group by t1.b,t1.a order by 1;" + qt_right_semi_right_side "select t2.b from test1 t1 right semi join (select * from test2 where b=105) t2 on t1.a=t2.a group by t2.b,t2.a order by 1;" + qt_right_anti_right_side "select t2.b from test1 t1 right anti join (select * from test2 where b=105) t2 on t1.a=t2.a group by t2.b,t2.a order by 1;" +} \ No newline at end of file diff --git a/regression-test/suites/nereids_rules_p0/mv/agg_without_roll_up/aggregate_without_roll_up.groovy b/regression-test/suites/nereids_rules_p0/mv/agg_without_roll_up/aggregate_without_roll_up.groovy index c43bde59acaa3d..92f54f3dd456e8 100644 --- a/regression-test/suites/nereids_rules_p0/mv/agg_without_roll_up/aggregate_without_roll_up.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/agg_without_roll_up/aggregate_without_roll_up.groovy @@ -373,7 +373,7 @@ suite("aggregate_without_roll_up") { "max(o_totalprice) as max_total, " + "min(o_totalprice) as min_total, " + "count(*) as count_all, " + - "count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) as distinct_count " + + "bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) as distinct_count " + "from lineitem " + "left join orders on lineitem.l_orderkey = orders.o_orderkey and l_shipdate = o_orderdate " + "group by " + @@ -571,7 +571,7 @@ suite("aggregate_without_roll_up") { "max(o_totalprice) as max_total, " + "min(o_totalprice) as min_total, " + "count(*) as count_all, " + - "count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) as distinct_count " + + "bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) as distinct_count " + "from lineitem " + "left join orders on lineitem.l_orderkey = orders.o_orderkey and l_shipdate = o_orderdate " + "group by " + @@ -661,7 +661,7 @@ suite("aggregate_without_roll_up") { "max(o_totalprice) as max_total, " + "min(o_totalprice) as min_total, " + "count(*) as count_all, " + - "count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) as distinct_count " + + "bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) as distinct_count " + "from lineitem " + "left join orders on lineitem.l_orderkey = orders.o_orderkey and l_shipdate = o_orderdate " + "group by " + From 8514df82ca87716dd1e5b175817b6beac411fb60 Mon Sep 17 00:00:00 2001 From: Pxl Date: Mon, 2 Dec 2024 14:29:04 +0800 Subject: [PATCH 108/399] [Bug](runtime-filter) check rf dependency is set and fix wrong throw status (#44838) ### What problem does this PR solve? 1. check rf dependency is set 2. fix wrong throw status, introduced by https://github.com/apache/doris/issues/44697 3. catch throw exception on buildsink::close --- be/src/exprs/runtime_filter.h | 5 +++-- be/src/pipeline/exec/hashjoin_build_sink.cpp | 3 ++- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/be/src/exprs/runtime_filter.h b/be/src/exprs/runtime_filter.h index a1fdfbf07d239d..63ef3e2dbd4653 100644 --- a/be/src/exprs/runtime_filter.h +++ b/be/src/exprs/runtime_filter.h @@ -355,8 +355,9 @@ class IRuntimeFilter { const std::shared_ptr& dependency); int64_t get_synced_size() const { - if (_synced_size == -1) { - throw Status::InternalError("sync filter size meet error, filter: {}", debug_string()); + if (_synced_size == -1 || !_dependency) { + throw Exception(doris::ErrorCode::INTERNAL_ERROR, + "sync filter size meet error, filter: {}", debug_string()); } return _synced_size; } diff --git a/be/src/pipeline/exec/hashjoin_build_sink.cpp b/be/src/pipeline/exec/hashjoin_build_sink.cpp index cec0c77da8a61d..864927e1721923 100644 --- a/be/src/pipeline/exec/hashjoin_build_sink.cpp +++ b/be/src/pipeline/exec/hashjoin_build_sink.cpp @@ -156,7 +156,8 @@ Status HashJoinBuildSinkLocalState::close(RuntimeState* state, Status exec_statu uint64_t hash_table_size = block ? block->rows() : 0; { SCOPED_TIMER(_runtime_filter_init_timer); - RETURN_IF_ERROR(_runtime_filter_slots->init_filters(state, hash_table_size)); + RETURN_IF_ERROR_OR_CATCH_EXCEPTION( + _runtime_filter_slots->init_filters(state, hash_table_size)); RETURN_IF_ERROR(_runtime_filter_slots->ignore_filters(state)); } if (hash_table_size > 1) { From fa13a700770349521c39ac88e75b5da7f313c4a9 Mon Sep 17 00:00:00 2001 From: Sun Chenyang Date: Mon, 2 Dec 2024 14:49:30 +0800 Subject: [PATCH 109/399] [feat] (inverted index) show index file size (#44120) ### What problem does this PR solve? Problem Summary: 1. Show the data in detail, including the size of the inverted index file and the data file. ``` mysql > show data all; +------------------------+-----------+--------------+-------------+-----------------+-----------------+-----------------+-----------------+ | TableName | ReplicaCount | LocalTotalSize| LocalDataSize | LocalIndexSize | RemoteTotalSize| RemoteDataSize | RemoteIndexSize | +------------------------+-----------+--------------+-------------+-----------------+-----------------+-----------------+-----------------+ | test_show_index_data_p2 | 1 | 291.534 MB | 133.697 MB | 157.837 MB | 0.000 | 0.000 | 0.000 | | Total | 1 | 291.534 MB | 133.697 MB | 157.837 MB | 0.000 | 0.000 | 0.000 | | Quota | 1024.000 TB | 1073741824 | | | | | | | Left | 1024.000 TB | 1073741823 | | | | | | +------------------------+-----------+--------------+-------------+-----------------+-----------------+-----------------+-----------------+ 4 rows in set (0.00 sec) ``` ``` msql> show data all from test_show_index_data_p2; +------------------------+------------------------+-----------+--------------+----------+-------------+-----------------+-----------------+-----------------+-----------------+ | TableName | IndexName | ReplicaCount | RowCount| LocalTotalSize | LocalDataSize | LocalIndexSize | RemoteTotalSize | RemoteDataSize | RemoteIndexSize | +------------------------+------------------------+-----------+--------------+----------+-------------+-----------------+-----------------+-----------------+-----------------+ | test_show_index_data_p2 | test_show_index_data_p2 | 1 | 19697882 | 291.534 MB | 157.837 MB | 133.697 MN | 0.000 | 0.000 | 0.000 | | | Total | 1 | | 291.534 MB | 133.697 MB | 157.837 MB | 0.000 | 0.000 | 0.000 | +------------------------+------------------------+-----------+--------------+----------+-------------+-----------------+-----------------+-----------------+-----------------+ 2 rows in set (0.00 sec) ``` 2. It is possible to obtain the sizes of data and index files by querying the system tables ``` msyql > select * from information_schema.tables where TABLE_NAME = "test_show_index_data_p2"; +-------------------------+-------------------------------+---------------------+----------------+-------------+-------------+-------------+------------+------------+---------------------+---------------------+------------+---------+------------+---------------+----------------+ | TABLE_CATALOG | TABLE_SCHEMA | TABLE_NAME | TABLE_TYPE | ENGINE | VERSION | ROW_FORMAT | TABLE_ROWS | AVG_ROW_LENGTH | DATA_LENGTH | MAX_DATA_LENGTH | INDEX_LENGTH | DATA_FREE | AUTO_INCREMENT | CREATE_TIME | UPDATE_TIME | CHECK_TIME | TABLE_COLLATION | CHECKSUM | CREATE_OPTIONS | TABLE_COMMENT | +-------------------------+-------------------------------+---------------------+----------------+-------------+-------------+-------------+------------+--------------+-------------+----------------+--------------+-----------+---------------+---------------------+---------------------+------------+----------------+---------+---------------+---------------+ | internal | regression_test_inverted_index_p2_show_data | test_show_index_data_p2 | BASE TABLE | Doris | NULL | NULL | 19697882 | 15 | 140191631 | NULL | 165504277 | NULL | NULL | 2024-11-18 15:22:32 | 2024-11-18 15:24:52 | NULL | utf-8 | NULL | NULL | | +-------------------------+-------------------------------+---------------------+----------------+-------------+-------------+-------------+------------+--------------+-------------+----------------+--------------+-----------+---------------+---------------------+---------------------+------------+----------------+---------+---------------+---------------+ 1 row in set (0.02 sec) ``` ### Release note 1. Added `show data all;` to retrieve the detailed file size. 2. Fixed the semantics of `DATA_LENGTH` and `INDEX_LENGTH` in the system `table information_schema.tables`. --- be/src/cloud/cloud_base_compaction.cpp | 5 + be/src/cloud/cloud_cumulative_compaction.cpp | 4 + be/src/cloud/cloud_full_compaction.cpp | 4 + be/src/cloud/cloud_schema_change_job.cpp | 6 + .../schema_scanner/schema_tables_scanner.cpp | 16 +- be/src/olap/tablet.cpp | 4 + be/src/olap/tablet_manager.cpp | 4 + be/src/olap/tablet_meta.h | 46 ++ cloud/src/meta-service/keys.cpp | 8 + cloud/src/meta-service/keys.h | 6 + cloud/src/meta-service/meta_service.cpp | 2 + cloud/src/meta-service/meta_service_job.cpp | 22 +- .../meta_service_tablet_stats.cpp | 111 +++- .../meta-service/meta_service_tablet_stats.h | 2 + cloud/src/meta-service/meta_service_txn.cpp | 26 +- cloud/src/meta-service/txn_lazy_committer.cpp | 4 +- cloud/test/http_encode_key_test.cpp | 32 +- cloud/test/keys_test.cpp | 34 ++ cloud/test/meta_service_http_test.cpp | 14 + cloud/test/meta_service_job_test.cpp | 60 ++- cloud/test/meta_service_test.cpp | 24 + cloud/test/schema_kv_test.cpp | 6 + .../org/apache/doris/nereids/DorisParser.g4 | 3 +- fe/fe-core/src/main/cup/sql_parser.cup | 21 +- .../apache/doris/analysis/ShowDataStmt.java | 484 +++++++++++------- .../doris/catalog/CloudTabletStatMgr.java | 20 +- .../doris/catalog/MaterializedIndex.java | 40 ++ .../org/apache/doris/catalog/OlapTable.java | 55 +- .../org/apache/doris/catalog/Replica.java | 19 + .../java/org/apache/doris/catalog/Table.java | 3 + .../org/apache/doris/catalog/TableIf.java | 2 + .../apache/doris/catalog/TabletStatMgr.java | 17 +- .../doris/datasource/ExternalTable.java | 5 + .../doris/service/FrontendServiceImpl.java | 1 + .../doris/analysis/ShowDataStmtTest.java | 6 +- gensrc/proto/cloud.proto | 10 +- gensrc/thrift/BackendService.thrift | 7 +- gensrc/thrift/FrontendService.thrift | 1 + gensrc/thrift/MasterService.thrift | 4 + .../system/test_query_sys_tables.out | 6 +- .../query_p0/system/test_query_sys_tables.out | 6 +- .../show_data/test_show_index_data.groovy | 399 +++++++++++++++ .../show_data/test_show_index_data_p2.groovy | 397 ++++++++++++++ 43 files changed, 1702 insertions(+), 244 deletions(-) create mode 100644 regression-test/suites/inverted_index_p1/show_data/test_show_index_data.groovy create mode 100644 regression-test/suites/inverted_index_p2/show_data/test_show_index_data_p2.groovy diff --git a/be/src/cloud/cloud_base_compaction.cpp b/be/src/cloud/cloud_base_compaction.cpp index 88d83000e95dfa..9742e57dcf9d34 100644 --- a/be/src/cloud/cloud_base_compaction.cpp +++ b/be/src/cloud/cloud_base_compaction.cpp @@ -125,6 +125,7 @@ Status CloudBaseCompaction::prepare_compact() { _input_row_num += rs->num_rows(); _input_segments += rs->num_segments(); _input_rowsets_data_size += rs->data_disk_size(); + _input_rowsets_index_size += rs->index_disk_size(); _input_rowsets_total_size += rs->total_disk_size(); } LOG_INFO("start CloudBaseCompaction, tablet_id={}, range=[{}-{}]", _tablet->tablet_id(), @@ -320,6 +321,10 @@ Status CloudBaseCompaction::modify_rowsets() { compaction_job->add_output_versions(_output_rowset->end_version()); compaction_job->add_txn_id(_output_rowset->txn_id()); compaction_job->add_output_rowset_ids(_output_rowset->rowset_id().to_string()); + compaction_job->set_index_size_input_rowsets(_input_rowsets_index_size); + compaction_job->set_segment_size_input_rowsets(_input_rowsets_data_size); + compaction_job->set_index_size_output_rowsets(_output_rowset->index_disk_size()); + compaction_job->set_segment_size_output_rowsets(_output_rowset->data_disk_size()); DeleteBitmapPtr output_rowset_delete_bitmap = nullptr; if (_tablet->keys_type() == KeysType::UNIQUE_KEYS && diff --git a/be/src/cloud/cloud_cumulative_compaction.cpp b/be/src/cloud/cloud_cumulative_compaction.cpp index 2f08082f51b5f3..c466c35e2a2ab6 100644 --- a/be/src/cloud/cloud_cumulative_compaction.cpp +++ b/be/src/cloud/cloud_cumulative_compaction.cpp @@ -263,6 +263,10 @@ Status CloudCumulativeCompaction::modify_rowsets() { compaction_job->add_output_versions(_output_rowset->end_version()); compaction_job->add_txn_id(_output_rowset->txn_id()); compaction_job->add_output_rowset_ids(_output_rowset->rowset_id().to_string()); + compaction_job->set_index_size_input_rowsets(_input_rowsets_index_size); + compaction_job->set_segment_size_input_rowsets(_input_rowsets_data_size); + compaction_job->set_index_size_output_rowsets(_output_rowset->index_disk_size()); + compaction_job->set_segment_size_output_rowsets(_output_rowset->data_disk_size()); DBUG_EXECUTE_IF("CloudCumulativeCompaction::modify_rowsets.enable_spin_wait", { LOG(INFO) << "CloudCumulativeCompaction::modify_rowsets.enable_spin_wait, start"; diff --git a/be/src/cloud/cloud_full_compaction.cpp b/be/src/cloud/cloud_full_compaction.cpp index c27b728c93d29b..bce00c9a2e74f6 100644 --- a/be/src/cloud/cloud_full_compaction.cpp +++ b/be/src/cloud/cloud_full_compaction.cpp @@ -216,6 +216,10 @@ Status CloudFullCompaction::modify_rowsets() { compaction_job->add_output_versions(_output_rowset->end_version()); compaction_job->add_txn_id(_output_rowset->txn_id()); compaction_job->add_output_rowset_ids(_output_rowset->rowset_id().to_string()); + compaction_job->set_index_size_input_rowsets(_input_rowsets_index_size); + compaction_job->set_segment_size_input_rowsets(_input_rowsets_data_size); + compaction_job->set_index_size_output_rowsets(_output_rowset->index_disk_size()); + compaction_job->set_segment_size_output_rowsets(_output_rowset->data_disk_size()); DeleteBitmapPtr output_rowset_delete_bitmap = nullptr; if (_tablet->keys_type() == KeysType::UNIQUE_KEYS && diff --git a/be/src/cloud/cloud_schema_change_job.cpp b/be/src/cloud/cloud_schema_change_job.cpp index 0bab742c3ad6e5..1cc4d052a81d69 100644 --- a/be/src/cloud/cloud_schema_change_job.cpp +++ b/be/src/cloud/cloud_schema_change_job.cpp @@ -340,17 +340,23 @@ Status CloudSchemaChangeJob::_convert_historical_rowsets(const SchemaChangeParam int64_t num_output_rows = 0; int64_t size_output_rowsets = 0; int64_t num_output_segments = 0; + int64_t index_size_output_rowsets = 0; + int64_t segment_size_output_rowsets = 0; for (auto& rs : _output_rowsets) { sc_job->add_txn_ids(rs->txn_id()); sc_job->add_output_versions(rs->end_version()); num_output_rows += rs->num_rows(); size_output_rowsets += rs->total_disk_size(); num_output_segments += rs->num_segments(); + index_size_output_rowsets += rs->index_disk_size(); + segment_size_output_rowsets += rs->data_disk_size(); } sc_job->set_num_output_rows(num_output_rows); sc_job->set_size_output_rowsets(size_output_rowsets); sc_job->set_num_output_segments(num_output_segments); sc_job->set_num_output_rowsets(_output_rowsets.size()); + sc_job->set_index_size_output_rowsets(index_size_output_rowsets); + sc_job->set_segment_size_output_rowsets(segment_size_output_rowsets); } _output_cumulative_point = std::min(_output_cumulative_point, sc_job->alter_version() + 1); sc_job->set_output_cumulative_point(_output_cumulative_point); diff --git a/be/src/exec/schema_scanner/schema_tables_scanner.cpp b/be/src/exec/schema_scanner/schema_tables_scanner.cpp index 23710b81971c15..3aba0dfcc4f83c 100644 --- a/be/src/exec/schema_scanner/schema_tables_scanner.cpp +++ b/be/src/exec/schema_scanner/schema_tables_scanner.cpp @@ -236,7 +236,7 @@ Status SchemaTablesScanner::_fill_block_impl(vectorized::Block* block) { std::vector srcs(table_num); for (int i = 0; i < table_num; ++i) { const TTableStatus& tbl_status = _table_result.tables[i]; - if (tbl_status.__isset.avg_row_length) { + if (tbl_status.__isset.data_length) { srcs[i] = tbl_status.data_length; datas[i] = srcs.data() + i; } else { @@ -248,7 +248,19 @@ Status SchemaTablesScanner::_fill_block_impl(vectorized::Block* block) { // max_data_length { RETURN_IF_ERROR(fill_dest_column_for_range(block, 10, null_datas)); } // index_length - { RETURN_IF_ERROR(fill_dest_column_for_range(block, 11, null_datas)); } + { + std::vector srcs(table_num); + for (int i = 0; i < table_num; ++i) { + const TTableStatus& tbl_status = _table_result.tables[i]; + if (tbl_status.__isset.index_length) { + srcs[i] = tbl_status.index_length; + datas[i] = srcs.data() + i; + } else { + datas[i] = nullptr; + } + } + RETURN_IF_ERROR(fill_dest_column_for_range(block, 11, datas)); + } // data_free { RETURN_IF_ERROR(fill_dest_column_for_range(block, 12, null_datas)); } // auto_increment diff --git a/be/src/olap/tablet.cpp b/be/src/olap/tablet.cpp index 0d04984d0e06ba..379fb6eec3cb43 100644 --- a/be/src/olap/tablet.cpp +++ b/be/src/olap/tablet.cpp @@ -1692,6 +1692,10 @@ void Tablet::build_tablet_report_info(TTabletInfo* tablet_info, // tablet may not have cooldowned data, but the storage policy is set tablet_info->__set_cooldown_term(_cooldown_conf.term); } + tablet_info->__set_local_index_size(_tablet_meta->tablet_local_index_size()); + tablet_info->__set_local_segment_size(_tablet_meta->tablet_local_segment_size()); + tablet_info->__set_remote_index_size(_tablet_meta->tablet_remote_index_size()); + tablet_info->__set_remote_segment_size(_tablet_meta->tablet_remote_segment_size()); } void Tablet::report_error(const Status& st) { diff --git a/be/src/olap/tablet_manager.cpp b/be/src/olap/tablet_manager.cpp index d6a944dbc39853..a18dc0b4a161b6 100644 --- a/be/src/olap/tablet_manager.cpp +++ b/be/src/olap/tablet_manager.cpp @@ -1083,6 +1083,10 @@ void TabletManager::build_all_report_tablets_info(std::map* t_tablet_stat.__set_total_version_count(tablet_info.total_version_count); t_tablet_stat.__set_visible_version_count(tablet_info.visible_version_count); t_tablet_stat.__set_visible_version(tablet_info.version); + t_tablet_stat.__set_local_index_size(tablet_info.local_index_size); + t_tablet_stat.__set_local_segment_size(tablet_info.local_segment_size); + t_tablet_stat.__set_remote_index_size(tablet_info.remote_index_size); + t_tablet_stat.__set_remote_segment_size(tablet_info.remote_segment_size); }; for_each_tablet(handler, filter_all_tablets); diff --git a/be/src/olap/tablet_meta.h b/be/src/olap/tablet_meta.h index fb0895604a19fe..34794ef6d0a760 100644 --- a/be/src/olap/tablet_meta.h +++ b/be/src/olap/tablet_meta.h @@ -170,6 +170,12 @@ class TabletMeta : public MetadataAdder { size_t tablet_local_size() const; // Remote disk space occupied by tablet. size_t tablet_remote_size() const; + + size_t tablet_local_index_size() const; + size_t tablet_local_segment_size() const; + size_t tablet_remote_index_size() const; + size_t tablet_remote_segment_size() const; + size_t version_count() const; size_t stale_version_count() const; size_t version_count_cross_with_range(const Version& range) const; @@ -668,6 +674,46 @@ inline size_t TabletMeta::tablet_remote_size() const { return total_size; } +inline size_t TabletMeta::tablet_local_index_size() const { + size_t total_size = 0; + for (auto& rs : _rs_metas) { + if (rs->is_local()) { + total_size += rs->index_disk_size(); + } + } + return total_size; +} + +inline size_t TabletMeta::tablet_local_segment_size() const { + size_t total_size = 0; + for (auto& rs : _rs_metas) { + if (rs->is_local()) { + total_size += rs->data_disk_size(); + } + } + return total_size; +} + +inline size_t TabletMeta::tablet_remote_index_size() const { + size_t total_size = 0; + for (auto& rs : _rs_metas) { + if (!rs->is_local()) { + total_size += rs->index_disk_size(); + } + } + return total_size; +} + +inline size_t TabletMeta::tablet_remote_segment_size() const { + size_t total_size = 0; + for (auto& rs : _rs_metas) { + if (!rs->is_local()) { + total_size += rs->data_disk_size(); + } + } + return total_size; +} + inline size_t TabletMeta::version_count() const { return _rs_metas.size(); } diff --git a/cloud/src/meta-service/keys.cpp b/cloud/src/meta-service/keys.cpp index 820d349084e35f..a518b6e264d20d 100644 --- a/cloud/src/meta-service/keys.cpp +++ b/cloud/src/meta-service/keys.cpp @@ -382,6 +382,14 @@ void stats_tablet_num_segs_key(const StatsTabletKeyInfo& in, std::string* out) { stats_tablet_key(in, out); encode_bytes(STATS_KEY_SUFFIX_NUM_SEGS, out); } +void stats_tablet_index_size_key(const StatsTabletKeyInfo& in, std::string* out) { + stats_tablet_key(in, out); + encode_bytes(STATS_KEY_SUFFIX_INDEX_SIZE, out); +} +void stats_tablet_segment_size_key(const StatsTabletKeyInfo& in, std::string* out) { + stats_tablet_key(in, out); + encode_bytes(STATS_KEY_SUFFIX_SEGMENT_SIZE, out); +} //============================================================================== // Job keys diff --git a/cloud/src/meta-service/keys.h b/cloud/src/meta-service/keys.h index c63af925b8f4de..855171c1dc648a 100644 --- a/cloud/src/meta-service/keys.h +++ b/cloud/src/meta-service/keys.h @@ -51,6 +51,8 @@ // 0x01 "stats" ${instance_id} "tablet" ${table_id} ${index_id} ${partition_id} ${tablet_id} "num_rows" -> int64 // 0x01 "stats" ${instance_id} "tablet" ${table_id} ${index_id} ${partition_id} ${tablet_id} "num_rowsets" -> int64 // 0x01 "stats" ${instance_id} "tablet" ${table_id} ${index_id} ${partition_id} ${tablet_id} "num_segs" -> int64 +// 0x01 "stats" ${instance_id} "tablet" ${table_id} ${index_id} ${partition_id} ${tablet_id} "index_size" -> int64 +// 0x01 "stats" ${instance_id} "tablet" ${table_id} ${index_id} ${partition_id} ${tablet_id} "segment_size"-> int64 // // 0x01 "recycle" ${instance_id} "index" ${index_id} -> RecycleIndexPB // 0x01 "recycle" ${instance_id} "partition" ${partition_id} -> RecyclePartitionPB @@ -83,6 +85,8 @@ static constexpr std::string_view STATS_KEY_SUFFIX_DATA_SIZE = "data_size"; static constexpr std::string_view STATS_KEY_SUFFIX_NUM_ROWS = "num_rows"; static constexpr std::string_view STATS_KEY_SUFFIX_NUM_ROWSETS = "num_rowsets"; static constexpr std::string_view STATS_KEY_SUFFIX_NUM_SEGS = "num_segs"; +static constexpr std::string_view STATS_KEY_SUFFIX_INDEX_SIZE = "index_size"; +static constexpr std::string_view STATS_KEY_SUFFIX_SEGMENT_SIZE = "segment_size"; // clang-format off /** @@ -247,6 +251,8 @@ void stats_tablet_data_size_key(const StatsTabletKeyInfo& in, std::string* out); void stats_tablet_num_rows_key(const StatsTabletKeyInfo& in, std::string* out); void stats_tablet_num_rowsets_key(const StatsTabletKeyInfo& in, std::string* out); void stats_tablet_num_segs_key(const StatsTabletKeyInfo& in, std::string* out); +void stats_tablet_index_size_key(const StatsTabletKeyInfo& in, std::string* out); +void stats_tablet_segment_size_key(const StatsTabletKeyInfo& in, std::string* out); static inline std::string stats_tablet_key(const StatsTabletKeyInfo& in) { std::string s; stats_tablet_key(in, &s); return s; } void job_recycle_key(const JobRecycleKeyInfo& in, std::string* out); diff --git a/cloud/src/meta-service/meta_service.cpp b/cloud/src/meta-service/meta_service.cpp index 974b3091a73e95..ddf4ee3889c00e 100644 --- a/cloud/src/meta-service/meta_service.cpp +++ b/cloud/src/meta-service/meta_service.cpp @@ -1642,6 +1642,8 @@ void MetaServiceImpl::get_tablet_stats(::google::protobuf::RpcController* contro #ifdef NDEBUG // Force data size >= 0 to reduce the losses caused by bugs if (tablet_stats->data_size() < 0) tablet_stats->set_data_size(0); + if (tablet_stats->index_size() < 0) tablet_stats->set_index_size(0); + if (tablet_stats->segment_size() < 0) tablet_stats->set_segment_size(0); #endif } } diff --git a/cloud/src/meta-service/meta_service_job.cpp b/cloud/src/meta-service/meta_service_job.cpp index d1c8df15870de7..cc80d5bd138980 100644 --- a/cloud/src/meta-service/meta_service_job.cpp +++ b/cloud/src/meta-service/meta_service_job.cpp @@ -701,6 +701,8 @@ void process_compaction_job(MetaServiceCode& code, std::string& msg, std::string stats->set_data_size(stats->data_size() + (compaction.size_output_rowsets() - compaction.size_input_rowsets())); stats->set_num_rowsets(stats->num_rowsets() + (compaction.num_output_rowsets() - compaction.num_input_rowsets())); stats->set_num_segments(stats->num_segments() + (compaction.num_output_segments() - compaction.num_input_segments())); + stats->set_index_size(stats->index_size() + (compaction.index_size_output_rowsets() - compaction.index_size_input_rowsets())); + stats->set_segment_size(stats->segment_size() + (compaction.segment_size_output_rowsets() - compaction.segment_size_input_rowsets())); stats->set_last_cumu_compaction_time_ms(now * 1000); // clang-format on } else if (compaction.type() == TabletCompactionJobPB::BASE) { @@ -710,6 +712,8 @@ void process_compaction_job(MetaServiceCode& code, std::string& msg, std::string stats->set_data_size(stats->data_size() + (compaction.size_output_rowsets() - compaction.size_input_rowsets())); stats->set_num_rowsets(stats->num_rowsets() + (compaction.num_output_rowsets() - compaction.num_input_rowsets())); stats->set_num_segments(stats->num_segments() + (compaction.num_output_segments() - compaction.num_input_segments())); + stats->set_index_size(stats->index_size() + (compaction.index_size_output_rowsets() - compaction.index_size_input_rowsets())); + stats->set_segment_size(stats->segment_size() + (compaction.segment_size_output_rowsets() - compaction.segment_size_input_rowsets())); stats->set_last_base_compaction_time_ms(now * 1000); // clang-format on } else if (compaction.type() == TabletCompactionJobPB::FULL) { @@ -724,6 +728,8 @@ void process_compaction_job(MetaServiceCode& code, std::string& msg, std::string stats->set_data_size(stats->data_size() + (compaction.size_output_rowsets() - compaction.size_input_rowsets())); stats->set_num_rowsets(stats->num_rowsets() + (compaction.num_output_rowsets() - compaction.num_input_rowsets())); stats->set_num_segments(stats->num_segments() + (compaction.num_output_segments() - compaction.num_input_segments())); + stats->set_index_size(stats->index_size() + (compaction.index_size_output_rowsets() - compaction.index_size_input_rowsets())); + stats->set_segment_size(stats->segment_size() + (compaction.segment_size_output_rowsets() - compaction.segment_size_input_rowsets())); stats->set_last_full_compaction_time_ms(now * 1000); // clang-format on } else { @@ -738,10 +744,14 @@ void process_compaction_job(MetaServiceCode& code, std::string& msg, std::string << " stats.data_size=" << stats->data_size() << " stats.num_rowsets=" << stats->num_rowsets() << " stats.num_segments=" << stats->num_segments() + << " stats.index_size=" << stats->index_size() + << " stats.segment_size=" << stats->segment_size() << " detached_stats.num_rows=" << detached_stats.num_rows << " detached_stats.data_size=" << detached_stats.data_size << " detached_stats.num_rowset=" << detached_stats.num_rowsets << " detached_stats.num_segments=" << detached_stats.num_segs + << " detached_stats.index_size=" << detached_stats.index_size + << " detached_stats.segment_size=" << detached_stats.segment_size << " compaction.size_output_rowsets=" << compaction.size_output_rowsets() << " compaction.size_input_rowsets=" << compaction.size_input_rowsets(); txn->put(stats_key, stats_val); @@ -752,10 +762,14 @@ void process_compaction_job(MetaServiceCode& code, std::string& msg, std::string << " stats.data_size=" << stats->data_size() << " stats.num_rowsets=" << stats->num_rowsets() << " stats.num_segments=" << stats->num_segments() + << " stats.index_size=" << stats->index_size() + << " stats.segment_size=" << stats->segment_size() << " detached_stats.num_rows=" << detached_stats.num_rows << " detached_stats.data_size=" << detached_stats.data_size << " detached_stats.num_rowset=" << detached_stats.num_rowsets << " detached_stats.num_segments=" << detached_stats.num_segs + << " detached_stats.index_size=" << detached_stats.index_size + << " detached_stats.segment_size=" << detached_stats.segment_size << " compaction.size_output_rowsets=" << compaction.size_output_rowsets() << " compaction.size_input_rowsets=" << compaction.size_input_rowsets(); @@ -1133,6 +1147,8 @@ void process_schema_change_job(MetaServiceCode& code, std::string& msg, std::str int64_t size_remove_rowsets = 0; int64_t num_remove_rowsets = 0; int64_t num_remove_segments = 0; + int64_t index_size_remove_rowsets = 0; + int64_t segment_size_remove_rowsets = 0; auto rs_start = meta_rowset_key({instance_id, new_tablet_id, 2}); auto rs_end = meta_rowset_key({instance_id, new_tablet_id, schema_change.alter_version() + 1}); @@ -1162,9 +1178,11 @@ void process_schema_change_job(MetaServiceCode& code, std::string& msg, std::str } num_remove_rows += rs.num_rows(); - size_remove_rowsets += rs.data_disk_size(); + size_remove_rowsets += rs.total_disk_size(); ++num_remove_rowsets; num_remove_segments += rs.num_segments(); + index_size_remove_rowsets += rs.index_disk_size(); + segment_size_remove_rowsets += rs.data_disk_size(); auto recycle_key = recycle_rowset_key({instance_id, new_tablet_id, rs.rowset_id_v2()}); RecycleRowsetPB recycle_rowset; @@ -1199,6 +1217,8 @@ void process_schema_change_job(MetaServiceCode& code, std::string& msg, std::str stats->set_data_size(stats->data_size() + (schema_change.size_output_rowsets() - size_remove_rowsets)); stats->set_num_rowsets(stats->num_rowsets() + (schema_change.num_output_rowsets() - num_remove_rowsets)); stats->set_num_segments(stats->num_segments() + (schema_change.num_output_segments() - num_remove_segments)); + stats->set_index_size(stats->index_size() + (schema_change.index_size_output_rowsets() - index_size_remove_rowsets)); + stats->set_segment_size(stats->segment_size() + (schema_change.segment_size_output_rowsets() - segment_size_remove_rowsets)); // clang-format on auto stats_key = stats_tablet_key( {instance_id, new_table_id, new_index_id, new_partition_id, new_tablet_id}); diff --git a/cloud/src/meta-service/meta_service_tablet_stats.cpp b/cloud/src/meta-service/meta_service_tablet_stats.cpp index cecccbd67673ad..4cbf629c9a3185 100644 --- a/cloud/src/meta-service/meta_service_tablet_stats.cpp +++ b/cloud/src/meta-service/meta_service_tablet_stats.cpp @@ -45,7 +45,8 @@ void internal_get_tablet_stats(MetaServiceCode& code, std::string& msg, Transact auto end_key = stats_tablet_key({instance_id, idx.table_id(), idx.index_id(), idx.partition_id(), idx.tablet_id() + 1}); // clang-format on std::vector> stats_kvs; - stats_kvs.reserve(5); // aggregate + data_size + num_rows + num_rowsets + num_segments + stats_kvs.reserve( + 7); // aggregate + data_size + num_rows + num_rowsets + num_segments + index_size + segment_size std::unique_ptr it; do { @@ -93,7 +94,7 @@ int get_detached_tablet_stats(const std::vector int64 + std::string tablet_stat_index_size_key; + stats_tablet_index_size_key( + {instance_id, tablet_stat.idx().table_id(), tablet_stat.idx().index_id(), + tablet_stat.idx().partition_id(), tablet_stat.idx().tablet_id()}, + &tablet_stat_index_size_key); + int64_t tablet_stat_index_size = 0; + std::string tablet_stat_index_size_value(sizeof(tablet_stat_index_size), '\0'); + memcpy(tablet_stat_index_size_value.data(), &tablet_stat_index_size, + sizeof(tablet_stat_index_size)); + txn->put(tablet_stat_index_size_key, tablet_stat_index_size_value); + // set tablet stats segment size = 0 + // 0x01 "stats" ${instance_id} "tablet" ${table_id} ${index_id} ${partition_id} ${tablet_id} "segment_size" -> int64 + std::string tablet_stat_segment_size_key; + stats_tablet_segment_size_key( + {instance_id, tablet_stat.idx().table_id(), tablet_stat.idx().index_id(), + tablet_stat.idx().partition_id(), tablet_stat.idx().tablet_id()}, + &tablet_stat_segment_size_key); + int64_t tablet_stat_segment_size = 0; + std::string tablet_stat_segment_size_value(sizeof(tablet_stat_segment_size), '\0'); + memcpy(tablet_stat_segment_size_value.data(), &tablet_stat_segment_size, + sizeof(tablet_stat_segment_size)); + txn->put(tablet_stat_segment_size_key, tablet_stat_segment_size_value); } } @@ -396,6 +433,76 @@ MetaServiceResponseStatus check_new_tablet_stats( .tag("data size", tablet_stat_data_size) .tag("check data size", tablet_stat_data_size_check); } + + // check index size + std::string tablet_stat_index_size_key; + stats_tablet_index_size_key( + {instance_id, tablet_stat_ptr->idx().table_id(), tablet_stat_ptr->idx().index_id(), + tablet_stat_ptr->idx().partition_id(), tablet_stat_ptr->idx().tablet_id()}, + &tablet_stat_index_size_key); + int64_t tablet_stat_index_size = 0; + std::string tablet_stat_index_size_value(sizeof(tablet_stat_index_size), '\0'); + err = txn->get(tablet_stat_index_size_key, &tablet_stat_index_size_value); + if (err != TxnErrorCode::TXN_OK && err != TxnErrorCode::TXN_KEY_NOT_FOUND) { + st.set_code(cast_as(err)); + return st; + } + int64_t tablet_stat_index_size_check; + + if (tablet_stat_index_size_value.size() != sizeof(tablet_stat_index_size_check)) + [[unlikely]] { + LOG(WARNING) << " malformed tablet stats value v.index_size=" + << tablet_stat_index_size_value.size() + << " value=" << hex(tablet_stat_index_size_value); + } + std::memcpy(&tablet_stat_index_size_check, tablet_stat_index_size_value.data(), + sizeof(tablet_stat_index_size_check)); + if constexpr (std::endian::native == std::endian::big) { + tablet_stat_index_size_check = bswap_64(tablet_stat_index_size_check); + } + if (tablet_stat_index_size_check != tablet_stat_index_size && + // ditto + (tablet_stat_index_size_check > 2 * tablet_stat_index_size || + tablet_stat_index_size > 2 * tablet_stat_index_size_check)) { + LOG_WARNING("[fix tablet stats]:index size check failed") + .tag("index size", tablet_stat_index_size) + .tag("check index size", tablet_stat_index_size_check); + } + + // check data size + std::string tablet_stat_segment_size_key; + stats_tablet_segment_size_key( + {instance_id, tablet_stat_ptr->idx().table_id(), tablet_stat_ptr->idx().index_id(), + tablet_stat_ptr->idx().partition_id(), tablet_stat_ptr->idx().tablet_id()}, + &tablet_stat_segment_size_key); + int64_t tablet_stat_segment_size = 0; + std::string tablet_stat_segment_size_value(sizeof(tablet_stat_segment_size), '\0'); + err = txn->get(tablet_stat_segment_size_key, &tablet_stat_segment_size_value); + if (err != TxnErrorCode::TXN_OK && err != TxnErrorCode::TXN_KEY_NOT_FOUND) { + st.set_code(cast_as(err)); + return st; + } + int64_t tablet_stat_segment_size_check; + + if (tablet_stat_segment_size_value.size() != sizeof(tablet_stat_segment_size_check)) + [[unlikely]] { + LOG(WARNING) << " malformed tablet stats value v.segment_size=" + << tablet_stat_segment_size_value.size() + << " value=" << hex(tablet_stat_segment_size_value); + } + std::memcpy(&tablet_stat_segment_size_check, tablet_stat_segment_size_value.data(), + sizeof(tablet_stat_segment_size_check)); + if constexpr (std::endian::native == std::endian::big) { + tablet_stat_segment_size_check = bswap_64(tablet_stat_segment_size_check); + } + if (tablet_stat_segment_size_check != tablet_stat_segment_size && + // ditto + (tablet_stat_segment_size_check > 2 * tablet_stat_segment_size || + tablet_stat_segment_size > 2 * tablet_stat_segment_size_check)) { + LOG_WARNING("[fix tablet stats]:segment size check failed") + .tag("segment size", tablet_stat_segment_size) + .tag("check segment size", tablet_stat_segment_size_check); + } } return st; diff --git a/cloud/src/meta-service/meta_service_tablet_stats.h b/cloud/src/meta-service/meta_service_tablet_stats.h index a7aea5885a8e1a..30cbaf56f6dc80 100644 --- a/cloud/src/meta-service/meta_service_tablet_stats.h +++ b/cloud/src/meta-service/meta_service_tablet_stats.h @@ -31,6 +31,8 @@ struct TabletStats { int64_t num_rows = 0; int64_t num_rowsets = 0; int64_t num_segs = 0; + int64_t index_size = 0; + int64_t segment_size = 0; }; // Get tablet stats and detached tablet stats via `txn`. If an error occurs, `code` will be set to non OK. diff --git a/cloud/src/meta-service/meta_service_txn.cpp b/cloud/src/meta-service/meta_service_txn.cpp index 58930f6edfc63b..5d696220b72dab 100644 --- a/cloud/src/meta-service/meta_service_txn.cpp +++ b/cloud/src/meta-service/meta_service_txn.cpp @@ -880,6 +880,12 @@ void update_tablet_stats(const StatsTabletKeyInfo& info, const TabletStats& stat std::string num_segs_key; stats_tablet_num_segs_key(info, &num_segs_key); txn->atomic_add(num_segs_key, stats.num_segs); + std::string index_size_key; + stats_tablet_index_size_key(info, &index_size_key); + txn->atomic_add(index_size_key, stats.index_size); + std::string segment_size_key; + stats_tablet_segment_size_key(info, &segment_size_key); + txn->atomic_add(segment_size_key, stats.segment_size); } std::string num_rowsets_key; stats_tablet_num_rowsets_key(info, &num_rowsets_key); @@ -906,6 +912,8 @@ void update_tablet_stats(const StatsTabletKeyInfo& info, const TabletStats& stat stats_pb.set_num_rows(stats_pb.num_rows() + stats.num_rows); stats_pb.set_num_rowsets(stats_pb.num_rowsets() + stats.num_rowsets); stats_pb.set_num_segments(stats_pb.num_segments() + stats.num_segs); + stats_pb.set_index_size(stats_pb.index_size() + stats.index_size); + stats_pb.set_segment_size(stats_pb.segment_size() + stats.segment_size); stats_pb.SerializeToString(&val); txn->put(key, val); LOG(INFO) << "put stats_tablet_key key=" << hex(key); @@ -1234,6 +1242,8 @@ void commit_txn_immediately( stats.num_rows += i.num_rows(); ++stats.num_rowsets; stats.num_segs += i.num_segments(); + stats.index_size += i.index_disk_size(); + stats.segment_size += i.data_disk_size(); } // for tmp_rowsets_meta process_mow_when_commit_txn(request, instance_id, code, msg, txn, table_id_tablet_ids); @@ -1864,10 +1874,12 @@ void commit_txn_eventually( for (auto& [_, i] : tmp_rowsets_meta) { // Accumulate affected rows auto& stats = tablet_stats[i.tablet_id()]; - stats.data_size += i.data_disk_size(); + stats.data_size += i.total_disk_size(); stats.num_rows += i.num_rows(); ++stats.num_rowsets; stats.num_segs += i.num_segments(); + stats.index_size += i.index_disk_size(); + stats.segment_size += i.data_disk_size(); } // calculate table stats from tablets stats @@ -2238,10 +2250,12 @@ void commit_txn_with_sub_txn(const CommitTxnRequest* request, CommitTxnResponse* // Accumulate affected rows auto& stats = tablet_stats[tablet_id]; - stats.data_size += i.data_disk_size(); + stats.data_size += i.total_disk_size(); stats.num_rows += i.num_rows(); ++stats.num_rowsets; stats.num_segs += i.num_segments(); + stats.index_size += i.index_disk_size(); + stats.segment_size += i.data_disk_size(); } // for tmp_rowsets_meta } @@ -2351,6 +2365,12 @@ void commit_txn_with_sub_txn(const CommitTxnRequest* request, CommitTxnResponse* auto& num_segs_key = kv_pool.emplace_back(); stats_tablet_num_segs_key(info, &num_segs_key); txn->atomic_add(num_segs_key, stats.num_segs); + auto& index_size_key = kv_pool.emplace_back(); + stats_tablet_index_size_key(info, &index_size_key); + txn->atomic_add(index_size_key, stats.index_size); + auto& segment_size_key = kv_pool.emplace_back(); + stats_tablet_segment_size_key(info, &segment_size_key); + txn->atomic_add(segment_size_key, stats.segment_size); } auto& num_rowsets_key = kv_pool.emplace_back(); stats_tablet_num_rowsets_key(info, &num_rowsets_key); @@ -2379,6 +2399,8 @@ void commit_txn_with_sub_txn(const CommitTxnRequest* request, CommitTxnResponse* stats_pb.set_num_rows(stats_pb.num_rows() + stats.num_rows); stats_pb.set_num_rowsets(stats_pb.num_rowsets() + stats.num_rowsets); stats_pb.set_num_segments(stats_pb.num_segments() + stats.num_segs); + stats_pb.set_index_size(stats_pb.index_size() + stats.index_size); + stats_pb.set_segment_size(stats_pb.segment_size() + stats.segment_size); stats_pb.SerializeToString(&val); txn->put(key, val); LOG(INFO) << "put stats_tablet_key, key=" << hex(key); diff --git a/cloud/src/meta-service/txn_lazy_committer.cpp b/cloud/src/meta-service/txn_lazy_committer.cpp index 25d36faab06796..9859c2b0ed1ae3 100644 --- a/cloud/src/meta-service/txn_lazy_committer.cpp +++ b/cloud/src/meta-service/txn_lazy_committer.cpp @@ -174,10 +174,12 @@ void convert_tmp_rowsets( // Accumulate affected rows auto& stats = tablet_stats[tmp_rowset_pb.tablet_id()]; - stats.data_size += tmp_rowset_pb.data_disk_size(); + stats.data_size += tmp_rowset_pb.total_disk_size(); stats.num_rows += tmp_rowset_pb.num_rows(); ++stats.num_rowsets; stats.num_segs += tmp_rowset_pb.num_segments(); + stats.index_size += tmp_rowset_pb.index_disk_size(); + stats.segment_size += tmp_rowset_pb.data_disk_size(); } for (auto& [tablet_id, stats] : tablet_stats) { diff --git a/cloud/test/http_encode_key_test.cpp b/cloud/test/http_encode_key_test.cpp index 77f800846076a9..d245682474349d 100644 --- a/cloud/test/http_encode_key_test.cpp +++ b/cloud/test/http_encode_key_test.cpp @@ -310,7 +310,7 @@ txn_id=126419752960)", }, R"({"creation_time":"12345","label":"label_1"})", }, - Input { // aggregated_stats + full detached_stats, there are 5 KVs in total + Input { // aggregated_stats + full detached_stats, there are 7 KVs in total "StatsTabletKey", "instance_id=gavin-instance&table_id=10086&index_id=10010&part_id=10000&tablet_id=1008601", { @@ -327,15 +327,17 @@ txn_id=126419752960)", idx->set_index_id(100010); idx->set_partition_id(10000); idx->set_tablet_id(1008601); - pb.set_data_size(1); + pb.set_data_size(2); pb.set_num_rows(10); pb.set_num_rowsets(11); pb.set_num_segments(12); + pb.set_index_size(1); + pb.set_segment_size(1); return {pb.SerializeAsString(), {"\x01\x00\x00\x00\x00\x00\x00\x00",8}, {"\x02\x00\x00\x00\x00\x00\x00\x00",8}, {"\x03\x00\x00\x00\x00\x00\x00\x00",8}, {"\x04\x00\x00\x00\x00\x00\x00\x00",8}}; }, - R"(aggregated_stats: {"idx":{"table_id":"10086","index_id":"100010","partition_id":"10000","tablet_id":"1008601"},"data_size":"1","num_rows":"10","num_rowsets":"11","num_segments":"12"} -detached_stats: {"data_size":"1","num_rows":"2","num_rowsets":"3","num_segments":"4"} -merged_stats: {"idx":{"table_id":"10086","index_id":"100010","partition_id":"10000","tablet_id":"1008601"},"data_size":"2","num_rows":"12","num_rowsets":"14","num_segments":"16"} + R"(aggregated_stats: {"idx":{"table_id":"10086","index_id":"100010","partition_id":"10000","tablet_id":"1008601"},"data_size":"2","num_rows":"10","num_rowsets":"11","num_segments":"12","index_size":"1","segment_size":"1"} +detached_stats: {"data_size":"1","num_rows":"2","num_rowsets":"3","num_segments":"4","index_size":"0","segment_size":"0"} +merged_stats: {"idx":{"table_id":"10086","index_id":"100010","partition_id":"10000","tablet_id":"1008601"},"data_size":"3","num_rows":"12","num_rowsets":"14","num_segments":"16","index_size":"1","segment_size":"1"} )", }, Input { // aggregated_stats + half detached_stats (num_segs == 0, there is num_rowsets detached stats) @@ -352,15 +354,17 @@ merged_stats: {"idx":{"table_id":"10086","index_id":"100010","partition_id":"100 idx->set_index_id(100010); idx->set_partition_id(10000); idx->set_tablet_id(1008602); - pb.set_data_size(1); + pb.set_data_size(2); pb.set_num_rows(10); pb.set_num_rowsets(11); pb.set_num_segments(12); + pb.set_index_size(1); + pb.set_segment_size(1); return {pb.SerializeAsString(), {"\x03\x00\x00\x00\x00\x00\x00\x00",8}}; }, - R"(aggregated_stats: {"idx":{"table_id":"10086","index_id":"100010","partition_id":"10000","tablet_id":"1008602"},"data_size":"1","num_rows":"10","num_rowsets":"11","num_segments":"12"} -detached_stats: {"data_size":"0","num_rows":"0","num_rowsets":"3","num_segments":"0"} -merged_stats: {"idx":{"table_id":"10086","index_id":"100010","partition_id":"10000","tablet_id":"1008602"},"data_size":"1","num_rows":"10","num_rowsets":"14","num_segments":"12"} + R"(aggregated_stats: {"idx":{"table_id":"10086","index_id":"100010","partition_id":"10000","tablet_id":"1008602"},"data_size":"2","num_rows":"10","num_rowsets":"11","num_segments":"12","index_size":"1","segment_size":"1"} +detached_stats: {"data_size":"0","num_rows":"0","num_rowsets":"3","num_segments":"0","index_size":"0","segment_size":"0"} +merged_stats: {"idx":{"table_id":"10086","index_id":"100010","partition_id":"10000","tablet_id":"1008602"},"data_size":"2","num_rows":"10","num_rowsets":"14","num_segments":"12","index_size":"1","segment_size":"1"} )", }, Input { // aggregated_stats only, the legacy @@ -376,15 +380,17 @@ merged_stats: {"idx":{"table_id":"10086","index_id":"100010","partition_id":"100 idx->set_index_id(100010); idx->set_partition_id(10000); idx->set_tablet_id(1008602); - pb.set_data_size(1); + pb.set_data_size(2); pb.set_num_rows(10); pb.set_num_rowsets(11); pb.set_num_segments(12); + pb.set_index_size(1); + pb.set_segment_size(1); return {pb.SerializeAsString()}; }, - R"(aggregated_stats: {"idx":{"table_id":"10086","index_id":"100010","partition_id":"10000","tablet_id":"1008602"},"data_size":"1","num_rows":"10","num_rowsets":"11","num_segments":"12"} -detached_stats: {"data_size":"0","num_rows":"0","num_rowsets":"0","num_segments":"0"} -merged_stats: {"idx":{"table_id":"10086","index_id":"100010","partition_id":"10000","tablet_id":"1008602"},"data_size":"1","num_rows":"10","num_rowsets":"11","num_segments":"12"} + R"(aggregated_stats: {"idx":{"table_id":"10086","index_id":"100010","partition_id":"10000","tablet_id":"1008602"},"data_size":"2","num_rows":"10","num_rowsets":"11","num_segments":"12","index_size":"1","segment_size":"1"} +detached_stats: {"data_size":"0","num_rows":"0","num_rowsets":"0","num_segments":"0","index_size":"0","segment_size":"0"} +merged_stats: {"idx":{"table_id":"10086","index_id":"100010","partition_id":"10000","tablet_id":"1008602"},"data_size":"2","num_rows":"10","num_rowsets":"11","num_segments":"12","index_size":"1","segment_size":"1"} )", }, Input { diff --git a/cloud/test/keys_test.cpp b/cloud/test/keys_test.cpp index a92a685cda45f8..52d840e1f5ce6b 100644 --- a/cloud/test/keys_test.cpp +++ b/cloud/test/keys_test.cpp @@ -803,6 +803,40 @@ TEST(KeysTest, StatsKeysTest) { EXPECT_EQ("num_segs", dec_stats_suffix); } + + // 0x01 "stats" ${instance_id} "tablet" ${table_id} ${index_id} ${partition_id} ${tablet_id} "index_size" -> int64 + { + StatsTabletKeyInfo stats_key {instance_id, table_id, index_id, partition_id, tablet_id}; + std::string encoded_stats_key0; + stats_tablet_index_size_key(stats_key, &encoded_stats_key0); + std::cout << hex(encoded_stats_key0) << std::endl; + + std::string dec_stats_suffix; + + std::string_view key_sv(encoded_stats_key0); + expect_stats_prefix(key_sv); + ASSERT_EQ(decode_bytes(&key_sv, &dec_stats_suffix), 0); + ASSERT_TRUE(key_sv.empty()); + + EXPECT_EQ("index_size", dec_stats_suffix); + } + + // 0x01 "stats" ${instance_id} "tablet" ${table_id} ${index_id} ${partition_id} ${tablet_id} "segment_size" -> int64 + { + StatsTabletKeyInfo stats_key {instance_id, table_id, index_id, partition_id, tablet_id}; + std::string encoded_stats_key0; + stats_tablet_segment_size_key(stats_key, &encoded_stats_key0); + std::cout << hex(encoded_stats_key0) << std::endl; + + std::string dec_stats_suffix; + + std::string_view key_sv(encoded_stats_key0); + expect_stats_prefix(key_sv); + ASSERT_EQ(decode_bytes(&key_sv, &dec_stats_suffix), 0); + ASSERT_TRUE(key_sv.empty()); + + EXPECT_EQ("segment_size", dec_stats_suffix); + } } TEST(KeysTest, JobKeysTest) { diff --git a/cloud/test/meta_service_http_test.cpp b/cloud/test/meta_service_http_test.cpp index 4360efeb4422a9..81c322303a5a8c 100644 --- a/cloud/test/meta_service_http_test.cpp +++ b/cloud/test/meta_service_http_test.cpp @@ -1257,6 +1257,8 @@ TEST(MetaServiceHttpTest, GetTabletStatsTest) { EXPECT_EQ(res.tablet_stats(0).num_rows(), 0); EXPECT_EQ(res.tablet_stats(0).num_rowsets(), 1); EXPECT_EQ(res.tablet_stats(0).num_segments(), 0); + EXPECT_EQ(res.tablet_stats(0).index_size(), 0); + EXPECT_EQ(res.tablet_stats(0).segment_size(), 0); { GetTabletStatsRequest req; auto idx = req.add_tablet_idx(); @@ -1288,6 +1290,16 @@ TEST(MetaServiceHttpTest, GetTabletStatsTest) { &data_size_key); ASSERT_EQ(txn->get(data_size_key, &data_size_val), TxnErrorCode::TXN_OK); EXPECT_EQ(*(int64_t*)data_size_val.data(), 22000); + std::string index_size_key, index_size_val; + stats_tablet_index_size_key({mock_instance, table_id, index_id, partition_id, tablet_id}, + &index_size_key); + ASSERT_EQ(txn->get(index_size_key, &index_size_val), TxnErrorCode::TXN_OK); + EXPECT_EQ(*(int64_t*)index_size_val.data(), 2000); + std::string segment_size_key, segment_size_val; + stats_tablet_segment_size_key({mock_instance, table_id, index_id, partition_id, tablet_id}, + &segment_size_key); + ASSERT_EQ(txn->get(segment_size_key, &segment_size_val), TxnErrorCode::TXN_OK); + EXPECT_EQ(*(int64_t*)segment_size_val.data(), 20000); std::string num_rows_key, num_rows_val; stats_tablet_num_rows_key({mock_instance, table_id, index_id, partition_id, tablet_id}, &num_rows_key); @@ -1312,6 +1324,8 @@ TEST(MetaServiceHttpTest, GetTabletStatsTest) { EXPECT_EQ(res.tablet_stats(0).num_rows(), 400); EXPECT_EQ(res.tablet_stats(0).num_rowsets(), 5); EXPECT_EQ(res.tablet_stats(0).num_segments(), 4); + EXPECT_EQ(res.tablet_stats(0).index_size(), 4000); + EXPECT_EQ(res.tablet_stats(0).segment_size(), 40000); { GetTabletStatsRequest req; auto idx = req.add_tablet_idx(); diff --git a/cloud/test/meta_service_job_test.cpp b/cloud/test/meta_service_job_test.cpp index f0323eebb790be..e1dbb0089e2e73 100644 --- a/cloud/test/meta_service_job_test.cpp +++ b/cloud/test/meta_service_job_test.cpp @@ -95,7 +95,9 @@ doris::RowsetMetaCloudPB create_rowset(int64_t tablet_id, int64_t start_version, rowset.set_end_version(end_version); rowset.set_num_segments(1); rowset.set_num_rows(num_rows); - rowset.set_data_disk_size(num_rows * 100); + rowset.set_total_disk_size(num_rows * 100); + rowset.set_data_disk_size(num_rows * 50); + rowset.set_index_disk_size(num_rows * 50); rowset.mutable_tablet_schema()->set_schema_version(0); rowset.set_txn_expiration(::time(nullptr)); // Required by DCHECK return rowset; @@ -114,11 +116,13 @@ void insert_rowsets(TxnKv* txn_kv, int64_t table_id, int64_t index_id, int64_t p std::unique_ptr txn; ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK) << tablet_id; std::deque buffer; - int data_size = 0, num_rows = 0, num_seg = 0; + int data_size = 0, num_rows = 0, num_seg = 0, index_size = 0, segment_size = 0; for (auto& rowset : rowsets) { - data_size += rowset.data_disk_size(); + data_size += rowset.total_disk_size(); num_rows += rowset.num_rows(); num_seg += rowset.num_segments(); + index_size += rowset.index_disk_size(); + segment_size += rowset.data_disk_size(); auto& key = buffer.emplace_back(); auto& val = buffer.emplace_back(); meta_rowset_key({instance_id, tablet_id, rowset.end_version()}, &key); @@ -138,6 +142,12 @@ void insert_rowsets(TxnKv* txn_kv, int64_t table_id, int64_t index_id, int64_t p std::string num_segs_key; stats_tablet_num_segs_key(info, &num_segs_key); txn->atomic_add(num_segs_key, num_seg); + std::string index_size_key; + stats_tablet_index_size_key(info, &index_size_key); + txn->atomic_add(index_size_key, index_size); + std::string segment_size_key; + stats_tablet_segment_size_key(info, &segment_size_key); + txn->atomic_add(segment_size_key, segment_size); ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK) << tablet_id; } @@ -241,7 +251,11 @@ void finish_schema_change_job( sc->add_output_versions(rowset.end_version()); sc->set_num_output_rows(sc->num_output_rows() + rowset.num_rows()); sc->set_num_output_segments(sc->num_output_segments() + rowset.num_segments()); - sc->set_size_output_rowsets(sc->size_output_rowsets() + rowset.data_disk_size()); + sc->set_size_output_rowsets(sc->size_output_rowsets() + rowset.total_disk_size()); + sc->set_index_size_output_rowsets(sc->index_size_output_rowsets() + + rowset.index_disk_size()); + sc->set_segment_size_output_rowsets(sc->segment_size_output_rowsets() + + rowset.data_disk_size()); } sc->set_num_output_rowsets(output_rowsets.size()); } @@ -853,6 +867,10 @@ TEST(MetaServiceJobTest, CompactionJobTest) { compaction->set_num_input_segments(dist(rng)); compaction->set_size_input_rowsets(dist(rng)); compaction->set_size_output_rowsets(dist(rng)); + compaction->set_index_size_input_rowsets(dist(rng)); + compaction->set_segment_size_output_rowsets(dist(rng)); + compaction->set_index_size_input_rowsets(dist(rng)); + compaction->set_segment_size_output_rowsets(dist(rng)); compaction->set_type(type); tablet_stats_pb.set_cumulative_compaction_cnt(dist(rng)); @@ -863,6 +881,8 @@ TEST(MetaServiceJobTest, CompactionJobTest) { tablet_stats_pb.set_data_size(dist(rng) + compaction->size_input_rowsets()); tablet_stats_pb.set_num_rowsets(dist(rng) + compaction->num_input_rowsets()); tablet_stats_pb.set_num_segments(dist(rng) + compaction->num_input_segments()); + tablet_stats_pb.set_index_size(dist(rng) + compaction->index_size_input_rowsets()); + tablet_stats_pb.set_segment_size(dist(rng) + compaction->segment_size_input_rowsets()); std::string tablet_stats_val = tablet_stats_pb.SerializeAsString(); ASSERT_FALSE(tablet_stats_val.empty()); @@ -981,6 +1001,8 @@ TEST(MetaServiceJobTest, CompactionJobTest) { EXPECT_EQ(stats.data_size() , tablet_stats_pb.data_size() + (req.job().compaction(0).size_output_rowsets() - req.job().compaction(0).size_input_rowsets())); EXPECT_EQ(stats.num_rowsets() , tablet_stats_pb.num_rowsets() + (req.job().compaction(0).num_output_rowsets() - req.job().compaction(0).num_input_rowsets())); EXPECT_EQ(stats.num_segments() , tablet_stats_pb.num_segments() + (req.job().compaction(0).num_output_segments() - req.job().compaction(0).num_input_segments())); + EXPECT_EQ(stats.index_size() , tablet_stats_pb.index_size() + (req.job().compaction(0).index_size_output_rowsets() - req.job().compaction(0).index_size_input_rowsets())); + EXPECT_EQ(stats.segment_size() , tablet_stats_pb.segment_size() + (req.job().compaction(0).segment_size_output_rowsets() - req.job().compaction(0).segment_size_input_rowsets())); // clang-format on // Check job removed, tablet meta updated @@ -1157,6 +1179,8 @@ TEST(MetaServiceJobTest, CompactionJobWithMoWTest) { tablet_stats_pb.set_data_size(dist(rng) + compaction->size_input_rowsets()); tablet_stats_pb.set_num_rowsets(dist(rng) + compaction->num_input_rowsets()); tablet_stats_pb.set_num_segments(dist(rng) + compaction->num_input_segments()); + tablet_stats_pb.set_index_size(dist(rng) + compaction->index_size_input_rowsets()); + tablet_stats_pb.set_segment_size(dist(rng) + compaction->segment_size_input_rowsets()); std::string tablet_stats_val = tablet_stats_pb.SerializeAsString(); ASSERT_FALSE(tablet_stats_val.empty()); @@ -1391,12 +1415,16 @@ TEST(MetaServiceJobTest, SchemaChangeJobTest) { EXPECT_EQ(res.stats().num_rowsets(), 6); EXPECT_EQ(res.stats().num_segments(), 5); EXPECT_EQ(res.stats().data_size(), 50000); + EXPECT_EQ(res.stats().index_size(), 25000); + EXPECT_EQ(res.stats().segment_size(), 25000); TabletStatsPB tablet_stats; get_tablet_stats(meta_service.get(), new_tablet_id, tablet_stats); EXPECT_EQ(tablet_stats.num_rows(), 500); EXPECT_EQ(tablet_stats.num_rowsets(), 6); EXPECT_EQ(tablet_stats.num_segments(), 5); EXPECT_EQ(tablet_stats.data_size(), 50000); + EXPECT_EQ(tablet_stats.index_size(), 25000); + EXPECT_EQ(tablet_stats.segment_size(), 25000); std::unique_ptr txn; ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); @@ -1465,12 +1493,16 @@ TEST(MetaServiceJobTest, SchemaChangeJobTest) { EXPECT_EQ(res.stats().num_rowsets(), 6); EXPECT_EQ(res.stats().num_segments(), 5); EXPECT_EQ(res.stats().data_size(), 50000); + EXPECT_EQ(res.stats().index_size(), 25000); + EXPECT_EQ(res.stats().segment_size(), 25000); TabletStatsPB tablet_stats; get_tablet_stats(meta_service.get(), new_tablet_id, tablet_stats); EXPECT_EQ(tablet_stats.num_rows(), 500); EXPECT_EQ(tablet_stats.num_rowsets(), 6); EXPECT_EQ(tablet_stats.num_segments(), 5); EXPECT_EQ(tablet_stats.data_size(), 50000); + EXPECT_EQ(tablet_stats.index_size(), 25000); + EXPECT_EQ(tablet_stats.segment_size(), 25000); std::unique_ptr txn; ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); @@ -1628,12 +1660,16 @@ TEST(MetaServiceJobTest, RetrySchemaChangeJobTest) { EXPECT_EQ(res.stats().num_rowsets(), 6); EXPECT_EQ(res.stats().num_segments(), 5); EXPECT_EQ(res.stats().data_size(), 50000); + EXPECT_EQ(res.stats().index_size(), 25000); + EXPECT_EQ(res.stats().segment_size(), 25000); TabletStatsPB tablet_stats; get_tablet_stats(meta_service.get(), new_tablet_id, tablet_stats); EXPECT_EQ(tablet_stats.num_rows(), 500); EXPECT_EQ(tablet_stats.num_rowsets(), 6); EXPECT_EQ(tablet_stats.num_segments(), 5); EXPECT_EQ(tablet_stats.data_size(), 50000); + EXPECT_EQ(tablet_stats.index_size(), 25000); + EXPECT_EQ(tablet_stats.segment_size(), 25000); std::unique_ptr txn; ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); @@ -1942,10 +1978,14 @@ TEST(MetaServiceJobTest, ConcurrentCompactionTest) { compaction->add_output_rowset_ids(output_rowset.rowset_id_v2()); compaction->set_output_cumulative_point(11); compaction->set_size_input_rowsets(60000); + compaction->set_index_size_input_rowsets(30000); + compaction->set_segment_size_input_rowsets(30000); compaction->set_num_input_rows(600); compaction->set_num_input_rowsets(6); compaction->set_num_input_segments(6); compaction->set_size_output_rowsets(10000); + compaction->set_index_size_output_rowsets(5000); + compaction->set_segment_size_output_rowsets(5000); compaction->set_num_output_rows(100); compaction->set_num_output_rowsets(1); compaction->set_num_output_segments(1); @@ -1958,12 +1998,16 @@ TEST(MetaServiceJobTest, ConcurrentCompactionTest) { EXPECT_EQ(res.stats().num_rowsets(), 6); EXPECT_EQ(res.stats().num_segments(), 5); EXPECT_EQ(res.stats().data_size(), 50000); + EXPECT_EQ(res.stats().index_size(), 25000); + EXPECT_EQ(res.stats().segment_size(), 25000); TabletStatsPB tablet_stats; get_tablet_stats(meta_service.get(), tablet_id, tablet_stats); EXPECT_EQ(tablet_stats.num_rows(), 500); EXPECT_EQ(tablet_stats.num_rowsets(), 6); EXPECT_EQ(tablet_stats.num_segments(), 5); EXPECT_EQ(tablet_stats.data_size(), 50000); + EXPECT_EQ(tablet_stats.index_size(), 25000); + EXPECT_EQ(tablet_stats.segment_size(), 25000); ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); // Check tmp rowsets @@ -2046,10 +2090,14 @@ TEST(MetaServiceJobTest, ConcurrentCompactionTest) { compaction->add_output_rowset_ids(output_rowset.rowset_id_v2()); compaction->set_output_cumulative_point(5); compaction->set_size_input_rowsets(30000); + compaction->set_index_size_input_rowsets(15000); + compaction->set_segment_size_input_rowsets(15000); compaction->set_num_input_rows(300); compaction->set_num_input_rowsets(3); compaction->set_num_input_segments(3); compaction->set_size_output_rowsets(10000); + compaction->set_index_size_output_rowsets(5000); + compaction->set_segment_size_output_rowsets(5000); compaction->set_num_output_rows(100); compaction->set_num_output_rowsets(1); compaction->set_num_output_segments(1); @@ -2062,12 +2110,16 @@ TEST(MetaServiceJobTest, ConcurrentCompactionTest) { EXPECT_EQ(res.stats().num_rowsets(), 4); EXPECT_EQ(res.stats().num_segments(), 3); EXPECT_EQ(res.stats().data_size(), 30000); + EXPECT_EQ(res.stats().index_size(), 15000); + EXPECT_EQ(res.stats().segment_size(), 15000); TabletStatsPB tablet_stats; get_tablet_stats(meta_service.get(), tablet_id, tablet_stats); EXPECT_EQ(tablet_stats.num_rows(), 300); EXPECT_EQ(tablet_stats.num_rowsets(), 4); EXPECT_EQ(tablet_stats.num_segments(), 3); EXPECT_EQ(tablet_stats.data_size(), 30000); + EXPECT_EQ(tablet_stats.index_size(), 15000); + EXPECT_EQ(tablet_stats.segment_size(), 15000); ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); // Check tmp rowsets diff --git a/cloud/test/meta_service_test.cpp b/cloud/test/meta_service_test.cpp index 1f93b228c6de08..d2dd80f6871e3e 100644 --- a/cloud/test/meta_service_test.cpp +++ b/cloud/test/meta_service_test.cpp @@ -4429,6 +4429,8 @@ TEST(MetaServiceTest, GetTabletStatsTest) { EXPECT_EQ(res.tablet_stats(0).num_rows(), 0); EXPECT_EQ(res.tablet_stats(0).num_rowsets(), 1); EXPECT_EQ(res.tablet_stats(0).num_segments(), 0); + EXPECT_EQ(res.tablet_stats(0).index_size(), 0); + EXPECT_EQ(res.tablet_stats(0).segment_size(), 0); // Insert rowset config::split_tablet_stats = false; ASSERT_NO_FATAL_FAILURE( @@ -4448,6 +4450,16 @@ TEST(MetaServiceTest, GetTabletStatsTest) { &data_size_key); ASSERT_EQ(txn->get(data_size_key, &data_size_val), TxnErrorCode::TXN_OK); EXPECT_EQ(*(int64_t*)data_size_val.data(), 22000); + std::string index_size_key, index_size_val; + stats_tablet_index_size_key({mock_instance, table_id, index_id, partition_id, tablet_id}, + &index_size_key); + ASSERT_EQ(txn->get(index_size_key, &index_size_val), TxnErrorCode::TXN_OK); + EXPECT_EQ(*(int64_t*)index_size_val.data(), 2000); + std::string segment_size_key, segment_size_val; + stats_tablet_segment_size_key({mock_instance, table_id, index_id, partition_id, tablet_id}, + &segment_size_key); + ASSERT_EQ(txn->get(segment_size_key, &segment_size_val), TxnErrorCode::TXN_OK); + EXPECT_EQ(*(int64_t*)segment_size_val.data(), 20000); std::string num_rows_key, num_rows_val; stats_tablet_num_rows_key({mock_instance, table_id, index_id, partition_id, tablet_id}, &num_rows_key); @@ -4472,6 +4484,8 @@ TEST(MetaServiceTest, GetTabletStatsTest) { EXPECT_EQ(res.tablet_stats(0).num_rows(), 400); EXPECT_EQ(res.tablet_stats(0).num_rowsets(), 5); EXPECT_EQ(res.tablet_stats(0).num_segments(), 4); + EXPECT_EQ(res.tablet_stats(0).index_size(), 4000); + EXPECT_EQ(res.tablet_stats(0).segment_size(), 40000); } TEST(MetaServiceTest, GetDeleteBitmapUpdateLock) { @@ -7771,6 +7785,8 @@ TEST(MetaServiceTest, UpdateTmpRowsetTest) { // simulate that there are new segments added to this rowset rowset.set_num_segments(rowset.num_segments() + 3); rowset.set_num_rows(rowset.num_rows() + 1000); + rowset.set_total_disk_size(rowset.total_disk_size() + 11000); + rowset.set_index_disk_size(rowset.index_disk_size() + 1000); rowset.set_data_disk_size(rowset.data_disk_size() + 10000); ASSERT_NO_FATAL_FAILURE(update_tmp_rowset(meta_service.get(), rowset, res)); @@ -7789,6 +7805,8 @@ TEST(MetaServiceTest, UpdateTmpRowsetTest) { ASSERT_EQ(doris::BEGIN_PARTIAL_UPDATE, fetchedRowsetMeta.rowset_state()); ASSERT_EQ(rowset.num_segments(), fetchedRowsetMeta.num_segments()); ASSERT_EQ(rowset.num_rows(), fetchedRowsetMeta.num_rows()); + ASSERT_EQ(rowset.total_disk_size(), fetchedRowsetMeta.total_disk_size()); + ASSERT_EQ(rowset.index_disk_size(), fetchedRowsetMeta.index_disk_size()); ASSERT_EQ(rowset.data_disk_size(), fetchedRowsetMeta.data_disk_size()); ASSERT_NO_FATAL_FAILURE(commit_txn(meta_service.get(), db_id, txn_id, label)); @@ -7819,6 +7837,8 @@ TEST(MetaServiceTest, UpdateTmpRowsetTest) { // simulate that there are new segments added to this rowset rowset.set_num_segments(rowset.num_segments() + 3); rowset.set_num_rows(rowset.num_rows() + 1000); + rowset.set_total_disk_size(rowset.total_disk_size() + 11000); + rowset.set_index_disk_size(rowset.index_disk_size() + 1000); rowset.set_data_disk_size(rowset.data_disk_size() + 10000); // repeated calls to update_tmp_rowset will all success @@ -7844,6 +7864,8 @@ TEST(MetaServiceTest, UpdateTmpRowsetTest) { ASSERT_EQ(doris::BEGIN_PARTIAL_UPDATE, fetchedRowsetMeta.rowset_state()); ASSERT_EQ(rowset.num_segments(), fetchedRowsetMeta.num_segments()); ASSERT_EQ(rowset.num_rows(), fetchedRowsetMeta.num_rows()); + ASSERT_EQ(rowset.total_disk_size(), fetchedRowsetMeta.total_disk_size()); + ASSERT_EQ(rowset.index_disk_size(), fetchedRowsetMeta.index_disk_size()); ASSERT_EQ(rowset.data_disk_size(), fetchedRowsetMeta.data_disk_size()); ASSERT_NO_FATAL_FAILURE(commit_txn(meta_service.get(), db_id, txn_id, label)); @@ -7869,6 +7891,8 @@ TEST(MetaServiceTest, UpdateTmpRowsetTest) { // simulate that there are new segments added to this rowset rowset.set_num_segments(rowset.num_segments() + 3); rowset.set_num_rows(rowset.num_rows() + 1000); + rowset.set_total_disk_size(rowset.total_disk_size() + 11000); + rowset.set_index_disk_size(rowset.index_disk_size() + 1000); rowset.set_data_disk_size(rowset.data_disk_size() + 10000); ASSERT_NO_FATAL_FAILURE(update_tmp_rowset(meta_service.get(), rowset, res)); diff --git a/cloud/test/schema_kv_test.cpp b/cloud/test/schema_kv_test.cpp index 07f658175c806f..52e54f5e494b7f 100644 --- a/cloud/test/schema_kv_test.cpp +++ b/cloud/test/schema_kv_test.cpp @@ -210,6 +210,8 @@ TEST(DetachSchemaKVTest, TabletTest) { EXPECT_EQ(get_rowset_res.stats().num_rowsets(), 1); EXPECT_EQ(get_rowset_res.stats().num_segments(), 0); EXPECT_EQ(get_rowset_res.stats().data_size(), 0); + EXPECT_EQ(get_rowset_res.stats().index_size(), 0); + EXPECT_EQ(get_rowset_res.stats().segment_size(), 0); } // new MS batch create tablets with write_schema_kv=true @@ -481,6 +483,8 @@ TEST(DetachSchemaKVTest, RowsetTest) { EXPECT_EQ(get_rowset_res.stats().num_rowsets(), 2); EXPECT_EQ(get_rowset_res.stats().num_segments(), 1); EXPECT_EQ(get_rowset_res.stats().data_size(), 11000); + EXPECT_EQ(get_rowset_res.stats().index_size(), 1000); + EXPECT_EQ(get_rowset_res.stats().segment_size(), 10000); } // new MS read rowsets committed by both old and new MS @@ -530,6 +534,8 @@ TEST(DetachSchemaKVTest, RowsetTest) { EXPECT_EQ(get_rowset_res->stats().num_rowsets(), 26); EXPECT_EQ(get_rowset_res->stats().num_segments(), 25); EXPECT_EQ(get_rowset_res->stats().data_size(), 275000); + EXPECT_EQ(get_rowset_res->stats().index_size(), 25000); + EXPECT_EQ(get_rowset_res->stats().segment_size(), 250000); if (schema != nullptr) { auto schema_version = get_rowset_res->rowset_meta(10).schema_version(); get_rowset_res->mutable_rowset_meta(10)->mutable_tablet_schema()->set_schema_version(3); diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index 6e2a518ec20982..a372a993ac6f22 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -324,7 +324,8 @@ unsupportedShowStatement ((FROM | IN) database=multipartIdentifier)? wildWhere? sortClause? limitClause? #showAlterTable | SHOW DATA SKEW FROM baseTableRef #showDataSkew - | SHOW DATA (FROM tableName=multipartIdentifier)? sortClause? propertyClause? #showData + | SHOW DATA (ALL)? (FROM tableName=multipartIdentifier)? + sortClause? propertyClause? #showData | SHOW TEMPORARY? PARTITIONS FROM tableName=multipartIdentifier wildWhere? sortClause? limitClause? #showPartitions | SHOW TABLET tabletId=INTEGER_VALUE #showTabletId diff --git a/fe/fe-core/src/main/cup/sql_parser.cup b/fe/fe-core/src/main/cup/sql_parser.cup index 03f1f02f158f84..05804343bc3417 100644 --- a/fe/fe-core/src/main/cup/sql_parser.cup +++ b/fe/fe-core/src/main/cup/sql_parser.cup @@ -1014,6 +1014,8 @@ nonterminal List copy_select_expr_list; //genearted column nonterminal Boolean opt_generated_always; +nonterminal Boolean opt_detailed; + precedence nonassoc COMMA; precedence nonassoc STRING_LITERAL; precedence nonassoc KW_COLUMNS; @@ -4493,13 +4495,13 @@ show_param ::= RESULT = new ShowDataSkewStmt(table_ref); :} /* Show data statement: used to show data size of specified range */ - | KW_DATA order_by_clause:orderByClause opt_properties:prop + | KW_DATA opt_detailed:detailed order_by_clause:orderByClause opt_properties:prop {: - RESULT = new ShowDataStmt(null, orderByClause, prop); + RESULT = new ShowDataStmt(null, orderByClause, prop, detailed); :} - | KW_DATA KW_FROM table_name:dbTblName order_by_clause:orderByClause + | KW_DATA opt_detailed:detailed KW_FROM table_name:dbTblName order_by_clause:orderByClause {: - RESULT = new ShowDataStmt(dbTblName, orderByClause, null); + RESULT = new ShowDataStmt(dbTblName, orderByClause, null, detailed); :} | opt_tmp:tmp KW_PARTITIONS KW_FROM table_name:tblName opt_wild_where order_by_clause:orderByClause limit_clause: limitClause {: @@ -5063,6 +5065,17 @@ cancel_param ::= :} ; +opt_detailed ::= + /* empty */ + {: + RESULT = false; + :} + | KW_ALL + {: + RESULT = true; + :} + ; + // Delete stmt delete_stmt ::= KW_DELETE KW_FROM table_name:table opt_partition_names:partitionNames opt_table_alias:alias opt_using_clause:fromClause where_clause:wherePredicate diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDataStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDataStmt.java index 26b77e0b3d5d26..398b0d7ec05d42 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDataStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDataStmt.java @@ -100,26 +100,64 @@ public class ShowDataStmt extends ShowStmt implements NotFallbackInParser { new ImmutableList.Builder().add("TableName").add("IndexName").add("Size").add("ReplicaCount") .add("RowCount").add("RemoteSize").build(); + private static final ShowResultSetMetaData SHOW_DETAILED_TABLE_DATA_META_DATA = + ShowResultSetMetaData.builder() + .addColumn(new Column("TableName", ScalarType.createVarchar(20))) + .addColumn(new Column("ReplicaCount", ScalarType.createVarchar(20))) + .addColumn(new Column("LocalTotalSize", ScalarType.createVarchar(30))) + .addColumn(new Column("LocalDataSize", ScalarType.createVarchar(30))) + .addColumn(new Column("LocalIndexSize", ScalarType.createVarchar(30))) + .addColumn(new Column("RemoteTotalSize", ScalarType.createVarchar(30))) + .addColumn(new Column("RemoteDataSize", ScalarType.createVarchar(30))) + .addColumn(new Column("RemoteIndexSize", ScalarType.createVarchar(30))) + .build(); + + private static final ShowResultSetMetaData SHOW_DETAILED_INDEX_DATA_META_DATA = + ShowResultSetMetaData.builder() + .addColumn(new Column("TableName", ScalarType.createVarchar(20))) + .addColumn(new Column("IndexName", ScalarType.createVarchar(20))) + .addColumn(new Column("ReplicaCount", ScalarType.createVarchar(20))) + .addColumn(new Column("RowCount", ScalarType.createVarchar(20))) + .addColumn(new Column("LocalTotalSize", ScalarType.createVarchar(30))) + .addColumn(new Column("LocalDataSize", ScalarType.createVarchar(30))) + .addColumn(new Column("LocalIndexSize", ScalarType.createVarchar(30))) + .addColumn(new Column("RemoteTotalSize", ScalarType.createVarchar(30))) + .addColumn(new Column("RemoteDataSize", ScalarType.createVarchar(30))) + .addColumn(new Column("RemoteIndexSize", ScalarType.createVarchar(30))) + .build(); + TableName tableName; String dbName; - List> totalRows; + List> totalRows = Lists.newArrayList(); List> totalRowsObject = Lists.newArrayList(); - private List orderByElements; - private List orderByPairs; + private List orderByElements = null; + private List orderByPairs = null; - private final Map properties; + private Map properties = null; + private boolean detailed = true; private static final String WAREHOUSE = "entire_warehouse"; private static final String DB_LIST = "db_names"; - public ShowDataStmt(TableName tableName, List orderByElements, Map properties) { + private long totalSize = 0; + private long totalReplicaCount = 0; + private long totalRemoteSize = 0; + private long totalLocalInvertedSize = 0; + private long totalLocalSegmentSize = 0; + private long totalRemoteInvertedSize = 0; + private long totalRemoteSegmentSize = 0; + + public ShowDataStmt(TableName tableName, List orderByElements, + Map properties, boolean detailed) { this.tableName = tableName; - this.totalRows = Lists.newArrayList(); this.orderByElements = orderByElements; this.properties = properties; + this.detailed = detailed; } + public ShowDataStmt() {} + @Override public void analyze(Analyzer analyzer) throws UserException { super.analyze(analyzer); @@ -156,208 +194,245 @@ public void analyze(Analyzer analyzer) throws UserException { } } - if (tableName == null) { - db.readLock(); - try { - long totalSize = 0; - long totalReplicaCount = 0; - long totalRemoteSize = 0; - // sort by table name - List

tables = db.getTables(); - SortedSet
sortedTables = new TreeSet<>(new Comparator
() { - @Override - public int compare(Table t1, Table t2) { - return t1.getName().compareTo(t2.getName()); - } - }); - - for (Table table : tables) { - if (!Env.getCurrentEnv().getAccessManager() - .checkTblPriv(ConnectContext.get(), InternalCatalog.INTERNAL_CATALOG_NAME, dbName, - table.getName(), - PrivPredicate.SHOW)) { - continue; - } - sortedTables.add(table); - } + if (hasTable()) { + if (!Env.getCurrentEnv().getAccessManager().checkTblPriv(ConnectContext.get(), tableName, + PrivPredicate.SHOW)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_TABLEACCESS_DENIED_ERROR, "SHOW DATA", + ConnectContext.get().getQualifiedUser(), + ConnectContext.get().getRemoteIP(), + dbName + ": " + tableName); + } + OlapTable table = (OlapTable) db.getTableOrMetaException(tableName.getTbl(), TableType.OLAP); + getSingleTableStats(table); + } else { + getSingleDbStats(db); + } + return; + } - for (Table table : sortedTables) { - if (!table.isManagedTable()) { - continue; - } + private void collectDbStats(Database db) { + // sort by table name + List
tables = db.getTables(); + SortedSet
sortedTables = new TreeSet<>(new Comparator
() { + @Override + public int compare(Table t1, Table t2) { + return t1.getName().compareTo(t2.getName()); + } + }); - OlapTable olapTable = (OlapTable) table; - long tableSize = 0; - long replicaCount = 0; - long remoteSize = 0; + for (Table table : tables) { + if (!Env.getCurrentEnv().getAccessManager() + .checkTblPriv(ConnectContext.get(), InternalCatalog.INTERNAL_CATALOG_NAME, dbName, + table.getName(), + PrivPredicate.SHOW)) { + continue; + } + sortedTables.add(table); + } - tableSize = olapTable.getDataSize(); - replicaCount = olapTable.getReplicaCount(); - remoteSize = olapTable.getRemoteDataSize(); + for (Table table : sortedTables) { + if (!table.isManagedTable()) { + continue; + } - //|TableName|Size|ReplicaCount|RemoteSize - List row = Arrays.asList(table.getName(), tableSize, replicaCount, remoteSize); - totalRowsObject.add(row); + OlapTable olapTable = (OlapTable) table; + long tableSize = 0; + long replicaCount = 0; + long remoteSize = 0; - totalSize += tableSize; - totalReplicaCount += replicaCount; - totalRemoteSize += remoteSize; - } // end for tables + tableSize = olapTable.getDataSize(); + replicaCount = olapTable.getReplicaCount(); + remoteSize = olapTable.getRemoteDataSize(); - // sort by - if (orderByPairs != null && !orderByPairs.isEmpty()) { - // k-> index, v-> isDesc - Map sortMap = Maps.newLinkedHashMap(); - for (OrderByPair orderByPair : orderByPairs) { - sortMap.put(orderByPair.getIndex(), orderByPair.isDesc()); + if (!detailed) { + totalRowsObject.add(Arrays.asList(table.getName(), tableSize, replicaCount, remoteSize)); + } else { + long localIndexSize = olapTable.getLocalIndexFileSize(); + long localSegmentSize = olapTable.getLocalSegmentSize(); + long remoteIndexSize = olapTable.getRemoteIndexFileSize(); + long remoteSegmentSize = olapTable.getRemoteSegmentSize(); + totalRowsObject.add(Arrays.asList(table.getName(), tableSize, replicaCount, remoteSize, + localIndexSize, localSegmentSize, remoteIndexSize, remoteSegmentSize)); + totalLocalInvertedSize += localIndexSize; + totalLocalSegmentSize += localSegmentSize; + totalRemoteInvertedSize += remoteIndexSize; + totalRemoteSegmentSize += remoteSegmentSize; + } - } - totalRowsObject.sort(sortRows(sortMap)); - } + totalSize += tableSize; + totalReplicaCount += replicaCount; + totalRemoteSize += remoteSize; + } // end for tables + } - // for output - for (List row : totalRowsObject) { - //|TableName|Size|ReplicaCount|RemoteSize - Pair tableSizePair = DebugUtil.getByteUint((long) row.get(1)); - String readableSize = DebugUtil.DECIMAL_FORMAT_SCALE_3.format(tableSizePair.first) + " " - + tableSizePair.second; - Pair remoteSizePair = DebugUtil.getByteUint((long) row.get(3)); - String remoteReadableSize = DebugUtil.DECIMAL_FORMAT_SCALE_3.format(remoteSizePair.first) + " " - + remoteSizePair.second; - List result = Arrays.asList(String.valueOf(row.get(0)), - readableSize, String.valueOf(row.get(2)), remoteReadableSize); - totalRows.add(result); - } + private void collectTableStats(OlapTable table) { + // sort by index name + Map indexNames = table.getIndexNameToId(); + Map sortedIndexNames = new TreeMap(); + for (Map.Entry entry : indexNames.entrySet()) { + sortedIndexNames.put(entry.getKey(), entry.getValue()); + } - Pair totalSizePair = DebugUtil.getByteUint(totalSize); - String readableSize = DebugUtil.DECIMAL_FORMAT_SCALE_3.format(totalSizePair.first) + " " - + totalSizePair.second; - Pair totalRemoteSizePair = DebugUtil.getByteUint(totalRemoteSize); - String remoteReadableSize = DebugUtil.DECIMAL_FORMAT_SCALE_3.format(totalRemoteSizePair.first) + " " - + totalRemoteSizePair.second; - List total = Arrays.asList("Total", readableSize, String.valueOf(totalReplicaCount), - remoteReadableSize); - totalRows.add(total); - - // quota - long quota = db.getDataQuota(); - long replicaQuota = db.getReplicaQuota(); - Pair quotaPair = DebugUtil.getByteUint(quota); - String readableQuota = DebugUtil.DECIMAL_FORMAT_SCALE_3.format(quotaPair.first) + " " - + quotaPair.second; - - List quotaRow = Arrays.asList("Quota", readableQuota, String.valueOf(replicaQuota), ""); - totalRows.add(quotaRow); - - // left - long left = Math.max(0, quota - totalSize); - long replicaCountLeft = Math.max(0, replicaQuota - totalReplicaCount); - Pair leftPair = DebugUtil.getByteUint(left); - String readableLeft = DebugUtil.DECIMAL_FORMAT_SCALE_3.format(leftPair.first) + " " - + leftPair.second; - List leftRow = Arrays.asList("Left", readableLeft, String.valueOf(replicaCountLeft), ""); - totalRows.add(leftRow); - } finally { - db.readUnlock(); + for (Long indexId : sortedIndexNames.values()) { + long indexSize = 0; + long indexReplicaCount = 0; + long indexRowCount = 0; + long indexRemoteSize = 0; + long localIndexSize = 0; + long localSegmentSize = 0; + long remoteIndexSize = 0; + long remoteSegmentSize = 0; + for (Partition partition : table.getAllPartitions()) { + MaterializedIndex mIndex = partition.getIndex(indexId); + indexSize += mIndex.getDataSize(false); + indexReplicaCount += mIndex.getReplicaCount(); + indexRowCount += mIndex.getRowCount() == -1 ? 0 : mIndex.getRowCount(); + indexRemoteSize += mIndex.getRemoteDataSize(); + localIndexSize += mIndex.getLocalIndexSize(); + localSegmentSize += mIndex.getLocalSegmentSize(); + remoteIndexSize += mIndex.getRemoteIndexSize(); + remoteSegmentSize += mIndex.getRemoteSegmentSize(); } - } else { - if (!Env.getCurrentEnv().getAccessManager().checkTblPriv(ConnectContext.get(), tableName, - PrivPredicate.SHOW)) { - ErrorReport.reportAnalysisException(ErrorCode.ERR_TABLEACCESS_DENIED_ERROR, "SHOW DATA", - ConnectContext.get().getQualifiedUser(), - ConnectContext.get().getRemoteIP(), - dbName + ": " + tableName); + + String indexName = table.getIndexNameById(indexId); + if (!detailed) { + totalRowsObject.add(Arrays.asList(tableName, indexName, indexSize, indexReplicaCount, + indexRowCount, indexRemoteSize)); + } else { + totalRowsObject.add(Arrays.asList(tableName, indexName, indexSize, indexReplicaCount, indexRowCount, + indexRemoteSize, localIndexSize, localSegmentSize, remoteIndexSize, remoteSegmentSize)); } - OlapTable olapTable = (OlapTable) db - .getTableOrMetaException(tableName.getTbl(), TableType.OLAP); - long totalSize = 0; - long totalReplicaCount = 0; - long totalRemoteSize = 0; - olapTable.readLock(); - try { - // sort by index name - Map indexNames = olapTable.getIndexNameToId(); - Map sortedIndexNames = new TreeMap(); - for (Map.Entry entry : indexNames.entrySet()) { - sortedIndexNames.put(entry.getKey(), entry.getValue()); - } + totalSize += indexSize; + totalReplicaCount += indexReplicaCount; + totalRemoteSize += indexRemoteSize; + totalLocalInvertedSize += localIndexSize; + totalLocalSegmentSize += localSegmentSize; + totalRemoteInvertedSize += remoteIndexSize; + totalRemoteSegmentSize += remoteSegmentSize; + } // end for indices + } - for (Long indexId : sortedIndexNames.values()) { - long indexSize = 0; - long indexReplicaCount = 0; - long indexRowCount = 0; - long indexRemoteSize = 0; - for (Partition partition : olapTable.getAllPartitions()) { - MaterializedIndex mIndex = partition.getIndex(indexId); - indexSize += mIndex.getDataSize(false); - indexReplicaCount += mIndex.getReplicaCount(); - indexRowCount += mIndex.getRowCount() == -1 ? 0 : mIndex.getRowCount(); - indexRemoteSize += mIndex.getRemoteDataSize(); - } + private void sortResult() { + if (orderByPairs != null && !orderByPairs.isEmpty()) { + // k-> index, v-> isDesc + Map sortMap = Maps.newLinkedHashMap(); + for (OrderByPair orderByPair : orderByPairs) { + sortMap.put(orderByPair.getIndex(), orderByPair.isDesc()); - String indexName = olapTable.getIndexNameById(indexId); - // .add("TableName").add("IndexName").add("Size").add("ReplicaCount").add("RowCount") - // .add("RemoteSize") - List row = Arrays.asList(tableName, indexName, indexSize, indexReplicaCount, - indexRowCount, indexRemoteSize); - totalRowsObject.add(row); - - totalSize += indexSize; - totalReplicaCount += indexReplicaCount; - totalRemoteSize += indexRemoteSize; - } // end for indices - - // sort by - if (orderByPairs != null && !orderByPairs.isEmpty()) { - // k-> index, v-> isDesc - Map sortMap = Maps.newLinkedHashMap(); - for (OrderByPair orderByPair : orderByPairs) { - sortMap.put(orderByPair.getIndex(), orderByPair.isDesc()); + } + totalRowsObject.sort(sortRows(sortMap)); + } + } - } - totalRowsObject.sort(sortRows(sortMap)); - } + private void buildDbStatsOutput(long quota, long replicaQuota) { + // for output + for (List row : totalRowsObject) { + if (!detailed) { + totalRows.add(Arrays.asList(String.valueOf(row.get(0)), + DebugUtil.printByteWithUnit((long) row.get(1)), String.valueOf(row.get(2)), + DebugUtil.printByteWithUnit((long) row.get(3)))); + } else { + totalRows.add(Arrays.asList(String.valueOf(row.get(0)), String.valueOf(row.get(2)), + DebugUtil.printByteWithUnit((long) row.get(1)), DebugUtil.printByteWithUnit((long) row.get(5)), + DebugUtil.printByteWithUnit((long) row.get(4)), DebugUtil.printByteWithUnit((long) row.get(3)), + DebugUtil.printByteWithUnit((long) row.get(7)), + DebugUtil.printByteWithUnit((long) row.get(6)))); + } + } - // for output - for (int index = 0; index <= totalRowsObject.size() - 1; index++) { - //| TableName| IndexName | Size | ReplicaCount | RowCount | RemoteSize - List row = totalRowsObject.get(index); - List result; - Pair tableSizePair = DebugUtil.getByteUint((long) row.get(2)); - String readableSize = DebugUtil.DECIMAL_FORMAT_SCALE_3.format(tableSizePair.first) - + " " + tableSizePair.second; - Pair remoteSizePair = DebugUtil.getByteUint((long) row.get(5)); - String remoteReadableSize = DebugUtil.DECIMAL_FORMAT_SCALE_3.format(remoteSizePair.first) + " " - + remoteSizePair.second; - if (index == 0) { - result = Arrays.asList(tableName.getTbl(), String.valueOf(row.get(1)), - readableSize, String.valueOf(row.get(3)), - String.valueOf(row.get(4)), remoteReadableSize); - } else { - result = Arrays.asList("", String.valueOf(row.get(1)), - readableSize, String.valueOf(row.get(3)), - String.valueOf(row.get(4)), remoteReadableSize); - } - totalRows.add(result); - } + long left = Math.max(0, quota - totalSize); + long replicaCountLeft = Math.max(0, replicaQuota - totalReplicaCount); - Pair totalSizePair = DebugUtil.getByteUint(totalSize); - String readableSize = DebugUtil.DECIMAL_FORMAT_SCALE_3.format(totalSizePair.first) + " " - + totalSizePair.second; - Pair totalRemoteSizePair = DebugUtil.getByteUint(totalRemoteSize); - String remoteReadableSize = DebugUtil.DECIMAL_FORMAT_SCALE_3.format(totalRemoteSizePair.first) + " " - + totalRemoteSizePair.second; - List row = Arrays.asList("", "Total", readableSize, String.valueOf(totalReplicaCount), "", - remoteReadableSize); - totalRows.add(row); - } finally { - olapTable.readUnlock(); + if (!detailed) { + totalRows.add(Arrays.asList("Total", DebugUtil.printByteWithUnit(totalSize), + String.valueOf(totalReplicaCount), DebugUtil.printByteWithUnit(totalRemoteSize))); + totalRows.add(Arrays.asList("Quota", DebugUtil.printByteWithUnit(quota), + String.valueOf(replicaQuota), "")); + totalRows.add(Arrays.asList("Left", DebugUtil.printByteWithUnit(left), + String.valueOf(replicaCountLeft), "")); + } else { + totalRows.add(Arrays.asList("Total", String.valueOf(totalReplicaCount), + DebugUtil.printByteWithUnit(totalSize), + DebugUtil.printByteWithUnit(totalLocalSegmentSize), + DebugUtil.printByteWithUnit(totalLocalInvertedSize), + DebugUtil.printByteWithUnit(totalRemoteSize), + DebugUtil.printByteWithUnit(totalRemoteSegmentSize), + DebugUtil.printByteWithUnit(totalRemoteInvertedSize))); + totalRows.add(Arrays.asList("Quota", String.valueOf(replicaQuota), + DebugUtil.printByteWithUnit(quota), "", "", "", "", "")); + totalRows.add(Arrays.asList("Left", String.valueOf(replicaCountLeft), + DebugUtil.printByteWithUnit(left), "", "", "", "", "")); + } + } + + private void buildTableStatsOutput() { + for (int index = 0; index < totalRowsObject.size(); index++) { + List row = totalRowsObject.get(index); + String indexName = index == 0 ? tableName.getTbl() : ""; + if (!detailed) { + totalRows.add(Arrays.asList(indexName, String.valueOf(row.get(1)), + DebugUtil.printByteWithUnit((long) row.get(2)), String.valueOf(row.get(3)), + String.valueOf(row.get(4)), DebugUtil.printByteWithUnit((long) row.get(5)))); + } else { + totalRows.add(Arrays.asList(indexName, String.valueOf(row.get(1)), + String.valueOf(row.get(3)), String.valueOf(row.get(4)), + DebugUtil.printByteWithUnit((long) row.get(2)), DebugUtil.printByteWithUnit((long) row.get(7)), + DebugUtil.printByteWithUnit((long) row.get(6)), DebugUtil.printByteWithUnit((long) row.get(5)), + DebugUtil.printByteWithUnit((long) row.get(9)), + DebugUtil.printByteWithUnit((long) row.get(8)))); } } + + // Total + if (!detailed) { + totalRows.add(Arrays.asList("", "Total", DebugUtil.printByteWithUnit(totalSize), + String.valueOf(totalReplicaCount), "", DebugUtil.printByteWithUnit(totalRemoteSize))); + } else { + totalRows.add(Arrays.asList("", "Total", String.valueOf(totalReplicaCount), "", + DebugUtil.printByteWithUnit(totalSize), DebugUtil.printByteWithUnit(totalLocalSegmentSize), + DebugUtil.printByteWithUnit(totalLocalInvertedSize), + DebugUtil.printByteWithUnit(totalRemoteSize), + DebugUtil.printByteWithUnit(totalRemoteSegmentSize), + DebugUtil.printByteWithUnit(totalRemoteInvertedSize))); + } + } + + // |TableName|Size|ReplicaCount|RemoteSize| + // |TableName|ReplicaCount|LocalTotalSize|LocalDataSize|LocalIndexSize| + // |RemoteTotalSize|RemoteDataSize|RemoteIndexSize| + private void getSingleDbStats(Database db) { + db.readLock(); + long quota = 0; + long replicaQuota = 0; + try { + collectDbStats(db); + quota = db.getDataQuota(); + replicaQuota = db.getReplicaQuota(); + } finally { + db.readUnlock(); + } + // sort by + sortResult(); + buildDbStatsOutput(quota, replicaQuota); } - public static int analyzeColumn(String columnName, String tableName) throws AnalysisException { + // |TableName|IndexName|Size|ReplicaCount|RowCount|RemoteSize| + // |TableName|IndexName|ReplicaCount||RowCount|LocalTotalSize |LocalDataSize |LocalIndexSize| + // |RemoteTotalSize|RemoteDataSize|RemoteIndexSize| + private void getSingleTableStats(OlapTable table) { + table.readLock(); + try { + collectTableStats(table); + } finally { + table.readUnlock(); + } + // sort by + sortResult(); + buildTableStatsOutput(); + } + + private int analyzeColumn(String columnName, String tableName) throws AnalysisException { ImmutableList titles = SHOW_TABLE_DATA_META_DATA_ORIGIN; if (tableName != null) { titles = SHOW_INDEX_DATA_META_DATA_ORIGIN; @@ -371,7 +446,7 @@ public static int analyzeColumn(String columnName, String tableName) throws Anal throw new AnalysisException("Title name[" + columnName + "] does not exist"); } - private static Comparator> sortRows(Map sortMap) { + private Comparator> sortRows(Map sortMap) { Ordering ordering = Ordering.natural(); return new Comparator>() { @@ -400,28 +475,41 @@ public List> getResultRows() throws AnalysisException { @Override public ShowResultSetMetaData getMetaData() { - String value = null; if (properties != null) { - value = properties.get(WAREHOUSE); - } - if (value != null && value.equals("true")) { - return SHOW_WAREHOUSE_DATA_META_DATA; + String value = properties.get(WAREHOUSE); + if (value != null && value.equals("true")) { + return SHOW_WAREHOUSE_DATA_META_DATA; + } } if (Strings.isNullOrEmpty(dbName)) { return SHOW_DATABASE_DATA_META_DATA; } - if (tableName != null) { - return SHOW_INDEX_DATA_META_DATA; + + if (hasTable()) { + if (!detailed) { + return SHOW_INDEX_DATA_META_DATA; + } else { + return SHOW_DETAILED_INDEX_DATA_META_DATA; + } } else { - return SHOW_TABLE_DATA_META_DATA; + if (!detailed) { + return SHOW_TABLE_DATA_META_DATA; + } else { + return SHOW_DETAILED_TABLE_DATA_META_DATA; + } } } @Override public String toSql() { StringBuilder builder = new StringBuilder(); - builder.append("SHOW DATA"); + builder.append("SHOW "); + if (detailed) { + builder.append("DETAILED DATA"); + } else { + builder.append("DATA"); + } if (tableName != null) { builder.append(" FROM "); @@ -445,6 +533,7 @@ public String toString() { return toSql(); } + // |DBName|DataSize|RecycleSize| private boolean getDbStatsByProperties() { if (properties == null) { return false; @@ -507,6 +596,7 @@ private boolean getDbStatsByProperties() { return false; } + // |DbId|DbName|Size|RemoteSize|RecycleSize|RecycleRemoteSize| private void getAllDbStats() throws AnalysisException { // check auth if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN)) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/CloudTabletStatMgr.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/CloudTabletStatMgr.java index 309b433572c942..3babb0e001a204 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/CloudTabletStatMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/CloudTabletStatMgr.java @@ -165,6 +165,8 @@ protected void runAfterCatalogReady() { Long tableDataSize = 0L; Long tableTotalReplicaDataSize = 0L; + Long tableTotalLocalIndexSize = 0L; + Long tableTotalLocalSegmentSize = 0L; Long tableReplicaCount = 0L; @@ -185,6 +187,8 @@ protected void runAfterCatalogReady() { long tabletRowsetCount = 0L; long tabletSegmentCount = 0L; long tabletRowCount = 0L; + long tabletIndexSize = 0L; + long tabletSegmentSize = 0L; for (Replica replica : tablet.getReplicas()) { if (replica.getDataSize() > tabletDataSize) { @@ -204,6 +208,13 @@ protected void runAfterCatalogReady() { tabletSegmentCount = replica.getSegmentCount(); } + if (replica.getLocalInvertedIndexSize() > tabletIndexSize) { + tabletIndexSize = replica.getLocalInvertedIndexSize(); + } + if (replica.getLocalSegmentSize() > tabletSegmentSize) { + tabletSegmentSize = replica.getLocalSegmentSize(); + } + tableReplicaCount++; } @@ -214,6 +225,8 @@ protected void runAfterCatalogReady() { tableRowsetCount += tabletRowsetCount; tableSegmentCount += tabletSegmentCount; + tableTotalLocalIndexSize += tabletIndexSize; + tableTotalLocalSegmentSize += tabletSegmentSize; } // end for tablets index.setRowCountReported(true); index.setRowCount(indexRowCount); @@ -223,7 +236,8 @@ protected void runAfterCatalogReady() { // this is only one thread to update table statistics, readLock is enough olapTable.setStatistics(new OlapTable.Statistics(db.getName(), table.getName(), tableDataSize, tableTotalReplicaDataSize, 0L, - tableReplicaCount, tableRowCount, tableRowsetCount, tableSegmentCount)); + tableReplicaCount, tableRowCount, tableRowsetCount, tableSegmentCount, + tableTotalLocalIndexSize, tableTotalLocalSegmentSize, 0L, 0L)); LOG.debug("finished to set row num for table: {} in database: {}", table.getName(), db.getFullName()); } finally { @@ -232,7 +246,7 @@ protected void runAfterCatalogReady() { newCloudTableStatsMap.put(Pair.of(dbId, table.getId()), new OlapTable.Statistics(db.getName(), table.getName(), tableDataSize, tableTotalReplicaDataSize, 0L, - tableReplicaCount, tableRowCount, tableRowsetCount, tableSegmentCount)); + tableReplicaCount, tableRowCount, tableRowsetCount, tableSegmentCount, 0L, 0L, 0L, 0L)); } } this.cloudTableStatsMap = newCloudTableStatsMap; @@ -253,6 +267,8 @@ private void updateTabletStat(GetTabletStatsResponse response) { replica.setRowsetCount(stat.getNumRowsets()); replica.setSegmentCount(stat.getNumSegments()); replica.setRowCount(stat.getNumRows()); + replica.setLocalInvertedIndexSize(stat.getIndexSize()); + replica.setLocalSegmentSize(stat.getSegmentSize()); } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/MaterializedIndex.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/MaterializedIndex.java index 6e4f6a2248e719..5e367b538cd74c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/MaterializedIndex.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/MaterializedIndex.java @@ -197,6 +197,46 @@ public long getReplicaCount() { return replicaCount; } + public long getLocalIndexSize() { + long localIndexSize = 0; + for (Tablet tablet : getTablets()) { + for (Replica replica : tablet.getReplicas()) { + localIndexSize += replica.getLocalInvertedIndexSize(); + } + } + return localIndexSize; + } + + public long getLocalSegmentSize() { + long localSegmentSize = 0; + for (Tablet tablet : getTablets()) { + for (Replica replica : tablet.getReplicas()) { + localSegmentSize += replica.getLocalSegmentSize(); + } + } + return localSegmentSize; + } + + public long getRemoteIndexSize() { + long remoteIndexSize = 0; + for (Tablet tablet : getTablets()) { + for (Replica replica : tablet.getReplicas()) { + remoteIndexSize += replica.getRemoteInvertedIndexSize(); + } + } + return remoteIndexSize; + } + + public long getRemoteSegmentSize() { + long remoteSegmentSize = 0; + for (Tablet tablet : getTablets()) { + for (Replica replica : tablet.getReplicas()) { + remoteSegmentSize += replica.getRemoteSegmentSize(); + } + } + return remoteSegmentSize; + } + public int getTabletOrderIdx(long tabletId) { int idx = 0; for (Tablet tablet : tablets) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java index 8925e483c29df7..50b1dc6ab3954f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java @@ -1688,11 +1688,22 @@ public long getAvgRowLength() { public long getDataLength() { long dataSize = 0; for (Map.Entry entry : idToPartition.entrySet()) { - dataSize += entry.getValue().getBaseIndex().getDataSize(false); + dataSize += entry.getValue().getBaseIndex().getLocalSegmentSize(); + dataSize += entry.getValue().getBaseIndex().getRemoteSegmentSize(); } return dataSize; } + @Override + public long getIndexLength() { + long indexSize = 0; + for (Map.Entry entry : idToPartition.entrySet()) { + indexSize += entry.getValue().getBaseIndex().getLocalIndexSize(); + indexSize += entry.getValue().getBaseIndex().getRemoteIndexSize(); + } + return indexSize; + } + // Get the signature string of this table with specified partitions. // This method is used to determine whether the tables have the same schema. // Contains: @@ -3387,6 +3398,18 @@ public static class Statistics { @Getter private Long segmentCount; + @Getter + private Long localInvertedIndexSize; // multi replicas + + @Getter + private Long localSegmentSize; // multi replicas + + @Getter + private Long remoteInvertedIndexSize; // single replica + + @Getter + private Long remoteSegmentSize; // single replica + public Statistics() { this.dbName = null; this.tableName = null; @@ -3401,13 +3424,18 @@ public Statistics() { this.rowCount = 0L; this.rowsetCount = 0L; this.segmentCount = 0L; - + this.localInvertedIndexSize = 0L; + this.localSegmentSize = 0L; + this.remoteInvertedIndexSize = 0L; + this.remoteSegmentSize = 0L; } public Statistics(String dbName, String tableName, Long dataSize, Long totalReplicaDataSize, Long remoteDataSize, Long replicaCount, Long rowCount, - Long rowsetCount, Long segmentCount) { + Long rowsetCount, Long segmentCount, + Long localInvertedIndexSize, Long localSegmentSize, + Long remoteInvertedIndexSize, Long remoteSegmentSize) { this.dbName = dbName; this.tableName = tableName; @@ -3422,6 +3450,11 @@ public Statistics(String dbName, String tableName, this.rowCount = rowCount; this.rowsetCount = rowsetCount; this.segmentCount = segmentCount; + + this.localInvertedIndexSize = localInvertedIndexSize; + this.localSegmentSize = localSegmentSize; + this.remoteInvertedIndexSize = remoteInvertedIndexSize; + this.remoteSegmentSize = remoteSegmentSize; } } @@ -3445,6 +3478,22 @@ public long getReplicaCount() { return statistics.getReplicaCount(); } + public long getLocalIndexFileSize() { + return statistics.getLocalInvertedIndexSize(); + } + + public long getLocalSegmentSize() { + return statistics.getLocalSegmentSize(); + } + + public long getRemoteIndexFileSize() { + return statistics.getRemoteInvertedIndexSize(); + } + + public long getRemoteSegmentSize() { + return statistics.getRemoteSegmentSize(); + } + public boolean isShadowIndex(long indexId) { String indexName = getIndexNameById(indexId); if (indexName != null && indexName.startsWith(org.apache.doris.alter.SchemaChangeHandler.SHADOW_NAME_PREFIX)) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Replica.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Replica.java index e86bd25cb5e926..d248cc40839988 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Replica.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Replica.java @@ -26,6 +26,8 @@ import org.apache.doris.thrift.TUniqueId; import com.google.gson.annotations.SerializedName; +import lombok.Getter; +import lombok.Setter; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -122,6 +124,23 @@ public static class ReplicaContext { @SerializedName(value = "lsvh", alternate = {"lastSuccessVersionHash"}) private long lastSuccessVersionHash = 0L; + @Setter + @Getter + @SerializedName(value = "lis", alternate = {"localInvertedIndexSize"}) + private Long localInvertedIndexSize = 0L; + @Setter + @Getter + @SerializedName(value = "lss", alternate = {"localSegmentSize"}) + private Long localSegmentSize = 0L; + @Setter + @Getter + @SerializedName(value = "ris", alternate = {"remoteInvertedIndexSize"}) + private Long remoteInvertedIndexSize = 0L; + @Setter + @Getter + @SerializedName(value = "rss", alternate = {"remoteSegmentSize"}) + private Long remoteSegmentSize = 0L; + private volatile long totalVersionCount = -1; private volatile long visibleVersionCount = -1; diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java index ed37a936e10819..0f1d91087d2988 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java @@ -446,6 +446,9 @@ public long getDataLength() { return 0; } + public long getIndexLength() { + return 0; + } public TTableDescriptor toThrift() { return null; diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/TableIf.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/TableIf.java index 8f6e924f44a54d..4761ac9d86db2a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/TableIf.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/TableIf.java @@ -174,6 +174,8 @@ default int getBaseColumnIdxByName(String colName) { long getAvgRowLength(); + long getIndexLength(); + long getLastCheckTime(); String getComment(boolean escapeQuota); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/TabletStatMgr.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/TabletStatMgr.java index aa46c362e38ba9..f79ed89215b4d3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/TabletStatMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/TabletStatMgr.java @@ -114,6 +114,11 @@ protected void runAfterCatalogReady() { Long tableDataSize = 0L; Long tableTotalReplicaDataSize = 0L; + Long tableTotalLocalIndexSize = 0L; + Long tableTotalLocalSegmentSize = 0L; + Long tableTotalRemoteIndexSize = 0L; + Long tableTotalRemoteSegmentSize = 0L; + Long tableRemoteDataSize = 0L; Long tableReplicaCount = 0L; @@ -171,6 +176,10 @@ protected void runAfterCatalogReady() { tabletRemoteDataSize = replica.getRemoteDataSize(); } tableReplicaCount++; + tableTotalLocalIndexSize += replica.getLocalInvertedIndexSize(); + tableTotalLocalSegmentSize += replica.getLocalSegmentSize(); + tableTotalRemoteIndexSize += replica.getRemoteInvertedIndexSize(); + tableTotalRemoteSegmentSize += replica.getRemoteSegmentSize(); } tableDataSize += tabletDataSize; @@ -196,7 +205,9 @@ protected void runAfterCatalogReady() { // this is only one thread to update table statistics, readLock is enough olapTable.setStatistics(new OlapTable.Statistics(db.getName(), table.getName(), tableDataSize, tableTotalReplicaDataSize, - tableRemoteDataSize, tableReplicaCount, tableRowCount, 0L, 0L)); + tableRemoteDataSize, tableReplicaCount, tableRowCount, 0L, 0L, + tableTotalLocalIndexSize, tableTotalLocalSegmentSize, + tableTotalRemoteIndexSize, tableTotalRemoteSegmentSize)); if (LOG.isDebugEnabled()) { LOG.debug("finished to set row num for table: {} in database: {}", @@ -220,6 +231,10 @@ private void updateTabletStat(Long beId, TTabletStatResult result) { if (replica != null) { replica.setDataSize(stat.getDataSize()); replica.setRemoteDataSize(stat.getRemoteDataSize()); + replica.setLocalInvertedIndexSize(stat.getLocalIndexSize()); + replica.setLocalSegmentSize(stat.getLocalSegmentSize()); + replica.setRemoteInvertedIndexSize(stat.getRemoteIndexSize()); + replica.setRemoteSegmentSize(stat.getRemoteSegmentSize()); replica.setRowCount(stat.getRowCount()); replica.setTotalVersionCount(stat.getTotalVersionCount()); replica.setVisibleVersionCount(stat.isSetVisibleVersionCount() ? stat.getVisibleVersionCount() diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalTable.java index 6c72d0a67d4732..bd1e36e7bc968b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalTable.java @@ -239,6 +239,11 @@ public long getDataLength() { return 0; } + @Override + public long getIndexLength() { + return 0; + } + @Override public long getCreateTime() { return 0; diff --git a/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java b/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java index e35fd2dc852322..cf9e8e82ce112f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java +++ b/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java @@ -659,6 +659,7 @@ public TListTableStatusResult listTableStatus(TGetTablesParams params) throws TE status.setRows(table.getCachedRowCount()); status.setDataLength(table.getDataLength()); status.setAvgRowLength(table.getAvgRowLength()); + status.setIndexLength(table.getIndexLength()); tablesResult.add(status); } finally { table.readUnlock(); diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowDataStmtTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowDataStmtTest.java index 9b421de0c397bd..83d230274f6f29 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowDataStmtTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowDataStmtTest.java @@ -139,7 +139,7 @@ public void setUp() throws UserException { @Test public void testNormal() throws AnalysisException, UserException { - ShowDataStmt stmt = new ShowDataStmt(null, null, null); + ShowDataStmt stmt = new ShowDataStmt(null, null, null, false); stmt.analyze(analyzer); Assert.assertEquals("SHOW DATA", stmt.toString()); Assert.assertEquals(4, stmt.getMetaData().getColumnCount()); @@ -151,7 +151,7 @@ public void testNormal() throws AnalysisException, UserException { OrderByElement orderByElementTwo = new OrderByElement(slotRefTwo, false, false); stmt = new ShowDataStmt(new TableName(InternalCatalog.INTERNAL_CATALOG_NAME, "testDb", "test_tbl"), - Arrays.asList(orderByElementOne, orderByElementTwo), null); + Arrays.asList(orderByElementOne, orderByElementTwo), null, false); stmt.analyze(analyzer); Assert.assertEquals( "SHOW DATA FROM `testDb`.`test_tbl` ORDER BY `ReplicaCount` DESC, `Size` DESC", @@ -159,7 +159,7 @@ public void testNormal() throws AnalysisException, UserException { Assert.assertEquals(6, stmt.getMetaData().getColumnCount()); Assert.assertEquals(true, stmt.hasTable()); - stmt = new ShowDataStmt(null, Arrays.asList(orderByElementOne, orderByElementTwo), null); + stmt = new ShowDataStmt(null, Arrays.asList(orderByElementOne, orderByElementTwo), null, false); stmt.analyze(analyzer); Assert.assertEquals("SHOW DATA ORDER BY `ReplicaCount` DESC, `Size` DESC", stmt.toString()); diff --git a/gensrc/proto/cloud.proto b/gensrc/proto/cloud.proto index 381075074335c5..c113868a2c3286 100644 --- a/gensrc/proto/cloud.proto +++ b/gensrc/proto/cloud.proto @@ -544,6 +544,10 @@ message TabletCompactionJobPB { optional int64 delete_bitmap_lock_initiator = 24; optional int64 full_compaction_cnt = 25; // prepare optional bool check_input_versions_range = 26; + optional int64 index_size_input_rowsets = 27; + optional int64 segment_size_input_rowsets = 28; + optional int64 index_size_output_rowsets = 29; + optional int64 segment_size_output_rowsets = 30; } message TabletSchemaChangeJobPB { @@ -563,6 +567,8 @@ message TabletSchemaChangeJobPB { optional int64 output_cumulative_point = 12; optional bool is_inverted_index_change = 13 [default = false]; optional int64 delete_bitmap_lock_initiator = 14; + optional int64 index_size_output_rowsets = 15; + optional int64 segment_size_output_rowsets = 16; } message TabletJobInfoPB { @@ -576,7 +582,7 @@ message TabletJobInfoPB { //============================================================================== message TabletStatsPB { optional TabletIndexPB idx = 1; - optional int64 data_size = 2; + optional int64 data_size = 2; // data_size = index_size + segment_size optional int64 num_rows = 3; optional int64 num_rowsets = 4; optional int64 num_segments = 5; @@ -588,6 +594,8 @@ message TabletStatsPB { optional int64 last_cumu_compaction_time_ms = 11; optional int64 full_compaction_cnt = 12; optional int64 last_full_compaction_time_ms = 13; + optional int64 index_size = 14; + optional int64 segment_size = 15; } message ObjectFilePB { diff --git a/gensrc/thrift/BackendService.thrift b/gensrc/thrift/BackendService.thrift index 7f073b2b71095a..aed248adfe891a 100644 --- a/gensrc/thrift/BackendService.thrift +++ b/gensrc/thrift/BackendService.thrift @@ -32,13 +32,18 @@ struct TExportTaskRequest { struct TTabletStat { 1: required i64 tablet_id - // local data size + // local data size = local inverted index file size + local segment file size 2: optional i64 data_size 3: optional i64 row_count 4: optional i64 total_version_count + // remote data size = remote inverted index file size + remote segment file size 5: optional i64 remote_data_size 6: optional i64 visible_version_count 7: optional i64 visible_version + 8: optional i64 local_index_size = 0 // .idx + 9: optional i64 local_segment_size = 0 // .dat + 10: optional i64 remote_index_size = 0 // .idx + 11: optional i64 remote_segment_size = 0 // .dat } struct TTabletStatResult { diff --git a/gensrc/thrift/FrontendService.thrift b/gensrc/thrift/FrontendService.thrift index e2af8937425d0c..246cc999562934 100644 --- a/gensrc/thrift/FrontendService.thrift +++ b/gensrc/thrift/FrontendService.thrift @@ -350,6 +350,7 @@ struct TTableStatus { 11: optional i64 rows; 12: optional i64 avg_row_length 13: optional i64 data_length; + 14: optional i64 index_length; } struct TListTableStatusResult { diff --git a/gensrc/thrift/MasterService.thrift b/gensrc/thrift/MasterService.thrift index 9d8cd9111ba5c1..a045fa3adcb5bd 100644 --- a/gensrc/thrift/MasterService.thrift +++ b/gensrc/thrift/MasterService.thrift @@ -47,6 +47,10 @@ struct TTabletInfo { 19: optional i64 cooldown_term 20: optional Types.TUniqueId cooldown_meta_id 21: optional i64 visible_version_count + 22: optional i64 local_index_size = 0 // .idx + 23: optional i64 local_segment_size = 0 // .dat + 24: optional i64 remote_index_size = 0 // .idx + 25: optional i64 remote_segment_size = 0 // .dat // For cloud 1000: optional bool is_persistent diff --git a/regression-test/data/nereids_p0/system/test_query_sys_tables.out b/regression-test/data/nereids_p0/system/test_query_sys_tables.out index a43229a87adead..16b71543a11302 100644 --- a/regression-test/data/nereids_p0/system/test_query_sys_tables.out +++ b/regression-test/data/nereids_p0/system/test_query_sys_tables.out @@ -25,9 +25,9 @@ internal test_query_sys_db_5 \N internal test_query_sys_db_6 \N -- !tables -- -internal test_query_sys_tb_4 BASE TABLE 0 \N \N -internal test_query_sys_tb_5 BASE TABLE 0 \N \N -internal test_query_sys_tb_6 BASE TABLE 0 \N \N +internal test_query_sys_tb_4 BASE TABLE 0 \N 0 +internal test_query_sys_tb_5 BASE TABLE 0 \N 0 +internal test_query_sys_tb_6 BASE TABLE 0 \N 0 -- !session_variables -- wait_timeout 30000 diff --git a/regression-test/data/query_p0/system/test_query_sys_tables.out b/regression-test/data/query_p0/system/test_query_sys_tables.out index cdd89914994c63..0100314ef799a1 100644 --- a/regression-test/data/query_p0/system/test_query_sys_tables.out +++ b/regression-test/data/query_p0/system/test_query_sys_tables.out @@ -158,9 +158,9 @@ internal test_query_sys_db_2 \N internal test_query_sys_db_3 \N -- !tables -- -internal test_query_sys_tb_1 BASE TABLE 0 \N \N -internal test_query_sys_tb_2 BASE TABLE 0 \N \N -internal test_query_sys_tb_3 BASE TABLE 0 \N \N +internal test_query_sys_tb_1 BASE TABLE 0 \N 0 +internal test_query_sys_tb_2 BASE TABLE 0 \N 0 +internal test_query_sys_tb_3 BASE TABLE 0 \N 0 -- !session_variables -- wait_timeout 30000 diff --git a/regression-test/suites/inverted_index_p1/show_data/test_show_index_data.groovy b/regression-test/suites/inverted_index_p1/show_data/test_show_index_data.groovy new file mode 100644 index 00000000000000..671fe907782f2c --- /dev/null +++ b/regression-test/suites/inverted_index_p1/show_data/test_show_index_data.groovy @@ -0,0 +1,399 @@ +// 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. + +import java.util.concurrent.Executors +import java.util.concurrent.TimeUnit +import org.awaitility.Awaitility + +enum FileSizeChange { + LARGER, + SMALLER, + UNCHANGED +} + +suite("test_show_index_data", "p1") { + + def show_table_name = "test_show_index_data" + + def load_json_data = { table_name, file_name -> + // load the json data + streamLoad { + table "${table_name}" + + // set http request header params + set 'read_json_by_line', 'true' + set 'format', 'json' + set 'max_filter_ratio', '0.1' + file file_name // import json file + time 10000 // limit inflight 10s + + // if declared a check callback, the default check condition will ignore. + // So you must check all condition + + check { result, exception, startTime, endTime -> + if (exception != null) { + throw exception + } + logger.info("Stream load ${file_name} result: ${result}".toString()) + def json = parseJson(result) + assertEquals("success", json.Status.toLowerCase()) + // assertEquals(json.NumberTotalRows, json.NumberLoadedRows + json.NumberUnselectedRows) + assertTrue(json.NumberLoadedRows > 0 && json.LoadBytes > 0) + } + } + } + + def backendId_to_backendIP = [:] + def backendId_to_backendHttpPort = [:] + getBackendIpHttpPort(backendId_to_backendIP, backendId_to_backendHttpPort); + + + + def convert_size = { str -> + def matcher = str =~ /(\d+(\.\d+)?)\s*(KB|MB|GB|TB|B)/ + if (matcher) { + def value = matcher[0][1] as double + def unit = matcher[0][3] + logger.info("value is: " + value + " unit is: " + unit) + def result = 0.0 + switch (unit) { + case 'KB': + result = value * 1024 + break + case 'MB': + result = value * 1024 * 1024 + break + case 'GB': + result = value * 1024 * 1024 * 1024 + break + case 'B': + result = value + break + default: + throw new IllegalArgumentException("Unknown unit: $unit") + } + + return result + } else { + return 0 + } + } + + sql "DROP TABLE IF EXISTS ${show_table_name}" + sql """ + CREATE TABLE ${show_table_name} ( + `@timestamp` int(11) NULL, + `clientip` varchar(20) NULL, + `request` varchar(500) NULL, + `status` int NULL, + `size` int NULL, + INDEX clientip_idx (`clientip`) USING INVERTED COMMENT '', + INDEX request_idx (`request`) USING INVERTED PROPERTIES("parser" = "unicode") COMMENT '' + ) ENGINE=OLAP + DUPLICATE KEY(`@timestamp`, `clientip`) + DISTRIBUTED BY HASH(`@timestamp`) BUCKETS 1 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1", + "compaction_policy" = "time_series", + "time_series_compaction_file_count_threshold" = "20", + "disable_auto_compaction" = "true" + ); + """ + + def compaction = { + + def tablets = sql_return_maparray """ show tablets from ${show_table_name}; """ + + for (def tablet in tablets) { + int beforeSegmentCount = 0 + String tablet_id = tablet.TabletId + (code, out, err) = curl("GET", tablet.CompactionStatus) + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + for (String rowset in (List) tabletJson.rowsets) { + beforeSegmentCount += Integer.parseInt(rowset.split(" ")[1]) + } + assertEquals(beforeSegmentCount, 10) + } + + // trigger compactions for all tablets in ${tableName} + for (def tablet in tablets) { + String tablet_id = tablet.TabletId + backend_id = tablet.BackendId + (code, out, err) = be_run_full_compaction(backendId_to_backendIP.get(backend_id), backendId_to_backendHttpPort.get(backend_id), tablet_id) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactJson = parseJson(out.trim()) + assertEquals("success", compactJson.status.toLowerCase()) + } + + // wait for all compactions done + for (def tablet in tablets) { + Awaitility.await().atMost(30, TimeUnit.MINUTES).untilAsserted(() -> { + Thread.sleep(30000) + String tablet_id = tablet.TabletId + backend_id = tablet.BackendId + (code, out, err) = be_get_compaction_status(backendId_to_backendIP.get(backend_id), backendId_to_backendHttpPort.get(backend_id), tablet_id) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("compaction task for this tablet is not running", compactionStatus.msg.toLowerCase()) + }); + } + + + for (def tablet in tablets) { + int afterSegmentCount = 0 + String tablet_id = tablet.TabletId + (code, out, err) = curl("GET", tablet.CompactionStatus) + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + for (String rowset in (List) tabletJson.rowsets) { + logger.info("rowset is: " + rowset) + afterSegmentCount += Integer.parseInt(rowset.split(" ")[1]) + } + assertEquals(afterSegmentCount, 1) + } + } + + double localIndexSize = 0 + double localSegmentSize = 0 + + def check_size_equal = { double result1, double result2 -> + double tolerance = 0.1 * Math.max(result1, result2); + return Math.abs(result1 - result2) <= tolerance; + } + + def check_show_data = { FileSizeChange expect_idx, FileSizeChange expect_data -> + Thread.sleep(90000) + Awaitility.await().atMost(5, TimeUnit.MINUTES).untilAsserted(() -> { + Thread.sleep(10000) + def result = sql """ show data all;""" + logger.info("show data all; result is: ${result}") + def currentLocalIndexSize = convert_size.call(result[0][4]) + def currentSegmentIndexSize = convert_size.call(result[0][3]) + + if (expect_idx == FileSizeChange.LARGER) { + assertTrue(currentLocalIndexSize > localIndexSize) + } else if (expect_idx == FileSizeChange.SMALLER) { + assertTrue(currentLocalIndexSize < localIndexSize) + } else { + assertTrue(check_size_equal(currentLocalIndexSize, localIndexSize)) + } + + if (expect_data == FileSizeChange.LARGER) { + assertTrue(currentSegmentIndexSize > localSegmentSize) + } else if (expect_data == FileSizeChange.SMALLER) { + assertTrue(currentSegmentIndexSize < localSegmentSize) + } else { + assertTrue(check_size_equal(currentSegmentIndexSize, localSegmentSize)) + } + assertTrue(currentLocalIndexSize != 0) + assertTrue(currentSegmentIndexSize != 0) + localIndexSize = currentLocalIndexSize + localSegmentSize = currentSegmentIndexSize + + def result2 = sql """ select * from information_schema.tables where TABLE_NAME = '${show_table_name}' """ + logger.info("result 2 is: ${result2}") + def currentLocalIndexSize2 = result2[0][11] as double + def currentSegmentIndexSize2 = result2[0][9] as double + logger.info("currentLocalIndexSize2 is: ${currentLocalIndexSize2}, currentSegmentIndexSize2 is: ${currentSegmentIndexSize2}") + assertTrue(check_size_equal(currentLocalIndexSize, currentLocalIndexSize2)) + assertTrue(check_size_equal(currentSegmentIndexSize, currentSegmentIndexSize2)) + logger.info("show data all localIndexSize is: " + localIndexSize) + logger.info("show data all localSegmentSize is: " + localSegmentSize) + }); + } + + def schema_change = { + def tablets = sql_return_maparray """ show tablets from ${show_table_name}; """ + Set rowsetids = new HashSet<>(); + for (def tablet in tablets) { + String tablet_id = tablet.TabletId + (code, out, err) = curl("GET", tablet.CompactionStatus) + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + for (String rowset in (List) tabletJson.rowsets) { + int segmentCount = Integer.parseInt(rowset.split(" ")[1]) + if (segmentCount == 0) { + continue; + } + String rowsetid = rowset.split(" ")[4]; + rowsetids.add(rowsetid) + logger.info("rowsetid: " + rowsetid) + } + } + sql """ alter table ${show_table_name} drop column clientip""" + Awaitility.await().atMost(30, TimeUnit.MINUTES).untilAsserted(() -> { + Thread.sleep(30000) + tablets = sql_return_maparray """ show tablets from ${show_table_name}; """ + for (def tablet in tablets) { + String tablet_id = tablet.TabletId + (code, out, err) = curl("GET", tablet.CompactionStatus) + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + for (String rowset in (List) tabletJson.rowsets) { + int segmentCount = Integer.parseInt(rowset.split(" ")[1]) + if (segmentCount == 0) { + continue; + } + String rowsetid = rowset.split(" ")[4]; + logger.info("rowsetid: " + rowsetid) + assertTrue(!rowsetids.contains(rowsetid)) + } + } + }); + } + + def build_index = { + + def tablets = sql_return_maparray """ show tablets from ${show_table_name}; """ + Set rowsetids = new HashSet<>(); + for (def tablet in tablets) { + String tablet_id = tablet.TabletId + (code, out, err) = curl("GET", tablet.CompactionStatus) + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + for (String rowset in (List) tabletJson.rowsets) { + int segmentCount = Integer.parseInt(rowset.split(" ")[1]) + if (segmentCount == 0) { + continue; + } + String rowsetid = rowset.split(" ")[4]; + rowsetids.add(rowsetid) + logger.info("rowsetid: " + rowsetid) + } + } + sql """ ALTER TABLE ${show_table_name} ADD INDEX status_idx (status) using inverted; """ + if (!isCloudMode()) { + sql """ build index status_idx on ${show_table_name}""" + } + Awaitility.await().atMost(30, TimeUnit.MINUTES).untilAsserted(() -> { + Thread.sleep(30000) + tablets = sql_return_maparray """ show tablets from ${show_table_name}; """ + for (def tablet in tablets) { + String tablet_id = tablet.TabletId + (code, out, err) = curl("GET", tablet.CompactionStatus) + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + for (String rowset in (List) tabletJson.rowsets) { + int segmentCount = Integer.parseInt(rowset.split(" ")[1]) + if (segmentCount == 0) { + continue; + } + String rowsetid = rowset.split(" ")[4]; + logger.info("rowsetid: " + rowsetid) + assertTrue(!rowsetids.contains(rowsetid)) + } + } + }); + } + + def drop_index = { + def tablets = sql_return_maparray """ show tablets from ${show_table_name}; """ + Set rowsetids = new HashSet<>(); + for (def tablet in tablets) { + String tablet_id = tablet.TabletId + (code, out, err) = curl("GET", tablet.CompactionStatus) + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + for (String rowset in (List) tabletJson.rowsets) { + int segmentCount = Integer.parseInt(rowset.split(" ")[1]) + if (segmentCount == 0) { + continue; + } + String rowsetid = rowset.split(" ")[4]; + rowsetids.add(rowsetid) + logger.info("rowsetid: " + rowsetid) + } + } + sql """ DROP INDEX status_idx on ${show_table_name}""" + Awaitility.await().atMost(30, TimeUnit.MINUTES).untilAsserted(() -> { + Thread.sleep(30000) + tablets = sql_return_maparray """ show tablets from ${show_table_name}; """ + for (def tablet in tablets) { + String tablet_id = tablet.TabletId + (code, out, err) = curl("GET", tablet.CompactionStatus) + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + for (String rowset in (List) tabletJson.rowsets) { + int segmentCount = Integer.parseInt(rowset.split(" ")[1]) + if (segmentCount == 0) { + continue; + } + String rowsetid = rowset.split(" ")[4]; + logger.info("rowsetid: " + rowsetid) + assertTrue(!rowsetids.contains(rowsetid)) + } + } + }); + } + + // 1. load data + def executor = Executors.newFixedThreadPool(5) + (1..10).each { i -> + executor.submit { + def fileName = "documents-" + i + ".json" + load_json_data.call(show_table_name, """${getS3Url()}/regression/inverted_index_cases/httplogs/${fileName}""") + } + } + executor.shutdown() + executor.awaitTermination(30, TimeUnit.MINUTES) + + // 2. check show data + check_show_data.call(FileSizeChange.LARGER, FileSizeChange.LARGER) + + // 3. compaction + compaction.call() + + // 4. check show data + check_show_data.call(FileSizeChange.SMALLER, FileSizeChange.SMALLER) + + // 5. schema change + schema_change.call() + + // 6.check show data + check_show_data.call(FileSizeChange.SMALLER, FileSizeChange.SMALLER) + + // 7. build index + build_index.call() + + // 8.check show data + check_show_data.call(FileSizeChange.LARGER, FileSizeChange.UNCHANGED) + + // 9. drop index + drop_index.call() + + // 10.check show data + check_show_data.call(FileSizeChange.SMALLER, FileSizeChange.UNCHANGED) +} diff --git a/regression-test/suites/inverted_index_p2/show_data/test_show_index_data_p2.groovy b/regression-test/suites/inverted_index_p2/show_data/test_show_index_data_p2.groovy new file mode 100644 index 00000000000000..2839a8a47b04aa --- /dev/null +++ b/regression-test/suites/inverted_index_p2/show_data/test_show_index_data_p2.groovy @@ -0,0 +1,397 @@ +// 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. + +import java.util.concurrent.Executors +import java.util.concurrent.TimeUnit +import org.awaitility.Awaitility + +enum FileSizeChange { + LARGER, + SMALLER, + UNCHANGED +} + +suite("test_show_index_data_p2", "p2") { + + def show_table_name = "test_show_index_data_p2" + + def load_json_data = { table_name, file_name -> + // load the json data + streamLoad { + table "${table_name}" + + // set http request header params + set 'read_json_by_line', 'true' + set 'format', 'json' + set 'max_filter_ratio', '0.1' + file file_name // import json file + time 10000 // limit inflight 10s + + // if declared a check callback, the default check condition will ignore. + // So you must check all condition + + check { result, exception, startTime, endTime -> + if (exception != null) { + throw exception + } + logger.info("Stream load ${file_name} result: ${result}".toString()) + def json = parseJson(result) + assertEquals("success", json.Status.toLowerCase()) + // assertEquals(json.NumberTotalRows, json.NumberLoadedRows + json.NumberUnselectedRows) + assertTrue(json.NumberLoadedRows > 0 && json.LoadBytes > 0) + } + } + } + + def backendId_to_backendIP = [:] + def backendId_to_backendHttpPort = [:] + getBackendIpHttpPort(backendId_to_backendIP, backendId_to_backendHttpPort); + + + + def convert_size = { str -> + def matcher = str =~ /(\d+(\.\d+)?)\s*(KB|MB|GB|TB|B)/ + if (matcher) { + def value = matcher[0][1] as double + def unit = matcher[0][3] + logger.info("value is: " + value + " unit is: " + unit) + def result = 0.0 + switch (unit) { + case 'KB': + result = value * 1024 + break + case 'MB': + result = value * 1024 * 1024 + break + case 'GB': + result = value * 1024 * 1024 * 1024 + break + case 'B': + result = value + break + default: + throw new IllegalArgumentException("Unknown unit: $unit") + } + + return result + } else { + return 0 + } + } + + sql "DROP TABLE IF EXISTS ${show_table_name}" + sql """ + CREATE TABLE ${show_table_name} ( + `@timestamp` int(11) NULL, + `clientip` varchar(20) NULL, + `request` varchar(500) NULL, + `status` int NULL, + `size` int NULL, + INDEX clientip_idx (`clientip`) USING INVERTED COMMENT '', + INDEX request_idx (`request`) USING INVERTED PROPERTIES("parser" = "unicode") COMMENT '' + ) ENGINE=OLAP + DUPLICATE KEY(`@timestamp`, `clientip`) + DISTRIBUTED BY HASH(`@timestamp`) BUCKETS 1 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1", + "compaction_policy" = "time_series", + "time_series_compaction_file_count_threshold" = "20", + "disable_auto_compaction" = "true" + ); + """ + + def compaction = { + def tablets = sql_return_maparray """ show tablets from ${show_table_name}; """ + for (def tablet in tablets) { + int beforeSegmentCount = 0 + String tablet_id = tablet.TabletId + (code, out, err) = curl("GET", tablet.CompactionStatus) + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + for (String rowset in (List) tabletJson.rowsets) { + beforeSegmentCount += Integer.parseInt(rowset.split(" ")[1]) + } + assertEquals(beforeSegmentCount, 110) + } + + // trigger compactions for all tablets in ${tableName} + for (def tablet in tablets) { + String tablet_id = tablet.TabletId + backend_id = tablet.BackendId + (code, out, err) = be_run_full_compaction(backendId_to_backendIP.get(backend_id), backendId_to_backendHttpPort.get(backend_id), tablet_id) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactJson = parseJson(out.trim()) + assertEquals("success", compactJson.status.toLowerCase()) + } + + // wait for all compactions done + for (def tablet in tablets) { + Awaitility.await().atMost(60, TimeUnit.MINUTES).untilAsserted(() -> { + Thread.sleep(30000) + String tablet_id = tablet.TabletId + backend_id = tablet.BackendId + (code, out, err) = be_get_compaction_status(backendId_to_backendIP.get(backend_id), backendId_to_backendHttpPort.get(backend_id), tablet_id) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("compaction task for this tablet is not running", compactionStatus.msg.toLowerCase()) + }); + } + + for (def tablet in tablets) { + int afterSegmentCount = 0 + String tablet_id = tablet.TabletId + (code, out, err) = curl("GET", tablet.CompactionStatus) + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + for (String rowset in (List) tabletJson.rowsets) { + logger.info("rowset is: " + rowset) + afterSegmentCount += Integer.parseInt(rowset.split(" ")[1]) + } + assertEquals(afterSegmentCount, 1) + } + + } + + double localIndexSize = 0 + double localSegmentSize = 0 + + def check_size_equal = { double result1, double result2 -> + double tolerance = 0.1 * Math.max(result1, result2); + return Math.abs(result1 - result2) <= tolerance; + } + + def check_show_data = { FileSizeChange expect_idx, FileSizeChange expect_data -> + Thread.sleep(90000) + Awaitility.await().atMost(10, TimeUnit.MINUTES).untilAsserted(() -> { + Thread.sleep(10000) + def result = sql """ show data all;""" + logger.info("show data all; result is: ${result}") + def currentLocalIndexSize = convert_size.call(result[0][4]) + def currentSegmentIndexSize = convert_size.call(result[0][3]) + + if (expect_idx == FileSizeChange.LARGER) { + assertTrue(currentLocalIndexSize > localIndexSize) + } else if (expect_idx == FileSizeChange.SMALLER) { + assertTrue(currentLocalIndexSize < localIndexSize) + } else { + assertTrue(check_size_equal(currentLocalIndexSize, localIndexSize)) + } + + if (expect_data == FileSizeChange.LARGER) { + assertTrue(currentSegmentIndexSize > localSegmentSize) + } else if (expect_data == FileSizeChange.SMALLER) { + assertTrue(currentSegmentIndexSize < localSegmentSize) + } else { + assertTrue(check_size_equal(currentSegmentIndexSize, localSegmentSize)) + } + + assertTrue(currentLocalIndexSize != 0) + assertTrue(currentSegmentIndexSize != 0) + localIndexSize = currentLocalIndexSize + localSegmentSize = currentSegmentIndexSize + + def result2 = sql """ select * from information_schema.tables where TABLE_NAME = '${show_table_name}' """ + logger.info("result 2 is: ${result2}") + def currentLocalIndexSize2 = result2[0][11] as double + def currentSegmentIndexSize2 = result2[0][9] as double + logger.info("currentLocalIndexSize2 is: ${currentLocalIndexSize2}, currentSegmentIndexSize2 is: ${currentSegmentIndexSize2}") + assertTrue(check_size_equal(currentLocalIndexSize, currentLocalIndexSize2)) + assertTrue(check_size_equal(currentSegmentIndexSize, currentSegmentIndexSize2)) + logger.info("show data all localIndexSize is: " + localIndexSize) + logger.info("show data all localSegmentSize is: " + localSegmentSize) + }); + } + + def schema_change = { + def tablets = sql_return_maparray """ show tablets from ${show_table_name}; """ + Set rowsetids = new HashSet<>(); + for (def tablet in tablets) { + String tablet_id = tablet.TabletId + (code, out, err) = curl("GET", tablet.CompactionStatus) + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + for (String rowset in (List) tabletJson.rowsets) { + int segmentCount = Integer.parseInt(rowset.split(" ")[1]) + if (segmentCount == 0) { + continue; + } + String rowsetid = rowset.split(" ")[4]; + rowsetids.add(rowsetid) + logger.info("rowsetid: " + rowsetid) + } + } + sql """ alter table ${show_table_name} drop column clientip""" + Awaitility.await().atMost(60, TimeUnit.MINUTES).untilAsserted(() -> { + Thread.sleep(30000) + tablets = sql_return_maparray """ show tablets from ${show_table_name}; """ + for (def tablet in tablets) { + String tablet_id = tablet.TabletId + (code, out, err) = curl("GET", tablet.CompactionStatus) + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + for (String rowset in (List) tabletJson.rowsets) { + int segmentCount = Integer.parseInt(rowset.split(" ")[1]) + if (segmentCount == 0) { + continue; + } + String rowsetid = rowset.split(" ")[4]; + logger.info("rowsetid: " + rowsetid) + assertTrue(!rowsetids.contains(rowsetid)) + } + } + }); + } + + def build_index = { + def tablets = sql_return_maparray """ show tablets from ${show_table_name}; """ + Set rowsetids = new HashSet<>(); + for (def tablet in tablets) { + String tablet_id = tablet.TabletId + (code, out, err) = curl("GET", tablet.CompactionStatus) + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + for (String rowset in (List) tabletJson.rowsets) { + int segmentCount = Integer.parseInt(rowset.split(" ")[1]) + if (segmentCount == 0) { + continue; + } + String rowsetid = rowset.split(" ")[4]; + rowsetids.add(rowsetid) + logger.info("rowsetid: " + rowsetid) + } + } + sql """ ALTER TABLE ${show_table_name} ADD INDEX status_idx (status) using inverted; """ + if (!isCloudMode()) { + sql """ build index status_idx on ${show_table_name}""" + } + Awaitility.await().atMost(60, TimeUnit.MINUTES).untilAsserted(() -> { + Thread.sleep(30000) + tablets = sql_return_maparray """ show tablets from ${show_table_name}; """ + for (def tablet in tablets) { + String tablet_id = tablet.TabletId + (code, out, err) = curl("GET", tablet.CompactionStatus) + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + for (String rowset in (List) tabletJson.rowsets) { + int segmentCount = Integer.parseInt(rowset.split(" ")[1]) + if (segmentCount == 0) { + continue; + } + String rowsetid = rowset.split(" ")[4]; + logger.info("rowsetid: " + rowsetid) + assertTrue(!rowsetids.contains(rowsetid)) + } + } + }); + } + + def drop_index = { + def tablets = sql_return_maparray """ show tablets from ${show_table_name}; """ + Set rowsetids = new HashSet<>(); + for (def tablet in tablets) { + String tablet_id = tablet.TabletId + (code, out, err) = curl("GET", tablet.CompactionStatus) + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + for (String rowset in (List) tabletJson.rowsets) { + int segmentCount = Integer.parseInt(rowset.split(" ")[1]) + if (segmentCount == 0) { + continue; + } + String rowsetid = rowset.split(" ")[4]; + rowsetids.add(rowsetid) + logger.info("rowsetid: " + rowsetid) + } + } + sql """ DROP INDEX status_idx on ${show_table_name}""" + Awaitility.await().atMost(60, TimeUnit.MINUTES).untilAsserted(() -> { + Thread.sleep(30000) + tablets = sql_return_maparray """ show tablets from ${show_table_name}; """ + for (def tablet in tablets) { + String tablet_id = tablet.TabletId + (code, out, err) = curl("GET", tablet.CompactionStatus) + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + for (String rowset in (List) tabletJson.rowsets) { + int segmentCount = Integer.parseInt(rowset.split(" ")[1]) + if (segmentCount == 0) { + continue; + } + String rowsetid = rowset.split(" ")[4]; + logger.info("rowsetid: " + rowsetid) + assertTrue(!rowsetids.contains(rowsetid)) + } + } + }); + } + + // 1. load data + def executor = Executors.newFixedThreadPool(5) + (1..110).each { i -> + executor.submit { + def fileName = "documents-" + i + ".json" + load_json_data.call(show_table_name, """${getS3Url()}/regression/inverted_index_cases/httplogs/${fileName}""") + } + } + executor.shutdown() + executor.awaitTermination(60, TimeUnit.MINUTES) + + // 2. check show data + check_show_data.call(FileSizeChange.LARGER, FileSizeChange.LARGER) + + // 3. compaction + compaction.call() + + // 4. check show data + check_show_data.call(FileSizeChange.SMALLER, FileSizeChange.LARGER) + + // 5. schema change + schema_change.call() + + // 6.check show data + check_show_data.call(FileSizeChange.SMALLER, FileSizeChange.SMALLER) + + // 7. build index + build_index.call() + + // 8.check show data + check_show_data.call(FileSizeChange.LARGER, FileSizeChange.UNCHANGED) + + // 9. drop index + drop_index.call() + + // 10.check show data + check_show_data.call(FileSizeChange.SMALLER, FileSizeChange.UNCHANGED) +} From b2b0ac76c6c0e8533673486ba6db7b04b9ec4627 Mon Sep 17 00:00:00 2001 From: airborne12 Date: Mon, 2 Dec 2024 14:54:22 +0800 Subject: [PATCH 110/399] [feat](test) add bloomfilter ut case (#44351) add bloomfilter index UT test case for BE --- be/src/olap/bloom_filter.hpp | 272 ---------- .../segment_v2/block_split_bloom_filter.h | 1 - be/src/olap/rowset/segment_v2/bloom_filter.h | 2 +- .../segment_v2/bloom_filter_index_reader.cpp | 1 + .../segment_v2/bloom_filter_index_writer.cpp | 29 +- be/test/olap/bloom_filter_test.cpp | 160 ------ .../bloom_filter_index_reader_writer_test.cpp | 485 +++++++++++++++++- 7 files changed, 488 insertions(+), 462 deletions(-) delete mode 100644 be/src/olap/bloom_filter.hpp delete mode 100644 be/test/olap/bloom_filter_test.cpp diff --git a/be/src/olap/bloom_filter.hpp b/be/src/olap/bloom_filter.hpp deleted file mode 100644 index 5c7cb5f9e6419f..00000000000000 --- a/be/src/olap/bloom_filter.hpp +++ /dev/null @@ -1,272 +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. - -#ifndef DORIS_BE_SRC_OLAP_COLUMN_FILE_BLOOM_FILTER_HPP -#define DORIS_BE_SRC_OLAP_COLUMN_FILE_BLOOM_FILTER_HPP - -#include - -#include -#include - -#include "olap/olap_define.h" -#include "olap/utils.h" -#include "util/hash_util.hpp" - -namespace doris { - -static const uint64_t DEFAULT_SEED = 104729; -static const uint64_t BLOOM_FILTER_NULL_HASHCODE = 2862933555777941757ULL; - -struct BloomFilterIndexHeader { - uint64_t block_count; - BloomFilterIndexHeader() : block_count(0) {} -} __attribute__((packed)); - -// Bare metal bit set implementation. For performance reasons, this implementation does not -// check for index bounds nor expand the bit set if the specified index is greater than the size. -class BitSet { -public: - BitSet() : _data(nullptr), _data_len(0) {} - - ~BitSet() { SAFE_DELETE_ARRAY(_data); } - - // Init BitSet with given bit_num, which will align up to uint64_t - bool init(uint32_t bit_num) { - if (bit_num <= 0) { - return false; - } - - _data_len = (bit_num + sizeof(uint64_t) * 8 - 1) / (sizeof(uint64_t) * 8); - _data = new (std::nothrow) uint64_t[_data_len]; - if (_data == nullptr) { - return false; - } - - memset(_data, 0, _data_len * sizeof(uint64_t)); - return true; - } - - // Init BitSet with given buffer - bool init(uint64_t* data, uint32_t data_len) { - _data = data; - _data_len = data_len; - return true; - } - - // Set the bit specified by param, note that uint64_t type contains 2^6 bits - void set(uint32_t index) { _data[index >> 6] |= 1L << (index % 64); } - - // Return true if the bit specified by param is set - bool get(uint32_t index) const { return (_data[index >> 6] & (1L << (index % 64))) != 0; } - - // Merge with another BitSet by byte, return false when the length is not equal - bool merge(const BitSet& set) { - if (_data_len != set.data_len()) { - return false; - } - - for (uint32_t i = 0; i < _data_len; ++i) { - _data[i] |= set.data()[i]; - } - - return true; - } - - // Convert BitSet to string to convenient debug and test - std::string to_string() const { - uint32_t bit_num = _data_len * sizeof(uint64_t) * 8; - std::string str(bit_num, '0'); - for (uint32_t i = 0; i < bit_num; ++i) { - if ((_data[i >> 6] & (1L << i)) != 0) { - str[i] = '1'; - } - } - - return str; - } - - uint64_t* data() const { return _data; } - - uint32_t data_len() const { return _data_len; } - - uint32_t bit_num() const { return _data_len * sizeof(uint64_t) * 8; } - - void clear() { memset(_data, 0, _data_len * sizeof(uint64_t)); } - - void reset() { - _data = NULL; - _data_len = 0; - } - -private: - uint64_t* _data; - uint32_t _data_len; -}; - -class BloomFilter { -public: - BloomFilter() : _bit_num(0), _hash_function_num(0) {} - ~BloomFilter() {} - - // Create BloomFilter with given entry num and fpp, which is used for loading data - bool init(int64_t expected_entries, double fpp) { - uint32_t bit_num = _optimal_bit_num(expected_entries, fpp); - if (!_bit_set.init(bit_num)) { - return false; - } - - _bit_num = _bit_set.bit_num(); - _hash_function_num = _optimal_hash_function_num(expected_entries, _bit_num); - return true; - } - - // Create BloomFilter with given entry num and default fpp - bool init(int64_t expected_entries) { - return this->init(expected_entries, BLOOM_FILTER_DEFAULT_FPP); - } - - // Init BloomFilter with given buffer, which is used for query - bool init(uint64_t* data, uint32_t len, uint32_t hash_function_num) { - _bit_num = sizeof(uint64_t) * 8 * len; - _hash_function_num = hash_function_num; - return _bit_set.init(data, len); - } - - // Compute hash value of given buffer and add to BloomFilter - void add_bytes(const char* buf, uint32_t len) { - uint64_t hash = buf == nullptr ? BLOOM_FILTER_NULL_HASHCODE - : HashUtil::hash64(buf, len, DEFAULT_SEED); - add_hash(hash); - } - - // Generate multiple hash value according to following rule: - // new_hash_value = hash_high_part + (i * hash_low_part) - void add_hash(uint64_t hash) { - uint32_t hash1 = (uint32_t)hash; - uint32_t hash2 = (uint32_t)(hash >> 32); - - for (uint32_t i = 0; i < _hash_function_num; ++i) { - uint64_t combine_hash = hash1 + hash2 * i; - uint32_t index = combine_hash % _bit_num; - _bit_set.set(index); - } - } - - // Compute hash value of given buffer and verify whether exist in BloomFilter - bool test_bytes(const char* buf, uint32_t len) const { - uint64_t hash = buf == nullptr ? BLOOM_FILTER_NULL_HASHCODE - : HashUtil::hash64(buf, len, DEFAULT_SEED); - return test_hash(hash); - } - - // Verify whether hash value in BloomFilter - bool test_hash(uint64_t hash) const { - uint32_t hash1 = (uint32_t)hash; - uint32_t hash2 = (uint32_t)(hash >> 32); - - for (uint32_t i = 0; i < _hash_function_num; ++i) { - uint64_t combine_hash = hash1 + hash2 * i; - uint32_t index = combine_hash % _bit_num; - if (!_bit_set.get(index)) { - return false; - } - } - - return true; - } - - // Merge with another BloomFilter, return false when the length - // and hash function number is not equal - bool merge(const BloomFilter& that) { - if (_bit_num == that.bit_num() && _hash_function_num == that.hash_function_num()) { - _bit_set.merge(that.bit_set()); - return true; - } - - return false; - } - - void clear() { _bit_set.clear(); } - - void reset() { - _bit_num = 0; - _hash_function_num = 0; - _bit_set.reset(); - } - - uint32_t bit_num() const { return _bit_num; } - - uint32_t hash_function_num() const { return _hash_function_num; } - - const BitSet& bit_set() const { return _bit_set; } - - uint64_t* bit_set_data() const { return _bit_set.data(); } - - uint32_t bit_set_data_len() const { return _bit_set.data_len(); } - - // Convert BloomFilter to string to convenient debug and test - std::string to_string() const { - std::stringstream bf_stream; - bf_stream << "bit_num:" << _bit_num << " hash_function_num:" << _hash_function_num - << " bit_set:" << _bit_set.to_string(); - return bf_stream.str(); - } - - // Get points which set by given buffer in the BitSet - std::string get_bytes_points_string(const char* buf, uint32_t len) const { - uint64_t hash = buf == nullptr ? BLOOM_FILTER_NULL_HASHCODE - : HashUtil::hash64(buf, len, DEFAULT_SEED); - uint32_t hash1 = (uint32_t)hash; - uint32_t hash2 = (uint32_t)(hash >> 32); - - std::stringstream stream; - for (uint32_t i = 0; i < _hash_function_num; ++i) { - if (i != 0) { - stream << "-"; - } - - uint32_t combine_hash = hash1 + hash2 * i; - uint32_t index = combine_hash % _bit_num; - stream << index; - } - - return stream.str(); - } - -private: - // Compute the optimal bit number according to the following rule: - // m = -n * ln(fpp) / (ln(2) ^ 2) - uint32_t _optimal_bit_num(int64_t n, double fpp) { - return (uint32_t)(-n * log(fpp) / (log(2) * log(2))); - } - - // Compute the optimal hash function number according to the following rule: - // k = round(m * ln(2) / n) - uint32_t _optimal_hash_function_num(int64_t n, uint32_t m) { - uint32_t k = (uint32_t)round(m * log(2) / n); - return k > 1 ? k : 1; - } - - BitSet _bit_set; - uint32_t _bit_num; - uint32_t _hash_function_num; -}; - -} // namespace doris - -#endif // DORIS_BE_SRC_OLAP_COLUMN_FILE_BLOOM_FILTER_HPP diff --git a/be/src/olap/rowset/segment_v2/block_split_bloom_filter.h b/be/src/olap/rowset/segment_v2/block_split_bloom_filter.h index f68ddd7e74bfc5..8dc470d9da4f88 100644 --- a/be/src/olap/rowset/segment_v2/block_split_bloom_filter.h +++ b/be/src/olap/rowset/segment_v2/block_split_bloom_filter.h @@ -34,7 +34,6 @@ class BlockSplitBloomFilter : public BloomFilter { void add_hash(uint64_t hash) override; bool test_hash(uint64_t hash) const override; - bool contains(const BloomFilter&) const override { return true; } private: // Bytes in a tiny Bloom filter block. diff --git a/be/src/olap/rowset/segment_v2/bloom_filter.h b/be/src/olap/rowset/segment_v2/bloom_filter.h index a7845d1ca36704..4f4adf0fd12283 100644 --- a/be/src/olap/rowset/segment_v2/bloom_filter.h +++ b/be/src/olap/rowset/segment_v2/bloom_filter.h @@ -186,7 +186,7 @@ class BloomFilter { /// Checks if this contains everything from another bloom filter. /// Bloom filters must have equal size and seed. - virtual bool contains(const BloomFilter& bf_) const = 0; + virtual bool contains(const BloomFilter& bf_) const { return true; }; virtual char* data() const { return _data; } diff --git a/be/src/olap/rowset/segment_v2/bloom_filter_index_reader.cpp b/be/src/olap/rowset/segment_v2/bloom_filter_index_reader.cpp index 609d21ce4f5c22..8c63c25d20acee 100644 --- a/be/src/olap/rowset/segment_v2/bloom_filter_index_reader.cpp +++ b/be/src/olap/rowset/segment_v2/bloom_filter_index_reader.cpp @@ -70,6 +70,7 @@ Status BloomFilterIndexIterator::read_bloom_filter(rowid_t ordinal, auto column = data_type->create_column(); RETURN_IF_ERROR(_bloom_filter_iter.seek_to_ordinal(ordinal)); + DCHECK(current_bloom_filter_index() == ordinal); size_t num_read = num_to_read; RETURN_IF_ERROR(_bloom_filter_iter.next_batch(&num_read, column)); DCHECK(num_to_read == num_read); diff --git a/be/src/olap/rowset/segment_v2/bloom_filter_index_writer.cpp b/be/src/olap/rowset/segment_v2/bloom_filter_index_writer.cpp index edc6102703f492..3f9fb94df0a844 100644 --- a/be/src/olap/rowset/segment_v2/bloom_filter_index_writer.cpp +++ b/be/src/olap/rowset/segment_v2/bloom_filter_index_writer.cpp @@ -21,6 +21,7 @@ #include #include +#include #include #include #include @@ -68,15 +69,12 @@ class BloomFilterIndexWriterImpl : public BloomFilterIndexWriter { explicit BloomFilterIndexWriterImpl(const BloomFilterOptions& bf_options, const TypeInfo* type_info) - : _bf_options(bf_options), - _type_info(type_info), - _has_null(false), - _bf_buffer_size(0) {} + : _bf_options(bf_options), _type_info(type_info) {} ~BloomFilterIndexWriterImpl() override = default; Status add_values(const void* values, size_t count) override { - const CppType* v = (const CppType*)values; + const auto* v = (const CppType*)values; for (int i = 0; i < count; ++i) { if (_values.find(*v) == _values.end()) { if constexpr (_is_slice_type()) { @@ -105,7 +103,7 @@ class BloomFilterIndexWriterImpl : public BloomFilterIndexWriter { bf->set_has_null(_has_null); for (auto& v : _values) { if constexpr (_is_slice_type()) { - Slice* s = (Slice*)&v; + auto* s = (Slice*)&v; bf->add_bytes(s->data, s->size); } else { bf->add_bytes((char*)&v, sizeof(CppType)); @@ -160,11 +158,11 @@ class BloomFilterIndexWriterImpl : public BloomFilterIndexWriter { static constexpr bool _is_int128() { return field_type == FieldType::OLAP_FIELD_TYPE_LARGEINT; } private: - BloomFilterOptions _bf_options; - const TypeInfo* _type_info; + BloomFilterOptions _bf_options {}; + const TypeInfo* _type_info = nullptr; vectorized::Arena _arena; - bool _has_null; - uint64_t _bf_buffer_size; + bool _has_null = false; + uint64_t _bf_buffer_size = 0; // distinct values ValueDict _values; std::vector> _bfs; @@ -173,7 +171,7 @@ class BloomFilterIndexWriterImpl : public BloomFilterIndexWriter { } // namespace Status PrimaryKeyBloomFilterIndexWriterImpl::add_values(const void* values, size_t count) { - const Slice* v = (const Slice*)values; + const auto* v = (const Slice*)values; for (int i = 0; i < count; ++i) { Slice new_value; RETURN_IF_CATCH_EXCEPTION(_type_info->deep_copy(&new_value, v, &_arena)); @@ -189,7 +187,7 @@ Status PrimaryKeyBloomFilterIndexWriterImpl::flush() { RETURN_IF_ERROR(bf->init(_values.size(), _bf_options.fpp, _bf_options.strategy)); bf->set_has_null(_has_null); for (auto& v : _values) { - Slice* s = (Slice*)&v; + auto* s = (Slice*)&v; bf->add_bytes(s->data, s->size); } _bf_buffer_size += bf->size(); @@ -205,7 +203,7 @@ Status PrimaryKeyBloomFilterIndexWriterImpl::flush() { Status PrimaryKeyBloomFilterIndexWriterImpl::finish(io::FileWriter* file_writer, ColumnIndexMetaPB* index_meta) { - if (_values.size() > 0) { + if (!_values.empty()) { RETURN_IF_ERROR(flush()); } index_meta->set_type(BLOOM_FILTER_INDEX); @@ -246,7 +244,7 @@ NGramBloomFilterIndexWriterImpl::NGramBloomFilterIndexWriterImpl( } Status NGramBloomFilterIndexWriterImpl::add_values(const void* values, size_t count) { - const Slice* src = reinterpret_cast(values); + const auto* src = reinterpret_cast(values); for (int i = 0; i < count; ++i, ++src) { if (src->size < _gram_size) { continue; @@ -339,7 +337,8 @@ Status NGramBloomFilterIndexWriterImpl::create(const BloomFilterOptions& bf_opti case FieldType::OLAP_FIELD_TYPE_CHAR: case FieldType::OLAP_FIELD_TYPE_VARCHAR: case FieldType::OLAP_FIELD_TYPE_STRING: - res->reset(new NGramBloomFilterIndexWriterImpl(bf_options, gram_size, gram_bf_size)); + *res = std::make_unique(bf_options, gram_size, + gram_bf_size); break; default: return Status::NotSupported("unsupported type for ngram bloom filter index:{}", diff --git a/be/test/olap/bloom_filter_test.cpp b/be/test/olap/bloom_filter_test.cpp deleted file mode 100644 index 32bf88e2c4d684..00000000000000 --- a/be/test/olap/bloom_filter_test.cpp +++ /dev/null @@ -1,160 +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 "olap/bloom_filter.hpp" - -#include -#include -#include - -#include - -#include "gtest/gtest_pred_impl.h" - -using std::string; - -namespace doris { - -class TestBloomFilter : public testing::Test { -public: - virtual ~TestBloomFilter() {} - - virtual void SetUp() {} - virtual void TearDown() {} -}; - -// Init BloomFilter with different item number and fpp, -// and verify bit_num and hash_function_num calculated by BloomFilter -TEST_F(TestBloomFilter, init_bloom_filter) { - { - BloomFilter bf; - bf.init(1024); - EXPECT_EQ(6400, bf.bit_num()); - EXPECT_EQ(4, bf.hash_function_num()); - } - - { - BloomFilter bf; - bf.init(1024, 0.01); - EXPECT_EQ(9856, bf.bit_num()); - EXPECT_EQ(7, bf.hash_function_num()); - } - - { - BloomFilter bf; - bf.init(10240, 0.1); - EXPECT_EQ(49088, bf.bit_num()); - EXPECT_EQ(3, bf.hash_function_num()); - } - - { - BloomFilter bf; - uint32_t data_len = 100; - uint32_t hash_function_num = 4; - uint64_t* data = new uint64_t[data_len]; - - bf.init(data, data_len, hash_function_num); - EXPECT_EQ(6400, bf.bit_num()); - EXPECT_EQ(4, bf.hash_function_num()); - EXPECT_EQ(data, bf.bit_set_data()); - - bf.reset(); - EXPECT_EQ(0, bf.bit_num()); - EXPECT_EQ(0, bf.hash_function_num()); - EXPECT_EQ(nullptr, bf.bit_set_data()); - delete[] data; - } -} - -// Add different buffer to BloomFilter and verify existence -TEST_F(TestBloomFilter, add_and_test_bytes) { - string bytes; - BloomFilter bf; - bf.init(1024); - - bf.add_bytes(nullptr, 0); - EXPECT_TRUE(bf.test_bytes(nullptr, 0)); - - bytes = "hello"; - bf.add_bytes(bytes.c_str(), bytes.size()); - EXPECT_TRUE(bf.test_bytes(bytes.c_str(), bytes.size())); - - bytes = "doris"; - bf.add_bytes(bytes.c_str(), bytes.size()); - EXPECT_TRUE(bf.test_bytes(bytes.c_str(), bytes.size())); - - BloomFilter new_bf; - new_bf.init(1024); - - bytes = "world"; - new_bf.add_bytes(bytes.c_str(), bytes.size()); - EXPECT_TRUE(bf.merge(new_bf)); - EXPECT_TRUE(bf.test_bytes(bytes.c_str(), bytes.size())); -} - -// Print bloom filter buffer and points of specified string -TEST_F(TestBloomFilter, bloom_filter_info) { - string bytes; - BloomFilter bf; - bf.init(8, 0.1); - - bytes = "doris"; - bf.add_bytes(bytes.c_str(), bytes.size()); - string buffer_expect = - "bit_num:64 hash_function_num:6 " - "bit_set:0000100000000000100000010000000000010000001000000000000000000100"; - string buffer = bf.to_string(); - EXPECT_TRUE(buffer_expect == buffer); - - string points_expect = "4-23-42-61-16-35"; - string points = bf.get_bytes_points_string(bytes.c_str(), bytes.size()); - EXPECT_TRUE(points_expect == points); - - bytes = "a"; - points = bf.get_bytes_points_string(bytes.c_str(), bytes.size()); - LOG(WARNING) << "bytes=" << bytes << " points=" << points; - - bytes = "ab"; - points = bf.get_bytes_points_string(bytes.c_str(), bytes.size()); - LOG(WARNING) << "bytes=" << bytes << " points=" << points; - - bytes = "b"; - points = bf.get_bytes_points_string(bytes.c_str(), bytes.size()); - LOG(WARNING) << "bytes=" << bytes << " points=" << points; - - bytes = "ba"; - points = bf.get_bytes_points_string(bytes.c_str(), bytes.size()); - LOG(WARNING) << "bytes=" << bytes << " points=" << points; - - bytes = "c"; - points = bf.get_bytes_points_string(bytes.c_str(), bytes.size()); - LOG(WARNING) << "bytes=" << bytes << " points=" << points; - - bytes = "bc"; - points = bf.get_bytes_points_string(bytes.c_str(), bytes.size()); - LOG(WARNING) << "bytes=" << bytes << " points=" << points; - - bytes = "ac"; - points = bf.get_bytes_points_string(bytes.c_str(), bytes.size()); - LOG(WARNING) << "bytes=" << bytes << " points=" << points; - - bytes = "abc"; - points = bf.get_bytes_points_string(bytes.c_str(), bytes.size()); - LOG(WARNING) << "bytes=" << bytes << " points=" << points; -} - -} // namespace doris diff --git a/be/test/olap/rowset/segment_v2/bloom_filter_index_reader_writer_test.cpp b/be/test/olap/rowset/segment_v2/bloom_filter_index_reader_writer_test.cpp index 69cb343f04bf91..813952595efcfd 100644 --- a/be/test/olap/rowset/segment_v2/bloom_filter_index_reader_writer_test.cpp +++ b/be/test/olap/rowset/segment_v2/bloom_filter_index_reader_writer_test.cpp @@ -32,6 +32,7 @@ #include "io/fs/file_writer.h" #include "io/fs/local_file_system.h" #include "olap/decimal12.h" +#include "olap/itoken_extractor.h" #include "olap/olap_common.h" #include "olap/rowset/segment_v2/bloom_filter.h" #include "olap/rowset/segment_v2/bloom_filter_index_reader.h" @@ -62,18 +63,19 @@ template Status write_bloom_filter_index_file(const std::string& file_name, const void* values, size_t value_count, size_t null_count, ColumnIndexMetaPB* index_meta, - bool use_primary_key_bloom_filter = false) { + bool use_primary_key_bloom_filter = false, double fpp = 0.05) { const auto* type_info = get_scalar_type_info(); using CppType = typename CppTypeTraits::CppType; std::string fname = dname + "/" + file_name; auto fs = io::global_local_filesystem(); { + size_t expect_size = 0; io::FileWriterPtr file_writer; RETURN_IF_ERROR(fs->create_file(fname, &file_writer)); std::unique_ptr bloom_filter_index_writer; BloomFilterOptions bf_options; - + bf_options.fpp = fpp; // Set the expected FPP if (use_primary_key_bloom_filter) { RETURN_IF_ERROR(PrimaryKeyBloomFilterIndexWriterImpl::create( bf_options, type_info, &bloom_filter_index_writer)); @@ -91,12 +93,26 @@ Status write_bloom_filter_index_file(const std::string& file_name, const void* v bloom_filter_index_writer->add_nulls(null_count); } RETURN_IF_ERROR(bloom_filter_index_writer->flush()); + auto bf_size = BloomFilter::optimal_bit_num(num, fpp) / 8; + expect_size += bf_size + 1; i += 1024; } + if (value_count == 3072) { + RETURN_IF_ERROR(bloom_filter_index_writer->add_values(vals + 3071, 1)); + auto bf_size = BloomFilter::optimal_bit_num(1, fpp) / 8; + expect_size += bf_size + 1; + } RETURN_IF_ERROR(bloom_filter_index_writer->finish(file_writer.get(), index_meta)); EXPECT_TRUE(file_writer->close().ok()); EXPECT_EQ(BLOOM_FILTER_INDEX, index_meta->type()); EXPECT_EQ(bf_options.strategy, index_meta->bloom_filter_index().hash_strategy()); + if constexpr (!field_is_slice_type(type)) { + EXPECT_EQ(expect_size, bloom_filter_index_writer->size()); + } + if (use_primary_key_bloom_filter) { + std::cout << "primary key bf size is " << bloom_filter_index_writer->size() + << std::endl; + } } return Status::OK(); } @@ -128,7 +144,7 @@ Status test_bloom_filter_index_reader_writer_template( BloomFilterIndexReader* reader = nullptr; std::unique_ptr iter; get_bloom_filter_reader_iter(file_name, meta, &reader, &iter); - + EXPECT_EQ(reader->algorithm(), BloomFilterAlgorithmPB::BLOCK_BLOOM_FILTER); // page 0 std::unique_ptr bf; RETURN_IF_ERROR(iter->read_bloom_filter(0, &bf)); @@ -171,7 +187,7 @@ Status test_bloom_filter_index_reader_writer_template( } TEST_F(BloomFilterIndexReaderWriterTest, test_int) { - size_t num = 1024 * 3 - 1; + size_t num = 1024 * 3; int* val = new int[num]; for (int i = 0; i < num; ++i) { // there will be 3 bloom filter pages @@ -186,8 +202,58 @@ TEST_F(BloomFilterIndexReaderWriterTest, test_int) { delete[] val; } +TEST_F(BloomFilterIndexReaderWriterTest, test_string) { + size_t num = 1024 * 3; + std::vector val_strings(num); + for (size_t i = 0; i < num; ++i) { + val_strings[i] = "string_test_" + std::to_string(i + 1); + } + Slice* val = new Slice[num]; + for (size_t i = 0; i < num; ++i) { + val[i] = Slice(val_strings[i]); + } + + std::string file_name = "bloom_filter_string"; + Slice not_exist_value("string_test_not_exist"); + auto st = test_bloom_filter_index_reader_writer_template( + file_name, val, num, 1, ¬_exist_value, true); + EXPECT_TRUE(st.ok()); + delete[] val; +} + +TEST_F(BloomFilterIndexReaderWriterTest, test_unsigned_int) { + size_t num = 1024 * 3; + uint32_t* val = new uint32_t[num]; + for (size_t i = 0; i < num; ++i) { + val[i] = static_cast(i + 1); + } + + std::string file_name = "bloom_filter_unsigned_int"; + uint32_t not_exist_value = 0xFFFFFFFF; + auto st = + test_bloom_filter_index_reader_writer_template( + file_name, val, num, 1, ¬_exist_value); + EXPECT_TRUE(st.ok()); + delete[] val; +} + +TEST_F(BloomFilterIndexReaderWriterTest, test_smallint) { + size_t num = 1024 * 3; + int16_t* val = new int16_t[num]; + for (size_t i = 0; i < num; ++i) { + val[i] = static_cast(i + 1); + } + + std::string file_name = "bloom_filter_smallint"; + int16_t not_exist_value = -1; + auto st = test_bloom_filter_index_reader_writer_template( + file_name, val, num, 1, ¬_exist_value); + EXPECT_TRUE(st.ok()); + delete[] val; +} + TEST_F(BloomFilterIndexReaderWriterTest, test_bigint) { - size_t num = 1024 * 3 - 1; + size_t num = 1024 * 3; int64_t* val = new int64_t[num]; for (int i = 0; i < num; ++i) { // there will be 3 bloom filter pages @@ -203,7 +269,7 @@ TEST_F(BloomFilterIndexReaderWriterTest, test_bigint) { } TEST_F(BloomFilterIndexReaderWriterTest, test_largeint) { - size_t num = 1024 * 3 - 1; + size_t num = 1024 * 3; int128_t* val = new int128_t[num]; for (int i = 0; i < num; ++i) { // there will be 3 bloom filter pages @@ -219,7 +285,7 @@ TEST_F(BloomFilterIndexReaderWriterTest, test_largeint) { } TEST_F(BloomFilterIndexReaderWriterTest, test_varchar_type) { - size_t num = 1024 * 3 - 1; + size_t num = 1024 * 3; std::string* val = new std::string[num]; for (int i = 0; i < num; ++i) { // there will be 3 bloom filter pages @@ -240,7 +306,7 @@ TEST_F(BloomFilterIndexReaderWriterTest, test_varchar_type) { } TEST_F(BloomFilterIndexReaderWriterTest, test_char) { - size_t num = 1024 * 3 - 1; + size_t num = 1024 * 3; std::string* val = new std::string[num]; for (int i = 0; i < num; ++i) { // there will be 3 bloom filter pages @@ -261,7 +327,7 @@ TEST_F(BloomFilterIndexReaderWriterTest, test_char) { } TEST_F(BloomFilterIndexReaderWriterTest, test_date) { - size_t num = 1024 * 3 - 1; + size_t num = 1024 * 3; uint24_t* val = new uint24_t[num]; for (int i = 0; i < num; ++i) { // there will be 3 bloom filter pages @@ -277,7 +343,7 @@ TEST_F(BloomFilterIndexReaderWriterTest, test_date) { } TEST_F(BloomFilterIndexReaderWriterTest, test_datetime) { - size_t num = 1024 * 3 - 1; + size_t num = 1024 * 3; int64_t* val = new int64_t[num]; for (int i = 0; i < num; ++i) { // there will be 3 bloom filter pages @@ -293,7 +359,7 @@ TEST_F(BloomFilterIndexReaderWriterTest, test_datetime) { } TEST_F(BloomFilterIndexReaderWriterTest, test_decimal) { - size_t num = 1024 * 3 - 1; + size_t num = 1024 * 3; decimal12_t* val = new decimal12_t[num]; for (int i = 0; i < num; ++i) { // there will be 3 bloom filter pages @@ -308,8 +374,29 @@ TEST_F(BloomFilterIndexReaderWriterTest, test_decimal) { delete[] val; } +TEST_F(BloomFilterIndexReaderWriterTest, test_primary_key_bloom_filter_index_char) { + size_t num = 1024 * 3; + std::string* val = new std::string[num]; + for (int i = 0; i < num; ++i) { + // there will be 3 bloom filter pages + val[i] = "primary_key_" + std::to_string(10000 + i); + } + Slice* slices = new Slice[num]; + for (int i = 0; i < num; ++i) { + // there will be 3 bloom filter pages + slices[i] = Slice(val[i].c_str(), val[i].size()); + } + std::string file_name = "primary_key_bloom_filter_index_char"; + Slice not_exist_value("primary_key_not_exist_char"); + auto st = test_bloom_filter_index_reader_writer_template( + file_name, slices, num, 1, ¬_exist_value, true, true); + EXPECT_TRUE(st.ok()); + delete[] val; + delete[] slices; +} + TEST_F(BloomFilterIndexReaderWriterTest, test_primary_key_bloom_filter_index) { - size_t num = 1024 * 3 - 1; + size_t num = 1024 * 3; std::vector val_strings(num); for (size_t i = 0; i < num; ++i) { val_strings[i] = "primary_key_" + std::to_string(i); @@ -328,7 +415,7 @@ TEST_F(BloomFilterIndexReaderWriterTest, test_primary_key_bloom_filter_index) { } TEST_F(BloomFilterIndexReaderWriterTest, test_primary_key_bloom_filter_index_int) { - size_t num = 1024 * 3 - 1; + size_t num = 1024 * 3; int* val = new int[num]; for (int i = 0; i < num; ++i) { // there will be 3 bloom filter pages @@ -344,5 +431,377 @@ TEST_F(BloomFilterIndexReaderWriterTest, test_primary_key_bloom_filter_index_int delete[] val; } +TEST_F(BloomFilterIndexReaderWriterTest, test_datev2) { + size_t num = 1024 * 3; + uint32_t* val = new uint32_t[num]; + for (size_t i = 0; i < num; ++i) { + val[i] = 20210101 + i; // YYYYMMDD + } + + std::string file_name = "bloom_filter_datev2"; + uint32_t not_exist_value = 20211231; + auto st = test_bloom_filter_index_reader_writer_template( + file_name, val, num, 1, ¬_exist_value); + EXPECT_TRUE(st.ok()); + delete[] val; +} + +TEST_F(BloomFilterIndexReaderWriterTest, test_datetimev2) { + size_t num = 1024 * 3; + uint64_t* val = new uint64_t[num]; + for (size_t i = 0; i < num; ++i) { + val[i] = 20210101000000 + i; // YYYYMMDDHHMMSS + } + + std::string file_name = "bloom_filter_datetimev2"; + uint64_t not_exist_value = 20211231235959; + auto st = test_bloom_filter_index_reader_writer_template( + file_name, val, num, 1, ¬_exist_value); + EXPECT_TRUE(st.ok()); + delete[] val; +} + +TEST_F(BloomFilterIndexReaderWriterTest, test_decimal32) { + size_t num = 1024 * 3; + int32_t* val = new int32_t[num]; + for (size_t i = 0; i < num; ++i) { + val[i] = static_cast(i * 100 + 1); + } + + std::string file_name = "bloom_filter_decimal32"; + int32_t not_exist_value = 99999; + auto st = test_bloom_filter_index_reader_writer_template( + file_name, val, num, 1, ¬_exist_value); + EXPECT_TRUE(st.ok()); + delete[] val; +} + +TEST_F(BloomFilterIndexReaderWriterTest, test_decimal64) { + size_t num = 1024 * 3; + ; + int64_t* val = new int64_t[num]; + for (size_t i = 0; i < num; ++i) { + val[i] = static_cast(i * 1000 + 123); + } + + std::string file_name = "bloom_filter_decimal64"; + int64_t not_exist_value = 9999999; + auto st = test_bloom_filter_index_reader_writer_template( + file_name, val, num, 1, ¬_exist_value); + EXPECT_TRUE(st.ok()); + delete[] val; +} + +TEST_F(BloomFilterIndexReaderWriterTest, test_ipv4) { + size_t num = 1024 * 3; // 3072 + uint32_t* val = new uint32_t[num]; + for (size_t i = 0; i < num; ++i) { + val[i] = (192 << 24) | (168 << 16) | (i & 0xFFFF); + } + + std::string file_name = "bloom_filter_ipv4"; + uint32_t not_exist_value = (10 << 24) | (0 << 16) | (0 << 8) | 1; // 10.0.0.1 + auto st = test_bloom_filter_index_reader_writer_template( + file_name, val, num, 1, ¬_exist_value); + EXPECT_TRUE(st.ok()); + delete[] val; +} + +TEST_F(BloomFilterIndexReaderWriterTest, test_decimal128i) { + size_t num = 1024 * 3; + int128_t* val = new int128_t[num]; + + int128_t base_value = int128_t(1000000000ULL) * int128_t(1000000000ULL); + + for (size_t i = 0; i < num; ++i) { + val[i] = base_value + int128_t(i); + } + + std::string file_name = "bloom_filter_decimal128i"; + int128_t not_exist_value = int128_t(9999999999999999999ULL); + + auto st = + test_bloom_filter_index_reader_writer_template( + file_name, val, num, 1, ¬_exist_value); + EXPECT_TRUE(st.ok()); + delete[] val; +} + +TEST_F(BloomFilterIndexReaderWriterTest, test_decimal256) { + size_t num = 1024 * 3; + using Decimal256Type = wide::Int256; + + Decimal256Type* val = new Decimal256Type[num]; + + Decimal256Type base_value = Decimal256Type(1000000000ULL); // 1e9 + base_value *= Decimal256Type(1000000000ULL); // base_value = 1e18 + base_value *= Decimal256Type(100000000ULL); // base_value = 1e26 + base_value *= Decimal256Type(100000000ULL); // base_value = 1e34 + base_value *= Decimal256Type(10000ULL); // base_value = 1e38 + + for (size_t i = 0; i < num; ++i) { + val[i] = base_value + Decimal256Type(i); + } + + std::string file_name = "bloom_filter_decimal256"; + + Decimal256Type not_exist_value = base_value + Decimal256Type(9999999ULL); + + auto st = test_bloom_filter_index_reader_writer_template( + file_name, val, num, 1, ¬_exist_value); + EXPECT_TRUE(st.ok()); + delete[] val; +} + +TEST_F(BloomFilterIndexReaderWriterTest, test_ipv6) { + size_t num = 1024 * 3; + uint128_t* val = new uint128_t[num]; + for (size_t i = 0; i < num; ++i) { + val[i] = (uint128_t(0x20010DB800000000) << 64) | uint128_t(i); + } + + std::string file_name = "bloom_filter_ipv6"; + uint128_t not_exist_value = (uint128_t(0x20010DB800000000) << 64) | uint128_t(999999); + + auto st = test_bloom_filter_index_reader_writer_template( + file_name, val, num, 1, ¬_exist_value); + EXPECT_TRUE(st.ok()); + delete[] val; +} + +template +Status write_ngram_bloom_filter_index_file(const std::string& file_name, Slice* values, + size_t num_values, const TypeInfo* type_info, + BloomFilterIndexWriter* bf_index_writer, + ColumnIndexMetaPB* meta) { + auto fs = io::global_local_filesystem(); + std::string fname = dname + "/" + file_name; + io::FileWriterPtr file_writer; + Status st = fs->create_file(fname, &file_writer); + EXPECT_TRUE(st.ok()) << st.to_string(); + + size_t i = 0; + while (i < num_values) { + size_t num = std::min(static_cast(1024), num_values - i); + st = bf_index_writer->add_values(values + i, num); + EXPECT_TRUE(st.ok()); + st = bf_index_writer->flush(); + EXPECT_TRUE(st.ok()); + i += num; + } + bf_index_writer->add_nulls(1); + st = bf_index_writer->finish(file_writer.get(), meta); + EXPECT_TRUE(st.ok()) << "Writer finish status: " << st.to_string(); + EXPECT_TRUE(file_writer->close().ok()); + + return Status::OK(); +} + +Status read_and_test_ngram_bloom_filter_index_file(const std::string& file_name, size_t num_values, + uint8_t gram_size, uint16_t bf_size, + const ColumnIndexMetaPB& meta, + const std::vector& test_patterns) { + BloomFilterIndexReader* reader = nullptr; + std::unique_ptr iter; + get_bloom_filter_reader_iter(file_name, meta, &reader, &iter); + EXPECT_EQ(reader->algorithm(), BloomFilterAlgorithmPB::NGRAM_BLOOM_FILTER); + + NgramTokenExtractor extractor(gram_size); + uint16_t gram_bf_size = bf_size; + + size_t total_pages = (num_values + 1023) / 1024; + for (size_t page = 0; page < total_pages; ++page) { + std::unique_ptr bf; + auto st = iter->read_bloom_filter(page, &bf); + EXPECT_TRUE(st.ok()); + + for (const auto& pattern : test_patterns) { + std::unique_ptr query_bf; + st = BloomFilter::create(NGRAM_BLOOM_FILTER, &query_bf, gram_bf_size); + EXPECT_TRUE(st.ok()); + + if (extractor.string_like_to_bloom_filter(pattern.data(), pattern.size(), *query_bf)) { + bool contains = bf->contains(*query_bf); + bool expected = false; + if ((page == 0 && (pattern == "ngram15" || pattern == "ngram1000")) || + (page == 1 && pattern == "ngram1499")) { + expected = true; + } + EXPECT_EQ(contains, expected) << "Pattern: " << pattern << ", Page: " << page; + } + } + } + + delete reader; + return Status::OK(); +} + +template +Status test_ngram_bloom_filter_index_reader_writer(const std::string& file_name, Slice* values, + size_t num_values, uint8_t gram_size, + uint16_t bf_size) { + const auto* type_info = get_scalar_type_info(); + ColumnIndexMetaPB meta; + + BloomFilterOptions bf_options; + std::unique_ptr bf_index_writer; + RETURN_IF_ERROR(NGramBloomFilterIndexWriterImpl::create(bf_options, type_info, gram_size, + bf_size, &bf_index_writer)); + + RETURN_IF_ERROR(write_ngram_bloom_filter_index_file( + file_name, values, num_values, type_info, bf_index_writer.get(), &meta)); + + std::vector test_patterns = {"ngram15", "ngram1000", "ngram1499", + "non-existent-string"}; + + RETURN_IF_ERROR(read_and_test_ngram_bloom_filter_index_file(file_name, num_values, gram_size, + bf_size, meta, test_patterns)); + + return Status::OK(); +} + +TEST_F(BloomFilterIndexReaderWriterTest, test_ngram_bloom_filter) { + size_t num = 1500; + std::vector val(num); + for (size_t i = 0; i < num; ++i) { + val[i] = "ngram" + std::to_string(i); + } + std::vector slices(num); + for (size_t i = 0; i < num; ++i) { + slices[i] = Slice(val[i].data(), val[i].size()); + } + + uint8_t gram_size = 5; + uint16_t bf_size = 65535; + + auto st = test_ngram_bloom_filter_index_reader_writer( + "bloom_filter_ngram_varchar", slices.data(), num, gram_size, bf_size); + EXPECT_TRUE(st.ok()); + st = test_ngram_bloom_filter_index_reader_writer( + "bloom_filter_ngram_char", slices.data(), num, gram_size, bf_size); + EXPECT_TRUE(st.ok()); + st = test_ngram_bloom_filter_index_reader_writer( + "bloom_filter_ngram_string", slices.data(), num, gram_size, bf_size); + EXPECT_TRUE(st.ok()); + st = test_ngram_bloom_filter_index_reader_writer( + "bloom_filter_ngram_string", slices.data(), num, gram_size, bf_size); + EXPECT_FALSE(st.ok()); + EXPECT_EQ(st.code(), TStatusCode::NOT_IMPLEMENTED_ERROR); +} +void test_ngram_bloom_filter_with_size(uint16_t bf_size) { + const auto* type_info = get_scalar_type_info(); + ColumnIndexMetaPB meta; + + BloomFilterOptions bf_options; + size_t num = 1500; + std::vector val(num); + for (size_t i = 0; i < num; ++i) { + val[i] = "ngram" + std::to_string(i); + } + std::vector slices(num); + for (size_t i = 0; i < num; ++i) { + slices[i] = Slice(val[i].data(), val[i].size()); + } + size_t total_pages = (num + 1024 - 1) / 1024; + uint8_t gram_size = 5; + + std::unique_ptr bf_index_writer; + auto st = NGramBloomFilterIndexWriterImpl::create(bf_options, type_info, gram_size, bf_size, + &bf_index_writer); + EXPECT_TRUE(st.ok()); + + std::string file_name = "bloom_filter_ngram_varchar_size_" + std::to_string(bf_size); + st = write_ngram_bloom_filter_index_file( + file_name, slices.data(), num, type_info, bf_index_writer.get(), &meta); + EXPECT_TRUE(st.ok()); + EXPECT_EQ(bf_index_writer->size(), static_cast(bf_size) * total_pages); +} + +TEST_F(BloomFilterIndexReaderWriterTest, test_ngram_bloom_filter_size) { + std::vector bf_sizes = {1024, 2048, 4096, 8192, 16384, 32768, 65535}; + for (uint16_t bf_size : bf_sizes) { + test_ngram_bloom_filter_with_size(bf_size); + } +} + +TEST_F(BloomFilterIndexReaderWriterTest, test_unsupported_type) { + auto type_info = get_scalar_type_info(); + BloomFilterOptions bf_options; + std::unique_ptr bloom_filter_index_writer; + auto st = BloomFilterIndexWriter::create(bf_options, type_info, &bloom_filter_index_writer); + EXPECT_FALSE(st.ok()); + EXPECT_EQ(st.code(), TStatusCode::NOT_IMPLEMENTED_ERROR); +} + +// Test function for verifying Bloom Filter FPP +void test_bloom_filter_fpp(double expected_fpp) { + size_t n = 10000; // Number of elements to insert into the Bloom Filter + size_t m = 100000; // Number of non-existent elements to test for false positives + + // Generate and insert elements into the Bloom Filter index + std::vector insert_values; + for (size_t i = 0; i < n; ++i) { + int64_t val = static_cast(i); + insert_values.push_back(val); + } + + // Write the Bloom Filter index to file + std::string file_name = "bloom_filter_fpp_test"; + ColumnIndexMetaPB index_meta; + Status st = write_bloom_filter_index_file( + file_name, insert_values.data(), n, 0, &index_meta, false, expected_fpp); + EXPECT_TRUE(st.ok()); + + // Read the Bloom Filter index + BloomFilterIndexReader* reader = nullptr; + std::unique_ptr iter; + get_bloom_filter_reader_iter(file_name, index_meta, &reader, &iter); + + // Read the Bloom Filter (only one page since we flushed once) + std::unique_ptr bf; + st = iter->read_bloom_filter(0, &bf); + EXPECT_TRUE(st.ok()); + + // Generate non-existent elements for testing false positive rate + std::unordered_set inserted_elements(insert_values.begin(), insert_values.end()); + std::unordered_set non_exist_elements; + std::vector test_values; + size_t max_value = n + m * 10; // Ensure test values are not in the inserted range + boost::mt19937_64 rng(12345); // Seed the random number generator for reproducibility + std::uniform_int_distribution dist(static_cast(n + 1), + static_cast(max_value)); + while (non_exist_elements.size() < m) { + int64_t val = dist(rng); + if (inserted_elements.find(val) == inserted_elements.end()) { + non_exist_elements.insert(val); + test_values.push_back(val); + } + } + + // Test non-existent elements and count false positives + size_t fp_count = 0; + for (const auto& val : test_values) { + if (bf->test_bytes(reinterpret_cast(&val), sizeof(int64_t))) { + fp_count++; + } + } + + // Compute actual false positive probability + double actual_fpp = static_cast(fp_count) / static_cast(m); + std::cout << "Expected FPP: " << expected_fpp << ", Actual FPP: " << actual_fpp << std::endl; + + // Verify that actual FPP is within the allowable error range + EXPECT_LE(actual_fpp, expected_fpp); + + delete reader; +} + +// Test case to run FPP tests with multiple expected FPP values +TEST_F(BloomFilterIndexReaderWriterTest, test_bloom_filter_fpp_multiple) { + std::vector fpp_values = {0.01, 0.02, 0.05}; + for (double fpp : fpp_values) { + test_bloom_filter_fpp(fpp); + } +} } // namespace segment_v2 } // namespace doris From 334b3435bc998e5e2405d8fe7b25ba16d0267f03 Mon Sep 17 00:00:00 2001 From: Vallish Pai Date: Mon, 2 Dec 2024 13:05:44 +0530 Subject: [PATCH 111/399] [Enhancement] (nereids)implement CreateEncryptKeyCommand in nereids (#44807) Issue Number: close #42591 --- .../org/apache/doris/nereids/DorisParser.g4 | 2 +- .../doris/catalog/EncryptKeyHelper.java | 10 ++- .../nereids/parser/LogicalPlanBuilder.java | 9 +++ .../doris/nereids/trees/plans/PlanType.java | 1 + .../commands/CreateEncryptkeyCommand.java | 73 +++++++++++++++++++ .../trees/plans/visitor/CommandVisitor.java | 5 ++ .../test_nereids_encrypt_test.groovy | 2 +- 7 files changed, 98 insertions(+), 4 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateEncryptkeyCommand.java diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index a372a993ac6f22..5200f714a090ce 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -188,6 +188,7 @@ supportedCreateStatement USING LEFT_PAREN booleanExpression RIGHT_PAREN #createRowPolicy | CREATE SQL_BLOCK_RULE (IF NOT EXISTS)? name=identifier properties=propertyClause? #createSqlBlockRule + | CREATE ENCRYPTKEY (IF NOT EXISTS)? multipartIdentifier AS STRING_LITERAL #createEncryptkey ; supportedAlterStatement @@ -767,7 +768,6 @@ unsupportedCreateStatement (CONDITIONS LEFT_PAREN workloadPolicyConditions RIGHT_PAREN)? (ACTIONS LEFT_PAREN workloadPolicyActions RIGHT_PAREN)? properties=propertyClause? #createWorkloadPolicy - | CREATE ENCRYPTKEY (IF NOT EXISTS)? multipartIdentifier AS STRING_LITERAL #createEncryptkey | CREATE STORAGE POLICY (IF NOT EXISTS)? name=identifier properties=propertyClause? #createStoragePolicy | BUILD INDEX name=identifier ON tableName=multipartIdentifier diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/EncryptKeyHelper.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/EncryptKeyHelper.java index 2e30f0ba86f326..934dd22f3a5b89 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/EncryptKeyHelper.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/EncryptKeyHelper.java @@ -32,10 +32,16 @@ public class EncryptKeyHelper { public static void createEncryptKey(CreateEncryptKeyStmt stmt) throws UserException { EncryptKeyName name = stmt.getEncryptKeyName(); - Database db = Env.getCurrentInternalCatalog().getDbOrDdlException(name.getDb()); - db.addEncryptKey(stmt.getEncryptKey(), stmt.isIfNotExists()); + createEncryptKey(name.getDb(), stmt.getEncryptKey(), stmt.isIfNotExists()); + } + + public static void createEncryptKey(String dbName, EncryptKey encryptKey, + boolean isIfNotExists) throws UserException { + Database db = Env.getCurrentInternalCatalog().getDbOrDdlException(dbName); + db.addEncryptKey(encryptKey, isIfNotExists); } + public static void replayCreateEncryptKey(EncryptKey encryptKey) throws MetaNotFoundException { String dbName = encryptKey.getEncryptKeyName().getDb(); Database db = Env.getCurrentInternalCatalog().getDbOrMetaException(dbName); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index dfe879e7d44dad..f94284145d5320 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -89,6 +89,7 @@ import org.apache.doris.nereids.DorisParser.ComplexColTypeListContext; import org.apache.doris.nereids.DorisParser.ComplexDataTypeContext; import org.apache.doris.nereids.DorisParser.ConstantContext; +import org.apache.doris.nereids.DorisParser.CreateEncryptkeyContext; import org.apache.doris.nereids.DorisParser.CreateMTMVContext; import org.apache.doris.nereids.DorisParser.CreateProcedureContext; import org.apache.doris.nereids.DorisParser.CreateRoutineLoadContext; @@ -476,6 +477,7 @@ import org.apache.doris.nereids.trees.plans.commands.CleanAllProfileCommand; import org.apache.doris.nereids.trees.plans.commands.Command; import org.apache.doris.nereids.trees.plans.commands.Constraint; +import org.apache.doris.nereids.trees.plans.commands.CreateEncryptkeyCommand; import org.apache.doris.nereids.trees.plans.commands.CreateJobCommand; import org.apache.doris.nereids.trees.plans.commands.CreateMTMVCommand; import org.apache.doris.nereids.trees.plans.commands.CreatePolicyCommand; @@ -4749,6 +4751,13 @@ public LogicalPlan visitDropRole(DropRoleContext ctx) { return new DropRoleCommand(ctx.name.getText(), ctx.EXISTS() != null); } + @Override + public LogicalPlan visitCreateEncryptkey(CreateEncryptkeyContext ctx) { + List nameParts = visitMultipartIdentifier(ctx.multipartIdentifier()); + return new CreateEncryptkeyCommand(new EncryptKeyName(nameParts), ctx.EXISTS() != null, + stripQuotes(ctx.STRING_LITERAL().getText())); + } + @Override public LogicalPlan visitDropEncryptkey(DropEncryptkeyContext ctx) { List nameParts = visitMultipartIdentifier(ctx.name); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index 14a1b13a4667df..d7331d5a65d738 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -237,5 +237,6 @@ public enum PlanType { RECOVER_TABLE_COMMAND, RECOVER_PARTITION_COMMAND, REPLAY_COMMAND, + CREATE_ENCRYPTKEY_COMMAND, CREATE_ROUTINE_LOAD_COMMAND } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateEncryptkeyCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateEncryptkeyCommand.java new file mode 100644 index 00000000000000..f8fc7c3cc12c5e --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateEncryptkeyCommand.java @@ -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. + +package org.apache.doris.nereids.trees.plans.commands; + +import org.apache.doris.analysis.EncryptKeyName; +import org.apache.doris.analysis.StmtType; +import org.apache.doris.catalog.EncryptKey; +import org.apache.doris.catalog.EncryptKeyHelper; +import org.apache.doris.catalog.Env; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.StmtExecutor; + +import com.google.common.base.Strings; + +/** CreateEncryptkeyCommand */ +public class CreateEncryptkeyCommand extends Command implements ForwardWithSync { + private final boolean ifNotExists; + private final EncryptKeyName encryptKeyName; + private final String keyString; + + public CreateEncryptkeyCommand(EncryptKeyName encryptKeyName, boolean ifNotExists, String keyString) { + super(PlanType.CREATE_ENCRYPTKEY_COMMAND); + this.ifNotExists = ifNotExists; + this.encryptKeyName = encryptKeyName; + this.keyString = keyString; + } + + @Override + public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { + // check operation privilege + if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, "ADMIN"); + } + + encryptKeyName.analyze(ctx); + if (Strings.isNullOrEmpty(keyString)) { + throw new AnalysisException("keyString can not be null or empty string."); + } + EncryptKeyHelper.createEncryptKey(encryptKeyName.getDb(), + new EncryptKey(encryptKeyName, keyString), ifNotExists); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitCreateEncryptKeyCommand(this, context); + } + + @Override + public StmtType stmtType() { + return StmtType.CREATE; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index d386d097a93b49..033290d37132e8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -33,6 +33,7 @@ import org.apache.doris.nereids.trees.plans.commands.CancelWarmUpJobCommand; import org.apache.doris.nereids.trees.plans.commands.CleanAllProfileCommand; import org.apache.doris.nereids.trees.plans.commands.Command; +import org.apache.doris.nereids.trees.plans.commands.CreateEncryptkeyCommand; import org.apache.doris.nereids.trees.plans.commands.CreateJobCommand; import org.apache.doris.nereids.trees.plans.commands.CreateMTMVCommand; import org.apache.doris.nereids.trees.plans.commands.CreatePolicyCommand; @@ -175,6 +176,10 @@ default R visitExportCommand(ExportCommand exportCommand, C context) { return visitCommand(exportCommand, context); } + default R visitCreateEncryptKeyCommand(CreateEncryptkeyCommand createEncryptKeyCommand, C context) { + return visitCommand(createEncryptKeyCommand, context); + } + default R visitCreateTableCommand(CreateTableCommand createTableCommand, C context) { return visitCommand(createTableCommand, context); } diff --git a/regression-test/suites/nereids_p0/test_nereids_encrypt_test.groovy b/regression-test/suites/nereids_p0/test_nereids_encrypt_test.groovy index 2fab616580faaa..8954090bb4c31d 100644 --- a/regression-test/suites/nereids_p0/test_nereids_encrypt_test.groovy +++ b/regression-test/suites/nereids_p0/test_nereids_encrypt_test.groovy @@ -20,7 +20,7 @@ suite("test_nereids_encrypt_test") { sql """ create database IF NOT EXISTS ${dbName}; """ sql """ use ${dbName}; """ checkNereidsExecute("drop encryptkey if exists ${encryptkeyName}") - sql """CREATE ENCRYPTKEY ${encryptkeyName} AS "ABCD123456789";""" + checkNereidsExecute("""CREATE ENCRYPTKEY ${encryptkeyName} AS "ABCD123456789";""") qt_check_encrypt_1("SHOW ENCRYPTKEYS FROM ${dbName}") checkNereidsExecute("drop encryptkey ${encryptkeyName}") qt_check_encrypt_2("SHOW ENCRYPTKEYS FROM ${dbName}") From 54fe0b463a9dd29a562539bdc351929aebb0b064 Mon Sep 17 00:00:00 2001 From: Vallish Pai Date: Mon, 2 Dec 2024 13:06:03 +0530 Subject: [PATCH 112/399] [Enhancement] (nereids)implement CreateRoleCommand in nereids (#44814) Issue Number: close #42597 --- .../org/apache/doris/nereids/DorisParser.g4 | 2 +- .../apache/doris/mysql/privilege/Auth.java | 4 + .../nereids/parser/LogicalPlanBuilder.java | 9 +++ .../doris/nereids/trees/plans/PlanType.java | 1 + .../plans/commands/CreateRoleCommand.java | 75 +++++++++++++++++++ .../trees/plans/commands/DropRoleCommand.java | 4 - .../trees/plans/visitor/CommandVisitor.java | 5 ++ .../ddl/account/test_nereids_role.groovy | 4 +- .../ddl/alter/test_nereids_role.groovy | 66 ---------------- 9 files changed, 97 insertions(+), 73 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateRoleCommand.java delete mode 100644 regression-test/suites/nereids_p0/ddl/alter/test_nereids_role.groovy diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index 5200f714a090ce..3a377ba05bc707 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -181,6 +181,7 @@ supportedCreateStatement | CREATE (EXTERNAL)? TABLE (IF NOT EXISTS)? name=multipartIdentifier LIKE existedTable=multipartIdentifier (WITH ROLLUP (rollupNames=identifierList)?)? #createTableLike + | CREATE ROLE (IF NOT EXISTS)? name=identifier (COMMENT STRING_LITERAL)? #createRole | CREATE ROW POLICY (IF NOT EXISTS)? name=identifier ON table=multipartIdentifier AS type=(RESTRICTIVE | PERMISSIVE) @@ -751,7 +752,6 @@ unsupportedCreateStatement (SUPERUSER | DEFAULT ROLE role=STRING_LITERAL)? passwordOption (COMMENT STRING_LITERAL)? #createUser | CREATE (READ ONLY)? REPOSITORY name=identifier WITH storageBackend #createRepository - | CREATE ROLE (IF NOT EXISTS)? name=identifier (COMMENT STRING_LITERAL)? #createRole | CREATE FILE name=STRING_LITERAL ((FROM | IN) database=identifier)? properties=propertyClause #createFile | CREATE INDEX (IF NOT EXISTS)? name=identifier diff --git a/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/Auth.java b/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/Auth.java index f1277a1cd0a3b5..8a13c6dd125548 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/Auth.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/Auth.java @@ -1019,6 +1019,10 @@ public void createRole(CreateRoleStmt stmt) throws DdlException { createRoleInternal(stmt.getRole(), stmt.isSetIfNotExists(), stmt.getComment(), false); } + public void createRole(String role, boolean ignoreIfExists, String comment) throws DdlException { + createRoleInternal(role, ignoreIfExists, comment, false); + } + public void alterRole(AlterRoleStmt stmt) throws DdlException { alterRoleInternal(stmt.getRole(), stmt.getComment(), false); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index f94284145d5320..2d55baffb1ebd9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -92,6 +92,7 @@ import org.apache.doris.nereids.DorisParser.CreateEncryptkeyContext; import org.apache.doris.nereids.DorisParser.CreateMTMVContext; import org.apache.doris.nereids.DorisParser.CreateProcedureContext; +import org.apache.doris.nereids.DorisParser.CreateRoleContext; import org.apache.doris.nereids.DorisParser.CreateRoutineLoadContext; import org.apache.doris.nereids.DorisParser.CreateRowPolicyContext; import org.apache.doris.nereids.DorisParser.CreateSqlBlockRuleContext; @@ -482,6 +483,7 @@ import org.apache.doris.nereids.trees.plans.commands.CreateMTMVCommand; import org.apache.doris.nereids.trees.plans.commands.CreatePolicyCommand; import org.apache.doris.nereids.trees.plans.commands.CreateProcedureCommand; +import org.apache.doris.nereids.trees.plans.commands.CreateRoleCommand; import org.apache.doris.nereids.trees.plans.commands.CreateSqlBlockRuleCommand; import org.apache.doris.nereids.trees.plans.commands.CreateTableCommand; import org.apache.doris.nereids.trees.plans.commands.CreateTableLikeCommand; @@ -4671,6 +4673,13 @@ public LogicalPlan visitAlterRole(AlterRoleContext ctx) { return new AlterRoleCommand(ctx.role.getText(), comment); } + @Override + public LogicalPlan visitCreateRole(CreateRoleContext ctx) { + String comment = ctx.STRING_LITERAL() == null ? "" : LogicalPlanBuilderAssistant.escapeBackSlash( + ctx.STRING_LITERAL().getText().substring(1, ctx.STRING_LITERAL().getText().length() - 1)); + return new CreateRoleCommand(ctx.EXISTS() != null, ctx.name.getText(), comment); + } + @Override public LogicalPlan visitShowFrontends(ShowFrontendsContext ctx) { String detail = (ctx.name != null) ? ctx.name.getText() : null; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index d7331d5a65d738..0cc0e5594075b4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -171,6 +171,7 @@ public enum PlanType { SHOW_CREATE_PROCEDURE_COMMAND, CREATE_VIEW_COMMAND, CLEAN_ALL_PROFILE_COMMAND, + CREATE_ROLE_COMMAND, ALTER_ROLE_COMMAND, ALTER_VIEW_COMMAND, ALTER_STORAGE_VAULT, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateRoleCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateRoleCommand.java new file mode 100644 index 00000000000000..1596993ba669f2 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateRoleCommand.java @@ -0,0 +1,75 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.commands; + +import org.apache.doris.analysis.StmtType; +import org.apache.doris.catalog.Env; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.Config; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.common.FeNameFormat; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.StmtExecutor; + +import com.google.common.base.Strings; + +/** + * Create role command + */ +public class CreateRoleCommand extends Command implements ForwardWithSync { + private boolean ifNotExists; + private String role; + private String comment; + + /** + * ctor of this command. + */ + public CreateRoleCommand(boolean ifNotExists, String role, String comment) { + super(PlanType.CREATE_ROLE_COMMAND); + this.ifNotExists = ifNotExists; + this.role = role; + this.comment = Strings.nullToEmpty(comment); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitCreateRoleCommand(this, context); + } + + @Override + public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { + if (Config.access_controller_type.equalsIgnoreCase("ranger-doris")) { + throw new AnalysisException("Create role is prohibited when Ranger is enabled."); + } + FeNameFormat.checkRoleName(role, false /* can not be admin */, "Can not create role"); + // check if current user has GRANT priv on GLOBAL level. + if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), PrivPredicate.GRANT)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, "CREATE ROLE"); + } + Env.getCurrentEnv().getAuth().createRole(role, ifNotExists, comment); + } + + @Override + public StmtType stmtType() { + return StmtType.CREATE; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropRoleCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropRoleCommand.java index 2f7858aa6ae180..5b74d32cf6a4df 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropRoleCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropRoleCommand.java @@ -29,14 +29,10 @@ import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.StmtExecutor; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; - /** * drop roles command */ public class DropRoleCommand extends DropCommand { - public static final Logger LOG = LogManager.getLogger(DropRoleCommand.class); private final boolean ifExists; private final String role; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index 033290d37132e8..d1d424738f4924 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -38,6 +38,7 @@ import org.apache.doris.nereids.trees.plans.commands.CreateMTMVCommand; import org.apache.doris.nereids.trees.plans.commands.CreatePolicyCommand; import org.apache.doris.nereids.trees.plans.commands.CreateProcedureCommand; +import org.apache.doris.nereids.trees.plans.commands.CreateRoleCommand; import org.apache.doris.nereids.trees.plans.commands.CreateSqlBlockRuleCommand; import org.apache.doris.nereids.trees.plans.commands.CreateTableCommand; import org.apache.doris.nereids.trees.plans.commands.CreateTableLikeCommand; @@ -486,6 +487,10 @@ default R visitCreateSqlBlockRuleCommand(CreateSqlBlockRuleCommand dropRoleComma return visitCommand(dropRoleCommand, context); } + default R visitCreateRoleCommand(CreateRoleCommand createRoleCommand, C context) { + return visitCommand(createRoleCommand, context); + } + default R visitDropRoleCommand(DropRoleCommand dropRoleCommand, C context) { return visitCommand(dropRoleCommand, context); } diff --git a/regression-test/suites/nereids_p0/ddl/account/test_nereids_role.groovy b/regression-test/suites/nereids_p0/ddl/account/test_nereids_role.groovy index 4e2cd4ca6375d2..a3bee25cf5822a 100644 --- a/regression-test/suites/nereids_p0/ddl/account/test_nereids_role.groovy +++ b/regression-test/suites/nereids_p0/ddl/account/test_nereids_role.groovy @@ -19,7 +19,7 @@ import org.junit.Assert; suite("test_nereids_role") { def role= 'nereids_account_role_test' - def user = 'acount_role_user_test' + def user = 'nereids_acount_role_user_test' def dbName = 'nereids_account_role_test_db' def pwd = 'C123_567p' @@ -28,7 +28,7 @@ suite("test_nereids_role") { sql """DROP DATABASE IF EXISTS ${dbName}""" sql """CREATE DATABASE ${dbName}""" - sql """CREATE ROLE ${role}""" + checkNereidsExecute("CREATE ROLE ${role}") sql """GRANT SELECT_PRIV ON ${context.config.defaultDb} TO ROLE '${role}'""" sql """GRANT SELECT_PRIV ON ${dbName} TO ROLE '${role}'""" sql """CREATE USER '${user}' IDENTIFIED BY '${pwd}' DEFAULT ROLE '${role}'""" diff --git a/regression-test/suites/nereids_p0/ddl/alter/test_nereids_role.groovy b/regression-test/suites/nereids_p0/ddl/alter/test_nereids_role.groovy deleted file mode 100644 index f23982188269b7..00000000000000 --- a/regression-test/suites/nereids_p0/ddl/alter/test_nereids_role.groovy +++ /dev/null @@ -1,66 +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. - -import org.junit.Assert; - -suite("test_nereids_role_x", "account") { - def role= 'nereids_account_role_test_x' - def user = 'nereids_acount_role_user_test_x' - def dbName = 'nereids_account_role_test_db_x' - def pwd = 'C123_567p' - - try_sql("DROP ROLE ${role}") - try_sql("DROP USER ${user}") - sql """DROP DATABASE IF EXISTS ${dbName}""" - sql """CREATE DATABASE IF NOT EXISTS ${dbName}""" - sql """CREATE ROLE ${role}""" - sql """GRANT SELECT_PRIV ON internal.${context.config.defaultDb}.* TO ROLE '${role}'""" - sql """GRANT SELECT_PRIV ON internal.${dbName}.* TO ROLE '${role}'""" - sql """CREATE USER '${user}' IDENTIFIED BY '${pwd}' DEFAULT ROLE '${role}'""" - def result1 = connect(user=user, password="${pwd}", url=context.config.jdbcUrl) { - sql "show databases like '${dbName}'" - } - assertEquals(result1.size(), 1) - - sql """REVOKE SELECT_PRIV ON ${dbName} FROM ROLE '${role}'""" - def result2 = connect(user=user, password="${pwd}", url=context.config.jdbcUrl) { - sql "show databases like '${dbName}'" - } - assertEquals(result2.size(), 0) - - sql """DROP USER ${user}""" - sql """DROP ROLE ${role}""" - sql """DROP DATABASE ${dbName}""" - - // test comment - // create role with comment - sql """CREATE ROLE ${role} comment 'account_p0_account_role_test_comment_create'""" - def roles_create = sql """show roles""" - logger.info("roles_create: " + roles_create.toString()) - assertTrue(roles_create.toString().contains("account_p0_account_role_test_comment_create")) - // alter role with comment - checkNereidsExecute("ALTER ROLE ${role} comment 'account_p0_account_role_test_comment_alter';"); - def roles_alter = sql """show roles""" - logger.info("roles_alter: " + roles_alter.toString()) - assertTrue(roles_alter.toString().contains("account_p0_account_role_test_comment_alter")) - // drop role - checkNereidsExecute("""DROP ROLE ${role}""") - def roles_drop = sql """show roles""" - logger.info("roles_drop: " + roles_drop.toString()) - assertFalse(roles_drop.toString().contains("account_p0_account_role_test_comment_alter")) -} - From 53ea2a0a1d18900a3ad5bca8d1b1d5fe1a98111c Mon Sep 17 00:00:00 2001 From: 924060929 Date: Mon, 2 Dec 2024 16:40:38 +0800 Subject: [PATCH 113/399] [fix](coordinator) fix NereidsCoordinator can not interrupt query in fe (#44795) fix NereidsCoordinator can not interrupt query in fe, the QueryProcessor should set the status to failed --- .../apache/doris/qe/AbstractJobProcessor.java | 118 ++++++++++++++++++ .../org/apache/doris/qe/JobProcessor.java | 5 +- .../apache/doris/qe/NereidsCoordinator.java | 5 +- .../doris/qe/runtime/LoadProcessor.java | 74 ++--------- .../qe/runtime/PipelineExecutionTask.java | 2 +- .../runtime/PipelineExecutionTaskBuilder.java | 2 +- .../doris/qe/runtime/QueryProcessor.java | 18 ++- 7 files changed, 142 insertions(+), 82 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/qe/AbstractJobProcessor.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/AbstractJobProcessor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/AbstractJobProcessor.java new file mode 100644 index 00000000000000..2858de25d578fc --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/AbstractJobProcessor.java @@ -0,0 +1,118 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.qe; + +import org.apache.doris.common.Status; +import org.apache.doris.common.util.DebugUtil; +import org.apache.doris.qe.runtime.BackendFragmentId; +import org.apache.doris.qe.runtime.MultiFragmentsPipelineTask; +import org.apache.doris.qe.runtime.PipelineExecutionTask; +import org.apache.doris.qe.runtime.SingleFragmentPipelineTask; +import org.apache.doris.thrift.TReportExecStatusParams; +import org.apache.doris.thrift.TUniqueId; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableMap; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.Map; +import java.util.Map.Entry; +import java.util.Objects; +import java.util.Optional; + +/** AbstractJobProcessor */ +public abstract class AbstractJobProcessor implements JobProcessor { + private final Logger logger = LogManager.getLogger(getClass()); + + protected final CoordinatorContext coordinatorContext; + protected volatile Optional executionTask; + protected volatile Optional> backendFragmentTasks; + + public AbstractJobProcessor(CoordinatorContext coordinatorContext) { + this.coordinatorContext = Objects.requireNonNull(coordinatorContext, "coordinatorContext can not be null"); + this.executionTask = Optional.empty(); + this.backendFragmentTasks = Optional.empty(); + } + + protected abstract void doProcessReportExecStatus( + TReportExecStatusParams params, SingleFragmentPipelineTask fragmentTask); + + @Override + public final void setPipelineExecutionTask(PipelineExecutionTask pipelineExecutionTask) { + Preconditions.checkArgument(pipelineExecutionTask != null, "sqlPipelineTask can not be null"); + + this.executionTask = Optional.of(pipelineExecutionTask); + Map backendFragmentTasks + = buildBackendFragmentTasks(pipelineExecutionTask); + this.backendFragmentTasks = Optional.of(backendFragmentTasks); + + afterSetPipelineExecutionTask(pipelineExecutionTask); + } + + protected void afterSetPipelineExecutionTask(PipelineExecutionTask pipelineExecutionTask) {} + + @Override + public final void updateFragmentExecStatus(TReportExecStatusParams params) { + SingleFragmentPipelineTask fragmentTask = backendFragmentTasks.get().get( + new BackendFragmentId(params.getBackendId(), params.getFragmentId())); + if (fragmentTask == null || !fragmentTask.processReportExecStatus(params)) { + return; + } + + TUniqueId queryId = coordinatorContext.queryId; + Status status = new Status(params.status); + // for now, abort the query if we see any error except if the error is cancelled + // and returned_all_results_ is true. + // (UpdateStatus() initiates cancellation, if it hasn't already been initiated) + if (!status.ok()) { + if (coordinatorContext.isEos() && status.isCancelled()) { + logger.warn("Query {} has returned all results, fragment_id={} instance_id={}, be={}" + + " is reporting failed status {}", + DebugUtil.printId(queryId), params.getFragmentId(), + DebugUtil.printId(params.getFragmentInstanceId()), + params.getBackendId(), + status.toString()); + } else { + logger.warn("one instance report fail, query_id={} fragment_id={} instance_id={}, be={}," + + " error message: {}", + DebugUtil.printId(queryId), params.getFragmentId(), + DebugUtil.printId(params.getFragmentInstanceId()), + params.getBackendId(), status.toString()); + coordinatorContext.updateStatusIfOk(status); + } + } + doProcessReportExecStatus(params, fragmentTask); + } + + private Map buildBackendFragmentTasks( + PipelineExecutionTask executionTask) { + ImmutableMap.Builder backendFragmentTasks + = ImmutableMap.builder(); + for (Entry backendTask : executionTask.getChildrenTasks().entrySet()) { + Long backendId = backendTask.getKey(); + for (Entry fragmentIdToTask : backendTask.getValue() + .getChildrenTasks().entrySet()) { + Integer fragmentId = fragmentIdToTask.getKey(); + SingleFragmentPipelineTask fragmentTask = fragmentIdToTask.getValue(); + backendFragmentTasks.put(new BackendFragmentId(backendId, fragmentId), fragmentTask); + } + } + return backendFragmentTasks.build(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/JobProcessor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/JobProcessor.java index ede218848c7221..7e4042dde3c3bd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/JobProcessor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/JobProcessor.java @@ -19,9 +19,12 @@ import org.apache.doris.common.Status; import org.apache.doris.qe.runtime.PipelineExecutionTask; +import org.apache.doris.thrift.TReportExecStatusParams; public interface JobProcessor { - void setSqlPipelineTask(PipelineExecutionTask pipelineExecutionTask); + void setPipelineExecutionTask(PipelineExecutionTask pipelineExecutionTask); void cancel(Status cancelReason); + + void updateFragmentExecStatus(TReportExecStatusParams params); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/NereidsCoordinator.java b/fe/fe-core/src/main/java/org/apache/doris/qe/NereidsCoordinator.java index a9d6becc7fafe3..a6f24806ed74aa 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/NereidsCoordinator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/NereidsCoordinator.java @@ -233,10 +233,7 @@ public boolean isDone() { @Override public void updateFragmentExecStatus(TReportExecStatusParams params) { - JobProcessor jobProcessor = coordinatorContext.getJobProcessor(); - if (jobProcessor instanceof LoadProcessor) { - coordinatorContext.asLoadProcessor().updateFragmentExecStatus(params); - } + coordinatorContext.getJobProcessor().updateFragmentExecStatus(params); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/LoadProcessor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/LoadProcessor.java index 3a448521fca0bf..fb32919d834a54 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/LoadProcessor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/LoadProcessor.java @@ -24,46 +24,39 @@ import org.apache.doris.datasource.hive.HMSTransaction; import org.apache.doris.datasource.iceberg.IcebergTransaction; import org.apache.doris.nereids.util.Utils; +import org.apache.doris.qe.AbstractJobProcessor; import org.apache.doris.qe.CoordinatorContext; -import org.apache.doris.qe.JobProcessor; import org.apache.doris.qe.LoadContext; import org.apache.doris.thrift.TFragmentInstanceReport; import org.apache.doris.thrift.TReportExecStatusParams; import org.apache.doris.thrift.TStatusCode; import org.apache.doris.thrift.TUniqueId; -import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import java.util.List; import java.util.Map; -import java.util.Map.Entry; -import java.util.Objects; import java.util.Optional; import java.util.concurrent.TimeUnit; -public class LoadProcessor implements JobProcessor { +public class LoadProcessor extends AbstractJobProcessor { private static final Logger LOG = LogManager.getLogger(LoadProcessor.class); - public final CoordinatorContext coordinatorContext; public final LoadContext loadContext; public final long jobId; // this latch is used to wait finish for load, for example, insert into statement // MarkedCountDownLatch: // key: fragmentId, value: backendId - private volatile Optional executionTask; private volatile Optional> latch; - private volatile Optional> backendFragmentTasks; private volatile List topFragmentTasks; public LoadProcessor(CoordinatorContext coordinatorContext, long jobId) { - this.coordinatorContext = Objects.requireNonNull(coordinatorContext, "coordinatorContext can not be null"); + super(coordinatorContext); + this.loadContext = new LoadContext(); - this.executionTask = Optional.empty(); this.latch = Optional.empty(); this.backendFragmentTasks = Optional.empty(); @@ -87,14 +80,8 @@ public LoadProcessor(CoordinatorContext coordinatorContext, long jobId) { } @Override - public void setSqlPipelineTask(PipelineExecutionTask pipelineExecutionTask) { - Preconditions.checkArgument(pipelineExecutionTask != null, "sqlPipelineTask can not be null"); - - this.executionTask = Optional.of(pipelineExecutionTask); - Map backendFragmentTasks - = buildBackendFragmentTasks(pipelineExecutionTask); - this.backendFragmentTasks = Optional.of(backendFragmentTasks); - + protected void afterSetPipelineExecutionTask(PipelineExecutionTask pipelineExecutionTask) { + Map backendFragmentTasks = this.backendFragmentTasks.get(); MarkedCountDownLatch latch = new MarkedCountDownLatch<>(backendFragmentTasks.size()); for (BackendFragmentId backendFragmentId : backendFragmentTasks.keySet()) { latch.addMark(backendFragmentId.fragmentId, backendFragmentId.backendId); @@ -168,34 +155,9 @@ public boolean await(long timeout, TimeUnit unit) throws InterruptedException { return latch.get().await(timeout, unit); } - public void updateFragmentExecStatus(TReportExecStatusParams params) { - SingleFragmentPipelineTask fragmentTask = backendFragmentTasks.get().get( - new BackendFragmentId(params.getBackendId(), params.getFragmentId())); - if (fragmentTask == null || !fragmentTask.processReportExecStatus(params)) { - return; - } - TUniqueId queryId = coordinatorContext.queryId; - Status status = new Status(params.status); - // for now, abort the query if we see any error except if the error is cancelled - // and returned_all_results_ is true. - // (UpdateStatus() initiates cancellation, if it hasn't already been initiated) - if (!status.ok()) { - if (coordinatorContext.isEos() && status.isCancelled()) { - LOG.warn("Query {} has returned all results, fragment_id={} instance_id={}, be={}" - + " is reporting failed status {}", - DebugUtil.printId(queryId), params.getFragmentId(), - DebugUtil.printId(params.getFragmentInstanceId()), - params.getBackendId(), - status.toString()); - } else { - LOG.warn("one instance report fail, query_id={} fragment_id={} instance_id={}, be={}," - + " error message: {}", - DebugUtil.printId(queryId), params.getFragmentId(), - DebugUtil.printId(params.getFragmentInstanceId()), - params.getBackendId(), status.toString()); - coordinatorContext.updateStatusIfOk(status); - } - } + + @Override + protected void doProcessReportExecStatus(TReportExecStatusParams params, SingleFragmentPipelineTask fragmentTask) { LoadContext loadContext = coordinatorContext.asLoadProcessor().loadContext; if (params.isSetDeltaUrls()) { loadContext.updateDeltaUrls(params.getDeltaUrls()); @@ -234,7 +196,7 @@ public void updateFragmentExecStatus(TReportExecStatusParams params) { if (fragmentTask.isDone()) { if (LOG.isDebugEnabled()) { LOG.debug("Query {} fragment {} is marked done", - DebugUtil.printId(queryId), params.getFragmentId()); + DebugUtil.printId(coordinatorContext.queryId), params.getFragmentId()); } latch.get().markedCountDown(params.getFragmentId(), params.getBackendId()); } @@ -258,22 +220,6 @@ public void updateFragmentExecStatus(TReportExecStatusParams params) { } } - private Map buildBackendFragmentTasks( - PipelineExecutionTask executionTask) { - ImmutableMap.Builder backendFragmentTasks - = ImmutableMap.builder(); - for (Entry backendTask : executionTask.getChildrenTasks().entrySet()) { - Long backendId = backendTask.getKey(); - for (Entry fragmentIdToTask : backendTask.getValue() - .getChildrenTasks().entrySet()) { - Integer fragmentId = fragmentIdToTask.getKey(); - SingleFragmentPipelineTask fragmentTask = fragmentIdToTask.getValue(); - backendFragmentTasks.put(new BackendFragmentId(backendId, fragmentId), fragmentTask); - } - } - return backendFragmentTasks.build(); - } - /* * Check the state of backends in needCheckBackendExecStates. * return true if all of them are OK. Otherwise, return false. diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/PipelineExecutionTask.java b/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/PipelineExecutionTask.java index 8c1b9714c35db8..ae87d59d075d12 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/PipelineExecutionTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/PipelineExecutionTask.java @@ -102,7 +102,7 @@ public void execute() throws Exception { @Override public String toString() { - return "SqlPipelineTask(\n" + return "PipelineExecutionTask(\n" + childrenTasks.allTasks() .stream() .map(multiFragmentsPipelineTask -> " " + multiFragmentsPipelineTask) diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/PipelineExecutionTaskBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/PipelineExecutionTaskBuilder.java index fd00bf0e3e8536..0da6f4a5fe2e43 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/PipelineExecutionTaskBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/PipelineExecutionTaskBuilder.java @@ -61,7 +61,7 @@ private PipelineExecutionTask buildTask(CoordinatorContext coordinatorContext, backendServiceProxy, buildMultiFragmentTasks(coordinatorContext, backendServiceProxy, workerToFragmentsParam) ); - coordinatorContext.getJobProcessor().setSqlPipelineTask(pipelineExecutionTask); + coordinatorContext.getJobProcessor().setPipelineExecutionTask(pipelineExecutionTask); return pipelineExecutionTask; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/QueryProcessor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/QueryProcessor.java index 2ec38e8cc8e3ea..a5a5100faece1a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/QueryProcessor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/QueryProcessor.java @@ -25,13 +25,14 @@ import org.apache.doris.nereids.trees.plans.distribute.worker.job.AssignedJob; import org.apache.doris.planner.DataSink; import org.apache.doris.planner.ResultSink; +import org.apache.doris.qe.AbstractJobProcessor; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.CoordinatorContext; -import org.apache.doris.qe.JobProcessor; import org.apache.doris.qe.ResultReceiver; import org.apache.doris.qe.RowBatch; import org.apache.doris.rpc.RpcException; import org.apache.doris.thrift.TNetworkAddress; +import org.apache.doris.thrift.TReportExecStatusParams; import org.apache.doris.thrift.TStatusCode; import com.google.common.base.Strings; @@ -44,24 +45,21 @@ import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.Optional; import java.util.concurrent.CopyOnWriteArrayList; -public class QueryProcessor implements JobProcessor { +public class QueryProcessor extends AbstractJobProcessor { private static final Logger LOG = LogManager.getLogger(QueryProcessor.class); // constant fields private final long limitRows; // mutable field - private Optional sqlPipelineTask; - private final CoordinatorContext coordinatorContext; private final List runningReceivers; private int receiverOffset; private long numReceivedRows; public QueryProcessor(CoordinatorContext coordinatorContext, List runningReceivers) { - this.coordinatorContext = Objects.requireNonNull(coordinatorContext, "coordinatorContext can not be null"); + super(coordinatorContext); this.runningReceivers = new CopyOnWriteArrayList<>( Objects.requireNonNull(runningReceivers, "runningReceivers can not be null") ); @@ -69,8 +67,6 @@ public QueryProcessor(CoordinatorContext coordinatorContext, List { + this.executionTask.ifPresent(sqlPipelineTask -> { for (MultiFragmentsPipelineTask fragmentsTask : sqlPipelineTask.getChildrenTasks().values()) { fragmentsTask.cancelExecute(cancelReason); } From 2cde7b0839d1607ef6dd619b7d9db95bc4c211ee Mon Sep 17 00:00:00 2001 From: 924060929 Date: Mon, 2 Dec 2024 16:41:17 +0800 Subject: [PATCH 114/399] [fix](nereids) fix NereidsCoordinator compute wrong result when exists CTE (#44753) fix NereidsCoordinator compute wrong result when exists CTE, introduced by #41730 --- .../doris/qe/runtime/ThriftPlansBuilder.java | 11 +++- .../distribute/test_multicast_sink.groovy | 60 +++++++++++++++++++ 2 files changed, 68 insertions(+), 3 deletions(-) create mode 100644 regression-test/suites/nereids_syntax_p0/distribute/test_multicast_sink.groovy diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/ThriftPlansBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/ThriftPlansBuilder.java index a02ee90e901cd5..54bc0b24d3e821 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/ThriftPlansBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/ThriftPlansBuilder.java @@ -240,14 +240,19 @@ private static Map computeExchangeSenderNum(PipelineDistribute return senderNum; } - private static void setMultiCastDestinationThrift(PipelineDistributedPlan fragmentPlan) { + private static void setMultiCastDestinationThriftIfNotSet(PipelineDistributedPlan fragmentPlan) { MultiCastDataSink multiCastDataSink = (MultiCastDataSink) fragmentPlan.getFragmentJob().getFragment().getSink(); List> destinationList = multiCastDataSink.getDestinations(); List dataStreamSinks = multiCastDataSink.getDataStreamSinks(); for (int i = 0; i < dataStreamSinks.size(); i++) { - DataStreamSink realSink = dataStreamSinks.get(i); List destinations = destinationList.get(i); + if (!destinations.isEmpty()) { + // we should only set destination only once, + // because all backends share the same MultiCastDataSink object + continue; + } + DataStreamSink realSink = dataStreamSinks.get(i); for (Entry> kv : fragmentPlan.getDestinations().entrySet()) { DataSink sink = kv.getKey(); if (sink == realSink) { @@ -318,7 +323,7 @@ private static TPipelineFragmentParams fragmentToThriftIfAbsent( List nonMultiCastDestinations; if (fragment.getSink() instanceof MultiCastDataSink) { nonMultiCastDestinations = Lists.newArrayList(); - setMultiCastDestinationThrift(fragmentPlan); + setMultiCastDestinationThriftIfNotSet(fragmentPlan); } else { nonMultiCastDestinations = nonMultiCastDestinationToThrift(fragmentPlan); } diff --git a/regression-test/suites/nereids_syntax_p0/distribute/test_multicast_sink.groovy b/regression-test/suites/nereids_syntax_p0/distribute/test_multicast_sink.groovy new file mode 100644 index 00000000000000..eeeaad06d5ee22 --- /dev/null +++ b/regression-test/suites/nereids_syntax_p0/distribute/test_multicast_sink.groovy @@ -0,0 +1,60 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("test_multicast_sink") { + multi_sql """ + drop table if exists table_1_undef_partitions2_keys3_properties4_distributed_by5; + CREATE TABLE `table_1_undef_partitions2_keys3_properties4_distributed_by5` ( + `col_int_undef_signed` int NULL, + `col_int_undef_signed_not_null` int NOT NULL, + `col_date_undef_signed` date NULL, + `col_date_undef_signed_not_null` date NOT NULL, + `col_varchar_10__undef_signed` varchar(10) NULL, + `col_varchar_10__undef_signed_not_null` varchar(10) NOT NULL, + `col_varchar_1024__undef_signed` varchar(1024) NULL, + `col_varchar_1024__undef_signed_not_null` varchar(1024) NOT NULL, + `pk` int NULL + ) ENGINE=OLAP + DUPLICATE KEY(`col_int_undef_signed`, `col_int_undef_signed_not_null`, `col_date_undef_signed`) + DISTRIBUTED BY HASH(`pk`) BUCKETS 10 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + insert into table_1_undef_partitions2_keys3_properties4_distributed_by5 values(3, 6, '2023-12-17', '2023-12-17', 'ok', 'v', 'want', 'z', 0); + set enable_nereids_distribute_planner=true; + set parallel_pipeline_task_num = 1; + """ + + for (def i in 0..<100) { + test { + sql """ + WITH cte1 AS( + SELECT t1.`pk` + FROM table_1_undef_partitions2_keys3_properties4_distributed_by5 AS t1 + ORDER BY t1.pk + ) + SELECT cte1.`pk` AS pk1 + FROM cte1 + LEFT OUTER JOIN cte1 AS alias1 + ON cte1 . `pk` = alias1 . `pk` + WHERE cte1.`pk` < 3 + LIMIT 66666666 + """ + result([[0]]) + } + } +} From 30b4df9d4f6d3c1f719775b7104c6105af95872d Mon Sep 17 00:00:00 2001 From: xyf <116467462+xyfsjq@users.noreply.github.com> Date: Mon, 2 Dec 2024 17:30:37 +0800 Subject: [PATCH 115/399] [fix](profile) Change the check of whether the username is admin and root to check whether the user specifically corresponds to admin permissions (#41714) --- .../rest/manager/QueryProfileAction.java | 2 +- .../test_http_permissions_check_auth.groovy | 66 +++++++++++++++++++ 2 files changed, 67 insertions(+), 1 deletion(-) create mode 100644 regression-test/suites/http_auth_check/test_http_permissions_check_auth.groovy diff --git a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/manager/QueryProfileAction.java b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/manager/QueryProfileAction.java index 8c3dfa73aea759..0886edb56fb2f6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/manager/QueryProfileAction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/manager/QueryProfileAction.java @@ -419,7 +419,7 @@ private ResponseEntity getProfileFromAllFrontends(HttpServletRequest request, St private void checkAuthByUserAndQueryId(String queryId) throws AuthenticationException { String user = ConnectContext.get().getCurrentUserIdentity().getQualifiedUser(); - if (!user.equalsIgnoreCase(Auth.ADMIN_USER) && !user.equalsIgnoreCase(Auth.ROOT_USER)) { + if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN)) { ProfileManager.getInstance().checkAuthByUserAndQueryId(user, queryId); } } diff --git a/regression-test/suites/http_auth_check/test_http_permissions_check_auth.groovy b/regression-test/suites/http_auth_check/test_http_permissions_check_auth.groovy new file mode 100644 index 00000000000000..36c6241a244db7 --- /dev/null +++ b/regression-test/suites/http_auth_check/test_http_permissions_check_auth.groovy @@ -0,0 +1,66 @@ +// 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. + +import org.junit.Assert; + +suite("test_http_permissions_check_auth","p0,auth") { + String suiteName = "test_http_permissions_check_auth" + String tableName = "${suiteName}_table" + String user = "${suiteName}_user" + String pwd = 'C123_567p' + try_sql("DROP USER ${user}") + sql """CREATE USER '${user}' IDENTIFIED BY '${pwd}'""" + sql """drop table if exists `${tableName}`""" + sql """ + CREATE TABLE `${tableName}` ( + `k1` int, + `k2` int + ) ENGINE=OLAP + DISTRIBUTED BY random BUCKETS auto + PROPERTIES ('replication_num' = '1') ; + """ + sql """insert into ${tableName} values(1,1)""" + sql """set session_context = 'trace_id:mmn9';""" + sql """select * from ${tableName};""" + + def get_queryid_by_traceid = { check_func -> + httpTest { + basicAuthorization "${user}","${pwd}" + endpoint "${context.config.feHttpAddress}" + uri "/rest/v2/manager/query/trace_id/mmn9" + op "get" + check check_func + } + } + + get_queryid_by_traceid.call() { + respCode, body -> + log.info("body:${body}") + assertTrue("${body}".contains("Bad Request")) + } + + sql """grant 'admin' to ${user}""" + + get_queryid_by_traceid.call() { + respCode, body -> + log.info("body:${body}") + assertTrue("${body}".contains("success")) + } + + sql """drop table if exists `${tableName}`""" + try_sql("DROP USER ${user}") +} From 0995618169d807d815d0abb023ff22eb7b3ce656 Mon Sep 17 00:00:00 2001 From: xzj7019 Date: Mon, 2 Dec 2024 17:44:17 +0800 Subject: [PATCH 116/399] [opt](nereids) remove enable nereids rules usage (#44769) Related PR: (@35925) Problem Summary: (1) Remove enable_nereids_rules usage, and replaced with use_cbo_rule hint. (2) Current case with use_cbo_rule which has deleted the enable_nereids_rules can't really tracking the rule's impact since the enable_nereids_rules related checking can't be passed. This pr activates these part of plan change. --- .../rules/rewrite/PushDownAggThroughJoin.java | 10 - .../PushDownAggThroughJoinOneSide.java | 10 - .../rewrite/PushDownDistinctThroughJoin.java | 7 - .../eager_aggregate/basic.out | 46 +- .../eager_aggregate/basic_one_side.out | 46 +- .../push_down_count_through_join_one_side.out | 472 ++++++++++++------ .../push_down_max_through_join.out | 168 ++++--- .../push_down_min_through_join.out | 166 +++--- .../push_down_sum_through_join_one_side.out | 254 +++++++--- .../push_down_distinct_through_join.out | 11 +- ...sh_down_count_through_join_one_side.groovy | 1 - ...sh_down_limit_distinct_through_join.groovy | 1 - .../constraints/query23.groovy | 1 - .../tpcds_sf100/constraints/query23.groovy | 1 - 14 files changed, 780 insertions(+), 414 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownAggThroughJoin.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownAggThroughJoin.java index f003d2ac2cc918..665e3294a93a72 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownAggThroughJoin.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownAggThroughJoin.java @@ -85,11 +85,6 @@ public List buildRules() { ); }) .thenApply(ctx -> { - Set enableNereidsRules = ctx.cascadesContext.getConnectContext() - .getSessionVariable().getEnableNereidsRules(); - if (!enableNereidsRules.contains(RuleType.PUSH_DOWN_AGG_THROUGH_JOIN.type())) { - return null; - } LogicalAggregate> agg = ctx.root; return pushAgg(agg, agg.child(), ImmutableList.of()); }) @@ -109,11 +104,6 @@ public List buildRules() { ); }) .thenApply(ctx -> { - Set enableNereidsRules = ctx.cascadesContext.getConnectContext() - .getSessionVariable().getEnableNereidsRules(); - if (!enableNereidsRules.contains(RuleType.PUSH_DOWN_AGG_THROUGH_JOIN.type())) { - return null; - } LogicalAggregate>> agg = ctx.root; return pushAgg(agg, agg.child().child(), agg.child().getProjects()); }) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownAggThroughJoinOneSide.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownAggThroughJoinOneSide.java index c5d3d0fb49a0a5..9319b09cc30470 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownAggThroughJoinOneSide.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownAggThroughJoinOneSide.java @@ -79,11 +79,6 @@ public List buildRules() { && (f.children().isEmpty() || f.child(0) instanceof Slot))); }) .thenApply(ctx -> { - Set enableNereidsRules = ctx.cascadesContext.getConnectContext() - .getSessionVariable().getEnableNereidsRules(); - if (!enableNereidsRules.contains(RuleType.PUSH_DOWN_AGG_THROUGH_JOIN_ONE_SIDE.type())) { - return null; - } LogicalAggregate> agg = ctx.root; return pushMinMaxSumCount(agg, agg.child(), ImmutableList.of()); }) @@ -101,11 +96,6 @@ public List buildRules() { && (f.children().isEmpty() || f.child(0) instanceof Slot)); }) .thenApply(ctx -> { - Set enableNereidsRules = ctx.cascadesContext.getConnectContext() - .getSessionVariable().getEnableNereidsRules(); - if (!enableNereidsRules.contains(RuleType.PUSH_DOWN_AGG_THROUGH_JOIN_ONE_SIDE.type())) { - return null; - } LogicalAggregate>> agg = ctx.root; return pushMinMaxSumCount(agg, agg.child().child(), agg.child().getProjects()); }) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownDistinctThroughJoin.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownDistinctThroughJoin.java index fa705a08d8db8a..533452c2f28f61 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownDistinctThroughJoin.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownDistinctThroughJoin.java @@ -18,7 +18,6 @@ package org.apache.doris.nereids.rules.rewrite; import org.apache.doris.nereids.jobs.JobContext; -import org.apache.doris.nereids.rules.RuleType; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.algebra.Relation; import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; @@ -30,7 +29,6 @@ import com.google.common.collect.ImmutableList; -import java.util.Set; import java.util.function.Function; /** @@ -39,11 +37,6 @@ public class PushDownDistinctThroughJoin extends DefaultPlanRewriter implements CustomRewriter { @Override public Plan rewriteRoot(Plan plan, JobContext context) { - Set enableNereidsRules = context.getCascadesContext().getConnectContext() - .getSessionVariable().getEnableNereidsRules(); - if (!enableNereidsRules.contains(RuleType.PUSH_DOWN_DISTINCT_THROUGH_JOIN.type())) { - return null; - } return plan.accept(this, context); } diff --git a/regression-test/data/nereids_rules_p0/eager_aggregate/basic.out b/regression-test/data/nereids_rules_p0/eager_aggregate/basic.out index ba18189efcad82..19b1b3cdb85590 100644 --- a/regression-test/data/nereids_rules_p0/eager_aggregate/basic.out +++ b/regression-test/data/nereids_rules_p0/eager_aggregate/basic.out @@ -40,14 +40,17 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((a.device_id = b.device_id)) otherCondition=() ---------filter((a.event_id = 'ad_click')) -----------PhysicalOlapScan[com_dd_library] ---------filter((cast(experiment_id as DOUBLE) = 37.0)) -----------PhysicalOlapScan[shunt_log_com_dd_library] +--------hashAgg[LOCAL] +----------hashAgg[LOCAL] +------------filter((a.event_id = 'ad_click')) +--------------PhysicalOlapScan[com_dd_library] +--------hashAgg[LOCAL] +----------filter((cast(experiment_id as DOUBLE) = 37.0)) +------------PhysicalOlapScan[shunt_log_com_dd_library] Hint log: -Used: -UnUsed: use_push_down_agg_through_join use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: use_push_down_agg_through_join SyntaxError: -- !with_hint_2 -- @@ -55,13 +58,16 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((a.device_id = b.device_id)) otherCondition=() ---------PhysicalOlapScan[com_dd_library] ---------filter((cast(experiment_id as DOUBLE) = 73.0)) -----------PhysicalOlapScan[shunt_log_com_dd_library] +--------hashAgg[LOCAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[com_dd_library] +--------hashAgg[LOCAL] +----------filter((cast(experiment_id as DOUBLE) = 73.0)) +------------PhysicalOlapScan[shunt_log_com_dd_library] Hint log: -Used: -UnUsed: use_push_down_agg_through_join use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: use_push_down_agg_through_join SyntaxError: -- !with_hint_3 -- @@ -69,13 +75,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((a.device_id = b.device_id)) otherCondition=() ---------PhysicalOlapScan[com_dd_library] +--------hashAgg[LOCAL] +----------PhysicalOlapScan[com_dd_library] --------filter((cast(experiment_id as DOUBLE) = 73.0)) ----------PhysicalOlapScan[shunt_log_com_dd_library] Hint log: -Used: -UnUsed: use_push_down_agg_through_join use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: use_push_down_agg_through_join SyntaxError: -- !with_hint_4 -- @@ -83,11 +90,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((a.device_id = b.device_id)) otherCondition=() ---------PhysicalOlapScan[com_dd_library] ---------PhysicalOlapScan[shunt_log_com_dd_library] +--------hashAgg[LOCAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[com_dd_library] +--------hashAgg[LOCAL] +----------PhysicalOlapScan[shunt_log_com_dd_library] Hint log: -Used: -UnUsed: use_push_down_agg_through_join use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: use_push_down_agg_through_join SyntaxError: diff --git a/regression-test/data/nereids_rules_p0/eager_aggregate/basic_one_side.out b/regression-test/data/nereids_rules_p0/eager_aggregate/basic_one_side.out index aaf6afeca1e9f6..66f1729e2e3816 100644 --- a/regression-test/data/nereids_rules_p0/eager_aggregate/basic_one_side.out +++ b/regression-test/data/nereids_rules_p0/eager_aggregate/basic_one_side.out @@ -40,14 +40,17 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((a.device_id = b.device_id)) otherCondition=() ---------filter((a.event_id = 'ad_click')) -----------PhysicalOlapScan[com_dd_library_one_side] ---------filter((cast(experiment_id as DOUBLE) = 37.0)) -----------PhysicalOlapScan[shunt_log_com_dd_library_one_side] +--------hashAgg[LOCAL] +----------hashAgg[LOCAL] +------------filter((a.event_id = 'ad_click')) +--------------PhysicalOlapScan[com_dd_library_one_side] +--------hashAgg[LOCAL] +----------filter((cast(experiment_id as DOUBLE) = 37.0)) +------------PhysicalOlapScan[shunt_log_com_dd_library_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_2 -- @@ -55,13 +58,16 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((a.device_id = b.device_id)) otherCondition=() ---------PhysicalOlapScan[com_dd_library_one_side] ---------filter((cast(experiment_id as DOUBLE) = 73.0)) -----------PhysicalOlapScan[shunt_log_com_dd_library_one_side] +--------hashAgg[LOCAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[com_dd_library_one_side] +--------hashAgg[LOCAL] +----------filter((cast(experiment_id as DOUBLE) = 73.0)) +------------PhysicalOlapScan[shunt_log_com_dd_library_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_3 -- @@ -69,13 +75,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((a.device_id = b.device_id)) otherCondition=() ---------PhysicalOlapScan[com_dd_library_one_side] +--------hashAgg[LOCAL] +----------PhysicalOlapScan[com_dd_library_one_side] --------filter((cast(experiment_id as DOUBLE) = 73.0)) ----------PhysicalOlapScan[shunt_log_com_dd_library_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_4 -- @@ -83,11 +90,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((a.device_id = b.device_id)) otherCondition=() ---------PhysicalOlapScan[com_dd_library_one_side] ---------PhysicalOlapScan[shunt_log_com_dd_library_one_side] +--------hashAgg[LOCAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[com_dd_library_one_side] +--------hashAgg[LOCAL] +----------PhysicalOlapScan[shunt_log_com_dd_library_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: diff --git a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_count_through_join_one_side.out b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_count_through_join_one_side.out index 8267eb3e38ff91..251bc580d3279d 100644 --- a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_count_through_join_one_side.out +++ b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_count_through_join_one_side.out @@ -404,12 +404,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------PhysicalOlapScan[count_t_one_side] ---------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_left_join -- @@ -456,12 +462,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------PhysicalOlapScan[count_t_one_side] ---------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_left_anti_join -- @@ -482,12 +494,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=((t1.name < t2.name)) ---------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[count_t_one_side] --------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_with_aggregate -- @@ -508,13 +522,19 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------PhysicalOlapScan[count_t_one_side] ---------filter((count_t_one_side.score > 10)) -----------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------filter((count_t_one_side.score > 10)) +------------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_outer_join -- @@ -535,13 +555,19 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------PhysicalOlapScan[count_t_one_side] ---------filter((count_t_one_side.score > 10)) -----------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------filter((count_t_one_side.score > 10)) +------------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_having -- @@ -550,12 +576,18 @@ PhysicalResultSink ----hashAgg[GLOBAL] ------hashAgg[LOCAL] --------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -----------PhysicalOlapScan[count_t_one_side] -----------PhysicalOlapScan[count_t_one_side] +----------hashAgg[GLOBAL] +------------hashAgg[LOCAL] +--------------hashAgg[GLOBAL] +----------------hashAgg[LOCAL] +------------------PhysicalOlapScan[count_t_one_side] +----------hashAgg[GLOBAL] +------------hashAgg[LOCAL] +--------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_mixed_aggregates -- @@ -563,12 +595,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[count_t_one_side] --------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_multi_table_join -- @@ -576,14 +610,26 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() ---------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -----------PhysicalOlapScan[count_t_one_side] -----------PhysicalOlapScan[count_t_one_side] ---------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +------------------hashAgg[GLOBAL] +--------------------hashAgg[LOCAL] +----------------------hashAgg[GLOBAL] +------------------------hashAgg[LOCAL] +--------------------------PhysicalOlapScan[count_t_one_side] +------------------hashAgg[GLOBAL] +--------------------hashAgg[LOCAL] +----------------------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_with_order_by -- @@ -593,12 +639,18 @@ PhysicalResultSink ------hashAgg[GLOBAL] --------hashAgg[LOCAL] ----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -------------PhysicalOlapScan[count_t_one_side] -------------PhysicalOlapScan[count_t_one_side] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------hashAgg[GLOBAL] +------------------hashAgg[LOCAL] +--------------------PhysicalOlapScan[count_t_one_side] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_multiple_equal_conditions -- @@ -606,12 +658,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------PhysicalOlapScan[count_t_one_side] ---------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_equal_conditions_with_aggregate -- @@ -619,12 +677,16 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------PhysicalOlapScan[count_t_one_side] ---------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_equal_conditions_non_aggregate_selection -- @@ -632,12 +694,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------PhysicalOlapScan[count_t_one_side] ---------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_equal_conditions_non_aggregate_selection_with_aggregate -- @@ -645,12 +713,20 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------PhysicalOlapScan[count_t_one_side] ---------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_with_where_clause -- @@ -658,13 +734,19 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------PhysicalOlapScan[count_t_one_side] ---------filter((t1.score > 50)) -----------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------filter((t1.score > 50)) +------------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_varied_aggregates -- @@ -687,12 +769,18 @@ PhysicalResultSink ------hashAgg[GLOBAL] --------hashAgg[LOCAL] ----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -------------PhysicalOlapScan[count_t_one_side] -------------PhysicalOlapScan[count_t_one_side] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------hashAgg[GLOBAL] +------------------hashAgg[LOCAL] +--------------------PhysicalOlapScan[count_t_one_side] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_alias_multiple_equal_conditions -- @@ -700,12 +788,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1_alias.id = t2_alias.id) and (t1_alias.name = t2_alias.name)) otherCondition=() ---------PhysicalOlapScan[count_t_one_side] ---------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_complex_join_condition -- @@ -713,12 +807,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.score = t2.score)) otherCondition=(( not (name = name))) ---------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[count_t_one_side] --------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_function_processed_columns -- @@ -739,14 +835,20 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------filter((count_t_one_side.id < 100)) -----------PhysicalOlapScan[count_t_one_side] ---------filter((count_t_one_side.score > 20) and (t1.id < 100)) -----------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------filter((count_t_one_side.id < 100)) +--------------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------filter((count_t_one_side.score > 20) and (t1.id < 100)) +------------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_basic -- @@ -754,12 +856,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------PhysicalOlapScan[count_t_one_side] ---------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_left_join -- @@ -806,12 +914,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------PhysicalOlapScan[count_t_one_side] ---------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_left_anti_join -- @@ -832,12 +946,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=((t1.name < t2.name)) ---------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[count_t_one_side] --------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_with_aggregate -- @@ -845,12 +961,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------PhysicalOlapScan[count_t_one_side] ---------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_subquery -- @@ -858,13 +980,19 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------PhysicalOlapScan[count_t_one_side] ---------filter((count_t_one_side.score > 10)) -----------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------filter((count_t_one_side.score > 10)) +------------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_outer_join -- @@ -885,13 +1013,19 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------PhysicalOlapScan[count_t_one_side] ---------filter((count_t_one_side.score > 10)) -----------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------filter((count_t_one_side.score > 10)) +------------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_having -- @@ -900,12 +1034,18 @@ PhysicalResultSink ----hashAgg[GLOBAL] ------hashAgg[LOCAL] --------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -----------PhysicalOlapScan[count_t_one_side] -----------PhysicalOlapScan[count_t_one_side] +----------hashAgg[GLOBAL] +------------hashAgg[LOCAL] +--------------hashAgg[GLOBAL] +----------------hashAgg[LOCAL] +------------------PhysicalOlapScan[count_t_one_side] +----------hashAgg[GLOBAL] +------------hashAgg[LOCAL] +--------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_multi_table_join -- @@ -913,14 +1053,26 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() ---------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -----------PhysicalOlapScan[count_t_one_side] -----------PhysicalOlapScan[count_t_one_side] ---------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +------------------hashAgg[GLOBAL] +--------------------hashAgg[LOCAL] +----------------------hashAgg[GLOBAL] +------------------------hashAgg[LOCAL] +--------------------------PhysicalOlapScan[count_t_one_side] +------------------hashAgg[GLOBAL] +--------------------hashAgg[LOCAL] +----------------------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_with_order_by -- @@ -930,12 +1082,18 @@ PhysicalResultSink ------hashAgg[GLOBAL] --------hashAgg[LOCAL] ----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -------------PhysicalOlapScan[count_t_one_side] -------------PhysicalOlapScan[count_t_one_side] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------hashAgg[GLOBAL] +------------------hashAgg[LOCAL] +--------------------PhysicalOlapScan[count_t_one_side] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_multiple_equal_conditions -- @@ -943,12 +1101,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------PhysicalOlapScan[count_t_one_side] ---------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_equal_conditions_non_aggregate_selection -- @@ -956,12 +1120,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------PhysicalOlapScan[count_t_one_side] ---------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_with_where_clause -- @@ -969,13 +1139,19 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------PhysicalOlapScan[count_t_one_side] ---------filter((t1.score > 50)) -----------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------filter((t1.score > 50)) +------------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_varied_aggregates -- @@ -998,12 +1174,18 @@ PhysicalResultSink ------hashAgg[GLOBAL] --------hashAgg[LOCAL] ----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -------------PhysicalOlapScan[count_t_one_side] -------------PhysicalOlapScan[count_t_one_side] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------hashAgg[GLOBAL] +------------------hashAgg[LOCAL] +--------------------PhysicalOlapScan[count_t_one_side] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_complex_join_condition -- @@ -1011,12 +1193,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.score = t2.score)) otherCondition=(( not (name = name))) ---------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[count_t_one_side] --------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_nested_queries -- @@ -1024,14 +1208,20 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------filter((count_t_one_side.id < 100)) -----------PhysicalOlapScan[count_t_one_side] ---------filter((count_t_one_side.score > 20) and (t1.id < 100)) -----------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------filter((count_t_one_side.id < 100)) +--------------PhysicalOlapScan[count_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------filter((count_t_one_side.score > 20) and (t1.id < 100)) +------------------PhysicalOlapScan[count_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !shape -- diff --git a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_max_through_join.out b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_max_through_join.out index 79b4ed890ded95..7cf9813e5b7181 100644 --- a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_max_through_join.out +++ b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_max_through_join.out @@ -224,12 +224,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------PhysicalOlapScan[max_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[max_t] --------PhysicalOlapScan[max_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_left_join -- @@ -276,12 +278,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------PhysicalOlapScan[max_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[max_t] --------PhysicalOlapScan[max_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_left_anti_join -- @@ -302,12 +306,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=((t1.name < t2.name)) ---------PhysicalOlapScan[max_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[max_t] --------PhysicalOlapScan[max_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_with_aggregate -- @@ -329,12 +335,14 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------PhysicalOlapScan[max_t] ---------filter((max_t.score > 10)) -----------PhysicalOlapScan[max_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------filter((max_t.score > 10)) +--------------PhysicalOlapScan[max_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_outer_join -- @@ -356,12 +364,14 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------PhysicalOlapScan[max_t] ---------filter((max_t.score > 10)) -----------PhysicalOlapScan[max_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------filter((max_t.score > 10)) +--------------PhysicalOlapScan[max_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_having -- @@ -370,12 +380,14 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------PhysicalOlapScan[max_t] ---------filter((t1.score > 100)) -----------PhysicalOlapScan[max_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------filter((t1.score > 100)) +--------------PhysicalOlapScan[max_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_mixed_aggregates -- @@ -383,12 +395,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------PhysicalOlapScan[max_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[max_t] --------PhysicalOlapScan[max_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_multi_table_join -- @@ -396,14 +410,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() ---------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -----------PhysicalOlapScan[max_t] -----------PhysicalOlapScan[max_t] --------PhysicalOlapScan[max_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------------hashAgg[GLOBAL] +----------------hashAgg[LOCAL] +------------------PhysicalOlapScan[max_t] +--------------PhysicalOlapScan[max_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_with_order_by -- @@ -413,12 +431,14 @@ PhysicalResultSink ------hashAgg[GLOBAL] --------hashAgg[LOCAL] ----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -------------PhysicalOlapScan[max_t] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[max_t] ------------PhysicalOlapScan[max_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_multiple_equal_conditions -- @@ -426,12 +446,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------PhysicalOlapScan[max_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[max_t] --------PhysicalOlapScan[max_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_equal_conditions_with_aggregate -- @@ -439,12 +461,16 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------PhysicalOlapScan[max_t] ---------PhysicalOlapScan[max_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[max_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[max_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_equal_conditions_non_aggregate_selection -- @@ -452,12 +478,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------PhysicalOlapScan[max_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[max_t] --------PhysicalOlapScan[max_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_equal_conditions_non_aggregate_selection_with_aggregate -- @@ -465,12 +493,16 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------PhysicalOlapScan[max_t] ---------PhysicalOlapScan[max_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[max_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[max_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_with_where_clause -- @@ -479,12 +511,14 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------PhysicalOlapScan[max_t] ---------filter((t1.score > 50)) -----------PhysicalOlapScan[max_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------filter((t1.score > 50)) +--------------PhysicalOlapScan[max_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_varied_aggregates -- @@ -507,12 +541,14 @@ PhysicalResultSink ------hashAgg[GLOBAL] --------hashAgg[LOCAL] ----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -------------PhysicalOlapScan[max_t] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[max_t] ------------PhysicalOlapScan[max_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_alias_multiple_equal_conditions -- @@ -520,12 +556,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1_alias.id = t2_alias.id) and (t1_alias.name = t2_alias.name)) otherCondition=() ---------PhysicalOlapScan[max_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[max_t] --------PhysicalOlapScan[max_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_complex_join_condition -- @@ -533,12 +571,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.score = t2.score)) otherCondition=(( not (name = name))) ---------PhysicalOlapScan[max_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[max_t] --------PhysicalOlapScan[max_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_function_processed_columns -- @@ -561,11 +601,13 @@ PhysicalResultSink ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------filter((max_t.id < 100)) ----------PhysicalOlapScan[max_t] ---------filter((max_t.score > 20) and (t1.id < 100)) -----------PhysicalOlapScan[max_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------filter((max_t.score > 20) and (t1.id < 100)) +--------------PhysicalOlapScan[max_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: diff --git a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_min_through_join.out b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_min_through_join.out index d4efe7df093503..e435bcd4efd335 100644 --- a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_min_through_join.out +++ b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_min_through_join.out @@ -224,12 +224,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------PhysicalOlapScan[min_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[min_t] --------PhysicalOlapScan[min_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_left_join -- @@ -276,12 +278,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------PhysicalOlapScan[min_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[min_t] --------PhysicalOlapScan[min_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_left_anti_join -- @@ -302,12 +306,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=((t1.name < t2.name)) ---------PhysicalOlapScan[min_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[min_t] --------PhysicalOlapScan[min_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_with_aggregate -- @@ -329,12 +335,14 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------PhysicalOlapScan[min_t] ---------filter((min_t.score > 10)) -----------PhysicalOlapScan[min_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------filter((min_t.score > 10)) +--------------PhysicalOlapScan[min_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_outer_join -- @@ -356,12 +364,14 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------PhysicalOlapScan[min_t] ---------filter((min_t.score > 10)) -----------PhysicalOlapScan[min_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------filter((min_t.score > 10)) +--------------PhysicalOlapScan[min_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_having -- @@ -370,12 +380,14 @@ PhysicalResultSink ----hashAgg[GLOBAL] ------hashAgg[LOCAL] --------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -----------PhysicalOlapScan[min_t] +----------hashAgg[GLOBAL] +------------hashAgg[LOCAL] +--------------PhysicalOlapScan[min_t] ----------PhysicalOlapScan[min_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_mixed_aggregates -- @@ -383,12 +395,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------PhysicalOlapScan[min_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[min_t] --------PhysicalOlapScan[min_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_multi_table_join -- @@ -396,14 +410,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() ---------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -----------PhysicalOlapScan[min_t] -----------PhysicalOlapScan[min_t] --------PhysicalOlapScan[min_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------------hashAgg[GLOBAL] +----------------hashAgg[LOCAL] +------------------PhysicalOlapScan[min_t] +--------------PhysicalOlapScan[min_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_with_order_by -- @@ -413,12 +431,14 @@ PhysicalResultSink ------hashAgg[GLOBAL] --------hashAgg[LOCAL] ----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -------------PhysicalOlapScan[min_t] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[min_t] ------------PhysicalOlapScan[min_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_multiple_equal_conditions -- @@ -426,12 +446,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------PhysicalOlapScan[min_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[min_t] --------PhysicalOlapScan[min_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_equal_conditions_with_aggregate -- @@ -439,12 +461,16 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------PhysicalOlapScan[min_t] ---------PhysicalOlapScan[min_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[min_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[min_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_equal_conditions_non_aggregate_selection -- @@ -452,12 +478,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------PhysicalOlapScan[min_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[min_t] --------PhysicalOlapScan[min_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_equal_conditions_non_aggregate_selection_with_aggregate -- @@ -465,12 +493,16 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------PhysicalOlapScan[min_t] ---------PhysicalOlapScan[min_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[min_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[min_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_with_where_clause -- @@ -479,12 +511,14 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------PhysicalOlapScan[min_t] ---------filter((t1.score > 50)) -----------PhysicalOlapScan[min_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------filter((t1.score > 50)) +--------------PhysicalOlapScan[min_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_varied_aggregates -- @@ -507,12 +541,14 @@ PhysicalResultSink ------hashAgg[GLOBAL] --------hashAgg[LOCAL] ----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -------------PhysicalOlapScan[min_t] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[min_t] ------------PhysicalOlapScan[min_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_alias_multiple_equal_conditions -- @@ -520,12 +556,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1_alias.id = t2_alias.id) and (t1_alias.name = t2_alias.name)) otherCondition=() ---------PhysicalOlapScan[min_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[min_t] --------PhysicalOlapScan[min_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_complex_join_condition -- @@ -533,12 +571,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.score = t2.score)) otherCondition=(( not (name = name))) ---------PhysicalOlapScan[min_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[min_t] --------PhysicalOlapScan[min_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_function_processed_columns -- @@ -561,11 +601,13 @@ PhysicalResultSink ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------filter((min_t.id < 100)) ----------PhysicalOlapScan[min_t] ---------filter((min_t.score > 20) and (t1.id < 100)) -----------PhysicalOlapScan[min_t] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------filter((min_t.score > 20) and (t1.id < 100)) +--------------PhysicalOlapScan[min_t] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: diff --git a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_sum_through_join_one_side.out b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_sum_through_join_one_side.out index 5013dc7dbdfb93..3a945f58e29e3c 100644 --- a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_sum_through_join_one_side.out +++ b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_sum_through_join_one_side.out @@ -224,12 +224,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------PhysicalOlapScan[sum_t_one_side] ---------PhysicalOlapScan[sum_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[sum_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[sum_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_left_join -- @@ -276,12 +282,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------PhysicalOlapScan[sum_t_one_side] ---------PhysicalOlapScan[sum_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[sum_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[sum_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_left_anti_join -- @@ -302,12 +314,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=((t1.name < t2.name)) ---------PhysicalOlapScan[sum_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[sum_t_one_side] --------PhysicalOlapScan[sum_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_with_aggregate -- @@ -328,13 +342,19 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------PhysicalOlapScan[sum_t_one_side] ---------filter((sum_t_one_side.score > 10)) -----------PhysicalOlapScan[sum_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[sum_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------filter((sum_t_one_side.score > 10)) +------------------PhysicalOlapScan[sum_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_outer_join -- @@ -355,13 +375,19 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------PhysicalOlapScan[sum_t_one_side] ---------filter((sum_t_one_side.score > 10)) -----------PhysicalOlapScan[sum_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[sum_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------filter((sum_t_one_side.score > 10)) +------------------PhysicalOlapScan[sum_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_having -- @@ -370,12 +396,18 @@ PhysicalResultSink ----hashAgg[GLOBAL] ------hashAgg[LOCAL] --------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -----------PhysicalOlapScan[sum_t_one_side] -----------PhysicalOlapScan[sum_t_one_side] +----------hashAgg[GLOBAL] +------------hashAgg[LOCAL] +--------------PhysicalOlapScan[sum_t_one_side] +----------hashAgg[GLOBAL] +------------hashAgg[LOCAL] +--------------hashAgg[GLOBAL] +----------------hashAgg[LOCAL] +------------------PhysicalOlapScan[sum_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_mixed_aggregates -- @@ -383,12 +415,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------PhysicalOlapScan[sum_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[sum_t_one_side] --------PhysicalOlapScan[sum_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_multi_table_join -- @@ -396,14 +430,26 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() ---------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -----------PhysicalOlapScan[sum_t_one_side] -----------PhysicalOlapScan[sum_t_one_side] ---------PhysicalOlapScan[sum_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[sum_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +------------------hashAgg[GLOBAL] +--------------------hashAgg[LOCAL] +----------------------PhysicalOlapScan[sum_t_one_side] +------------------hashAgg[GLOBAL] +--------------------hashAgg[LOCAL] +----------------------hashAgg[GLOBAL] +------------------------hashAgg[LOCAL] +--------------------------PhysicalOlapScan[sum_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_with_order_by -- @@ -413,12 +459,18 @@ PhysicalResultSink ------hashAgg[GLOBAL] --------hashAgg[LOCAL] ----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -------------PhysicalOlapScan[sum_t_one_side] -------------PhysicalOlapScan[sum_t_one_side] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[sum_t_one_side] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------hashAgg[GLOBAL] +------------------hashAgg[LOCAL] +--------------------PhysicalOlapScan[sum_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_multiple_equal_conditions -- @@ -426,12 +478,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------PhysicalOlapScan[sum_t_one_side] ---------PhysicalOlapScan[sum_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[sum_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[sum_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_equal_conditions_with_aggregate -- @@ -439,12 +497,16 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------PhysicalOlapScan[sum_t_one_side] ---------PhysicalOlapScan[sum_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[sum_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[sum_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_equal_conditions_non_aggregate_selection -- @@ -452,12 +514,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------PhysicalOlapScan[sum_t_one_side] ---------PhysicalOlapScan[sum_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[sum_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[sum_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_equal_conditions_non_aggregate_selection_with_aggregate -- @@ -465,12 +533,20 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------PhysicalOlapScan[sum_t_one_side] ---------PhysicalOlapScan[sum_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[sum_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[sum_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_with_where_clause -- @@ -478,13 +554,19 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------PhysicalOlapScan[sum_t_one_side] ---------filter((t1.score > 50)) -----------PhysicalOlapScan[sum_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[sum_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------filter((t1.score > 50)) +------------------PhysicalOlapScan[sum_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_varied_aggregates -- @@ -507,12 +589,18 @@ PhysicalResultSink ------hashAgg[GLOBAL] --------hashAgg[LOCAL] ----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -------------PhysicalOlapScan[sum_t_one_side] -------------PhysicalOlapScan[sum_t_one_side] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[sum_t_one_side] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------hashAgg[GLOBAL] +------------------hashAgg[LOCAL] +--------------------PhysicalOlapScan[sum_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_alias_multiple_equal_conditions -- @@ -520,12 +608,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1_alias.id = t2_alias.id) and (t1_alias.name = t2_alias.name)) otherCondition=() ---------PhysicalOlapScan[sum_t_one_side] ---------PhysicalOlapScan[sum_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[sum_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[sum_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_complex_join_condition -- @@ -533,12 +627,14 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.score = t2.score)) otherCondition=(( not (name = name))) ---------PhysicalOlapScan[sum_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[sum_t_one_side] --------PhysicalOlapScan[sum_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: -- !with_hint_groupby_pushdown_function_processed_columns -- @@ -559,13 +655,19 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------filter((sum_t_one_side.id < 100)) -----------PhysicalOlapScan[sum_t_one_side] ---------filter((sum_t_one_side.score > 20) and (t1.id < 100)) -----------PhysicalOlapScan[sum_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------filter((sum_t_one_side.id < 100)) +--------------PhysicalOlapScan[sum_t_one_side] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------filter((sum_t_one_side.score > 20) and (t1.id < 100)) +------------------PhysicalOlapScan[sum_t_one_side] Hint log: -Used: -UnUsed: use_push_down_agg_through_join_one_side +Used: use_push_down_agg_through_join_one_side +UnUsed: SyntaxError: diff --git a/regression-test/data/nereids_rules_p0/push_down_distinct_through_join/push_down_distinct_through_join.out b/regression-test/data/nereids_rules_p0/push_down_distinct_through_join/push_down_distinct_through_join.out index 65baed4f1194a0..4c5c60acd31a8e 100644 --- a/regression-test/data/nereids_rules_p0/push_down_distinct_through_join/push_down_distinct_through_join.out +++ b/regression-test/data/nereids_rules_p0/push_down_distinct_through_join/push_down_distinct_through_join.out @@ -15,14 +15,15 @@ SyntaxError: PhysicalResultSink --hashAgg[LOCAL] ----hashJoin[INNER_JOIN] hashCondition=((t1.id = t3.id)) otherCondition=() -------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------PhysicalOlapScan[t] ---------PhysicalOlapScan[t] +------hashAgg[LOCAL] +--------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +----------PhysicalOlapScan[t] +----------PhysicalOlapScan[t] ------PhysicalOlapScan[t] Hint log: -Used: -UnUsed: use_push_down_distinct_through_join +Used: use_push_down_distinct_through_join +UnUsed: SyntaxError: -- !basic_sql -- diff --git a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_through_join_one_side.groovy b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_through_join_one_side.groovy index e551fa04c9110a..031430a48265ae 100644 --- a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_through_join_one_side.groovy +++ b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_through_join_one_side.groovy @@ -453,7 +453,6 @@ suite("push_down_count_through_join_one_side") { insert into dwd_tracking_sensor_init_tmp_ymd values(1, '2024-08-19', 'click'), (1, '2024-08-19', 'click'); """ sql """ - set ENABLE_NEREIDS_RULES = "PUSH_DOWN_AGG_THROUGH_JOIN_ONE_SIDE"; set disable_join_reorder=true; """ diff --git a/regression-test/suites/nereids_rules_p0/push_down_limit_distinct/push_down_limit_distinct_through_join.groovy b/regression-test/suites/nereids_rules_p0/push_down_limit_distinct/push_down_limit_distinct_through_join.groovy index f53b100c29aff2..22a59dd1cbdf46 100644 --- a/regression-test/suites/nereids_rules_p0/push_down_limit_distinct/push_down_limit_distinct_through_join.groovy +++ b/regression-test/suites/nereids_rules_p0/push_down_limit_distinct/push_down_limit_distinct_through_join.groovy @@ -21,7 +21,6 @@ suite("push_down_limit_distinct_through_join") { SET enable_fallback_to_original_planner=false; set runtime_filter_mode=OFF; SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'; - SET ENABLE_NEREIDS_RULES=push_down_limit_distinct_through_join; set push_topn_to_agg = true; SET disable_join_reorder=true; set topn_opt_limit_threshold=1024; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/constraints/query23.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/constraints/query23.groovy index 96e49d61bfa89f..9010034d49273e 100644 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/constraints/query23.groovy +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/constraints/query23.groovy @@ -37,7 +37,6 @@ suite("query23") { set disable_nereids_rules='PRUNE_EMPTY_PARTITION'; set push_topn_to_agg = true; set topn_opt_limit_threshold=1024; - set enable_nereids_rules='ELIMINATE_GROUP_BY_KEY'; """ def ds = """with frequent_ss_items as diff --git a/regression-test/suites/new_shapes_p0/tpcds_sf100/constraints/query23.groovy b/regression-test/suites/new_shapes_p0/tpcds_sf100/constraints/query23.groovy index b17e9c6d3d0e7f..82db5a725ce44a 100644 --- a/regression-test/suites/new_shapes_p0/tpcds_sf100/constraints/query23.groovy +++ b/regression-test/suites/new_shapes_p0/tpcds_sf100/constraints/query23.groovy @@ -40,7 +40,6 @@ suite("query23") { set disable_nereids_rules='PRUNE_EMPTY_PARTITION'; set push_topn_to_agg = true; set topn_opt_limit_threshold=1024; - set enable_nereids_rules='ELIMINATE_GROUP_BY_KEY'; """ def ds = """with frequent_ss_items as From 6f87e35bced5be8f6aec0f6b4a8eb84b496c2efa Mon Sep 17 00:00:00 2001 From: seawinde Date: Mon, 2 Dec 2024 19:14:32 +0800 Subject: [PATCH 117/399] [fix](mtmv) Fix data wrong when query with table operator such as TABLESAMPLE or tablet and so on (#43030) Related PR: #28064 Materialized view is as following: CREATE MATERIALIZED VIEW mv1 BUILD IMMEDIATE REFRESH AUTO ON MANUAL DISTRIBUTED BY RANDOM BUCKETS 2 PROPERTIES ('replication_num' = '1') AS select * from orders If run query as following, should rewrite fail by materialized view above to make sure data correctness select * from orders TABLET(110); select * from orders index query_index_test; select * from orders TABLESAMPLE(20 percent); select * from orders_partition PARTITION (day_2); At before, this would rewrite by materialized view succesfully and the result data is wrong, This pr fix this. --- .../org/apache/doris/mtmv/MTMVPlanUtil.java | 6 + .../nereids/rules/analysis/BindRelation.java | 4 + .../mv/InitMaterializationContextHook.java | 5 + .../exploration/mv/MaterializedViewUtils.java | 20 +- .../rules/exploration/mv/StructInfo.java | 7 + .../rules/rewrite/PruneEmptyPartition.java | 7 +- .../rules/rewrite/PruneOlapScanTablet.java | 6 +- .../plans/commands/info/CreateMTMVInfo.java | 3 + .../trees/plans/logical/LogicalOlapScan.java | 55 +++-- .../rewrite/PruneOlapScanTabletTest.java | 21 +- .../doris/nereids/sqltest/SqlTestBase.java | 3 + .../doris/regression/suite/Suite.groovy | 16 ++ .../is_in_debug_mode/is_in_debug_mode.groovy | 156 ++++++++++++++ .../with_table_operator.groovy | 195 ++++++++++++++++++ 14 files changed, 482 insertions(+), 22 deletions(-) create mode 100644 regression-test/suites/nereids_rules_p0/mv/is_in_debug_mode/is_in_debug_mode.groovy create mode 100644 regression-test/suites/nereids_rules_p0/mv/with_table_operator/with_table_operator.groovy diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java index c0cd47bd5a0f2f..576e87b44f8d3e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java @@ -58,6 +58,12 @@ public static ConnectContext createMTMVContext(MTMV mtmv) { ctx.setCurrentUserIdentity(UserIdentity.ADMIN); ctx.getState().reset(); ctx.setThreadLocalInfo(); + // Debug session variable should be disabled when refreshed + ctx.getSessionVariable().skipDeletePredicate = false; + ctx.getSessionVariable().skipDeleteBitmap = false; + ctx.getSessionVariable().skipDeleteSign = false; + ctx.getSessionVariable().skipStorageEngineMerge = false; + ctx.getSessionVariable().showHiddenColumns = false; ctx.getSessionVariable().allowModifyMaterializedViewData = true; // Disable add default limit rule to avoid refresh data wrong ctx.getSessionVariable().setDisableNereidsRules( diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java index cba3afca6f08d3..c62dda5a539df7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java @@ -243,6 +243,10 @@ private LogicalPlan makeOlapScan(TableIf table, UnboundRelation unboundRelation, unboundRelation.getTableSample()); } } + if (!tabletIds.isEmpty()) { + // This tabletIds is set manually, so need to set specifiedTabletIds + scan = scan.withManuallySpecifiedTabletIds(tabletIds); + } if (needGenerateLogicalAggForRandomDistAggTable(scan)) { // it's a random distribution agg table // add agg on olap scan diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/InitMaterializationContextHook.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/InitMaterializationContextHook.java index 2e8baecf165639..f9ea00e178b5b7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/InitMaterializationContextHook.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/InitMaterializationContextHook.java @@ -79,6 +79,11 @@ public void initMaterializationContext(CascadesContext cascadesContext) { * @param cascadesContext current cascadesContext in the planner */ protected void doInitMaterializationContext(CascadesContext cascadesContext) { + if (cascadesContext.getConnectContext().getSessionVariable().isInDebugMode()) { + LOG.info(String.format("MaterializationContext init return because is in debug mode, current queryId is %s", + cascadesContext.getConnectContext().getQueryIdentifier())); + return; + } // Only collect the table or mv which query use directly, to avoid useless mv partition in rewrite TableCollectorContext collectorContext = new TableCollectorContext(Sets.newHashSet(), false); try { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtils.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtils.java index 484abd11f01e72..ee4b002007ea05 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtils.java @@ -347,7 +347,15 @@ public Plan visitLogicalResultSink(LogicalResultSink logicalResu planner.getCascadesContext().getMemo().getRoot().getStatistics(), null); } - private static final class TableQueryOperatorChecker extends DefaultPlanVisitor { + /** + * Check the query if Contains query operator + * Such sql as following should return true + * select * from orders TABLET(10098) because TABLET(10098) should return true + * select * from orders_partition PARTITION (day_2) because PARTITION (day_2) + * select * from orders index query_index_test because index query_index_test + * select * from orders TABLESAMPLE(20 percent) because TABLESAMPLE(20 percent) + * */ + public static final class TableQueryOperatorChecker extends DefaultPlanVisitor { public static final TableQueryOperatorChecker INSTANCE = new TableQueryOperatorChecker(); @Override @@ -358,12 +366,20 @@ public Boolean visitLogicalRelation(LogicalRelation relation, Void context) { if (relation instanceof LogicalOlapScan) { LogicalOlapScan logicalOlapScan = (LogicalOlapScan) relation; if (logicalOlapScan.getTableSample().isPresent()) { + // Contain sample, select * from orders TABLESAMPLE(20 percent) return true; } - if (!logicalOlapScan.getSelectedTabletIds().isEmpty()) { + if (!logicalOlapScan.getManuallySpecifiedTabletIds().isEmpty()) { + // Contain tablets, select * from orders TABLET(10098) because TABLET(10098) return true; } if (!logicalOlapScan.getManuallySpecifiedPartitions().isEmpty()) { + // Contain specified partitions, select * from orders_partition PARTITION (day_2) + return true; + } + if (logicalOlapScan.getSelectedIndexId() != logicalOlapScan.getTable().getBaseIndexId()) { + // Contains select index or use sync mv in rbo rewrite + // select * from orders index query_index_test return true; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/StructInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/StructInfo.java index 526ec7030d2db5..5a84ab787d7bd5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/StructInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/StructInfo.java @@ -28,6 +28,7 @@ import org.apache.doris.nereids.jobs.joinorder.hypergraph.node.StructInfoNode; import org.apache.doris.nereids.memo.Group; import org.apache.doris.nereids.memo.GroupExpression; +import org.apache.doris.nereids.rules.exploration.mv.MaterializedViewUtils.TableQueryOperatorChecker; import org.apache.doris.nereids.rules.exploration.mv.Predicates.SplitPredicate; import org.apache.doris.nereids.trees.copier.DeepCopierContext; import org.apache.doris.nereids.trees.copier.LogicalPlanDeepCopier; @@ -36,6 +37,7 @@ import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.SlotReference; import org.apache.doris.nereids.trees.expressions.literal.Literal; +import org.apache.doris.nereids.trees.plans.AbstractPlan; import org.apache.doris.nereids.trees.plans.GroupPlan; import org.apache.doris.nereids.trees.plans.JoinType; import org.apache.doris.nereids.trees.plans.ObjectId; @@ -323,6 +325,11 @@ public static StructInfo of(Plan originalPlan, @Nullable Plan topPlan, @Nullable cascadesContext); valid = valid && hyperGraph.getNodes().stream().allMatch(n -> ((StructInfoNode) n).getExpressions() != null); + // if relationList has any relation which contains table operator, + // such as query with sample, index, table, is invalid + boolean invalid = relationList.stream().anyMatch(relation -> + ((AbstractPlan) relation).accept(TableQueryOperatorChecker.INSTANCE, null)); + valid = valid && !invalid; // collect predicate from top plan which not in hyper graph Set topPlanPredicates = new LinkedHashSet<>(); topPlan.accept(PREDICATE_COLLECTOR, topPlanPredicates); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneEmptyPartition.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneEmptyPartition.java index 5465211c73e793..c7b8f452afbe66 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneEmptyPartition.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneEmptyPartition.java @@ -36,11 +36,16 @@ public Rule build() { return logicalOlapScan().thenApply(ctx -> { LogicalOlapScan scan = ctx.root; OlapTable table = scan.getTable(); - List ids = table.selectNonEmptyPartitionIds(scan.getSelectedPartitionIds()); + List partitionIdsToPrune = scan.getSelectedPartitionIds(); + List ids = table.selectNonEmptyPartitionIds(partitionIdsToPrune); if (ids.isEmpty()) { return new LogicalEmptyRelation(ConnectContext.get().getStatementContext().getNextRelationId(), scan.getOutput()); } + if (partitionIdsToPrune.equals(ids)) { + // Not Prune actually, return directly + return null; + } return scan.withSelectedPartitionIds(ids); }).toRule(RuleType.PRUNE_EMPTY_PARTITION); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneOlapScanTablet.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneOlapScanTablet.java index 4b86b7a3706f0e..fc212550b6f364 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneOlapScanTablet.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneOlapScanTablet.java @@ -53,7 +53,7 @@ public Rule build() { LogicalOlapScan olapScan = filter.child(); OlapTable table = olapScan.getTable(); Builder selectedTabletIdsBuilder = ImmutableList.builder(); - if (olapScan.getSelectedTabletIds().isEmpty()) { + if (olapScan.getManuallySpecifiedTabletIds().isEmpty()) { for (Long id : olapScan.getSelectedPartitionIds()) { Partition partition = table.getPartition(id); MaterializedIndex index = partition.getIndex(olapScan.getSelectedIndexId()); @@ -64,10 +64,10 @@ public Rule build() { partition.getDistributionInfo())); } } else { - selectedTabletIdsBuilder.addAll(olapScan.getSelectedTabletIds()); + selectedTabletIdsBuilder.addAll(olapScan.getManuallySpecifiedTabletIds()); } List selectedTabletIds = selectedTabletIdsBuilder.build(); - if (new HashSet<>(selectedTabletIds).equals(new HashSet<>(olapScan.getSelectedTabletIds()))) { + if (new HashSet<>(selectedTabletIds).equals(new HashSet<>(olapScan.getManuallySpecifiedTabletIds()))) { return null; } return filter.withChildren(olapScan.withSelectedTabletIds(selectedTabletIds)); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java index 5c6c71c92e8111..f9e298fd9d40d4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java @@ -157,6 +157,9 @@ public void analyze(ConnectContext ctx) throws Exception { if (!InternalCatalog.INTERNAL_CATALOG_NAME.equals(mvName.getCtl())) { throw new AnalysisException("Only support creating asynchronous materialized views in internal catalog"); } + if (ctx.getSessionVariable().isInDebugMode()) { + throw new AnalysisException("Create materialized view fail, because is in debug mode"); + } try { FeNameFormat.checkTableName(mvName.getTbl()); } catch (org.apache.doris.common.AnalysisException e) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalOlapScan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalOlapScan.java index 90ceb24231340c..2216e58c4fa3b8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalOlapScan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalOlapScan.java @@ -99,6 +99,12 @@ public class LogicalOlapScan extends LogicalCatalogRelation implements OlapScan */ private final List selectedTabletIds; + /** + * Selected tablet ids to read data from, this would be set if user query with tablets manually + * Such as select * from orders TABLET(100); + */ + private final List manuallySpecifiedTabletIds; + /////////////////////////////////////////////////////////////////////////// // Members for partition ids. /////////////////////////////////////////////////////////////////////////// @@ -127,12 +133,16 @@ public LogicalOlapScan(RelationId id, OlapTable table) { this(id, table, ImmutableList.of()); } + /** + * LogicalOlapScan construct method + */ public LogicalOlapScan(RelationId id, OlapTable table, List qualifier) { this(id, table, qualifier, Optional.empty(), Optional.empty(), table.getPartitionIds(), false, ImmutableList.of(), -1, false, PreAggStatus.unset(), ImmutableList.of(), ImmutableList.of(), - Maps.newHashMap(), Optional.empty(), false, ImmutableMap.of()); + Maps.newHashMap(), Optional.empty(), false, ImmutableMap.of(), + ImmutableList.of()); } public LogicalOlapScan(RelationId id, OlapTable table, List qualifier, List tabletIds, @@ -140,7 +150,7 @@ public LogicalOlapScan(RelationId id, OlapTable table, List qualifier, L this(id, table, qualifier, Optional.empty(), Optional.empty(), table.getPartitionIds(), false, tabletIds, -1, false, PreAggStatus.unset(), ImmutableList.of(), hints, Maps.newHashMap(), - tableSample, false, ImmutableMap.of()); + tableSample, false, ImmutableMap.of(), ImmutableList.of()); } public LogicalOlapScan(RelationId id, OlapTable table, List qualifier, List specifiedPartitions, @@ -149,7 +159,7 @@ public LogicalOlapScan(RelationId id, OlapTable table, List qualifier, L // must use specifiedPartitions here for prune partition by sql like 'select * from t partition p1' specifiedPartitions, false, tabletIds, -1, false, PreAggStatus.unset(), specifiedPartitions, hints, Maps.newHashMap(), - tableSample, false, ImmutableMap.of()); + tableSample, false, ImmutableMap.of(), ImmutableList.of()); } public LogicalOlapScan(RelationId id, OlapTable table, List qualifier, List tabletIds, @@ -158,7 +168,8 @@ public LogicalOlapScan(RelationId id, OlapTable table, List qualifier, L this(id, table, qualifier, Optional.empty(), Optional.empty(), selectedPartitionIds, false, tabletIds, selectedIndexId, true, preAggStatus, - specifiedPartitions, hints, Maps.newHashMap(), tableSample, true, ImmutableMap.of()); + specifiedPartitions, hints, Maps.newHashMap(), tableSample, true, ImmutableMap.of(), + ImmutableList.of()); } /** @@ -171,7 +182,7 @@ public LogicalOlapScan(RelationId id, Table table, List qualifier, PreAggStatus preAggStatus, List specifiedPartitions, List hints, Map, Slot> cacheSlotWithSlotName, Optional tableSample, boolean directMvScan, - Map>> colToSubPathsMap) { + Map>> colToSubPathsMap, List specifiedTabletIds) { super(id, PlanType.LOGICAL_OLAP_SCAN, table, qualifier, groupExpression, logicalProperties); Preconditions.checkArgument(selectedPartitionIds != null, @@ -182,6 +193,7 @@ public LogicalOlapScan(RelationId id, Table table, List qualifier, this.indexSelected = indexSelected; this.preAggStatus = preAggStatus; this.manuallySpecifiedPartitions = ImmutableList.copyOf(specifiedPartitions); + this.manuallySpecifiedTabletIds = ImmutableList.copyOf(specifiedTabletIds); if (selectedPartitionIds.isEmpty()) { this.selectedPartitionIds = ImmutableList.of(); @@ -240,6 +252,7 @@ public boolean equals(Object o) { && partitionPruned == that.partitionPruned && Objects.equals(preAggStatus, that.preAggStatus) && Objects.equals(selectedTabletIds, that.selectedTabletIds) && Objects.equals(manuallySpecifiedPartitions, that.manuallySpecifiedPartitions) + && Objects.equals(manuallySpecifiedTabletIds, that.manuallySpecifiedTabletIds) && Objects.equals(selectedPartitionIds, that.selectedPartitionIds) && Objects.equals(hints, that.hints) && Objects.equals(tableSample, that.tableSample); @@ -248,8 +261,8 @@ public boolean equals(Object o) { @Override public int hashCode() { return Objects.hash(super.hashCode(), selectedIndexId, indexSelected, preAggStatus, cacheSlotWithSlotName, - selectedTabletIds, partitionPruned, manuallySpecifiedPartitions, selectedPartitionIds, hints, - tableSample); + selectedTabletIds, partitionPruned, manuallySpecifiedTabletIds, manuallySpecifiedPartitions, + selectedPartitionIds, hints, tableSample); } @Override @@ -258,7 +271,7 @@ public LogicalOlapScan withGroupExpression(Optional groupExpres groupExpression, Optional.of(getLogicalProperties()), selectedPartitionIds, partitionPruned, selectedTabletIds, selectedIndexId, indexSelected, preAggStatus, manuallySpecifiedPartitions, - hints, cacheSlotWithSlotName, tableSample, directMvScan, colToSubPathsMap); + hints, cacheSlotWithSlotName, tableSample, directMvScan, colToSubPathsMap, manuallySpecifiedTabletIds); } @Override @@ -267,7 +280,7 @@ public Plan withGroupExprLogicalPropChildren(Optional groupExpr return new LogicalOlapScan(relationId, (Table) table, qualifier, groupExpression, logicalProperties, selectedPartitionIds, partitionPruned, selectedTabletIds, selectedIndexId, indexSelected, preAggStatus, manuallySpecifiedPartitions, - hints, cacheSlotWithSlotName, tableSample, directMvScan, colToSubPathsMap); + hints, cacheSlotWithSlotName, tableSample, directMvScan, colToSubPathsMap, manuallySpecifiedTabletIds); } public LogicalOlapScan withSelectedPartitionIds(List selectedPartitionIds) { @@ -275,7 +288,7 @@ public LogicalOlapScan withSelectedPartitionIds(List selectedPartitionIds) Optional.empty(), Optional.of(getLogicalProperties()), selectedPartitionIds, true, selectedTabletIds, selectedIndexId, indexSelected, preAggStatus, manuallySpecifiedPartitions, - hints, cacheSlotWithSlotName, tableSample, directMvScan, colToSubPathsMap); + hints, cacheSlotWithSlotName, tableSample, directMvScan, colToSubPathsMap, manuallySpecifiedTabletIds); } public LogicalOlapScan withMaterializedIndexSelected(long indexId) { @@ -283,7 +296,7 @@ public LogicalOlapScan withMaterializedIndexSelected(long indexId) { Optional.empty(), Optional.of(getLogicalProperties()), selectedPartitionIds, partitionPruned, selectedTabletIds, indexId, true, PreAggStatus.unset(), manuallySpecifiedPartitions, hints, cacheSlotWithSlotName, - tableSample, directMvScan, colToSubPathsMap); + tableSample, directMvScan, colToSubPathsMap, manuallySpecifiedTabletIds); } public LogicalOlapScan withSelectedTabletIds(List selectedTabletIds) { @@ -291,7 +304,7 @@ public LogicalOlapScan withSelectedTabletIds(List selectedTabletIds) { Optional.empty(), Optional.of(getLogicalProperties()), selectedPartitionIds, partitionPruned, selectedTabletIds, selectedIndexId, indexSelected, preAggStatus, manuallySpecifiedPartitions, - hints, cacheSlotWithSlotName, tableSample, directMvScan, colToSubPathsMap); + hints, cacheSlotWithSlotName, tableSample, directMvScan, colToSubPathsMap, manuallySpecifiedTabletIds); } public LogicalOlapScan withPreAggStatus(PreAggStatus preAggStatus) { @@ -299,7 +312,7 @@ public LogicalOlapScan withPreAggStatus(PreAggStatus preAggStatus) { Optional.empty(), Optional.of(getLogicalProperties()), selectedPartitionIds, partitionPruned, selectedTabletIds, selectedIndexId, indexSelected, preAggStatus, manuallySpecifiedPartitions, - hints, cacheSlotWithSlotName, tableSample, directMvScan, colToSubPathsMap); + hints, cacheSlotWithSlotName, tableSample, directMvScan, colToSubPathsMap, manuallySpecifiedTabletIds); } public LogicalOlapScan withColToSubPathsMap(Map>> colToSubPathsMap) { @@ -307,7 +320,15 @@ public LogicalOlapScan withColToSubPathsMap(Map>> colTo Optional.empty(), Optional.empty(), selectedPartitionIds, partitionPruned, selectedTabletIds, selectedIndexId, indexSelected, preAggStatus, manuallySpecifiedPartitions, - hints, cacheSlotWithSlotName, tableSample, directMvScan, colToSubPathsMap); + hints, cacheSlotWithSlotName, tableSample, directMvScan, colToSubPathsMap, manuallySpecifiedTabletIds); + } + + public LogicalOlapScan withManuallySpecifiedTabletIds(List manuallySpecifiedTabletIds) { + return new LogicalOlapScan(relationId, (Table) table, qualifier, + Optional.empty(), Optional.of(getLogicalProperties()), + selectedPartitionIds, partitionPruned, selectedTabletIds, + selectedIndexId, indexSelected, preAggStatus, manuallySpecifiedPartitions, + hints, cacheSlotWithSlotName, tableSample, directMvScan, colToSubPathsMap, manuallySpecifiedTabletIds); } @Override @@ -317,7 +338,7 @@ public LogicalOlapScan withRelationId(RelationId relationId) { Optional.empty(), Optional.empty(), selectedPartitionIds, false, selectedTabletIds, selectedIndexId, indexSelected, preAggStatus, manuallySpecifiedPartitions, - hints, Maps.newHashMap(), tableSample, directMvScan, colToSubPathsMap); + hints, Maps.newHashMap(), tableSample, directMvScan, colToSubPathsMap, selectedTabletIds); } @Override @@ -333,6 +354,10 @@ public List getSelectedTabletIds() { return selectedTabletIds; } + public List getManuallySpecifiedTabletIds() { + return manuallySpecifiedTabletIds; + } + @Override public long getSelectedIndexId() { return selectedIndexId; diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PruneOlapScanTabletTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PruneOlapScanTabletTest.java index e7940cdfb9b519..385657cd3f229e 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PruneOlapScanTabletTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PruneOlapScanTabletTest.java @@ -28,6 +28,7 @@ import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.Partition; import org.apache.doris.catalog.PrimitiveType; +import org.apache.doris.nereids.sqltest.SqlTestBase; import org.apache.doris.nereids.trees.expressions.EqualTo; import org.apache.doris.nereids.trees.expressions.GreaterThanEqual; import org.apache.doris.nereids.trees.expressions.InPredicate; @@ -51,8 +52,9 @@ import org.junit.jupiter.api.Test; import java.util.List; +import java.util.Objects; -class PruneOlapScanTabletTest implements MemoPatternMatchSupported { +class PruneOlapScanTabletTest extends SqlTestBase implements MemoPatternMatchSupported { @Test void testPruneOlapScanTablet(@Mocked OlapTable olapTable, @@ -154,4 +156,21 @@ void testPruneOlapScanTablet(@Mocked OlapTable olapTable, ) ); } + + @Test + void testPruneOlapScanTabletWithManually() { + String sql = "select * from T4 TABLET(110) where id > 8"; + PlanChecker.from(connectContext) + .analyze(sql) + .applyTopDown(new PruneOlapScanTablet()) + .matches( + logicalFilter( + logicalOlapScan().when(s -> + Objects.equals(s.getSelectedTabletIds(), Lists.newArrayList(110L)) + && Objects.equals(s.getManuallySpecifiedTabletIds(), + Lists.newArrayList(110L)) + ) + ) + ); + } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/sqltest/SqlTestBase.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/sqltest/SqlTestBase.java index f5b301dcc49b25..9ebbc22a85c826 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/sqltest/SqlTestBase.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/sqltest/SqlTestBase.java @@ -87,6 +87,9 @@ protected void runBeforeAll() throws Exception { + " score bigint\n" + ")\n" + "DUPLICATE KEY(id)\n" + + "AUTO PARTITION BY LIST(`id`)\n" + + "(\n" + + ")\n" + "DISTRIBUTED BY HASH(id) BUCKETS 1\n" + "PROPERTIES (\n" + " \"replication_num\" = \"1\"\n" diff --git a/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy b/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy index 62e6032578fd6a..744edfe54cbc24 100644 --- a/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy +++ b/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy @@ -1876,6 +1876,22 @@ class Suite implements GroovyInterceptable { sql "analyze table ${db}.${mv_name} with sync;" } + def create_async_partition_mv = { db, mv_name, mv_sql, partition_col -> + + sql """DROP MATERIALIZED VIEW IF EXISTS ${db}.${mv_name}""" + sql""" + CREATE MATERIALIZED VIEW ${db}.${mv_name} + BUILD IMMEDIATE REFRESH COMPLETE ON MANUAL + PARTITION BY ${partition_col} + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS ${mv_sql} + """ + def job_name = getJobName(db, mv_name); + waitingMTMVTaskFinished(job_name) + sql "analyze table ${db}.${mv_name} with sync;" + } + // mv not part in rewrite process void mv_not_part_in(query_sql, mv_name, sync_cbo_rewrite = enable_sync_mv_cost_based_rewrite()) { logger.info("query_sql = " + query_sql + ", mv_names = " + mv_name + ", sync_cbo_rewrite = " + sync_cbo_rewrite) diff --git a/regression-test/suites/nereids_rules_p0/mv/is_in_debug_mode/is_in_debug_mode.groovy b/regression-test/suites/nereids_rules_p0/mv/is_in_debug_mode/is_in_debug_mode.groovy new file mode 100644 index 00000000000000..15d93e32f65dc2 --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/mv/is_in_debug_mode/is_in_debug_mode.groovy @@ -0,0 +1,156 @@ +// 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. + +import org.junit.Assert + +suite("is_in_debug_mode") { + String db = context.config.getDbNameByFile(context.file) + sql "use ${db}" + sql "set runtime_filter_mode=OFF"; + sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'" + + sql """ + drop table if exists orders; + """ + + sql """ + CREATE TABLE IF NOT EXISTS orders ( + o_orderkey INTEGER NOT NULL, + o_custkey INTEGER NOT NULL, + o_orderstatus CHAR(1) NOT NULL, + o_totalprice DECIMALV3(15,2) NOT NULL, + o_orderdate DATE NOT NULL, + o_orderpriority CHAR(15) NOT NULL, + o_clerk CHAR(15) NOT NULL, + o_shippriority INTEGER NOT NULL, + O_COMMENT VARCHAR(79) NOT NULL + ) + UNIQUE KEY(o_orderkey, o_custkey) + DISTRIBUTED BY HASH(o_orderkey) BUCKETS 3 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + sql """ + insert into orders values + (1, 1, 'o', 9.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (1, 1, 'o', 10.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (1, 1, 'o', 10.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (1, 1, 'o', 10.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (2, 1, 'o', 11.5, '2023-12-09', 'a', 'b', 1, 'yy'), + (2, 1, 'o', 11.5, '2023-12-09', 'a', 'b', 1, 'yy'), + (2, 1, 'o', 11.5, '2023-12-09', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 12.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 12.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 12.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 33.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm'), + (4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm'), + (4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm'), + (5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi'), + (5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi'), + (5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi'), + (5, 2, 'o', 1.2, '2023-12-12', 'c','d',2, 'mi'); + """ + + create_async_mv(db, "basic_mv", """ + select * from orders where o_orderkey > 1; + """) + + sql """set skip_delete_sign = true;""" + mv_not_part_in("""select * from orders where o_orderkey > 1;""", "basic_mv") + try { + sql """ + CREATE MATERIALIZED VIEW test_create_mv + BUILD IMMEDIATE REFRESH COMPLETE ON MANUAL + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS select * from orders where o_orderkey > 2; + """ + } catch (Exception e) { + Assert.assertTrue(e.getMessage().contains("because is in debug mode")) + } + sql """set skip_delete_sign = false;""" + + + sql """set skip_storage_engine_merge = true;""" + mv_not_part_in("""select * from orders where o_orderkey > 1;""", "basic_mv") + try { + sql """ + CREATE MATERIALIZED VIEW test_create_mv + BUILD IMMEDIATE REFRESH COMPLETE ON MANUAL + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS select * from orders where o_orderkey > 2; + """ + } catch (Exception e) { + Assert.assertTrue(e.getMessage().contains("because is in debug mode")) + } + sql """set skip_storage_engine_merge = false;""" + + + sql """set skip_delete_bitmap = true;""" + mv_not_part_in("""select * from orders where o_orderkey > 1;""", "basic_mv") + try { + sql """ + CREATE MATERIALIZED VIEW test_create_mv + BUILD IMMEDIATE REFRESH COMPLETE ON MANUAL + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS select * from orders where o_orderkey > 2; + """ + } catch (Exception e) { + Assert.assertTrue(e.getMessage().contains("because is in debug mode")) + } + sql """set skip_delete_bitmap = false;""" + + + sql """set skip_delete_predicate = true;""" + mv_not_part_in("""select * from orders where o_orderkey > 1;""", "basic_mv") + try { + sql """ + CREATE MATERIALIZED VIEW test_create_mv + BUILD IMMEDIATE REFRESH COMPLETE ON MANUAL + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS select * from orders where o_orderkey > 2; + """ + } catch (Exception e) { + Assert.assertTrue(e.getMessage().contains("because is in debug mode")) + } + sql """set skip_delete_predicate = false;""" + + + sql """set show_hidden_columns = true;""" + mv_not_part_in("""select * from orders where o_orderkey > 1;""", "basic_mv") + try { + sql """ + CREATE MATERIALIZED VIEW test_create_mv + BUILD IMMEDIATE REFRESH COMPLETE ON MANUAL + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS select * from orders where o_orderkey > 2; + """ + } catch (Exception e) { + Assert.assertTrue(e.getMessage().contains("because is in debug mode")) + } + sql """set show_hidden_columns = false;""" + + sql """drop materialized view if exists basic_mv""" +} + diff --git a/regression-test/suites/nereids_rules_p0/mv/with_table_operator/with_table_operator.groovy b/regression-test/suites/nereids_rules_p0/mv/with_table_operator/with_table_operator.groovy new file mode 100644 index 00000000000000..5ba2c05053e2a2 --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/mv/with_table_operator/with_table_operator.groovy @@ -0,0 +1,195 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("with_table_operator") { + String db = context.config.getDbNameByFile(context.file) + sql "use ${db}" + sql "set runtime_filter_mode=OFF"; + sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'" + + sql """ + drop table if exists orders; + """ + + sql """ + CREATE TABLE IF NOT EXISTS orders ( + o_orderkey INTEGER NOT NULL, + o_custkey INTEGER NOT NULL, + o_orderstatus CHAR(1) NOT NULL, + o_totalprice DECIMALV3(15,2) NOT NULL, + o_orderdate DATE NOT NULL, + o_orderpriority CHAR(15) NOT NULL, + o_clerk CHAR(15) NOT NULL, + o_shippriority INTEGER NOT NULL, + O_COMMENT VARCHAR(79) NOT NULL + ) + DUPLICATE KEY(o_orderkey, o_custkey) + DISTRIBUTED BY HASH(o_orderkey) BUCKETS 3 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + sql """ + insert into orders values + (1, 1, 'o', 9.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (1, 1, 'o', 10.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (1, 1, 'o', 10.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (1, 1, 'o', 10.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (2, 1, 'o', 11.5, '2023-12-09', 'a', 'b', 1, 'yy'), + (2, 1, 'o', 11.5, '2023-12-09', 'a', 'b', 1, 'yy'), + (2, 1, 'o', 11.5, '2023-12-09', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 12.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 12.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 12.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 33.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm'), + (4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm'), + (4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm'), + (5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi'), + (5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi'), + (5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi'), + (5, 2, 'o', 1.2, '2023-12-12', 'c','d',2, 'mi'); + """ + + sql """ + drop table if exists orders_partition; + """ + + sql """ + CREATE TABLE IF NOT EXISTS orders_partition ( + o_orderkey INTEGER NOT NULL, + o_custkey INTEGER NOT NULL, + o_orderstatus CHAR(1) NOT NULL, + o_totalprice DECIMALV3(15,2) NOT NULL, + o_orderdate DATE NOT NULL, + o_orderpriority CHAR(15) NOT NULL, + o_clerk CHAR(15) NOT NULL, + o_shippriority INTEGER NOT NULL, + O_COMMENT VARCHAR(79) NOT NULL + ) + DUPLICATE KEY(o_orderkey, o_custkey) + PARTITION BY RANGE(o_orderdate) ( + PARTITION `day_2` VALUES LESS THAN ('2023-12-9'), + PARTITION `day_3` VALUES LESS THAN ("2023-12-11"), + PARTITION `day_4` VALUES LESS THAN ("2023-12-30") + ) + DISTRIBUTED BY HASH(o_orderkey) BUCKETS 3 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + sql """ + insert into orders_partition values + (1, 1, 'o', 9.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (1, 1, 'o', 10.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (1, 1, 'o', 10.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (1, 1, 'o', 10.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (2, 1, 'o', 11.5, '2023-12-09', 'a', 'b', 1, 'yy'), + (2, 1, 'o', 11.5, '2023-12-09', 'a', 'b', 1, 'yy'), + (2, 1, 'o', 11.5, '2023-12-09', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 12.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 12.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 12.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 33.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm'), + (4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm'), + (4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm'), + (5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi'), + (5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi'), + (5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi'), + (5, 2, 'o', 1.2, '2023-12-12', 'c','d',2, 'mi'); + """ + + create_async_mv(db, "complete_agg_mv", """ + select count(*),o_orderdate + from orders + group by o_orderdate; + """) + + create_async_mv(db, "complete_mv", """ + select * from orders; + """) + + create_async_partition_mv(db, "partition_agg_mv", """ + select count(*),o_orderdate + from orders_partition + group by o_orderdate; + """, "(o_orderdate)") + + create_async_partition_mv(db, "partition_mv", """ + select * from orders_partition; + """, "(o_orderdate)") + + // query rewrite with tablet should fail + mv_rewrite_all_fail(""" + select count(*),o_orderdate + from orders TABLET(110) + group by o_orderdate; + """, ["complete_mv", "complete_agg_mv"]) + + mv_rewrite_all_fail("""select * from orders TABLET(110);""", ["complete_mv", "complete_agg_mv"]) + + // query rewrite with index should fail + createMV(""" + create materialized view query_index_test + as + select o_orderdate, count(*) + from orders + group by o_orderdate; + """) + mv_rewrite_all_fail(""" + select * from orders index query_index_test; + """, ["complete_mv", "complete_agg_mv"]) + + // query rewrite with sample should fail + mv_rewrite_all_fail(""" + select count(*),o_orderdate + from orders TABLESAMPLE(20 percent) + group by o_orderdate; + """, ["complete_mv", "complete_agg_mv"]) + + mv_rewrite_all_fail("""select * from orders TABLESAMPLE(20 percent);""", ["complete_mv", "complete_agg_mv"]) + + // query rewrite with partition should fail + mv_rewrite_all_fail(""" + select count(*),o_orderdate + from orders_partition PARTITION (day_2) + group by o_orderdate; + """, ["partition_mv", "partition_agg_mv"]) + + mv_rewrite_all_fail("""select * from orders_partition PARTITION (day_2);""", ["partition_mv", "partition_agg_mv"]) + + + sql """ + DROP MATERIALIZED VIEW IF EXISTS complete_agg_mv; + """ + sql """ + DROP MATERIALIZED VIEW IF EXISTS partition_agg_mv; + """ + sql """ + DROP MATERIALIZED VIEW IF EXISTS complete_mv; + """ + sql """ + DROP MATERIALIZED VIEW IF EXISTS partition_mv; + """ + sql """ + DROP MATERIALIZED VIEW IF EXISTS query_index_test on orders; + """ +} + From ce9c7174fb5ea04781d042d72b69a677793bb965 Mon Sep 17 00:00:00 2001 From: zhangdong Date: Mon, 2 Dec 2024 19:45:27 +0800 Subject: [PATCH 118/399] [fix](auth) Prohibit deleting admin user (#44751) --- .../apache/doris/analysis/DropUserStmt.java | 4 +- .../apache/doris/analysis/UserIdentity.java | 4 ++ .../trees/plans/commands/DropUserCommand.java | 4 +- .../suites/account_p0/test_system_user.groovy | 39 +++++++++++++++++++ 4 files changed, 47 insertions(+), 4 deletions(-) create mode 100644 regression-test/suites/account_p0/test_system_user.groovy diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/DropUserStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/DropUserStmt.java index f9bb218744fc85..5f9872e42ad7cc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/DropUserStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/DropUserStmt.java @@ -63,8 +63,8 @@ public void analyze(Analyzer analyzer) throws AnalysisException, UserException { userIdent.analyze(); - if (userIdent.isRootUser()) { - ErrorReport.reportAnalysisException(ErrorCode.ERR_COMMON_ERROR, "Can not drop root user"); + if (userIdent.isSystemUser()) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_COMMON_ERROR, "Can not drop system user"); } // only user with GLOBAL level's GRANT_PRIV can drop user. diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/UserIdentity.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/UserIdentity.java index 122f28ca45092b..6898915dab73ce 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/UserIdentity.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/UserIdentity.java @@ -189,6 +189,10 @@ public boolean isAdminUser() { return user.equals(Auth.ADMIN_USER); } + public boolean isSystemUser() { + return isRootUser() || isAdminUser(); + } + public TUserIdentity toThrift() { Preconditions.checkState(isAnalyzed); TUserIdentity tUserIdent = new TUserIdentity(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropUserCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropUserCommand.java index 094ce06c36143e..5c4fb407e40213 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropUserCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropUserCommand.java @@ -55,8 +55,8 @@ public void doRun(ConnectContext ctx, StmtExecutor executor) throws Exception { userIdent.analyze(); - if (userIdent.isRootUser()) { - ErrorReport.reportAnalysisException(ErrorCode.ERR_COMMON_ERROR, "Can not drop root user"); + if (userIdent.isSystemUser()) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_COMMON_ERROR, "Can not drop system user"); } // only user with GLOBAL level's GRANT_PRIV can drop user. diff --git a/regression-test/suites/account_p0/test_system_user.groovy b/regression-test/suites/account_p0/test_system_user.groovy new file mode 100644 index 00000000000000..1805f1669ea570 --- /dev/null +++ b/regression-test/suites/account_p0/test_system_user.groovy @@ -0,0 +1,39 @@ +// 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. + +import org.junit.Assert; + +suite("test_system_user") { + test { + sql """ + create user `root`; + """ + exception "root" + } + test { + sql """ + drop user `root`; + """ + exception "system" + } + test { + sql """ + drop user `admin`; + """ + exception "system" + } +} From 70b0a08815b1ea4e39a8e39f8f067ea27c795abb Mon Sep 17 00:00:00 2001 From: feiniaofeiafei Date: Mon, 2 Dec 2024 20:04:14 +0800 Subject: [PATCH 119/399] [fix](nereids) fix merge_percentile_to_array when has same agg function (#44783) Related PR: #34313 Problem Summary The original PR did not handle the following scenario: ```sql SELECT SUM(a), PERCENTILE(pk, 0.1) AS c1, PERCENTILE(pk, 0.1) AS c2, PERCENTILE(pk, 0.4) AS c3 FROM test_merge_percentile; ``` In this case, the aggregate outputs include two identical functions (PERCENTILE(pk, 0.1)). When constructing the LogicalProject, a map was used where the key is the child of an Alias and the value is the Alias itself. However, this approach loses information when two Aliases share the same child. This PR modifies the map structure to use the child of an Alias as the key and a list of Alias objects as the value. This ensures that all Alias instances with the same child are preserved, resolving the issue of lost information in such cases. --- .../rules/rewrite/MergePercentileToArray.java | 26 +++++++++---------- .../merge_percentile_to_array.out | 12 +++++++++ .../merge_percentile_to_array.groovy | 4 +++ 3 files changed, 28 insertions(+), 14 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/MergePercentileToArray.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/MergePercentileToArray.java index f92ad84bde8525..fe81adf13bf29d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/MergePercentileToArray.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/MergePercentileToArray.java @@ -152,10 +152,10 @@ private Plan doMerge(LogicalAggregate aggregate) { (List) (List) newPercentileArrays); ImmutableList.Builder newProjectOutputExpressions = ImmutableList.builder(); newProjectOutputExpressions.addAll((List) (List) notChangeForProject); - Map existsAliasMap = Maps.newHashMap(); + Map> existsAliasMap = Maps.newHashMap(); // existsAliasMap is used to keep upper plan refer the same expr for (Alias alias : existsAliases) { - existsAliasMap.put(alias.child(), alias); + existsAliasMap.computeIfAbsent(alias.child(), k -> new ArrayList<>()).add(alias); } Map slotMap = Maps.newHashMap(); // slotMap is used to find the correspondence @@ -169,20 +169,22 @@ private Plan doMerge(LogicalAggregate aggregate) { for (Map.Entry> entry : funcMap.entrySet()) { for (int i = 0; i < entry.getValue().size(); i++) { AggregateFunction aggFunc = entry.getValue().get(i); - Alias originAlias = existsAliasMap.get(aggFunc); - DistinctAndExpr distinctAndExpr = new DistinctAndExpr(aggFunc.child(0), aggFunc.isDistinct()); - Alias newAlias = new Alias(originAlias.getExprId(), new ElementAt(slotMap.get(distinctAndExpr), - new IntegerLiteral(i + 1)), originAlias.getName()); - newProjectOutputExpressions.add(newAlias); + List originAliases = existsAliasMap.get(aggFunc); + for (Alias originAlias : originAliases) { + DistinctAndExpr distinctAndExpr = new DistinctAndExpr(aggFunc.child(0), aggFunc.isDistinct()); + Alias newAlias = new Alias(originAlias.getExprId(), new ElementAt(slotMap.get(distinctAndExpr), + new IntegerLiteral(i + 1)), originAlias.getName()); + newProjectOutputExpressions.add(newAlias); + } } } newProjectOutputExpressions.addAll(groupBySlots); - return new LogicalProject(newProjectOutputExpressions.build(), newAggregate); + return new LogicalProject<>(newProjectOutputExpressions.build(), newAggregate); } private static class DistinctAndExpr { - private Expression expression; - private boolean isDistinct; + private final Expression expression; + private final boolean isDistinct; public DistinctAndExpr(Expression expression, boolean isDistinct) { this.expression = expression; @@ -193,10 +195,6 @@ public Expression getExpression() { return expression; } - public boolean isDistinct() { - return isDistinct; - } - @Override public boolean equals(Object o) { if (this == o) { diff --git a/regression-test/data/nereids_rules_p0/merge_percentile_to_array/merge_percentile_to_array.out b/regression-test/data/nereids_rules_p0/merge_percentile_to_array/merge_percentile_to_array.out index b495302e80d3c8..1b2f876cfba50a 100644 --- a/regression-test/data/nereids_rules_p0/merge_percentile_to_array/merge_percentile_to_array.out +++ b/regression-test/data/nereids_rules_p0/merge_percentile_to_array/merge_percentile_to_array.out @@ -41,3 +41,15 @@ 7.0 \N \N 7.0 7.0 7 +-- !same_percentile -- +52 1.0 1.0 2.0 + +-- !same_percentile_group_by -- +\N 6.0 6.0 6.0 +2 3.0 3.0 3.0 +25 3.0 3.0 3.0 +4 2.0 2.0 2.0 +5 1.0 1.0 1.6 +7 6.0 6.0 6.0 +9 1.2 1.2 1.8 + diff --git a/regression-test/suites/nereids_rules_p0/merge_percentile_to_array/merge_percentile_to_array.groovy b/regression-test/suites/nereids_rules_p0/merge_percentile_to_array/merge_percentile_to_array.groovy index 2071d75ae85d4e..5bb13c6336c264 100644 --- a/regression-test/suites/nereids_rules_p0/merge_percentile_to_array/merge_percentile_to_array.groovy +++ b/regression-test/suites/nereids_rules_p0/merge_percentile_to_array/merge_percentile_to_array.groovy @@ -57,4 +57,8 @@ suite("merge_percentile_to_array") { percentile(abs(a), 0.55) as c2 from test_merge_percentile group by a) t; """ + order_qt_same_percentile """select sum(a),percentile(pk, 0.1) as c1 , percentile(pk, 0.1) as c2 , + percentile(pk, 0.4) as c2 from test_merge_percentile;""" + order_qt_same_percentile_group_by """select sum(a),percentile(pk, 0.1) as c1 , percentile(pk, 0.1) as c2 , + percentile(pk, 0.4) as c2 from test_merge_percentile group by a;""" } \ No newline at end of file From 21e1d6d8a7ccb08848e7be61ae574c7ec4b27e69 Mon Sep 17 00:00:00 2001 From: Mingxi <71588583+joker-star-l@users.noreply.github.com> Date: Mon, 2 Dec 2024 20:27:10 +0800 Subject: [PATCH 120/399] [fix](logstash) remove ShortNameResolver to solve thread race problem (#44598) remove ShortNameResolver to solve thread race problem --- .../logstash/lib/logstash/outputs/doris.rb | 50 +--------------- .../lib/logstash/util/shortname_resolver.rb | 58 ------------------- .../logstash/logstash-output-doris.gemspec | 1 - 3 files changed, 2 insertions(+), 107 deletions(-) delete mode 100644 extension/logstash/lib/logstash/util/shortname_resolver.rb diff --git a/extension/logstash/lib/logstash/outputs/doris.rb b/extension/logstash/lib/logstash/outputs/doris.rb index 02e7591b0a354b..21d3ee6e752b08 100644 --- a/extension/logstash/lib/logstash/outputs/doris.rb +++ b/extension/logstash/lib/logstash/outputs/doris.rb @@ -21,7 +21,6 @@ require "logstash/outputs/base" require "logstash/namespace" require "logstash/json" -require "logstash/util/shortname_resolver" require 'logstash/util/formater' require "uri" require "securerandom" @@ -67,8 +66,6 @@ class LogStash::Outputs::Doris < LogStash::Outputs::Base config :save_file, :validate => :string, :default => "failed.data" - config :host_resolve_ttl_sec, :validate => :number, :default => 120 - config :max_retries, :validate => :number, :default => -1 config :log_request, :validate => :boolean, :default => true @@ -92,10 +89,6 @@ def print_plugin_info() def register @http_query = "/api/#{@db}/#{@table}/_stream_load" - @hostnames_pool = - parse_http_hosts(@http_hosts, - ShortNameResolver.new(ttl: @host_resolve_ttl_sec, logger: @logger)) - @request_headers = make_request_headers @logger.info("request headers: ", @request_headers) @@ -141,39 +134,6 @@ def register print_plugin_info() end # def register - private - - def parse_http_hosts(hosts, resolver) - ip_re = /^[\d]+\.[\d]+\.[\d]+\.[\d]+$/ - - lambda { - hosts.flat_map { |h| - scheme = URI(h).scheme - host = URI(h).host - port = URI(h).port - path = URI(h).path - - if ip_re !~ host - resolver.get_addresses(host).map { |ip| - "#{scheme}://#{ip}:#{port}#{path}" - } - else - [h] - end - } - } - end - - private - - def get_host_addresses() - begin - @hostnames_pool.call - rescue Exception => ex - @logger.error('Error while resolving host', :error => ex.to_s) - end - end - def multi_receive(events) return if events.empty? send_events(events) @@ -191,8 +151,6 @@ def send_events(events) # @logger.info("get event num: #{event_num}") @logger.debug("get documents: #{documents}") - hosts = get_host_addresses() - http_headers = @request_headers.dup if !@group_commit # only set label if group_commit is off_mode or not set, since lable can not be used with group_commit @@ -202,7 +160,7 @@ def send_events(events) req_count = 0 sleep_for = 1 while true - response = make_request(documents, http_headers, hosts, @http_query, hosts.sample) + response = make_request(documents, http_headers, @http_query, @http_hosts.sample) req_count += 1 response_json = {} @@ -246,11 +204,7 @@ def send_events(events) end private - def make_request(documents, http_headers, hosts, query, host = "") - if host == "" - host = hosts.pop - end - + def make_request(documents, http_headers, query, host) url = host + query if @log_request or @logger.debug? diff --git a/extension/logstash/lib/logstash/util/shortname_resolver.rb b/extension/logstash/lib/logstash/util/shortname_resolver.rb deleted file mode 100644 index 1437ccba007988..00000000000000 --- a/extension/logstash/lib/logstash/util/shortname_resolver.rb +++ /dev/null @@ -1,58 +0,0 @@ -=begin -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. -=end -require 'resolv' -require 'mini_cache' - -class ShortNameResolver - def initialize(ttl:, logger:) - @ttl = ttl - @store = MiniCache::Store.new - @logger = logger - end - - private - def resolve_cached(shortname) - @store.get_or_set(shortname) do - addresses = resolve(shortname) - raise "Bad shortname '#{shortname}'" if addresses.empty? - MiniCache::Data.new(addresses, expires_in: @ttl) - end - end - - private - def resolve(shortname) - addresses = Resolv::DNS.open do |dns| - dns.getaddresses(shortname).map { |r| r.to_s } - end - - @logger.info("Resolved shortname '#{shortname}' to addresses #{addresses}") - - return addresses - end - - public - def get_address(shortname) - return resolve_cached(shortname).sample - end - - public - def get_addresses(shortname) - return resolve_cached(shortname) - end -end diff --git a/extension/logstash/logstash-output-doris.gemspec b/extension/logstash/logstash-output-doris.gemspec index 689b93503f6a8b..30341b83156ed4 100644 --- a/extension/logstash/logstash-output-doris.gemspec +++ b/extension/logstash/logstash-output-doris.gemspec @@ -38,7 +38,6 @@ Gem::Specification.new do |s| # Gem dependencies s.add_runtime_dependency "logstash-core-plugin-api", ">= 1.60", "<= 2.99" - s.add_runtime_dependency 'mini_cache', ">= 1.0.0", "< 2.0.0" s.add_runtime_dependency "rest-client", '~> 2.1' s.add_development_dependency 'logstash-devutils', '~> 1.3' From 303e27597e3f6c2ee82033ab90593ae8d2599628 Mon Sep 17 00:00:00 2001 From: bobhan1 Date: Mon, 2 Dec 2024 21:42:18 +0800 Subject: [PATCH 121/399] [Fix](TPartitionVersionInfo) Fix duplicate `TPartitionVersionInfo` in `PublishVersionTask.partitionVersionInfos` (#44846) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What problem does this PR solve? Problem Summary: When FE handles BEs' tablet report and finds that there exists some expired txns on BE, it will generate publish version task. `TPartitionVersionInfo` with same values may be added to `transactionsToPublish` under same txn id many times when partitions involved in this failed txn involves many tablets on this BE. Because it uses `ArrayListMultimap`, these duplicate values may occupy a lot of memories when the number of tablets is large. ### Release note Fixed the issue of FE's memory occupation growing too fast in cases of persistent load and clone failures on merge-on-write tables. 修复了在merge-on-write表上有持续的导入失败和clone失败的情况下,FE使用内存增长过快的问题。 --- .../apache/doris/catalog/TabletInvertedIndex.java | 13 +++++++------ .../java/org/apache/doris/master/ReportHandler.java | 9 +++++---- 2 files changed, 12 insertions(+), 10 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/TabletInvertedIndex.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/TabletInvertedIndex.java index 4a421dc7b2b1ef..a51d1f55014fcc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/TabletInvertedIndex.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/TabletInvertedIndex.java @@ -37,13 +37,14 @@ import org.apache.doris.transaction.TransactionStatus; import com.google.common.base.Preconditions; -import com.google.common.collect.ArrayListMultimap; import com.google.common.collect.HashBasedTable; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.LinkedHashMultimap; import com.google.common.collect.ListMultimap; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Ordering; +import com.google.common.collect.SetMultimap; import com.google.common.collect.Sets; import com.google.common.collect.Table; import com.google.common.collect.TreeMultimap; @@ -135,7 +136,7 @@ public void tabletReport(long backendId, Map backendTablets, Set tabletFoundInMeta, ListMultimap tabletMigrationMap, Map partitionVersionSyncMap, - Map> transactionsToPublish, + Map> transactionsToPublish, ListMultimap transactionsToClear, ListMultimap tabletRecoveryMap, List tabletToUpdate, @@ -314,7 +315,7 @@ && isLocal(tabletMeta.getStorageMedium())) { } private void handleBackendTransactions(long backendId, List transactionIds, long tabletId, - TabletMeta tabletMeta, Map> transactionsToPublish, + TabletMeta tabletMeta, Map> transactionsToPublish, ListMultimap transactionsToClear) { GlobalTransactionMgrIface transactionMgr = Env.getCurrentGlobalTransactionMgr(); long partitionId = tabletMeta.getPartitionId(); @@ -376,15 +377,15 @@ private TPartitionVersionInfo generatePartitionVersionInfoWhenReport(Transaction } private void publishPartition(TransactionState transactionState, long transactionId, TabletMeta tabletMeta, - long partitionId, Map> transactionsToPublish) { + long partitionId, Map> transactionsToPublish) { TPartitionVersionInfo versionInfo = generatePartitionVersionInfoWhenReport(transactionState, transactionId, tabletMeta, partitionId); if (versionInfo != null) { synchronized (transactionsToPublish) { - ListMultimap map = transactionsToPublish.get( + SetMultimap map = transactionsToPublish.get( transactionState.getDbId()); if (map == null) { - map = ArrayListMultimap.create(); + map = LinkedHashMultimap.create(); transactionsToPublish.put(transactionState.getDbId(), map); } map.put(transactionId, versionInfo); diff --git a/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java b/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java index c5c72eae3c5a8b..06047e2cf16682 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java @@ -95,6 +95,7 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Queues; +import com.google.common.collect.SetMultimap; import com.google.common.collect.Sets; import org.apache.commons.collections.CollectionUtils; import org.apache.commons.lang3.StringUtils; @@ -503,7 +504,7 @@ public void tabletReport(long backendId, Map backendTablets, Map partitionVersionSyncMap = Maps.newConcurrentMap(); // dbid -> txn id -> [partition info] - Map> transactionsToPublish = Maps.newHashMap(); + Map> transactionsToPublish = Maps.newHashMap(); ListMultimap transactionsToClear = LinkedListMultimap.create(); // db id -> tablet id @@ -1148,14 +1149,14 @@ private static void handleMigration(ListMultimap tabletMet } private static void handleRepublishVersionInfo( - Map> transactionsToPublish, long backendId) { + Map> transactionsToPublish, long backendId) { AgentBatchTask batchTask = new AgentBatchTask(); long createPublishVersionTaskTime = System.currentTimeMillis(); for (Long dbId : transactionsToPublish.keySet()) { - ListMultimap map = transactionsToPublish.get(dbId); + SetMultimap map = transactionsToPublish.get(dbId); for (long txnId : map.keySet()) { PublishVersionTask task = new PublishVersionTask(backendId, txnId, dbId, - map.get(txnId), createPublishVersionTaskTime); + Lists.newArrayList(map.get(txnId)), createPublishVersionTaskTime); batchTask.addTask(task); // add to AgentTaskQueue for handling finish report. AgentTaskQueue.addTask(task); From ee3f09d00d70daa8689a1d855f9dc0e433e658f5 Mon Sep 17 00:00:00 2001 From: bobhan1 Date: Tue, 3 Dec 2024 10:49:41 +0800 Subject: [PATCH 122/399] [opt](ms) print segment key bounds size when prepare/commit/update rowset meta (#44715) --- cloud/src/meta-service/meta_service.cpp | 32 ++++++++++++++++++-- cloud/src/meta-service/meta_service_helper.h | 1 + 2 files changed, 30 insertions(+), 3 deletions(-) diff --git a/cloud/src/meta-service/meta_service.cpp b/cloud/src/meta-service/meta_service.cpp index ddf4ee3889c00e..9da5750d8d83f9 100644 --- a/cloud/src/meta-service/meta_service.cpp +++ b/cloud/src/meta-service/meta_service.cpp @@ -1033,14 +1033,18 @@ void MetaServiceImpl::prepare_rowset(::google::protobuf::RpcController* controll prepare_rowset.SerializeToString(&val); DCHECK_GT(prepare_rowset.expiration(), 0); txn->put(prepare_rs_key, val); + std::size_t segment_key_bounds_bytes = get_segments_key_bounds_bytes(rowset_meta); LOG(INFO) << "put prepare_rs_key " << hex(prepare_rs_key) << " value_size " << val.size() - << " txn_id " << request->txn_id(); + << " txn_id " << request->txn_id() << " segment_key_bounds_bytes " + << segment_key_bounds_bytes; err = txn->commit(); if (err != TxnErrorCode::TXN_OK) { if (err == TxnErrorCode::TXN_VALUE_TOO_LARGE) { LOG(WARNING) << "failed to prepare rowset, err=value too large" << ", txn_id=" << request->txn_id() << ", tablet_id=" << tablet_id << ", rowset_id=" << rowset_id + << ", rowset_meta_bytes=" << rowset_meta.ByteSizeLong() + << ", segment_key_bounds_bytes=" << segment_key_bounds_bytes << ", rowset_meta=" << rowset_meta.ShortDebugString(); } code = cast_as(err); @@ -1167,15 +1171,18 @@ void MetaServiceImpl::commit_rowset(::google::protobuf::RpcController* controlle DCHECK_GT(rowset_meta.txn_expiration(), 0); auto tmp_rs_val = rowset_meta.SerializeAsString(); txn->put(tmp_rs_key, tmp_rs_val); + std::size_t segment_key_bounds_bytes = get_segments_key_bounds_bytes(rowset_meta); LOG(INFO) << "put tmp_rs_key " << hex(tmp_rs_key) << " delete recycle_rs_key " << hex(recycle_rs_key) << " value_size " << tmp_rs_val.size() << " txn_id " - << request->txn_id(); + << request->txn_id() << " segment_key_bounds_bytes " << segment_key_bounds_bytes; err = txn->commit(); if (err != TxnErrorCode::TXN_OK) { if (err == TxnErrorCode::TXN_VALUE_TOO_LARGE) { LOG(WARNING) << "failed to commit rowset, err=value too large" << ", txn_id=" << request->txn_id() << ", tablet_id=" << tablet_id << ", rowset_id=" << rowset_id + << ", rowset_meta_bytes=" << rowset_meta.ByteSizeLong() + << ", segment_key_bounds_bytes=" << segment_key_bounds_bytes << ", rowset_meta=" << rowset_meta.ShortDebugString(); } code = cast_as(err); @@ -1267,10 +1274,21 @@ void MetaServiceImpl::update_tmp_rowset(::google::protobuf::RpcController* contr } txn->put(update_key, update_val); + std::size_t segment_key_bounds_bytes = get_segments_key_bounds_bytes(rowset_meta); LOG(INFO) << "xxx put " - << "update_rowset_key " << hex(update_key) << " value_size " << update_val.size(); + << "update_rowset_key " << hex(update_key) << " value_size " << update_val.size() + << " segment_key_bounds_bytes " << segment_key_bounds_bytes; err = txn->commit(); if (err != TxnErrorCode::TXN_OK) { + if (err == TxnErrorCode::TXN_VALUE_TOO_LARGE) { + const auto& rowset_id = rowset_meta.rowset_id_v2(); + LOG(WARNING) << "failed to update tmp rowset, err=value too large" + << ", txn_id=" << request->txn_id() << ", tablet_id=" << tablet_id + << ", rowset_id=" << rowset_id + << ", rowset_meta_bytes=" << rowset_meta.ByteSizeLong() + << ", segment_key_bounds_bytes=" << segment_key_bounds_bytes + << ", rowset_meta=" << rowset_meta.ShortDebugString(); + } code = cast_as(err); ss << "failed to update rowset meta, err=" << err; msg = ss.str(); @@ -2372,4 +2390,12 @@ MetaServiceResponseStatus MetaServiceImpl::fix_tablet_stats(std::string cloud_un return st; } +std::size_t get_segments_key_bounds_bytes(const doris::RowsetMetaCloudPB& rowset_meta) { + size_t ret {0}; + for (const auto& key_bounds : rowset_meta.segments_key_bounds()) { + ret += key_bounds.ByteSizeLong(); + } + return ret; +} + } // namespace doris::cloud diff --git a/cloud/src/meta-service/meta_service_helper.h b/cloud/src/meta-service/meta_service_helper.h index 4ef98ea2fb7b0c..5355ac0eb61ed2 100644 --- a/cloud/src/meta-service/meta_service_helper.h +++ b/cloud/src/meta-service/meta_service_helper.h @@ -248,4 +248,5 @@ void get_tablet_idx(MetaServiceCode& code, std::string& msg, Transaction* txn, bool is_dropped_tablet(Transaction* txn, const std::string& instance_id, int64_t index_id, int64_t partition_id); +std::size_t get_segments_key_bounds_bytes(const doris::RowsetMetaCloudPB& rowset_meta); } // namespace doris::cloud From c382dcca72134288b8e785d0c5d8715ce55a0837 Mon Sep 17 00:00:00 2001 From: walter Date: Tue, 3 Dec 2024 10:50:25 +0800 Subject: [PATCH 123/399] [fix](catalog) Fix infinity partition key toSql (#44834) --- .../main/java/org/apache/doris/catalog/PartitionKey.java | 3 +-- .../java/org/apache/doris/catalog/PartitionKeyTest.java | 6 ++++++ 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/PartitionKey.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/PartitionKey.java index f9640ccae14023..29bfda8b201d20 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/PartitionKey.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/PartitionKey.java @@ -373,15 +373,14 @@ public String toSql() { if (expr == MaxLiteral.MAX_VALUE || expr.isNullLiteral()) { value = expr.toSql(); sb.append(value); - continue; } else { value = "\"" + expr.getRealValue() + "\""; if (expr instanceof DateLiteral) { DateLiteral dateLiteral = (DateLiteral) expr; value = dateLiteral.toSql(); } + sb.append(value); } - sb.append(value); if (keys.size() - 1 != i) { sb.append(", "); diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/PartitionKeyTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/PartitionKeyTest.java index affd5a4cc2c397..c6b45c0b136e61 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/catalog/PartitionKeyTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/PartitionKeyTest.java @@ -268,4 +268,10 @@ public void testSerialization() throws Exception { dis.close(); Files.deleteIfExists(path); } + + @Test + public void testMaxValueToSql() throws Exception { + PartitionKey key = PartitionKey.createInfinityPartitionKey(allColumns, true); + Assert.assertEquals("(MAXVALUE, MAXVALUE, MAXVALUE, MAXVALUE, MAXVALUE, MAXVALUE, MAXVALUE)", key.toSql()); + } } From b0f2c17fee2e6b5de0213db5c7b4f5a8b6e740b0 Mon Sep 17 00:00:00 2001 From: meiyi Date: Tue, 3 Dec 2024 11:25:39 +0800 Subject: [PATCH 124/399] [fix](cluster key) fix some case (#44881) --- .../data/compaction/test_full_compaction.out | 4 + .../compaction/test_full_compaction_ck.out | 47 +++++ .../test_generated_column_nereids.out | 12 +- .../org/apache/doris/regression/Config.groovy | 10 +- .../compaction/test_full_compaction.groovy | 5 + .../compaction/test_full_compaction_ck.groovy | 189 ++++++++++++++++++ .../test_generated_column_nereids.groovy | 22 +- 7 files changed, 273 insertions(+), 16 deletions(-) create mode 100644 regression-test/data/compaction/test_full_compaction_ck.out create mode 100644 regression-test/suites/compaction/test_full_compaction_ck.groovy diff --git a/regression-test/data/compaction/test_full_compaction.out b/regression-test/data/compaction/test_full_compaction.out index b25fdad93145cc..aaab47b4c3678b 100644 --- a/regression-test/data/compaction/test_full_compaction.out +++ b/regression-test/data/compaction/test_full_compaction.out @@ -41,3 +41,7 @@ 2 200 3 0 +-- !select_final2 -- +1 100 +2 200 + diff --git a/regression-test/data/compaction/test_full_compaction_ck.out b/regression-test/data/compaction/test_full_compaction_ck.out new file mode 100644 index 00000000000000..f76314e79a7579 --- /dev/null +++ b/regression-test/data/compaction/test_full_compaction_ck.out @@ -0,0 +1,47 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !1 -- +1 1 +2 2 + +-- !2 -- +1 10 +2 20 + +-- !3 -- +1 100 +2 200 + +-- !4 -- +1 100 +2 200 +3 300 + +-- !5 -- +1 100 +2 200 +3 100 + +-- !6 -- +1 100 +2 200 + +-- !skip_delete -- +1 1 +1 10 +1 100 +2 2 +2 20 +2 200 +3 100 +3 100 +3 300 + +-- !select_final -- +1 100 +2 200 +3 100 + +-- !select_final2 -- +1 100 +2 200 + diff --git a/regression-test/data/ddl_p0/test_create_table_generated_column/test_generated_column_nereids.out b/regression-test/data/ddl_p0/test_create_table_generated_column/test_generated_column_nereids.out index b7757222ab4991..24f2f19c5769a5 100644 --- a/regression-test/data/ddl_p0/test_create_table_generated_column/test_generated_column_nereids.out +++ b/regression-test/data/ddl_p0/test_create_table_generated_column/test_generated_column_nereids.out @@ -170,12 +170,6 @@ c double No false \N NONE,STORED GENERATED b int Yes false \N NONE d int Yes false \N NONE,STORED GENERATED --- !test_update -- -1 - --- !test_update_generated_column -- -1 20 21 - -- !gen_col_unique_key -- 0 @@ -221,3 +215,9 @@ d int Yes false \N NONE,STORED GENERATED -- !agg_replace_null -- 1 2 3 4 13 +-- !test_update -- +1 + +-- !test_update_generated_column -- +1 20 21 + diff --git a/regression-test/framework/src/main/groovy/org/apache/doris/regression/Config.groovy b/regression-test/framework/src/main/groovy/org/apache/doris/regression/Config.groovy index 379d177fbc280d..2f632f5c9252a7 100644 --- a/regression-test/framework/src/main/groovy/org/apache/doris/regression/Config.groovy +++ b/regression-test/framework/src/main/groovy/org/apache/doris/regression/Config.groovy @@ -1028,7 +1028,15 @@ class Config { excludeDirectorySet.add("fault_injection_p0/partial_update") excludeDirectorySet.add("fault_injection_p0/flexible") excludeDirectorySet.add("doc") - List excludeCases = ["test_table_properties", "test_default_hll", "test_default_pi", "test_full_compaction", "test_full_compaction_by_table_id", "test_create_table", "txn_insert", "test_update_mow", "test_new_update", "test_update_unique", "test_partial_update_generated_column", "nereids_partial_update_native_insert_stmt", "partial_update", "nereids_update_on_current_timestamp", "update_on_current_timestamp", "test_default_bitmap_empty", "nereids_delete_mow_partial_update", "delete_mow_partial_update", "partial_update_seq_col", "nereids_partial_update_native_insert_stmt_complex", "regression_test_variant_delete_and_update", "test_unique_table_auto_inc_partial_update_correct_stream_load", "test_unique_table_auto_inc", "test_unique_table_auto_inc_partial_update_correct_insert", "test_update_schema_change", "test_partial_update_rowset_not_found_fault_injection"] + excludeDirectorySet.add("schema_change_p0/unique_ck") + List excludeCases = ["test_table_properties", "test_create_table" + , "test_default_hll", "test_default_pi", "test_default_bitmap_empty" + // partial update + , "txn_insert", "test_update_schema_change", "test_generated_column_update", "test_nested_type_with_rowstore", "test_partial_update_generated_column", "nereids_partial_update_native_insert_stmt" + , "partial_update", "nereids_update_on_current_timestamp", "update_on_current_timestamp", "nereids_delete_mow_partial_update", "delete_mow_partial_update", "test_unique_table_auto_inc" + , "test_unique_table_auto_inc_partial_update_correct_insert", "partial_update_seq_col", "nereids_partial_update_native_insert_stmt_complex", "regression_test_variant_delete_and_update" + , "test_unique_table_auto_inc_partial_update_correct_stream_load", "test_update_mow", "test_new_update", "test_update_unique", "nereids_partial_update_native_insert_seq_col" + , "test_partial_update_rowset_not_found_fault_injection"] for (def excludeCase in excludeCases) { excludeSuiteWildcard.add(excludeCase) } diff --git a/regression-test/suites/compaction/test_full_compaction.groovy b/regression-test/suites/compaction/test_full_compaction.groovy index 60f52f6f5a55a0..ed2d2ce5bc5e46 100644 --- a/regression-test/suites/compaction/test_full_compaction.groovy +++ b/regression-test/suites/compaction/test_full_compaction.groovy @@ -177,6 +177,11 @@ suite("test_full_compaction") { // make sure all hidden data has been deleted // (1,100)(2,200) qt_select_final """select * from ${tableName} order by user_id""" + + sql "SET skip_delete_predicate = false" + sql "SET skip_delete_sign = false" + sql "SET skip_delete_bitmap = false" + qt_select_final2 """select * from ${tableName} order by user_id""" } finally { try_sql("DROP TABLE IF EXISTS ${tableName}") } diff --git a/regression-test/suites/compaction/test_full_compaction_ck.groovy b/regression-test/suites/compaction/test_full_compaction_ck.groovy new file mode 100644 index 00000000000000..8e2f86c596de65 --- /dev/null +++ b/regression-test/suites/compaction/test_full_compaction_ck.groovy @@ -0,0 +1,189 @@ +// 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. + +import org.codehaus.groovy.runtime.IOGroovyMethods + +suite("test_full_compaction_ck") { + def tableName = "test_full_compaction_ck" + + try { + String backend_id; + + def backendId_to_backendIP = [:] + def backendId_to_backendHttpPort = [:] + getBackendIpHttpPort(backendId_to_backendIP, backendId_to_backendHttpPort); + + backend_id = backendId_to_backendIP.keySet()[0] + def (code, out, err) = show_be_config(backendId_to_backendIP.get(backend_id), backendId_to_backendHttpPort.get(backend_id)) + logger.info("Show config: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def configList = parseJson(out.trim()) + assert configList instanceof List + + boolean disableAutoCompaction = true + for (Object ele in (List) configList) { + assert ele instanceof List + if (((List) ele)[0] == "disable_auto_compaction") { + disableAutoCompaction = Boolean.parseBoolean(((List) ele)[2]) + } + } + + sql """ DROP TABLE IF EXISTS ${tableName} """ + sql """ + CREATE TABLE ${tableName} ( + `user_id` INT NOT NULL, `value` INT NOT NULL) + UNIQUE KEY(`user_id`) + CLUSTER BY(`value`) + DISTRIBUTED BY HASH(`user_id`) + BUCKETS 1 + PROPERTIES ("replication_allocation" = "tag.location.default: 1", + "disable_auto_compaction" = "true", + "enable_mow_light_delete" = "false", + "enable_unique_key_merge_on_write" = "true");""" + + // version1 (1,1)(2,2) + sql """ INSERT INTO ${tableName} VALUES + (1,1),(2,2) + """ + qt_1 """select * from ${tableName} order by user_id""" + + + // version2 (1,10)(2,20) + sql """ INSERT INTO ${tableName} VALUES + (1,10),(2,20) + """ + qt_2 """select * from ${tableName} order by user_id""" + + + // version3 (1,100)(2,200) + sql """ INSERT INTO ${tableName} VALUES + (1,100),(2,200) + """ + qt_3 """select * from ${tableName} order by user_id""" + + + // version4 (1,100)(2,200)(3,300) + sql """ INSERT INTO ${tableName} VALUES + (3,300) + """ + qt_4 """select * from ${tableName} order by user_id""" + + + // version5 (1,100)(2,200)(3,100) + sql """update ${tableName} set value = 100 where user_id = 3""" + qt_5 """select * from ${tableName} order by user_id""" + + + // version6 (1,100)(2,200) + sql """delete from ${tableName} where user_id = 3""" + qt_6 """select * from ${tableName} order by user_id""" + + sql "SET skip_delete_predicate = true" + sql "SET skip_delete_sign = true" + sql "SET skip_delete_bitmap = true" + // show all hidden data + // (1,10)(1,100)(2,2)(2,20)(2,200)(3,300)(3,100) + qt_skip_delete """select * from ${tableName} order by user_id, value""" + + //TabletId,ReplicaId,BackendId,SchemaHash,Version,LstSuccessVersion,LstFailedVersion,LstFailedTime,LocalDataSize,RemoteDataSize,RowCount,State,LstConsistencyCheckTime,CheckVersion,VersionCount,PathHash,MetaUrl,CompactionStatus + def tablets = sql_return_maparray """ show tablets from ${tableName}; """ + + def replicaNum = get_table_replica_num(tableName) + logger.info("get table replica num: " + replicaNum) + // before full compaction, there are 7 rowsets. + int rowsetCount = 0 + for (def tablet in tablets) { + String tablet_id = tablet.TabletId + (code, out, err) = curl("GET", tablet.CompactionStatus) + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + rowsetCount +=((List) tabletJson.rowsets).size() + } + assert (rowsetCount == 7 * replicaNum) + + // trigger full compactions for all tablets in ${tableName} + for (def tablet in tablets) { + String tablet_id = tablet.TabletId + backend_id = tablet.BackendId + times = 1 + + do{ + (code, out, err) = be_run_full_compaction(backendId_to_backendIP.get(backend_id), backendId_to_backendHttpPort.get(backend_id), tablet_id) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + ++times + sleep(2000) + } while (parseJson(out.trim()).status.toLowerCase()!="success" && times<=10) + + def compactJson = parseJson(out.trim()) + if (compactJson.status.toLowerCase() == "fail") { + assertEquals(disableAutoCompaction, false) + logger.info("Compaction was done automatically!") + } + if (disableAutoCompaction) { + assertEquals("success", compactJson.status.toLowerCase()) + } + } + + // wait for full compaction done + for (def tablet in tablets) { + boolean running = true + do { + Thread.sleep(1000) + String tablet_id = tablet.TabletId + backend_id = tablet.BackendId + (code, out, err) = be_get_compaction_status(backendId_to_backendIP.get(backend_id), backendId_to_backendHttpPort.get(backend_id), tablet_id) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } while (running) + } + + // after full compaction, there is only 1 rowset. + + rowsetCount = 0 + for (def tablet in tablets) { + String tablet_id = tablet.TabletId + (code, out, err) = curl("GET", tablet.CompactionStatus) + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + rowsetCount +=((List) tabletJson.rowsets).size() + } + def cloudMode = isCloudMode() + if (cloudMode) { + assert (rowsetCount == 2) + } else { + assert (rowsetCount == 1 * replicaNum) + } + + // make sure all hidden data has been deleted + // (1,100)(2,200) + qt_select_final """select * from ${tableName} order by user_id""" + + sql "SET skip_delete_predicate = false" + sql "SET skip_delete_sign = false" + sql "SET skip_delete_bitmap = false" + qt_select_final2 """select * from ${tableName} order by user_id""" + } finally { + // try_sql("DROP TABLE IF EXISTS ${tableName}") + } +} diff --git a/regression-test/suites/ddl_p0/test_create_table_generated_column/test_generated_column_nereids.groovy b/regression-test/suites/ddl_p0/test_create_table_generated_column/test_generated_column_nereids.groovy index 36fbf9d80aa663..b22b7d82ae34c8 100644 --- a/regression-test/suites/ddl_p0/test_create_table_generated_column/test_generated_column_nereids.groovy +++ b/regression-test/suites/ddl_p0/test_create_table_generated_column/test_generated_column_nereids.groovy @@ -134,15 +134,6 @@ suite("test_generated_column") { qt_describe "describe gencol_refer_gencol" - //test update - sql "drop table if exists test_gen_col_update" - sql """create table test_gen_col_update (a int, b int, c int as (a+b)) - unique key(a) - distributed by hash(a) properties("replication_num"="1")""" - sql "insert into test_gen_col_update values(1,3,default)" - qt_test_update "update test_gen_col_update set b=20" - qt_test_update_generated_column "select * from test_gen_col_update" - // test unique table, generated column is not key sql "drop table if exists test_gen_col_unique_key" qt_gen_col_unique_key """create table test_gen_col_unique_key(a int,b int,c int generated always as (abs(a+b)) not null) @@ -233,4 +224,17 @@ suite("test_generated_column") { PROPERTIES("replication_num" = "1");""" exception "The generated columns can be key columns, or value columns of replace and replace_if_not_null aggregation type." } + + //test update + sql "drop table if exists test_gen_col_update" + sql """create table test_gen_col_update (a int, b int, c int as (a+b)) + unique key(a) + distributed by hash(a) properties("replication_num"="1")""" + sql "insert into test_gen_col_update values(1,3,default)" + if (!isClusterKeyEnabled()) { + qt_test_update "update test_gen_col_update set b=20" + qt_test_update_generated_column "select * from test_gen_col_update" + } else { + // errCode = 2, detailMessage = The value specified for generated column 'c' in table 'test_gen_col_update' is not allowed + } } \ No newline at end of file From 829b4b79d178ad878fbc20f4057b77583ef26af7 Mon Sep 17 00:00:00 2001 From: HappenLee Date: Tue, 3 Dec 2024 11:31:14 +0800 Subject: [PATCH 125/399] [Refactor](query) refactor lock in fragment mgr and change std::unorder_map to phmap (#44821) --- be/src/runtime/fragment_mgr.cpp | 262 +++++++++++++++---------------- be/src/runtime/fragment_mgr.h | 29 ++-- be/src/runtime/load_channel.cpp | 3 +- be/src/runtime/load_stream.cpp | 2 +- be/src/runtime/runtime_state.cpp | 2 +- 5 files changed, 140 insertions(+), 158 deletions(-) diff --git a/be/src/runtime/fragment_mgr.cpp b/be/src/runtime/fragment_mgr.cpp index 1e72fa756d3dd3..f96e4152500808 100644 --- a/be/src/runtime/fragment_mgr.cpp +++ b/be/src/runtime/fragment_mgr.cpp @@ -269,8 +269,11 @@ void FragmentMgr::stop() { // Only me can delete { - std::lock_guard lock(_lock); + std::unique_lock lock(_query_ctx_map_mutex); _query_ctx_map.clear(); + } + { + std::unique_lock lock(_pipeline_map_mutex); _pipeline_map.clear(); } _thread_pool->shutdown(); @@ -583,11 +586,7 @@ Status FragmentMgr::start_query_execution(const PExecPlanFragmentStartRequest* r TUniqueId query_id; query_id.__set_hi(request->query_id().hi()); query_id.__set_lo(request->query_id().lo()); - std::shared_ptr q_ctx = nullptr; - { - std::lock_guard lock(_lock); - q_ctx = _get_or_erase_query_ctx(query_id); - } + auto q_ctx = get_query_ctx(query_id); if (q_ctx) { q_ctx->set_ready_to_execute(Status::OK()); LOG_INFO("Query {} start execution", print_id(query_id)); @@ -602,114 +601,107 @@ Status FragmentMgr::start_query_execution(const PExecPlanFragmentStartRequest* r void FragmentMgr::remove_pipeline_context( std::shared_ptr f_context) { - { - std::lock_guard lock(_lock); - auto query_id = f_context->get_query_id(); - int64 now = duration_cast( - std::chrono::system_clock::now().time_since_epoch()) - .count(); - g_fragment_executing_count << -1; - g_fragment_last_active_time.set_value(now); - _pipeline_map.erase({query_id, f_context->get_fragment_id()}); - } + auto query_id = f_context->get_query_id(); + int64 now = duration_cast( + std::chrono::system_clock::now().time_since_epoch()) + .count(); + g_fragment_executing_count << -1; + g_fragment_last_active_time.set_value(now); + + std::unique_lock lock(_pipeline_map_mutex); + _pipeline_map.erase({query_id, f_context->get_fragment_id()}); } -std::shared_ptr FragmentMgr::_get_or_erase_query_ctx(const TUniqueId& query_id) { +std::shared_ptr FragmentMgr::get_query_ctx(const TUniqueId& query_id) { + std::shared_lock lock(_query_ctx_map_mutex); auto search = _query_ctx_map.find(query_id); if (search != _query_ctx_map.end()) { if (auto q_ctx = search->second.lock()) { return q_ctx; - } else { - LOG(WARNING) << "Query context (query id = " << print_id(query_id) - << ") has been released."; - _query_ctx_map.erase(search); - return nullptr; } } return nullptr; } -std::shared_ptr FragmentMgr::get_or_erase_query_ctx_with_lock( - const TUniqueId& query_id) { - std::unique_lock lock(_lock); - return _get_or_erase_query_ctx(query_id); -} - -template -Status FragmentMgr::_get_query_ctx(const Params& params, TUniqueId query_id, bool pipeline, - QuerySource query_source, - std::shared_ptr& query_ctx) { +Status FragmentMgr::_get_or_create_query_ctx(const TPipelineFragmentParams& params, + TUniqueId query_id, bool pipeline, + QuerySource query_source, + std::shared_ptr& query_ctx) { DBUG_EXECUTE_IF("FragmentMgr._get_query_ctx.failed", { return Status::InternalError("FragmentMgr._get_query_ctx.failed, query id {}", print_id(query_id)); }); + + // Find _query_ctx_map, in case some other request has already + // create the query fragments context. + query_ctx = get_query_ctx(query_id); if (params.is_simplified_param) { // Get common components from _query_ctx_map - std::lock_guard lock(_lock); - if (auto q_ctx = _get_or_erase_query_ctx(query_id)) { - query_ctx = q_ctx; - } else { + if (!query_ctx) { return Status::InternalError( "Failed to get query fragments context. Query {} may be timeout or be " "cancelled. host: {}", print_id(query_id), BackendOptions::get_localhost()); } } else { - // Find _query_ctx_map, in case some other request has already - // create the query fragments context. - std::lock_guard lock(_lock); - if (auto q_ctx = _get_or_erase_query_ctx(query_id)) { - query_ctx = q_ctx; - return Status::OK(); - } + if (!query_ctx) { + std::unique_lock lock(_query_ctx_map_mutex); + // Only one thread need create query ctx. other thread just get query_ctx in _query_ctx_map. + auto search = _query_ctx_map.find(query_id); + if (search != _query_ctx_map.end()) { + query_ctx = search->second.lock(); + } - // First time a fragment of a query arrived. print logs. - LOG(INFO) << "query_id: " << print_id(query_id) << ", coord_addr: " << params.coord - << ", total fragment num on current host: " << params.fragment_num_on_host - << ", fe process uuid: " << params.query_options.fe_process_uuid - << ", query type: " << params.query_options.query_type - << ", report audit fe:" << params.current_connect_fe; - - // This may be a first fragment request of the query. - // Create the query fragments context. - query_ctx = QueryContext::create_shared(query_id, _exec_env, params.query_options, - params.coord, params.is_nereids, - params.current_connect_fe, query_source); - SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(query_ctx->query_mem_tracker); - RETURN_IF_ERROR(DescriptorTbl::create(&(query_ctx->obj_pool), params.desc_tbl, - &(query_ctx->desc_tbl))); - // set file scan range params - if (params.__isset.file_scan_params) { - query_ctx->file_scan_range_params_map = params.file_scan_params; - } + if (!query_ctx) { + // First time a fragment of a query arrived. print logs. + LOG(INFO) << "query_id: " << print_id(query_id) << ", coord_addr: " << params.coord + << ", total fragment num on current host: " << params.fragment_num_on_host + << ", fe process uuid: " << params.query_options.fe_process_uuid + << ", query type: " << params.query_options.query_type + << ", report audit fe:" << params.current_connect_fe; + + // This may be a first fragment request of the query. + // Create the query fragments context. + query_ctx = QueryContext::create_shared(query_id, _exec_env, params.query_options, + params.coord, params.is_nereids, + params.current_connect_fe, query_source); + SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(query_ctx->query_mem_tracker); + RETURN_IF_ERROR(DescriptorTbl::create(&(query_ctx->obj_pool), params.desc_tbl, + &(query_ctx->desc_tbl))); + // set file scan range params + if (params.__isset.file_scan_params) { + query_ctx->file_scan_range_params_map = params.file_scan_params; + } - query_ctx->query_globals = params.query_globals; + query_ctx->query_globals = params.query_globals; - if (params.__isset.resource_info) { - query_ctx->user = params.resource_info.user; - query_ctx->group = params.resource_info.group; - query_ctx->set_rsc_info = true; - } + if (params.__isset.resource_info) { + query_ctx->user = params.resource_info.user; + query_ctx->group = params.resource_info.group; + query_ctx->set_rsc_info = true; + } - _set_scan_concurrency(params, query_ctx.get()); - - if (params.__isset.workload_groups && !params.workload_groups.empty()) { - uint64_t tg_id = params.workload_groups[0].id; - WorkloadGroupPtr workload_group_ptr = - _exec_env->workload_group_mgr()->get_task_group_by_id(tg_id); - if (workload_group_ptr != nullptr) { - RETURN_IF_ERROR(workload_group_ptr->add_query(query_id, query_ctx)); - RETURN_IF_ERROR(query_ctx->set_workload_group(workload_group_ptr)); - _exec_env->runtime_query_statistics_mgr()->set_workload_group_id(print_id(query_id), - tg_id); - } else { - LOG(WARNING) << "Query/load id: " << print_id(query_ctx->query_id()) - << "can't find its workload group " << tg_id; + _set_scan_concurrency(params, query_ctx.get()); + + if (params.__isset.workload_groups && !params.workload_groups.empty()) { + uint64_t tg_id = params.workload_groups[0].id; + WorkloadGroupPtr workload_group_ptr = + _exec_env->workload_group_mgr()->get_task_group_by_id(tg_id); + if (workload_group_ptr != nullptr) { + RETURN_IF_ERROR(workload_group_ptr->add_query(query_id, query_ctx)); + RETURN_IF_ERROR(query_ctx->set_workload_group(workload_group_ptr)); + _exec_env->runtime_query_statistics_mgr()->set_workload_group_id( + print_id(query_id), tg_id); + } else { + LOG(WARNING) << "Query/load id: " << print_id(query_ctx->query_id()) + << "can't find its workload group " << tg_id; + } + } + // There is some logic in query ctx's dctor, we could not check if exists and delete the + // temp query ctx now. For example, the query id maybe removed from workload group's queryset. + _query_ctx_map.insert({query_id, query_ctx}); } } - // There is some logic in query ctx's dctor, we could not check if exists and delete the - // temp query ctx now. For example, the query id maybe removed from workload group's queryset. - _query_ctx_map.insert(std::make_pair(query_ctx->query_id(), query_ctx)); } return Status::OK(); } @@ -723,13 +715,13 @@ std::string FragmentMgr::dump_pipeline_tasks(int64_t duration) { fmt::memory_buffer debug_string_buffer; size_t i = 0; { - std::lock_guard lock(_lock); fmt::format_to(debug_string_buffer, "{} pipeline fragment contexts are still running! duration_limit={}\n", _pipeline_map.size(), duration); - timespec now; clock_gettime(CLOCK_MONOTONIC, &now); + + std::shared_lock lock(_pipeline_map_mutex); for (auto& it : _pipeline_map) { auto elapsed = it.second->elapsed_time() / 1000000000.0; if (elapsed < duration) { @@ -748,7 +740,7 @@ std::string FragmentMgr::dump_pipeline_tasks(int64_t duration) { } std::string FragmentMgr::dump_pipeline_tasks(TUniqueId& query_id) { - if (auto q_ctx = _get_or_erase_query_ctx(query_id)) { + if (auto q_ctx = get_query_ctx(query_id)) { return q_ctx->print_all_pipeline_context(); } else { return fmt::format( @@ -767,7 +759,8 @@ Status FragmentMgr::exec_plan_fragment(const TPipelineFragmentParams& params, << apache::thrift::ThriftDebugString(params.query_options).c_str(); std::shared_ptr query_ctx; - RETURN_IF_ERROR(_get_query_ctx(params, params.query_id, true, query_source, query_ctx)); + RETURN_IF_ERROR( + _get_or_create_query_ctx(params, params.query_id, true, query_source, query_ctx)); SCOPED_ATTACH_TASK(query_ctx.get()); int64_t duration_ns = 0; std::shared_ptr context = @@ -800,16 +793,8 @@ Status FragmentMgr::exec_plan_fragment(const TPipelineFragmentParams& params, } { - // (query_id, fragment_id) is executed only on one BE, locks _pipeline_map. - std::lock_guard lock(_lock); for (const auto& local_param : params.local_params) { const TUniqueId& fragment_instance_id = local_param.fragment_instance_id; - auto iter = _pipeline_map.find({params.query_id, params.fragment_id}); - if (iter != _pipeline_map.end()) { - return Status::InternalError( - "exec_plan_fragment query_id({}) input duplicated fragment_id({})", - print_id(params.query_id), params.fragment_id); - } query_ctx->fragment_instance_ids.push_back(fragment_instance_id); } @@ -818,7 +803,15 @@ Status FragmentMgr::exec_plan_fragment(const TPipelineFragmentParams& params, .count(); g_fragment_executing_count << 1; g_fragment_last_active_time.set_value(now); - // TODO: simplify this mapping + + // (query_id, fragment_id) is executed only on one BE, locks _pipeline_map. + std::unique_lock lock(_pipeline_map_mutex); + auto iter = _pipeline_map.find({params.query_id, params.fragment_id}); + if (iter != _pipeline_map.end()) { + return Status::InternalError( + "exec_plan_fragment query_id({}) input duplicated fragment_id({})", + print_id(params.query_id), params.fragment_id); + } _pipeline_map.insert({{params.query_id, params.fragment_id}, context}); } @@ -848,8 +841,7 @@ void FragmentMgr::cancel_query(const TUniqueId query_id, const Status reason) { std::shared_ptr query_ctx = nullptr; std::vector all_instance_ids; { - std::lock_guard state_lock(_lock); - if (auto q_ctx = _get_or_erase_query_ctx(query_id)) { + if (auto q_ctx = get_query_ctx(query_id)) { query_ctx = q_ctx; // Copy instanceids to avoid concurrent modification. // And to reduce the scope of lock. @@ -862,7 +854,7 @@ void FragmentMgr::cancel_query(const TUniqueId query_id, const Status reason) { } query_ctx->cancel(reason); { - std::lock_guard state_lock(_lock); + std::unique_lock l(_query_ctx_map_mutex); _query_ctx_map.erase(query_id); } LOG(INFO) << "Query " << print_id(query_id) @@ -898,7 +890,7 @@ void FragmentMgr::cancel_worker() { std::vector> ctx; { - std::lock_guard lock(_lock); + std::shared_lock lock(_pipeline_map_mutex); ctx.reserve(_pipeline_map.size()); for (auto& pipeline_itr : _pipeline_map) { ctx.push_back(pipeline_itr.second); @@ -910,29 +902,34 @@ void FragmentMgr::cancel_worker() { std::unordered_map, BrpcItem> brpc_stub_with_queries; { - std::lock_guard lock(_lock); - for (auto it = _query_ctx_map.begin(); it != _query_ctx_map.end();) { - if (auto q_ctx = it->second.lock()) { - if (q_ctx->is_timeout(now)) { - LOG_WARNING("Query {} is timeout", print_id(it->first)); - queries_timeout.push_back(it->first); - } else if (config::enable_brpc_connection_check) { - auto brpc_stubs = q_ctx->get_using_brpc_stubs(); - for (auto& item : brpc_stubs) { - if (!brpc_stub_with_queries.contains(item.second)) { - brpc_stub_with_queries.emplace(item.second, - BrpcItem {item.first, {q_ctx}}); - } else { - brpc_stub_with_queries[item.second].queries.emplace_back(q_ctx); + { + // TODO: Now only the cancel worker do the GC the _query_ctx_map. each query must + // do erase the finish query unless in _query_ctx_map. Rethink the logic is ok + std::unique_lock lock(_query_ctx_map_mutex); + for (auto it = _query_ctx_map.begin(); it != _query_ctx_map.end();) { + if (auto q_ctx = it->second.lock()) { + if (q_ctx->is_timeout(now)) { + LOG_WARNING("Query {} is timeout", print_id(it->first)); + queries_timeout.push_back(it->first); + } else if (config::enable_brpc_connection_check) { + auto brpc_stubs = q_ctx->get_using_brpc_stubs(); + for (auto& item : brpc_stubs) { + if (!brpc_stub_with_queries.contains(item.second)) { + brpc_stub_with_queries.emplace(item.second, + BrpcItem {item.first, {q_ctx}}); + } else { + brpc_stub_with_queries[item.second].queries.emplace_back(q_ctx); + } } } + ++it; + } else { + it = _query_ctx_map.erase(it); } - ++it; - } else { - it = _query_ctx_map.erase(it); } } + std::shared_lock lock(_query_ctx_map_mutex); // We use a very conservative cancel strategy. // 0. If there are no running frontends, do not cancel any queries. // 1. If query's process uuid is zero, do not cancel @@ -1215,7 +1212,7 @@ Status FragmentMgr::apply_filterv2(const PPublishFilterRequestV2* request, const auto& fragment_ids = request->fragment_ids(); { - std::unique_lock lock(_lock); + std::shared_lock lock(_pipeline_map_mutex); for (auto fragment_id : fragment_ids) { auto iter = _pipeline_map.find({UniqueId(request->query_id()).to_thrift(), fragment_id}); @@ -1267,8 +1264,7 @@ Status FragmentMgr::send_filter_size(const PSendFilterSizeRequest* request) { TUniqueId query_id; query_id.__set_hi(queryid.hi); query_id.__set_lo(queryid.lo); - std::lock_guard lock(_lock); - if (auto q_ctx = _get_or_erase_query_ctx(query_id)) { + if (auto q_ctx = get_query_ctx(query_id)) { query_ctx = q_ctx; } else { return Status::EndOfFile( @@ -1291,8 +1287,7 @@ Status FragmentMgr::sync_filter_size(const PSyncFilterSizeRequest* request) { TUniqueId query_id; query_id.__set_hi(queryid.hi); query_id.__set_lo(queryid.lo); - std::lock_guard lock(_lock); - if (auto q_ctx = _get_or_erase_query_ctx(query_id)) { + if (auto q_ctx = get_query_ctx(query_id)) { query_ctx = q_ctx; } else { return Status::EndOfFile( @@ -1312,8 +1307,7 @@ Status FragmentMgr::merge_filter(const PMergeFilterRequest* request, TUniqueId query_id; query_id.__set_hi(queryid.hi); query_id.__set_lo(queryid.lo); - std::lock_guard lock(_lock); - if (auto q_ctx = _get_or_erase_query_ctx(query_id)) { + if (auto q_ctx = get_query_ctx(query_id)) { query_ctx = q_ctx; } else { return Status::EndOfFile( @@ -1330,7 +1324,7 @@ Status FragmentMgr::merge_filter(const PMergeFilterRequest* request, void FragmentMgr::get_runtime_query_info(std::vector* query_info_list) { { - std::lock_guard lock(_lock); + std::unique_lock lock(_query_ctx_map_mutex); for (auto iter = _query_ctx_map.begin(); iter != _query_ctx_map.end();) { if (auto q_ctx = iter->second.lock()) { WorkloadQueryInfo workload_query_info; @@ -1353,19 +1347,9 @@ Status FragmentMgr::get_realtime_exec_status(const TUniqueId& query_id, return Status::InvalidArgument("exes_status is nullptr"); } - std::shared_ptr query_context = nullptr; - - { - std::lock_guard lock(_lock); - if (auto q_ctx = _get_or_erase_query_ctx(query_id)) { - query_context = q_ctx; - } else { - return Status::NotFound("Query {} has been released", print_id(query_id)); - } - } - + std::shared_ptr query_context = get_query_ctx(query_id); if (query_context == nullptr) { - return Status::NotFound("Query {} not found", print_id(query_id)); + return Status::NotFound("Query {} not found or released", print_id(query_id)); } *exec_status = query_context->get_realtime_exec_status(); diff --git a/be/src/runtime/fragment_mgr.h b/be/src/runtime/fragment_mgr.h index 0eac0469683961..63d666788d0a5f 100644 --- a/be/src/runtime/fragment_mgr.h +++ b/be/src/runtime/fragment_mgr.h @@ -133,7 +133,7 @@ class FragmentMgr : public RestMonitorIface { ThreadPool* get_thread_pool() { return _thread_pool.get(); } int32_t running_query_num() { - std::unique_lock ctx_lock(_lock); + std::shared_lock lock(_query_ctx_map_mutex); return _query_ctx_map.size(); } @@ -145,7 +145,7 @@ class FragmentMgr : public RestMonitorIface { Status get_realtime_exec_status(const TUniqueId& query_id, TReportExecStatusParams* exec_status); - std::shared_ptr get_or_erase_query_ctx_with_lock(const TUniqueId& query_id); + std::shared_ptr get_query_ctx(const TUniqueId& query_id); private: struct BrpcItem { @@ -153,14 +153,12 @@ class FragmentMgr : public RestMonitorIface { std::vector> queries; }; - std::shared_ptr _get_or_erase_query_ctx(const TUniqueId& query_id); - template void _set_scan_concurrency(const Param& params, QueryContext* query_ctx); - template - Status _get_query_ctx(const Params& params, TUniqueId query_id, bool pipeline, - QuerySource query_type, std::shared_ptr& query_ctx); + Status _get_or_create_query_ctx(const TPipelineFragmentParams& params, TUniqueId query_id, + bool pipeline, QuerySource query_type, + std::shared_ptr& query_ctx); void _check_brpc_available(const std::shared_ptr& brpc_stub, const BrpcItem& brpc_item); @@ -168,20 +166,21 @@ class FragmentMgr : public RestMonitorIface { // This is input params ExecEnv* _exec_env = nullptr; + // The lock protect the `_pipeline_map` + std::shared_mutex _pipeline_map_mutex; + // (QueryID, FragmentID) -> PipelineFragmentContext + phmap::flat_hash_map, + std::shared_ptr> + _pipeline_map; + // The lock should only be used to protect the structures in fragment manager. Has to be // used in a very small scope because it may dead lock. For example, if the _lock is used // in prepare stage, the call path is prepare --> expr prepare --> may call allocator // when allocate failed, allocator may call query_is_cancelled, query is callced will also // call _lock, so that there is dead lock. - std::mutex _lock; - - // (QueryID, FragmentID) -> PipelineFragmentContext - std::unordered_map, - std::shared_ptr> - _pipeline_map; - + std::shared_mutex _query_ctx_map_mutex; // query id -> QueryContext - std::unordered_map> _query_ctx_map; + phmap::flat_hash_map> _query_ctx_map; std::unordered_map> _bf_size_map; CountDownLatch _stop_background_threads_latch; diff --git a/be/src/runtime/load_channel.cpp b/be/src/runtime/load_channel.cpp index 0cb313747b0373..dd426f1ab81d3e 100644 --- a/be/src/runtime/load_channel.cpp +++ b/be/src/runtime/load_channel.cpp @@ -45,8 +45,7 @@ LoadChannel::LoadChannel(const UniqueId& load_id, int64_t timeout_s, bool is_hig _backend_id(backend_id), _enable_profile(enable_profile) { std::shared_ptr query_context = - ExecEnv::GetInstance()->fragment_mgr()->get_or_erase_query_ctx_with_lock( - _load_id.to_thrift()); + ExecEnv::GetInstance()->fragment_mgr()->get_query_ctx(_load_id.to_thrift()); std::shared_ptr mem_tracker = nullptr; WorkloadGroupPtr wg_ptr = nullptr; diff --git a/be/src/runtime/load_stream.cpp b/be/src/runtime/load_stream.cpp index 752e2ff95b2917..60da45fa685fbf 100644 --- a/be/src/runtime/load_stream.cpp +++ b/be/src/runtime/load_stream.cpp @@ -428,7 +428,7 @@ LoadStream::LoadStream(PUniqueId load_id, LoadStreamMgr* load_stream_mgr, bool e TUniqueId load_tid = ((UniqueId)load_id).to_thrift(); #ifndef BE_TEST std::shared_ptr query_context = - ExecEnv::GetInstance()->fragment_mgr()->get_or_erase_query_ctx_with_lock(load_tid); + ExecEnv::GetInstance()->fragment_mgr()->get_query_ctx(load_tid); if (query_context != nullptr) { _query_thread_context = {load_tid, query_context->query_mem_tracker, query_context->workload_group()}; diff --git a/be/src/runtime/runtime_state.cpp b/be/src/runtime/runtime_state.cpp index 344180bad771ac..072c6e87f740a6 100644 --- a/be/src/runtime/runtime_state.cpp +++ b/be/src/runtime/runtime_state.cpp @@ -295,7 +295,7 @@ Status RuntimeState::init(const TUniqueId& fragment_instance_id, const TQueryOpt } std::weak_ptr RuntimeState::get_query_ctx_weak() { - return _exec_env->fragment_mgr()->get_or_erase_query_ctx_with_lock(_query_ctx->query_id()); + return _exec_env->fragment_mgr()->get_query_ctx(_query_ctx->query_id()); } void RuntimeState::init_mem_trackers(const std::string& name, const TUniqueId& id) { From 3e3948c87fb285a5bf36fdf5d70bdebaeffc33ae Mon Sep 17 00:00:00 2001 From: Mryange Date: Tue, 3 Dec 2024 11:33:40 +0800 Subject: [PATCH 126/399] [env](compile)open compile check in some function agg file (#44549) ### What problem does this PR solve? This is part of the changes because modifying too much at once can easily cause conflicts with other PRs. --- .../aggregate_functions/aggregate_function.h | 3 +++ ...gregate_function_approx_count_distinct.cpp | 1 + ...aggregate_function_approx_count_distinct.h | 6 ++++-- .../aggregate_function_avg.cpp | 1 + .../aggregate_function_avg.h | 16 +++++++++++---- .../aggregate_function_avg_weighted.cpp | 1 + .../aggregate_function_avg_weighted.h | 5 ++++- .../aggregate_function_binary.h | 3 +++ .../aggregate_function_bit.cpp | 1 + .../aggregate_function_bit.h | 4 +++- .../aggregate_function_bitmap.cpp | 7 ++++++- .../aggregate_function_bitmap.h | 4 +++- .../aggregate_function_bitmap_agg.cpp | 7 ++++++- .../aggregate_function_bitmap_agg.h | 4 +++- .../aggregate_function_collect.cpp | 1 + .../aggregate_function_collect.h | 3 +++ .../aggregate_function_combinator.h | 3 +++ .../aggregate_function_corr.cpp | 1 + .../aggregate_function_count.cpp | 1 + .../aggregate_function_count.h | 3 +++ .../aggregate_function_count_by_enum.cpp | 1 + .../aggregate_function_count_by_enum.h | 7 ++++--- .../aggregate_function_covar.cpp | 1 + .../aggregate_function_covar.h | 9 +++++---- .../aggregate_function_distinct.cpp | 1 + .../aggregate_function_distinct.h | 3 +++ .../aggregate_function_foreach.cpp | 1 + .../aggregate_function_foreach.h | 3 +++ ...gregate_function_group_array_intersect.cpp | 1 + ...aggregate_function_group_array_intersect.h | 3 +++ .../aggregate_function_group_concat.cpp | 1 + .../aggregate_function_group_concat.h | 3 +++ .../aggregate_function_histogram.cpp | 1 + .../aggregate_function_histogram.h | 3 +++ .../aggregate_function_hll_union_agg.cpp | 1 + .../aggregate_function_hll_union_agg.h | 6 ++++-- .../aggregate_function_java_udaf.h | 18 +++++++++++------ .../aggregate_function_kurtosis.cpp | 1 + .../aggregate_function_linear_histogram.cpp | 1 + .../aggregate_function_linear_histogram.h | 6 +++++- .../aggregate_function_map.cpp | 1 + .../aggregate_function_map.h | 3 +++ .../aggregate_function_max_by.cpp | 1 + .../aggregate_function_min_by.cpp | 1 + .../aggregate_function_min_max.cpp | 1 + .../aggregate_function_min_max.h | 14 ++++++++----- .../aggregate_function_min_max_by.h | 3 +++ .../aggregate_function_null.h | 3 +++ .../aggregate_function_orthogonal_bitmap.cpp | 1 + .../aggregate_function_orthogonal_bitmap.h | 6 +++++- .../aggregate_function_product.h | 3 +++ .../aggregate_function_quantile_state.cpp | 1 + .../aggregate_function_quantile_state.h | 3 +++ .../aggregate_function_reader.cpp | 1 + .../aggregate_function_reader.h | 3 +++ .../aggregate_function_reader_first_last.h | 4 +++- .../aggregate_function_regr_union.cpp | 1 + .../aggregate_function_regr_union.h | 17 +++++++++------- .../aggregate_function_retention.cpp | 1 + .../aggregate_function_retention.h | 4 +++- .../aggregate_function_sequence_match.cpp | 1 + .../aggregate_function_sequence_match.h | 6 +++++- .../aggregate_function_simple_factory.cpp | 1 + .../aggregate_function_simple_factory.h | 3 +++ .../aggregate_function_skew.cpp | 1 + .../aggregate_function_sort.cpp | 1 + .../aggregate_function_sort.h | 3 +++ .../aggregate_function_state_merge.h | 3 +++ .../aggregate_function_state_union.h | 3 +++ .../aggregate_function_statistic.h | 16 ++++++++++----- .../aggregate_function_stddev.cpp | 1 + .../aggregate_function_stddev.h | 20 ++++++++++--------- .../aggregate_function_sum.cpp | 1 + .../aggregate_function_sum.h | 3 +++ .../aggregate_function_topn.cpp | 1 + .../aggregate_function_topn.h | 3 +++ .../aggregate_function_uniq.cpp | 1 + .../aggregate_function_uniq.h | 3 +++ ...aggregate_function_uniq_distribute_key.cpp | 1 + .../aggregate_function_uniq_distribute_key.h | 3 +++ .../aggregate_function_window.cpp | 1 + .../aggregate_function_window.h | 7 +++++-- .../aggregate_function_window_funnel.cpp | 1 + .../aggregate_function_window_funnel.h | 10 ++++++++-- .../vec/aggregate_functions/factory_helpers.h | 3 +++ be/src/vec/aggregate_functions/helpers.h | 3 +++ be/src/vec/aggregate_functions/moments.h | 4 +++- be/src/vec/common/string_buffer.hpp | 2 +- 88 files changed, 260 insertions(+), 64 deletions(-) diff --git a/be/src/vec/aggregate_functions/aggregate_function.h b/be/src/vec/aggregate_functions/aggregate_function.h index 32fc9d5efce771..e0ec2bef62fc2a 100644 --- a/be/src/vec/aggregate_functions/aggregate_function.h +++ b/be/src/vec/aggregate_functions/aggregate_function.h @@ -36,6 +36,7 @@ #include "vec/data_types/data_type_string.h" namespace doris::vectorized { +#include "common/compile_check_begin.h" class Arena; class IColumn; @@ -598,3 +599,5 @@ class AggregateFunctionGuard { }; } // namespace doris::vectorized + +#include "common/compile_check_end.h" diff --git a/be/src/vec/aggregate_functions/aggregate_function_approx_count_distinct.cpp b/be/src/vec/aggregate_functions/aggregate_function_approx_count_distinct.cpp index 18662bf66cf38c..8bf6c32c0872de 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_approx_count_distinct.cpp +++ b/be/src/vec/aggregate_functions/aggregate_function_approx_count_distinct.cpp @@ -29,6 +29,7 @@ #include "vec/functions/function.h" namespace doris::vectorized { +#include "common/compile_check_begin.h" AggregateFunctionPtr create_aggregate_function_approx_count_distinct( const std::string& name, const DataTypes& argument_types, const bool result_is_nullable, diff --git a/be/src/vec/aggregate_functions/aggregate_function_approx_count_distinct.h b/be/src/vec/aggregate_functions/aggregate_function_approx_count_distinct.h index d267499e059818..3ef22be9fca74c 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_approx_count_distinct.h +++ b/be/src/vec/aggregate_functions/aggregate_function_approx_count_distinct.h @@ -38,6 +38,7 @@ #include "vec/io/io_helper.h" namespace doris { +#include "common/compile_check_begin.h" namespace vectorized { class Arena; class BufferReadable; @@ -64,8 +65,7 @@ struct AggregateFunctionApproxCountDistinctData { void write(BufferWritable& buf) const { std::string result; result.resize(hll_data.max_serialized_size()); - int size = hll_data.serialize((uint8_t*)result.data()); - result.resize(size); + result.resize(hll_data.serialize((uint8_t*)result.data())); write_binary(result, buf); } @@ -136,3 +136,5 @@ class AggregateFunctionApproxCountDistinct final }; } // namespace doris::vectorized + +#include "common/compile_check_end.h" diff --git a/be/src/vec/aggregate_functions/aggregate_function_avg.cpp b/be/src/vec/aggregate_functions/aggregate_function_avg.cpp index 6a6711f90f983e..6109f0b0c601cd 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_avg.cpp +++ b/be/src/vec/aggregate_functions/aggregate_function_avg.cpp @@ -25,6 +25,7 @@ #include "vec/core/field.h" namespace doris::vectorized { +#include "common/compile_check_begin.h" template struct Avg { diff --git a/be/src/vec/aggregate_functions/aggregate_function_avg.h b/be/src/vec/aggregate_functions/aggregate_function_avg.h index 62fbb8078ea949..8b24db692aef05 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_avg.h +++ b/be/src/vec/aggregate_functions/aggregate_function_avg.h @@ -41,6 +41,7 @@ #include "vec/io/io_helper.h" namespace doris { +#include "common/compile_check_begin.h" namespace vectorized { class Arena; class BufferReadable; @@ -72,7 +73,8 @@ struct AggregateFunctionAvgData { ResultT result() const { if constexpr (std::is_floating_point_v) { if constexpr (std::numeric_limits::is_iec559) { - return static_cast(sum) / count; /// allow division by zero + return static_cast(sum) / + static_cast(count); /// allow division by zero } } @@ -91,7 +93,7 @@ struct AggregateFunctionAvgData { if constexpr (IsDecimal256) { return static_cast(sum / T(count)); } else { - return static_cast(sum) / count; + return static_cast(sum) / static_cast(count); } } } @@ -124,7 +126,11 @@ class AggregateFunctionAvg final IsDecimalV2, ColumnDecimal, std::conditional_t, ColumnDecimal, ColumnFloat64>>; + // The result calculated by PercentileApprox is an approximate value, + // so the underlying storage uses float. The following calls will involve + // an implicit cast to float. + using DataType = typename Data::ResultType; /// ctor for native types AggregateFunctionAvg(const DataTypes& argument_types_) : IAggregateFunctionDataHelper>(argument_types_), @@ -148,9 +154,9 @@ class AggregateFunctionAvg final const auto& column = assert_cast(*columns[0]); if constexpr (IsDecimalNumber) { - this->data(place).sum += column.get_data()[row_num].value; + this->data(place).sum += (DataType)column.get_data()[row_num].value; } else { - this->data(place).sum += column.get_data()[row_num]; + this->data(place).sum += (DataType)column.get_data()[row_num]; } ++this->data(place).count; } @@ -282,3 +288,5 @@ class AggregateFunctionAvg final }; } // namespace doris::vectorized + +#include "common/compile_check_end.h" diff --git a/be/src/vec/aggregate_functions/aggregate_function_avg_weighted.cpp b/be/src/vec/aggregate_functions/aggregate_function_avg_weighted.cpp index fc5df5303fd15d..70a707b02e992b 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_avg_weighted.cpp +++ b/be/src/vec/aggregate_functions/aggregate_function_avg_weighted.cpp @@ -21,6 +21,7 @@ #include "vec/aggregate_functions/helpers.h" namespace doris::vectorized { +#include "common/compile_check_begin.h" void register_aggregate_function_avg_weighted(AggregateFunctionSimpleFactory& factory) { factory.register_function_both("avg_weighted", creator_with_type::creator); diff --git a/be/src/vec/aggregate_functions/aggregate_function_avg_weighted.h b/be/src/vec/aggregate_functions/aggregate_function_avg_weighted.h index b59a3dccf0cea8..d1a5921b45039f 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_avg_weighted.h +++ b/be/src/vec/aggregate_functions/aggregate_function_avg_weighted.h @@ -35,6 +35,7 @@ #include "vec/io/io_helper.h" namespace doris { +#include "common/compile_check_begin.h" namespace vectorized { class Arena; class BufferReadable; @@ -57,7 +58,7 @@ struct AggregateFunctionAvgWeightedData { DecimalV2Value value = binary_cast(data_val); data_sum = data_sum + (double(value) * weight_val); } else { - data_sum = data_sum + (data_val * weight_val); + data_sum = data_sum + (double(data_val) * weight_val); } weight_sum = weight_sum + weight_val; } @@ -138,3 +139,5 @@ class AggregateFunctionAvgWeight final }; } // namespace doris::vectorized + +#include "common/compile_check_end.h" diff --git a/be/src/vec/aggregate_functions/aggregate_function_binary.h b/be/src/vec/aggregate_functions/aggregate_function_binary.h index 9fba9d11a1013a..fd5fc55d253661 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_binary.h +++ b/be/src/vec/aggregate_functions/aggregate_function_binary.h @@ -36,6 +36,7 @@ #include "vec/io/io_helper.h" namespace doris::vectorized { +#include "common/compile_check_begin.h" template typename Moments> struct StatFunc { @@ -127,3 +128,5 @@ AggregateFunctionPtr create_with_two_basic_numeric_types(const DataTypePtr& firs } } // namespace doris::vectorized + +#include "common/compile_check_end.h" diff --git a/be/src/vec/aggregate_functions/aggregate_function_bit.cpp b/be/src/vec/aggregate_functions/aggregate_function_bit.cpp index 97a6c0e92fa723..981ced1fbd5a46 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_bit.cpp +++ b/be/src/vec/aggregate_functions/aggregate_function_bit.cpp @@ -24,6 +24,7 @@ #include "vec/aggregate_functions/helpers.h" namespace doris::vectorized { +#include "common/compile_check_begin.h" void register_aggregate_function_bit(AggregateFunctionSimpleFactory& factory) { factory.register_function_both( diff --git a/be/src/vec/aggregate_functions/aggregate_function_bit.h b/be/src/vec/aggregate_functions/aggregate_function_bit.h index 1ab01b03ceea38..d9760fdd30080b 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_bit.h +++ b/be/src/vec/aggregate_functions/aggregate_function_bit.h @@ -30,6 +30,7 @@ #include "vec/io/io_helper.h" namespace doris { +#include "common/compile_check_begin.h" namespace vectorized { class Arena; class BufferReadable; @@ -142,4 +143,5 @@ class AggregateFunctionBitwise final } }; -} // namespace doris::vectorized \ No newline at end of file +} // namespace doris::vectorized +#include "common/compile_check_end.h" diff --git a/be/src/vec/aggregate_functions/aggregate_function_bitmap.cpp b/be/src/vec/aggregate_functions/aggregate_function_bitmap.cpp index e9c86d4b9556da..47ddf2d81b6a71 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_bitmap.cpp +++ b/be/src/vec/aggregate_functions/aggregate_function_bitmap.cpp @@ -23,6 +23,7 @@ #include "vec/data_types/data_type_nullable.h" namespace doris::vectorized { +#include "common/compile_check_begin.h" template class AggregateFunctionTemplate> AggregateFunctionPtr create_with_int_data_type(const DataTypes& argument_type) { @@ -33,7 +34,11 @@ AggregateFunctionPtr create_with_int_data_type(const DataTypes& argument_type) { return std::make_shared>>( \ argument_type); \ } - FOR_INTEGER_TYPES(DISPATCH) + // Keep consistent with the FE definition; the function does not have an int128 type. + DISPATCH(Int8) + DISPATCH(Int16) + DISPATCH(Int32) + DISPATCH(Int64) #undef DISPATCH LOG(WARNING) << "with unknowed type, failed in create_with_int_data_type bitmap_union_int" << " and type is: " << argument_type[0]->get_name(); diff --git a/be/src/vec/aggregate_functions/aggregate_function_bitmap.h b/be/src/vec/aggregate_functions/aggregate_function_bitmap.h index b0619a63e1ffe8..fb17b0a80be092 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_bitmap.h +++ b/be/src/vec/aggregate_functions/aggregate_function_bitmap.h @@ -38,6 +38,7 @@ #include "vec/data_types/data_type_number.h" namespace doris { +#include "common/compile_check_begin.h" namespace vectorized { class Arena; class BufferReadable; @@ -432,4 +433,5 @@ AggregateFunctionPtr create_aggregate_function_bitmap_union(const std::string& n const DataTypes& argument_types, const bool result_is_nullable); -} // namespace doris::vectorized \ No newline at end of file +} // namespace doris::vectorized +#include "common/compile_check_end.h" diff --git a/be/src/vec/aggregate_functions/aggregate_function_bitmap_agg.cpp b/be/src/vec/aggregate_functions/aggregate_function_bitmap_agg.cpp index 0b95ddfd46f0d5..2a2c86303f3000 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_bitmap_agg.cpp +++ b/be/src/vec/aggregate_functions/aggregate_function_bitmap_agg.cpp @@ -23,6 +23,7 @@ #include "vec/data_types/data_type_nullable.h" namespace doris::vectorized { +#include "common/compile_check_begin.h" template AggregateFunctionPtr create_with_int_data_type(const DataTypes& argument_types) { @@ -32,7 +33,11 @@ AggregateFunctionPtr create_with_int_data_type(const DataTypes& argument_types) if (which.idx == TypeIndex::TYPE) { \ return std::make_shared>(argument_types); \ } - FOR_INTEGER_TYPES(DISPATCH) + // Keep consistent with the FE definition; the function does not have an int128 type. + DISPATCH(Int8) + DISPATCH(Int16) + DISPATCH(Int32) + DISPATCH(Int64) #undef DISPATCH LOG(WARNING) << "with unknown type, failed in create_with_int_data_type bitmap_union_int" << " and type is: " << argument_types[0]->get_name(); diff --git a/be/src/vec/aggregate_functions/aggregate_function_bitmap_agg.h b/be/src/vec/aggregate_functions/aggregate_function_bitmap_agg.h index 5747faf1b8e8c1..bff32aa606ccd2 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_bitmap_agg.h +++ b/be/src/vec/aggregate_functions/aggregate_function_bitmap_agg.h @@ -31,6 +31,7 @@ #include "vec/data_types/data_type_bitmap.h" namespace doris { +#include "common/compile_check_begin.h" namespace vectorized { class Arena; class BufferReadable; @@ -226,4 +227,5 @@ class AggregateFunctionBitmapAgg final } }; -} // namespace doris::vectorized \ No newline at end of file +} // namespace doris::vectorized +#include "common/compile_check_end.h" diff --git a/be/src/vec/aggregate_functions/aggregate_function_collect.cpp b/be/src/vec/aggregate_functions/aggregate_function_collect.cpp index d726b7c6355318..a4853ff98659a0 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_collect.cpp +++ b/be/src/vec/aggregate_functions/aggregate_function_collect.cpp @@ -26,6 +26,7 @@ #include "vec/aggregate_functions/helpers.h" namespace doris::vectorized { +#include "common/compile_check_begin.h" template AggregateFunctionPtr do_create_agg_function_collect(bool distinct, const DataTypes& argument_types, diff --git a/be/src/vec/aggregate_functions/aggregate_function_collect.h b/be/src/vec/aggregate_functions/aggregate_function_collect.h index da310c6e0cc4c2..2d18a56313f3f9 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_collect.h +++ b/be/src/vec/aggregate_functions/aggregate_function_collect.h @@ -46,6 +46,7 @@ #include "vec/io/var_int.h" namespace doris { +#include "common/compile_check_begin.h" namespace vectorized { class Arena; } // namespace vectorized @@ -836,3 +837,5 @@ class AggregateFunctionCollect }; } // namespace doris::vectorized + +#include "common/compile_check_end.h" diff --git a/be/src/vec/aggregate_functions/aggregate_function_combinator.h b/be/src/vec/aggregate_functions/aggregate_function_combinator.h index 1593d74ed4e59d..0908ac8d0278f1 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_combinator.h +++ b/be/src/vec/aggregate_functions/aggregate_function_combinator.h @@ -26,6 +26,7 @@ #include "vec/data_types/data_type.h" namespace doris::vectorized { +#include "common/compile_check_begin.h" /** Aggregate function combinator allows to take one aggregate function * and transform it to another aggregate function. @@ -69,3 +70,5 @@ class IAggregateFunctionCombinator { }; } // namespace doris::vectorized + +#include "common/compile_check_end.h" diff --git a/be/src/vec/aggregate_functions/aggregate_function_corr.cpp b/be/src/vec/aggregate_functions/aggregate_function_corr.cpp index cdaab6e086f4a5..e0a51ca6629a06 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_corr.cpp +++ b/be/src/vec/aggregate_functions/aggregate_function_corr.cpp @@ -21,6 +21,7 @@ #include "vec/core/types.h" namespace doris::vectorized { +#include "common/compile_check_begin.h" template struct CorrMoment { diff --git a/be/src/vec/aggregate_functions/aggregate_function_count.cpp b/be/src/vec/aggregate_functions/aggregate_function_count.cpp index 5cfe5af41982f6..72d12cf65fe9d0 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_count.cpp +++ b/be/src/vec/aggregate_functions/aggregate_function_count.cpp @@ -26,6 +26,7 @@ #include "vec/aggregate_functions/factory_helpers.h" namespace doris::vectorized { +#include "common/compile_check_begin.h" AggregateFunctionPtr create_aggregate_function_count(const std::string& name, const DataTypes& argument_types, diff --git a/be/src/vec/aggregate_functions/aggregate_function_count.h b/be/src/vec/aggregate_functions/aggregate_function_count.h index 7b54d074683b04..630994a7967957 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_count.h +++ b/be/src/vec/aggregate_functions/aggregate_function_count.h @@ -41,6 +41,7 @@ #include "vec/io/var_int.h" namespace doris { +#include "common/compile_check_begin.h" namespace vectorized { class Arena; class BufferReadable; @@ -321,3 +322,5 @@ class AggregateFunctionCountNotNullUnary final }; } // namespace doris::vectorized + +#include "common/compile_check_end.h" diff --git a/be/src/vec/aggregate_functions/aggregate_function_count_by_enum.cpp b/be/src/vec/aggregate_functions/aggregate_function_count_by_enum.cpp index 093b31d57db554..20235d9e2ef2e9 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_count_by_enum.cpp +++ b/be/src/vec/aggregate_functions/aggregate_function_count_by_enum.cpp @@ -26,6 +26,7 @@ #include "vec/core/types.h" namespace doris::vectorized { +#include "common/compile_check_begin.h" AggregateFunctionPtr create_aggregate_function_count_by_enum(const std::string& name, const DataTypes& argument_types, diff --git a/be/src/vec/aggregate_functions/aggregate_function_count_by_enum.h b/be/src/vec/aggregate_functions/aggregate_function_count_by_enum.h index 1f5093de68263e..543ae55f872da6 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_count_by_enum.h +++ b/be/src/vec/aggregate_functions/aggregate_function_count_by_enum.h @@ -32,6 +32,7 @@ #include "vec/io/io_helper.h" namespace doris::vectorized { +#include "common/compile_check_begin.h" struct CountByEnumData { std::unordered_map cbe; @@ -46,8 +47,7 @@ void build_json_from_vec(rapidjson::StringBuffer& buffer, doc.SetArray(); rapidjson::Document::AllocatorType& allocator = doc.GetAllocator(); - int vec_size_number = data_vec.size(); - for (int idx = 0; idx < vec_size_number; ++idx) { + for (size_t idx = 0; idx < data_vec.size(); ++idx) { rapidjson::Value obj(rapidjson::kObjectType); rapidjson::Value obj_cbe(rapidjson::kObjectType); @@ -239,4 +239,5 @@ class AggregateFunctionCountByEnum final size_t arg_count; }; -} // namespace doris::vectorized \ No newline at end of file +} // namespace doris::vectorized +#include "common/compile_check_end.h" diff --git a/be/src/vec/aggregate_functions/aggregate_function_covar.cpp b/be/src/vec/aggregate_functions/aggregate_function_covar.cpp index 4c5fe1321952d6..d9c091fb601868 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_covar.cpp +++ b/be/src/vec/aggregate_functions/aggregate_function_covar.cpp @@ -28,6 +28,7 @@ #include "vec/data_types/data_type_nullable.h" namespace doris::vectorized { +#include "common/compile_check_begin.h" template