diff --git a/.asf.yaml b/.asf.yaml index 5aa08854a67219..0a2e83fcd51aea 100644 --- a/.asf.yaml +++ b/.asf.yaml @@ -109,8 +109,8 @@ github: - shuke987 - wm1581066 - KassieZ - - yujun777 - gavinchou + - yujun777 notifications: pullrequests_status: commits@doris.apache.org diff --git a/be/src/agent/task_worker_pool.cpp b/be/src/agent/task_worker_pool.cpp index b9a55518c46a78..183a5744aae10f 100644 --- a/be/src/agent/task_worker_pool.cpp +++ b/be/src/agent/task_worker_pool.cpp @@ -190,7 +190,8 @@ void alter_tablet(StorageEngine& engine, const TAgentTaskRequest& agent_task_req Status res = Status::OK(); try { DCHECK(agent_task_req.alter_tablet_req_v2.__isset.job_id); - SchemaChangeJob job(engine, agent_task_req.alter_tablet_req_v2); + SchemaChangeJob job(engine, agent_task_req.alter_tablet_req_v2, + std::to_string(agent_task_req.alter_tablet_req_v2.job_id)); status = job.process_alter_tablet(agent_task_req.alter_tablet_req_v2); } catch (const Exception& e) { status = e.to_status(); diff --git a/be/src/cloud/cloud_schema_change_job.cpp b/be/src/cloud/cloud_schema_change_job.cpp index e919e3f37aa492..d1d323e19a9dce 100644 --- a/be/src/cloud/cloud_schema_change_job.cpp +++ b/be/src/cloud/cloud_schema_change_job.cpp @@ -201,6 +201,10 @@ Status CloudSchemaChangeJob::_convert_historical_rowsets(const SchemaChangeParam "Don't support to add materialized view by linked schema change"); } + LOG(INFO) << "schema change type, sc_sorting: " << sc_sorting + << ", sc_directly: " << sc_directly << ", base_tablet=" << _base_tablet->tablet_id() + << ", new_tablet=" << _new_tablet->tablet_id(); + // 2. Generate historical data converter auto sc_procedure = get_sc_procedure(changer, sc_sorting); diff --git a/be/src/clucene b/be/src/clucene index ef95e67ae31234..ff2cd82f9e545a 160000 --- a/be/src/clucene +++ b/be/src/clucene @@ -1 +1 @@ -Subproject commit ef95e67ae3123409f006072194f742a079603159 +Subproject commit ff2cd82f9e545a24318f1256eba312b4d0562a82 diff --git a/be/src/exec/exec_node.h b/be/src/exec/exec_node.h index 5d7b3a91651757..f2303068437b2f 100644 --- a/be/src/exec/exec_node.h +++ b/be/src/exec/exec_node.h @@ -127,7 +127,7 @@ class ExecNode { bool has_output_row_descriptor() const { return _output_row_descriptor != nullptr; } // If use projection, we should clear `_origin_block`. void clear_origin_block() { - _origin_block.clear_column_data(_row_descriptor.num_materialized_slots()); + _origin_block.clear_column_data(intermediate_row_desc().num_materialized_slots()); } // Emit data, both need impl with method: sink @@ -326,8 +326,8 @@ class ExecNode { std::shared_ptr _query_statistics = nullptr; //_keep_origin is used to avoid copying during projection, - // currently set to true only in the nestloop join. - bool _keep_origin = false; + // currently set to false only in the nestloop join. + bool _keep_origin = true; private: static Status create_tree_helper(RuntimeState* state, ObjectPool* pool, diff --git a/be/src/exec/schema_scanner.cpp b/be/src/exec/schema_scanner.cpp index 32197e37e1e21b..908bb7b6f09f03 100644 --- a/be/src/exec/schema_scanner.cpp +++ b/be/src/exec/schema_scanner.cpp @@ -37,6 +37,7 @@ #include "exec/schema_scanner/schema_partitions_scanner.h" #include "exec/schema_scanner/schema_processlist_scanner.h" #include "exec/schema_scanner/schema_profiling_scanner.h" +#include "exec/schema_scanner/schema_routine_scanner.h" #include "exec/schema_scanner/schema_rowsets_scanner.h" #include "exec/schema_scanner/schema_schema_privileges_scanner.h" #include "exec/schema_scanner/schema_schemata_scanner.h" @@ -161,6 +162,8 @@ std::unique_ptr SchemaScanner::create(TSchemaTableType::type type return SchemaWorkloadGroupsScanner::create_unique(); case TSchemaTableType::SCH_PROCESSLIST: return SchemaProcessListScanner::create_unique(); + case TSchemaTableType::SCH_PROCEDURES: + return SchemaRoutinesScanner::create_unique(); default: return SchemaDummyScanner::create_unique(); break; diff --git a/be/src/exec/schema_scanner/schema_routine_scanner.cpp b/be/src/exec/schema_scanner/schema_routine_scanner.cpp new file mode 100644 index 00000000000000..7db46ada650bc6 --- /dev/null +++ b/be/src/exec/schema_scanner/schema_routine_scanner.cpp @@ -0,0 +1,172 @@ +// 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 "exec/schema_scanner/schema_routine_scanner.h" + +#include "runtime/client_cache.h" +#include "runtime/exec_env.h" +#include "runtime/runtime_state.h" +#include "util/thrift_rpc_helper.h" +#include "vec/common/string_ref.h" +#include "vec/core/block.h" +#include "vec/data_types/data_type_factory.hpp" + +namespace doris { +std::vector SchemaRoutinesScanner::_s_tbls_columns = { + {"SPECIFIC_NAME", TYPE_VARCHAR, sizeof(StringRef), true}, + {"ROUTINE_CATALOG", TYPE_VARCHAR, sizeof(StringRef), true}, + {"ROUTINE_SCHEMA", TYPE_VARCHAR, sizeof(StringRef), true}, + {"ROUTINE_NAME", TYPE_VARCHAR, sizeof(StringRef), true}, + {"ROUTINE_TYPE", TYPE_VARCHAR, sizeof(StringRef), true}, + {"DTD_IDENTIFIER", TYPE_VARCHAR, sizeof(StringRef), true}, + {"ROUTINE_BODY", TYPE_VARCHAR, sizeof(StringRef), true}, + {"ROUTINE_DEFINITION", TYPE_VARCHAR, sizeof(StringRef), true}, + {"EXTERNAL_NAME", TYPE_VARCHAR, sizeof(StringRef), true}, + {"EXTERNAL_LANGUAGE", TYPE_VARCHAR, sizeof(StringRef), true}, + {"PARAMETER_STYLE", TYPE_VARCHAR, sizeof(StringRef), true}, + {"IS_DETERMINISTIC", TYPE_VARCHAR, sizeof(StringRef), true}, + {"SQL_DATA_ACCESS", TYPE_VARCHAR, sizeof(StringRef), true}, + {"SQL_PATH", TYPE_VARCHAR, sizeof(StringRef), true}, + {"SECURITY_TYPE", TYPE_VARCHAR, sizeof(StringRef), true}, + {"CREATED", TYPE_DATETIME, sizeof(int64_t), true}, + {"LAST_ALTERED", TYPE_DATETIME, sizeof(int64_t), true}, + {"SQL_MODE", TYPE_VARCHAR, sizeof(StringRef), true}, + {"ROUTINE_COMMENT", TYPE_VARCHAR, sizeof(StringRef), true}, + {"DEFINER", TYPE_VARCHAR, sizeof(StringRef), true}, + {"CHARACTER_SET_CLIENT", TYPE_VARCHAR, sizeof(StringRef), true}, + {"COLLATION_CONNECTION", TYPE_VARCHAR, sizeof(StringRef), true}, + {"DATABASE_COLLATION", TYPE_VARCHAR, sizeof(StringRef), true}, +}; + +SchemaRoutinesScanner::SchemaRoutinesScanner() + : SchemaScanner(_s_tbls_columns, TSchemaTableType::SCH_PROCEDURES) {} + +Status SchemaRoutinesScanner::start(RuntimeState* state) { + _block_rows_limit = state->batch_size(); + _rpc_timeout = state->execution_timeout() * 1000; + return Status::OK(); +} + +Status SchemaRoutinesScanner::get_block_from_fe() { + TNetworkAddress master_addr = ExecEnv::GetInstance()->master_info()->network_address; + TSchemaTableRequestParams schema_table_request_params; + for (int i = 0; i < _s_tbls_columns.size(); i++) { + schema_table_request_params.__isset.columns_name = true; + schema_table_request_params.columns_name.emplace_back(_s_tbls_columns[i].name); + } + schema_table_request_params.__set_current_user_ident(*_param->common_param->current_user_ident); + TFetchSchemaTableDataRequest request; + request.__set_schema_table_name(TSchemaTableName::ROUTINES_INFO); + request.__set_schema_table_params(schema_table_request_params); + TFetchSchemaTableDataResult result; + RETURN_IF_ERROR(ThriftRpcHelper::rpc( + master_addr.hostname, master_addr.port, + [&request, &result](FrontendServiceConnection& client) { + client->fetchSchemaTableData(result, request); + }, + _rpc_timeout)); + Status status(Status::create(result.status)); + if (!status.ok()) { + LOG(WARNING) << "fetch routines from FE failed, errmsg=" << status; + return status; + } + std::vector result_data = result.data_batch; + _routines_block = vectorized::Block::create_unique(); + for (int i = 0; i < _s_tbls_columns.size(); ++i) { + TypeDescriptor descriptor(_s_tbls_columns[i].type); + auto data_type = vectorized::DataTypeFactory::instance().create_data_type(descriptor, true); + _routines_block->insert(vectorized::ColumnWithTypeAndName( + data_type->create_column(), data_type, _s_tbls_columns[i].name)); + } + _routines_block->reserve(_block_rows_limit); + if (result_data.size() > 0) { + int col_size = result_data[0].column_value.size(); + if (col_size != _s_tbls_columns.size()) { + return Status::InternalError("routine table schema is not match for FE and BE"); + } + } + auto insert_string_value = [&](int col_index, std::string str_val, vectorized::Block* block) { + vectorized::MutableColumnPtr mutable_col_ptr; + mutable_col_ptr = std::move(*block->get_by_position(col_index).column).assume_mutable(); + auto* nullable_column = + reinterpret_cast(mutable_col_ptr.get()); + vectorized::IColumn* col_ptr = &nullable_column->get_nested_column(); + reinterpret_cast(col_ptr)->insert_data(str_val.data(), + str_val.size()); + nullable_column->get_null_map_data().emplace_back(0); + }; + auto insert_datetime_value = [&](int col_index, const std::vector& datas, + vectorized::Block* block) { + vectorized::MutableColumnPtr mutable_col_ptr; + mutable_col_ptr = std::move(*block->get_by_position(col_index).column).assume_mutable(); + auto* nullable_column = + reinterpret_cast(mutable_col_ptr.get()); + vectorized::IColumn* col_ptr = &nullable_column->get_nested_column(); + auto data = datas[0]; + reinterpret_cast*>(col_ptr)->insert_data( + reinterpret_cast(data), 0); + nullable_column->get_null_map_data().emplace_back(0); + }; + + for (int i = 0; i < result_data.size(); i++) { + TRow row = result_data[i]; + + for (int j = 0; j < _s_tbls_columns.size(); j++) { + if (_s_tbls_columns[j].type == TYPE_DATETIME) { + std::vector datas(1); + VecDateTimeValue src[1]; + src[0].from_date_str(row.column_value[j].stringVal.data(), + row.column_value[j].stringVal.size()); + datas[0] = src; + insert_datetime_value(j, datas, _routines_block.get()); + } else { + insert_string_value(j, row.column_value[j].stringVal, _routines_block.get()); + } + } + } + return Status::OK(); +} + +Status SchemaRoutinesScanner::get_next_block(vectorized::Block* block, bool* eos) { + if (!_is_init) { + return Status::InternalError("Used before initialized."); + } + + if (nullptr == block || nullptr == eos) { + return Status::InternalError("input pointer is nullptr."); + } + + if (_routines_block == nullptr) { + RETURN_IF_ERROR(get_block_from_fe()); + _total_rows = _routines_block->rows(); + } + + if (_row_idx == _total_rows) { + *eos = true; + return Status::OK(); + } + + int current_batch_rows = std::min(_block_rows_limit, _total_rows - _row_idx); + vectorized::MutableBlock mblock = vectorized::MutableBlock::build_mutable_block(block); + mblock.add_rows(_routines_block.get(), _row_idx, current_batch_rows); + _row_idx += current_batch_rows; + + *eos = _row_idx == _total_rows; + return Status::OK(); +} + +} // namespace doris \ No newline at end of file diff --git a/be/src/exec/schema_scanner/schema_routine_scanner.h b/be/src/exec/schema_scanner/schema_routine_scanner.h new file mode 100644 index 00000000000000..543f9e8e8f684a --- /dev/null +++ b/be/src/exec/schema_scanner/schema_routine_scanner.h @@ -0,0 +1,52 @@ +// 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 "common/status.h" +#include "exec/schema_scanner.h" + +namespace doris { +class RuntimeState; +namespace vectorized { +class Block; +} // namespace vectorized + +class SchemaRoutinesScanner : public SchemaScanner { + ENABLE_FACTORY_CREATOR(SchemaRoutinesScanner); + +public: + SchemaRoutinesScanner(); + ~SchemaRoutinesScanner() override = default; + + Status start(RuntimeState* state) override; + Status get_next_block(vectorized::Block* block, bool* eos) override; + + static std::vector _s_tbls_columns; + +private: + Status get_block_from_fe(); + + int _block_rows_limit = 4096; + int _row_idx = 0; + int _total_rows = 0; + std::unique_ptr _routines_block = nullptr; + int _rpc_timeout = 3000; +}; +}; // namespace doris diff --git a/be/src/exec/schema_scanner/schema_workload_groups_scanner.cpp b/be/src/exec/schema_scanner/schema_workload_groups_scanner.cpp index 24e23a3e336ae8..03bf9782dcd814 100644 --- a/be/src/exec/schema_scanner/schema_workload_groups_scanner.cpp +++ b/be/src/exec/schema_scanner/schema_workload_groups_scanner.cpp @@ -41,6 +41,7 @@ std::vector SchemaWorkloadGroupsScanner::_s_tbls_colu {"MIN_REMOTE_SCAN_THREAD_NUM", TYPE_BIGINT, sizeof(int64_t), true}, {"SPILL_THRESHOLD_LOW_WATERMARK", TYPE_VARCHAR, sizeof(StringRef), true}, {"SPILL_THRESHOLD_HIGH_WATERMARK", TYPE_VARCHAR, sizeof(StringRef), true}, + {"TAG", TYPE_VARCHAR, sizeof(StringRef), true}, }; SchemaWorkloadGroupsScanner::SchemaWorkloadGroupsScanner() diff --git a/be/src/olap/hll.h b/be/src/olap/hll.h index 1f97f30ae8bbbd..0ae5539163c904 100644 --- a/be/src/olap/hll.h +++ b/be/src/olap/hll.h @@ -109,7 +109,7 @@ class HyperLogLog { } } - HyperLogLog(HyperLogLog&& other) { + HyperLogLog(HyperLogLog&& other) noexcept { this->_type = other._type; switch (other._type) { case HLL_DATA_EMPTY: @@ -131,7 +131,7 @@ class HyperLogLog { } } - HyperLogLog& operator=(HyperLogLog&& other) { + HyperLogLog& operator=(HyperLogLog&& other) noexcept { if (this != &other) { if (_registers != nullptr) { delete[] _registers; 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 078668cb0cf230..87827464f155b2 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index_writer.cpp +++ b/be/src/olap/rowset/segment_v2/inverted_index_writer.cpp @@ -80,8 +80,11 @@ class InvertedIndexColumnWriterImpl : public InvertedIndexColumnWriter { explicit InvertedIndexColumnWriterImpl(const std::string& field_name, InvertedIndexFileWriter* index_file_writer, - const TabletIndex* index_meta) - : _index_meta(index_meta), _index_file_writer(index_file_writer) { + const TabletIndex* index_meta, + const bool single_field = true) + : _single_field(single_field), + _index_meta(index_meta), + _index_file_writer(index_file_writer) { _parser_type = get_inverted_index_parser_type_from_string( get_parser_string_from_properties(_index_meta->properties())); _value_key_coder = get_key_coder(field_type); @@ -237,9 +240,14 @@ class InvertedIndexColumnWriterImpl : public InvertedIndexColumnWriter { RETURN_IF_ERROR(create_char_string_reader(_char_string_reader)); RETURN_IF_ERROR(create_analyzer(_analyzer)); RETURN_IF_ERROR(create_index_writer(_index_writer)); - RETURN_IF_ERROR(create_field(&_field)); _doc = std::make_unique(); - _doc->add(*_field); + if (_single_field) { + RETURN_IF_ERROR(create_field(&_field)); + _doc->add(*_field); + } else { + // array's inverted index do need create field first + _doc->setNeedResetFieldData(true); + } return Status::OK(); } @@ -282,11 +290,9 @@ class InvertedIndexColumnWriterImpl : public InvertedIndexColumnWriter { return Status::OK(); } - void new_fulltext_field(const char* field_value_data, size_t field_value_size) { - if (_parser_type == InvertedIndexParserType::PARSER_ENGLISH || - _parser_type == InvertedIndexParserType::PARSER_CHINESE || - _parser_type == InvertedIndexParserType::PARSER_UNICODE || - _parser_type == InvertedIndexParserType::PARSER_STANDARD) { + void new_inverted_index_field(const char* field_value_data, size_t field_value_size) { + if (_parser_type != InvertedIndexParserType::PARSER_UNKNOWN && + _parser_type != InvertedIndexParserType::PARSER_NONE) { new_char_token_stream(field_value_data, field_value_size, _field); } else { new_field_char_value(field_value_data, field_value_size, _field); @@ -328,7 +334,7 @@ class InvertedIndexColumnWriterImpl : public InvertedIndexColumnWriter { (_parser_type != InvertedIndexParserType::PARSER_NONE && v->empty())) { RETURN_IF_ERROR(add_null_document()); } else { - new_fulltext_field(v->get_data(), v->get_size()); + new_inverted_index_field(v->get_data(), v->get_size()); RETURN_IF_ERROR(add_document()); } ++v; @@ -348,39 +354,58 @@ class InvertedIndexColumnWriterImpl : public InvertedIndexColumnWriter { } const auto* offsets = reinterpret_cast(offsets_ptr); if constexpr (field_is_slice_type(field_type)) { - if (_field == nullptr || _index_writer == nullptr) { - LOG(ERROR) << "field or index writer is null in inverted index writer."; - return Status::InternalError( - "field or index writer is null in inverted index writer"); + if (_index_writer == nullptr) { + LOG(ERROR) << "index writer is null in inverted index writer."; + return Status::InternalError("index writer is null in inverted index writer"); } auto ignore_above_value = get_parser_ignore_above_value_from_properties(_index_meta->properties()); auto ignore_above = std::stoi(ignore_above_value); for (int i = 0; i < count; ++i) { // offsets[i+1] is now row element count - std::vector strings; // [0, 3, 6] // [10,20,30] [20,30,40], [30,40,50] auto start_off = offsets[i]; auto end_off = offsets[i + 1]; + // TODO(Amory).later we use object pool to avoid field creation + lucene::document::Field* new_field = nullptr; + CL_NS(analysis)::TokenStream* ts = nullptr; for (auto j = start_off; j < end_off; ++j) { if (null_map[j] == 1) { continue; } + // now we temp create field . later make a pool + if (Status st = create_field(&new_field); st != Status::OK()) { + LOG(ERROR) + << "create field " << string(_field_name.begin(), _field_name.end()) + << " error:" << st; + return st; + } auto* v = (Slice*)((const uint8_t*)value_ptr + j * field_size); - strings.emplace_back(v->get_data(), v->get_size()); - } - - auto value = join(strings, " "); - // only ignore_above UNTOKENIZED strings and empty strings not tokenized - if ((_parser_type == InvertedIndexParserType::PARSER_NONE && - value.length() > ignore_above) || - (_parser_type != InvertedIndexParserType::PARSER_NONE && value.empty())) { - RETURN_IF_ERROR(add_null_document()); - } else { - new_fulltext_field(value.c_str(), value.length()); - RETURN_IF_ERROR(add_document()); + if ((_parser_type == InvertedIndexParserType::PARSER_NONE && + v->get_size() > ignore_above) || + (_parser_type != InvertedIndexParserType::PARSER_NONE && v->empty())) { + // is here a null value? + // TODO. Maybe here has performance problem for large size string. + continue; + } else { + if (_parser_type != InvertedIndexParserType::PARSER_UNKNOWN && + _parser_type != InvertedIndexParserType::PARSER_NONE) { + // in this case stream need to delete after add_document, because the + // stream can not reuse for different field + _char_string_reader->init(v->get_data(), v->get_size(), false); + ts = _analyzer->tokenStream(new_field->name(), + _char_string_reader.get()); + new_field->setValue(ts); + } else { + new_field_char_value(v->get_data(), v->get_size(), new_field); + } + _doc->add(*new_field); + } } + RETURN_IF_ERROR(add_document()); + _doc->clear(); + _CLDELETE(ts); _rid++; } } else if constexpr (field_is_numeric_type(field_type)) { @@ -426,7 +451,7 @@ class InvertedIndexColumnWriterImpl : public InvertedIndexColumnWriter { item_data_ptr = (uint8_t*)item_data_ptr + field_size; } auto value = join(strings, " "); - new_fulltext_field(value.c_str(), value.length()); + new_inverted_index_field(value.c_str(), value.length()); _rid++; RETURN_IF_ERROR(add_document()); values++; @@ -579,6 +604,7 @@ class InvertedIndexColumnWriterImpl : public InvertedIndexColumnWriter { std::unique_ptr _doc = nullptr; lucene::document::Field* _field = nullptr; + bool _single_field = true; std::unique_ptr _index_writer = nullptr; std::unique_ptr _analyzer = nullptr; std::unique_ptr _char_string_reader = nullptr; @@ -598,11 +624,13 @@ Status InvertedIndexColumnWriter::create(const Field* field, const auto* typeinfo = field->type_info(); FieldType type = typeinfo->type(); std::string field_name = field->name(); + bool single_field = true; if (type == FieldType::OLAP_FIELD_TYPE_ARRAY) { const auto* array_typeinfo = dynamic_cast(typeinfo); if (array_typeinfo != nullptr) { typeinfo = array_typeinfo->item_type_info(); type = typeinfo->type(); + single_field = false; } else { return Status::NotSupported("unsupported array type for inverted index: " + std::to_string(int(type))); @@ -610,10 +638,10 @@ Status InvertedIndexColumnWriter::create(const Field* field, } switch (type) { -#define M(TYPE) \ - case TYPE: \ - *res = std::make_unique>( \ - field_name, index_file_writer, index_meta); \ +#define M(TYPE) \ + case TYPE: \ + *res = std::make_unique>( \ + field_name, index_file_writer, index_meta, single_field); \ break; M(FieldType::OLAP_FIELD_TYPE_TINYINT) M(FieldType::OLAP_FIELD_TYPE_SMALLINT) diff --git a/be/src/olap/schema_change.cpp b/be/src/olap/schema_change.cpp index 1a0836014ae155..a4e81184b109d1 100644 --- a/be/src/olap/schema_change.cpp +++ b/be/src/olap/schema_change.cpp @@ -347,7 +347,7 @@ Status BlockChanger::change_block(vectorized::Block* ref_block, assert_cast(new_col->assume_mutable().get()); new_nullable_col->change_nested_column(ref_col); - new_nullable_col->get_null_map_data().resize_fill(new_nullable_col->size()); + new_nullable_col->get_null_map_data().resize_fill(ref_col->size()); } else { // nullable to not nullable: // suppose column `c_phone` is originally varchar(16) NOT NULL, @@ -394,11 +394,22 @@ Status BlockChanger::_check_cast_valid(vectorized::ColumnPtr ref_column, return Status::DataQualityError("Null data is changed to not nullable"); } } else { - const auto* new_null_map = + const auto& null_map_column = vectorized::check_and_get_column(new_column) - ->get_null_map_column() - .get_data() - .data(); + ->get_null_map_column(); + const auto& nested_column = + vectorized::check_and_get_column(new_column) + ->get_nested_column(); + const auto* new_null_map = null_map_column.get_data().data(); + + if (null_map_column.size() != new_column->size() || + nested_column.size() != new_column->size()) { + DCHECK(false); + return Status::InternalError( + "null_map_column size is changed, null_map_column_size={}, " + "new_column_size={}", + null_map_column.size(), new_column->size()); + } bool is_changed = false; for (size_t i = 0; i < ref_column->size(); i++) { @@ -713,7 +724,7 @@ Status SchemaChangeJob::process_alter_tablet(const TAlterTabletReqV2& request) { } SchemaChangeJob::SchemaChangeJob(StorageEngine& local_storage_engine, - const TAlterTabletReqV2& request) + const TAlterTabletReqV2& request, const std::string& job_id) : _local_storage_engine(local_storage_engine) { _base_tablet = _local_storage_engine.tablet_manager()->get_tablet(request.base_tablet_id); _new_tablet = _local_storage_engine.tablet_manager()->get_tablet(request.new_tablet_id); @@ -726,6 +737,7 @@ SchemaChangeJob::SchemaChangeJob(StorageEngine& local_storage_engine, // the complete variant is constructed by reading all the sub-columns of the variant. _new_tablet_schema = _new_tablet->tablet_schema()->copy_without_extracted_columns(); } + _job_id = job_id; } // In the past schema change and rollup will create new tablet and will wait for txns starting before the task to finished @@ -1017,7 +1029,7 @@ Status SchemaChangeJob::_convert_historical_rowsets(const SchemaChangeParams& sc int64_t* real_alter_version) { LOG(INFO) << "begin to convert historical rowsets for new_tablet from base_tablet." << " base_tablet=" << _base_tablet->tablet_id() - << ", new_tablet=" << _new_tablet->tablet_id(); + << ", new_tablet=" << _new_tablet->tablet_id() << ", job_id=" << _job_id; // find end version int32_t end_version = -1; @@ -1291,6 +1303,7 @@ Status SchemaChangeJob::parse_request(const SchemaChangeParams& sc_params, // use directly schema change instead. if (!(*sc_directly) && !(*sc_sorting)) { // check has remote rowset + // work for cloud and cold storage for (const auto& rs_reader : sc_params.ref_rowset_readers) { if (!rs_reader->rowset()->is_local()) { *sc_directly = true; diff --git a/be/src/olap/schema_change.h b/be/src/olap/schema_change.h index 2e1198e181a5c3..dc77cff9d06b44 100644 --- a/be/src/olap/schema_change.h +++ b/be/src/olap/schema_change.h @@ -274,7 +274,8 @@ struct SchemaChangeParams { class SchemaChangeJob { public: - SchemaChangeJob(StorageEngine& local_storage_engine, const TAlterTabletReqV2& request); + SchemaChangeJob(StorageEngine& local_storage_engine, const TAlterTabletReqV2& request, + const std::string& job_id); Status process_alter_tablet(const TAlterTabletReqV2& request); bool tablet_in_converting(int64_t tablet_id); @@ -323,5 +324,6 @@ class SchemaChangeJob { std::shared_mutex _mutex; std::unordered_set _tablet_ids_in_converting; std::set _supported_functions; + std::string _job_id; }; } // namespace doris diff --git a/be/src/pipeline/exec/join_probe_operator.cpp b/be/src/pipeline/exec/join_probe_operator.cpp index 5c89075e8b9a98..c78e5423709cf5 100644 --- a/be/src/pipeline/exec/join_probe_operator.cpp +++ b/be/src/pipeline/exec/join_probe_operator.cpp @@ -87,9 +87,7 @@ Status JoinProbeLocalState::_build_output_block( // and you could see a 'todo' in the Thrift definition. // Here, we have refactored it, but considering upgrade compatibility, we still need to retain the old code. if (!output_block->mem_reuse()) { - vectorized::MutableBlock tmp( - vectorized::VectorizedUtils::create_columns_with_type_and_name(p.row_desc())); - output_block->swap(tmp.to_block()); + output_block->swap(origin_block->clone_empty()); } output_block->swap(*origin_block); return Status::OK(); diff --git a/be/src/pipeline/exec/partitioned_aggregation_sink_operator.cpp b/be/src/pipeline/exec/partitioned_aggregation_sink_operator.cpp index 0a3eb7c70639e1..87eae081d08562 100644 --- a/be/src/pipeline/exec/partitioned_aggregation_sink_operator.cpp +++ b/be/src/pipeline/exec/partitioned_aggregation_sink_operator.cpp @@ -170,6 +170,9 @@ Status PartitionedAggSinkOperatorX::sink(doris::RuntimeState* state, vectorized: if (revocable_mem_size(state) > 0) { RETURN_IF_ERROR(revoke_memory(state)); } else { + for (auto& partition : local_state._shared_state->spill_partitions) { + RETURN_IF_ERROR(partition->finish_current_spilling(eos)); + } local_state._dependency->set_ready_to_read(); } } @@ -244,8 +247,17 @@ Status PartitionedAggSinkLocalState::revoke_memory(RuntimeState* state) { } } }}; + + auto execution_context = state->get_task_execution_context(); + _shared_state_holder = _shared_state->shared_from_this(); + status = ExecEnv::GetInstance()->spill_stream_mgr()->get_async_task_thread_pool()->submit_func( - [this, &parent, state] { + [this, &parent, state, execution_context] { + auto execution_context_lock = execution_context.lock(); + if (!execution_context_lock) { + LOG(INFO) << "execution_context released, maybe query was cancelled."; + return Status::Cancelled("Cancelled"); + } SCOPED_ATTACH_TASK(state); SCOPED_TIMER(Base::_spill_timer); Defer defer {[&]() { diff --git a/be/src/pipeline/exec/partitioned_aggregation_sink_operator.h b/be/src/pipeline/exec/partitioned_aggregation_sink_operator.h index 9628fb5766e7b6..5e61738681235d 100644 --- a/be/src/pipeline/exec/partitioned_aggregation_sink_operator.h +++ b/be/src/pipeline/exec/partitioned_aggregation_sink_operator.h @@ -276,6 +276,11 @@ class PartitionedAggSinkLocalState std::mutex _spill_lock; std::condition_variable _spill_cv; + /// Resources in shared state will be released when the operator is closed, + /// but there may be asynchronous spilling tasks at this time, which can lead to conflicts. + /// So, we need hold the pointer of shared state. + std::shared_ptr _shared_state_holder; + // temp structures during spilling vectorized::MutableColumns key_columns_; vectorized::MutableColumns value_columns_; diff --git a/be/src/pipeline/exec/partitioned_aggregation_source_operator.cpp b/be/src/pipeline/exec/partitioned_aggregation_source_operator.cpp index 82bb25e674cb6b..916a4e82681428 100644 --- a/be/src/pipeline/exec/partitioned_aggregation_source_operator.cpp +++ b/be/src/pipeline/exec/partitioned_aggregation_source_operator.cpp @@ -196,9 +196,18 @@ Status PartitionedAggLocalState::initiate_merge_spill_partition_agg_data(Runtime RETURN_IF_ERROR(Base::_shared_state->in_mem_shared_state->reset_hash_table()); _dependency->Dependency::block(); + auto execution_context = state->get_task_execution_context(); + _shared_state_holder = _shared_state->shared_from_this(); RETURN_IF_ERROR( ExecEnv::GetInstance()->spill_stream_mgr()->get_async_task_thread_pool()->submit_func( - [this, state] { + [this, state, execution_context] { + auto execution_context_lock = execution_context.lock(); + if (!execution_context_lock) { + LOG(INFO) << "execution_context released, maybe query was cancelled."; + // FIXME: return status is meaningless? + return Status::Cancelled("Cancelled"); + } + SCOPED_ATTACH_TASK(state); Defer defer {[&]() { if (!_status.ok()) { diff --git a/be/src/pipeline/exec/partitioned_aggregation_source_operator.h b/be/src/pipeline/exec/partitioned_aggregation_source_operator.h index cc408c07e21759..ac63402f227023 100644 --- a/be/src/pipeline/exec/partitioned_aggregation_source_operator.h +++ b/be/src/pipeline/exec/partitioned_aggregation_source_operator.h @@ -63,6 +63,11 @@ class PartitionedAggLocalState final : public PipelineXSpillLocalState _shared_state_holder; + std::unique_ptr _internal_runtime_profile; RuntimeProfile::Counter* _get_results_timer = nullptr; RuntimeProfile::Counter* _serialize_result_timer = nullptr; diff --git a/be/src/pipeline/exec/partitioned_hash_join_probe_operator.cpp b/be/src/pipeline/exec/partitioned_hash_join_probe_operator.cpp index 93ce24e8e72695..25ac9d333b165b 100644 --- a/be/src/pipeline/exec/partitioned_hash_join_probe_operator.cpp +++ b/be/src/pipeline/exec/partitioned_hash_join_probe_operator.cpp @@ -174,29 +174,37 @@ Status PartitionedHashJoinProbeLocalState::spill_build_block(RuntimeState* state auto* spill_io_pool = ExecEnv::GetInstance()->spill_stream_mgr()->get_spill_io_thread_pool( build_spilling_stream->get_spill_root_dir()); - return spill_io_pool->submit_func([state, &build_spilling_stream, &mutable_block, this] { - (void)state; // avoid ut compile error - SCOPED_ATTACH_TASK(state); - if (_spill_status_ok) { - auto build_block = mutable_block->to_block(); - DCHECK_EQ(mutable_block->rows(), 0); - auto st = build_spilling_stream->spill_block(build_block, false); - if (!st.ok()) { - std::unique_lock lock(_spill_lock); - _spill_status_ok = false; - _spill_status = std::move(st); - } else { - COUNTER_UPDATE(_spill_build_rows, build_block.rows()); - COUNTER_UPDATE(_spill_build_blocks, 1); - } - } - --_spilling_task_count; + auto execution_context = state->get_task_execution_context(); + _shared_state_holder = _shared_state->shared_from_this(); + return spill_io_pool->submit_func( + [execution_context, state, &build_spilling_stream, &mutable_block, this] { + auto execution_context_lock = execution_context.lock(); + if (!execution_context_lock) { + LOG(INFO) << "execution_context released, maybe query was cancelled."; + return; + } + (void)state; // avoid ut compile error + SCOPED_ATTACH_TASK(state); + if (_spill_status_ok) { + auto build_block = mutable_block->to_block(); + DCHECK_EQ(mutable_block->rows(), 0); + auto st = build_spilling_stream->spill_block(build_block, false); + if (!st.ok()) { + std::unique_lock lock(_spill_lock); + _spill_status_ok = false; + _spill_status = std::move(st); + } else { + COUNTER_UPDATE(_spill_build_rows, build_block.rows()); + COUNTER_UPDATE(_spill_build_blocks, 1); + } + } + --_spilling_task_count; - if (_spilling_task_count == 0) { - std::unique_lock lock(_spill_lock); - _dependency->set_ready(); - } - }); + if (_spilling_task_count == 0) { + std::unique_lock lock(_spill_lock); + _dependency->set_ready(); + } + }); } Status PartitionedHashJoinProbeLocalState::spill_probe_blocks(RuntimeState* state, @@ -223,34 +231,42 @@ Status PartitionedHashJoinProbeLocalState::spill_probe_blocks(RuntimeState* stat } if (!blocks.empty()) { - return spill_io_pool->submit_func([state, &blocks, &spilling_stream, this] { - (void)state; // avoid ut compile error - SCOPED_ATTACH_TASK(state); - COUNTER_UPDATE(_spill_probe_blocks, blocks.size()); - while (!blocks.empty()) { - auto block = std::move(blocks.back()); - blocks.pop_back(); - if (_spill_status_ok) { - auto st = spilling_stream->spill_block(block, false); - if (!st.ok()) { - std::unique_lock lock(_spill_lock); - _spill_status_ok = false; - _spill_status = std::move(st); - break; + auto execution_context = state->get_task_execution_context(); + _shared_state_holder = _shared_state->shared_from_this(); + return spill_io_pool->submit_func( + [execution_context, state, &blocks, spilling_stream, this] { + auto execution_context_lock = execution_context.lock(); + if (!execution_context_lock) { + LOG(INFO) << "execution_context released, maybe query was cancelled."; + _dependency->set_ready(); + return; + } + SCOPED_ATTACH_TASK(state); + COUNTER_UPDATE(_spill_probe_blocks, blocks.size()); + while (!blocks.empty() && !state->is_cancelled()) { + auto block = std::move(blocks.back()); + blocks.pop_back(); + if (_spill_status_ok) { + auto st = spilling_stream->spill_block(block, false); + if (!st.ok()) { + std::unique_lock lock(_spill_lock); + _spill_status_ok = false; + _spill_status = std::move(st); + break; + } + COUNTER_UPDATE(_spill_probe_rows, block.rows()); + } else { + break; + } } - COUNTER_UPDATE(_spill_probe_rows, block.rows()); - } else { - break; - } - } - --_spilling_task_count; + --_spilling_task_count; - if (_spilling_task_count == 0) { - std::unique_lock lock(_spill_lock); - _dependency->set_ready(); - } - }); + if (_spilling_task_count == 0) { + std::unique_lock lock(_spill_lock); + _dependency->set_ready(); + } + }); } else { --_spilling_task_count; if (_spilling_task_count == 0) { @@ -296,7 +312,14 @@ Status PartitionedHashJoinProbeLocalState::recovery_build_blocks_from_disk(Runti auto& mutable_block = _shared_state->partitioned_build_blocks[partition_index]; DCHECK(mutable_block != nullptr); - auto read_func = [this, state, &spilled_stream, &mutable_block] { + auto execution_context = state->get_task_execution_context(); + _shared_state_holder = _shared_state->shared_from_this(); + auto read_func = [this, state, &spilled_stream, &mutable_block, execution_context] { + auto execution_context_lock = execution_context.lock(); + if (!execution_context_lock) { + LOG(INFO) << "execution_context released, maybe query was cancelled."; + return; + } Defer defer([this] { --_spilling_task_count; }); (void)state; // avoid ut compile error SCOPED_ATTACH_TASK(state); @@ -363,7 +386,14 @@ Status PartitionedHashJoinProbeLocalState::recovery_probe_blocks_from_disk(Runti auto& blocks = _probe_blocks[partition_index]; /// TODO: maybe recovery more blocks each time. - auto read_func = [this, state, &spilled_stream, &blocks] { + auto execution_context = state->get_task_execution_context(); + _shared_state_holder = _shared_state->shared_from_this(); + auto read_func = [this, execution_context, state, &spilled_stream, &blocks] { + auto execution_context_lock = execution_context.lock(); + if (!execution_context_lock) { + LOG(INFO) << "execution_context released, maybe query was cancelled."; + return; + } Defer defer([this] { --_spilling_task_count; }); (void)state; // avoid ut compile error SCOPED_ATTACH_TASK(state); diff --git a/be/src/pipeline/exec/partitioned_hash_join_probe_operator.h b/be/src/pipeline/exec/partitioned_hash_join_probe_operator.h index adbaf19314f3f7..7337017fde661d 100644 --- a/be/src/pipeline/exec/partitioned_hash_join_probe_operator.h +++ b/be/src/pipeline/exec/partitioned_hash_join_probe_operator.h @@ -80,6 +80,11 @@ class PartitionedHashJoinProbeLocalState final std::vector> _partitioned_blocks; std::map> _probe_blocks; + /// Resources in shared state will be released when the operator is closed, + /// but there may be asynchronous spilling tasks at this time, which can lead to conflicts. + /// So, we need hold the pointer of shared state. + std::shared_ptr _shared_state_holder; + std::vector _probe_spilling_streams; std::unique_ptr _partitioner; diff --git a/be/src/pipeline/exec/partitioned_hash_join_sink_operator.cpp b/be/src/pipeline/exec/partitioned_hash_join_sink_operator.cpp index 20b3531c0cd77c..a63f22c1329289 100644 --- a/be/src/pipeline/exec/partitioned_hash_join_sink_operator.cpp +++ b/be/src/pipeline/exec/partitioned_hash_join_sink_operator.cpp @@ -74,7 +74,14 @@ Status PartitionedHashJoinSinkLocalState::revoke_memory(RuntimeState* state) { auto* spill_io_pool = ExecEnv::GetInstance()->spill_stream_mgr()->get_async_task_thread_pool(); DCHECK(spill_io_pool != nullptr); - auto st = spill_io_pool->submit_func([this, state, spilling_stream, i] { + auto execution_context = state->get_task_execution_context(); + _shared_state_holder = _shared_state->shared_from_this(); + auto st = spill_io_pool->submit_func([this, execution_context, state, spilling_stream, i] { + auto execution_context_lock = execution_context.lock(); + if (!execution_context_lock) { + LOG(INFO) << "execution_context released, maybe query was cancelled."; + return; + } (void)state; // avoid ut compile error SCOPED_ATTACH_TASK(state); _spill_to_disk(i, spilling_stream); diff --git a/be/src/pipeline/exec/partitioned_hash_join_sink_operator.h b/be/src/pipeline/exec/partitioned_hash_join_sink_operator.h index f2d5ca3e1403a1..44081bb0caae07 100644 --- a/be/src/pipeline/exec/partitioned_hash_join_sink_operator.h +++ b/be/src/pipeline/exec/partitioned_hash_join_sink_operator.h @@ -66,6 +66,11 @@ class PartitionedHashJoinSinkLocalState Status _spill_status; std::mutex _spill_status_lock; + /// Resources in shared state will be released when the operator is closed, + /// but there may be asynchronous spilling tasks at this time, which can lead to conflicts. + /// So, we need hold the pointer of shared state. + std::shared_ptr _shared_state_holder; + std::unique_ptr _partitioner; RuntimeProfile::Counter* _partition_timer = nullptr; diff --git a/be/src/pipeline/exec/spill_sort_sink_operator.cpp b/be/src/pipeline/exec/spill_sort_sink_operator.cpp index 15edcf3c8d1f12..6e70ba5fcaf9e5 100644 --- a/be/src/pipeline/exec/spill_sort_sink_operator.cpp +++ b/be/src/pipeline/exec/spill_sort_sink_operator.cpp @@ -214,11 +214,20 @@ Status SpillSortSinkLocalState::revoke_memory(RuntimeState* state) { if (!_eos) { Base::_dependency->Dependency::block(); } + + auto execution_context = state->get_task_execution_context(); + _shared_state_holder = _shared_state->shared_from_this(); status = ExecEnv::GetInstance() ->spill_stream_mgr() ->get_spill_io_thread_pool(_spilling_stream->get_spill_root_dir()) - ->submit_func([this, state, &parent] { + ->submit_func([this, state, &parent, execution_context] { + auto execution_context_lock = execution_context.lock(); + if (!execution_context_lock) { + LOG(INFO) << "execution_context released, maybe query was cancelled."; + return Status::OK(); + } + SCOPED_ATTACH_TASK(state); Defer defer {[&]() { if (!_shared_state->sink_status.ok()) { diff --git a/be/src/pipeline/exec/spill_sort_sink_operator.h b/be/src/pipeline/exec/spill_sort_sink_operator.h index aba85abece3215..ae5a3bcb8c7d83 100644 --- a/be/src/pipeline/exec/spill_sort_sink_operator.h +++ b/be/src/pipeline/exec/spill_sort_sink_operator.h @@ -48,6 +48,11 @@ class SpillSortSinkLocalState : public PipelineXSpillSinkLocalState _shared_state_holder; + std::unique_ptr _runtime_state; std::unique_ptr _internal_runtime_profile; RuntimeProfile::Counter* _partial_sort_timer = nullptr; diff --git a/be/src/pipeline/exec/spill_sort_source_operator.cpp b/be/src/pipeline/exec/spill_sort_source_operator.cpp index f82c015a17510b..c772b6b2797e65 100644 --- a/be/src/pipeline/exec/spill_sort_source_operator.cpp +++ b/be/src/pipeline/exec/spill_sort_source_operator.cpp @@ -90,7 +90,15 @@ Status SpillSortLocalState::initiate_merge_sort_spill_streams(RuntimeState* stat } }}; - auto spill_func = [this, state, &parent] { + auto execution_context = state->get_task_execution_context(); + _shared_state_holder = _shared_state->shared_from_this(); + auto spill_func = [this, state, &parent, execution_context] { + auto execution_context_lock = execution_context.lock(); + if (!execution_context_lock) { + LOG(INFO) << "execution_context released, maybe query was cancelled."; + return Status::OK(); + } + SCOPED_TIMER(_spill_merge_sort_timer); SCOPED_ATTACH_TASK(state); Defer defer {[&]() { diff --git a/be/src/pipeline/exec/spill_sort_source_operator.h b/be/src/pipeline/exec/spill_sort_source_operator.h index ae76c4e1b58191..8132dd5a56c9e3 100644 --- a/be/src/pipeline/exec/spill_sort_source_operator.h +++ b/be/src/pipeline/exec/spill_sort_source_operator.h @@ -57,6 +57,11 @@ class SpillSortLocalState final : public PipelineXSpillLocalState _shared_state_holder; + int64_t _external_sort_bytes_threshold = 134217728; // 128M std::vector _current_merging_streams; std::unique_ptr _merger; diff --git a/be/src/pipeline/pipeline_fragment_context.cpp b/be/src/pipeline/pipeline_fragment_context.cpp index 8600a98b32cc15..0e075b90ebca64 100644 --- a/be/src/pipeline/pipeline_fragment_context.cpp +++ b/be/src/pipeline/pipeline_fragment_context.cpp @@ -459,7 +459,11 @@ void PipelineFragmentContext::trigger_report_if_necessary() { // _runtime_state->load_channel_profile()->compute_time_in_profile(); // TODO load channel profile add timer _runtime_state->load_channel_profile()->pretty_print(&ss); } - VLOG_FILE << ss.str(); + + VLOG_FILE << "Query " << print_id(this->get_query_id()) << " fragment " + << this->get_fragment_id() << " instance " + << print_id(this->get_fragment_instance_id()) << " profile:\n" + << ss.str(); } auto st = send_report(false); if (!st.ok()) { @@ -917,7 +921,9 @@ void PipelineFragmentContext::_close_fragment_instance() { if (_runtime_state->load_channel_profile()) { _runtime_state->load_channel_profile()->pretty_print(&ss); } - LOG(INFO) << ss.str(); + + LOG_INFO("Query {} fragment {} instance {} profile:\n {}", print_id(this->_query_id), + this->_fragment_id, print_id(this->get_fragment_instance_id()), ss.str()); } // all submitted tasks done _exec_env->fragment_mgr()->remove_pipeline_context( diff --git a/be/src/pipeline/pipeline_x/dependency.h b/be/src/pipeline/pipeline_x/dependency.h index fe95c1c4470058..622c175edcbc6a 100644 --- a/be/src/pipeline/pipeline_x/dependency.h +++ b/be/src/pipeline/pipeline_x/dependency.h @@ -57,6 +57,8 @@ static constexpr auto TIME_UNIT_DEPENDENCY_LOG = 30 * 1000L * 1000L * 1000L; static_assert(TIME_UNIT_DEPENDENCY_LOG < SLOW_DEPENDENCY_THRESHOLD); struct BasicSharedState { + ENABLE_FACTORY_CREATOR(BasicSharedState) + template TARGET* cast() { DCHECK(dynamic_cast(this)) @@ -184,7 +186,9 @@ class Dependency : public std::enable_shared_from_this { std::mutex _always_ready_lock; }; -struct FakeSharedState final : public BasicSharedState {}; +struct FakeSharedState final : public BasicSharedState { + ENABLE_FACTORY_CREATOR(FakeSharedState) +}; struct FakeDependency final : public Dependency { public: @@ -325,6 +329,7 @@ class RuntimeFilterDependency final : public Dependency { }; struct AggSharedState : public BasicSharedState { + ENABLE_FACTORY_CREATOR(AggSharedState) public: AggSharedState() { agg_data = std::make_unique(); @@ -418,8 +423,10 @@ struct AggSharedState : public BasicSharedState { }; struct AggSpillPartition; -struct PartitionedAggSharedState : public BasicSharedState { -public: +struct PartitionedAggSharedState : public BasicSharedState, + public std::enable_shared_from_this { + ENABLE_FACTORY_CREATOR(PartitionedAggSharedState) + PartitionedAggSharedState() = default; ~PartitionedAggSharedState() override = default; @@ -480,11 +487,15 @@ struct AggSpillPartition { }; using AggSpillPartitionSPtr = std::shared_ptr; struct SortSharedState : public BasicSharedState { + ENABLE_FACTORY_CREATOR(SortSharedState) public: std::unique_ptr sorter; }; -struct SpillSortSharedState : public BasicSharedState { +struct SpillSortSharedState : public BasicSharedState, + public std::enable_shared_from_this { + ENABLE_FACTORY_CREATOR(SpillSortSharedState) + SpillSortSharedState() = default; ~SpillSortSharedState() override = default; @@ -512,6 +523,8 @@ struct SpillSortSharedState : public BasicSharedState { }; struct UnionSharedState : public BasicSharedState { + ENABLE_FACTORY_CREATOR(UnionSharedState) + public: UnionSharedState(int child_count = 1) : data_queue(child_count), _child_count(child_count) {}; int child_count() const { return _child_count; } @@ -527,6 +540,8 @@ struct MultiCastSharedState : public BasicSharedState { }; struct AnalyticSharedState : public BasicSharedState { + ENABLE_FACTORY_CREATOR(AnalyticSharedState) + public: AnalyticSharedState() = default; @@ -560,6 +575,7 @@ struct JoinSharedState : public BasicSharedState { }; struct HashJoinSharedState : public JoinSharedState { + ENABLE_FACTORY_CREATOR(HashJoinSharedState) // mark the join column whether support null eq std::vector is_null_safe_eq_join; // mark the build hash table whether it needs to store null value @@ -576,13 +592,18 @@ struct HashJoinSharedState : public JoinSharedState { bool probe_ignore_null = false; }; -struct PartitionedHashJoinSharedState : public HashJoinSharedState { +struct PartitionedHashJoinSharedState + : public HashJoinSharedState, + public std::enable_shared_from_this { + ENABLE_FACTORY_CREATOR(PartitionedHashJoinSharedState) + std::vector> partitioned_build_blocks; std::vector spilled_streams; bool need_to_spill = false; }; struct NestedLoopJoinSharedState : public JoinSharedState { + ENABLE_FACTORY_CREATOR(NestedLoopJoinSharedState) // if true, left child has no more rows to process bool left_side_eos = false; // Visited flags for each row in build side. @@ -592,6 +613,7 @@ struct NestedLoopJoinSharedState : public JoinSharedState { }; struct PartitionSortNodeSharedState : public BasicSharedState { + ENABLE_FACTORY_CREATOR(PartitionSortNodeSharedState) public: std::queue blocks_buffer; std::mutex buffer_mutex; @@ -610,6 +632,7 @@ class AsyncWriterDependency final : public Dependency { }; struct SetSharedState : public BasicSharedState { + ENABLE_FACTORY_CREATOR(SetSharedState) public: /// default init vectorized::Block build_block; // build to source diff --git a/be/src/pipeline/pipeline_x/operator.cpp b/be/src/pipeline/pipeline_x/operator.cpp index 6ee9ccb13c4d89..989b1ee00a517d 100644 --- a/be/src/pipeline/pipeline_x/operator.cpp +++ b/be/src/pipeline/pipeline_x/operator.cpp @@ -167,7 +167,7 @@ Status OperatorXBase::close(RuntimeState* state) { } void PipelineXLocalStateBase::clear_origin_block() { - _origin_block.clear_column_data(_parent->_row_descriptor.num_materialized_slots()); + _origin_block.clear_column_data(_parent->intermediate_row_desc().num_materialized_slots()); } Status OperatorXBase::do_projections(RuntimeState* state, vectorized::Block* origin_block, @@ -299,7 +299,7 @@ std::shared_ptr DataSinkOperatorX::create_shar return nullptr; } else { std::shared_ptr ss = nullptr; - ss.reset(new typename LocalStateType::SharedStateType()); + ss = LocalStateType::SharedStateType::create_shared(); ss->id = operator_id(); for (auto& dest : dests_id()) { ss->related_op_ids.insert(dest); diff --git a/be/src/pipeline/pipeline_x/operator.h b/be/src/pipeline/pipeline_x/operator.h index 56991d43105fd5..c375efb924dcbc 100644 --- a/be/src/pipeline/pipeline_x/operator.h +++ b/be/src/pipeline/pipeline_x/operator.h @@ -328,8 +328,8 @@ class OperatorXBase : public OperatorBase { int _parallel_tasks = 0; //_keep_origin is used to avoid copying during projection, - // currently set to true only in the nestloop join. - bool _keep_origin = false; + // currently set to false only in the nestloop join. + bool _keep_origin = true; }; template diff --git a/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.cpp b/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.cpp index 5ed5dc8d59808e..adab6408dde7f0 100644 --- a/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.cpp +++ b/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.cpp @@ -990,7 +990,7 @@ Status PipelineXFragmentContext::_create_operator(ObjectPool* pool, const TPlanN break; } case TPlanNodeType::AGGREGATION_NODE: { - if (tnode.agg_node.aggregate_functions.empty() && + if (tnode.agg_node.aggregate_functions.empty() && !_runtime_state->enable_agg_spill() && request.query_options.__isset.enable_distinct_streaming_aggregation && request.query_options.enable_distinct_streaming_aggregation) { op.reset(new DistinctStreamingAggOperatorX(pool, next_operator_id(), tnode, descs)); @@ -1351,7 +1351,9 @@ void PipelineXFragmentContext::_close_fragment_instance() { if (_runtime_state->load_channel_profile()) { _runtime_state->load_channel_profile()->pretty_print(&ss); } - LOG(INFO) << ss.str(); + + LOG_INFO("Query {} fragment {} profile:\n {}", print_id(this->_query_id), + this->_fragment_id, ss.str()); } // all submitted tasks done _exec_env->fragment_mgr()->remove_pipeline_context( diff --git a/be/src/runtime/exec_env.h b/be/src/runtime/exec_env.h index 8a71ca5ba44648..f3be4d8033363e 100644 --- a/be/src/runtime/exec_env.h +++ b/be/src/runtime/exec_env.h @@ -214,6 +214,9 @@ class ExecEnv { MemTableMemoryLimiter* memtable_memory_limiter() { return _memtable_memory_limiter.get(); } WalManager* wal_mgr() { return _wal_manager.get(); } #ifdef BE_TEST + void set_tmp_file_dir(std::unique_ptr tmp_file_dirs) { + this->_tmp_file_dirs = std::move(tmp_file_dirs); + } void set_ready() { this->_s_ready = true; } void set_not_ready() { this->_s_ready = false; } void set_memtable_memory_limiter(MemTableMemoryLimiter* limiter) { diff --git a/be/src/service/doris_main.cpp b/be/src/service/doris_main.cpp index 5960db992143b5..a218c7be6f18a7 100644 --- a/be/src/service/doris_main.cpp +++ b/be/src/service/doris_main.cpp @@ -619,6 +619,8 @@ int main(int argc, char** argv) { LOG(INFO) << "Be server stopped"; brpc_service.reset(nullptr); LOG(INFO) << "Brpc service stopped"; + service.reset(); + LOG(INFO) << "Backend Service stopped"; exec_env->destroy(); doris::ThreadLocalHandle::del_thread_local_if_count_is_zero(); LOG(INFO) << "Doris main exited."; diff --git a/be/src/util/bitmap_value.h b/be/src/util/bitmap_value.h index 6bc88d4f0a1b69..b7a046921b5d6d 100644 --- a/be/src/util/bitmap_value.h +++ b/be/src/util/bitmap_value.h @@ -1208,7 +1208,7 @@ class BitmapValue { } } - BitmapValue(BitmapValue&& other) { + BitmapValue(BitmapValue&& other) noexcept { _type = other._type; switch (other._type) { case EMPTY: @@ -1265,7 +1265,7 @@ class BitmapValue { return buf; } - BitmapValue& operator=(BitmapValue&& other) { + BitmapValue& operator=(BitmapValue&& other) noexcept { if (this == &other) { return *this; } diff --git a/be/src/util/quantile_state.h b/be/src/util/quantile_state.h index c3b8cf83001935..e2b782cf268fbd 100644 --- a/be/src/util/quantile_state.h +++ b/be/src/util/quantile_state.h @@ -46,6 +46,11 @@ class QuantileState { QuantileState(); explicit QuantileState(float compression); explicit QuantileState(const Slice& slice); + QuantileState& operator=(const QuantileState& other) noexcept = default; + QuantileState(const QuantileState& other) noexcept = default; + QuantileState& operator=(QuantileState&& other) noexcept = default; + QuantileState(QuantileState&& other) noexcept = default; + void set_compression(float compression); bool deserialize(const Slice& slice); size_t serialize(uint8_t* dst) const; diff --git a/be/src/vec/aggregate_functions/aggregate_function_simple_factory.cpp b/be/src/vec/aggregate_functions/aggregate_function_simple_factory.cpp index c33b8b50609635..2069e8ae8a49de 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_simple_factory.cpp +++ b/be/src/vec/aggregate_functions/aggregate_function_simple_factory.cpp @@ -31,6 +31,7 @@ void register_aggregate_function_combinator_distinct(AggregateFunctionSimpleFact void register_aggregate_function_combinator_foreach(AggregateFunctionSimpleFactory& factory); void register_aggregate_function_sum(AggregateFunctionSimpleFactory& factory); +void register_aggregate_function_sum0(AggregateFunctionSimpleFactory& factory); void register_aggregate_function_minmax(AggregateFunctionSimpleFactory& factory); void register_aggregate_function_min_by(AggregateFunctionSimpleFactory& factory); void register_aggregate_function_max_by(AggregateFunctionSimpleFactory& factory); @@ -70,6 +71,7 @@ AggregateFunctionSimpleFactory& AggregateFunctionSimpleFactory::instance() { static AggregateFunctionSimpleFactory instance; std::call_once(oc, [&]() { register_aggregate_function_sum(instance); + register_aggregate_function_sum0(instance); register_aggregate_function_minmax(instance); register_aggregate_function_min_by(instance); register_aggregate_function_max_by(instance); diff --git a/be/src/vec/aggregate_functions/aggregate_function_sum.cpp b/be/src/vec/aggregate_functions/aggregate_function_sum.cpp index 3ee7dc6ff48333..e0676957d467df 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_sum.cpp +++ b/be/src/vec/aggregate_functions/aggregate_function_sum.cpp @@ -31,4 +31,8 @@ void register_aggregate_function_sum(AggregateFunctionSimpleFactory& factory) { "sum_decimal256", creator_with_type::creator); } +void register_aggregate_function_sum0(AggregateFunctionSimpleFactory& factory) { + factory.register_function_both("sum0", creator_with_type::creator); +} + } // namespace doris::vectorized diff --git a/be/src/vec/aggregate_functions/aggregate_function_uniq.h b/be/src/vec/aggregate_functions/aggregate_function_uniq.h index 72be9e0183397c..2e8855134ebd31 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_uniq.h +++ b/be/src/vec/aggregate_functions/aggregate_function_uniq.h @@ -36,7 +36,6 @@ #include "vec/common/assert_cast.h" #include "vec/common/hash_table/hash.h" #include "vec/common/hash_table/phmap_fwd_decl.h" -#include "vec/common/sip_hash.h" #include "vec/common/string_ref.h" #include "vec/common/uint128.h" #include "vec/core/types.h" @@ -64,17 +63,14 @@ template struct AggregateFunctionUniqExactData { static constexpr bool is_string_key = std::is_same_v; using Key = std::conditional_t; - using Hash = std::conditional_t>; + using Hash = HashCRC32; using Set = flat_hash_set; // TODO: replace SipHash with xxhash to speed up static UInt128 ALWAYS_INLINE get_key(const StringRef& value) { - UInt128 key; - SipHash hash; - hash.update(value.data, value.size); - hash.get128(key.low, key.high); - return key; + auto hash_value = XXH_INLINE_XXH128(value.data, value.size, 0); + return UInt128 {hash_value.high64, hash_value.low64}; } Set set; 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 6d4f82e845b60e..99a301294dc727 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 @@ -19,6 +19,7 @@ #include +#include "common/exception.h" #include "common/status.h" #include "vec/columns/column.h" #include "vec/columns/column_object.h" @@ -95,6 +96,28 @@ void DataTypeObjectSerDe::read_one_cell_from_jsonb(IColumn& column, const JsonbV variant.insert(field); } +void DataTypeObjectSerDe::write_column_to_arrow(const IColumn& column, const NullMap* null_map, + arrow::ArrayBuilder* array_builder, int start, + int end, const cctz::time_zone& ctz) const { + const auto* var = check_and_get_column(column); + auto& builder = assert_cast(*array_builder); + for (size_t i = start; i < end; ++i) { + if (null_map && (*null_map)[i]) { + checkArrowStatus(builder.AppendNull(), column.get_name(), + array_builder->type()->name()); + } else { + std::string serialized_value; + if (!var->serialize_one_row_to_string(i, &serialized_value)) { + throw doris::Exception(ErrorCode::INTERNAL_ERROR, "Failed to serialize variant {}", + var->dump_structure()); + } + checkArrowStatus(builder.Append(serialized_value.data(), + static_cast(serialized_value.size())), + column.get_name(), array_builder->type()->name()); + } + } +} + } // namespace vectorized } // namespace doris \ No newline at end of file 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 4d78d67352ddca..c589a1af92a0aa 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 @@ -73,10 +73,7 @@ class DataTypeObjectSerDe : public DataTypeSerDe { void write_column_to_arrow(const IColumn& column, const NullMap* null_map, arrow::ArrayBuilder* array_builder, int start, int 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 { throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, diff --git a/be/src/vec/exec/join/vjoin_node_base.cpp b/be/src/vec/exec/join/vjoin_node_base.cpp index 9b954811ee9a02..a9e25e7626bbd8 100644 --- a/be/src/vec/exec/join/vjoin_node_base.cpp +++ b/be/src/vec/exec/join/vjoin_node_base.cpp @@ -185,8 +185,7 @@ Status VJoinNodeBase::_build_output_block(Block* origin_block, Block* output_blo // and you could see a 'todo' in the Thrift definition. // Here, we have refactored it, but considering upgrade compatibility, we still need to retain the old code. if (!output_block->mem_reuse()) { - MutableBlock tmp(VectorizedUtils::create_columns_with_type_and_name(row_desc())); - output_block->swap(tmp.to_block()); + output_block->swap(origin_block->clone_empty()); } output_block->swap(*origin_block); return Status::OK(); diff --git a/be/src/vec/functions/uuid_numeric.cpp b/be/src/vec/functions/uuid_numeric.cpp index 012d0d488a1b9b..221cbd7d34c337 100644 --- a/be/src/vec/functions/uuid_numeric.cpp +++ b/be/src/vec/functions/uuid_numeric.cpp @@ -22,10 +22,7 @@ #include "common/status.h" #include "runtime/large_int_value.h" #include "vec/columns/column_vector.h" -#include "vec/columns/columns_number.h" #include "vec/common/hash_table/hash.h" -#include "vec/common/sip_hash.h" -#include "vec/common/uint128.h" #include "vec/core/block.h" #include "vec/core/types.h" #include "vec/data_types/data_type_number.h" @@ -100,7 +97,6 @@ class UuidNumeric : public IFunction { } // TODO(zhiqiang): May be override open function? - Status execute_impl(FunctionContext* /*context*/, Block& block, const ColumnNumbers& /*arguments*/, size_t result, size_t input_rows_count) const override { @@ -140,14 +136,16 @@ class UuidNumeric : public IFunction { UInt64 randomSeed() const { struct timespec times {}; + clock_gettime(CLOCK_MONOTONIC, ×); /// Not cryptographically secure as time, pid and stack address can be predictable. - - SipHash hash; - hash.update(times.tv_nsec); - hash.update(times.tv_sec); - hash.update((uintptr_t)pthread_self()); - - return hash.get64(); + auto ret = HashUtil::xxHash64WithSeed(reinterpret_cast(×.tv_nsec), + sizeof(times.tv_nsec), 0); + ret = HashUtil::xxHash64WithSeed(reinterpret_cast(×.tv_sec), + sizeof(times.tv_sec), ret); + ret = HashUtil::xxHash64WithSeed(reinterpret_cast((uintptr_t)pthread_self()), + sizeof(pthread_t), ret); + + return ret; } }; diff --git a/be/src/vec/olap/vgeneric_iterators.cpp b/be/src/vec/olap/vgeneric_iterators.cpp index 2d45c884a78e41..c973a7131f9ff7 100644 --- a/be/src/vec/olap/vgeneric_iterators.cpp +++ b/be/src/vec/olap/vgeneric_iterators.cpp @@ -422,9 +422,9 @@ Status VUnionIterator::current_block_row_locations(std::vector* loc RowwiseIteratorUPtr new_merge_iterator(std::vector&& inputs, int sequence_id_idx, bool is_unique, bool is_reverse, uint64_t* merged_rows) { - if (inputs.size() == 1) { - return std::move(inputs[0]); - } + // when the size of inputs is 1, we also need to use VMergeIterator, because the + // next_block_view function only be implemented in VMergeIterator. The reason why + // the size of inputs is 1 is that the segment was filtered out by zone map or others. return std::make_unique(std::move(inputs), sequence_id_idx, is_unique, is_reverse, merged_rows); } diff --git a/be/test/olap/rowset/segment_v2/inverted_index_array_test.cpp b/be/test/olap/rowset/segment_v2/inverted_index_array_test.cpp new file mode 100644 index 00000000000000..74e9827db25199 --- /dev/null +++ b/be/test/olap/rowset/segment_v2/inverted_index_array_test.cpp @@ -0,0 +1,225 @@ +// 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 +#include +#include +#include +#include +#include + +#include +#include + +#include "gtest/gtest_pred_impl.h" +#include "io/fs/file_writer.h" +#include "io/fs/local_file_system.h" +#include "olap/rowset/segment_v2/inverted_index_compound_reader.h" +#include "olap/rowset/segment_v2/inverted_index_file_writer.h" +#include "olap/rowset/segment_v2/inverted_index_fs_directory.h" +#include "olap/rowset/segment_v2/inverted_index_writer.h" +#include "olap/rowset/segment_v2/zone_map_index.h" +#include "olap/tablet_schema.h" +#include "olap/tablet_schema_helper.h" +#include "runtime/exec_env.h" +#include "util/slice.h" +#include "vec/columns/column_array.h" +#include "vec/columns/column_nullable.h" +#include "vec/columns/column_string.h" +#include "vec/common/pod_array_fwd.h" +#include "vec/core/field.h" +#include "vec/core/types.h" +#include "vec/data_types/data_type_array.h" +#include "vec/data_types/data_type_factory.hpp" +#include "vec/olap/olap_data_convertor.h" + +using namespace lucene::index; +using doris::segment_v2::InvertedIndexFileWriter; + +namespace doris { +namespace segment_v2 { + +class InvertedIndexArrayTest : public testing::Test { +public: + const std::string kTestDir = "./ut_dir/inverted_index_array_test"; + + void check_terms_stats(string dir_str, string file_str) { + auto fs = io::global_local_filesystem(); + std::unique_ptr reader = std::make_unique( + DorisFSDirectoryFactory::getDirectory(fs, dir_str.c_str()), file_str.c_str(), 4096); + std::cout << "Term statistics for " << file_str << std::endl; + std::cout << "==================================" << std::endl; + lucene::store::Directory* dir = reader.get(); + + IndexReader* r = IndexReader::open(dir); + + printf("Max Docs: %d\n", r->maxDoc()); + printf("Num Docs: %d\n", r->numDocs()); + + TermEnum* te = r->terms(); + int32_t nterms; + for (nterms = 0; te->next(); nterms++) { + /* empty */ + std::string token = + lucene_wcstoutf8string(te->term(false)->text(), te->term(false)->textLength()); + + printf("Term: %s ", token.c_str()); + printf("Freq: %d\n", te->docFreq()); + } + printf("Term count: %d\n\n", nterms); + te->close(); + _CLLDELETE(te); + + r->close(); + _CLLDELETE(r); + reader->close(); + } + + void SetUp() override { + auto st = io::global_local_filesystem()->delete_directory(kTestDir); + ASSERT_TRUE(st.ok()) << st; + st = io::global_local_filesystem()->create_directory(kTestDir); + ASSERT_TRUE(st.ok()) << st; + config::enable_write_index_searcher_cache = false; + std::vector paths; + paths.emplace_back(kTestDir, 1024); + auto tmp_file_dirs = std::make_unique(paths); + st = tmp_file_dirs->init(); + if (!st.OK()) { + std::cout << "init tmp file dirs error:" << st.to_string() << std::endl; + return; + } + ExecEnv::GetInstance()->set_tmp_file_dir(std::move(tmp_file_dirs)); + } + void TearDown() override { + EXPECT_TRUE(io::global_local_filesystem()->delete_directory(kTestDir).ok()); + } + + void test_string(std::string testname, Field* field) { + EXPECT_TRUE(field->type() == FieldType::OLAP_FIELD_TYPE_ARRAY); + std::string filename = kTestDir + "/" + testname; + auto fs = io::global_local_filesystem(); + + io::FileWriterPtr file_writer; + EXPECT_TRUE(fs->create_file(filename, &file_writer).ok()); + auto index_meta_pb = std::make_unique(); + index_meta_pb->set_index_type(IndexType::INVERTED); + index_meta_pb->set_index_id(26033); + index_meta_pb->set_index_name("index_inverted_arr1"); + index_meta_pb->clear_col_unique_id(); + index_meta_pb->add_col_unique_id(0); + + TabletIndex idx_meta; + idx_meta.index_type(); + idx_meta.init_from_pb(*index_meta_pb.get()); + auto index_file_writer = std::make_unique( + fs, file_writer->path().parent_path(), file_writer->path().filename(), + InvertedIndexStorageFormatPB::V1); + std::unique_ptr _inverted_index_builder = nullptr; + EXPECT_EQ(InvertedIndexColumnWriter::create(field, &_inverted_index_builder, + index_file_writer.get(), &idx_meta), + Status::OK()); + vectorized::PaddedPODArray _slice; + _slice.resize(5); + + vectorized::Array a1, a2; + a1.push_back("amory"); + a1.push_back("doris"); + a2.push_back(vectorized::Null()); + a2.push_back("amory"); + a2.push_back("commiter"); + + vectorized::DataTypePtr s1 = std::make_shared( + std::make_shared()); + vectorized::DataTypePtr au = std::make_shared(s1); + vectorized::MutableColumnPtr col = au->create_column(); + col->insert(a1); + col->insert(a2); + vectorized::ColumnPtr column_array = std::move(col); + vectorized::ColumnWithTypeAndName type_and_name(column_array, au, "arr1"); + + vectorized::PaddedPODArray _offsets; + _offsets.reserve(3); + _offsets.emplace_back(0); + _offsets.emplace_back(2); + _offsets.emplace_back(5); + const uint8_t* offsets_ptr = (const uint8_t*)(_offsets.data()); + + auto* col_arr = assert_cast(column_array.get()); + const vectorized::UInt8* nested_null_map = + assert_cast(col_arr->get_data_ptr().get()) + ->get_null_map_data() + .data(); + auto* col_arr_str = assert_cast( + assert_cast(col_arr->get_data_ptr().get()) + ->get_nested_column_ptr() + .get()); + const char* char_data = (const char*)(col_arr_str->get_chars().data()); + const vectorized::ColumnString::Offset* offset_cur = col_arr_str->get_offsets().data(); + const vectorized::ColumnString::Offset* offset_end = offset_cur + 5; + + Slice* slice = _slice.data(); + size_t string_offset = *(offset_cur - 1); + const vectorized::UInt8* nullmap_cur = nested_null_map; + while (offset_cur != offset_end) { + if (!*nullmap_cur) { + slice->data = const_cast(char_data + string_offset); + slice->size = *offset_cur - string_offset; + } else { + slice->data = nullptr; + slice->size = 0; + } + string_offset = *offset_cur; + ++nullmap_cur; + ++slice; + ++offset_cur; + } + + auto field_size = field->get_sub_field(0)->size(); + Status st = _inverted_index_builder->add_array_values( + field_size, reinterpret_cast(_slice.data()), + reinterpret_cast(nested_null_map), offsets_ptr, 2); + EXPECT_EQ(st, Status::OK()); + EXPECT_EQ(_inverted_index_builder->finish(), Status::OK()); + EXPECT_EQ(index_file_writer->close(), Status::OK()); + + { + std::cout << "dir: " << file_writer->path().parent_path().string() << std::endl; + string idx_file_name = file_writer->path().filename().string() + "_26033.idx"; + std::cout << "file: " << file_writer->path().filename().string() << std::endl; + check_terms_stats(file_writer->path().parent_path().string(), idx_file_name); + } + } +}; + +TEST_F(InvertedIndexArrayTest, ArrayString) { + TabletColumn arrayTabletColumn; + arrayTabletColumn.set_unique_id(0); + arrayTabletColumn.set_name("arr1"); + arrayTabletColumn.set_type(FieldType::OLAP_FIELD_TYPE_ARRAY); + TabletColumn arraySubColumn; + arraySubColumn.set_unique_id(1); + arraySubColumn.set_name("arr_sub_string"); + arraySubColumn.set_type(FieldType::OLAP_FIELD_TYPE_STRING); + arrayTabletColumn.add_sub_column(arraySubColumn); + Field* field = FieldFactory::create(arrayTabletColumn); + test_string("InvertedIndexArray", field); + delete field; +} + +} // namespace segment_v2 +} // namespace doris diff --git a/cloud/src/meta-service/http_encode_key.cpp b/cloud/src/meta-service/http_encode_key.cpp index aa0c3c4dac95fc..f70f2accb500bd 100644 --- a/cloud/src/meta-service/http_encode_key.cpp +++ b/cloud/src/meta-service/http_encode_key.cpp @@ -140,7 +140,8 @@ static std::unordered_map{p}.get()); } , parse}} , {"TxnIndexKey", {{"instance_id", "txn_id"}, [](param_type& p) { return txn_index_key(KeyInfoSetter{p}.get()); } , parse}} , {"TxnRunningKey", {{"instance_id", "db_id", "txn_id"}, [](param_type& p) { return txn_running_key(KeyInfoSetter{p}.get()); } , parse}} , - {"VersionKey", {{"instance_id", "db_id", "tbl_id", "partition_id"}, [](param_type& p) { return version_key(KeyInfoSetter{p}.get()); } , parse}} , + {"PartitionVersionKey", {{"instance_id", "db_id", "tbl_id", "partition_id"}, [](param_type& p) { return partition_version_key(KeyInfoSetter{p}.get()); } , parse}} , + {"TableVersionKey", {{"instance_id", "db_id", "tbl_id"}, [](param_type& p) { return table_version_key(KeyInfoSetter{p}.get()); } , parse}} , {"MetaRowsetKey", {{"instance_id", "tablet_id", "version"}, [](param_type& p) { return meta_rowset_key(KeyInfoSetter{p}.get()); } , parse}} , {"MetaRowsetTmpKey", {{"instance_id", "txn_id", "tablet_id"}, [](param_type& p) { return meta_rowset_tmp_key(KeyInfoSetter{p}.get()); } , parse}} , {"MetaTabletKey", {{"instance_id", "table_id", "index_id", "part_id", "tablet_id"}, [](param_type& p) { return meta_tablet_key(KeyInfoSetter{p}.get()); } , parse}} , diff --git a/cloud/src/meta-service/keys.cpp b/cloud/src/meta-service/keys.cpp index f3e89dfbc66d1f..6e883681696a0d 100644 --- a/cloud/src/meta-service/keys.cpp +++ b/cloud/src/meta-service/keys.cpp @@ -43,7 +43,8 @@ namespace doris::cloud { [[maybe_unused]] static const char* TXN_KEY_INFIX_INDEX = "txn_index"; [[maybe_unused]] static const char* TXN_KEY_INFIX_RUNNING = "txn_running"; -[[maybe_unused]] static const char* VERSION_KEY_INFIX = "partition"; +[[maybe_unused]] static const char* PARTITION_VERSION_KEY_INFIX = "partition"; +[[maybe_unused]] static const char* TABLE_VERSION_KEY_INFIX = "table"; [[maybe_unused]] static const char* META_KEY_INFIX_ROWSET = "rowset"; [[maybe_unused]] static const char* META_KEY_INFIX_ROWSET_TMP = "rowset_tmp"; @@ -114,9 +115,9 @@ static void encode_prefix(const T& t, std::string* key) { InstanceKeyInfo, TxnLabelKeyInfo, TxnInfoKeyInfo, TxnIndexKeyInfo, TxnRunningKeyInfo, MetaRowsetKeyInfo, MetaRowsetTmpKeyInfo, MetaTabletKeyInfo, MetaTabletIdxKeyInfo, MetaSchemaKeyInfo, - MetaDeleteBitmapInfo, MetaDeleteBitmapUpdateLockInfo, MetaPendingDeleteBitmapInfo, VersionKeyInfo, + MetaDeleteBitmapInfo, MetaDeleteBitmapUpdateLockInfo, MetaPendingDeleteBitmapInfo, PartitionVersionKeyInfo, RecycleIndexKeyInfo, RecyclePartKeyInfo, RecycleRowsetKeyInfo, RecycleTxnKeyInfo, RecycleStageKeyInfo, - StatsTabletKeyInfo, + StatsTabletKeyInfo, TableVersionKeyInfo, JobTabletKeyInfo, JobRecycleKeyInfo, RLJobProgressKeyInfo, CopyJobKeyInfo, CopyFileKeyInfo, MetaRowsetSchemaKeyInfo, StorageVaultKeyInfo>); @@ -139,7 +140,8 @@ static void encode_prefix(const T& t, std::string* key) { || std::is_same_v || std::is_same_v) { encode_bytes(META_KEY_PREFIX, key); - } else if constexpr (std::is_same_v) { + } else if constexpr (std::is_same_v + || std::is_same_v) { encode_bytes(VERSION_KEY_PREFIX, key); } else if constexpr (std::is_same_v || std::is_same_v @@ -217,9 +219,22 @@ void txn_running_key(const TxnRunningKeyInfo& in, std::string* out) { // Version keys //============================================================================== -void version_key(const VersionKeyInfo& in, std::string* out) { +std::string version_key_prefix(std::string_view instance_id) { + std::string out; + encode_prefix(TableVersionKeyInfo {instance_id, 0, 0}, &out); + return out; +} + +void table_version_key(const TableVersionKeyInfo& in, std::string* out) { + encode_prefix(in, out); // 0x01 "version" ${instance_id} + encode_bytes(TABLE_VERSION_KEY_INFIX, out); // "table" + encode_int64(std::get<1>(in), out); // db_id + encode_int64(std::get<2>(in), out); // tbl_id +} + +void partition_version_key(const PartitionVersionKeyInfo& in, std::string* out) { encode_prefix(in, out); // 0x01 "version" ${instance_id} - encode_bytes(VERSION_KEY_INFIX, out); // "partition" + encode_bytes(PARTITION_VERSION_KEY_INFIX, out); // "partition" encode_int64(std::get<1>(in), out); // db_id encode_int64(std::get<2>(in), out); // tbl_id encode_int64(std::get<3>(in), out); // partition_id diff --git a/cloud/src/meta-service/keys.h b/cloud/src/meta-service/keys.h index edc9afcf4e8a78..483332a133c0b1 100644 --- a/cloud/src/meta-service/keys.h +++ b/cloud/src/meta-service/keys.h @@ -34,6 +34,7 @@ // 0x01 "txn" ${instance_id} "txn_running" ${db_id} ${txn_id} -> TxnRunningPB // // 0x01 "version" ${instance_id} "partition" ${db_id} ${tbl_id} ${partition_id} -> VersionPB +// 0x01 "version" ${instance_id} "table" ${db_id} ${tbl_id} -> int64 // // 0x01 "meta" ${instance_id} "rowset" ${tablet_id} ${version} -> RowsetMetaCloudPB // 0x01 "meta" ${instance_id} "rowset_tmp" ${txn_id} ${tablet_id} -> RowsetMetaCloudPB @@ -115,7 +116,7 @@ using TxnIndexKeyInfo = BasicKeyInfo<3 , std::tuple> using TxnRunningKeyInfo = BasicKeyInfo<5 , std::tuple>; // 0:instance_id 1:db_id 2:tbl_id 3:partition_id -using VersionKeyInfo = BasicKeyInfo<6 , std::tuple>; +using PartitionVersionKeyInfo = BasicKeyInfo<6 , std::tuple>; // 0:instance_id 1:tablet_id 2:version using MetaRowsetKeyInfo = BasicKeyInfo<7 , std::tuple>; @@ -181,6 +182,9 @@ using RLJobProgressKeyInfo = BasicKeyInfo<25, std::tuple>; +// 0:instance_id 1:db_id 2:table_id +using TableVersionKeyInfo = BasicKeyInfo<27, std::tuple>; + void instance_key(const InstanceKeyInfo& in, std::string* out); static inline std::string instance_key(const InstanceKeyInfo& in) { std::string s; instance_key(in, &s); return s; } @@ -197,8 +201,11 @@ static inline std::string txn_info_key(const TxnInfoKeyInfo& in) { std::string s static inline std::string txn_index_key(const TxnIndexKeyInfo& in) { std::string s; txn_index_key(in, &s); return s; } static inline std::string txn_running_key(const TxnRunningKeyInfo& in) { std::string s; txn_running_key(in, &s); return s; } -void version_key(const VersionKeyInfo& in, std::string* out); -static inline std::string version_key(const VersionKeyInfo& in) { std::string s; version_key(in, &s); return s; } +std::string version_key_prefix(std::string_view instance_id); +void partition_version_key(const PartitionVersionKeyInfo& in, std::string* out); +static inline std::string partition_version_key(const PartitionVersionKeyInfo& in) { std::string s; partition_version_key(in, &s); return s; } +void table_version_key(const TableVersionKeyInfo& in, std::string* out); +static inline std::string table_version_key(const TableVersionKeyInfo& in) { std::string s; table_version_key(in, &s); return s; } std::string meta_key_prefix(std::string_view instance_id); void meta_rowset_key(const MetaRowsetKeyInfo& in, std::string* out); diff --git a/cloud/src/meta-service/meta_service.cpp b/cloud/src/meta-service/meta_service.cpp index 5f2a0d39066673..d9c9048793cb71 100644 --- a/cloud/src/meta-service/meta_service.cpp +++ b/cloud/src/meta-service/meta_service.cpp @@ -204,13 +204,19 @@ void MetaServiceImpl::get_version(::google::protobuf::RpcController* controller, cloud_unique_id = request->cloud_unique_id(); } + bool is_table_version = false; + if (request->has_is_table_version()) { + is_table_version = request->is_table_version(); + } + int64_t db_id = request->has_db_id() ? request->db_id() : -1; int64_t table_id = request->has_table_id() ? request->table_id() : -1; int64_t partition_id = request->has_partition_id() ? request->partition_id() : -1; - if (db_id == -1 || table_id == -1 || partition_id == -1) { + if (db_id == -1 || table_id == -1 || (!is_table_version && partition_id == -1)) { msg = "params error, db_id=" + std::to_string(db_id) + " table_id=" + std::to_string(table_id) + - " partition_id=" + std::to_string(partition_id); + " partition_id=" + std::to_string(partition_id) + + " is_table_version=" + std::to_string(is_table_version); code = MetaServiceCode::INVALID_ARGUMENT; LOG(WARNING) << msg; return; @@ -224,9 +230,12 @@ void MetaServiceImpl::get_version(::google::protobuf::RpcController* controller, return; } RPC_RATE_LIMIT(get_version) - VersionKeyInfo ver_key_info {instance_id, db_id, table_id, partition_id}; std::string ver_key; - version_key(ver_key_info, &ver_key); + if (is_table_version) { + table_version_key({instance_id, db_id, table_id}, &ver_key); + } else { + partition_version_key({instance_id, db_id, table_id, partition_id}, &ver_key); + } std::unique_ptr txn; TxnErrorCode err = txn_kv_->create_txn(&txn); @@ -237,17 +246,22 @@ void MetaServiceImpl::get_version(::google::protobuf::RpcController* controller, } std::string ver_val; - VersionPB version_pb; // 0 for success get a key, 1 for key not found, negative for error err = txn->get(ver_key, &ver_val); VLOG_DEBUG << "xxx get version_key=" << hex(ver_key); if (err == TxnErrorCode::TXN_OK) { - if (!version_pb.ParseFromString(ver_val)) { - code = MetaServiceCode::PROTOBUF_PARSE_ERR; - msg = "malformed version value"; - return; + if (is_table_version) { + int64_t version = *reinterpret_cast(ver_val.data()); + response->set_version(version); + } else { + VersionPB version_pb; + if (!version_pb.ParseFromString(ver_val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = "malformed version value"; + return; + } + response->set_version(version_pb.version()); } - response->set_version(version_pb.version()); { TEST_SYNC_POINT_CALLBACK("get_version_code", &code); } return; } else if (err == TxnErrorCode::TXN_KEY_NOT_FOUND) { @@ -270,12 +284,18 @@ void MetaServiceImpl::batch_get_version(::google::protobuf::RpcController* contr cloud_unique_id = request->cloud_unique_id(); } + bool is_table_version = false; + if (request->has_is_table_version()) { + is_table_version = request->is_table_version(); + } + if (request->db_ids_size() == 0 || request->table_ids_size() == 0 || - request->table_ids_size() != request->partition_ids_size() || - request->db_ids_size() != request->partition_ids_size()) { + (!is_table_version && request->table_ids_size() != request->partition_ids_size()) || + (!is_table_version && request->db_ids_size() != request->partition_ids_size())) { msg = "param error, num db_ids=" + std::to_string(request->db_ids_size()) + " num table_ids=" + std::to_string(request->table_ids_size()) + - " num partition_ids=" + std::to_string(request->partition_ids_size()); + " num partition_ids=" + std::to_string(request->partition_ids_size()) + + " is_table_version=" + std::to_string(request->is_table_version()); code = MetaServiceCode::INVALID_ARGUMENT; LOG(WARNING) << msg; return; @@ -308,7 +328,12 @@ void MetaServiceImpl::batch_get_version(::google::protobuf::RpcController* contr int64_t db_id = request->db_ids(i); int64_t table_id = request->table_ids(i); int64_t partition_id = request->partition_ids(i); - std::string ver_key = version_key({instance_id, db_id, table_id, partition_id}); + std::string ver_key; + if (is_table_version) { + table_version_key({instance_id, db_id, table_id}, &ver_key); + } else { + partition_version_key({instance_id, db_id, table_id, partition_id}, &ver_key); + } // TODO(walter) support batch get. std::string ver_val; @@ -316,13 +341,18 @@ void MetaServiceImpl::batch_get_version(::google::protobuf::RpcController* contr TEST_SYNC_POINT_CALLBACK("batch_get_version_err", &err); VLOG_DEBUG << "xxx get version_key=" << hex(ver_key); if (err == TxnErrorCode::TXN_OK) { - VersionPB version_pb; - if (!version_pb.ParseFromString(ver_val)) { - code = MetaServiceCode::PROTOBUF_PARSE_ERR; - msg = "malformed version value"; - break; + if (is_table_version) { + int64_t version = *reinterpret_cast(ver_val.data()); + response->add_versions(version); + } else { + VersionPB version_pb; + if (!version_pb.ParseFromString(ver_val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = "malformed version value"; + break; + } + response->add_versions(version_pb.version()); } - response->add_versions(version_pb.version()); } else if (err == TxnErrorCode::TXN_KEY_NOT_FOUND) { // return -1 if the target version is not exists. response->add_versions(-1); diff --git a/cloud/src/meta-service/meta_service_partition.cpp b/cloud/src/meta-service/meta_service_partition.cpp index 17c67fb22ffd0c..c5a7db7d9028b1 100644 --- a/cloud/src/meta-service/meta_service_partition.cpp +++ b/cloud/src/meta-service/meta_service_partition.cpp @@ -222,6 +222,16 @@ void MetaServiceImpl::commit_index(::google::protobuf::RpcController* controller LOG_INFO("remove recycle index").tag("key", hex(key)); txn->remove(key); } + + if (request->has_db_id() && request->has_is_new_table() && request->is_new_table()) { + // init table version, for create and truncate table + std::string key = table_version_key({instance_id, request->db_id(), request->table_id()}); + std::string val(sizeof(int64_t), 0); + *reinterpret_cast(val.data()) = (int64_t)1; + txn->put(key, val); + LOG_INFO("put table version").tag("key", hex(key)); + } + err = txn->commit(); if (err != TxnErrorCode::TXN_OK) { code = cast_as(err); @@ -493,6 +503,14 @@ void MetaServiceImpl::commit_partition(::google::protobuf::RpcController* contro LOG_INFO("remove recycle partition").tag("key", hex(key)); txn->remove(key); } + + // update table versions + if (request->has_db_id()) { + std::string ver_key = table_version_key({instance_id, request->db_id(), request->table_id()}); + txn->atomic_add(ver_key, 1); + LOG_INFO("update table version").tag("ver_key", hex(ver_key)); + } + err = txn->commit(); if (err != TxnErrorCode::TXN_OK) { code = cast_as(err); @@ -513,8 +531,7 @@ void MetaServiceImpl::drop_partition(::google::protobuf::RpcController* controll } RPC_RATE_LIMIT(drop_partition) - if (request->partition_ids().empty() || request->index_ids().empty() || - !request->has_table_id()) { + if (request->partition_ids().empty() || request->index_ids().empty() || !request->has_table_id()) { code = MetaServiceCode::INVALID_ARGUMENT; msg = "empty partition_ids or index_ids or table_id"; return; @@ -579,6 +596,14 @@ void MetaServiceImpl::drop_partition(::google::protobuf::RpcController* controll } } if (!need_commit) return; + + // update table versions + if (request->has_db_id()) { + std::string ver_key = table_version_key({instance_id, request->db_id(), request->table_id()}); + txn->atomic_add(ver_key, 1); + LOG_INFO("update table version").tag("ver_key", hex(ver_key)); + } + err = txn->commit(); if (err != TxnErrorCode::TXN_OK) { code = cast_as(err); diff --git a/cloud/src/meta-service/meta_service_txn.cpp b/cloud/src/meta-service/meta_service_txn.cpp index 03251ee0f071a7..0c7f57385b3d7f 100644 --- a/cloud/src/meta-service/meta_service_txn.cpp +++ b/cloud/src/meta-service/meta_service_txn.cpp @@ -864,7 +864,7 @@ void MetaServiceImpl::commit_txn(::google::protobuf::RpcController* controller, int64_t table_id = tablet_ids[tablet_id].table_id(); int64_t partition_id = i.partition_id(); - std::string ver_key = version_key({instance_id, db_id, table_id, partition_id}); + std::string ver_key = partition_version_key({instance_id, db_id, table_id, partition_id}); int64_t version = -1; std::string ver_val_str; int64_t new_version = -1; @@ -983,11 +983,11 @@ void MetaServiceImpl::commit_txn(::google::protobuf::RpcController* controller, txn->put(i.first, ver_val); put_size += i.first.size() + ver_val.size(); - LOG(INFO) << "xxx put version_key=" << hex(i.first) << " version:" << i.second + LOG(INFO) << "xxx put partition_version_key=" << hex(i.first) << " version:" << i.second << " txn_id=" << txn_id; std::string_view ver_key = i.first; - //VersionKeyInfo {instance_id, db_id, table_id, partition_id} + //PartitionVersionKeyInfo {instance_id, db_id, table_id, partition_id} ver_key.remove_prefix(1); // Remove key space std::vector, int, int>> out; int ret = decode_key(&ver_key, &out); @@ -1009,6 +1009,15 @@ void MetaServiceImpl::commit_txn(::google::protobuf::RpcController* controller, response->add_versions(i.second); } + // Save table versions + num_put_keys += table_id_tablet_ids.size(); + for (auto& i : table_id_tablet_ids) { + std::string ver_key = table_version_key({instance_id, db_id, i.first}); + txn->atomic_add(ver_key, 1); + put_size += ver_key.size(); + LOG(INFO) << "xxx atomic add table_version_key=" << hex(ver_key) << " txn_id=" << txn_id; + } + LOG(INFO) << " before update txn_info=" << txn_info.ShortDebugString(); // Update txn_info diff --git a/cloud/src/recycler/meta_checker.cpp b/cloud/src/recycler/meta_checker.cpp index 86a17019399053..00a27aa9a29419 100644 --- a/cloud/src/recycler/meta_checker.cpp +++ b/cloud/src/recycler/meta_checker.cpp @@ -171,7 +171,7 @@ bool MetaChecker::check_fdb_by_fe_meta(MYSQL* conn) { int num_row = mysql_num_rows(result); for (int i = 0; i < num_row; ++i) { MYSQL_ROW row = mysql_fetch_row(result); - TabletInfo tablet_info; + TabletInfo tablet_info = { 0 }; tablet_info.tablet_id = atoll(row[0]); tablet_info.schema_version = atoll(row[4]); tablets.push_back(std::move(tablet_info)); @@ -197,7 +197,7 @@ bool MetaChecker::check_fdb_by_fe_meta(MYSQL* conn) { tablet_info.partition_id = atoll(row[6]); tablet_info.index_id = atoll(row[7]); - PartitionInfo partition_info; + PartitionInfo partition_info = { 0 }; partition_info.db_id = atoll(row[4]); partition_info.table_id = atoll(row[5]); partition_info.partition_id = atoll(row[6]); diff --git a/cloud/src/recycler/recycler.cpp b/cloud/src/recycler/recycler.cpp index 1ca1e05f741e57..cafcd2131d879e 100644 --- a/cloud/src/recycler/recycler.cpp +++ b/cloud/src/recycler/recycler.cpp @@ -527,9 +527,8 @@ int InstanceRecycler::recycle_deleted_instance() { std::string start_txn_key = txn_key_prefix(instance_id_); std::string end_txn_key = txn_key_prefix(instance_id_ + '\x00'); txn->remove(start_txn_key, end_txn_key); - // 0:instance_id 1:db_id 2:tbl_id 3:partition_id - std::string start_version_key = version_key({instance_id_, 0, 0, 0}); - std::string end_version_key = version_key({instance_id_, INT64_MAX, 0, 0}); + std::string start_version_key = version_key_prefix(instance_id_); + std::string end_version_key = version_key_prefix(instance_id_ + '\x00'); txn->remove(start_version_key, end_version_key); std::string start_meta_key = meta_key_prefix(instance_id_); std::string end_meta_key = meta_key_prefix(instance_id_ + '\x00'); @@ -758,7 +757,7 @@ int InstanceRecycler::recycle_partitions() { // Elements in `partition_keys` has the same lifetime as `it` in `scan_and_recycle` std::vector partition_keys; - std::vector version_keys; + std::vector partition_version_keys; auto recycle_func = [&, this](std::string_view k, std::string_view v) -> int { ++num_scanned; RecyclePartitionPB part_pb; @@ -831,18 +830,18 @@ int InstanceRecycler::recycle_partitions() { check_recycle_task(instance_id_, task_name, num_scanned, num_recycled, start_time); partition_keys.push_back(k); if (part_pb.db_id() > 0) { - version_keys.push_back(version_key( + partition_version_keys.push_back(partition_version_key( {instance_id_, part_pb.db_id(), part_pb.table_id(), partition_id})); } } return ret; }; - auto loop_done = [&partition_keys, &version_keys, this]() -> int { + auto loop_done = [&partition_keys, &partition_version_keys, this]() -> int { if (partition_keys.empty()) return 0; std::unique_ptr> defer((int*)0x01, [&](int*) { partition_keys.clear(); - version_keys.clear(); + partition_version_keys.clear(); }); std::unique_ptr txn; TxnErrorCode err = txn_kv_->create_txn(&txn); @@ -853,7 +852,7 @@ int InstanceRecycler::recycle_partitions() { for (auto& k : partition_keys) { txn->remove(k); } - for (auto& k : version_keys) { + for (auto& k : partition_version_keys) { txn->remove(k); } err = txn->commit(); @@ -872,24 +871,24 @@ int InstanceRecycler::recycle_versions() { int num_scanned = 0; int num_recycled = 0; - LOG_INFO("begin to recycle partition versions").tag("instance_id", instance_id_); + LOG_INFO("begin to recycle table and partition versions").tag("instance_id", instance_id_); auto start_time = steady_clock::now(); std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { auto cost = duration(steady_clock::now() - start_time).count(); - LOG_INFO("recycle partition versions finished, cost={}s", cost) + LOG_INFO("recycle table and partition versions finished, cost={}s", cost) .tag("instance_id", instance_id_) .tag("num_scanned", num_scanned) .tag("num_recycled", num_recycled); }); - auto version_key_begin = version_key({instance_id_, 0, 0, 0}); - auto version_key_end = version_key({instance_id_, INT64_MAX, 0, 0}); + auto version_key_begin = partition_version_key({instance_id_, 0, 0, 0}); + auto version_key_end = partition_version_key({instance_id_, INT64_MAX, 0, 0}); int64_t last_scanned_table_id = 0; bool is_recycled = false; // Is last scanned kv recycled auto recycle_func = [&num_scanned, &num_recycled, &last_scanned_table_id, &is_recycled, this]( - std::string_view k, std::string_view) { + std::string_view k, std::string_view) { ++num_scanned; auto k1 = k; k1.remove_prefix(1); @@ -916,16 +915,20 @@ int InstanceRecycler::recycle_versions() { if (err != TxnErrorCode::TXN_OK) { return -1; } - if (iter->has_next()) { // Table is useful, should not recycle partiton versions + if (iter->has_next()) { // Table is useful, should not recycle table and partition versions return 0; } - // Remove all version kvs of this table auto db_id = std::get(std::get<0>(out[3])); - auto table_version_key_begin = version_key({instance_id_, db_id, table_id, 0}); - auto table_version_key_end = version_key({instance_id_, db_id, table_id, INT64_MAX}); - txn->remove(table_version_key_begin, table_version_key_end); - LOG(WARNING) << "remove version kv, begin=" << hex(table_version_key_begin) - << " end=" << hex(table_version_key_end); + // 1. Remove all partition version kvs of this table + auto partition_version_key_begin = partition_version_key({instance_id_, db_id, table_id, 0}); + auto partition_version_key_end = partition_version_key({instance_id_, db_id, table_id, INT64_MAX}); + txn->remove(partition_version_key_begin, partition_version_key_end); + LOG(WARNING) << "remove partition version kv, begin=" << hex(partition_version_key_begin) + << " end=" << hex(partition_version_key_end); + // 2. Remove the table version kv of this table + auto tbl_version_key = table_version_key({instance_id_, db_id, table_id}); + txn->remove(tbl_version_key); + LOG(WARNING) << "remove table version kv " << hex(tbl_version_key); err = txn->commit(); if (err != TxnErrorCode::TXN_OK) { return -1; diff --git a/cloud/test/http_encode_key_test.cpp b/cloud/test/http_encode_key_test.cpp index ac5eade9735859..25182d54489f71 100644 --- a/cloud/test/http_encode_key_test.cpp +++ b/cloud/test/http_encode_key_test.cpp @@ -168,7 +168,7 @@ txn_id=126419752960)", R"({"table_ids":["10001"]})", }, Input { - "VersionKey", + "PartitionVersionKey", "instance_id=gavin-instance&db_id=10086&tbl_id=10010&partition_id=10000", "011076657273696f6e000110676176696e2d696e7374616e6365000110706172746974696f6e000112000000000000276612000000000000271a120000000000002710", []() -> std::string { @@ -178,6 +178,17 @@ txn_id=126419752960)", }, R"({"version":"10"})", }, + Input { + "TableVersionKey", + "instance_id=gavin-instance&db_id=10086&tbl_id=10010", + "011076657273696f6e000110676176696e2d696e7374616e63650001107461626c65000112000000000000276612000000000000271a", + []() -> std::string { + VersionPB pb; + pb.set_version(10); + return pb.SerializeAsString(); + }, + R"({"version":"10"})", + }, Input { "MetaRowsetKey", "instance_id=gavin-instance&tablet_id=10086&version=10010", diff --git a/cloud/test/keys_test.cpp b/cloud/test/keys_test.cpp index 614a17374d0bf5..ce7cbb0551e1bd 100644 --- a/cloud/test/keys_test.cpp +++ b/cloud/test/keys_test.cpp @@ -298,14 +298,14 @@ TEST(KeysTest, VersionKeyTest) { using namespace doris::cloud; std::string instance_id = "instance_id_deadbeef"; - // 0x01 "version" ${instance_id} "version_id" ${db_id} ${tbl_id} ${partition_id} -> ${version} + // 0x01 "version" ${instance_id} "partition" ${db_id} ${tbl_id} ${partition_id} -> ${version} { int64_t db_id = 11111; - int64_t tablet_id = 10086; + int64_t table_id = 10086; int64_t partition_id = 9998; - VersionKeyInfo v_key {instance_id, db_id, tablet_id, partition_id}; + PartitionVersionKeyInfo v_key {instance_id, db_id, table_id, partition_id}; std::string encoded_version_key0; - version_key(v_key, &encoded_version_key0); + partition_version_key(v_key, &encoded_version_key0); std::cout << "version key after encode: " << hex(encoded_version_key0) << std::endl; std::string dec_instance_id; @@ -329,12 +329,50 @@ TEST(KeysTest, VersionKeyTest) { EXPECT_EQ("partition", dec_version_infix); EXPECT_EQ(instance_id, dec_instance_id); EXPECT_EQ(db_id, dec_db_id); - EXPECT_EQ(tablet_id, dec_table_id); + EXPECT_EQ(table_id, dec_table_id); EXPECT_EQ(partition_id, dec_partition_id); std::get<3>(v_key) = partition_id + 1; std::string encoded_version_key1; - version_key(v_key, &encoded_version_key1); + partition_version_key(v_key, &encoded_version_key1); + std::cout << "version key after encode: " << hex(encoded_version_key1) << std::endl; + + ASSERT_GT(encoded_version_key1, encoded_version_key0); + } + + // 0x01 "version" ${instance_id} "table" ${db_id} ${tbl_id} -> ${version} + { + int64_t db_id = 11111; + int64_t table_id = 10010; + TableVersionKeyInfo v_key {instance_id, db_id, table_id}; + std::string encoded_version_key0; + table_version_key(v_key, &encoded_version_key0); + std::cout << "version key after encode: " << hex(encoded_version_key0) << std::endl; + + std::string dec_instance_id; + int64_t dec_db_id = 0; + int64_t dec_table_id = 0; + + std::string_view key_sv(encoded_version_key0); + std::string dec_version_prefix; + std::string dec_version_infix; + remove_user_space_prefix(&key_sv); + ASSERT_EQ(decode_bytes(&key_sv, &dec_version_prefix), 0); + ASSERT_EQ(decode_bytes(&key_sv, &dec_instance_id), 0); + ASSERT_EQ(decode_bytes(&key_sv, &dec_version_infix), 0); + ASSERT_EQ(decode_int64(&key_sv, &dec_db_id), 0) << hex(key_sv); + ASSERT_EQ(decode_int64(&key_sv, &dec_table_id), 0); + ASSERT_TRUE(key_sv.empty()); + + EXPECT_EQ("version", dec_version_prefix); + EXPECT_EQ("table", dec_version_infix); + EXPECT_EQ(instance_id, dec_instance_id); + EXPECT_EQ(db_id, dec_db_id); + EXPECT_EQ(table_id, dec_table_id); + + std::get<2>(v_key) = table_id + 1; + std::string encoded_version_key1; + table_version_key(v_key, &encoded_version_key1); std::cout << "version key after encode: " << hex(encoded_version_key1) << std::endl; ASSERT_GT(encoded_version_key1, encoded_version_key0); diff --git a/cloud/test/meta_service_test.cpp b/cloud/test/meta_service_test.cpp index 2d8e0c89a5c406..a497cb566f4974 100644 --- a/cloud/test/meta_service_test.cpp +++ b/cloud/test/meta_service_test.cpp @@ -4247,6 +4247,8 @@ TEST(MetaServiceTest, IndexRequest) { auto tablet_val = tablet_pb.SerializeAsString(); RecycleIndexPB index_pb; auto index_key = recycle_index_key({instance_id, index_id}); + int64_t val_int = 0; + auto tbl_version_key = table_version_key({instance_id, 1, table_id}); std::string val; // ------------Test prepare index------------ @@ -4255,8 +4257,10 @@ TEST(MetaServiceTest, IndexRequest) { IndexResponse res; meta_service->prepare_index(&ctrl, &req, &res, nullptr); ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT); + req.set_db_id(1); req.set_table_id(table_id); req.add_index_ids(index_id); + req.set_is_new_table(true); // Last state UNKNOWN res.Clear(); meta_service->prepare_index(&ctrl, &req, &res, nullptr); @@ -4312,19 +4316,26 @@ TEST(MetaServiceTest, IndexRequest) { ASSERT_EQ(res.status().code(), MetaServiceCode::ALREADY_EXISTED); ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); ASSERT_EQ(txn->get(index_key, &val), TxnErrorCode::TXN_KEY_NOT_FOUND); + // Prepare index should not init table version + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(tbl_version_key, &val), TxnErrorCode::TXN_KEY_NOT_FOUND); // ------------Test commit index------------ reset_meta_service(); req.Clear(); meta_service->commit_index(&ctrl, &req, &res, nullptr); ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT); + req.set_db_id(1); req.set_table_id(table_id); req.add_index_ids(index_id); + req.set_is_new_table(true); // Last state UNKNOWN res.Clear(); meta_service->commit_index(&ctrl, &req, &res, nullptr); ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT); ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); ASSERT_EQ(txn->get(index_key, &val), TxnErrorCode::TXN_KEY_NOT_FOUND); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(tbl_version_key, &val), TxnErrorCode::TXN_KEY_NOT_FOUND); // Last state PREPARED reset_meta_service(); index_pb.set_state(RecycleIndexPB::PREPARED); @@ -4337,6 +4348,11 @@ TEST(MetaServiceTest, IndexRequest) { ASSERT_EQ(res.status().code(), MetaServiceCode::OK); ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); ASSERT_EQ(txn->get(index_key, &val), TxnErrorCode::TXN_KEY_NOT_FOUND); + // First commit index should init table version + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(tbl_version_key, &val), TxnErrorCode::TXN_OK); + val_int = *reinterpret_cast(val.data()); + ASSERT_EQ(val_int, 1); // Last state DROPPED reset_meta_service(); index_pb.set_state(RecycleIndexPB::DROPPED); @@ -4351,6 +4367,8 @@ TEST(MetaServiceTest, IndexRequest) { ASSERT_EQ(txn->get(index_key, &val), TxnErrorCode::TXN_OK); ASSERT_TRUE(index_pb.ParseFromString(val)); ASSERT_EQ(index_pb.state(), RecycleIndexPB::DROPPED); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(tbl_version_key, &val), TxnErrorCode::TXN_KEY_NOT_FOUND); // Last state RECYCLING reset_meta_service(); index_pb.set_state(RecycleIndexPB::RECYCLING); @@ -4365,6 +4383,8 @@ TEST(MetaServiceTest, IndexRequest) { ASSERT_EQ(txn->get(index_key, &val), TxnErrorCode::TXN_OK); ASSERT_TRUE(index_pb.ParseFromString(val)); ASSERT_EQ(index_pb.state(), RecycleIndexPB::RECYCLING); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(tbl_version_key, &val), TxnErrorCode::TXN_KEY_NOT_FOUND); // Last state UNKNOWN but tablet meta existed reset_meta_service(); ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); @@ -4375,13 +4395,17 @@ TEST(MetaServiceTest, IndexRequest) { ASSERT_EQ(res.status().code(), MetaServiceCode::OK); ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); ASSERT_EQ(txn->get(index_key, &val), TxnErrorCode::TXN_KEY_NOT_FOUND); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(tbl_version_key, &val), TxnErrorCode::TXN_KEY_NOT_FOUND); // ------------Test drop index------------ reset_meta_service(); req.Clear(); meta_service->drop_index(&ctrl, &req, &res, nullptr); ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT); + req.set_db_id(1); req.set_table_id(table_id); req.add_index_ids(index_id); + req.set_is_new_table(true); // Last state UNKNOWN res.Clear(); meta_service->drop_index(&ctrl, &req, &res, nullptr); @@ -4432,6 +4456,9 @@ TEST(MetaServiceTest, IndexRequest) { ASSERT_EQ(txn->get(index_key, &val), TxnErrorCode::TXN_OK); ASSERT_TRUE(index_pb.ParseFromString(val)); ASSERT_EQ(index_pb.state(), RecycleIndexPB::RECYCLING); + // Drop index should not init table version + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(tbl_version_key, &val), TxnErrorCode::TXN_KEY_NOT_FOUND); } TEST(MetaServiceTest, PartitionRequest) { @@ -4460,6 +4487,8 @@ TEST(MetaServiceTest, PartitionRequest) { auto tablet_val = tablet_pb.SerializeAsString(); RecyclePartitionPB partition_pb; auto partition_key = recycle_partition_key({instance_id, partition_id}); + int64_t val_int = 0; + auto tbl_version_key = table_version_key({instance_id, 1, table_id}); std::string val; // ------------Test prepare partition------------ brpc::Controller ctrl; @@ -4467,17 +4496,26 @@ TEST(MetaServiceTest, PartitionRequest) { PartitionResponse res; meta_service->prepare_partition(&ctrl, &req, &res, nullptr); ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT); + req.set_db_id(1); req.set_table_id(table_id); req.add_index_ids(index_id); req.add_partition_ids(partition_id); // Last state UNKNOWN res.Clear(); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->atomic_add(tbl_version_key, 1); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); meta_service->prepare_partition(&ctrl, &req, &res, nullptr); ASSERT_EQ(res.status().code(), MetaServiceCode::OK); ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); ASSERT_EQ(txn->get(partition_key, &val), TxnErrorCode::TXN_OK); ASSERT_TRUE(partition_pb.ParseFromString(val)); ASSERT_EQ(partition_pb.state(), RecyclePartitionPB::PREPARED); + // Prepare partition should not update table version + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(tbl_version_key, &val), TxnErrorCode::TXN_OK); + val_int = *reinterpret_cast(val.data()); + ASSERT_EQ(val_int, 1); // Last state PREPARED res.Clear(); meta_service->prepare_partition(&ctrl, &req, &res, nullptr); @@ -4525,11 +4563,14 @@ TEST(MetaServiceTest, PartitionRequest) { ASSERT_EQ(res.status().code(), MetaServiceCode::ALREADY_EXISTED); ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); ASSERT_EQ(txn->get(partition_key, &val), TxnErrorCode::TXN_KEY_NOT_FOUND); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(tbl_version_key, &val), TxnErrorCode::TXN_KEY_NOT_FOUND); // ------------Test commit partition------------ reset_meta_service(); req.Clear(); meta_service->commit_partition(&ctrl, &req, &res, nullptr); ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT); + req.set_db_id(1); req.set_table_id(table_id); req.add_index_ids(index_id); req.add_partition_ids(partition_id); @@ -4541,6 +4582,9 @@ TEST(MetaServiceTest, PartitionRequest) { ASSERT_EQ(txn->get(partition_key, &val), TxnErrorCode::TXN_KEY_NOT_FOUND); // Last state PREPARED reset_meta_service(); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->atomic_add(tbl_version_key, 1); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); partition_pb.set_state(RecyclePartitionPB::PREPARED); val = partition_pb.SerializeAsString(); ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); @@ -4551,8 +4595,16 @@ TEST(MetaServiceTest, PartitionRequest) { ASSERT_EQ(res.status().code(), MetaServiceCode::OK); ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); ASSERT_EQ(txn->get(partition_key, &val), TxnErrorCode::TXN_KEY_NOT_FOUND); + // Commit partition should update table version + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(tbl_version_key, &val), TxnErrorCode::TXN_OK); + val_int = *reinterpret_cast(val.data()); + ASSERT_EQ(val_int, 2); // Last state DROPPED reset_meta_service(); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->atomic_add(tbl_version_key, 1); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); partition_pb.set_state(RecyclePartitionPB::DROPPED); val = partition_pb.SerializeAsString(); ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); @@ -4565,8 +4617,15 @@ TEST(MetaServiceTest, PartitionRequest) { ASSERT_EQ(txn->get(partition_key, &val), TxnErrorCode::TXN_OK); ASSERT_TRUE(partition_pb.ParseFromString(val)); ASSERT_EQ(partition_pb.state(), RecyclePartitionPB::DROPPED); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(tbl_version_key, &val), TxnErrorCode::TXN_OK); + val_int = *reinterpret_cast(val.data()); + ASSERT_EQ(val_int, 1); // Last state RECYCLING reset_meta_service(); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->atomic_add(tbl_version_key, 1); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); partition_pb.set_state(RecyclePartitionPB::RECYCLING); val = partition_pb.SerializeAsString(); ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); @@ -4579,9 +4638,16 @@ TEST(MetaServiceTest, PartitionRequest) { ASSERT_EQ(txn->get(partition_key, &val), TxnErrorCode::TXN_OK); ASSERT_TRUE(partition_pb.ParseFromString(val)); ASSERT_EQ(partition_pb.state(), RecyclePartitionPB::RECYCLING); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(tbl_version_key, &val), TxnErrorCode::TXN_OK); + val_int = *reinterpret_cast(val.data()); + ASSERT_EQ(val_int, 1); // Last state UNKNOWN but tablet meta existed reset_meta_service(); ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->atomic_add(tbl_version_key, 1); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); txn->put(tablet_key, tablet_val); ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); res.Clear(); @@ -4589,6 +4655,10 @@ TEST(MetaServiceTest, PartitionRequest) { ASSERT_EQ(res.status().code(), MetaServiceCode::OK); ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); ASSERT_EQ(txn->get(partition_key, &val), TxnErrorCode::TXN_KEY_NOT_FOUND); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(tbl_version_key, &val), TxnErrorCode::TXN_OK); + val_int = *reinterpret_cast(val.data()); + ASSERT_EQ(val_int, 1); // Last state UNKNOWN and tablet meta existed, but request has no index ids reset_meta_service(); ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); @@ -4606,19 +4676,31 @@ TEST(MetaServiceTest, PartitionRequest) { req.Clear(); meta_service->drop_partition(&ctrl, &req, &res, nullptr); ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT); + req.set_db_id(1); req.set_table_id(table_id); req.add_index_ids(index_id); req.add_partition_ids(partition_id); // Last state UNKNOWN res.Clear(); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->atomic_add(tbl_version_key, 1); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); meta_service->drop_partition(&ctrl, &req, &res, nullptr); ASSERT_EQ(res.status().code(), MetaServiceCode::OK); ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); ASSERT_EQ(txn->get(partition_key, &val), TxnErrorCode::TXN_OK); ASSERT_TRUE(partition_pb.ParseFromString(val)); ASSERT_EQ(partition_pb.state(), RecyclePartitionPB::DROPPED); + // Drop partition should update table version + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(tbl_version_key, &val), TxnErrorCode::TXN_OK); + val_int = *reinterpret_cast(val.data()); + ASSERT_EQ(val_int, 2); // Last state PREPARED reset_meta_service(); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->atomic_add(tbl_version_key, 1); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); partition_pb.set_state(RecyclePartitionPB::PREPARED); val = partition_pb.SerializeAsString(); ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); @@ -4631,8 +4713,15 @@ TEST(MetaServiceTest, PartitionRequest) { ASSERT_EQ(txn->get(partition_key, &val), TxnErrorCode::TXN_OK); ASSERT_TRUE(partition_pb.ParseFromString(val)); ASSERT_EQ(partition_pb.state(), RecyclePartitionPB::DROPPED); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(tbl_version_key, &val), TxnErrorCode::TXN_OK); + val_int = *reinterpret_cast(val.data()); + ASSERT_EQ(val_int, 2); // Last state DROPPED reset_meta_service(); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->atomic_add(tbl_version_key, 1); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); partition_pb.set_state(RecyclePartitionPB::DROPPED); val = partition_pb.SerializeAsString(); ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); @@ -4645,8 +4734,15 @@ TEST(MetaServiceTest, PartitionRequest) { ASSERT_EQ(txn->get(partition_key, &val), TxnErrorCode::TXN_OK); ASSERT_TRUE(partition_pb.ParseFromString(val)); ASSERT_EQ(partition_pb.state(), RecyclePartitionPB::DROPPED); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(tbl_version_key, &val), TxnErrorCode::TXN_OK); + val_int = *reinterpret_cast(val.data()); + ASSERT_EQ(val_int, 1); // Last state RECYCLING reset_meta_service(); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->atomic_add(tbl_version_key, 1); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); partition_pb.set_state(RecyclePartitionPB::RECYCLING); val = partition_pb.SerializeAsString(); ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); @@ -4659,6 +4755,10 @@ TEST(MetaServiceTest, PartitionRequest) { ASSERT_EQ(txn->get(partition_key, &val), TxnErrorCode::TXN_OK); ASSERT_TRUE(partition_pb.ParseFromString(val)); ASSERT_EQ(partition_pb.state(), RecyclePartitionPB::RECYCLING); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(tbl_version_key, &val), TxnErrorCode::TXN_OK); + val_int = *reinterpret_cast(val.data()); + ASSERT_EQ(val_int, 1); } TEST(MetaServiceTxnStoreRetryableTest, MockGetVersion) { diff --git a/cloud/test/recycler_test.cpp b/cloud/test/recycler_test.cpp index 87c70833a30d22..de059d0df09be7 100644 --- a/cloud/test/recycler_test.cpp +++ b/cloud/test/recycler_test.cpp @@ -293,8 +293,8 @@ static int create_recycle_partiton(TxnKv* txn_kv, int64_t table_id, int64_t part return 0; } -static int create_version_kv(TxnKv* txn_kv, int64_t table_id, int64_t partition_id) { - auto key = version_key({instance_id, db_id, table_id, partition_id}); +static int create_partition_version_kv(TxnKv* txn_kv, int64_t table_id, int64_t partition_id) { + auto key = partition_version_key({instance_id, db_id, table_id, partition_id}); VersionPB version; version.set_version(1); auto val = version.SerializeAsString(); @@ -309,6 +309,21 @@ static int create_version_kv(TxnKv* txn_kv, int64_t table_id, int64_t partition_ return 0; } +static int create_table_version_kv(TxnKv* txn_kv, int64_t table_id) { + auto key = table_version_key({instance_id, db_id, table_id}); + std::string val(sizeof(int64_t), 0); + *reinterpret_cast(val.data()) = (int64_t) 1; + std::unique_ptr txn; + if (txn_kv->create_txn(&txn) != TxnErrorCode::TXN_OK) { + return -1; + } + txn->put(key, val); + if (txn->commit() != TxnErrorCode::TXN_OK) { + return -1; + } + return 0; +} + static int create_txn_label_kv(TxnKv* txn_kv, std::string label, int64_t db_id) { std::string txn_label_key_; std::string txn_label_val; @@ -1133,8 +1148,9 @@ TEST(RecyclerTest, recycle_versions) { } } for (auto partition_id : partition_ids) { - create_version_kv(txn_kv.get(), table_id, partition_id); + create_partition_version_kv(txn_kv.get(), table_id, partition_id); } + create_table_version_kv(txn_kv.get(), table_id); // Drop partitions for (int i = 0; i < 5; ++i) { create_recycle_partiton(txn_kv.get(), table_id, partition_ids[i], index_ids); @@ -1147,16 +1163,23 @@ TEST(RecyclerTest, recycle_versions) { // Recycle all partitions in table except 30006 ASSERT_EQ(recycler.recycle_partitions(), 0); ASSERT_EQ(recycler.recycle_versions(), 0); // `recycle_versions` should do nothing - // All version kvs except version of partition 30006 must have been deleted + // All partition version kvs except version of partition 30006 must have been deleted std::unique_ptr txn; ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); - auto key_begin = version_key({instance_id, db_id, table_id, 0}); - auto key_end = version_key({instance_id, db_id, table_id, INT64_MAX}); + auto partition_key_begin = partition_version_key({instance_id, db_id, table_id, 0}); + auto partition_key_end = partition_version_key({instance_id, db_id, table_id, INT64_MAX}); std::unique_ptr iter; - ASSERT_EQ(txn->get(key_begin, key_end, &iter), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(partition_key_begin, partition_key_end, &iter), TxnErrorCode::TXN_OK); + ASSERT_EQ(iter->size(), 1); + auto [pk, pv] = iter->next(); + EXPECT_EQ(pk, partition_version_key({instance_id, db_id, table_id, 30006})); + // Table 10000's table version must not be deleted + auto table_key_begin = table_version_key({instance_id, db_id, 0}); + auto table_key_end = table_version_key({instance_id, db_id, INT64_MAX}); + ASSERT_EQ(txn->get(table_key_begin, table_key_end, &iter), TxnErrorCode::TXN_OK); ASSERT_EQ(iter->size(), 1); - auto [k, v] = iter->next(); - EXPECT_EQ(k, version_key({instance_id, db_id, table_id, 30006})); + auto [tk, tv] = iter->next(); + EXPECT_EQ(tk, table_version_key({instance_id, db_id, 10000})); // Drop indexes for (auto index_id : index_ids) { @@ -1167,7 +1190,9 @@ TEST(RecyclerTest, recycle_versions) { // `recycle_versions` should delete all version kvs of the dropped table ASSERT_EQ(recycler.recycle_versions(), 0); ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); - ASSERT_EQ(txn->get(key_begin, key_end, &iter), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(partition_key_begin, partition_key_end, &iter), TxnErrorCode::TXN_OK); + ASSERT_EQ(iter->size(), 0); + ASSERT_EQ(txn->get(table_key_begin, table_key_end, &iter), TxnErrorCode::TXN_OK); ASSERT_EQ(iter->size(), 0); } @@ -1790,9 +1815,13 @@ TEST(RecyclerTest, recycle_deleted_instance) { for (size_t i = 0; i < 100; i++) { ASSERT_EQ(0, create_txn_label_kv(txn_kv.get(), fmt::format("fake_label{}", i), i)); } - // create version key + // create partition version key + for (size_t i = 101; i < 200; i += 2) { + ASSERT_EQ(0, create_partition_version_kv(txn_kv.get(), i, i + 1)); + } + // create table version key for (size_t i = 101; i < 200; i += 2) { - ASSERT_EQ(0, create_version_kv(txn_kv.get(), i, i + 1)); + ASSERT_EQ(0, create_table_version_kv(txn_kv.get(), i)); } // create meta key std::vector schemas; @@ -1851,8 +1880,18 @@ TEST(RecyclerTest, recycle_deleted_instance) { ASSERT_EQ(txn->get(start_txn_key, end_txn_key, &it), TxnErrorCode::TXN_OK); ASSERT_EQ(it->size(), 0); - std::string start_version_key = version_key({instance_id, 0, 0, 0}); - std::string end_version_key = version_key({instance_id, INT64_MAX, 0, 0}); + std::string start_partition_version_key = partition_version_key({instance_id, 0, 0, 0}); + std::string end_partition_version_key = partition_version_key({instance_id, INT64_MAX, 0, 0}); + ASSERT_EQ(txn->get(start_partition_version_key, end_partition_version_key, &it), TxnErrorCode::TXN_OK); + ASSERT_EQ(it->size(), 0); + + std::string start_table_version_key = table_version_key({instance_id, 0, 0}); + std::string end_table_version_key = table_version_key({instance_id, INT64_MAX, 0}); + ASSERT_EQ(txn->get(start_table_version_key, end_table_version_key, &it), TxnErrorCode::TXN_OK); + ASSERT_EQ(it->size(), 0); + + std::string start_version_key = version_key_prefix(instance_id); + std::string end_version_key = version_key_prefix(instance_id + '\x00'); ASSERT_EQ(txn->get(start_version_key, end_version_key, &it), TxnErrorCode::TXN_OK); ASSERT_EQ(it->size(), 0); diff --git a/conf/fe.conf b/conf/fe.conf index 8b8c3be747deac..91f7d048cb1de2 100644 --- a/conf/fe.conf +++ b/conf/fe.conf @@ -30,7 +30,7 @@ LOG_DIR = ${DORIS_HOME}/log JAVA_OPTS="-Djavax.security.auth.useSubjectCredsOnly=false -Xss4m -Xmx8192m -XX:+UnlockExperimentalVMOptions -XX:+UseG1GC -XX:MaxGCPauseMillis=200 -XX:+PrintGCDateStamps -XX:+PrintGCDetails -Xloggc:$DORIS_HOME/log/fe.gc.log.$CUR_DATE -XX:+UseGCLogFileRotation -XX:NumberOfGCLogFiles=10 -XX:GCLogFileSize=50M -Dlog4j2.formatMsgNoLookups=true" # For jdk 17, this JAVA_OPTS will be used as default JVM options -JAVA_OPTS_FOR_JDK_17="-Djavax.security.auth.useSubjectCredsOnly=false -Xmx8192m -Xms8192m -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=$DORIS_HOME/log/ -Xlog:gc*:$DORIS_HOME/log/fe.gc.log.$CUR_DATE:time,uptime:filecount=10,filesize=50M" +JAVA_OPTS_FOR_JDK_17="-Djavax.security.auth.useSubjectCredsOnly=false -Xmx8192m -Xms8192m -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=$DORIS_HOME/log/ -Xlog:gc*:$DORIS_HOME/log/fe.gc.log.$CUR_DATE:time,uptime:filecount=10,filesize=50M --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens java.base/jdk.internal.ref=ALL-UNNAMED" # Set your own JAVA_HOME # JAVA_HOME=/path/to/jdk/ diff --git a/docs/en/docs/sql-manual/sql-reference/Show-Statements/SHOW-VIEWS.md b/docs/en/docs/sql-manual/sql-reference/Show-Statements/SHOW-VIEWS.md new file mode 100644 index 00000000000000..5622be47fa3bd9 --- /dev/null +++ b/docs/en/docs/sql-manual/sql-reference/Show-Statements/SHOW-VIEWS.md @@ -0,0 +1,78 @@ +--- +{ + "title": "SHOW-VIEWS", + "language": "en" +} +--- + + + +## SHOW-VIEWS + +### Name + +SHOW VIEWS + +### Description + +This statement is used to display all logical views under the current db + +grammar: + +```sql +SHOW [FULL] VIEWS [LIKE] +```` + +illustrate: + +1. LIKE: Fuzzy query can be performed according to the table name + +### Example + + 1. Desplay all views under DB + + ```sql + MySQL [test]> show views; + +----------------+ + | Tables_in_test | + +----------------+ + | t1_view | + | t2_view | + +----------------+ + 2 rows in set (0.00 sec) + ``` + +2. Fuzzy query by view name + + ```sql + MySQL [test]> show views like '%t1%'; + +----------------+ + | Tables_in_test | + +----------------+ + | t1_view | + +----------------+ + 1 row in set (0.01 sec) + ``` + +### Keywords + + SHOW, VIEWS + +### Best Practice diff --git a/docs/zh-CN/docs/sql-manual/sql-reference/Show-Statements/SHOW-VIEWS.md b/docs/zh-CN/docs/sql-manual/sql-reference/Show-Statements/SHOW-VIEWS.md new file mode 100644 index 00000000000000..41262dba693109 --- /dev/null +++ b/docs/zh-CN/docs/sql-manual/sql-reference/Show-Statements/SHOW-VIEWS.md @@ -0,0 +1,78 @@ +--- +{ + "title": "SHOW-VIEWS", + "language": "zh-CN" +} +--- + + + +## SHOW-VIEWS + +### Name + +SHOW VIEWS + +### Description + +该语句用于展示当前 db 下所有的 logical view + +语法: + +```sql +SHOW [FULL] VIEWS [LIKE] | [WHERE where_condition] +``` + +说明: + +1. LIKE:可按照表名进行模糊查询 + +### Example + + 1. 查看DB下所有逻辑视图 + + ```sql + MySQL [test]> show views; + +----------------+ + | Tables_in_test | + +----------------+ + | t1_view | + | t2_view | + +----------------+ + 2 rows in set (0.00 sec) + ``` + +2. 按照VIEW名进行模糊查询 + + ```sql + MySQL [test]> show views like '%t1%'; + +----------------+ + | Tables_in_test | + +----------------+ + | t1_view | + +----------------+ + 1 row in set (0.01 sec) + ``` + +### Keywords + + SHOW, VIEWS + +### Best Practice diff --git a/fe/be-java-extensions/hudi-scanner/src/main/resources/package.xml b/fe/be-java-extensions/hudi-scanner/src/main/resources/package.xml index 206b7d3a7ca6d0..4bbb2610603363 100644 --- a/fe/be-java-extensions/hudi-scanner/src/main/resources/package.xml +++ b/fe/be-java-extensions/hudi-scanner/src/main/resources/package.xml @@ -25,13 +25,17 @@ under the License. jar false - - - ${project.build.outputDirectory} + + / - - **/*.class - - - + true + true + runtime + + + **/Log4j2Plugins.dat + + + + diff --git a/fe/be-java-extensions/java-common/src/main/resources/package.xml b/fe/be-java-extensions/java-common/src/main/resources/package.xml index 206b7d3a7ca6d0..4bbb2610603363 100644 --- a/fe/be-java-extensions/java-common/src/main/resources/package.xml +++ b/fe/be-java-extensions/java-common/src/main/resources/package.xml @@ -25,13 +25,17 @@ under the License. jar false - - - ${project.build.outputDirectory} + + / - - **/*.class - - - + true + true + runtime + + + **/Log4j2Plugins.dat + + + + diff --git a/fe/be-java-extensions/jdbc-scanner/src/main/resources/package.xml b/fe/be-java-extensions/jdbc-scanner/src/main/resources/package.xml index 206b7d3a7ca6d0..4bbb2610603363 100644 --- a/fe/be-java-extensions/jdbc-scanner/src/main/resources/package.xml +++ b/fe/be-java-extensions/jdbc-scanner/src/main/resources/package.xml @@ -25,13 +25,17 @@ under the License. jar false - - - ${project.build.outputDirectory} + + / - - **/*.class - - - + true + true + runtime + + + **/Log4j2Plugins.dat + + + + diff --git a/fe/be-java-extensions/max-compute-scanner/src/main/resources/package.xml b/fe/be-java-extensions/max-compute-scanner/src/main/resources/package.xml index 206b7d3a7ca6d0..4bbb2610603363 100644 --- a/fe/be-java-extensions/max-compute-scanner/src/main/resources/package.xml +++ b/fe/be-java-extensions/max-compute-scanner/src/main/resources/package.xml @@ -25,13 +25,17 @@ under the License. jar false - - - ${project.build.outputDirectory} + + / - - **/*.class - - - + true + true + runtime + + + **/Log4j2Plugins.dat + + + + diff --git a/fe/be-java-extensions/paimon-scanner/src/main/resources/package.xml b/fe/be-java-extensions/paimon-scanner/src/main/resources/package.xml index 206b7d3a7ca6d0..4bbb2610603363 100644 --- a/fe/be-java-extensions/paimon-scanner/src/main/resources/package.xml +++ b/fe/be-java-extensions/paimon-scanner/src/main/resources/package.xml @@ -25,13 +25,17 @@ under the License. jar false - - - ${project.build.outputDirectory} + + / - - **/*.class - - - + true + true + runtime + + + **/Log4j2Plugins.dat + + + + diff --git a/fe/be-java-extensions/preload-extensions/src/main/resources/package-deps.xml b/fe/be-java-extensions/preload-extensions/src/main/resources/package-deps.xml index 3ace22a7870027..1fe322c0bf3a66 100644 --- a/fe/be-java-extensions/preload-extensions/src/main/resources/package-deps.xml +++ b/fe/be-java-extensions/preload-extensions/src/main/resources/package-deps.xml @@ -31,6 +31,11 @@ under the License. false true runtime + + + **/Log4j2Plugins.dat + + diff --git a/fe/be-java-extensions/trino-connector-scanner/pom.xml b/fe/be-java-extensions/trino-connector-scanner/pom.xml index c9c821820a20ce..2c8b91e0a6c65a 100644 --- a/fe/be-java-extensions/trino-connector-scanner/pom.xml +++ b/fe/be-java-extensions/trino-connector-scanner/pom.xml @@ -33,11 +33,6 @@ under the License. - - org.apache.doris - hudi-scanner - ${project.version} - org.apache.doris java-common @@ -47,6 +42,11 @@ under the License. io.trino trino-main + + commons-io + commons-io + ${commons-io.version} + diff --git a/fe/be-java-extensions/trino-connector-scanner/src/main/java/org/apache/doris/trinoconnector/ProcessUtils.java b/fe/be-java-extensions/trino-connector-scanner/src/main/java/org/apache/doris/trinoconnector/ProcessUtils.java new file mode 100644 index 00000000000000..429b268e01ba1a --- /dev/null +++ b/fe/be-java-extensions/trino-connector-scanner/src/main/java/org/apache/doris/trinoconnector/ProcessUtils.java @@ -0,0 +1,74 @@ +// 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.trinoconnector; + +import org.apache.commons.io.FileUtils; + +import java.io.BufferedReader; +import java.io.File; +import java.io.IOException; +import java.io.InputStreamReader; +import java.lang.management.ManagementFactory; +import java.util.LinkedList; +import java.util.List; + +/** + * Utils for handling processes + */ +public class ProcessUtils { + public static long getCurrentProcId() { + try { + return ManagementFactory.getRuntimeMXBean().getPid(); + } catch (Exception e) { + throw new RuntimeException("Couldn't find PID of current JVM process.", e); + } + } + + public static List getChildProcessIds(long pid) { + try { + Process pgrep = (new ProcessBuilder("pgrep", "-P", String.valueOf(pid))).start(); + BufferedReader reader = new BufferedReader(new InputStreamReader(pgrep.getInputStream())); + List result = new LinkedList<>(); + String line; + while ((line = reader.readLine()) != null) { + result.add(Long.valueOf(line.trim())); + } + pgrep.waitFor(); + return result; + } catch (Exception e) { + throw new RuntimeException("Couldn't get child processes of PID " + pid, e); + } + } + + public static String getCommandLine(long pid) { + try { + return FileUtils.readFileToString(new File(String.format("/proc/%d/cmdline", pid))).trim(); + } catch (IOException e) { + return null; + } + } + + public static void killProcess(long pid) { + try { + Process kill = (new ProcessBuilder("kill", "-9", String.valueOf(pid))).start(); + kill.waitFor(); + } catch (Exception e) { + throw new RuntimeException("Couldn't kill process PID " + pid, e); + } + } +} diff --git a/fe/be-java-extensions/trino-connector-scanner/src/main/java/org/apache/doris/trinoconnector/TrinoConnectorCache.java b/fe/be-java-extensions/trino-connector-scanner/src/main/java/org/apache/doris/trinoconnector/TrinoConnectorCache.java index 0a076716b2fe77..e9d92a98f788ce 100644 --- a/fe/be-java-extensions/trino-connector-scanner/src/main/java/org/apache/doris/trinoconnector/TrinoConnectorCache.java +++ b/fe/be-java-extensions/trino-connector-scanner/src/main/java/org/apache/doris/trinoconnector/TrinoConnectorCache.java @@ -17,8 +17,6 @@ package org.apache.doris.trinoconnector; -import org.apache.doris.hudi.Utils; - import com.google.common.cache.CacheBuilder; import com.google.common.cache.CacheLoader; import com.google.common.cache.LoadingCache; @@ -115,12 +113,12 @@ private static TrinoConnectorCacheValue loadCache(TrinoConnectorCacheKey key) { ScheduledExecutorService executorService = Executors.newScheduledThreadPool(1); executorService.scheduleAtFixedRate(() -> { if (!isKilled.get()) { - List pids = Utils.getChildProcessIds( - Utils.getCurrentProcId()); + List pids = ProcessUtils.getChildProcessIds( + ProcessUtils.getCurrentProcId()); for (long pid : pids) { - String cmd = Utils.getCommandLine(pid); + String cmd = ProcessUtils.getCommandLine(pid); if (cmd != null && cmd.contains("org.openjdk.jol.vm.sa.AttachMain")) { - Utils.killProcess(pid); + ProcessUtils.killProcess(pid); isKilled.set(true); } } diff --git a/fe/be-java-extensions/trino-connector-scanner/src/main/resources/package.xml b/fe/be-java-extensions/trino-connector-scanner/src/main/resources/package.xml index 98fb269a547f8f..c4f838e54d1b7c 100644 --- a/fe/be-java-extensions/trino-connector-scanner/src/main/resources/package.xml +++ b/fe/be-java-extensions/trino-connector-scanner/src/main/resources/package.xml @@ -23,20 +23,11 @@ under the License. jar false - - - ${project.build.outputDirectory} - / - - **/*.class - - - - - + + \ No newline at end of file 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 8639f62af455fc..a0e7311af0bdd6 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 @@ -2341,10 +2341,16 @@ public class Config extends ConfigBase { }) public static int autobucket_min_buckets = 1; - @ConfField(description = {"Arrow Flight Server中所有用户token的缓存上限,超过后LRU淘汰,默认值为2000", + @ConfField(description = {"Arrow Flight Server中所有用户token的缓存上限,超过后LRU淘汰,默认值为512, " + + "并强制限制小于 qe_max_connection/2, 避免`Reach limit of connections`, " + + "因为arrow flight sql是无状态的协议,连接通常不会主动断开," + + "bearer token 从 cache 淘汰的同时会 unregister Connection.", "The cache limit of all user tokens in Arrow Flight Server. which will be eliminated by" - + "LRU rules after exceeding the limit, the default value is 2000."}) - public static int arrow_flight_token_cache_size = 2000; + + "LRU rules after exceeding the limit, the default value is 512, the mandatory limit is " + + "less than qe_max_connection/2 to avoid `Reach limit of connections`, " + + "because arrow flight sql is a stateless protocol, the connection is usually not actively " + + "disconnected, bearer token is evict from the cache will unregister ConnectContext."}) + public static int arrow_flight_token_cache_size = 512; @ConfField(description = {"Arrow Flight Server中用户token的存活时间,自上次写入后过期时间,单位分钟,默认值为4320,即3天", "The alive time of the user token in Arrow Flight Server, expire after write, unit minutes," diff --git a/fe/fe-core/pom.xml b/fe/fe-core/pom.xml index 21dd17d60b4bf0..307f9046cc6d25 100644 --- a/fe/fe-core/pom.xml +++ b/fe/fe-core/pom.xml @@ -35,7 +35,7 @@ under the License. 4.13.1 2.20.131 3.1.1-hw-46 - 3.3.5 + 8.2.7 @@ -418,9 +418,15 @@ under the License. hadoop-aliyun - org.apache.hadoop + com.qcloud.cos hadoop-cos ${tencentcos.version} + + + org.json + json + + com.aliyun.odps diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisLexer.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisLexer.g4 index 256888c2fc1409..33354199e67abd 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisLexer.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisLexer.g4 @@ -548,6 +548,7 @@ VALUE: 'VALUE'; VALUES: 'VALUES'; VARCHAR: 'VARCHAR'; VARIABLES: 'VARIABLES'; +VARIANT: 'VARIANT'; VERBOSE: 'VERBOSE'; VERSION: 'VERSION'; VIEW: 'VIEW'; 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 7aaf9fbde652c2..0b3a5b5c5b7888 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 @@ -899,6 +899,7 @@ primitiveColType: | type=DECIMALV3 | type=IPV4 | type=IPV6 + | type=VARIANT | type=ALL ; diff --git a/fe/fe-core/src/main/cup/sql_parser.cup b/fe/fe-core/src/main/cup/sql_parser.cup index d18f00146b9251..9738b361c6b019 100644 --- a/fe/fe-core/src/main/cup/sql_parser.cup +++ b/fe/fe-core/src/main/cup/sql_parser.cup @@ -45,6 +45,7 @@ import org.apache.doris.catalog.ArrayType; import org.apache.doris.catalog.MapType; import org.apache.doris.catalog.StructField; import org.apache.doris.catalog.StructType; +import org.apache.doris.catalog.TableIf.TableType; import org.apache.doris.catalog.View; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.FeConstants; @@ -659,6 +660,7 @@ terminal String KW_VERBOSE, KW_VERSION, KW_VIEW, + KW_VIEWS, KW_WARNINGS, KW_WEEK, KW_WHEN, @@ -3992,6 +3994,16 @@ show_param ::= {: RESULT = new ShowTableStmt(db, ctl, parser.isVerbose, parser.wild, parser.where); :} + /* show views */ + | opt_full KW_VIEWS opt_db:db opt_wild_where + {: + RESULT = new ShowTableStmt(db, null, parser.isVerbose, TableType.VIEW, parser.wild, parser.where); + :} + /* show views */ + | opt_full KW_VIEWS from_or_in ident:ctl DOT ident:db opt_wild_where + {: + RESULT = new ShowTableStmt(db, ctl, parser.isVerbose, TableType.VIEW, parser.wild, parser.where); + :} /* show table id */ | KW_TABLE INTEGER_LITERAL:tableId {: @@ -8080,6 +8092,8 @@ keyword ::= {: RESULT = id; :} | KW_VIEW:id {: RESULT = id; :} + | KW_VIEWS:id + {: RESULT = id; :} | KW_WARNINGS:id {: RESULT = id; :} | KW_WORK:id diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/CloudRollupJobV2.java b/fe/fe-core/src/main/java/org/apache/doris/alter/CloudRollupJobV2.java index 60980d4699f6bb..71ba5ef35cf169 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/CloudRollupJobV2.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/CloudRollupJobV2.java @@ -92,7 +92,7 @@ protected void onCreateRollupReplicaDone() throws AlterCancelException { rollupIndexList.add(rollupIndexId); try { ((CloudInternalCatalog) Env.getCurrentInternalCatalog()) - .commitMaterializedIndex(tableId, rollupIndexList); + .commitMaterializedIndex(dbId, tableId, rollupIndexList, false); } catch (Exception e) { LOG.warn("commitMaterializedIndex Exception:{}", e); throw new AlterCancelException(e.getMessage()); @@ -110,7 +110,7 @@ protected void onCancel() { while (true) { try { ((CloudInternalCatalog) Env.getCurrentInternalCatalog()) - .dropMaterializedIndex(tableId, rollupIndexList); + .dropMaterializedIndex(tableId, rollupIndexList, false); break; } catch (Exception e) { LOG.warn("tryTimes:{}, onCancel exception:", tryTimes, e); diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/CloudSchemaChangeJobV2.java b/fe/fe-core/src/main/java/org/apache/doris/alter/CloudSchemaChangeJobV2.java index 0739da03642471..8e3a198c2a4407 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/CloudSchemaChangeJobV2.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/CloudSchemaChangeJobV2.java @@ -83,8 +83,7 @@ protected void commitShadowIndex() throws AlterCancelException { indexIdMap.keySet().stream().collect(Collectors.toList()); try { ((CloudInternalCatalog) Env.getCurrentInternalCatalog()) - .commitMaterializedIndex(tableId, - shadowIdxList); + .commitMaterializedIndex(dbId, tableId, shadowIdxList, false); } catch (Exception e) { LOG.warn("commitMaterializedIndex exception:", e); throw new AlterCancelException(e.getMessage()); @@ -110,7 +109,7 @@ private void dropIndex(List idxList) { while (true) { try { ((CloudInternalCatalog) Env.getCurrentInternalCatalog()) - .dropMaterializedIndex(tableId, idxList); + .dropMaterializedIndex(tableId, idxList, false); break; } catch (Exception e) { LOG.warn("tryTimes:{}, dropIndex exception:", tryTimes, e); 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 fccc517f7a9ab9..2960d9a0744780 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 @@ -459,7 +459,7 @@ protected void runWaitingTxnJob() throws AlterCancelException { if (indexColumnMap.containsKey(SchemaChangeHandler.SHADOW_NAME_PREFIX + column.getName())) { Column newColumn = indexColumnMap .get(SchemaChangeHandler.SHADOW_NAME_PREFIX + column.getName()); - if (newColumn.getType() != column.getType()) { + if (!newColumn.getType().equals(column.getType())) { try { SlotRef slot = new SlotRef(destSlotDesc); slot.setCol(column.getName()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SystemHandler.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SystemHandler.java index e503e093787971..57e00f5ab1468c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/SystemHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SystemHandler.java @@ -33,24 +33,34 @@ import org.apache.doris.analysis.ModifyFrontendHostNameClause; import org.apache.doris.catalog.Database; import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.MysqlCompatibleDatabase; import org.apache.doris.catalog.OlapTable; +import org.apache.doris.catalog.Partition; +import org.apache.doris.catalog.ReplicaAllocation; +import org.apache.doris.catalog.Table; import org.apache.doris.catalog.TabletInvertedIndex; import org.apache.doris.common.Config; import org.apache.doris.common.DdlException; import org.apache.doris.common.UserException; +import org.apache.doris.common.util.DebugPointUtil; import org.apache.doris.common.util.NetUtils; import org.apache.doris.ha.FrontendNodeType; +import org.apache.doris.resource.Tag; import org.apache.doris.system.Backend; import org.apache.doris.system.SystemInfoService; import org.apache.doris.system.SystemInfoService.HostInfo; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import org.apache.commons.lang3.NotImplementedException; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; /* * SystemHandler is for @@ -235,7 +245,8 @@ public static List checkDecommission(List hostInfos) decommissionBackends.add(backend); } - // TODO(cmy): check if replication num can be met + checkDecommissionWithReplicaAllocation(decommissionBackends); + // TODO(cmy): check remaining space return decommissionBackends; @@ -258,12 +269,81 @@ public static List checkDecommissionByIds(List ids) decommissionBackends.add(backend); } - // TODO(cmy): check if replication num can be met + checkDecommissionWithReplicaAllocation(decommissionBackends); + // TODO(cmy): check remaining space return decommissionBackends; } + private static void checkDecommissionWithReplicaAllocation(List decommissionBackends) + throws DdlException { + if (Config.isCloudMode() || decommissionBackends.isEmpty() + || DebugPointUtil.isEnable("SystemHandler.decommission_no_check_replica_num")) { + return; + } + + Set decommissionTags = decommissionBackends.stream().map(be -> be.getLocationTag()) + .collect(Collectors.toSet()); + Map tagAvailBackendNums = Maps.newHashMap(); + for (Backend backend : Env.getCurrentSystemInfo().getAllBackends()) { + long beId = backend.getId(); + if (!backend.isScheduleAvailable() + || decommissionBackends.stream().anyMatch(be -> be.getId() == beId)) { + continue; + } + + Tag tag = backend.getLocationTag(); + if (tag != null) { + tagAvailBackendNums.put(tag, tagAvailBackendNums.getOrDefault(tag, 0) + 1); + } + } + + Env env = Env.getCurrentEnv(); + List dbIds = env.getInternalCatalog().getDbIds(); + for (Long dbId : dbIds) { + Database db = env.getInternalCatalog().getDbNullable(dbId); + if (db == null) { + continue; + } + + if (db instanceof MysqlCompatibleDatabase) { + continue; + } + + for (Table table : db.getTables()) { + table.readLock(); + try { + if (!table.needSchedule()) { + continue; + } + + OlapTable tbl = (OlapTable) table; + for (Partition partition : tbl.getAllPartitions()) { + ReplicaAllocation replicaAlloc = tbl.getPartitionInfo().getReplicaAllocation(partition.getId()); + for (Map.Entry entry : replicaAlloc.getAllocMap().entrySet()) { + Tag tag = entry.getKey(); + if (!decommissionTags.contains(tag)) { + continue; + } + int replicaNum = (int) entry.getValue(); + int backendNum = tagAvailBackendNums.getOrDefault(tag, 0); + if (replicaNum > backendNum) { + throw new DdlException("After decommission, partition " + partition.getName() + + " of table " + db.getName() + "." + tbl.getName() + + " 's replication allocation { " + replicaAlloc + + " } > available backend num " + backendNum + " on tag " + tag + + ", otherwise need to decrease the partition's replication num."); + } + } + } + } finally { + table.readUnlock(); + } + } + } + } + @Override public synchronized void cancel(CancelStmt stmt) throws DdlException { CancelAlterSystemStmt cancelAlterSystemStmt = (CancelAlterSystemStmt) stmt; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateRoleStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateRoleStmt.java index 0a60a3060c3d4d..9021402d48a26b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateRoleStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateRoleStmt.java @@ -18,6 +18,8 @@ package org.apache.doris.analysis; 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; @@ -60,6 +62,11 @@ public String getComment() { @Override public void analyze(Analyzer analyzer) throws UserException { super.analyze(analyzer); + + 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. diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateUserStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateUserStmt.java index 9446f918c23fc5..e64a5e2e7f81da 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateUserStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateUserStmt.java @@ -19,9 +19,12 @@ import org.apache.doris.catalog.Env; import org.apache.doris.cluster.ClusterNamespace; +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.common.LdapConfig; import org.apache.doris.common.UserException; import org.apache.doris.mysql.privilege.PrivPredicate; import org.apache.doris.mysql.privilege.Role; @@ -142,6 +145,11 @@ public String getComment() { @Override public void analyze(Analyzer analyzer) throws UserException { super.analyze(analyzer); + + if (Config.access_controller_type.equalsIgnoreCase("ranger-doris") && LdapConfig.ldap_authentication_enabled) { + throw new AnalysisException("Create user is prohibited when Ranger and LDAP are enabled at same time."); + } + userIdent.analyze(); if (userIdent.isRootUser()) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateWorkloadGroupStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateWorkloadGroupStmt.java index 3d48cad5cd6e50..92a60a94e55289 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateWorkloadGroupStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateWorkloadGroupStmt.java @@ -26,6 +26,7 @@ import org.apache.doris.common.util.PrintableMap; import org.apache.doris.mysql.privilege.PrivPredicate; import org.apache.doris.qe.ConnectContext; +import org.apache.doris.resource.workloadgroup.WorkloadGroup; import java.util.Map; @@ -69,6 +70,11 @@ public void analyze(Analyzer analyzer) throws UserException { if (properties == null || properties.isEmpty()) { throw new AnalysisException("Resource group properties can't be null"); } + + String wgTag = properties.get(WorkloadGroup.TAG); + if (wgTag != null) { + FeNameFormat.checkCommonName("workload group tag", wgTag); + } } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/DropRoleStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/DropRoleStmt.java index df087432a0a4f3..468b86579f4911 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/DropRoleStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/DropRoleStmt.java @@ -18,6 +18,8 @@ package org.apache.doris.analysis; 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; @@ -50,6 +52,11 @@ public String getRole() { @Override public void analyze(Analyzer analyzer) throws UserException { super.analyze(analyzer); + + if (Config.access_controller_type.equalsIgnoreCase("ranger-doris")) { + throw new AnalysisException("Drop role is prohibited when Ranger is enabled."); + } + FeNameFormat.checkRoleName(role, false /* can not be superuser */, "Can not drop role"); // check if current user has GRANT priv on GLOBAL level. 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 61b9fdc7ab53c4..4b8196ad638816 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 @@ -19,8 +19,10 @@ 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.LdapConfig; import org.apache.doris.common.UserException; import org.apache.doris.mysql.privilege.PrivPredicate; import org.apache.doris.qe.ConnectContext; @@ -53,6 +55,11 @@ public UserIdentity getUserIdentity() { @Override public void analyze(Analyzer analyzer) throws AnalysisException, UserException { super.analyze(analyzer); + + if (Config.access_controller_type.equalsIgnoreCase("ranger-doris") && LdapConfig.ldap_authentication_enabled) { + throw new AnalysisException("Drop user is prohibited when Ranger and LDAP are enabled at same time."); + } + userIdent.analyze(); if (userIdent.isRootUser()) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/GrantStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/GrantStmt.java index 7962eb822b852f..ae033a6ff39244 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/GrantStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/GrantStmt.java @@ -20,6 +20,7 @@ import org.apache.doris.catalog.AccessPrivilegeWithCols; 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; @@ -137,6 +138,11 @@ public Map> getColPrivileges() { @Override public void analyze(Analyzer analyzer) throws UserException { super.analyze(analyzer); + + if (Config.access_controller_type.equalsIgnoreCase("ranger-doris")) { + throw new AnalysisException("Grant is prohibited when Ranger is enabled."); + } + if (userIdent != null) { userIdent.analyze(); } else { diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ResourcePattern.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ResourcePattern.java index 6afeae3f4b816f..59c70f770a56ef 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ResourcePattern.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ResourcePattern.java @@ -22,6 +22,7 @@ import org.apache.doris.common.io.Text; import org.apache.doris.common.io.Writable; import org.apache.doris.mysql.privilege.Auth.PrivLevel; +import org.apache.doris.persist.gson.GsonPostProcessable; import org.apache.doris.persist.gson.GsonUtils; import com.google.common.base.Strings; @@ -34,7 +35,7 @@ // only the following 2 formats are allowed // * // resource -public class ResourcePattern implements Writable { +public class ResourcePattern implements Writable, GsonPostProcessable { @SerializedName(value = "resourceName") private String resourceName; @@ -48,9 +49,9 @@ public class ResourcePattern implements Writable { public static ResourcePattern ALL_STAGE; static { - ALL_GENERAL = new ResourcePattern("*", ResourceTypeEnum.GENERAL); - ALL_CLUSTER = new ResourcePattern("*", ResourceTypeEnum.CLUSTER); - ALL_STAGE = new ResourcePattern("*", ResourceTypeEnum.STAGE); + ALL_GENERAL = new ResourcePattern("%", ResourceTypeEnum.GENERAL); + ALL_CLUSTER = new ResourcePattern("%", ResourceTypeEnum.CLUSTER); + ALL_STAGE = new ResourcePattern("%", ResourceTypeEnum.STAGE); try { ALL_GENERAL.analyze(); @@ -65,7 +66,11 @@ private ResourcePattern() { } public ResourcePattern(String resourceName, ResourceTypeEnum type) { - this.resourceName = Strings.isNullOrEmpty(resourceName) ? "*" : resourceName; + // To be compatible with previous syntax + if ("*".equals(resourceName)) { + resourceName = "%"; + } + this.resourceName = Strings.isNullOrEmpty(resourceName) ? "%" : resourceName; resourceType = type; } @@ -86,15 +91,11 @@ public String getResourceName() { } public PrivLevel getPrivLevel() { - if (resourceName.equals("*")) { - return PrivLevel.GLOBAL; - } else { - return PrivLevel.RESOURCE; - } + return PrivLevel.RESOURCE; } public void analyze() throws AnalysisException { - if (!resourceName.equals("*")) { + if (!resourceName.equals("%")) { FeNameFormat.checkResourceName(resourceName, resourceType); } } @@ -130,4 +131,12 @@ public static ResourcePattern read(DataInput in) throws IOException { String json = Text.readString(in); return GsonUtils.GSON.fromJson(json, ResourcePattern.class); } + + @Override + public void gsonPostProcess() throws IOException { + // // To be compatible with previous syntax + if ("*".equals(resourceName)) { + resourceName = "%"; + } + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/RevokeStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/RevokeStmt.java index 70a44a4625cf4b..cbd5e0a83efe16 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/RevokeStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/RevokeStmt.java @@ -19,6 +19,7 @@ import org.apache.doris.catalog.AccessPrivilegeWithCols; import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.Config; import org.apache.doris.common.FeNameFormat; import org.apache.doris.mysql.privilege.ColPrivilegeKey; import org.apache.doris.mysql.privilege.PrivBitSet; @@ -121,6 +122,10 @@ public Map> getColPrivileges() { @Override public void analyze(Analyzer analyzer) throws AnalysisException { + if (Config.access_controller_type.equalsIgnoreCase("ranger-doris")) { + throw new AnalysisException("Revoke is prohibited when Ranger is enabled."); + } + if (userIdent != null) { userIdent.analyze(); } else { diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowColumnStatsStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowColumnStatsStmt.java index 37be76b20df09d..a4216f55661e16 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowColumnStatsStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowColumnStatsStmt.java @@ -32,6 +32,7 @@ import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.ShowResultSet; import org.apache.doris.qe.ShowResultSetMetaData; +import org.apache.doris.statistics.AnalysisManager; import org.apache.doris.statistics.ColStatsMeta; import org.apache.doris.statistics.ColumnStatistic; @@ -138,14 +139,15 @@ public TableIf getTable() { public ShowResultSet constructResultSet(List, ColumnStatistic>> columnStatistics) { List> result = Lists.newArrayList(); + AnalysisManager analysisManager = Env.getCurrentEnv().getAnalysisManager(); columnStatistics.forEach(p -> { if (p.second.isUnKnown) { return; } - List row = Lists.newArrayList(); - row.add(p.first.first); + // p data structure is Pair, ColumnStatistic> row.add(p.first.second); + row.add(p.first.first); row.add(String.valueOf(p.second.count)); row.add(String.valueOf(p.second.ndv)); row.add(String.valueOf(p.second.numNulls)); @@ -153,8 +155,7 @@ public ShowResultSet constructResultSet(List, ColumnSt row.add(String.valueOf(p.second.avgSizeByte)); row.add(String.valueOf(p.second.minExpr == null ? "N/A" : p.second.minExpr.toSql())); row.add(String.valueOf(p.second.maxExpr == null ? "N/A" : p.second.maxExpr.toSql())); - ColStatsMeta colStatsMeta = Env.getCurrentEnv().getAnalysisManager().findColStatsMeta(table.getId(), - p.first.first); + ColStatsMeta colStatsMeta = analysisManager.findColStatsMeta(table.getId(), p.first.first, p.first.second); row.add(String.valueOf(colStatsMeta == null ? "N/A" : colStatsMeta.analysisMethod)); row.add(String.valueOf(colStatsMeta == null ? "N/A" : colStatsMeta.analysisType)); row.add(String.valueOf(colStatsMeta == null ? "N/A" : colStatsMeta.jobType)); diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDataSkewStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDataSkewStmt.java index e0e3cfdc835f36..a1806278358010 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDataSkewStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDataSkewStmt.java @@ -20,7 +20,6 @@ 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.ErrorCode; import org.apache.doris.common.ErrorReport; import org.apache.doris.common.UserException; @@ -34,7 +33,7 @@ // show data skew from tbl [partition(p1, p2, ...)] public class ShowDataSkewStmt extends ShowStmt { public static final ImmutableList TITLE_NAMES = new ImmutableList.Builder() - .add("BucketIdx").add("AvgRowCount").add("AvgDataSize") + .add("PartitionName").add("BucketIdx").add("AvgRowCount").add("AvgDataSize") .add("Graph").add("Percent") .build(); @@ -58,10 +57,6 @@ public void analyze(Analyzer analyzer) throws UserException { ConnectContext.get().getRemoteIP(), tblRef.getName().getDb() + "." + tblRef.getName().getTbl()); } - PartitionNames partitionNames = tblRef.getPartitionNames(); - if (partitionNames == null || partitionNames.getPartitionNames().size() != 1) { - throw new AnalysisException("Should specify one and only one partition"); - } } public String getDbName() { diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTableStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTableStmt.java index 3dc3fa24aae6eb..f3e12f1aa92e4d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTableStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTableStmt.java @@ -20,6 +20,7 @@ import org.apache.doris.catalog.Column; import org.apache.doris.catalog.InfoSchemaDb; import org.apache.doris.catalog.ScalarType; +import org.apache.doris.catalog.TableIf.TableType; import org.apache.doris.cluster.ClusterNamespace; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.ErrorCode; @@ -40,8 +41,9 @@ public class ShowTableStmt extends ShowStmt { private static final String INVERTED_INDEX_STORAGE_FORMAT_COL = "Inverted_index_storage_format"; private String db; private String catalog; - private boolean isVerbose; - private String pattern; + private final boolean isVerbose; + private TableType type; + private final String pattern; private Expr where; private SelectStmt selectStmt; @@ -61,6 +63,12 @@ public ShowTableStmt(String db, String catalog, boolean isVerbose, String patter this.catalog = catalog; } + public ShowTableStmt(String db, String catalog, boolean isVerbose, TableType type, String pattern, + Expr where) { + this(db, catalog, isVerbose, pattern, where); + this.type = type; + } + public String getDb() { return db; } @@ -73,6 +81,10 @@ public boolean isVerbose() { return isVerbose; } + public TableType getType() { + return type; + } + public String getPattern() { return pattern; } @@ -120,6 +132,11 @@ public SelectStmt toSelectStmt(Analyzer analyzer) throws AnalysisException { selectList.addItem(item); aliasMap.put(new SlotRef(null, TYPE_COL), item.getExpr().clone(null)); } + if (type != null) { + BinaryPredicate viewFilter = new BinaryPredicate(BinaryPredicate.Operator.EQ, + new SlotRef(tablesTableName, "ENGINE"), new StringLiteral(type.toEngineName())); + where = CompoundPredicate.createConjunction(viewFilter, where); + } where = where.substitute(aliasMap); selectStmt = new SelectStmt(selectList, new FromClause(Lists.newArrayList(new TableRef(tablesTableName, null))), @@ -137,7 +154,18 @@ public String toSql() { if (isVerbose) { sb.append(" FULL"); } - sb.append(" TABLES"); + if (type != null) { + switch (type) { + // todo(only show views from now) + case VIEW: + sb.append(" VIEWS"); + break; + default: + sb.append(" TABLES"); + } + } else { + sb.append(" TABLES"); + } if (!Strings.isNullOrEmpty(db)) { if (!Strings.isNullOrEmpty(catalog)) { sb.append(" FROM ").append(catalog); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinAggregateFunctions.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinAggregateFunctions.java index e6134cfc31f9e5..49e6d66f50e1e2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinAggregateFunctions.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinAggregateFunctions.java @@ -50,6 +50,7 @@ import org.apache.doris.nereids.trees.expressions.functions.agg.MultiDistinctCount; import org.apache.doris.nereids.trees.expressions.functions.agg.MultiDistinctGroupConcat; import org.apache.doris.nereids.trees.expressions.functions.agg.MultiDistinctSum; +import org.apache.doris.nereids.trees.expressions.functions.agg.MultiDistinctSum0; import org.apache.doris.nereids.trees.expressions.functions.agg.Ndv; import org.apache.doris.nereids.trees.expressions.functions.agg.OrthogonalBitmapIntersect; import org.apache.doris.nereids.trees.expressions.functions.agg.OrthogonalBitmapIntersectCount; @@ -64,6 +65,7 @@ import org.apache.doris.nereids.trees.expressions.functions.agg.Stddev; import org.apache.doris.nereids.trees.expressions.functions.agg.StddevSamp; import org.apache.doris.nereids.trees.expressions.functions.agg.Sum; +import org.apache.doris.nereids.trees.expressions.functions.agg.Sum0; import org.apache.doris.nereids.trees.expressions.functions.agg.TopN; import org.apache.doris.nereids.trees.expressions.functions.agg.TopNArray; import org.apache.doris.nereids.trees.expressions.functions.agg.TopNWeighted; @@ -118,6 +120,7 @@ public class BuiltinAggregateFunctions implements FunctionHelper { agg(MultiDistinctCount.class, "multi_distinct_count"), agg(MultiDistinctGroupConcat.class, "multi_distinct_group_concat"), agg(MultiDistinctSum.class, "multi_distinct_sum"), + agg(MultiDistinctSum0.class, "multi_distinct_sum0"), agg(Ndv.class, "approx_count_distinct", "ndv"), agg(OrthogonalBitmapIntersect.class, "orthogonal_bitmap_intersect"), agg(OrthogonalBitmapIntersectCount.class, "orthogonal_bitmap_intersect_count"), @@ -132,6 +135,7 @@ public class BuiltinAggregateFunctions implements FunctionHelper { agg(Stddev.class, "stddev_pop", "stddev"), agg(StddevSamp.class, "stddev_samp"), agg(Sum.class, "sum"), + agg(Sum0.class, "sum0"), agg(TopN.class, "topn"), agg(TopNArray.class, "topn_array"), agg(TopNWeighted.class, "topn_weighted"), diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java index 1b479f364639b2..0c286cf6dd1ae6 100755 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java @@ -5518,9 +5518,14 @@ public void replaceTempPartition(Database db, OlapTable olapTable, ReplacePartit } } olapTable.replaceTempPartitions(partitionNames, tempPartitionNames, isStrictRange, useTempPartitionName); - long version = olapTable.getNextVersion(); + long version; long versionTime = System.currentTimeMillis(); - olapTable.updateVisibleVersionAndTime(version, versionTime); + if (Config.isNotCloudMode()) { + version = olapTable.getNextVersion(); + olapTable.updateVisibleVersionAndTime(version, versionTime); + } else { + version = olapTable.getVisibleVersion(); + } // write log ReplacePartitionOperationLog info = new ReplacePartitionOperationLog(db.getId(), db.getFullName(), olapTable.getId(), olapTable.getName(), diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/FunctionSet.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/FunctionSet.java index b69d52b2c11c1b..29ff913b8aabac 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/FunctionSet.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/FunctionSet.java @@ -46,6 +46,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Map.Entry; import java.util.Set; public class FunctionSet { @@ -924,6 +925,17 @@ private void initAggregateBuiltins() { // sum in multi distinct if (t.equals(Type.BIGINT) || t.equals(Type.LARGEINT) || t.equals(Type.DOUBLE)) { addBuiltin(AggregateFunction.createBuiltin("multi_distinct_sum", Lists.newArrayList(t), + t, + t, + "", + "", + "", + "", + null, + null, + "", + false, true, false, true)); + addBuiltin(AggregateFunction.createBuiltin("multi_distinct_sum0", Lists.newArrayList(t), t, t, "", @@ -937,6 +949,17 @@ private void initAggregateBuiltins() { } else if (t.equals(Type.MAX_DECIMALV2_TYPE)) { // vectorized addBuiltin(AggregateFunction.createBuiltin("multi_distinct_sum", Lists.newArrayList(t), + MULTI_DISTINCT_SUM_RETURN_TYPE.get(t), + Type.MAX_DECIMALV2_TYPE, + "", + "", + "", + "", + null, + null, + "", + false, true, false, true)); + addBuiltin(AggregateFunction.createBuiltin("multi_distinct_sum0", Lists.newArrayList(t), MULTI_DISTINCT_SUM_RETURN_TYPE.get(t), Type.MAX_DECIMALV2_TYPE, "", @@ -950,6 +973,17 @@ private void initAggregateBuiltins() { } else if (t.equals(Type.DECIMAL32)) { // vectorized addBuiltin(AggregateFunction.createBuiltin("multi_distinct_sum", Lists.newArrayList(t), + MULTI_DISTINCT_SUM_RETURN_TYPE.get(t), + Type.DECIMAL32, + "", + "", + "", + "", + null, + null, + "", + false, true, false, true)); + addBuiltin(AggregateFunction.createBuiltin("multi_distinct_sum0", Lists.newArrayList(t), MULTI_DISTINCT_SUM_RETURN_TYPE.get(t), Type.DECIMAL32, "", @@ -962,6 +996,17 @@ private void initAggregateBuiltins() { false, true, true, true)); } else if (t.equals(Type.DECIMAL64)) { addBuiltin(AggregateFunction.createBuiltin("multi_distinct_sum", Lists.newArrayList(t), + MULTI_DISTINCT_SUM_RETURN_TYPE.get(t), + Type.DECIMAL64, + "", + "", + "", + "", + null, + null, + "", + false, true, false, true)); + addBuiltin(AggregateFunction.createBuiltin("multi_distinct_sum0", Lists.newArrayList(t), MULTI_DISTINCT_SUM_RETURN_TYPE.get(t), Type.DECIMAL64, "", @@ -974,6 +1019,17 @@ private void initAggregateBuiltins() { false, true, true, true)); } else if (t.equals(Type.DECIMAL128)) { addBuiltin(AggregateFunction.createBuiltin("multi_distinct_sum", Lists.newArrayList(t), + MULTI_DISTINCT_SUM_RETURN_TYPE.get(t), + Type.DECIMAL128, + "", + "", + "", + "", + null, + null, + "", + false, true, false, true)); + addBuiltin(AggregateFunction.createBuiltin("multi_distinct_sum0", Lists.newArrayList(t), MULTI_DISTINCT_SUM_RETURN_TYPE.get(t), Type.DECIMAL128, "", @@ -1174,86 +1230,91 @@ private void initAggregateBuiltins() { } // Sum - String []sumNames = {"sum", "sum_distinct"}; - for (String name : sumNames) { - addBuiltin(AggregateFunction.createBuiltin(name, + // functionName(String) -> returnsNonNullOnEmpty(Boolean) + Map sumNames = ImmutableMap.of( + "sum", false, + "sum_distinct", false, + "sum0", true + ); + for (Entry nameWithReturn : sumNames.entrySet()) { + addBuiltin(AggregateFunction.createBuiltin(nameWithReturn.getKey(), Lists.newArrayList(Type.BOOLEAN), Type.BIGINT, Type.BIGINT, "", "", "", null, null, "", - null, false, true, false, true)); + null, false, true, nameWithReturn.getValue(), true)); - addBuiltin(AggregateFunction.createBuiltin(name, + addBuiltin(AggregateFunction.createBuiltin(nameWithReturn.getKey(), Lists.newArrayList(Type.TINYINT), Type.BIGINT, Type.BIGINT, "", "", "", null, null, "", - null, false, true, false, true)); - addBuiltin(AggregateFunction.createBuiltin(name, + null, false, true, nameWithReturn.getValue(), true)); + addBuiltin(AggregateFunction.createBuiltin(nameWithReturn.getKey(), Lists.newArrayList(Type.SMALLINT), Type.BIGINT, Type.BIGINT, "", "", "", null, null, "", - null, false, true, false, true)); - addBuiltin(AggregateFunction.createBuiltin(name, + null, false, true, nameWithReturn.getValue(), true)); + addBuiltin(AggregateFunction.createBuiltin(nameWithReturn.getKey(), Lists.newArrayList(Type.INT), Type.BIGINT, Type.BIGINT, "", "", "", null, null, "", - null, false, true, false, true)); - addBuiltin(AggregateFunction.createBuiltin(name, + null, false, true, nameWithReturn.getValue(), true)); + addBuiltin(AggregateFunction.createBuiltin(nameWithReturn.getKey(), Lists.newArrayList(Type.BIGINT), Type.BIGINT, Type.BIGINT, "", "", "", null, null, "", - null, false, true, false, true)); - addBuiltin(AggregateFunction.createBuiltin(name, + null, false, true, nameWithReturn.getValue(), true)); + addBuiltin(AggregateFunction.createBuiltin(nameWithReturn.getKey(), Lists.newArrayList(Type.DOUBLE), Type.DOUBLE, Type.DOUBLE, "", "", "", null, null, "", - null, false, true, false, true)); - addBuiltin(AggregateFunction.createBuiltin(name, + null, false, true, nameWithReturn.getValue(), true)); + addBuiltin(AggregateFunction.createBuiltin(nameWithReturn.getKey(), Lists.newArrayList(Type.MAX_DECIMALV2_TYPE), Type.MAX_DECIMALV2_TYPE, Type.MAX_DECIMALV2_TYPE, "", "", "", null, null, "", - null, false, true, false, true)); - addBuiltin(AggregateFunction.createBuiltin(name, + null, false, true, nameWithReturn.getValue(), true)); + addBuiltin(AggregateFunction.createBuiltin(nameWithReturn.getKey(), Lists.newArrayList(Type.DECIMAL32), ScalarType.DECIMAL128, Type.DECIMAL128, "", "", "", null, null, "", - null, false, true, false, true)); - addBuiltin(AggregateFunction.createBuiltin(name, + null, false, true, nameWithReturn.getValue(), true)); + addBuiltin(AggregateFunction.createBuiltin(nameWithReturn.getKey(), Lists.newArrayList(Type.DECIMAL64), Type.DECIMAL128, Type.DECIMAL128, "", "", "", null, null, "", - null, false, true, false, true)); - addBuiltin(AggregateFunction.createBuiltin(name, + null, false, true, nameWithReturn.getValue(), true)); + addBuiltin(AggregateFunction.createBuiltin(nameWithReturn.getKey(), Lists.newArrayList(Type.DECIMAL128), Type.DECIMAL128, Type.DECIMAL128, "", "", "", null, null, "", - null, false, true, false, true)); - addBuiltin(AggregateFunction.createBuiltin(name, + null, false, true, nameWithReturn.getValue(), true)); + addBuiltin(AggregateFunction.createBuiltin(nameWithReturn.getKey(), Lists.newArrayList(Type.LARGEINT), Type.LARGEINT, Type.LARGEINT, "", "", "", null, null, "", - null, false, true, false, true)); + null, false, true, nameWithReturn.getValue(), true)); } Type[] types = {Type.SMALLINT, Type.TINYINT, Type.INT, Type.BIGINT, Type.FLOAT, Type.DOUBLE, Type.CHAR, 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 33f0d9f9a57678..333b7b146acf27 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 @@ -34,6 +34,7 @@ import java.text.DecimalFormat; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Map; @@ -267,55 +268,70 @@ private static List> getDataSkew(String dbName, String tblName, Par List> result = Lists.newArrayList(); Env env = Env.getCurrentEnv(); - if (partitionNames == null || partitionNames.getPartitionNames().size() != 1) { - throw new DdlException("Should specify one and only one partitions"); - } - Database db = env.getInternalCatalog().getDbOrDdlException(dbName); OlapTable olapTable = db.getOlapTableOrDdlException(tblName); + if (olapTable.getPartitionNames().isEmpty()) { + throw new DdlException("Can not find any partition from " + dbName + "." + tblName); + } + + // patition -> isTmep + Map allPartionNames = new HashMap<>(); + if (partitionNames == null) { + for (Partition p : olapTable.getPartitions()) { + allPartionNames.put(p.getName(), false); + } + for (Partition p : olapTable.getTempPartitions()) { + allPartionNames.put(p.getName(), true); + } + } else { + for (String name : partitionNames.getPartitionNames()) { + allPartionNames.put(name, partitionNames.isTemp()); + } + } + olapTable.readLock(); try { Partition partition = null; // check partition - for (String partName : partitionNames.getPartitionNames()) { - partition = olapTable.getPartition(partName, partitionNames.isTemp()); + for (Map.Entry partName : allPartionNames.entrySet()) { + partition = olapTable.getPartition(partName.getKey(), partName.getValue()); if (partition == null) { throw new DdlException("Partition does not exist: " + partName); } - break; - } - DistributionInfo distributionInfo = partition.getDistributionInfo(); - List rowCountTabletInfos = Lists.newArrayListWithCapacity(distributionInfo.getBucketNum()); - List dataSizeTabletInfos = Lists.newArrayListWithCapacity(distributionInfo.getBucketNum()); - for (long i = 0; i < distributionInfo.getBucketNum(); i++) { - rowCountTabletInfos.add(0L); - dataSizeTabletInfos.add(0L); - } + DistributionInfo distributionInfo = partition.getDistributionInfo(); + List rowCountTabletInfos = Lists.newArrayListWithCapacity(distributionInfo.getBucketNum()); + List dataSizeTabletInfos = Lists.newArrayListWithCapacity(distributionInfo.getBucketNum()); + for (long i = 0; i < distributionInfo.getBucketNum(); i++) { + rowCountTabletInfos.add(0L); + dataSizeTabletInfos.add(0L); + } - long totalSize = 0; - for (MaterializedIndex mIndex : partition.getMaterializedIndices(IndexExtState.VISIBLE)) { - List tabletIds = mIndex.getTabletIdsInOrder(); - for (int i = 0; i < tabletIds.size(); i++) { - Tablet tablet = mIndex.getTablet(tabletIds.get(i)); - long rowCount = tablet.getRowCount(true); - long dataSize = tablet.getDataSize(true); - rowCountTabletInfos.set(i, rowCountTabletInfos.get(i) + rowCount); - dataSizeTabletInfos.set(i, dataSizeTabletInfos.get(i) + dataSize); - totalSize += dataSize; + long totalSize = 0; + for (MaterializedIndex mIndex : partition.getMaterializedIndices(IndexExtState.VISIBLE)) { + List tabletIds = mIndex.getTabletIdsInOrder(); + for (int i = 0; i < tabletIds.size(); i++) { + Tablet tablet = mIndex.getTablet(tabletIds.get(i)); + long rowCount = tablet.getRowCount(true); + long dataSize = tablet.getDataSize(true); + rowCountTabletInfos.set(i, rowCountTabletInfos.get(i) + rowCount); + dataSizeTabletInfos.set(i, dataSizeTabletInfos.get(i) + dataSize); + totalSize += dataSize; + } } - } - // graph - for (int i = 0; i < distributionInfo.getBucketNum(); i++) { - List row = Lists.newArrayList(); - row.add(String.valueOf(i)); - row.add(rowCountTabletInfos.get(i).toString()); - row.add(dataSizeTabletInfos.get(i).toString()); - row.add(graph(dataSizeTabletInfos.get(i), totalSize)); - row.add(totalSize == dataSizeTabletInfos.get(i) ? (totalSize == 0L ? "0.00%" : "100.00%") : - df.format((double) dataSizeTabletInfos.get(i) / totalSize)); - result.add(row); + // graph + for (int i = 0; i < distributionInfo.getBucketNum(); i++) { + List row = Lists.newArrayList(); + row.add(partName.getKey()); + row.add(String.valueOf(i)); + row.add(rowCountTabletInfos.get(i).toString()); + row.add(dataSizeTabletInfos.get(i).toString()); + row.add(graph(dataSizeTabletInfos.get(i), totalSize)); + row.add(totalSize == dataSizeTabletInfos.get(i) ? (totalSize == 0L ? "0.00%" : "100.00%") : + df.format((double) dataSizeTabletInfos.get(i) / totalSize)); + result.add(row); + } } } finally { olapTable.readUnlock(); 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 b6239f486cdeec..89bba35645de2c 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 @@ -37,6 +37,8 @@ import org.apache.doris.clone.TabletSchedCtx; import org.apache.doris.clone.TabletScheduler; import org.apache.doris.cloud.catalog.CloudPartition; +import org.apache.doris.cloud.proto.Cloud; +import org.apache.doris.cloud.qe.SnapshotProxy; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.Config; import org.apache.doris.common.DdlException; @@ -46,6 +48,7 @@ import org.apache.doris.common.UserException; import org.apache.doris.common.io.DeepCopy; import org.apache.doris.common.io.Text; +import org.apache.doris.common.profile.SummaryProfile; import org.apache.doris.common.util.PropertyAnalyzer; import org.apache.doris.common.util.Util; import org.apache.doris.mtmv.MTMVRelatedTableIf; @@ -53,7 +56,9 @@ import org.apache.doris.mtmv.MTMVVersionSnapshot; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.OriginStatement; +import org.apache.doris.qe.StmtExecutor; import org.apache.doris.resource.Tag; +import org.apache.doris.rpc.RpcException; import org.apache.doris.statistics.AnalysisInfo; import org.apache.doris.statistics.AnalysisInfo.AnalysisType; import org.apache.doris.statistics.BaseAnalysisTask; @@ -1279,11 +1284,11 @@ public boolean needReAnalyzeTable(TableStatsMeta tblStats) { if (tblStats == null) { return true; } - if (!tblStats.analyzeColumns().containsAll(getBaseSchema() + if (!tblStats.analyzeColumns().containsAll(getColumnIndexPairs(getSchemaAllIndexes(false) .stream() .filter(c -> !StatisticsUtil.isUnsupportedType(c.getType())) .map(Column::getName) - .collect(Collectors.toSet()))) { + .collect(Collectors.toSet())))) { return true; } long rowCount = getRowCount(); @@ -1296,34 +1301,20 @@ public boolean needReAnalyzeTable(TableStatsMeta tblStats) { } @Override - public Map> findReAnalyzeNeededPartitions() { - TableStatsMeta tableStats = Env.getCurrentEnv().getAnalysisManager().findTableStatsStatus(getId()); - Set allPartitions = getPartitionNames().stream().map(this::getPartition) - .filter(Partition::hasData).map(Partition::getName).collect(Collectors.toSet()); - if (tableStats == null) { - Map> ret = Maps.newHashMap(); - for (Column col : getSchemaAllIndexes(false)) { - if (StatisticsUtil.isUnsupportedType(col.getType())) { + public List> getColumnIndexPairs(Set columns) { + List> ret = Lists.newArrayList(); + // Check the schema of all indexes for each given column name, + // If the column name exists in the index, add the pair to return list. + for (String column : columns) { + for (MaterializedIndexMeta meta : indexIdToMeta.values()) { + Column col = meta.getColumnByName(column); + if (col == null || StatisticsUtil.isUnsupportedType(col.getType())) { continue; } - ret.put(col.getName(), allPartitions); + ret.add(Pair.of(getIndexNameById(meta.getIndexId()), column)); } - return ret; } - Map> colToPart = new HashMap<>(); - for (Column col : getSchemaAllIndexes(false)) { - if (StatisticsUtil.isUnsupportedType(col.getType())) { - continue; - } - long lastUpdateTime = tableStats.findColumnLastUpdateTime(col.getName()); - Set partitions = getPartitionNames().stream() - .map(this::getPartition) - .filter(Partition::hasData) - .filter(partition -> partition.getVisibleVersionTime() >= lastUpdateTime).map(Partition::getName) - .collect(Collectors.toSet()); - colToPart.put(col.getName(), partitions); - } - return colToPart; + return ret; } @Override @@ -2666,11 +2657,48 @@ public void updateVisibleVersionAndTime(long visibleVersion, long visibleVersion // During `getNextVersion` and `updateVisibleVersionAndTime` period, // the write lock on the table should be held continuously public long getNextVersion() { - return tableAttributes.getNextVersion(); + if (!Config.isCloudMode()) { + return tableAttributes.getNextVersion(); + } else { + // cloud mode should not reach here + if (LOG.isDebugEnabled()) { + LOG.debug("getNextVersion in Cloud mode in OlapTable {} ", getName()); + } + return getVisibleVersion() + 1; + } } public long getVisibleVersion() { - return tableAttributes.getVisibleVersion(); + if (Config.isNotCloudMode()) { + return tableAttributes.getVisibleVersion(); + } + // get version rpc + Cloud.GetVersionRequest request = Cloud.GetVersionRequest.newBuilder() + .setDbId(this.getDatabase().getId()) + .setTableId(this.id) + .setBatchMode(false) + .setIsTableVersion(true) + .build(); + + try { + Cloud.GetVersionResponse resp = getVersionFromMeta(request); + long version = -1; + if (resp.getStatus().getCode() == Cloud.MetaServiceCode.OK) { + version = resp.getVersion(); + } else { + assert resp.getStatus().getCode() == Cloud.MetaServiceCode.VERSION_NOT_FOUND; + version = 0; + } + if (LOG.isDebugEnabled()) { + LOG.debug("get version from meta service, version: {}, table: {}", version, getId()); + } + if (version == 0) { + version = 1; + } + return version; + } catch (RpcException e) { + throw new RuntimeException("get version from meta service failed"); + } } public long getVisibleVersionTime() { @@ -2719,6 +2747,19 @@ public String getPartitionName(long partitionId) throws AnalysisException { } } + private static Cloud.GetVersionResponse getVersionFromMeta(Cloud.GetVersionRequest req) + throws RpcException { + long startAt = System.nanoTime(); + try { + return SnapshotProxy.getVisibleVersion(req); + } finally { + SummaryProfile profile = getSummaryProfile(); + if (profile != null) { + profile.addGetTableVersionTime(System.nanoTime() - startAt); + } + } + } + @Override public boolean needAutoRefresh() { return true; @@ -2726,6 +2767,17 @@ public boolean needAutoRefresh() { @Override public boolean isPartitionColumnAllowNull() { - return false; + return true; + } + + private static SummaryProfile getSummaryProfile() { + ConnectContext ctx = ConnectContext.get(); + if (ctx != null) { + StmtExecutor executor = ctx.getExecutor(); + if (executor != null) { + return executor.getSummaryProfile(); + } + } + return null; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/SchemaTable.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/SchemaTable.java index f06c05839f979b..89d1c363c101af 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/SchemaTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/SchemaTable.java @@ -486,6 +486,7 @@ public class SchemaTable extends Table { .column("MIN_REMOTE_SCAN_THREAD_NUM", ScalarType.createType(PrimitiveType.BIGINT)) .column("SPILL_THRESHOLD_LOW_WATERMARK", ScalarType.createVarchar(256)) .column("SPILL_THRESHOLD_HIGH_WATERMARK", ScalarType.createVarchar(256)) + .column("TAG", ScalarType.createVarchar(256)) .build())) .put("processlist", new SchemaTable(SystemIdGenerator.getNextId(), "processlist", TableType.SCHEMA, builder().column("ID", ScalarType.createType(PrimitiveType.LARGEINT)) 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 35f5b14efc5ab9..52655fa064943d 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 @@ -24,6 +24,7 @@ import org.apache.doris.common.ErrorCode; import org.apache.doris.common.FeMetaVersion; import org.apache.doris.common.MetaNotFoundException; +import org.apache.doris.common.Pair; import org.apache.doris.common.io.Text; import org.apache.doris.common.io.Writable; import org.apache.doris.common.util.QueryableReentrantReadWriteLock; @@ -647,11 +648,6 @@ public boolean needReAnalyzeTable(TableStatsMeta tblStats) { return true; } - @Override - public Map> findReAnalyzeNeededPartitions() { - return Collections.emptyMap(); - } - @Override public List getChunkSizes() { throw new NotImplementedException("getChunkSized not implemented"); @@ -661,4 +657,9 @@ public List getChunkSizes() { public long fetchRowCount() { return 0; } + + @Override + public List> getColumnIndexPairs(Set columns) { + return Lists.newArrayList(); + } } 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 484dd3bb6ebfe3..f7c8b4b83252bf 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 @@ -25,6 +25,7 @@ import org.apache.doris.cluster.ClusterNamespace; import org.apache.doris.common.DdlException; import org.apache.doris.common.MetaNotFoundException; +import org.apache.doris.common.Pair; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.persist.AlterConstraintLog; import org.apache.doris.statistics.AnalysisInfo; @@ -184,7 +185,11 @@ default long getRowCountForNereids() { boolean needReAnalyzeTable(TableStatsMeta tblStats); - Map> findReAnalyzeNeededPartitions(); + /** + * @param columns Set of column names. + * @return List of pairs. Each pair is . For external table, index name is table name. + */ + List> getColumnIndexPairs(Set columns); // Get all the chunk sizes of this table. Now, only HMS external table implemented this interface. // For HMS external table, the return result is a list of all the files' size. diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudClusterChecker.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudClusterChecker.java index cc4a3c09d9cb7b..801f3166861497 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudClusterChecker.java +++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudClusterChecker.java @@ -94,19 +94,13 @@ private void checkToAddCluster(Map remoteClusterIdToPB, Set newTagMap = Tag.DEFAULT_BACKEND_TAG.toMap(); String clusterName = remoteClusterIdToPB.get(addId).getClusterName(); String clusterId = remoteClusterIdToPB.get(addId).getClusterId(); String publicEndpoint = remoteClusterIdToPB.get(addId).getPublicEndpoint(); String privateEndpoint = remoteClusterIdToPB.get(addId).getPrivateEndpoint(); - newTagMap.put(Tag.CLOUD_CLUSTER_NAME, clusterName); - newTagMap.put(Tag.CLOUD_CLUSTER_ID, clusterId); - newTagMap.put(Tag.CLOUD_CLUSTER_PUBLIC_ENDPOINT, publicEndpoint); - newTagMap.put(Tag.CLOUD_CLUSTER_PRIVATE_ENDPOINT, privateEndpoint); // For old versions that do no have status field set ClusterStatus clusterStatus = remoteClusterIdToPB.get(addId).hasClusterStatus() ? remoteClusterIdToPB.get(addId).getClusterStatus() : ClusterStatus.NORMAL; - newTagMap.put(Tag.CLOUD_CLUSTER_STATUS, String.valueOf(clusterStatus)); MetricRepo.registerCloudMetrics(clusterId, clusterName); //toAdd.forEach(i -> i.setTagMap(newTagMap)); List toAdd = new ArrayList<>(); @@ -117,6 +111,12 @@ private void checkToAddCluster(Map remoteClusterIdToPB, Set newTagMap = Tag.DEFAULT_BACKEND_TAG.toMap(); + newTagMap.put(Tag.CLOUD_CLUSTER_STATUS, String.valueOf(clusterStatus)); + newTagMap.put(Tag.CLOUD_CLUSTER_NAME, clusterName); + newTagMap.put(Tag.CLOUD_CLUSTER_ID, clusterId); + newTagMap.put(Tag.CLOUD_CLUSTER_PUBLIC_ENDPOINT, publicEndpoint); + newTagMap.put(Tag.CLOUD_CLUSTER_PRIVATE_ENDPOINT, privateEndpoint); newTagMap.put(Tag.CLOUD_UNIQUE_ID, node.getCloudUniqueId()); b.setTagMap(newTagMap); toAdd.add(b); @@ -250,13 +250,6 @@ private void checkDiffNode(Map remoteClusterIdToPB, updateStatus(currentBes, expectedBes); - // Attach tag to BEs - Map newTagMap = Tag.DEFAULT_BACKEND_TAG.toMap(); - newTagMap.put(Tag.CLOUD_CLUSTER_NAME, remoteClusterIdToPB.get(cid).getClusterName()); - newTagMap.put(Tag.CLOUD_CLUSTER_ID, remoteClusterIdToPB.get(cid).getClusterId()); - newTagMap.put(Tag.CLOUD_CLUSTER_PUBLIC_ENDPOINT, remoteClusterIdToPB.get(cid).getPublicEndpoint()); - newTagMap.put(Tag.CLOUD_CLUSTER_PRIVATE_ENDPOINT, remoteClusterIdToPB.get(cid).getPrivateEndpoint()); - diffNodes(toAdd, toDel, () -> { Map currentMap = new HashMap<>(); for (Backend be : currentBes) { @@ -280,6 +273,14 @@ private void checkDiffNode(Map remoteClusterIdToPB, if (node.hasIsSmoothUpgrade()) { b.setSmoothUpgradeDst(node.getIsSmoothUpgrade()); } + + // Attach tag to BEs + Map newTagMap = Tag.DEFAULT_BACKEND_TAG.toMap(); + newTagMap.put(Tag.CLOUD_CLUSTER_NAME, remoteClusterIdToPB.get(cid).getClusterName()); + newTagMap.put(Tag.CLOUD_CLUSTER_ID, remoteClusterIdToPB.get(cid).getClusterId()); + newTagMap.put(Tag.CLOUD_CLUSTER_PUBLIC_ENDPOINT, remoteClusterIdToPB.get(cid).getPublicEndpoint()); + newTagMap.put(Tag.CLOUD_CLUSTER_PRIVATE_ENDPOINT, + remoteClusterIdToPB.get(cid).getPrivateEndpoint()); newTagMap.put(Tag.CLOUD_UNIQUE_ID, node.getCloudUniqueId()); b.setTagMap(newTagMap); nodeMap.put(endpoint, b); @@ -350,8 +351,8 @@ private void checkFeNodesMapValid() { } private void getCloudObserverFes() { - Cloud.GetClusterResponse response = CloudSystemInfoService - .getCloudCluster(Config.cloud_sql_server_cluster_name, Config.cloud_sql_server_cluster_id, ""); + Cloud.GetClusterResponse response = cloudSystemInfoService.getCloudCluster( + Config.cloud_sql_server_cluster_name, Config.cloud_sql_server_cluster_id, ""); if (!response.hasStatus() || !response.getStatus().hasCode() || response.getStatus().getCode() != Cloud.MetaServiceCode.OK) { LOG.warn("failed to get cloud cluster due to incomplete response, " @@ -416,7 +417,7 @@ private void getCloudObserverFes() { return; } try { - CloudSystemInfoService.updateFrontends(toAdd, toDel); + cloudSystemInfoService.updateFrontends(toAdd, toDel); } catch (DdlException e) { LOG.warn("update cloud frontends exception e: {}, msg: {}", e, e.getMessage()); } @@ -426,7 +427,7 @@ private void getCloudBackends() { Map> clusterIdToBackend = cloudSystemInfoService.getCloudClusterIdToBackend(); //rpc to ms, to get mysql user can use cluster_id // NOTE: rpc args all empty, use cluster_unique_id to get a instance's all cluster info. - Cloud.GetClusterResponse response = CloudSystemInfoService.getCloudCluster("", "", ""); + Cloud.GetClusterResponse response = cloudSystemInfoService.getCloudCluster("", "", ""); if (!response.hasStatus() || !response.getStatus().hasCode() || (response.getStatus().getCode() != Cloud.MetaServiceCode.OK && response.getStatus().getCode() != MetaServiceCode.CLUSTER_NOT_FOUND)) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudEnv.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudEnv.java index 613fef3be685f5..7c37f1dbcffa98 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudEnv.java +++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudEnv.java @@ -83,8 +83,8 @@ public static String genFeNodeNameFromMeta(String host, int port, long timeMs) { private Cloud.NodeInfoPB getLocalTypeFromMetaService() { // get helperNodes from ms - Cloud.GetClusterResponse response = CloudSystemInfoService.getCloudCluster( - Config.cloud_sql_server_cluster_name, Config.cloud_sql_server_cluster_id, ""); + Cloud.GetClusterResponse response = ((CloudSystemInfoService) Env.getCurrentSystemInfo()) + .getCloudCluster(Config.cloud_sql_server_cluster_name, Config.cloud_sql_server_cluster_id, ""); if (!response.hasStatus() || !response.getStatus().hasCode() || response.getStatus().getCode() != Cloud.MetaServiceCode.OK) { LOG.warn("failed to get cloud cluster due to incomplete response, " @@ -392,7 +392,7 @@ public void checkCloudClusterPriv(String clusterName) throws DdlException { public void changeCloudCluster(String clusterName, ConnectContext ctx) throws DdlException { checkCloudClusterPriv(clusterName); - CloudSystemInfoService.waitForAutoStart(clusterName); + ((CloudSystemInfoService) Env.getCurrentSystemInfo()).waitForAutoStart(clusterName); try { ((CloudSystemInfoService) Env.getCurrentSystemInfo()).addCloudCluster(clusterName, ""); } catch (UserException e) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudPartition.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudPartition.java index 01603823752762..60e567ce9fe001 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudPartition.java +++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudPartition.java @@ -22,8 +22,7 @@ import org.apache.doris.catalog.Partition; import org.apache.doris.cloud.proto.Cloud; import org.apache.doris.cloud.proto.Cloud.MetaServiceCode; -import org.apache.doris.cloud.rpc.MetaServiceProxy; -import org.apache.doris.common.Config; +import org.apache.doris.cloud.qe.SnapshotProxy; import org.apache.doris.common.profile.SummaryProfile; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.StmtExecutor; @@ -38,10 +37,6 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; import java.util.concurrent.locks.ReentrantLock; import java.util.stream.Collectors; @@ -328,7 +323,7 @@ private static Cloud.GetVersionResponse getVersionFromMeta(Cloud.GetVersionReque throws RpcException { long startAt = System.nanoTime(); try { - return getVersionFromMetaInner(req); + return SnapshotProxy.getVisibleVersion(req); } finally { SummaryProfile profile = getSummaryProfile(); if (profile != null) { @@ -337,55 +332,6 @@ private static Cloud.GetVersionResponse getVersionFromMeta(Cloud.GetVersionReque } } - private static Cloud.GetVersionResponse getVersionFromMetaInner(Cloud.GetVersionRequest req) - throws RpcException { - for (int retryTime = 0; retryTime < Config.cloud_meta_service_rpc_failed_retry_times; retryTime++) { - try { - long deadline = System.currentTimeMillis() + Config.default_get_version_from_ms_timeout_second * 1000L; - Future future = - MetaServiceProxy.getInstance().getVisibleVersionAsync(req); - - Cloud.GetVersionResponse resp = null; - while (resp == null) { - try { - resp = future.get(Math.max(0, deadline - System.currentTimeMillis()), TimeUnit.MILLISECONDS); - } catch (InterruptedException e) { - LOG.warn("get version from meta service: future get interrupted exception"); - } - } - - if (resp.hasStatus() && (resp.getStatus().getCode() == MetaServiceCode.OK - || resp.getStatus().getCode() == MetaServiceCode.VERSION_NOT_FOUND)) { - if (LOG.isDebugEnabled()) { - LOG.debug("get version from meta service, code: {}", resp.getStatus().getCode()); - } - return resp; - } - - LOG.warn("get version from meta service failed, status: {}, retry time: {}", - resp.getStatus(), retryTime); - } catch (RpcException | ExecutionException | TimeoutException | RuntimeException e) { - LOG.warn("get version from meta service failed, retry times: {} exception: ", retryTime, e); - } - - // sleep random millis [20, 200] ms, retry rpc failed - int randomMillis = 20 + (int) (Math.random() * (200 - 20)); - if (retryTime > Config.cloud_meta_service_rpc_failed_retry_times / 2) { - // sleep random millis [500, 1000] ms, retry rpc failed - randomMillis = 500 + (int) (Math.random() * (1000 - 500)); - } - try { - Thread.sleep(randomMillis); - } catch (InterruptedException e) { - LOG.warn("get version from meta service: sleep get interrupted exception"); - } - } - - LOG.warn("get version from meta service failed after retry {} times", - Config.cloud_meta_service_rpc_failed_retry_times); - throw new RpcException("get version from meta service", "failed after retry n times"); - } - private static boolean isEmptyPartitionPruneDisabled() { ConnectContext ctx = ConnectContext.get(); if (ctx != null && ctx.getSessionVariable().getDisableEmptyPartitionPrune()) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudReplica.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudReplica.java index a51e332a784218..aebc66128ed844 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudReplica.java +++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudReplica.java @@ -163,7 +163,7 @@ private long getBackendIdImpl(String cluster) { // if cluster is SUSPENDED, wait try { - CloudSystemInfoService.waitForAutoStart(cluster); + ((CloudSystemInfoService) Env.getCurrentSystemInfo()).waitForAutoStart(cluster); } catch (DdlException e) { // this function cant throw exception. so just log it LOG.warn("cant resume cluster {}", cluster); diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/datasource/CloudInternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/datasource/CloudInternalCatalog.java index f48c38de698b29..b473f80e126959 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/cloud/datasource/CloudInternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/datasource/CloudInternalCatalog.java @@ -361,12 +361,13 @@ protected void beforeCreatePartitions(long dbId, long tableId, List partit } @Override - protected void afterCreatePartitions(long tableId, List partitionIds, List indexIds) + protected void afterCreatePartitions(long dbId, long tableId, List partitionIds, List indexIds, + boolean isCreateTable) throws DdlException { if (partitionIds == null) { - commitMaterializedIndex(tableId, indexIds); + commitMaterializedIndex(dbId, tableId, indexIds, isCreateTable); } else { - commitPartition(tableId, partitionIds, indexIds); + commitPartition(dbId, tableId, partitionIds, indexIds); } } @@ -406,11 +407,13 @@ private void preparePartition(long dbId, long tableId, List partitionIds, } } - private void commitPartition(long tableId, List partitionIds, List indexIds) throws DdlException { + private void commitPartition(long dbId, long tableId, List partitionIds, List indexIds) + throws DdlException { Cloud.PartitionRequest.Builder partitionRequestBuilder = Cloud.PartitionRequest.newBuilder(); partitionRequestBuilder.setCloudUniqueId(Config.cloud_unique_id); partitionRequestBuilder.addAllPartitionIds(partitionIds); partitionRequestBuilder.addAllIndexIds(indexIds); + partitionRequestBuilder.setDbId(dbId); partitionRequestBuilder.setTableId(tableId); final Cloud.PartitionRequest partitionRequest = partitionRequestBuilder.build(); @@ -469,11 +472,14 @@ public void prepareMaterializedIndex(Long tableId, List indexIds, long exp } } - public void commitMaterializedIndex(Long tableId, List indexIds) throws DdlException { + public void commitMaterializedIndex(long dbId, long tableId, List indexIds, boolean isCreateTable) + throws DdlException { Cloud.IndexRequest.Builder indexRequestBuilder = Cloud.IndexRequest.newBuilder(); indexRequestBuilder.setCloudUniqueId(Config.cloud_unique_id); indexRequestBuilder.addAllIndexIds(indexIds); + indexRequestBuilder.setDbId(dbId); indexRequestBuilder.setTableId(tableId); + indexRequestBuilder.setIsNewTable(isCreateTable); final Cloud.IndexRequest indexRequest = indexRequestBuilder.build(); Cloud.IndexResponse response = null; @@ -562,7 +568,7 @@ public void eraseTableDropBackendReplicas(OlapTable olapTable, boolean isReplay) if (indexs.isEmpty()) { break; } - dropMaterializedIndex(olapTable.getId(), indexs); + dropMaterializedIndex(olapTable.getId(), indexs, true); } catch (Exception e) { LOG.warn("failed to drop index {} of table {}, try cnt {}, execption {}", indexs, olapTable.getId(), tryCnt, e); @@ -657,7 +663,7 @@ private void dropCloudPartition(long dbId, long tableId, List partitionIds } } - public void dropMaterializedIndex(Long tableId, List indexIds) throws DdlException { + public void dropMaterializedIndex(long tableId, List indexIds, boolean dropTable) throws DdlException { Cloud.IndexRequest.Builder indexRequestBuilder = Cloud.IndexRequest.newBuilder(); indexRequestBuilder.setCloudUniqueId(Config.cloud_unique_id); indexRequestBuilder.addAllIndexIds(indexIds); diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/load/CloudLoadManager.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/load/CloudLoadManager.java index 1d0bfc23f6c2de..5caa2108c592fe 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/cloud/load/CloudLoadManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/load/CloudLoadManager.java @@ -19,6 +19,7 @@ import org.apache.doris.analysis.InsertStmt; import org.apache.doris.analysis.LoadStmt; +import org.apache.doris.catalog.Env; import org.apache.doris.cloud.system.CloudSystemInfoService; import org.apache.doris.common.DdlException; import org.apache.doris.common.UserException; @@ -33,14 +34,14 @@ public CloudLoadManager(LoadJobScheduler loadJobScheduler) { @Override public long createLoadJobFromStmt(LoadStmt stmt) throws DdlException, UserException { - CloudSystemInfoService.waitForAutoStartCurrentCluster(); + ((CloudSystemInfoService) Env.getCurrentSystemInfo()).waitForAutoStartCurrentCluster(); return super.createLoadJobFromStmt(stmt); } @Override public long createLoadJobFromStmt(InsertStmt stmt) throws DdlException { - CloudSystemInfoService.waitForAutoStartCurrentCluster(); + ((CloudSystemInfoService) Env.getCurrentSystemInfo()).waitForAutoStartCurrentCluster(); return super.createLoadJobFromStmt(stmt); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/qe/SnapshotProxy.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/qe/SnapshotProxy.java new file mode 100644 index 00000000000000..1f4b20ab1dd457 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/qe/SnapshotProxy.java @@ -0,0 +1,92 @@ +// 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.cloud.qe; + +import org.apache.doris.cloud.proto.Cloud; +import org.apache.doris.cloud.rpc.MetaServiceProxy; +import org.apache.doris.common.Config; +import org.apache.doris.rpc.RpcException; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +public class SnapshotProxy { + private static final Logger LOG = LogManager.getLogger(SnapshotProxy.class); + + public static Cloud.GetVersionResponse getVisibleVersion(Cloud.GetVersionRequest request) throws RpcException { + int tryTimes = 0; + while (tryTimes++ < Config.meta_service_rpc_retry_times) { + Cloud.GetVersionResponse resp = getVisibleVersionInternal(request, + Config.default_get_version_from_ms_timeout_second * 1000); + if (resp.hasStatus() && (resp.getStatus().getCode() == Cloud.MetaServiceCode.OK + || resp.getStatus().getCode() == Cloud.MetaServiceCode.VERSION_NOT_FOUND)) { + if (LOG.isDebugEnabled()) { + LOG.debug("get version from meta service, code: {}", resp.getStatus().getCode()); + } + return resp; + } + + LOG.warn("get version from meta service failed, status: {}, retry time: {}", + resp.getStatus(), tryTimes); + + // sleep random millis, retry rpc failed + if (tryTimes > Config.meta_service_rpc_retry_times / 2) { + sleepSeveralMs(500, 1000); + } else { + sleepSeveralMs(20, 200); + } + } + + LOG.warn("get version from meta service failed after retry {} times", tryTimes); + throw new RpcException("get version from meta service", "failed after retry n times"); + } + + public static Cloud.GetVersionResponse getVisibleVersionInternal(Cloud.GetVersionRequest request, int timeoutMs) { + long deadline = System.currentTimeMillis() + timeoutMs; + Cloud.GetVersionResponse resp = null; + try { + Future future = + MetaServiceProxy.getInstance().getVisibleVersionAsync(request); + + while (resp == null) { + try { + resp = future.get(Math.max(0, deadline - System.currentTimeMillis()), TimeUnit.MILLISECONDS); + } catch (InterruptedException e) { + LOG.warn("get version from meta service: future get interrupted exception"); + } + } + } catch (RpcException | ExecutionException | TimeoutException | RuntimeException e) { + LOG.warn("get version from meta service failed, exception: ", e); + } + return resp; + } + + private static void sleepSeveralMs(int lowerMs, int upperMs) { + // sleep random millis [lowerMs, upperMs] ms + try { + Thread.sleep(lowerMs + (int) (Math.random() * (upperMs - lowerMs))); + } catch (InterruptedException e) { + LOG.warn("get snapshot from meta service: sleep get interrupted exception"); + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/system/CloudSystemInfoService.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/system/CloudSystemInfoService.java index 7171fbafd16267..59ba7340dfc65c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/cloud/system/CloudSystemInfoService.java +++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/system/CloudSystemInfoService.java @@ -89,7 +89,7 @@ public Pair>, TStorageMedium> selectBackendIdsForReplicaCrea * @param clusterId cluster id * @return */ - public static Cloud.GetClusterResponse getCloudCluster(String clusterName, String clusterId, String userName) { + public Cloud.GetClusterResponse getCloudCluster(String clusterName, String clusterId, String userName) { Cloud.GetClusterRequest.Builder builder = Cloud.GetClusterRequest.newBuilder(); builder.setCloudUniqueId(Config.cloud_unique_id) .setClusterName(clusterName).setClusterId(clusterId).setMysqlUserName(userName); @@ -261,8 +261,8 @@ public void updateCloudClusterMap(List toAdd, List toDel) { } - public static synchronized void updateFrontends(List toAdd, - List toDel) throws DdlException { + public synchronized void updateFrontends(List toAdd, List toDel) + throws DdlException { if (LOG.isDebugEnabled()) { LOG.debug("updateCloudFrontends toAdd={} toDel={}", toAdd, toDel); } @@ -570,7 +570,7 @@ public void setInstanceStatus(InstanceInfoPB.Status instanceStatus) { this.instanceStatus = instanceStatus; } - public static void waitForAutoStartCurrentCluster() throws DdlException { + public void waitForAutoStartCurrentCluster() throws DdlException { ConnectContext context = ConnectContext.get(); if (context != null) { String cloudCluster = context.getCloudCluster(); @@ -580,7 +580,7 @@ public static void waitForAutoStartCurrentCluster() throws DdlException { } } - public static String getClusterNameAutoStart(final String clusterName) { + public String getClusterNameAutoStart(final String clusterName) { if (!Strings.isNullOrEmpty(clusterName)) { return clusterName; } @@ -607,7 +607,7 @@ public static String getClusterNameAutoStart(final String clusterName) { return cloudClusterTypeAndName.clusterName; } - public static void waitForAutoStart(String clusterName) throws DdlException { + public void waitForAutoStart(String clusterName) throws DdlException { if (Config.isNotCloudMode()) { return; } @@ -616,7 +616,7 @@ public static void waitForAutoStart(String clusterName) throws DdlException { LOG.warn("auto start in cloud mode, but clusterName empty {}", clusterName); return; } - String clusterStatus = ((CloudSystemInfoService) Env.getCurrentSystemInfo()).getCloudStatusByName(clusterName); + String clusterStatus = getCloudStatusByName(clusterName); if (Strings.isNullOrEmpty(clusterStatus)) { // for cluster rename or cluster dropped LOG.warn("cant find clusterStatus in fe, clusterName {}", clusterName); @@ -631,8 +631,7 @@ public static void waitForAutoStart(String clusterName) throws DdlException { builder.setOp(Cloud.AlterClusterRequest.Operation.SET_CLUSTER_STATUS); Cloud.ClusterPB.Builder clusterBuilder = Cloud.ClusterPB.newBuilder(); - clusterBuilder.setClusterId(((CloudSystemInfoService) - Env.getCurrentSystemInfo()).getCloudClusterIdByName(clusterName)); + clusterBuilder.setClusterId(getCloudClusterIdByName(clusterName)); clusterBuilder.setClusterStatus(Cloud.ClusterStatus.TO_RESUME); builder.setCluster(clusterBuilder); @@ -671,7 +670,7 @@ public static void waitForAutoStart(String clusterName) throws DdlException { } catch (InterruptedException e) { LOG.info("change cluster sleep wait InterruptedException: ", e); } - clusterStatus = ((CloudSystemInfoService) Env.getCurrentSystemInfo()).getCloudStatusByName(clusterName); + clusterStatus = getCloudStatusByName(clusterName); } if (retryTime >= retryTimes) { // auto start timeout diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/profile/SummaryProfile.java b/fe/fe-core/src/main/java/org/apache/doris/common/profile/SummaryProfile.java index d4615dab820afc..7e9367d0abd07c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/profile/SummaryProfile.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/profile/SummaryProfile.java @@ -74,6 +74,8 @@ public class SummaryProfile { public static final String GET_PARTITION_VERSION_TIME = "Get Partition Version Time"; public static final String GET_PARTITION_VERSION_COUNT = "Get Partition Version Count"; public static final String GET_PARTITION_VERSION_BY_HAS_DATA_COUNT = "Get Partition Version Count (hasData)"; + public static final String GET_TABLE_VERSION_TIME = "Get Table Version Time"; + public static final String GET_TABLE_VERSION_COUNT = "Get Table Version Count"; public static final String PARSE_SQL_TIME = "Parse SQL Time"; public static final String NEREIDS_ANALYSIS_TIME = "Nereids Analysis Time"; @@ -93,7 +95,8 @@ public class SummaryProfile { PLAN_TIME, JOIN_REORDER_TIME, CREATE_SINGLE_NODE_TIME, QUERY_DISTRIBUTED_TIME, INIT_SCAN_NODE_TIME, FINALIZE_SCAN_NODE_TIME, GET_SPLITS_TIME, GET_PARTITIONS_TIME, GET_PARTITION_FILES_TIME, CREATE_SCAN_RANGE_TIME, GET_PARTITION_VERSION_TIME, - GET_PARTITION_VERSION_BY_HAS_DATA_COUNT, GET_PARTITION_VERSION_COUNT, SCHEDULE_TIME, FETCH_RESULT_TIME, + GET_PARTITION_VERSION_BY_HAS_DATA_COUNT, GET_PARTITION_VERSION_COUNT, GET_TABLE_VERSION_TIME, + GET_TABLE_VERSION_COUNT, SCHEDULE_TIME, FETCH_RESULT_TIME, WRITE_RESULT_TIME, WAIT_FETCH_RESULT_TIME, DORIS_VERSION, IS_NEREIDS, IS_PIPELINE, IS_CACHED, TOTAL_INSTANCES_NUM, INSTANCES_NUM_PER_BE, PARALLEL_FRAGMENT_EXEC_INSTANCE, TRACE_ID); @@ -115,6 +118,8 @@ public class SummaryProfile { builder.put(GET_PARTITION_VERSION_TIME, 1); builder.put(GET_PARTITION_VERSION_COUNT, 1); builder.put(GET_PARTITION_VERSION_BY_HAS_DATA_COUNT, 1); + builder.put(GET_TABLE_VERSION_TIME, 1); + builder.put(GET_TABLE_VERSION_COUNT, 1); EXECUTION_SUMMARY_KEYS_IDENTATION = builder.build(); } @@ -158,6 +163,8 @@ public class SummaryProfile { private long getPartitionVersionTime = 0; private long getPartitionVersionCount = 0; private long getPartitionVersionByHasDataCount = 0; + private long getTableVersionTime = 0; + private long getTableVersionCount = 0; public SummaryProfile(RuntimeProfile rootProfile) { summaryProfile = new RuntimeProfile(SUMMARY_PROFILE_NAME); @@ -235,6 +242,8 @@ private void updateExecutionSummaryProfile() { executionSummaryProfile.addInfoString(GET_PARTITION_VERSION_COUNT, getPrettyGetPartitionVersionCount()); executionSummaryProfile.addInfoString(GET_PARTITION_VERSION_BY_HAS_DATA_COUNT, getPrettyGetPartitionVersionByHasDataCount()); + executionSummaryProfile.addInfoString(GET_TABLE_VERSION_TIME, getPrettyGetTableVersionTime()); + executionSummaryProfile.addInfoString(GET_TABLE_VERSION_COUNT, getPrettyGetTableVersionCount()); } } @@ -347,6 +356,11 @@ public void addGetPartitionVersionTime(long ns) { this.getPartitionVersionCount += 1; } + public void addGetTableVersionTime(long ns) { + this.getTableVersionTime += ns; + this.getTableVersionCount += 1; + } + public void incGetPartitionVersionByHasDataCount() { this.getPartitionVersionByHasDataCount += 1; } @@ -593,4 +607,15 @@ private String getPrettyGetPartitionVersionByHasDataCount() { private String getPrettyGetPartitionVersionCount() { return RuntimeProfile.printCounter(getPartitionVersionCount, TUnit.UNIT); } + + private String getPrettyGetTableVersionTime() { + if (getTableVersionTime == 0) { + return "N/A"; + } + return RuntimeProfile.printCounter(getTableVersionTime, TUnit.TIME_NS); + } + + private String getPrettyGetTableVersionCount() { + return RuntimeProfile.printCounter(getTableVersionCount, TUnit.UNIT); + } } 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 7f82d0d38768bf..82390b916560a8 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 @@ -24,6 +24,7 @@ import org.apache.doris.catalog.TableIf; import org.apache.doris.catalog.constraint.Constraint; import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.Pair; import org.apache.doris.common.io.Text; import org.apache.doris.common.io.Writable; import org.apache.doris.common.util.Util; @@ -36,7 +37,7 @@ import org.apache.doris.statistics.util.StatisticsUtil; import org.apache.doris.thrift.TTableDescriptor; -import com.google.common.collect.Sets; +import com.google.common.collect.Lists; import com.google.gson.annotations.SerializedName; import lombok.Getter; import org.apache.commons.lang3.NotImplementedException; @@ -46,7 +47,6 @@ import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; -import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Optional; @@ -322,11 +322,12 @@ public boolean needReAnalyzeTable(TableStatsMeta tblStats) { if (tblStats == null) { return true; } - if (!tblStats.analyzeColumns().containsAll(getBaseSchema() + if (!tblStats.analyzeColumns().containsAll(getColumnIndexPairs( + getBaseSchema() .stream() .filter(c -> !StatisticsUtil.isUnsupportedType(c.getType())) .map(Column::getName) - .collect(Collectors.toSet()))) { + .collect(Collectors.toSet())))) { return true; } return System.currentTimeMillis() @@ -334,12 +335,17 @@ public boolean needReAnalyzeTable(TableStatsMeta tblStats) { } @Override - public Map> findReAnalyzeNeededPartitions() { - HashSet partitions = Sets.newHashSet(); - // TODO: Find a way to collect external table partitions that need to be analyzed. - partitions.add("Dummy Partition"); - return getBaseSchema().stream().filter(c -> !StatisticsUtil.isUnsupportedType(c.getType())) - .collect(Collectors.toMap(Column::getName, k -> partitions)); + public List> getColumnIndexPairs(Set columns) { + List> ret = Lists.newArrayList(); + for (String column : columns) { + Column col = getColumn(column); + if (col == null || StatisticsUtil.isUnsupportedType(col.getType())) { + continue; + } + // External table put table name as index name. + ret.add(Pair.of(String.valueOf(name), column)); + } + return ret; } @Override 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 91c919af3f88a3..31c882a05f0e91 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 @@ -1612,7 +1612,7 @@ public void addPartition(Database db, String tableName, AddPartitionClause addPa singlePartitionDesc.getTabletType(), storagePolicy, idGeneratorBuffer, binlogConfig, dataProperty.isStorageMediumSpecified(), null); - afterCreatePartitions(olapTable.getId(), partitionIds, indexIds); + afterCreatePartitions(db.getId(), olapTable.getId(), partitionIds, indexIds, false); // TODO cluster key ids // check again @@ -2033,7 +2033,8 @@ protected void beforeCreatePartitions(long dbId, long tableId, List partit throws DdlException { } - protected void afterCreatePartitions(long tableId, List partitionIds, List indexIds) + protected void afterCreatePartitions(long dbId, long tableId, List partitionIds, List indexIds, + boolean isCreateTable) throws DdlException { } @@ -2669,7 +2670,8 @@ private void createOlapTable(Database db, CreateTableStmt stmt) throws UserExcep binlogConfigForTask, partitionInfo.getDataProperty(partitionId).isStorageMediumSpecified(), keysDesc.getClusterKeysColumnIds()); - afterCreatePartitions(olapTable.getId(), null, olapTable.getIndexIdList()); + afterCreatePartitions(db.getId(), olapTable.getId(), null, + olapTable.getIndexIdList(), true); olapTable.addPartition(partition); } else if (partitionInfo.getType() == PartitionType.RANGE || partitionInfo.getType() == PartitionType.LIST) { @@ -2767,7 +2769,8 @@ private void createOlapTable(Database db, CreateTableStmt stmt) throws UserExcep olapTable.getPartitionInfo().getDataProperty(partition.getId()) .setStoragePolicy(partionStoragePolicy); } - afterCreatePartitions(olapTable.getId(), null, olapTable.getIndexIdList()); + afterCreatePartitions(db.getId(), olapTable.getId(), null, + olapTable.getIndexIdList(), true); } else { throw new DdlException("Unsupported partition method: " + partitionInfo.getType().name()); } @@ -3227,7 +3230,7 @@ public void truncateTable(TruncateTableStmt truncateTableStmt) throws DdlExcepti newPartitions.add(newPartition); } - afterCreatePartitions(copiedTbl.getId(), newPartitionIds, indexIds); + afterCreatePartitions(db.getId(), copiedTbl.getId(), newPartitionIds, indexIds, true); } catch (DdlException e) { // create partition failed, remove all newly created tablets diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergHMSExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergHMSExternalCatalog.java index 2ccf1e03e2825d..221aed4cd8c1d7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergHMSExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergHMSExternalCatalog.java @@ -17,16 +17,10 @@ package org.apache.doris.datasource.iceberg; -import org.apache.doris.common.Config; -import org.apache.doris.common.security.authentication.AuthenticationConfig; -import org.apache.doris.common.security.authentication.HadoopUGI; import org.apache.doris.datasource.CatalogProperty; -import org.apache.doris.datasource.hive.HMSCachedClient; -import org.apache.doris.datasource.hive.HiveMetadataOps; import org.apache.doris.datasource.property.PropertyConverter; import org.apache.doris.datasource.property.constants.HMSProperties; -import org.apache.hadoop.hive.conf.HiveConf; import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.hive.HiveCatalog; @@ -51,20 +45,8 @@ protected void initCatalog() { Map catalogProperties = new HashMap<>(); String metastoreUris = catalogProperty.getOrDefault(HMSProperties.HIVE_METASTORE_URIS, ""); catalogProperties.put(CatalogProperties.URI, metastoreUris); - HiveConf hiveConf = new HiveConf(); - for (Map.Entry kv : catalogProperty.getHadoopProperties().entrySet()) { - hiveConf.set(kv.getKey(), kv.getValue()); - } - hiveConf.set(HiveConf.ConfVars.METASTORE_CLIENT_SOCKET_TIMEOUT.name(), - String.valueOf(Config.hive_metastore_client_timeout_second)); - HadoopUGI.tryKrbLogin(this.getName(), AuthenticationConfig.getKerberosConfig(hiveConf, - AuthenticationConfig.HADOOP_KERBEROS_PRINCIPAL, - AuthenticationConfig.HADOOP_KERBEROS_KEYTAB)); - initS3Param(hiveConf); - HMSCachedClient cachedClient = HiveMetadataOps.createCachedClient(hiveConf, 1, null); - String location = cachedClient.getCatalogLocation("hive"); - catalogProperties.put(CatalogProperties.WAREHOUSE_LOCATION, location); hiveCatalog.initialize(icebergCatalogType, catalogProperties); catalog = hiveCatalog; } } + diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/PropertyConverter.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/PropertyConverter.java index 24457abc748333..b385fb838b3689 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/PropertyConverter.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/PropertyConverter.java @@ -39,9 +39,9 @@ import com.amazonaws.glue.catalog.util.AWSGlueConfig; import com.google.common.base.Strings; import com.google.common.collect.Maps; +import org.apache.hadoop.fs.CosFileSystem; +import org.apache.hadoop.fs.CosNConfigKeys; import org.apache.hadoop.fs.aliyun.oss.AliyunOSSFileSystem; -import org.apache.hadoop.fs.cosn.CosNConfigKeys; -import org.apache.hadoop.fs.cosn.CosNFileSystem; import org.apache.hadoop.fs.obs.OBSConstants; import org.apache.hadoop.fs.obs.OBSFileSystem; import org.apache.hadoop.fs.s3a.Constants; @@ -186,7 +186,7 @@ public static String getHadoopFSImplByScheme(String fsScheme) { } else if (fsScheme.equalsIgnoreCase("oss")) { return AliyunOSSFileSystem.class.getName(); } else if (fsScheme.equalsIgnoreCase("cosn")) { - return CosNFileSystem.class.getName(); + return CosFileSystem.class.getName(); } else { return S3AFileSystem.class.getName(); } @@ -352,8 +352,8 @@ private static Map convertToCOSProperties(Map pr cosProperties.put("fs.cosn.impl.disable.cache", "true"); cosProperties.put("fs.cosn.impl", getHadoopFSImplByScheme("cosn")); if (credential.isWhole()) { - cosProperties.put(CosNConfigKeys.COSN_SECRET_ID_KEY, credential.getAccessKey()); - cosProperties.put(CosNConfigKeys.COSN_SECRET_KEY_KEY, credential.getSecretKey()); + cosProperties.put(CosNConfigKeys.COSN_USERINFO_SECRET_ID_KEY, credential.getAccessKey()); + cosProperties.put(CosNConfigKeys.COSN_USERINFO_SECRET_KEY_KEY, credential.getSecretKey()); } // session token is unsupported for (Map.Entry entry : props.entrySet()) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/insertoverwrite/InsertOverwriteManager.java b/fe/fe-core/src/main/java/org/apache/doris/insertoverwrite/InsertOverwriteManager.java index 32d52d7e7af100..e0c46dde920b6e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/insertoverwrite/InsertOverwriteManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/insertoverwrite/InsertOverwriteManager.java @@ -56,8 +56,10 @@ public class InsertOverwriteManager extends MasterDaemon implements Writable { private Map> taskGroups = Maps.newConcurrentMap(); // for one task group, there may be different requests about changing a partition to new. // but we only change one time and save the relations in partitionPairs. they're protected by taskLocks + @SerializedName(value = "taskLocks") private Map taskLocks = Maps.newConcurrentMap(); // > + @SerializedName(value = "partitionPairs") private Map> partitionPairs = Maps.newConcurrentMap(); public InsertOverwriteManager() { 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 e3333f325858e3..d23c7534f8664e 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 @@ -95,9 +95,9 @@ private static Set getBaseTables(Plan plan) { TableCollectorContext collectorContext = new TableCollector.TableCollectorContext( com.google.common.collect.Sets - .newHashSet(TableType.values())); + .newHashSet(TableType.values()), true); plan.accept(TableCollector.INSTANCE, collectorContext); - List collectedTables = collectorContext.getCollectedTables(); + Set collectedTables = collectorContext.getCollectedTables(); return transferTableIfToInfo(collectedTables); } @@ -105,7 +105,7 @@ private static Set getBaseViews(Plan plan) { return Sets.newHashSet(); } - private static Set transferTableIfToInfo(List tables) { + private static Set transferTableIfToInfo(Set tables) { Set result = com.google.common.collect.Sets.newHashSet(); for (TableIf table : tables) { result.add(new BaseTableInfo(table)); diff --git a/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/ResourcePrivEntry.java b/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/ResourcePrivEntry.java index a41f3803088cf7..65546e49ebc5d1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/ResourcePrivEntry.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/ResourcePrivEntry.java @@ -27,11 +27,8 @@ import java.io.IOException; public class ResourcePrivEntry extends PrivEntry { - protected static final String ANY_RESOURCE = "*"; - protected PatternMatcher resourcePattern; protected String origResource; - protected boolean isAnyResource; protected ResourcePrivEntry() { } @@ -41,15 +38,12 @@ protected ResourcePrivEntry(PatternMatcher resourcePattern, super(privSet); this.resourcePattern = resourcePattern; this.origResource = origResource; - if (origResource.equals(ANY_RESOURCE)) { - isAnyResource = true; - } } public static ResourcePrivEntry create(String resourceName, PrivBitSet privs) throws AnalysisException, PatternMatcherException { PatternMatcher resourcePattern = PatternMatcher.createMysqlPattern( - resourceName.equals(ANY_RESOURCE) ? "%" : resourceName, + resourceName, CaseSensibility.RESOURCE.getCaseSensibility()); if (privs.containsNodePriv() || privs.containsDbTablePriv()) { throw new AnalysisException("Resource privilege can not contains node or db table privileges: " + privs); @@ -112,6 +106,5 @@ public void readFields(DataInput in) throws IOException { } catch (PatternMatcherException e) { throw new IOException(e); } - isAnyResource = origResource.equals(ANY_RESOURCE); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/ResourcePrivTable.java b/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/ResourcePrivTable.java index 010d1026c95734..93f8bb20eb9c24 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/ResourcePrivTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/ResourcePrivTable.java @@ -26,26 +26,15 @@ public class ResourcePrivTable extends PrivTable { private static final Logger LOG = LogManager.getLogger(ResourcePrivTable.class); - /* - * Return first priv which match the user@host on resourceName The returned priv will be - * saved in 'savedPrivs'. - */ public void getPrivs(String resourceName, PrivBitSet savedPrivs) { - ResourcePrivEntry matchedEntry = null; + // need check all entries, because may have 2 entries match resourceName, + // For example, if the resourceName is g1, there are two entry `%` and `g1` compound requirements for (PrivEntry entry : entries) { ResourcePrivEntry resourcePrivEntry = (ResourcePrivEntry) entry; // check resource - if (!resourcePrivEntry.getResourcePattern().match(resourceName)) { - continue; + if (resourcePrivEntry.getResourcePattern().match(resourceName)) { + savedPrivs.or(resourcePrivEntry.getPrivSet()); } - - matchedEntry = resourcePrivEntry; - break; - } - if (matchedEntry == null) { - return; } - - savedPrivs.or(matchedEntry.getPrivSet()); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/Role.java b/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/Role.java index b22849ea75ef39..e6090f87623011 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/Role.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/Role.java @@ -471,8 +471,8 @@ public boolean checkWorkloadGroupPriv(String workloadGroupName, PrivPredicate wa return true; } PrivBitSet savedPrivs = PrivBitSet.of(); - // Workload groups do not support global usage_priv, so only global admin_priv and usage_priv are checked. - if (checkGlobalInternal(PrivPredicate.ADMIN, savedPrivs) + // usage priv not in global, but grant_priv may in global + if (checkGlobalInternal(wanted, savedPrivs) || checkWorkloadGroupInternal(workloadGroupName, wanted, savedPrivs)) { return true; } @@ -573,22 +573,11 @@ private void grantPrivs(ResourcePattern resourcePattern, PrivBitSet privs) throw if (privs.isEmpty()) { return; } - // grant privs to user - switch (resourcePattern.getPrivLevel()) { - case GLOBAL: - grantGlobalPrivs(privs); - break; - case RESOURCE: - if (resourcePattern.isClusterResource()) { - grantCloudClusterPrivs(resourcePattern.getResourceName(), false, false, privs); - } else { - grantResourcePrivs(resourcePattern.getResourceName(), privs); - } - break; - default: - Preconditions.checkNotNull(null, resourcePattern.getPrivLevel()); + if (resourcePattern.isClusterResource()) { + grantCloudClusterPrivs(resourcePattern.getResourceName(), false, false, privs); + } else { + grantResourcePrivs(resourcePattern.getResourceName(), privs); } - } private void grantPrivs(WorkloadGroupPattern workloadGroupPattern, PrivBitSet privs) throws DdlException { diff --git a/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/WorkloadGroupPrivTable.java b/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/WorkloadGroupPrivTable.java index 994118c743102e..7b35b41f01f743 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/WorkloadGroupPrivTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/WorkloadGroupPrivTable.java @@ -20,20 +20,13 @@ public class WorkloadGroupPrivTable extends PrivTable { public void getPrivs(String workloadGroupName, PrivBitSet savedPrivs) { - WorkloadGroupPrivEntry matchedEntry = null; + // need check all entries, because may have 2 entries match workloadGroupName, + // For example, if the workloadGroupName is g1, there are two entry `%` and `g1` compound requirements for (PrivEntry entry : entries) { WorkloadGroupPrivEntry workloadGroupPrivEntry = (WorkloadGroupPrivEntry) entry; - if (!workloadGroupPrivEntry.getWorkloadGroupPattern().match(workloadGroupName)) { - continue; + if (workloadGroupPrivEntry.getWorkloadGroupPattern().match(workloadGroupName)) { + savedPrivs.or(workloadGroupPrivEntry.getPrivSet()); } - - matchedEntry = workloadGroupPrivEntry; - break; - } - if (matchedEntry == null) { - return; } - - savedPrivs.or(matchedEntry.getPrivSet()); } } 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 2985664f8b818a..a9ba3603f53442 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 @@ -2625,7 +2625,7 @@ public IndexDefinition visitIndexDef(IndexDefContext ctx) { String indexType = ctx.indexType != null ? ctx.indexType.getText().toUpperCase() : null; String comment = ctx.comment != null ? ctx.comment.getText() : ""; // change BITMAP index to INVERTED index - if (indexType.equalsIgnoreCase("BITMAP")) { + if ("BITMAP".equalsIgnoreCase(indexType)) { indexType = "INVERTED"; } return new IndexDefinition(indexName, indexCols, indexType, properties, comment); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/ColumnPruningPostProcessor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/ColumnPruningPostProcessor.java new file mode 100644 index 00000000000000..ff362b7bc9e010 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/ColumnPruningPostProcessor.java @@ -0,0 +1,102 @@ +// 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.processor.post; + +import org.apache.doris.nereids.CascadesContext; +import org.apache.doris.nereids.annotation.DependsRules; +import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.physical.AbstractPhysicalJoin; +import org.apache.doris.nereids.trees.plans.physical.AbstractPhysicalPlan; +import org.apache.doris.nereids.trees.plans.physical.PhysicalDistribute; +import org.apache.doris.nereids.trees.plans.physical.PhysicalProject; + +import java.util.ArrayList; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Prune column for Join-Cluster + */ +@DependsRules({ + MergeProjectPostProcessor.class +}) +public class ColumnPruningPostProcessor extends PlanPostProcessor { + @Override + public PhysicalProject visitPhysicalProject(PhysicalProject project, CascadesContext ctx) { + Plan child = project.child(); + Plan newChild = child.accept(this, ctx); + if (newChild instanceof AbstractPhysicalJoin) { + AbstractPhysicalJoin join = (AbstractPhysicalJoin) newChild; + Plan left = join.left(); + Plan right = join.right(); + Set leftOutput = left.getOutputSet(); + Set rightOutput = right.getOutputSet(); + + Set usedSlots = project.getProjects().stream().flatMap(ne -> ne.getInputSlots().stream()) + .collect(Collectors.toSet()); + usedSlots.addAll(join.getConditionSlot()); + + List leftNewProjections = new ArrayList<>(); + List rightNewProjections = new ArrayList<>(); + + for (Slot usedSlot : usedSlots) { + if (leftOutput.contains(usedSlot)) { + leftNewProjections.add(usedSlot); + } else if (rightOutput.contains(usedSlot)) { + rightNewProjections.add(usedSlot); + } + } + + Plan newLeft; + if (left instanceof PhysicalDistribute) { + newLeft = leftNewProjections.size() != leftOutput.size() && !leftNewProjections.isEmpty() + ? left.withChildren(new PhysicalProject<>(leftNewProjections, + left.getLogicalProperties(), left.child(0))) + : left; + } else { + newLeft = leftNewProjections.size() != leftOutput.size() && !leftNewProjections.isEmpty() + ? new PhysicalProject<>(leftNewProjections, left.getLogicalProperties(), + left).copyStatsAndGroupIdFrom((AbstractPhysicalPlan) left) + : left; + } + Plan newRight; + if (right instanceof PhysicalDistribute) { + newRight = rightNewProjections.size() != rightOutput.size() && !rightNewProjections.isEmpty() + ? right.withChildren(new PhysicalProject<>(rightNewProjections, + right.getLogicalProperties(), right.child(0))) + : right; + } else { + newRight = rightNewProjections.size() != rightOutput.size() && !rightNewProjections.isEmpty() + ? new PhysicalProject<>(rightNewProjections, right.getLogicalProperties(), + right).copyStatsAndGroupIdFrom((AbstractPhysicalPlan) right) + : right; + } + + if (newLeft != left || newRight != right) { + return (PhysicalProject) project.withChildren(join.withChildren(newLeft, newRight)); + } else { + return project; + } + } + return project; + } +} + diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/PlanPostProcessors.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/PlanPostProcessors.java index d89ef1acd2602e..60c1a74445e1ff 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/PlanPostProcessors.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/PlanPostProcessors.java @@ -59,6 +59,7 @@ public List getProcessors() { // add processor if we need Builder builder = ImmutableList.builder(); builder.add(new PushDownFilterThroughProject()); + builder.add(new ColumnPruningPostProcessor()); builder.add(new MergeProjectPostProcessor()); builder.add(new RecomputeLogicalPropertiesProcessor()); builder.add(new AddOffsetIntoDistribute()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/EliminateGroupByConstant.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/EliminateGroupByConstant.java index e683153e9a2966..4408e64487cfb2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/EliminateGroupByConstant.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/EliminateGroupByConstant.java @@ -26,11 +26,14 @@ import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; +import org.apache.doris.nereids.util.ExpressionUtils; import com.google.common.collect.ImmutableList; import com.google.common.collect.Sets; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.Set; /** @@ -54,6 +57,7 @@ public Rule build() { List groupByExprs = aggregate.getGroupByExpressions(); List outputExprs = aggregate.getOutputExpressions(); Set slotGroupByExprs = Sets.newLinkedHashSet(); + Map constantExprsReplaceMap = new HashMap<>(groupByExprs.size()); Expression lit = null; for (Expression expression : groupByExprs) { // NOTICE: we should not use the expression after fold as new aggregate's output or group expr @@ -64,13 +68,15 @@ public Rule build() { if (!foldExpression.isConstant()) { slotGroupByExprs.add(expression); } else { + constantExprsReplaceMap.put(expression, foldExpression); lit = expression; } } if (slotGroupByExprs.isEmpty() && lit != null) { slotGroupByExprs.add(lit); } - return aggregate.withGroupByAndOutput(ImmutableList.copyOf(slotGroupByExprs), outputExprs); + return aggregate.withGroupByAndOutput(ImmutableList.copyOf(slotGroupByExprs), + ExpressionUtils.replaceNamedExpressions(outputExprs, constantExprsReplaceMap)); }).toRule(RuleType.ELIMINATE_GROUP_BY_CONSTANT); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewRule.java index 32e90a67efa586..66852444cc0e2f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewRule.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewRule.java @@ -58,7 +58,6 @@ import java.util.ArrayList; import java.util.Collection; -import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Objects; @@ -69,8 +68,15 @@ * The abstract class for all materialized view rules */ public abstract class AbstractMaterializedViewRule implements ExplorationRuleFactory { - public static final HashSet SUPPORTED_JOIN_TYPE_SET = Sets.newHashSet(JoinType.INNER_JOIN, - JoinType.LEFT_OUTER_JOIN); + public static final Set SUPPORTED_JOIN_TYPE_SET = ImmutableSet.of( + JoinType.INNER_JOIN, + JoinType.LEFT_OUTER_JOIN, + JoinType.RIGHT_OUTER_JOIN, + JoinType.FULL_OUTER_JOIN, + JoinType.LEFT_SEMI_JOIN, + JoinType.RIGHT_SEMI_JOIN, + JoinType.LEFT_ANTI_JOIN, + JoinType.RIGHT_ANTI_JOIN); /** * The abstract template method for query rewrite, it contains the main logic, try to rewrite query by diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/ComparisonResult.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/ComparisonResult.java index c60eb9f94d3d0f..2ffdeb5718bff5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/ComparisonResult.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/ComparisonResult.java @@ -17,6 +17,7 @@ package org.apache.doris.nereids.rules.exploration.mv; +import org.apache.doris.common.Pair; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.Slot; @@ -27,6 +28,7 @@ import java.util.Collection; import java.util.List; import java.util.Set; +import java.util.stream.Collectors; /** * comparison result of view and query @@ -111,8 +113,14 @@ public Builder addViewExpressions(Collection expressions) return this; } - public Builder addViewNoNullableSlot(Set viewNoNullableSlot) { - viewNoNullableSlotBuilder.add(ImmutableSet.copyOf(viewNoNullableSlot)); + /**Add slots which should reject null slots in view*/ + public Builder addViewNoNullableSlot(Pair, Set> viewNoNullableSlotsPair) { + if (!viewNoNullableSlotsPair.first.isEmpty()) { + viewNoNullableSlotBuilder.add(viewNoNullableSlotsPair.first); + } + if (!viewNoNullableSlotsPair.second.isEmpty()) { + viewNoNullableSlotBuilder.add(viewNoNullableSlotsPair.second); + } return this; } @@ -127,7 +135,9 @@ public boolean isInvalid() { public ComparisonResult build() { Preconditions.checkArgument(valid, "Comparison result must be valid"); - return new ComparisonResult(queryBuilder.build(), queryAllPulledUpExpressionsBuilder.build(), + return new ComparisonResult(queryBuilder.build(), + queryAllPulledUpExpressionsBuilder.build().stream() + .filter(expr -> !expr.isInferred()).collect(Collectors.toList()), viewBuilder.build(), viewNoNullableSlotBuilder.build(), valid, ""); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/HyperGraphComparator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/HyperGraphComparator.java index 341caa88094118..956a974982ccec 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/HyperGraphComparator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/HyperGraphComparator.java @@ -73,7 +73,11 @@ public class HyperGraphComparator { private final Map> pullUpQueryExprWithEdge = new HashMap<>(); private final Map> pullUpViewExprWithEdge = new HashMap<>(); private final LogicalCompatibilityContext logicalCompatibilityContext; - private final Map>> inferredViewEdgeWithCond = new HashMap<>(); + // this records the slots which needs to reject null + // the key is the target join which should reject null, the value is a pair, the first value of the pair is the + // join type, the second value is also a pair which left represents the slots in the left of join that should + // reject null, right represents the slots in the right of join that should reject null. + private final Map, Set>>> inferredViewEdgeWithCond = new HashMap<>(); private List viewJoinEdgesAfterInferring; private List viewFilterEdgesAfterInferring; private final long eliminateViewNodesMap; @@ -263,7 +267,7 @@ private ComparisonResult buildComparisonRes() { } builder.addViewExpressions(rawFilter); } - for (Pair> inferredCond : inferredViewEdgeWithCond.values()) { + for (Pair, Set>> inferredCond : inferredViewEdgeWithCond.values()) { builder.addViewNoNullableSlot(inferredCond.second); } builder.addQueryAllPulledUpExpressions( @@ -485,11 +489,14 @@ private boolean tryInferEdge(JoinEdge query, JoinEdge view) { if (noNullableChild == null) { return false; } - Set noNullableSlot = Sets.union( - noNullableChild.first ? view.getJoin().left().getOutputSet() : ImmutableSet.of(), - noNullableChild.second ? view.getJoin().right().getOutputSet() : ImmutableSet.of() - ); - inferredViewEdgeWithCond.put(view, Pair.of(query.getJoinType(), noNullableSlot)); + Pair, Set> noNullableSlotSetPair = Pair.of(ImmutableSet.of(), ImmutableSet.of()); + if (noNullableChild.first) { + noNullableSlotSetPair.first = view.getJoin().left().getOutputSet(); + } + if (noNullableChild.second) { + noNullableSlotSetPair.second = view.getJoin().right().getOutputSet(); + } + inferredViewEdgeWithCond.put(view, Pair.of(query.getJoinType(), noNullableSlotSetPair)); } return true; } 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 8046639d1bac9d..914b213361348a 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 @@ -66,9 +66,9 @@ public void initMaterializationContext(CascadesContext cascadesContext) { return; } Plan rewritePlan = cascadesContext.getRewritePlan(); - TableCollectorContext collectorContext = new TableCollectorContext(Sets.newHashSet()); + TableCollectorContext collectorContext = new TableCollectorContext(Sets.newHashSet(), true); rewritePlan.accept(TableCollector.INSTANCE, collectorContext); - List collectedTables = collectorContext.getCollectedTables(); + Set collectedTables = collectorContext.getCollectedTables(); if (collectedTables.isEmpty()) { return; } 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 new file mode 100644 index 00000000000000..89999258713e50 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/MultiDistinctSum0.java @@ -0,0 +1,82 @@ +// 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.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.shape.UnaryExpression; +import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; +import org.apache.doris.nereids.types.BigIntType; +import org.apache.doris.nereids.types.DoubleType; +import org.apache.doris.nereids.types.LargeIntType; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.List; + +/** MultiDistinctSum0 */ +public class MultiDistinctSum0 extends AggregateFunction implements UnaryExpression, + ExplicitlyCastableSignature, ComputePrecisionForSum, MultiDistinction, AlwaysNotNullable { + + public static final List SIGNATURES = ImmutableList.of( + FunctionSignature.ret(BigIntType.INSTANCE).varArgs(BigIntType.INSTANCE), + FunctionSignature.ret(BigIntType.INSTANCE).varArgs(DoubleType.INSTANCE), + FunctionSignature.ret(BigIntType.INSTANCE).varArgs(LargeIntType.INSTANCE) + ); + + public MultiDistinctSum0(Expression arg0) { + super("multi_distinct_sum0", true, arg0); + } + + public MultiDistinctSum0(boolean distinct, Expression arg0) { + super("multi_distinct_sum0", true, arg0); + } + + @Override + public void checkLegalityBeforeTypeCoercion() { + if (child().getDataType().isDateLikeType()) { + throw new AnalysisException("Sum0 in multi distinct functions do not support Date/Datetime type"); + } + } + + @Override + public List getSignatures() { + return new Sum0(getArgument(0)).getSignatures(); + } + + @Override + public FunctionSignature searchSignature(List signatures) { + return new Sum0(getArgument(0)).searchSignature(signatures); + } + + @Override + public MultiDistinctSum0 withDistinctAndChildren(boolean distinct, List children) { + Preconditions.checkArgument(children.size() == 1); + return new MultiDistinctSum0(distinct, children.get(0)); + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitMultiDistinctSum0(this, context); + } +} 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 new file mode 100644 index 00000000000000..b25dd5fe60e786 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Sum0.java @@ -0,0 +1,125 @@ +// 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.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; +import org.apache.doris.nereids.trees.expressions.functions.window.SupportWindowAnalytic; +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; +import org.apache.doris.nereids.types.BooleanType; +import org.apache.doris.nereids.types.DataType; +import org.apache.doris.nereids.types.DecimalV3Type; +import org.apache.doris.nereids.types.DoubleType; +import org.apache.doris.nereids.types.FloatType; +import org.apache.doris.nereids.types.IntegerType; +import org.apache.doris.nereids.types.LargeIntType; +import org.apache.doris.nereids.types.SmallIntType; +import org.apache.doris.nereids.types.TinyIntType; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.List; + +/** + * 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, + SupportWindowAnalytic, CouldRollUp { + + public static final List SIGNATURES = ImmutableList.of( + FunctionSignature.ret(BigIntType.INSTANCE).args(BooleanType.INSTANCE), + FunctionSignature.ret(BigIntType.INSTANCE).args(TinyIntType.INSTANCE), + FunctionSignature.ret(BigIntType.INSTANCE).args(SmallIntType.INSTANCE), + FunctionSignature.ret(BigIntType.INSTANCE).args(IntegerType.INSTANCE), + FunctionSignature.ret(BigIntType.INSTANCE).args(BigIntType.INSTANCE), + FunctionSignature.ret(LargeIntType.INSTANCE).args(LargeIntType.INSTANCE), + FunctionSignature.ret(DecimalV3Type.WILDCARD).args(DecimalV3Type.WILDCARD), + FunctionSignature.ret(DoubleType.INSTANCE).args(DoubleType.INSTANCE) + ); + + /** + * constructor with 1 argument. + */ + public Sum0(Expression arg) { + this(false, arg); + } + + /** + * constructor with 2 argument. + */ + public Sum0(boolean distinct, Expression arg) { + super("sum0", distinct, arg); + } + + public MultiDistinctSum0 convertToMultiDistinct() { + Preconditions.checkArgument(distinct, + "can't convert to multi_distinct_sum because there is no distinct args"); + return new MultiDistinctSum0(false, child()); + } + + @Override + public void checkLegalityBeforeTypeCoercion() { + DataType argType = child().getDataType(); + if ((!argType.isNumericType() && !argType.isBooleanType() && !argType.isNullType()) + || argType.isOnlyMetricType()) { + throw new AnalysisException("sum0 requires a numeric or boolean parameter: " + this.toSql()); + } + } + + /** + * withDistinctAndChildren. + */ + @Override + public Sum0 withDistinctAndChildren(boolean distinct, List children) { + Preconditions.checkArgument(children.size() == 1); + return new Sum0(distinct, children.get(0)); + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitSum0(this, context); + } + + @Override + public List getSignatures() { + return SIGNATURES; + } + + @Override + public FunctionSignature searchSignature(List signatures) { + if (getArgument(0).getDataType() instanceof FloatType) { + return FunctionSignature.ret(DoubleType.INSTANCE).args(FloatType.INSTANCE); + } + return ExplicitlyCastableSignature.super.searchSignature(signatures); + } + + @Override + public Function constructRollUp(Expression param, Expression... varParams) { + return new Sum0(this.distinct, param); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/AggregateFunctionVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/AggregateFunctionVisitor.java index 594f9c754335aa..febe93974f264a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/AggregateFunctionVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/AggregateFunctionVisitor.java @@ -51,6 +51,7 @@ import org.apache.doris.nereids.trees.expressions.functions.agg.MultiDistinctCount; import org.apache.doris.nereids.trees.expressions.functions.agg.MultiDistinctGroupConcat; import org.apache.doris.nereids.trees.expressions.functions.agg.MultiDistinctSum; +import org.apache.doris.nereids.trees.expressions.functions.agg.MultiDistinctSum0; import org.apache.doris.nereids.trees.expressions.functions.agg.Ndv; import org.apache.doris.nereids.trees.expressions.functions.agg.NullableAggregateFunction; import org.apache.doris.nereids.trees.expressions.functions.agg.OrthogonalBitmapIntersect; @@ -66,6 +67,7 @@ import org.apache.doris.nereids.trees.expressions.functions.agg.Stddev; import org.apache.doris.nereids.trees.expressions.functions.agg.StddevSamp; import org.apache.doris.nereids.trees.expressions.functions.agg.Sum; +import org.apache.doris.nereids.trees.expressions.functions.agg.Sum0; import org.apache.doris.nereids.trees.expressions.functions.agg.TopN; import org.apache.doris.nereids.trees.expressions.functions.agg.TopNArray; import org.apache.doris.nereids.trees.expressions.functions.agg.TopNWeighted; @@ -162,6 +164,10 @@ default R visitMultiDistinctSum(MultiDistinctSum multiDistinctSum, C context) { return visitAggregateFunction(multiDistinctSum, context); } + default R visitMultiDistinctSum0(MultiDistinctSum0 multiDistinctSum0, C context) { + return visitAggregateFunction(multiDistinctSum0, context); + } + default R visitGroupBitAnd(GroupBitAnd groupBitAnd, C context) { return visitNullableAggregateFunction(groupBitAnd, context); } @@ -274,6 +280,10 @@ default R visitSum(Sum sum, C context) { return visitNullableAggregateFunction(sum, context); } + default R visitSum0(Sum0 sum0, C context) { + return visitAggregateFunction(sum0, context); + } + default R visitTopN(TopN topN, C context) { return visitAggregateFunction(topN, context); } 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 d7c502a9eb84c6..7e7f58db44db22 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 @@ -320,9 +320,9 @@ private PartitionDesc generatePartitionDesc(MTMVRelatedTableIf relatedTable, Con private void analyzeBaseTables(Plan plan) { TableCollectorContext collectorContext = - new TableCollector.TableCollectorContext(Sets.newHashSet(TableType.MATERIALIZED_VIEW)); + new TableCollector.TableCollectorContext(Sets.newHashSet(TableType.MATERIALIZED_VIEW), true); plan.accept(TableCollector.INSTANCE, collectorContext); - List collectedTables = collectorContext.getCollectedTables(); + Set collectedTables = collectorContext.getCollectedTables(); if (!CollectionUtils.isEmpty(collectedTables)) { throw new AnalysisException("can not contain MATERIALIZED_VIEW"); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/IndexDefinition.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/IndexDefinition.java index ba56cfa4a8bba9..09c670e1b8e934 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/IndexDefinition.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/IndexDefinition.java @@ -105,8 +105,8 @@ public void checkColumn(ColumnDefinition column, KeysType keysType, } if (!(colType.isDateLikeType() || colType.isDecimalLikeType() || colType.isIntegralType() || colType.isStringLikeType() - || colType.isBooleanType())) { - // TODO add colType.isVariantType() and colType.isAggState() + || colType.isBooleanType() || colType.isVariantType())) { + // TODO add colType.isAggState() throw new AnalysisException(colType + " is not supported in " + indexType.toString() + " index. " + "invalid index: " + name); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalTestScan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalTestScan.java index 7d326ade3d603b..94cae4b955d798 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalTestScan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalTestScan.java @@ -23,6 +23,7 @@ import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.PlanType; import org.apache.doris.nereids.trees.plans.RelationId; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; import org.apache.doris.nereids.util.Utils; import java.util.List; @@ -68,4 +69,9 @@ public Plan withGroupExprLogicalPropChildren(Optional groupExpr Optional logicalProperties, List children) { return new LogicalTestScan(relationId, table, qualifier, groupExpression, logicalProperties); } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitLogicalTestScan(this, context); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/AbstractPhysicalPlan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/AbstractPhysicalPlan.java index 1e9135d600e4e3..3ebf9390d84e81 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/AbstractPhysicalPlan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/AbstractPhysicalPlan.java @@ -149,7 +149,7 @@ public Plan getExplainPlan(ConnectContext ctx) { return this; } - public T copyStatsAndGroupIdFrom(T from) { + public AbstractPhysicalPlan copyStatsAndGroupIdFrom(T from) { T newPlan = (T) withPhysicalPropertiesAndStats( from.getPhysicalProperties(), from.getStats()); newPlan.setMutableState(MutableState.KEY_GROUP, from.getGroupIdAsString()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/RelationVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/RelationVisitor.java index adfe4991665216..5eaf0b7213648d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/RelationVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/RelationVisitor.java @@ -20,6 +20,7 @@ import org.apache.doris.nereids.analyzer.UnboundOneRowRelation; import org.apache.doris.nereids.analyzer.UnboundRelation; import org.apache.doris.nereids.analyzer.UnboundTVFRelation; +import org.apache.doris.nereids.trees.plans.logical.LogicalCatalogRelation; import org.apache.doris.nereids.trees.plans.logical.LogicalDeferMaterializeOlapScan; import org.apache.doris.nereids.trees.plans.logical.LogicalEmptyRelation; import org.apache.doris.nereids.trees.plans.logical.LogicalEsScan; @@ -31,6 +32,8 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalRelation; import org.apache.doris.nereids.trees.plans.logical.LogicalSchemaScan; import org.apache.doris.nereids.trees.plans.logical.LogicalTVFRelation; +import org.apache.doris.nereids.trees.plans.logical.LogicalTestScan; +import org.apache.doris.nereids.trees.plans.physical.PhysicalCatalogRelation; import org.apache.doris.nereids.trees.plans.physical.PhysicalDeferMaterializeOlapScan; import org.apache.doris.nereids.trees.plans.physical.PhysicalEmptyRelation; import org.apache.doris.nereids.trees.plans.physical.PhysicalEsScan; @@ -54,8 +57,16 @@ public interface RelationVisitor { R visitLogicalRelation(LogicalRelation logicalRelation, C context); + default R visitLogicalCatalogRelation(LogicalCatalogRelation catalogRelation, C context) { + return visitLogicalRelation(catalogRelation, context); + } + R visitPhysicalRelation(PhysicalRelation physicalRelation, C context); + default R visitPhysicalCatalogRelation(PhysicalCatalogRelation catalogRelation, C context) { + return visitPhysicalRelation(catalogRelation, context); + } + // ******************************* // unbound relations // ******************************* @@ -81,28 +92,28 @@ default R visitLogicalEmptyRelation(LogicalEmptyRelation emptyRelation, C contex } default R visitLogicalEsScan(LogicalEsScan esScan, C context) { - return visitLogicalRelation(esScan, context); + return visitLogicalCatalogRelation(esScan, context); } default R visitLogicalFileScan(LogicalFileScan fileScan, C context) { - return visitLogicalRelation(fileScan, context); + return visitLogicalCatalogRelation(fileScan, context); } default R visitLogicalJdbcScan(LogicalJdbcScan jdbcScan, C context) { - return visitLogicalRelation(jdbcScan, context); + return visitLogicalCatalogRelation(jdbcScan, context); } default R visitLogicalOdbcScan(LogicalOdbcScan odbcScan, C context) { - return visitLogicalRelation(odbcScan, context); + return visitLogicalCatalogRelation(odbcScan, context); } default R visitLogicalOlapScan(LogicalOlapScan olapScan, C context) { - return visitLogicalRelation(olapScan, context); + return visitLogicalCatalogRelation(olapScan, context); } default R visitLogicalDeferMaterializeOlapScan( LogicalDeferMaterializeOlapScan deferMaterializeOlapScan, C context) { - return visitLogicalRelation(deferMaterializeOlapScan, context); + return visitLogicalCatalogRelation(deferMaterializeOlapScan, context); } default R visitLogicalOneRowRelation(LogicalOneRowRelation oneRowRelation, C context) { @@ -110,13 +121,17 @@ default R visitLogicalOneRowRelation(LogicalOneRowRelation oneRowRelation, C con } default R visitLogicalSchemaScan(LogicalSchemaScan schemaScan, C context) { - return visitLogicalRelation(schemaScan, context); + return visitLogicalCatalogRelation(schemaScan, context); } default R visitLogicalTVFRelation(LogicalTVFRelation tvfRelation, C context) { return visitLogicalRelation(tvfRelation, context); } + default R visitLogicalTestScan(LogicalTestScan testScan, C context) { + return visitLogicalCatalogRelation(testScan, context); + } + // ******************************* // physical relations // ******************************* @@ -126,28 +141,28 @@ default R visitPhysicalEmptyRelation(PhysicalEmptyRelation emptyRelation, C cont } default R visitPhysicalEsScan(PhysicalEsScan esScan, C context) { - return visitPhysicalRelation(esScan, context); + return visitPhysicalCatalogRelation(esScan, context); } default R visitPhysicalFileScan(PhysicalFileScan fileScan, C context) { - return visitPhysicalRelation(fileScan, context); + return visitPhysicalCatalogRelation(fileScan, context); } default R visitPhysicalJdbcScan(PhysicalJdbcScan jdbcScan, C context) { - return visitPhysicalRelation(jdbcScan, context); + return visitPhysicalCatalogRelation(jdbcScan, context); } default R visitPhysicalOdbcScan(PhysicalOdbcScan odbcScan, C context) { - return visitPhysicalRelation(odbcScan, context); + return visitPhysicalCatalogRelation(odbcScan, context); } default R visitPhysicalOlapScan(PhysicalOlapScan olapScan, C context) { - return visitPhysicalRelation(olapScan, context); + return visitPhysicalCatalogRelation(olapScan, context); } default R visitPhysicalDeferMaterializeOlapScan( PhysicalDeferMaterializeOlapScan deferMaterializeOlapScan, C context) { - return visitPhysicalRelation(deferMaterializeOlapScan, context); + return visitPhysicalCatalogRelation(deferMaterializeOlapScan, context); } default R visitPhysicalOneRowRelation(PhysicalOneRowRelation oneRowRelation, C context) { @@ -155,7 +170,7 @@ default R visitPhysicalOneRowRelation(PhysicalOneRowRelation oneRowRelation, C c } default R visitPhysicalSchemaScan(PhysicalSchemaScan schemaScan, C context) { - return visitPhysicalRelation(schemaScan, context); + return visitPhysicalCatalogRelation(schemaScan, context); } default R visitPhysicalTVFRelation(PhysicalTVFRelation tvfRelation, C context) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/TableCollector.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/TableCollector.java index a3c874f6370444..3736ab60b0f6e8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/TableCollector.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/TableCollector.java @@ -17,33 +17,73 @@ package org.apache.doris.nereids.trees.plans.visitor; +import org.apache.doris.catalog.MTMV; import org.apache.doris.catalog.TableIf; import org.apache.doris.catalog.TableIf.TableType; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.mtmv.MTMVCache; +import org.apache.doris.mtmv.MTMVPlanUtil; import org.apache.doris.nereids.trees.plans.Plan; -import org.apache.doris.nereids.trees.plans.algebra.CatalogRelation; +import org.apache.doris.nereids.trees.plans.logical.LogicalCatalogRelation; +import org.apache.doris.nereids.trees.plans.physical.PhysicalCatalogRelation; import org.apache.doris.nereids.trees.plans.visitor.TableCollector.TableCollectorContext; -import java.util.ArrayList; -import java.util.List; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.HashSet; import java.util.Set; /** * Collect the table in plan * Note: will not get table if table is eliminated by EmptyRelation in rewrite. + * View expand is in RBO, if call this method with the plan after RBO, this will get base tables in view, or will not. + * Materialized view is extended or not can be controlled by the field expand */ -public class TableCollector extends DefaultPlanVisitor { +public class TableCollector extends DefaultPlanVisitor { public static final TableCollector INSTANCE = new TableCollector(); + private static final Logger LOG = LogManager.getLogger(TableCollector.class); @Override - public Void visit(Plan plan, TableCollectorContext context) { - if (plan instanceof CatalogRelation) { - TableIf table = ((CatalogRelation) plan).getTable(); - if (context.getTargetTableTypes().isEmpty() || context.getTargetTableTypes().contains(table.getType())) { - context.getCollectedTables().add(table); - } + public Plan visitLogicalCatalogRelation(LogicalCatalogRelation catalogRelation, TableCollectorContext context) { + TableIf table = catalogRelation.getTable(); + if (context.getTargetTableTypes().isEmpty() || context.getTargetTableTypes().contains(table.getType())) { + context.getCollectedTables().add(table); + } + if (table instanceof MTMV) { + expandMvAndCollect((MTMV) table, context); + } + return catalogRelation; + } + + @Override + public Plan visitPhysicalCatalogRelation(PhysicalCatalogRelation catalogRelation, TableCollectorContext context) { + TableIf table = catalogRelation.getTable(); + if (context.getTargetTableTypes().isEmpty() || context.getTargetTableTypes().contains(table.getType())) { + context.getCollectedTables().add(table); + } + if (table instanceof MTMV) { + expandMvAndCollect((MTMV) table, context); + } + return catalogRelation; + } + + private void expandMvAndCollect(MTMV mtmv, TableCollectorContext context) { + if (!context.isExpand()) { + return; + } + try { + MTMVCache expandedMv = MTMVCache.from(mtmv, MTMVPlanUtil.createMTMVContext(mtmv)); + expandedMv.getLogicalPlan().accept(this, context); + } catch (AnalysisException e) { + LOG.error(String.format( + "table collector expand fail, mtmv name is %s, targetTableTypes is %s", + mtmv.getName(), context.targetTableTypes), e); + throw new org.apache.doris.nereids.exceptions.AnalysisException( + String.format("expand mv and collect table fail, mv name is %s, mv sql is %s", + mtmv.getName(), mtmv.getQuerySql()), e); } - return super.visit(plan, context); } /** @@ -51,19 +91,26 @@ public Void visit(Plan plan, TableCollectorContext context) { * and the result of collect. */ public static final class TableCollectorContext { - private final List collectedTables = new ArrayList<>(); + private final Set collectedTables = new HashSet<>(); private final Set targetTableTypes; + // if expand the mv or not + private final boolean expand; - public TableCollectorContext(Set targetTableTypes) { + public TableCollectorContext(Set targetTableTypes, boolean expand) { this.targetTableTypes = targetTableTypes; + this.expand = expand; } - public List getCollectedTables() { + public Set getCollectedTables() { return collectedTables; } public Set getTargetTableTypes() { return targetTableTypes; } + + public boolean isExpand() { + return expand; + } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DataType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DataType.java index 470eee8153b856..e9905a44a41003 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DataType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DataType.java @@ -297,6 +297,9 @@ public static DataType convertPrimitiveFromStrings(List types, boolean u case "ipv6": dataType = IPv6Type.INSTANCE; break; + case "variant": + dataType = VariantType.INSTANCE; + break; default: throw new AnalysisException("Nereids do not support type: " + type); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/plsql/metastore/PlsqlManager.java b/fe/fe-core/src/main/java/org/apache/doris/plsql/metastore/PlsqlManager.java index 87cbd7b58a33f2..dc91d344f63421 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/plsql/metastore/PlsqlManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/plsql/metastore/PlsqlManager.java @@ -22,6 +22,7 @@ import org.apache.doris.common.io.Writable; import org.apache.doris.persist.gson.GsonUtils; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Maps; import com.google.gson.annotations.SerializedName; import org.apache.logging.log4j.LogManager; @@ -34,7 +35,20 @@ public class PlsqlManager implements Writable { private static final Logger LOG = LogManager.getLogger(PlsqlManager.class); - + public static final ImmutableList ROUTINE_INFO_TITLE_NAMES = new ImmutableList.Builder() + .add("SPECIFIC_NAME").add("ROUTINE_CATALOG").add("ROUTINE_SCHEMA").add("ROUTINE_NAME") + .add("ROUTINE_TYPE") + .add("DTD_IDENTIFIER").add("ROUTINE_BODY") + .add("ROUTINE_DEFINITION").add("EXTERNAL_NAME") + .add("EXTERNAL_LANGUAGE").add("PARAMETER_STYLE") + .add("IS_DETERMINISTIC") + .add("SQL_DATA_ACCESS").add("SQL_PATH") + .add("SECURITY_TYPE").add("CREATED") + .add("LAST_ALTERED").add("SQL_MODE") + .add("ROUTINE_COMMENT") + .add("DEFINER").add("CHARACTER_SET_CLIENT") + .add("COLLATION_CONNECTION").add("DATABASE_COLLATION") + .build(); @SerializedName(value = "nameToStoredProcedures") Map nameToStoredProcedures = Maps.newConcurrentMap(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java index 61beeb57ce698f..a0d369eafdec66 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java @@ -909,6 +909,9 @@ private void handleShowTable() throws AnalysisException { if (tbl.getName().startsWith(FeConstants.TEMP_MATERIZLIZE_DVIEW_PREFIX)) { continue; } + if (showTableStmt.getType() != null && tbl.getType() != showTableStmt.getType()) { + continue; + } if (matcher != null && !matcher.match(tbl.getName())) { continue; } @@ -2532,16 +2535,18 @@ private void handleShowColumnStats() throws AnalysisException { private void getStatsForAllColumns(List, ColumnStatistic>> columnStatistics, TableIf tableIf) throws AnalysisException { List resultRows = StatisticsRepository.queryColumnStatisticsForTable(tableIf.getId()); + // row[4] is index id, row[5] is column name. for (ResultRow row : resultRows) { - String indexName = "N/A"; + String indexName = tableIf.getName(); long indexId = Long.parseLong(row.get(4)); - if (indexId != -1) { - indexName = ((OlapTable) tableIf).getIndexNameById(indexId); - if (indexName == null) { - continue; - } + if (tableIf instanceof OlapTable) { + OlapTable olapTable = (OlapTable) tableIf; + indexName = olapTable.getIndexNameById(indexId == -1 ? olapTable.getBaseIndexId() : indexId); + } + if (indexName == null) { + continue; } - columnStatistics.add(Pair.of(Pair.of(row.get(5), indexName), ColumnStatistic.fromResultRow(row))); + columnStatistics.add(Pair.of(Pair.of(indexName, row.get(5)), ColumnStatistic.fromResultRow(row))); } } @@ -2558,28 +2563,29 @@ private void getStatsForSpecifiedColumns(List, ColumnS indexIds.add(-1L); } for (long indexId : indexIds) { - String indexName = "N/A"; - if (indexId != -1) { - indexName = ((OlapTable) tableIf).getIndexNameById(indexId); - if (indexName == null) { - continue; - } + String indexName = tableIf.getName(); + if (tableIf instanceof OlapTable) { + OlapTable olapTable = (OlapTable) tableIf; + indexName = olapTable.getIndexNameById(indexId == -1 ? olapTable.getBaseIndexId() : indexId); + } + if (indexName == null) { + continue; } // Show column statistics in columnStatisticsCache. if (showCache) { ColumnStatistic columnStatistic = Env.getCurrentEnv().getStatisticsCache().getColumnStatistics( tableIf.getDatabase().getCatalog().getId(), tableIf.getDatabase().getId(), tableIf.getId(), indexId, colName); - columnStatistics.add(Pair.of(Pair.of(colName, indexName), columnStatistic)); + columnStatistics.add(Pair.of(Pair.of(indexName, colName), columnStatistic)); } else if (partitionNames == null) { ColumnStatistic columnStatistic = StatisticsRepository.queryColumnStatisticsByName(tableIf.getId(), indexId, colName); - columnStatistics.add(Pair.of(Pair.of(colName, indexName), columnStatistic)); + columnStatistics.add(Pair.of(Pair.of(indexName, colName), columnStatistic)); } else { String finalIndexName = indexName; columnStatistics.addAll(StatisticsRepository.queryColumnStatisticsByPartitions(tableName, colName, partitionNames.getPartitionNames()) - .stream().map(s -> Pair.of(Pair.of(colName, finalIndexName), s)) + .stream().map(s -> Pair.of(Pair.of(finalIndexName, colName), s)) .collect(Collectors.toList())); } } @@ -2983,7 +2989,7 @@ private void handleShowAnalyzeTaskStatus() { if (table instanceof OlapTable && analysisInfo.indexId != -1) { row.add(((OlapTable) table).getIndexNameById(analysisInfo.indexId)); } else { - row.add("N/A"); + row.add(table.getName()); } row.add(analysisInfo.message); row.add(TimeUtils.DATETIME_FORMAT.format( 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 40af03d9237d51..783323b03fac72 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 @@ -820,7 +820,8 @@ private void handleQueryWithRetry(TUniqueId queryId) throws Exception { deadCloudClusterStatus); if (Strings.isNullOrEmpty(deadCloudClusterStatus) || ClusterStatus.valueOf(deadCloudClusterStatus) != ClusterStatus.NORMAL) { - CloudSystemInfoService.waitForAutoStart(deadCloudClusterClusterName); + ((CloudSystemInfoService) Env.getCurrentSystemInfo()) + .waitForAutoStart(deadCloudClusterClusterName); } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/resource/workloadgroup/WorkloadGroup.java b/fe/fe-core/src/main/java/org/apache/doris/resource/workloadgroup/WorkloadGroup.java index 4a220252afe17a..482d2f6f11a301 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/resource/workloadgroup/WorkloadGroup.java +++ b/fe/fe-core/src/main/java/org/apache/doris/resource/workloadgroup/WorkloadGroup.java @@ -71,6 +71,8 @@ public class WorkloadGroup implements Writable, GsonPostProcessable { public static final String SPILL_THRESHOLD_HIGH_WATERMARK = "spill_threshold_high_watermark"; + public static final String TAG = "tag"; + // NOTE(wb): all property is not required, some properties default value is set in be // default value is as followed // cpu_share=1024, memory_limit=0%(0 means not limit), enable_memory_overcommit=true @@ -78,7 +80,8 @@ public class WorkloadGroup implements Writable, GsonPostProcessable { .add(CPU_SHARE).add(MEMORY_LIMIT).add(ENABLE_MEMORY_OVERCOMMIT).add(MAX_CONCURRENCY) .add(MAX_QUEUE_SIZE).add(QUEUE_TIMEOUT).add(CPU_HARD_LIMIT).add(SCAN_THREAD_NUM) .add(MAX_REMOTE_SCAN_THREAD_NUM).add(MIN_REMOTE_SCAN_THREAD_NUM) - .add(SPILL_THRESHOLD_LOW_WATERMARK).add(SPILL_THRESHOLD_HIGH_WATERMARK).build(); + .add(SPILL_THRESHOLD_LOW_WATERMARK).add(SPILL_THRESHOLD_HIGH_WATERMARK) + .add(TAG).build(); public static final int SPILL_LOW_WATERMARK_DEFAULT_VALUE = 50; public static final int SPILL_HIGH_WATERMARK_DEFAULT_VALUE = 80; @@ -142,6 +145,9 @@ private WorkloadGroup(long id, String name, Map properties, long } this.properties.put(SPILL_THRESHOLD_HIGH_WATERMARK, highWatermarkStr); } + if (properties.containsKey(TAG)) { + this.properties.put(TAG, properties.get(TAG).toLowerCase()); + } resetQueueProperty(properties); } @@ -206,11 +212,25 @@ private static void checkProperties(Map properties) throws DdlEx if (properties.containsKey(CPU_HARD_LIMIT)) { String cpuHardLimit = properties.get(CPU_HARD_LIMIT); - if (cpuHardLimit.endsWith("%")) { - cpuHardLimit = cpuHardLimit.substring(0, cpuHardLimit.length() - 1); - } - if (!StringUtils.isNumeric(cpuHardLimit) || Long.parseLong(cpuHardLimit) <= 0) { - throw new DdlException(CPU_HARD_LIMIT + " " + cpuHardLimit + " requires a positive integer."); + String originValue = cpuHardLimit; + try { + boolean endWithSign = false; + if (cpuHardLimit.endsWith("%")) { + cpuHardLimit = cpuHardLimit.substring(0, cpuHardLimit.length() - 1); + endWithSign = true; + } + + int intVal = Integer.parseInt(cpuHardLimit); + if (endWithSign && intVal == -1) { + throw new NumberFormatException(); + } + if (!(intVal >= 1 && intVal <= 100) && -1 != intVal) { + throw new NumberFormatException(); + } + } catch (NumberFormatException e) { + throw new DdlException( + "workload group's " + WorkloadGroup.CPU_HARD_LIMIT + + " must be a positive integer[1,100] or -1, but input value is " + originValue); } } @@ -395,7 +415,9 @@ public void getProcNodeData(BaseProcResult result, QueryQueue qq) { if (CPU_HARD_LIMIT.equals(key)) { String val = properties.get(key); if (StringUtils.isEmpty(val)) { // cpu_hard_limit is not required - row.add("0%"); + row.add("-1"); + } else if ("-1".equals(val)) { + row.add(val); } else { row.add(val + "%"); } @@ -431,6 +453,13 @@ public void getProcNodeData(BaseProcResult result, QueryQueue qq) { row.add(qq == null ? "0" : String.valueOf(qq.getCurrentRunningQueryNum())); } else if (QueryQueue.WAITING_QUERY_NUM.equals(key)) { row.add(qq == null ? "0" : String.valueOf(qq.getCurrentWaitingQueryNum())); + } else if (TAG.equals(key)) { + String val = properties.get(key); + if (StringUtils.isEmpty(val)) { + row.add(""); + } else { + row.add(val); + } } else { row.add(properties.get(key)); } @@ -442,6 +471,10 @@ public int getCpuHardLimit() { return cpuHardLimit; } + public String getTag() { + return properties.get(TAG); + } + @Override public String toString() { return GsonUtils.GSON.toJson(this); 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 08de0ce338a61f..53269b15049985 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 @@ -48,6 +48,7 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.gson.annotations.SerializedName; +import org.apache.commons.lang3.StringUtils; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -59,7 +60,6 @@ import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.concurrent.locks.ReentrantReadWriteLock; public class WorkloadGroupMgr implements Writable, GsonPostProcessable { @@ -73,6 +73,7 @@ public class WorkloadGroupMgr implements Writable, GsonPostProcessable { .add(WorkloadGroup.SCAN_THREAD_NUM).add(WorkloadGroup.MAX_REMOTE_SCAN_THREAD_NUM) .add(WorkloadGroup.MIN_REMOTE_SCAN_THREAD_NUM) .add(WorkloadGroup.SPILL_THRESHOLD_LOW_WATERMARK).add(WorkloadGroup.SPILL_THRESHOLD_HIGH_WATERMARK) + .add(WorkloadGroup.TAG) .add(QueryQueue.RUNNING_QUERY_NUM).add(QueryQueue.WAITING_QUERY_NUM) .build(); @@ -301,38 +302,44 @@ public void createWorkloadGroup(CreateWorkloadGroupStmt stmt) throws DdlExceptio LOG.info("Create workload group success: {}", workloadGroup); } - private void checkGlobalUnlock(WorkloadGroup workloadGroup, WorkloadGroup old) throws DdlException { - double totalMemoryLimit = idToWorkloadGroup.values().stream().mapToDouble(WorkloadGroup::getMemoryLimitPercent) - .sum() + workloadGroup.getMemoryLimitPercent(); - if (!Objects.isNull(old)) { - totalMemoryLimit -= old.getMemoryLimitPercent(); - } - if (totalMemoryLimit > 100.0 + 1e-6) { - throw new DdlException( - "The sum of all workload group " + WorkloadGroup.MEMORY_LIMIT + " cannot be greater than 100.0%."); - } - - // 1, check new group - int newGroupCpuHardLimit = workloadGroup.getCpuHardLimit(); - if (newGroupCpuHardLimit > 100 || newGroupCpuHardLimit < 0) { - throw new DdlException( - "new group's " + WorkloadGroup.CPU_HARD_LIMIT - + " value can not be greater than 100% or less than or equal 0%"); - } - - // 2, check sum of all cpu hard limit + // NOTE: used for checking sum value of 100% for cpu_hard_limit and memory_limit + // when create/alter workload group with same tag. + // when oldWg is null it means caller is an alter stmt. + private void checkGlobalUnlock(WorkloadGroup newWg, WorkloadGroup oldWg) throws DdlException { + String wgTag = newWg.getTag(); + double sumOfAllMemLimit = 0; int sumOfAllCpuHardLimit = 0; for (Map.Entry entry : idToWorkloadGroup.entrySet()) { - if (old != null && entry.getKey() == old.getId()) { + WorkloadGroup wg = entry.getValue(); + if (!StringUtils.equals(wgTag, wg.getTag())) { + continue; + } + + if (oldWg != null && entry.getKey() == oldWg.getId()) { continue; } - sumOfAllCpuHardLimit += entry.getValue().getCpuHardLimit(); + + if (wg.getCpuHardLimit() > 0) { + sumOfAllCpuHardLimit += wg.getCpuHardLimit(); + } + if (wg.getMemoryLimitPercent() > 0) { + sumOfAllMemLimit += wg.getMemoryLimitPercent(); + } + } + + sumOfAllMemLimit += newWg.getMemoryLimitPercent(); + sumOfAllCpuHardLimit += newWg.getCpuHardLimit(); + + if (sumOfAllMemLimit > 100.0 + 1e-6) { + throw new DdlException( + "The sum of all workload group " + WorkloadGroup.MEMORY_LIMIT + " within tag " + wgTag + + " cannot be greater than 100.0%."); } - sumOfAllCpuHardLimit += newGroupCpuHardLimit; if (sumOfAllCpuHardLimit > 100) { - throw new DdlException("sum of all workload group " + WorkloadGroup.CPU_HARD_LIMIT - + " can not be greater than 100% "); + throw new DdlException( + "sum of all workload group " + WorkloadGroup.CPU_HARD_LIMIT + " within tag " + + wgTag + " can not be greater than 100% "); } } 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 992c512e4026b0..60ae26df757130 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 @@ -285,9 +285,9 @@ public class FrontendServiceImpl implements FrontendService.Iface { private MasterImpl masterImpl; private ExecuteEnv exeEnv; - // key is txn id,value is index of plan fragment instance, it's used by multi table request plan - private ConcurrentHashMap multiTableFragmentInstanceIdIndexMap = - new ConcurrentHashMap<>(64); + // key is txn id,value is index of plan fragment instance, it's used by multi + // table request plan + private ConcurrentHashMap multiTableFragmentInstanceIdIndexMap = new ConcurrentHashMap<>(64); private final Map proxyQueryIdToConnCtx = new ConcurrentHashMap<>(64); @@ -352,8 +352,9 @@ public TConfirmUnusedRemoteFilesResult confirmUnusedRemoteFiles(TConfirmUnusedRe } // check cooldownMetaId of all replicas are the same List replicas = Env.getCurrentEnv().getTabletInvertedIndex().getReplicas(info.tablet_id); - // FIXME(plat1ko): We only delete remote files when tablet is under a stable state: enough replicas and - // all replicas are alive. Are these conditions really sufficient or necessary? + // FIXME(plat1ko): We only delete remote files when tablet is under a stable + // state: enough replicas and + // all replicas are alive. Are these conditions really sufficient or necessary? if (replicas.size() < replicaNum) { LOG.info("num replicas are not enough, tablet={}", info.tablet_id); return; @@ -669,7 +670,6 @@ public TListTableMetadataNameIdsResult listTableMetadataNameIds(TGetTablesParams } PatternMatcher finalMatcher = matcher; - ExecutorService executor = Executors.newSingleThreadExecutor(); Future future = executor.submit(() -> { @@ -1013,7 +1013,8 @@ public TMasterOpResult forward(TMasterOpRequest params) throws TException { LOG.debug("receive forwarded stmt {} from FE: {}", params.getStmtId(), params.getClientNodeHost()); } ConnectContext context = new ConnectContext(null, true); - // Set current connected FE to the client address, so that we can know where this request come from. + // Set current connected FE to the client address, so that we can know where + // this request come from. context.setCurrentConnectedFEIp(params.getClientNodeHost()); if (Config.isCloudMode() && !Strings.isNullOrEmpty(params.getCloudCluster())) { context.setCloudCluster(params.getCloudCluster()); @@ -1359,7 +1360,8 @@ private List queryLoadCommitTables(TLoadTxnCommitRequest request, Databas OlapTable table = (OlapTable) db.getTableOrMetaException(tbl, TableType.OLAP); tables.add(table); } - // if it has multi table, use multi table and update multi table running transaction table ids + // if it has multi table, use multi table and update multi table running + // transaction table ids if (CollectionUtils.isNotEmpty(request.getTbls())) { List multiTableIds = tables.stream().map(Table::getId).collect(Collectors.toList()); Env.getCurrentGlobalTransactionMgr() @@ -1555,7 +1557,8 @@ public TLoadTxnCommitResult loadTxnCommit(TLoadTxnCommitRequest request) throws return result; } - // return true if commit success and publish success, return false if publish timeout + // return true if commit success and publish success, return false if publish + // timeout private boolean loadTxnCommitImpl(TLoadTxnCommitRequest request) throws UserException { if (request.isSetAuthCode()) { // TODO(cmy): find a way to check @@ -1633,7 +1636,8 @@ public TCommitTxnResult commitTxn(TCommitTxnRequest request) throws TException { return result; } - // return true if commit success and publish success, return false if publish timeout + // return true if commit success and publish success, return false if publish + // timeout private boolean commitTxnImpl(TCommitTxnRequest request) throws UserException { /// Check required arg: user, passwd, db, txn_id, commit_infos if (!request.isSetUser()) { @@ -1698,7 +1702,6 @@ private boolean commitTxnImpl(TCommitTxnRequest request) throws UserException { // Step 4: get timeout long timeoutMs = request.isSetThriftRpcTimeoutMs() ? request.getThriftRpcTimeoutMs() / 2 : 5000; - // Step 5: commit and publish return Env.getCurrentGlobalTransactionMgr() .commitAndPublishTransaction(db, tableList, @@ -1949,7 +1952,8 @@ public TStreamLoadPutResult streamLoadPut(TStreamLoadPutRequest request) { } /** - * For first-class multi-table scenarios, we should store the mapping between Txn and data source type in a common + * For first-class multi-table scenarios, we should store the mapping between + * Txn and data source type in a common * place. Since there is only Kafka now, we should do this first. */ private void buildMultiTableStreamLoadTask(StreamLoadTask baseTaskInfo, long txnId) { @@ -2002,7 +2006,7 @@ public TStreamLoadMultiTablePutResult streamLoadMultiTablePut(TStreamLoadPutRequ ctx.setCurrentUserIdentity(currentUser.get(0)); } LOG.info("one stream multi table load use cloud cluster {}", request.getCloudCluster()); - //ctx.setCloudCluster(); + // ctx.setCloudCluster(); if (!Strings.isNullOrEmpty(request.getCloudCluster())) { if (Strings.isNullOrEmpty(request.getUser())) { ctx.setCloudCluster(request.getCloudCluster()); @@ -2055,7 +2059,7 @@ public TStreamLoadMultiTablePutResult streamLoadMultiTablePut(TStreamLoadPutRequ RoutineLoadJob routineLoadJob = Env.getCurrentEnv().getRoutineLoadManager() .getRoutineLoadJobByMultiLoadTaskTxnId(request.getTxnId()); routineLoadJob.updateState(JobState.PAUSED, new ErrorReason(InternalErrorCode.CANNOT_RESUME_ERR, - "failed to get stream load plan, " + exception.getMessage()), false); + "failed to get stream load plan, " + exception.getMessage()), false); } catch (UserException e) { LOG.warn("catch update routine load job error.", e); } @@ -2112,9 +2116,9 @@ private HttpStreamParams initHttpStreamPlan(TStreamLoadPutRequest request, Conne coord.setQueryType(TQueryType.LOAD); TableIf table = httpStreamParams.getTable(); if (table instanceof OlapTable) { - boolean isEnableMemtableOnSinkNode = - ((OlapTable) table).getTableProperty().getUseSchemaLightChange() - ? coord.getQueryOptions().isEnableMemtableOnSinkNode() : false; + boolean isEnableMemtableOnSinkNode = ((OlapTable) table).getTableProperty().getUseSchemaLightChange() + ? coord.getQueryOptions().isEnableMemtableOnSinkNode() + : false; coord.getQueryOptions().setEnableMemtableOnSinkNode(isEnableMemtableOnSinkNode); } httpStreamParams.setParams(coord.getStreamLoadPlan()); @@ -2239,7 +2243,7 @@ private TExecPlanFragmentParams streamLoadPutImpl(TStreamLoadPutRequest request, try { if (!((OlapTable) table).getTableProperty().getUseSchemaLightChange() && (request.getGroupCommitMode() != null - && !request.getGroupCommitMode().equals("off_mode"))) { + && !request.getGroupCommitMode().equals("off_mode"))) { throw new UserException( "table light_schema_change is false, can't do stream load with group commit mode"); } @@ -2906,7 +2910,6 @@ private TGetBinlogResult getBinlogImpl(TGetBinlogRequest request, String clientI throw new UserException("prev_commit_seq is not set"); } - // step 1: check auth if (Strings.isNullOrEmpty(request.getToken())) { checkSingleTablePasswordAndPrivs(request.getUser(), request.getPasswd(), request.getDb(), @@ -2999,7 +3002,8 @@ public TGetSnapshotResult getSnapshot(TGetSnapshotRequest request) throws TExcep // getSnapshotImpl private TGetSnapshotResult getSnapshotImpl(TGetSnapshotRequest request, String clientIp) throws UserException { - // Step 1: Check all required arg: user, passwd, db, label_name, snapshot_name, snapshot_type + // Step 1: Check all required arg: user, passwd, db, label_name, snapshot_name, + // snapshot_type if (!request.isSetUser()) { throw new UserException("user is not set"); } @@ -3080,7 +3084,8 @@ public TRestoreSnapshotResult restoreSnapshot(TRestoreSnapshotRequest request) t // restoreSnapshotImpl private TRestoreSnapshotResult restoreSnapshotImpl(TRestoreSnapshotRequest request, String clientIp) throws UserException { - // Step 1: Check all required arg: user, passwd, db, label_name, repo_name, meta, info + // Step 1: Check all required arg: user, passwd, db, label_name, repo_name, + // meta, info if (!request.isSetUser()) { throw new UserException("user is not set"); } @@ -3347,7 +3352,6 @@ private TGetBinlogLagResult getBinlogLagImpl(TGetBinlogRequest request, String c throw new UserException("prev_commit_seq is not set"); } - // step 1: check auth if (Strings.isNullOrEmpty(request.getToken())) { checkSingleTablePasswordAndPrivs(request.getUser(), request.getPasswd(), request.getDb(), @@ -3597,7 +3601,7 @@ public TCreatePartitionResult createPartition(TCreatePartitionRequest request) t @Override public TReplacePartitionResult replacePartition(TReplacePartitionRequest request) throws TException { - LOG.info("Receive create partition request: {}", request); + LOG.info("Receive replace partition request: {}", request); long dbId = request.getDbId(); long tableId = request.getTableId(); List partitionIds = request.getPartitionIds(); @@ -3642,8 +3646,10 @@ public TReplacePartitionResult replacePartition(TReplacePartitionRequest request List allReqPartNames; // all request partitions try { taskLock.lock(); - // we dont lock the table. other thread in this txn will be controled by taskLock. - // if we have already replaced. dont do it again, but acquire the recorded new partition directly. + // we dont lock the table. other thread in this txn will be controled by + // taskLock. + // if we have already replaced. dont do it again, but acquire the recorded new + // partition directly. // if not by this txn, just let it fail naturally is ok. List replacedPartIds = overwriteManager.tryReplacePartitionIds(taskGroupId, partitionIds); // here if replacedPartIds still have null. this will throw exception. @@ -3653,7 +3659,8 @@ public TReplacePartitionResult replacePartition(TReplacePartitionRequest request .filter(i -> partitionIds.get(i) == replacedPartIds.get(i)) // equal means not replaced .mapToObj(partitionIds::get) .collect(Collectors.toList()); - // from here we ONLY deal the pending partitions. not include the dealed(by others). + // from here we ONLY deal the pending partitions. not include the dealed(by + // others). if (!pendingPartitionIds.isEmpty()) { // below two must have same order inner. List pendingPartitionNames = olapTable.uncheckedGetPartNamesById(pendingPartitionIds); @@ -3664,7 +3671,8 @@ public TReplacePartitionResult replacePartition(TReplacePartitionRequest request overwriteManager.registerTaskInGroup(taskGroupId, taskId); InsertOverwriteUtil.addTempPartitions(olapTable, pendingPartitionNames, tempPartitionNames); InsertOverwriteUtil.replacePartition(olapTable, pendingPartitionNames, tempPartitionNames); - // now temp partitions are bumped up and use new names. we get their ids and record them. + // now temp partitions are bumped up and use new names. we get their ids and + // record them. List newPartitionIds = new ArrayList(); for (String newPartName : pendingPartitionNames) { newPartitionIds.add(olapTable.getPartition(newPartName).getId()); @@ -3686,8 +3694,10 @@ public TReplacePartitionResult replacePartition(TReplacePartitionRequest request taskLock.unlock(); } - // build partition & tablets. now all partitions in allReqPartNames are replaced an recorded. - // so they won't be changed again. if other transaction changing it. just let it fail. + // build partition & tablets. now all partitions in allReqPartNames are replaced + // an recorded. + // so they won't be changed again. if other transaction changing it. just let it + // fail. List partitions = Lists.newArrayList(); List tablets = Lists.newArrayList(); PartitionInfo partitionInfo = olapTable.getPartitionInfo(); @@ -3796,7 +3806,7 @@ public TGetMetaResult getMeta(TGetMetaRequest request) throws TException { private TGetMetaResult getMetaImpl(TGetMetaRequest request, String clientIp) throws Exception { - // Step 1: check fields + // Step 1: check fields if (!request.isSetUser()) { throw new UserException("user is not set"); } @@ -3869,7 +3879,6 @@ private TGetMetaResult getMetaImpl(TGetMetaRequest request, String clientIp) } } - @Override public TGetColumnInfoResult getColumnInfo(TGetColumnInfoRequest request) { TGetColumnInfoResult result = new TGetColumnInfoResult(); @@ -3990,12 +3999,12 @@ public TStatus reportCommitTxnResult(TReportCommitTxnResultRequest request) thro // FE only has one master, this should not be a problem if (!Env.getCurrentEnv().isMaster()) { LOG.error("failed to handle load stats report: not master, backend:{}", - clientAddr); + clientAddr); return new TStatus(TStatusCode.NOT_MASTER); } LOG.info("receive load stats report request: {}, backend: {}, dbId: {}, txnId: {}, label: {}", - request, clientAddr, request.getDbId(), request.getTxnId(), request.getLabel()); + request, clientAddr, request.getDbId(), request.getTxnId(), request.getLabel()); try { byte[] receivedProtobufBytes = request.getPayload(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/service/arrowflight/DorisFlightSqlProducer.java b/fe/fe-core/src/main/java/org/apache/doris/service/arrowflight/DorisFlightSqlProducer.java index 20ea5836483ce4..2c7aaae4f2a475 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/service/arrowflight/DorisFlightSqlProducer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/service/arrowflight/DorisFlightSqlProducer.java @@ -238,16 +238,16 @@ private FlightInfo executeQueryStatement(String peerIdentity, ConnectContext con connectContext.getResultFlightServerAddr().port); List endpoints = Collections.singletonList(new FlightEndpoint(ticket, location)); // TODO Set in BE callback after query end, Client will not callback. - connectContext.setCommand(MysqlCommand.COM_SLEEP); return new FlightInfo(schema, descriptor, endpoints, -1, -1); } } catch (Exception e) { - connectContext.setCommand(MysqlCommand.COM_SLEEP); String errMsg = "get flight info statement failed, " + e.getMessage() + ", " + Util.getRootCauseMessage(e) + ", error code: " + connectContext.getState().getErrorCode() + ", error msg: " + connectContext.getState().getErrorMessage(); LOG.warn(errMsg, e); throw CallStatus.INTERNAL.withDescription(errMsg).withCause(e).toRuntimeException(); + } finally { + connectContext.setCommand(MysqlCommand.COM_SLEEP); } } @@ -306,6 +306,7 @@ public void createPreparedStatement(final ActionCreatePreparedStatementRequest r executorService.submit(() -> { ConnectContext connectContext = flightSessionsManager.getConnectContext(context.peerIdentity()); try { + connectContext.setCommand(MysqlCommand.COM_QUERY); final String query = request.getQuery(); String preparedStatementId = UUID.randomUUID().toString(); final ByteString handle = ByteString.copyFromUtf8(context.peerIdentity() + ":" + preparedStatementId); @@ -323,7 +324,6 @@ public void createPreparedStatement(final ActionCreatePreparedStatementRequest r Any.pack(buildCreatePreparedStatementResult(handle, parameterSchema, metaData)) .toByteArray())); } catch (Exception e) { - connectContext.setCommand(MysqlCommand.COM_SLEEP); String errMsg = "create prepared statement failed, " + e.getMessage() + ", " + Util.getRootCauseMessage(e) + ", error code: " + connectContext.getState().getErrorCode() + ", error msg: " + connectContext.getState().getErrorMessage(); @@ -333,6 +333,8 @@ public void createPreparedStatement(final ActionCreatePreparedStatementRequest r } catch (final Throwable t) { listener.onError(CallStatus.INTERNAL.withDescription("Unknown error: " + t).toRuntimeException()); return; + } finally { + connectContext.setCommand(MysqlCommand.COM_SLEEP); } listener.onCompleted(); }); diff --git a/fe/fe-core/src/main/java/org/apache/doris/service/arrowflight/DorisFlightSqlService.java b/fe/fe-core/src/main/java/org/apache/doris/service/arrowflight/DorisFlightSqlService.java index fe5a60f0cc24ca..a8ad9a05c93704 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/service/arrowflight/DorisFlightSqlService.java +++ b/fe/fe-core/src/main/java/org/apache/doris/service/arrowflight/DorisFlightSqlService.java @@ -47,7 +47,11 @@ public class DorisFlightSqlService { public DorisFlightSqlService(int port) { BufferAllocator allocator = new RootAllocator(); Location location = Location.forGrpcInsecure(FrontendOptions.getLocalHostAddress(), port); - this.flightTokenManager = new FlightTokenManagerImpl(Config.arrow_flight_token_cache_size, + // arrow_flight_token_cache_size less than qe_max_connection to avoid `Reach limit of connections`. + // arrow flight sql is a stateless protocol, connection is usually not actively disconnected. + // bearer token is evict from the cache will unregister ConnectContext. + this.flightTokenManager = new FlightTokenManagerImpl( + Math.min(Config.arrow_flight_token_cache_size, Config.qe_max_connection / 2), Config.arrow_flight_token_alive_time); this.flightSessionsManager = new FlightSessionsWithTokenManager(flightTokenManager); diff --git a/fe/fe-core/src/main/java/org/apache/doris/service/arrowflight/auth2/FlightBearerTokenAuthenticator.java b/fe/fe-core/src/main/java/org/apache/doris/service/arrowflight/auth2/FlightBearerTokenAuthenticator.java index ef6e28b034dd5e..9f4479c6bcd9d0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/service/arrowflight/auth2/FlightBearerTokenAuthenticator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/service/arrowflight/auth2/FlightBearerTokenAuthenticator.java @@ -86,7 +86,7 @@ AuthResult validateBearer(String token) { return createAuthResultWithBearerToken(token); } catch (IllegalArgumentException e) { LOG.error("Bearer token validation failed.", e); - throw CallStatus.UNAUTHENTICATED.toRuntimeException(); + throw CallStatus.UNAUTHENTICATED.withCause(e).withDescription(e.getMessage()).toRuntimeException(); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/service/arrowflight/sessions/FlightSessionsWithTokenManager.java b/fe/fe-core/src/main/java/org/apache/doris/service/arrowflight/sessions/FlightSessionsWithTokenManager.java index 26a48f0cfd2ead..b7e5ffa46466a6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/service/arrowflight/sessions/FlightSessionsWithTokenManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/service/arrowflight/sessions/FlightSessionsWithTokenManager.java @@ -47,42 +47,36 @@ public ConnectContext getConnectContext(String peerIdentity) { ConnectContext connectContext = ExecuteEnv.getInstance().getScheduler().getContext(peerIdentity); if (null == connectContext) { connectContext = createConnectContext(peerIdentity); - if (null == connectContext) { - flightTokenManager.invalidateToken(peerIdentity); - String err = "UserSession expire after access, need reauthorize."; - LOG.error(err); - throw CallStatus.UNAUTHENTICATED.withDescription(err).toRuntimeException(); - } return connectContext; } return connectContext; } catch (Exception e) { - LOG.warn("getConnectContext failed, " + e.getMessage(), e); + LOG.warn("get ConnectContext failed, " + e.getMessage(), e); throw CallStatus.INTERNAL.withDescription(Util.getRootCauseMessage(e)).withCause(e).toRuntimeException(); } } @Override public ConnectContext createConnectContext(String peerIdentity) { - try { - final FlightTokenDetails flightTokenDetails = flightTokenManager.validateToken(peerIdentity); - if (flightTokenDetails.getCreatedSession()) { - return null; - } - flightTokenDetails.setCreatedSession(true); - ConnectContext connectContext = FlightSessionsManager.buildConnectContext(peerIdentity, - flightTokenDetails.getUserIdentity(), flightTokenDetails.getRemoteIp()); - connectContext.setConnectionId(nextConnectionId.getAndAdd(1)); - connectContext.resetLoginTime(); - if (!ExecuteEnv.getInstance().getScheduler().registerConnection(connectContext)) { - connectContext.getState() - .setError(ErrorCode.ERR_TOO_MANY_USER_CONNECTIONS, "Reach limit of connections"); - throw CallStatus.UNAUTHENTICATED.withDescription("Reach limit of connections").toRuntimeException(); - } - return connectContext; - } catch (IllegalArgumentException e) { - LOG.error("Bearer token validation failed.", e); - throw CallStatus.UNAUTHENTICATED.toRuntimeException(); + final FlightTokenDetails flightTokenDetails = flightTokenManager.validateToken(peerIdentity); + if (flightTokenDetails.getCreatedSession()) { + flightTokenManager.invalidateToken(peerIdentity); + throw new IllegalArgumentException("UserSession expire after access, try reconnect, bearer token: " + + peerIdentity + ", a peerIdentity(bearer token) can only create a ConnectContext once. " + + "if ConnectContext is deleted without operation for a long time, it needs to be reconnected " + + "(at the same time obtain a new bearer token)."); + } + flightTokenDetails.setCreatedSession(true); + ConnectContext connectContext = FlightSessionsManager.buildConnectContext(peerIdentity, + flightTokenDetails.getUserIdentity(), flightTokenDetails.getRemoteIp()); + connectContext.setConnectionId(nextConnectionId.getAndAdd(1)); + connectContext.resetLoginTime(); + if (!ExecuteEnv.getInstance().getScheduler().registerConnection(connectContext)) { + String err = "Reach limit of connections, increase `qe_max_connection` in fe.conf, or decrease " + + "`arrow_flight_token_cache_size` to evict unused bearer tokens and it connections faster"; + connectContext.getState().setError(ErrorCode.ERR_TOO_MANY_USER_CONNECTIONS, err); + throw new IllegalArgumentException(err); } + return connectContext; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/service/arrowflight/tokens/FlightTokenManagerImpl.java b/fe/fe-core/src/main/java/org/apache/doris/service/arrowflight/tokens/FlightTokenManagerImpl.java index 54e53e931dd359..cd1b492de068cd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/service/arrowflight/tokens/FlightTokenManagerImpl.java +++ b/fe/fe-core/src/main/java/org/apache/doris/service/arrowflight/tokens/FlightTokenManagerImpl.java @@ -19,12 +19,16 @@ package org.apache.doris.service.arrowflight.tokens; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.service.ExecuteEnv; import org.apache.doris.service.arrowflight.auth2.FlightAuthResult; import com.google.common.base.Preconditions; import com.google.common.cache.CacheBuilder; import com.google.common.cache.CacheLoader; import com.google.common.cache.LoadingCache; +import com.google.common.cache.RemovalListener; +import com.google.common.cache.RemovalNotification; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -39,17 +43,32 @@ public class FlightTokenManagerImpl implements FlightTokenManager { private static final Logger LOG = LogManager.getLogger(FlightTokenManagerImpl.class); private final SecureRandom generator = new SecureRandom(); + private final int cacheSize; private final int cacheExpiration; private LoadingCache tokenCache; public FlightTokenManagerImpl(final int cacheSize, final int cacheExpiration) { + this.cacheSize = cacheSize; this.cacheExpiration = cacheExpiration; - this.tokenCache = CacheBuilder.newBuilder() - .maximumSize(cacheSize) + this.tokenCache = CacheBuilder.newBuilder().maximumSize(cacheSize) .expireAfterWrite(cacheExpiration, TimeUnit.MINUTES) - .build(new CacheLoader() { + .removalListener(new RemovalListener() { + @Override + public void onRemoval(RemovalNotification notification) { + // TODO: broadcast this message to other FE + LOG.info("evict bearer token: " + notification.getKey() + ", reason: " + + notification.getCause()); + ConnectContext context = ExecuteEnv.getInstance().getScheduler() + .getContext(notification.getKey()); + if (context != null) { + ExecuteEnv.getInstance().getScheduler().unregisterConnection(context); + LOG.info("unregister flight connect context after evict bearer token: " + + notification.getKey()); + } + } + }).build(new CacheLoader() { @Override public FlightTokenDetails load(String key) { return new FlightTokenDetails(); @@ -77,26 +96,32 @@ public FlightTokenDetails createToken(final String username, final FlightAuthRes flightAuthResult.getUserIdentity(), flightAuthResult.getRemoteIp()); tokenCache.put(token, flightTokenDetails); - LOG.trace("Created flight token for user: {}", username); + LOG.info("Created flight token for user: {}, token: {}", username, token); return flightTokenDetails; } @Override public FlightTokenDetails validateToken(final String token) throws IllegalArgumentException { final FlightTokenDetails value = getTokenDetails(token); + if (value.getToken().equals("")) { + throw new IllegalArgumentException("invalid bearer token: " + token + + ", try reconnect, bearer token may not be created, or may have been evict, search for this " + + "token in fe.log to see the evict reason. currently in fe.conf, `arrow_flight_token_cache_size`=" + + this.cacheSize + ", `arrow_flight_token_alive_time`=" + this.cacheExpiration); + } if (System.currentTimeMillis() >= value.getExpiresAt()) { - tokenCache.invalidate(token); // removes from the store as well - throw new IllegalArgumentException("token expired"); + tokenCache.invalidate(token); + throw new IllegalArgumentException("bearer token expired: " + token + ", try reconnect, " + + "currently in fe.conf, `arrow_flight_token_alive_time`=" + this.cacheExpiration); } - - LOG.trace("Validated flight token for user: {}", value.getUsername()); + LOG.info("Validated bearer token for user: {}", value.getUsername()); return value; } @Override public void invalidateToken(final String token) { - LOG.trace("Invalidate flight token, {}", token); - tokenCache.invalidate(token); // removes from the store as well + LOG.info("Invalidate bearer token, {}", token); + tokenCache.invalidate(token); } private FlightTokenDetails getTokenDetails(final String token) { @@ -105,7 +130,7 @@ private FlightTokenDetails getTokenDetails(final String token) { try { value = tokenCache.getUnchecked(token); } catch (CacheLoader.InvalidCacheLoadException ignored) { - throw new IllegalArgumentException("invalid token"); + throw new IllegalArgumentException("InvalidCacheLoadException, invalid bearer token: " + token); } return value; diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisInfo.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisInfo.java index c707107e0e0fb6..c167db2228d8cc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisInfo.java @@ -18,6 +18,7 @@ package org.apache.doris.statistics; import org.apache.doris.catalog.TableIf; +import org.apache.doris.common.Pair; import org.apache.doris.common.io.Text; import org.apache.doris.common.io.Writable; import org.apache.doris.persist.gson.GsonUtils; @@ -35,7 +36,6 @@ import java.io.IOException; import java.text.ParseException; import java.util.List; -import java.util.Map; import java.util.Set; import java.util.StringJoiner; @@ -95,8 +95,8 @@ public enum ScheduleType { @SerializedName("tblId") public final long tblId; - // TODO: Map here is wired, List is enough - public final Map> colToPartitions; + // Pair + public final List> jobColumns; public final Set partitionNames; @@ -200,7 +200,7 @@ public enum ScheduleType { public final boolean userInject; public AnalysisInfo(long jobId, long taskId, List taskIds, long catalogId, long dbId, long tblId, - Map> colToPartitions, Set partitionNames, String colName, Long indexId, + List> jobColumns, Set partitionNames, String colName, Long indexId, JobType jobType, AnalysisMode analysisMode, AnalysisMethod analysisMethod, AnalysisType analysisType, int samplePercent, long sampleRows, int maxBucketNum, long periodTimeInMs, String message, long lastExecTimeInMs, long timeCostInMs, AnalysisState state, ScheduleType scheduleType, @@ -213,7 +213,7 @@ public AnalysisInfo(long jobId, long taskId, List taskIds, long catalogId, this.catalogId = catalogId; this.dbId = dbId; this.tblId = tblId; - this.colToPartitions = colToPartitions; + this.jobColumns = jobColumns; this.partitionNames = partitionNames; this.colName = colName; this.indexId = indexId; @@ -268,8 +268,8 @@ public String toString() { if (maxBucketNum > 0) { sj.add("MaxBucketNum: " + maxBucketNum); } - if (colToPartitions != null) { - sj.add("colToPartitions: " + getColToPartitionStr()); + if (jobColumns != null) { + sj.add("jobColumns: " + getJobColumns()); } if (lastExecTimeInMs > 0) { sj.add("LastExecTime: " + StatisticsUtil.getReadableTime(lastExecTimeInMs)); @@ -301,12 +301,12 @@ public void addTaskId(long taskId) { taskIds.add(taskId); } - public String getColToPartitionStr() { - if (colToPartitions == null || colToPartitions.isEmpty()) { + public String getJobColumns() { + if (jobColumns == null || jobColumns.isEmpty()) { return ""; } Gson gson = new Gson(); - return gson.toJson(colToPartitions); + return gson.toJson(jobColumns); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisInfoBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisInfoBuilder.java index 22f3d22b3ce77c..00cf9f7b1bc560 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisInfoBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisInfoBuilder.java @@ -17,6 +17,7 @@ package org.apache.doris.statistics; +import org.apache.doris.common.Pair; import org.apache.doris.statistics.AnalysisInfo.AnalysisMethod; import org.apache.doris.statistics.AnalysisInfo.AnalysisMode; import org.apache.doris.statistics.AnalysisInfo.AnalysisType; @@ -26,7 +27,6 @@ import org.apache.logging.log4j.core.util.CronExpression; import java.util.List; -import java.util.Map; import java.util.Set; public class AnalysisInfoBuilder { @@ -36,7 +36,7 @@ public class AnalysisInfoBuilder { private long catalogId; private long dbId; private long tblId; - private Map> colToPartitions; + private List> jobColumns; private Set partitionNames; private String colName; private long indexId = -1L; @@ -75,7 +75,7 @@ public AnalysisInfoBuilder(AnalysisInfo info) { catalogId = info.catalogId; dbId = info.dbId; tblId = info.tblId; - colToPartitions = info.colToPartitions; + jobColumns = info.jobColumns; partitionNames = info.partitionNames; colName = info.colName; indexId = info.indexId; @@ -135,8 +135,8 @@ public AnalysisInfoBuilder setTblId(long tblId) { return this; } - public AnalysisInfoBuilder setColToPartitions(Map> colToPartitions) { - this.colToPartitions = colToPartitions; + public AnalysisInfoBuilder setJobColumns(List> jobColumns) { + this.jobColumns = jobColumns; return this; } @@ -276,7 +276,7 @@ public AnalysisInfoBuilder setUserInject(boolean userInject) { } public AnalysisInfo build() { - return new AnalysisInfo(jobId, taskId, taskIds, catalogId, dbId, tblId, colToPartitions, partitionNames, + return new AnalysisInfo(jobId, taskId, taskIds, catalogId, dbId, tblId, jobColumns, partitionNames, colName, indexId, jobType, analysisMode, analysisMethod, analysisType, samplePercent, sampleRows, maxBucketNum, periodTimeInMs, message, lastExecTimeInMs, timeCostInMs, state, scheduleType, externalTableLevelTask, partitionOnly, samplingPartition, isAllPartition, partitionCount, diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisJob.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisJob.java index f52764bd6c9f4a..5fd5e43be53f2b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisJob.java @@ -180,13 +180,13 @@ public void cancel() { public void deregisterJob() { analysisManager.removeJob(jobInfo.jobId); for (BaseAnalysisTask task : queryingTask) { - task.info.colToPartitions.clear(); + task.info.jobColumns.clear(); if (task.info.partitionNames != null) { task.info.partitionNames.clear(); } } for (BaseAnalysisTask task : queryFinished) { - task.info.colToPartitions.clear(); + task.info.jobColumns.clear(); if (task.info.partitionNames != null) { task.info.partitionNames.clear(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisManager.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisManager.java index 258c33305afc4d..66d6d38f381f2c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisManager.java @@ -38,6 +38,7 @@ import org.apache.doris.common.DdlException; import org.apache.doris.common.FeConstants; import org.apache.doris.common.FeMetaVersion; +import org.apache.doris.common.Pair; import org.apache.doris.common.ThreadPoolManager; import org.apache.doris.common.ThreadPoolManager.BlockedPolicy; import org.apache.doris.common.io.Text; @@ -82,7 +83,6 @@ import java.util.Collections; import java.util.Comparator; import java.util.HashMap; -import java.util.HashSet; import java.util.LinkedList; import java.util.List; import java.util.Map; @@ -219,7 +219,7 @@ public void createAnalysisJob(AnalyzeTblStmt stmt, boolean proxy) throws DdlExce @VisibleForTesting protected AnalysisInfo buildAndAssignJob(AnalyzeTblStmt stmt) throws DdlException { AnalysisInfo jobInfo = buildAnalysisJobInfo(stmt); - if (jobInfo.colToPartitions.isEmpty()) { + if (jobInfo.jobColumns.isEmpty()) { // No statistics need to be collected or updated return null; } @@ -292,51 +292,6 @@ private void sendJobId(List analysisInfos, boolean proxy) { } } - /** - * Gets the partitions for which statistics are to be collected. First verify that - * there are partitions that have been deleted but have historical statistics(invalid statistics), - * if there are these partitions, we need to delete them to avoid errors in summary table level statistics. - * Then get the partitions for which statistics need to be collected based on collection mode (incremental/full). - *

- * note: - * If there is no invalid statistics, it does not need to collect/update - * statistics if the following conditions are met: - * - in full collection mode, the partitioned table does not have partitions - * - in incremental collection mode, partition statistics already exist - *

- * TODO Supports incremental collection of statistics from materialized views - */ - private Map> validateAndGetPartitions(TableIf table, Set columnNames, - Set partitionNames, AnalysisType analysisType) throws DdlException { - - Map> columnToPartitions = columnNames.stream() - .collect(Collectors.toMap( - columnName -> columnName, - columnName -> new HashSet<>(partitionNames == null ? Collections.emptySet() : partitionNames) - )); - - if (analysisType == AnalysisType.HISTOGRAM) { - // Collecting histograms does not need to support incremental collection, - // and will automatically cover historical statistics - return columnToPartitions; - } - - if (table instanceof HMSExternalTable) { - // TODO Currently, we do not support INCREMENTAL collection for external table. - // One reason is external table partition id couldn't convert to a Long value. - // Will solve this problem later. - return columnToPartitions; - } - - if (analysisType == AnalysisType.FUNDAMENTALS) { - Map> result = table.findReAnalyzeNeededPartitions(); - result.keySet().retainAll(columnNames); - return result; - } - - return columnToPartitions; - } - // Make sure colName of job has all the column as this AnalyzeStmt specified, no matter whether it will be analyzed // or not. @VisibleForTesting @@ -362,12 +317,6 @@ public AnalysisInfo buildAnalysisJobInfo(AnalyzeTblStmt stmt) throws DdlExceptio infoBuilder.setCatalogId(stmt.getCatalogId()); infoBuilder.setDBId(stmt.getDbId()); infoBuilder.setTblId(stmt.getTable().getId()); - // TODO: Refactor later, DON'T MODIFY IT RIGHT NOW - StringJoiner stringJoiner = new StringJoiner(",", "[", "]"); - for (String colName : columnNames) { - stringJoiner.add(colName); - } - infoBuilder.setColName(stringJoiner.toString()); infoBuilder.setPartitionNames(partitionNames); infoBuilder.setPartitionOnly(partitionOnly); infoBuilder.setSamplingPartition(isSamplingPartition); @@ -391,20 +340,23 @@ public AnalysisInfo buildAnalysisJobInfo(AnalyzeTblStmt stmt) throws DdlExceptio if (analysisType == AnalysisType.HISTOGRAM) { int numBuckets = stmt.getNumBuckets(); - int maxBucketNum = numBuckets > 0 ? numBuckets - : StatisticConstants.HISTOGRAM_MAX_BUCKET_NUM; + int maxBucketNum = numBuckets > 0 ? numBuckets : StatisticConstants.HISTOGRAM_MAX_BUCKET_NUM; infoBuilder.setMaxBucketNum(maxBucketNum); } long periodTimeInMs = stmt.getPeriodTimeInMs(); infoBuilder.setPeriodTimeInMs(periodTimeInMs); - - Map> colToPartitions = validateAndGetPartitions(table, columnNames, - partitionNames, analysisType); - infoBuilder.setColToPartitions(colToPartitions); + List> jobColumns = table.getColumnIndexPairs(columnNames); + infoBuilder.setJobColumns(jobColumns); + StringJoiner stringJoiner = new StringJoiner(",", "[", "]"); + for (Pair pair : jobColumns) { + stringJoiner.add(pair.toString()); + } + infoBuilder.setColName(stringJoiner.toString()); infoBuilder.setTaskIds(Lists.newArrayList()); infoBuilder.setTblUpdateTime(table.getUpdateTime()); - infoBuilder.setEmptyJob(table instanceof OlapTable && table.getRowCount() == 0); + infoBuilder.setEmptyJob(table instanceof OlapTable && table.getRowCount() == 0 + && analysisMethod.equals(AnalysisMethod.SAMPLE)); return infoBuilder.build(); } @@ -420,35 +372,28 @@ public void recordAnalysisJob(AnalysisInfo jobInfo) { public void createTaskForEachColumns(AnalysisInfo jobInfo, Map analysisTasks, boolean isSync) throws DdlException { - Map> columnToPartitions = jobInfo.colToPartitions; + List> jobColumns = jobInfo.jobColumns; TableIf table = jobInfo.getTable(); - for (Entry> entry : columnToPartitions.entrySet()) { - String colName = entry.getKey(); - List indexIds = Lists.newArrayList(); - // Get index id this column belongs to for OlapTable. Set it to -1 for baseIndex id. - if (table instanceof OlapTable) { - indexIds = ((OlapTable) table).getMvColumnIndexIds(colName); - } else { - indexIds.add(-1L); - } + for (Pair pair : jobColumns) { AnalysisInfoBuilder colTaskInfoBuilder = new AnalysisInfoBuilder(jobInfo); - if (jobInfo.analysisType != AnalysisType.HISTOGRAM) { - colTaskInfoBuilder.setAnalysisType(AnalysisType.FUNDAMENTALS); - Map> colToParts = new HashMap<>(); - colToParts.put(colName, entry.getValue()); - colTaskInfoBuilder.setColToPartitions(colToParts); - } - for (long indexId : indexIds) { - long taskId = Env.getCurrentEnv().getNextId(); - AnalysisInfo analysisInfo = colTaskInfoBuilder.setColName(colName).setIndexId(indexId) - .setTaskId(taskId).setLastExecTimeInMs(System.currentTimeMillis()).build(); - analysisTasks.put(taskId, createTask(analysisInfo)); - jobInfo.addTaskId(taskId); - if (isSync) { - continue; + colTaskInfoBuilder.setAnalysisType(AnalysisType.FUNDAMENTALS); + long taskId = Env.getCurrentEnv().getNextId(); + long indexId = -1; + if (table instanceof OlapTable) { + OlapTable olapTable = (OlapTable) table; + indexId = olapTable.getIndexIdByName(pair.first); + if (indexId == olapTable.getBaseIndexId()) { + indexId = -1; } - replayCreateAnalysisTask(analysisInfo); } + AnalysisInfo analysisInfo = colTaskInfoBuilder.setColName(pair.second).setIndexId(indexId) + .setTaskId(taskId).setLastExecTimeInMs(System.currentTimeMillis()).build(); + analysisTasks.put(taskId, createTask(analysisInfo)); + jobInfo.addTaskId(taskId); + if (isSync) { + continue; + } + replayCreateAnalysisTask(analysisInfo); } } @@ -565,7 +510,9 @@ public void updateTableStats(AnalysisInfo jobInfo) { tableStats.update(jobInfo, tbl); logCreateTableStats(tableStats); } - jobInfo.colToPartitions.clear(); + if (jobInfo.jobColumns != null) { + jobInfo.jobColumns.clear(); + } if (jobInfo.partitionNames != null) { jobInfo.partitionNames.clear(); } @@ -712,7 +659,16 @@ public void invalidateLocalStats(long catalogId, long dbId, long tableId, indexIds.add(-1L); } for (long indexId : indexIds) { - tableStats.removeColumn(column); + String indexName = table.getName(); + if (table instanceof OlapTable) { + OlapTable olapTable = (OlapTable) table; + if (indexId == -1) { + indexName = olapTable.getIndexNameById(olapTable.getBaseIndexId()); + } else { + indexName = olapTable.getIndexNameById(indexId); + } + } + tableStats.removeColumn(indexName, column); statisticsCache.invalidate(tableId, indexId, column); } } @@ -1088,25 +1044,16 @@ public void registerSysJob(AnalysisInfo jobInfo, Map tas analysisJobIdToTaskMap.put(jobInfo.jobId, taskInfos); } - // Remove col stats status from TableStats if failed load some col stats after analyze corresponding column so that - // we could make sure it would be analyzed again soon if user or system submit job for that column again. - public void removeColStatsStatus(long tblId, String colName) { - TableStatsMeta tableStats = findTableStatsStatus(tblId); - if (tableStats != null) { - tableStats.removeColumn(colName); - } - } - public void removeTableStats(long tableId) { idToTblStats.remove(tableId); } - public ColStatsMeta findColStatsMeta(long tblId, String colName) { + public ColStatsMeta findColStatsMeta(long tblId, String indexName, String colName) { TableStatsMeta tableStats = findTableStatsStatus(tblId); if (tableStats == null) { return null; } - return tableStats.findColumnStatsMeta(colName); + return tableStats.findColumnStatsMeta(indexName, colName); } public AnalysisJob findJob(long id) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/BaseAnalysisTask.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/BaseAnalysisTask.java index 68767843507ad4..f871e8761a5e55 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/BaseAnalysisTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/BaseAnalysisTask.java @@ -82,6 +82,30 @@ public abstract class BaseAnalysisTask { + "NOW() " + "FROM `${catalogName}`.`${dbName}`.`${tblName}` ${index} ${sampleHints} ${limit}"; + protected static final String DUJ1_ANALYZE_STRING_TEMPLATE = "SELECT " + + "CONCAT('${tblId}', '-', '${idxId}', '-', '${colId}') AS `id`, " + + "${catalogId} AS `catalog_id`, " + + "${dbId} AS `db_id`, " + + "${tblId} AS `tbl_id`, " + + "${idxId} AS `idx_id`, " + + "'${colId}' AS `col_id`, " + + "NULL AS `part_id`, " + + "${rowCount} AS `row_count`, " + + "${ndvFunction} as `ndv`, " + + "IFNULL(SUM(IF(`t1`.`column_key` IS NULL, `t1`.`count`, 0)), 0) * ${scaleFactor} as `null_count`, " + + "SUBSTRING(CAST(${min} AS STRING), 1, 1024) AS `min`, " + + "SUBSTRING(CAST(${max} AS STRING), 1, 1024) AS `max`, " + + "${dataSizeFunction} * ${scaleFactor} AS `data_size`, " + + "NOW() " + + "FROM ( " + + " SELECT t0.`colValue` as `column_key`, COUNT(1) as `count` " + + " FROM " + + " (SELECT SUBSTRING(CAST(`${colName}` AS STRING), 1, 1024) AS `colValue` " + + " FROM `${catalogName}`.`${dbName}`.`${tblName}` ${index} " + + " ${sampleHints} ${limit}) as `t0` " + + " GROUP BY `t0`.`colValue` " + + ") as `t1` "; + protected static final String DUJ1_ANALYZE_TEMPLATE = "SELECT " + "CONCAT('${tblId}', '-', '${idxId}', '-', '${colId}') AS `id`, " + "${catalogId} AS `catalog_id`, " diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/ExternalAnalysisTask.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/ExternalAnalysisTask.java index ef1b795bd13827..287941be526635 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/ExternalAnalysisTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/ExternalAnalysisTask.java @@ -129,7 +129,11 @@ protected void getColumnStats() throws Exception { params.put("ndvFunction", "ROUND(NDV(`${colName}`) * ${scaleFactor})"); params.put("rowCount", "ROUND(count(1) * ${scaleFactor})"); } else { - sb.append(DUJ1_ANALYZE_TEMPLATE); + if (col.getType().isStringType()) { + sb.append(DUJ1_ANALYZE_STRING_TEMPLATE); + } else { + sb.append(DUJ1_ANALYZE_TEMPLATE); + } params.put("dataSizeFunction", getDataSizeFunction(col, true)); params.put("ndvFunction", getNdvFunction("ROUND(SUM(t1.count) * ${scaleFactor})")); params.put("rowCount", "ROUND(SUM(t1.count) * ${scaleFactor})"); diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/OlapAnalysisTask.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/OlapAnalysisTask.java index d26de9d9de7141..60bfcab6157377 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/OlapAnalysisTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/OlapAnalysisTask.java @@ -65,13 +65,8 @@ public OlapAnalysisTask(AnalysisInfo info) { } public void doExecute() throws Exception { - Set partitionNames = info.colToPartitions.get(info.colName); - if (StatisticsUtil.isEmptyTable(tbl, info.analysisMethod) - || partitionNames == null || partitionNames.isEmpty()) { - if (partitionNames == null) { - LOG.warn("Table {}.{}.{}, partitionNames for column {} is null. ColToPartitions:[{}]", - info.catalogId, info.dbId, info.tblId, info.colName, info.colToPartitions); - } + List> columnList = info.jobColumns; + if (StatisticsUtil.isEmptyTable(tbl, info.analysisMethod) || columnList == null || columnList.isEmpty()) { StatsId statsId = new StatsId(concatColumnStatsId(), info.catalogId, info.dbId, info.tblId, info.indexId, info.colName, null); job.appendBuf(this, Arrays.asList(new ColStatsData(statsId))); @@ -166,7 +161,11 @@ protected void doSample() throws Exception { sql = stringSubstitutor.replace(LINEAR_ANALYZE_TEMPLATE); } else { params.put("dataSizeFunction", getDataSizeFunction(col, true)); - sql = stringSubstitutor.replace(DUJ1_ANALYZE_TEMPLATE); + if (col.getType().isStringType()) { + sql = stringSubstitutor.replace(DUJ1_ANALYZE_STRING_TEMPLATE); + } else { + sql = stringSubstitutor.replace(DUJ1_ANALYZE_TEMPLATE); + } } LOG.info("Sample for column [{}]. Total rows [{}], rows to sample [{}], scale factor [{}], " + "limited [{}], distribute column [{}], partition column [{}], key column [{}], " diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsAutoCollector.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsAutoCollector.java index dbb7046467aec0..9ca971845b7e64 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsAutoCollector.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsAutoCollector.java @@ -25,6 +25,7 @@ import org.apache.doris.catalog.TableIf; import org.apache.doris.common.Config; import org.apache.doris.common.DdlException; +import org.apache.doris.common.Pair; import org.apache.doris.common.util.TimeUtils; import org.apache.doris.datasource.CatalogIf; import org.apache.doris.datasource.hive.HMSExternalTable; @@ -39,10 +40,9 @@ import java.time.LocalTime; import java.util.ArrayList; -import java.util.HashMap; import java.util.List; -import java.util.Map; import java.util.Set; +import java.util.StringJoiner; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; @@ -121,7 +121,7 @@ public void analyzeDb(DatabaseIf databaseIf) throws DdlException { analysisTaskExecutor.clear(); break; } - analysisInfo = getReAnalyzeRequiredPart(analysisInfo); + analysisInfo = getNeedAnalyzeColumns(analysisInfo); if (analysisInfo == null) { continue; } @@ -186,11 +186,7 @@ protected void createAnalyzeJobForTbl(DatabaseIf db, .setCatalogId(db.getCatalog().getId()) .setDBId(db.getId()) .setTblId(table.getId()) - .setColName( - table.getSchemaAllIndexes(false).stream() - .filter(c -> !StatisticsUtil.isUnsupportedType(c.getType())) - .map(Column::getName).collect(Collectors.joining(",")) - ) + .setColName(null) .setAnalysisType(AnalysisInfo.AnalysisType.FUNDAMENTALS) .setAnalysisMode(AnalysisInfo.AnalysisMode.INCREMENTAL) .setAnalysisMethod(analysisMethod) @@ -202,13 +198,14 @@ protected void createAnalyzeJobForTbl(DatabaseIf db, .setLastExecTimeInMs(System.currentTimeMillis()) .setJobType(JobType.SYSTEM) .setTblUpdateTime(table.getUpdateTime()) - .setEmptyJob(table instanceof OlapTable && table.getRowCount() == 0) + .setEmptyJob(table instanceof OlapTable && table.getRowCount() == 0 + && analysisMethod.equals(AnalysisMethod.SAMPLE)) .build(); analysisInfos.add(jobInfo); } @VisibleForTesting - protected AnalysisInfo getReAnalyzeRequiredPart(AnalysisInfo jobInfo) { + protected AnalysisInfo getNeedAnalyzeColumns(AnalysisInfo jobInfo) { TableIf table = StatisticsUtil.findTable(jobInfo.catalogId, jobInfo.dbId, jobInfo.tblId); // Skip tables that are too wide. if (table.getBaseSchema().size() > StatisticsUtil.getAutoAnalyzeTableWidthThreshold()) { @@ -218,26 +215,25 @@ protected AnalysisInfo getReAnalyzeRequiredPart(AnalysisInfo jobInfo) { AnalysisManager analysisManager = Env.getServingEnv().getAnalysisManager(); TableStatsMeta tblStats = analysisManager.findTableStatsStatus(table.getId()); - Map> needRunPartitions = null; - String colNames = jobInfo.colName; + List> needRunColumns = null; if (table.needReAnalyzeTable(tblStats)) { - needRunPartitions = table.findReAnalyzeNeededPartitions(); + needRunColumns = table.getColumnIndexPairs(table.getSchemaAllIndexes(false) + .stream().map(Column::getName).collect(Collectors.toSet())); } else if (table instanceof OlapTable && tblStats.newPartitionLoaded.get()) { OlapTable olapTable = (OlapTable) table; - needRunPartitions = new HashMap<>(); - Set partitionColumnNames = olapTable.getPartitionInfo().getPartitionColumns().stream() - .map(Column::getName).collect(Collectors.toSet()); - colNames = partitionColumnNames.stream().collect(Collectors.joining(",")); Set partitionNames = olapTable.getAllPartitions().stream() .map(Partition::getName).collect(Collectors.toSet()); - for (String column : partitionColumnNames) { - needRunPartitions.put(column, partitionNames); - } + needRunColumns = olapTable.getColumnIndexPairs(partitionNames); } - if (needRunPartitions == null || needRunPartitions.isEmpty()) { + if (needRunColumns == null || needRunColumns.isEmpty()) { return null; } - return new AnalysisInfoBuilder(jobInfo).setColName(colNames).setColToPartitions(needRunPartitions).build(); + StringJoiner stringJoiner = new StringJoiner(",", "[", "]"); + for (Pair pair : needRunColumns) { + stringJoiner.add(pair.toString()); + } + return new AnalysisInfoBuilder(jobInfo) + .setColName(stringJoiner.toString()).setJobColumns(needRunColumns).build(); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsCollector.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsCollector.java index 0985b9b2b9539e..ec187fe893af49 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsCollector.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsCollector.java @@ -61,7 +61,7 @@ protected void runAfterCatalogReady() { @VisibleForTesting protected void createSystemAnalysisJob(AnalysisInfo jobInfo) throws DdlException { - if (jobInfo.colToPartitions.isEmpty()) { + if (jobInfo.jobColumns.isEmpty()) { // No statistics need to be collected or updated return; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsRepository.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsRepository.java index 5ac9b7305c77fd..5caa5bd9751c1e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsRepository.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsRepository.java @@ -29,6 +29,7 @@ import org.apache.doris.statistics.util.DBObjects; import org.apache.doris.statistics.util.StatisticsUtil; +import com.google.common.collect.Lists; import com.google.common.collect.Maps; import org.apache.commons.text.StringSubstitutor; import org.apache.logging.log4j.LogManager; @@ -320,7 +321,7 @@ public static void alterColumnStatistics(AlterColumnStatsStmt alterColumnStatsSt AnalysisInfo mockedJobInfo = new AnalysisInfoBuilder() .setTblUpdateTime(System.currentTimeMillis()) .setColName("") - .setColToPartitions(Maps.newHashMap()) + .setJobColumns(Lists.newArrayList()) .setUserInject(true) .setJobType(AnalysisInfo.JobType.MANUAL) .build(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/TableStatsMeta.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/TableStatsMeta.java index 9231c6a2bc7cd1..3b9b1e2bead005 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/TableStatsMeta.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/TableStatsMeta.java @@ -21,11 +21,11 @@ import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.PartitionInfo; import org.apache.doris.catalog.TableIf; +import org.apache.doris.common.Pair; 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.statistics.AnalysisInfo.JobType; -import org.apache.doris.statistics.util.StatisticsUtil; import com.google.common.annotations.VisibleForTesting; import com.google.gson.annotations.SerializedName; @@ -33,8 +33,6 @@ import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; -import java.util.Arrays; -import java.util.List; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; @@ -64,7 +62,11 @@ public class TableStatsMeta implements Writable { public long updatedTime; @SerializedName("colNameToColStatsMeta") - private ConcurrentMap colNameToColStatsMeta = new ConcurrentHashMap<>(); + private ConcurrentMap deprecatedColNameToColStatsMeta = new ConcurrentHashMap<>(); + + @SerializedName("colToColStatsMeta") + // -> ColStatsMeta + private ConcurrentMap, ColStatsMeta> colToColStatsMeta = new ConcurrentHashMap<>(); @SerializedName("trigger") public JobType jobType; @@ -100,52 +102,34 @@ public static TableStatsMeta read(DataInput dataInput) throws IOException { String json = Text.readString(dataInput); TableStatsMeta tableStats = GsonUtils.GSON.fromJson(json, TableStatsMeta.class); // Might be null counterintuitively, for compatible - if (tableStats.colNameToColStatsMeta == null) { - tableStats.colNameToColStatsMeta = new ConcurrentHashMap<>(); + if (tableStats.colToColStatsMeta == null) { + tableStats.colToColStatsMeta = new ConcurrentHashMap<>(); } - return tableStats; - } - - public long findColumnLastUpdateTime(String colName) { - ColStatsMeta colStatsMeta = colNameToColStatsMeta.get(colName); - if (colStatsMeta == null) { - return 0; + if (tableStats.deprecatedColNameToColStatsMeta != null) { + tableStats.convertDeprecatedColStatsToNewVersion(); } - return colStatsMeta.updatedTime; - } - - public ColStatsMeta findColumnStatsMeta(String colName) { - return colNameToColStatsMeta.get(colName); + return tableStats; } - public void removeColumn(String colName) { - colNameToColStatsMeta.remove(colName); + public ColStatsMeta findColumnStatsMeta(String indexName, String colName) { + return colToColStatsMeta.get(Pair.of(indexName, colName)); } - public Set analyzeColumns() { - return colNameToColStatsMeta.keySet(); + public void removeColumn(String indexName, String colName) { + colToColStatsMeta.remove(Pair.of(indexName, colName)); } - public void reset() { - updatedTime = 0; - colNameToColStatsMeta.values().forEach(ColStatsMeta::clear); + public Set> analyzeColumns() { + return colToColStatsMeta.keySet(); } public void update(AnalysisInfo analyzedJob, TableIf tableIf) { updatedTime = analyzedJob.tblUpdateTime; userInjected = analyzedJob.userInject; - String colNameStr = analyzedJob.colName; - // colName field AnalyzeJob's format likes: "[col1, col2]", we need to remove brackets here - // TODO: Refactor this later - if (analyzedJob.colName.startsWith("[") && analyzedJob.colName.endsWith("]")) { - colNameStr = colNameStr.substring(1, colNameStr.length() - 1); - } - List cols = Arrays.stream(colNameStr.split(",")) - .map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toList()); - for (String col : cols) { - ColStatsMeta colStatsMeta = colNameToColStatsMeta.get(col); + for (Pair colPair : analyzedJob.jobColumns) { + ColStatsMeta colStatsMeta = colToColStatsMeta.get(colPair); if (colStatsMeta == null) { - colNameToColStatsMeta.put(col, new ColStatsMeta(updatedTime, + colToColStatsMeta.put(colPair, new ColStatsMeta(updatedTime, analyzedJob.analysisMethod, analyzedJob.analysisType, analyzedJob.jobType, 0)); } else { colStatsMeta.updatedTime = updatedTime; @@ -159,21 +143,27 @@ public void update(AnalysisInfo analyzedJob, TableIf tableIf) { if (tableIf instanceof OlapTable) { rowCount = analyzedJob.emptyJob ? 0 : tableIf.getRowCount(); } - if (!analyzedJob.emptyJob && analyzedJob.colToPartitions.keySet() - .containsAll(tableIf.getBaseSchema().stream() - .filter(c -> !StatisticsUtil.isUnsupportedType(c.getType())) - .map(Column::getName).collect(Collectors.toSet()))) { + if (analyzedJob.emptyJob) { + return; + } + if (analyzedJob.jobColumns.containsAll( + tableIf.getColumnIndexPairs( + tableIf.getSchemaAllIndexes(false).stream().map(Column::getName).collect(Collectors.toSet())))) { updatedRows.set(0); newPartitionLoaded.set(false); } if (tableIf instanceof OlapTable) { PartitionInfo partitionInfo = ((OlapTable) tableIf).getPartitionInfo(); - if (partitionInfo != null && analyzedJob.colToPartitions.keySet() - .containsAll(partitionInfo.getPartitionColumns().stream() - .map(Column::getName).collect(Collectors.toSet()))) { + if (partitionInfo != null && analyzedJob.jobColumns + .containsAll(tableIf.getColumnIndexPairs(partitionInfo.getPartitionColumns().stream() + .map(Column::getName).collect(Collectors.toSet())))) { newPartitionLoaded.set(false); } } } } + + public void convertDeprecatedColStatsToNewVersion() { + deprecatedColNameToColStatsMeta = null; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataGenerator.java b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataGenerator.java index c84939ffbe63ae..ba727063712c23 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataGenerator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataGenerator.java @@ -19,6 +19,7 @@ import org.apache.doris.analysis.UserIdentity; 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.SchemaTable; @@ -38,6 +39,9 @@ import org.apache.doris.job.task.AbstractTask; import org.apache.doris.mtmv.MTMVPartitionUtil; import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.plsql.metastore.PlsqlManager; +import org.apache.doris.plsql.metastore.PlsqlProcedureKey; +import org.apache.doris.plsql.metastore.PlsqlStoredProcedure; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.QeProcessorImpl; import org.apache.doris.qe.QeProcessorImpl.QueryInfo; @@ -92,6 +96,8 @@ public class MetadataGenerator { private static final ImmutableMap WORKLOAD_GROUPS_COLUMN_TO_INDEX; + private static final ImmutableMap ROUTINE_INFO_COLUMN_TO_INDEX; + static { ImmutableMap.Builder activeQueriesbuilder = new ImmutableMap.Builder(); List activeQueriesColList = SchemaTable.TABLE_MAP.get("active_queries").getFullSchema(); @@ -105,6 +111,12 @@ public class MetadataGenerator { workloadGroupBuilder.put(WorkloadGroupMgr.WORKLOAD_GROUP_PROC_NODE_TITLE_NAMES.get(i).toLowerCase(), i); } WORKLOAD_GROUPS_COLUMN_TO_INDEX = workloadGroupBuilder.build(); + + ImmutableMap.Builder routineInfoBuilder = new ImmutableMap.Builder(); + for (int i = 0; i < PlsqlManager.ROUTINE_INFO_TITLE_NAMES.size(); i++) { + routineInfoBuilder.put(PlsqlManager.ROUTINE_INFO_TITLE_NAMES.get(i).toLowerCase(), i); + } + ROUTINE_INFO_COLUMN_TO_INDEX = routineInfoBuilder.build(); } public static TFetchSchemaTableDataResult getMetadataTable(TFetchSchemaTableDataRequest request) throws TException { @@ -167,6 +179,10 @@ public static TFetchSchemaTableDataResult getSchemaTableData(TFetchSchemaTableDa result = workloadGroupsMetadataResult(schemaTableParams); columnIndex = WORKLOAD_GROUPS_COLUMN_TO_INDEX; break; + case ROUTINES_INFO: + result = routineInfoMetadataResult(schemaTableParams); + columnIndex = ROUTINE_INFO_COLUMN_TO_INDEX; + break; default: return errorResult("invalid schema table name."); } @@ -189,7 +205,7 @@ private static TFetchSchemaTableDataResult icebergMetadataResult(TMetadataTableR return errorResult("Iceberg metadata params is not set."); } - TIcebergMetadataParams icebergMetadataParams = params.getIcebergMetadataParams(); + TIcebergMetadataParams icebergMetadataParams = params.getIcebergMetadataParams(); TIcebergQueryType icebergQueryType = icebergMetadataParams.getIcebergQueryType(); IcebergMetadataCache icebergMetadataCache = Env.getCurrentEnv().getExtMetaCacheMgr().getIcebergMetadataCache(); List dataBatch = Lists.newArrayList(); @@ -376,7 +392,7 @@ private static TFetchSchemaTableDataResult frontendsDisksMetadataResult(TMetadat private static TFetchSchemaTableDataResult catalogsMetadataResult(TMetadataTableRequestParams params) { TFetchSchemaTableDataResult result = new TFetchSchemaTableDataResult(); - List info = Env.getCurrentEnv().getCatalogMgr().listCatalogs(); + List info = Env.getCurrentEnv().getCatalogMgr().listCatalogs(); List dataBatch = Lists.newArrayList(); for (CatalogIf catalog : info) { @@ -417,24 +433,25 @@ private static TFetchSchemaTableDataResult workloadGroupsMetadataResult(TSchemaT List dataBatch = Lists.newArrayList(); for (List rGroupsInfo : workloadGroupsInfo) { TRow trow = new TRow(); - trow.addToColumnValue(new TCell().setLongVal(Long.valueOf(rGroupsInfo.get(0)))); // id - trow.addToColumnValue(new TCell().setStringVal(rGroupsInfo.get(1))); // name + trow.addToColumnValue(new TCell().setLongVal(Long.valueOf(rGroupsInfo.get(0)))); // id + trow.addToColumnValue(new TCell().setStringVal(rGroupsInfo.get(1))); // name trow.addToColumnValue(new TCell().setLongVal(Long.valueOf(rGroupsInfo.get(2)))); // cpu_share - trow.addToColumnValue(new TCell().setStringVal(rGroupsInfo.get(3))); // mem_limit - trow.addToColumnValue(new TCell().setStringVal(rGroupsInfo.get(4))); // mem overcommit + trow.addToColumnValue(new TCell().setStringVal(rGroupsInfo.get(3))); // mem_limit + trow.addToColumnValue(new TCell().setStringVal(rGroupsInfo.get(4))); // mem overcommit trow.addToColumnValue(new TCell().setLongVal(Long.valueOf(rGroupsInfo.get(5)))); // max concurrent trow.addToColumnValue(new TCell().setLongVal(Long.valueOf(rGroupsInfo.get(6)))); // max queue size trow.addToColumnValue(new TCell().setLongVal(Long.valueOf(rGroupsInfo.get(7)))); // queue timeout - trow.addToColumnValue(new TCell().setStringVal(rGroupsInfo.get(8))); // cpu hard limit + trow.addToColumnValue(new TCell().setStringVal(rGroupsInfo.get(8))); // cpu hard limit trow.addToColumnValue(new TCell().setLongVal(Long.valueOf(rGroupsInfo.get(9)))); // scan thread num // max remote scan thread num trow.addToColumnValue(new TCell().setLongVal(Long.valueOf(rGroupsInfo.get(10)))); // min remote scan thread num trow.addToColumnValue(new TCell().setLongVal(Long.valueOf(rGroupsInfo.get(11)))); - trow.addToColumnValue(new TCell().setStringVal(rGroupsInfo.get(12))); - trow.addToColumnValue(new TCell().setStringVal(rGroupsInfo.get(13))); - trow.addToColumnValue(new TCell().setStringVal(rGroupsInfo.get(14))); - trow.addToColumnValue(new TCell().setStringVal(rGroupsInfo.get(15))); + trow.addToColumnValue(new TCell().setStringVal(rGroupsInfo.get(12))); // spill low watermark + trow.addToColumnValue(new TCell().setStringVal(rGroupsInfo.get(13))); // spill high watermark + trow.addToColumnValue(new TCell().setStringVal(rGroupsInfo.get(14))); // tag + trow.addToColumnValue(new TCell().setStringVal(rGroupsInfo.get(15))); // running query num + trow.addToColumnValue(new TCell().setStringVal(rGroupsInfo.get(16))); // waiting query num dataBatch.add(trow); } @@ -455,11 +472,11 @@ private static TFetchSchemaTableDataResult workloadSchedPolicyMetadataResult(TMe List dataBatch = Lists.newArrayList(); for (List policyRow : workloadPolicyList) { TRow trow = new TRow(); - trow.addToColumnValue(new TCell().setLongVal(Long.valueOf(policyRow.get(0)))); // id - trow.addToColumnValue(new TCell().setStringVal(policyRow.get(1))); // name - trow.addToColumnValue(new TCell().setStringVal(policyRow.get(2))); // condition - trow.addToColumnValue(new TCell().setStringVal(policyRow.get(3))); // action - trow.addToColumnValue(new TCell().setIntVal(Integer.valueOf(policyRow.get(4)))); // priority + trow.addToColumnValue(new TCell().setLongVal(Long.valueOf(policyRow.get(0)))); // id + trow.addToColumnValue(new TCell().setStringVal(policyRow.get(1))); // name + trow.addToColumnValue(new TCell().setStringVal(policyRow.get(2))); // condition + trow.addToColumnValue(new TCell().setStringVal(policyRow.get(3))); // action + trow.addToColumnValue(new TCell().setIntVal(Integer.valueOf(policyRow.get(4)))); // priority trow.addToColumnValue(new TCell().setBoolVal(Boolean.valueOf(policyRow.get(5)))); // enabled trow.addToColumnValue(new TCell().setIntVal(Integer.valueOf(policyRow.get(6)))); // version dataBatch.add(trow); @@ -552,7 +569,7 @@ private static TFetchSchemaTableDataResult queriesMetadataResult(TSchemaTableReq List relayResults = forwardToOtherFrontends(replayFetchSchemaTableReq); relayResults .forEach(rs -> rs.getDataBatch() - .forEach(row -> dataBatch.add(row))); + .forEach(row -> dataBatch.add(row))); } result.setDataBatch(dataBatch); @@ -753,4 +770,58 @@ private static TFetchSchemaTableDataResult taskMetadataResult(TMetadataTableRequ result.setStatus(new TStatus(TStatusCode.OK)); return result; } + + private static TFetchSchemaTableDataResult routineInfoMetadataResult(TSchemaTableRequestParams params) { + if (!params.isSetCurrentUserIdent()) { + return errorResult("current user ident is not set."); + } + + PlsqlManager plSqlClient = Env.getCurrentEnv().getPlsqlManager(); + + TFetchSchemaTableDataResult result = new TFetchSchemaTableDataResult(); + List dataBatch = Lists.newArrayList(); + + Map allProc = plSqlClient.getAllPlsqlStoredProcedures(); + for (Map.Entry entry : allProc.entrySet()) { + PlsqlStoredProcedure proc = entry.getValue(); + TRow trow = new TRow(); + trow.addToColumnValue(new TCell().setStringVal(proc.getName())); // SPECIFIC_NAME + trow.addToColumnValue(new TCell().setStringVal(Long.toString(proc.getCatalogId()))); // ROUTINE_CATALOG + CatalogIf catalog = Env.getCurrentEnv().getCatalogMgr().getCatalog(proc.getCatalogId()); + if (catalog != null) { + DatabaseIf db = catalog.getDbNullable(proc.getDbId()); + if (db != null) { + trow.addToColumnValue(new TCell().setStringVal(db.getFullName())); // ROUTINE_SCHEMA + } else { + trow.addToColumnValue(new TCell().setStringVal("")); // ROUTINE_SCHEMA + } + } else { + trow.addToColumnValue(new TCell().setStringVal("")); // ROUTINE_SCHEMA + } + trow.addToColumnValue(new TCell().setStringVal(proc.getName())); // ROUTINE_NAME + trow.addToColumnValue(new TCell().setStringVal("PROCEDURE")); // ROUTINE_TYPE + trow.addToColumnValue(new TCell().setStringVal("")); // DTD_IDENTIFIER + trow.addToColumnValue(new TCell().setStringVal(proc.getSource())); // ROUTINE_BODY + trow.addToColumnValue(new TCell().setStringVal("")); // ROUTINE_DEFINITION + trow.addToColumnValue(new TCell().setStringVal("NULL")); // EXTERNAL_NAME + trow.addToColumnValue(new TCell().setStringVal("")); // EXTERNAL_LANGUAGE + trow.addToColumnValue(new TCell().setStringVal("SQL")); // PARAMETER_STYLE + trow.addToColumnValue(new TCell().setStringVal("")); // IS_DETERMINISTIC + trow.addToColumnValue(new TCell().setStringVal("")); // SQL_DATA_ACCESS + trow.addToColumnValue(new TCell().setStringVal("NULL")); // SQL_PATH + trow.addToColumnValue(new TCell().setStringVal("DEFINER")); // SECURITY_TYPE + trow.addToColumnValue(new TCell().setStringVal(proc.getCreateTime())); // CREATED + trow.addToColumnValue(new TCell().setStringVal(proc.getModifyTime())); // LAST_ALTERED + trow.addToColumnValue(new TCell().setStringVal("")); // SQ_MODE + trow.addToColumnValue(new TCell().setStringVal("")); // ROUTINE_COMMENT + trow.addToColumnValue(new TCell().setStringVal(proc.getOwnerName())); // DEFINER + trow.addToColumnValue(new TCell().setStringVal("")); // CHARACTER_SET_CLIENT + trow.addToColumnValue(new TCell().setStringVal("")); // COLLATION_CONNECTION + trow.addToColumnValue(new TCell().setStringVal("")); // DATABASE_COLLATION + dataBatch.add(trow); + } + result.setDataBatch(dataBatch); + result.setStatus(new TStatus(TStatusCode.OK)); + return result; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/transaction/PublishVersionDaemon.java b/fe/fe-core/src/main/java/org/apache/doris/transaction/PublishVersionDaemon.java index f70f70595553dc..2ceebe2b53c128 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/transaction/PublishVersionDaemon.java +++ b/fe/fe-core/src/main/java/org/apache/doris/transaction/PublishVersionDaemon.java @@ -23,6 +23,7 @@ import org.apache.doris.catalog.Partition; import org.apache.doris.common.Config; import org.apache.doris.common.MetaNotFoundException; +import org.apache.doris.common.Pair; import org.apache.doris.common.util.DebugPointUtil; import org.apache.doris.common.util.MasterDaemon; import org.apache.doris.metric.MetricRepo; @@ -45,7 +46,6 @@ import java.util.Collections; import java.util.List; import java.util.Map; -import java.util.Map.Entry; import java.util.Set; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -225,8 +225,8 @@ private Map> getBaseTabletIdsForEachBe(TransactionState transact .flatMap(Collection::stream) .flatMap(tablet -> tablet.getBackendIds() - .stream().map(backendId -> Map.entry(backendId, tablet.getId()))) - .collect(Collectors.groupingBy(Entry::getKey, - Collectors.mapping(Entry::getValue, Collectors.toSet()))); + .stream().map(backendId -> Pair.of(backendId, tablet.getId()))) + .collect(Collectors.groupingBy(p -> p.first, + Collectors.mapping(p -> p.second, Collectors.toSet()))); } } diff --git a/fe/fe-core/src/main/jflex/sql_scanner.flex b/fe/fe-core/src/main/jflex/sql_scanner.flex index cd496a9fff3292..b931bda9e92fe2 100644 --- a/fe/fe-core/src/main/jflex/sql_scanner.flex +++ b/fe/fe-core/src/main/jflex/sql_scanner.flex @@ -505,6 +505,7 @@ import org.apache.doris.qe.SqlModeHelper; keywordMap.put("verbose", new Integer(SqlParserSymbols.KW_VERBOSE)); keywordMap.put("version", new Integer(SqlParserSymbols.KW_VERSION)); keywordMap.put("view", new Integer(SqlParserSymbols.KW_VIEW)); + keywordMap.put("views", new Integer(SqlParserSymbols.KW_VIEWS)); keywordMap.put("warnings", new Integer(SqlParserSymbols.KW_WARNINGS)); keywordMap.put("week", new Integer(SqlParserSymbols.KW_WEEK)); keywordMap.put("when", new Integer(SqlParserSymbols.KW_WHEN)); diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/GrantStmtTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/GrantStmtTest.java index 382e9348b79550..05451e44b25316 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/analysis/GrantStmtTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/GrantStmtTest.java @@ -108,8 +108,8 @@ public void testResourceNormal() throws UserException { stmt = new GrantStmt(new UserIdentity("testUser", "%"), null, new ResourcePattern("*", ResourceTypeEnum.GENERAL), privileges, ResourceTypeEnum.GENERAL); stmt.analyze(analyzer); - Assert.assertEquals(Auth.PrivLevel.GLOBAL, stmt.getResourcePattern().getPrivLevel()); - Assert.assertEquals("GRANT Usage_priv ON RESOURCE '*' TO 'testUser'@'%'", stmt.toSql()); + Assert.assertEquals(Auth.PrivLevel.RESOURCE, stmt.getResourcePattern().getPrivLevel()); + Assert.assertEquals("GRANT Usage_priv ON RESOURCE '%' TO 'testUser'@'%'", stmt.toSql()); } @Test(expected = AnalysisException.class) diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowReplicaTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowReplicaTest.java index be913cbad26048..21fe967a96dd68 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowReplicaTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowReplicaTest.java @@ -64,7 +64,7 @@ public void testShowReplicaDistribution() throws Exception { executor = new ShowExecutor(connectContext, skewStmt); resultSet = executor.execute(); Assert.assertEquals(10, resultSet.getResultRows().size()); - Assert.assertEquals(5, resultSet.getResultRows().get(0).size()); + Assert.assertEquals(6, resultSet.getResultRows().get(0).size()); // update tablets' data size and row count Database db = Env.getCurrentInternalCatalog().getDbOrAnalysisException("test"); @@ -87,8 +87,8 @@ public void testShowReplicaDistribution() throws Exception { executor = new ShowExecutor(connectContext, skewStmt); resultSet = executor.execute(); Assert.assertEquals(10, resultSet.getResultRows().size()); - Assert.assertEquals("4", resultSet.getResultRows().get(4).get(0)); - Assert.assertEquals(5, resultSet.getResultRows().get(0).size()); + Assert.assertEquals("4", resultSet.getResultRows().get(4).get(1)); + Assert.assertEquals(6, resultSet.getResultRows().get(0).size()); } @Test diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowTableStmtTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowTableStmtTest.java index cd72b41bafe689..5c0015fd03a5a8 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowTableStmtTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowTableStmtTest.java @@ -17,6 +17,7 @@ package org.apache.doris.analysis; +import org.apache.doris.catalog.TableIf.TableType; import org.apache.doris.common.AnalysisException; import org.apache.doris.mysql.privilege.AccessControllerManager; import org.apache.doris.mysql.privilege.MockedAuth; @@ -68,6 +69,28 @@ public void testNormal() throws AnalysisException { Assert.assertEquals("Table_type", stmt.getMetaData().getColumn(1).getName()); } + @Test + public void testShowViews() throws AnalysisException { + ShowTableStmt stmt = new ShowTableStmt("", null, false, TableType.VIEW, + null, null); + stmt.analyze(analyzer); + Assert.assertEquals("SHOW VIEWS FROM internal.testDb", stmt.toString()); + Assert.assertEquals("testDb", stmt.getDb()); + Assert.assertEquals(TableType.VIEW, stmt.getType()); + Assert.assertFalse(stmt.isVerbose()); + Assert.assertEquals(1, stmt.getMetaData().getColumnCount()); + Assert.assertEquals("Tables_in_testDb", stmt.getMetaData().getColumn(0).getName()); + + stmt = new ShowTableStmt("abc", null, true, TableType.VIEW, "bcd", null); + stmt.analyze(analyzer); + Assert.assertEquals("bcd", stmt.getPattern()); + Assert.assertEquals("SHOW FULL VIEWS FROM internal.abc LIKE 'bcd'", stmt.toString()); + Assert.assertEquals(4, stmt.getMetaData().getColumnCount()); + Assert.assertEquals("Tables_in_abc", stmt.getMetaData().getColumn(0).getName()); + Assert.assertEquals("Table_type", stmt.getMetaData().getColumn(1).getName()); + Assert.assertEquals(TableType.VIEW, stmt.getType()); + } + @Test public void testNoDb() { ShowTableStmt stmt = new ShowTableStmt("", null, false, null); diff --git a/fe/fe-core/src/test/java/org/apache/doris/mysql/privilege/AuthTest.java b/fe/fe-core/src/test/java/org/apache/doris/mysql/privilege/AuthTest.java index 5ed4c23d470948..ffd5e21ca96228 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/mysql/privilege/AuthTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/mysql/privilege/AuthTest.java @@ -18,6 +18,7 @@ package org.apache.doris.mysql.privilege; import org.apache.doris.analysis.Analyzer; +import org.apache.doris.analysis.CompoundPredicate.Operator; import org.apache.doris.analysis.CreateRoleStmt; import org.apache.doris.analysis.CreateUserStmt; import org.apache.doris.analysis.DropRoleStmt; @@ -1677,15 +1678,17 @@ public void testGrantRole() { } @Test - public void testResource() { + public void testResource() throws UserException { UserIdentity userIdentity = new UserIdentity("testUser", "%"); String role = "role0"; String resourceName = "spark0"; ResourcePattern resourcePattern = new ResourcePattern(resourceName, ResourceTypeEnum.GENERAL); - String anyResource = "*"; + String anyResource = "%"; ResourcePattern anyResourcePattern = new ResourcePattern(anyResource, ResourceTypeEnum.GENERAL); List usagePrivileges = Lists .newArrayList(new AccessPrivilegeWithCols(AccessPrivilege.USAGE_PRIV)); + List grantPrivileges = Lists + .newArrayList(new AccessPrivilegeWithCols(AccessPrivilege.GRANT_PRIV)); UserDesc userDesc = new UserDesc(userIdentity, "12345", true); // ------ grant|revoke resource to|from user ------ @@ -1833,8 +1836,9 @@ public void testResource() { Assert.fail(); } Assert.assertTrue(accessManager.checkResourcePriv(userIdentity, resourceName, PrivPredicate.USAGE)); - Assert.assertTrue(accessManager.checkGlobalPriv(userIdentity, PrivPredicate.USAGE)); - Assert.assertTrue(accessManager.checkGlobalPriv(userIdentity, PrivPredicate.SHOW_RESOURCES)); + // anyResource not belong to global auth + Assert.assertFalse(accessManager.checkGlobalPriv(userIdentity, PrivPredicate.USAGE)); + Assert.assertFalse(accessManager.checkGlobalPriv(userIdentity, PrivPredicate.SHOW_RESOURCES)); Assert.assertFalse(accessManager.checkGlobalPriv(userIdentity, PrivPredicate.SHOW)); // 3. revoke usage_priv on resource '*' from 'testUser'@'%' @@ -1891,7 +1895,7 @@ public void testResource() { Assert.fail(); } Assert.assertTrue(accessManager.checkResourcePriv(userIdentity, resourceName, PrivPredicate.USAGE)); - Assert.assertTrue(accessManager.checkGlobalPriv(userIdentity, PrivPredicate.USAGE)); + Assert.assertFalse(accessManager.checkGlobalPriv(userIdentity, PrivPredicate.USAGE)); // 3. revoke usage_priv on resource '*' from role 'role0' revokeStmt = new RevokeStmt(null, role, anyResourcePattern, usagePrivileges, ResourceTypeEnum.GENERAL); @@ -1960,10 +1964,26 @@ public void testResource() { GrantStmt grantStmt3 = new GrantStmt(userIdentity, "test_role", tablePattern, usagePrivileges); ExceptionChecker.expectThrowsWithMsg(AnalysisException.class, "Can not grant/revoke USAGE_PRIV to/from database or table", () -> grantStmt3.analyze(analyzer)); + + // 4.drop user + dropUser(userIdentity); + + // -------- test anyPattern has usage_priv and g1 has grant_priv ---------- + // 1. create user with no role + createUser(userIdentity); + // 2. grant usage_priv on workload group '%' to user + grant(new GrantStmt(userIdentity, null, anyResourcePattern, usagePrivileges, ResourceTypeEnum.GENERAL)); + // 3.grant grant_priv on workload group g1 + grant(new GrantStmt(userIdentity, null, resourcePattern, grantPrivileges, ResourceTypeEnum.GENERAL)); + Assert.assertTrue(accessManager.checkResourcePriv(userIdentity, resourceName, + PrivPredicate.of(PrivBitSet.of(Privilege.USAGE_PRIV, Privilege.GRANT_PRIV), + Operator.AND))); + // 4. drop user + dropUser(userIdentity); } @Test - public void testWorkloadGroupPriv() { + public void testWorkloadGroupPriv() throws UserException { UserIdentity userIdentity = new UserIdentity("testUser", "%"); String role = "role0"; String workloadGroupName = "g1"; @@ -1972,6 +1992,8 @@ public void testWorkloadGroupPriv() { WorkloadGroupPattern anyWorkloadGroupPattern = new WorkloadGroupPattern(anyWorkloadGroup); List usagePrivileges = Lists .newArrayList(new AccessPrivilegeWithCols(AccessPrivilege.USAGE_PRIV)); + List grantPrivileges = Lists + .newArrayList(new AccessPrivilegeWithCols(AccessPrivilege.GRANT_PRIV)); UserDesc userDesc = new UserDesc(userIdentity, "12345", true); // ------ grant|revoke workload group to|from user ------ @@ -2244,6 +2266,27 @@ public void testWorkloadGroupPriv() { GrantStmt grantStmt3 = new GrantStmt(userIdentity, "test_role", tablePattern, usagePrivileges); ExceptionChecker.expectThrowsWithMsg(AnalysisException.class, "Can not grant/revoke USAGE_PRIV to/from database or table", () -> grantStmt3.analyze(analyzer)); + // 4.drop user + dropUser(userIdentity); + + // -------- test anyPattern has usage_priv and g1 has grant_priv ---------- + // 1. create user with no role + createUser(userIdentity); + // 2. grant usage_priv on workload group '%' to user + grant(new GrantStmt(userIdentity, null, anyWorkloadGroupPattern, usagePrivileges)); + // 3.grant grant_priv on workload group g1 + grant(new GrantStmt(userIdentity, null, workloadGroupPattern, grantPrivileges)); + Assert.assertTrue(accessManager.checkWorkloadGroupPriv(userIdentity, workloadGroupName, + PrivPredicate.of(PrivBitSet.of(Privilege.USAGE_PRIV, Privilege.GRANT_PRIV), + Operator.AND))); + // 4. drop user + dropUser(userIdentity); + } + + private void dropUser(UserIdentity userIdentity) throws UserException { + DropUserStmt dropUserStmt = new DropUserStmt(userIdentity); + dropUserStmt.analyze(analyzer); + auth.dropUser(dropUserStmt); } private void createUser(UserIdentity userIdentity) throws UserException { diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/postprocess/ColumnPruningPostProcessorTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/postprocess/ColumnPruningPostProcessorTest.java new file mode 100644 index 00000000000000..0e65aa6d582875 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/postprocess/ColumnPruningPostProcessorTest.java @@ -0,0 +1,62 @@ +// 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.postprocess; + +import org.apache.doris.nereids.processor.post.ColumnPruningPostProcessor; +import org.apache.doris.nereids.rules.rewrite.InferFilterNotNull; +import org.apache.doris.nereids.trees.plans.JoinType; +import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; +import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; +import org.apache.doris.nereids.trees.plans.physical.PhysicalNestedLoopJoin; +import org.apache.doris.nereids.trees.plans.physical.PhysicalPlan; +import org.apache.doris.nereids.trees.plans.physical.PhysicalProject; +import org.apache.doris.nereids.util.LogicalPlanBuilder; +import org.apache.doris.nereids.util.MemoPatternMatchSupported; +import org.apache.doris.nereids.util.MemoTestUtils; +import org.apache.doris.nereids.util.PlanChecker; +import org.apache.doris.nereids.util.PlanConstructor; + +import com.google.common.collect.ImmutableList; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +class ColumnPruningPostProcessorTest implements MemoPatternMatchSupported { + private final LogicalOlapScan scan1 = PlanConstructor.newLogicalOlapScan(0, "t1", 0); + private final LogicalOlapScan scan2 = PlanConstructor.newLogicalOlapScan(1, "t2", 0); + + @Test + void test() { + LogicalPlan plan = new LogicalPlanBuilder(scan1) + .join(scan2, JoinType.INNER_JOIN, ImmutableList.of()) + .project(ImmutableList.of(0, 2)) + .build(); + + PhysicalPlan physicalPlan = PlanChecker.from(MemoTestUtils.createConnectContext(), plan) + .applyTopDown(new InferFilterNotNull()) + .implement() + .getPhysicalPlan(); + + ColumnPruningPostProcessor processor = new ColumnPruningPostProcessor(); + PhysicalPlan newPlan = (PhysicalPlan) physicalPlan.accept(processor, null); + + Assertions.assertTrue(newPlan instanceof PhysicalProject); + Assertions.assertTrue(newPlan.child(0) instanceof PhysicalNestedLoopJoin); + Assertions.assertTrue(newPlan.child(0).child(0) instanceof PhysicalProject); + Assertions.assertTrue(newPlan.child(0).child(1) instanceof PhysicalProject); + } +} 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 02fb18edbf7d15..8416bd396b04fc 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 @@ -269,7 +269,7 @@ public void getRelatedTableInfoTestWithoutGroupNullTest() { Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan(); Optional relatedTableInfo = MaterializedViewUtils.getRelatedTableInfo("l_shipdate", rewrittenPlan); - Assertions.assertFalse(relatedTableInfo.isPresent()); + Assertions.assertTrue(relatedTableInfo.isPresent()); }); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/PlanVisitorTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/PlanVisitorTest.java index 775bc05e1639a5..23cfc1cc452d8c 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/PlanVisitorTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/PlanVisitorTest.java @@ -37,7 +37,9 @@ import org.junit.jupiter.api.Test; import java.util.ArrayList; +import java.util.HashSet; import java.util.List; +import java.util.Set; import java.util.stream.Collectors; /** @@ -78,9 +80,31 @@ protected void runBeforeAll() throws Exception { + "\"replication_num\" = \"1\"\n" + ");"); + createTable("CREATE TABLE `table3` (\n" + + " `c1` bigint(20) NULL,\n" + + " `c2` bigint(20) NULL,\n" + + " `c3` bigint(20) not NULL,\n" + + " `k4` bitmap BITMAP_UNION NULL,\n" + + " `k5` bitmap BITMAP_UNION NULL\n" + + ") ENGINE=OLAP\n" + + "AGGREGATE KEY(`c1`, `c2`, `c3`)\n" + + "COMMENT 'OLAP'\n" + + "DISTRIBUTED BY HASH(`c2`) BUCKETS 1\n" + + "PROPERTIES (\n" + + "\"replication_num\" = \"1\"\n" + + ");"); + createView("CREATE VIEW `view1` AS SELECT t1.*, random() FROM\n" + "`table1` t1 LEFT JOIN\n" + "`table2` t2 ON t1.c1 = t2.c1;"); + + createMvByNereids("create materialized view mv1 BUILD IMMEDIATE REFRESH COMPLETE ON MANUAL\n" + + "DISTRIBUTED BY RANDOM BUCKETS 1\n" + + "PROPERTIES ('replication_num' = '1') \n" + + "as " + + "select t1.c1, t3.c2 " + + "from table1 t1 " + + "inner join table3 t3 on t1.c1= t3.c2;"); } @Test @@ -97,18 +121,16 @@ public void test1() { Assertions.assertEquals(1, collectResult.size()); Assertions.assertTrue(collectResult.get(0) instanceof Random); // Check get tables - TableCollectorContext collectorContext = - new TableCollector.TableCollectorContext(Sets.newHashSet(TableType.OLAP)); + TableCollectorContext collectorContext = new TableCollector.TableCollectorContext( + Sets.newHashSet(TableType.OLAP), true); physicalPlan.accept(TableCollector.INSTANCE, collectorContext); - Assertions.assertEquals(3, collectorContext.getCollectedTables().size()); - List expectedTables = new ArrayList<>(); + Set expectedTables = new HashSet<>(); expectedTables.add("table1"); expectedTables.add("table2"); - expectedTables.add("table2"); Assertions.assertEquals( collectorContext.getCollectedTables().stream() .map(TableIf::getName) - .collect(Collectors.toList()), + .collect(Collectors.toSet()), expectedTables); }); } @@ -128,21 +150,83 @@ public void test2() { Assertions.assertTrue(collectResult.get(0) instanceof Uuid); Assertions.assertTrue(collectResult.get(1) instanceof Random); // Check get tables - TableCollectorContext collectorContext = - new TableCollector.TableCollectorContext(Sets.newHashSet(TableType.OLAP)); + TableCollectorContext collectorContext = new TableCollector.TableCollectorContext( + Sets.newHashSet(TableType.OLAP), true); physicalPlan.accept(TableCollector.INSTANCE, collectorContext); - Assertions.assertEquals(4, collectorContext.getCollectedTables().size()); - List expectedTables = new ArrayList<>(); + Set expectedTables = new HashSet<>(); expectedTables.add("table1"); expectedTables.add("table2"); + Assertions.assertEquals( + collectorContext.getCollectedTables().stream() + .map(TableIf::getName) + .collect(Collectors.toSet()), + expectedTables); + }); + } + + @Test + public void test3() throws Exception { + PlanChecker.from(connectContext) + .checkPlannerResult("SELECT mv1.*, uuid() FROM mv1 " + + "INNER JOIN view1 on mv1.c1 = view1.c2 " + + "LEFT SEMI JOIN table2 ON mv1.c1 = table2.c1 " + + "WHERE mv1.c1 IN (SELECT c1 FROM table2) OR mv1.c1 < 10", + nereidsPlanner -> { + PhysicalPlan physicalPlan = nereidsPlanner.getPhysicalPlan(); + List> collectResult = new ArrayList<>(); + // Check nondeterministic collect + physicalPlan.accept(NondeterministicFunctionCollector.INSTANCE, collectResult); + // Check get tables + TableCollectorContext collectorContext = new TableCollector.TableCollectorContext( + Sets.newHashSet(TableType.OLAP), true); + physicalPlan.accept(TableCollector.INSTANCE, collectorContext); + Set expectedTables = new HashSet<>(); + expectedTables.add("table1"); expectedTables.add("table2"); - expectedTables.add("table2"); + expectedTables.add("table3"); Assertions.assertEquals( collectorContext.getCollectedTables().stream() .map(TableIf::getName) - .collect(Collectors.toList()), + .collect(Collectors.toSet()), expectedTables); + + TableCollectorContext collectorContextWithNoExpand = + new TableCollector.TableCollectorContext(Sets.newHashSet(TableType.OLAP), + false); + physicalPlan.accept(TableCollector.INSTANCE, collectorContextWithNoExpand); + Set expectedTablesWithNoExpand = new HashSet<>(); + expectedTablesWithNoExpand.add("table1"); + expectedTablesWithNoExpand.add("table2"); + Assertions.assertEquals( + collectorContextWithNoExpand.getCollectedTables().stream() + .map(TableIf::getName) + .collect(Collectors.toSet()), + expectedTablesWithNoExpand); + + TableCollectorContext mvCollectorContext = new TableCollector.TableCollectorContext( + Sets.newHashSet(TableType.MATERIALIZED_VIEW), true); + physicalPlan.accept(TableCollector.INSTANCE, mvCollectorContext); + Set expectedMvs = new HashSet<>(); + expectedMvs.add("mv1"); + Assertions.assertEquals( + mvCollectorContext.getCollectedTables().stream() + .map(TableIf::getName) + .collect(Collectors.toSet()), + expectedMvs); + + TableCollectorContext mvCollectorContextWithNoExpand = + new TableCollector.TableCollectorContext( + Sets.newHashSet(TableType.MATERIALIZED_VIEW), false); + physicalPlan.accept(TableCollector.INSTANCE, mvCollectorContextWithNoExpand); + Set expectedMvsWithNoExpand = new HashSet<>(); + expectedMvsWithNoExpand.add("mv1"); + Assertions.assertEquals( + mvCollectorContextWithNoExpand.getCollectedTables().stream() + .map(TableIf::getName) + .collect(Collectors.toSet()), + expectedMvsWithNoExpand); }); + dropMvByNereids("drop materialized view mv1"); } @Test diff --git a/fe/fe-core/src/test/java/org/apache/doris/qe/ShowExecutorTest.java b/fe/fe-core/src/test/java/org/apache/doris/qe/ShowExecutorTest.java index 114159142dceae..5b8b374ad5a027 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/qe/ShowExecutorTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/qe/ShowExecutorTest.java @@ -324,6 +324,16 @@ public void testShowTable() throws AnalysisException { Assert.assertFalse(resultSet.next()); } + @Test + public void testShowViews() throws AnalysisException { + ShowTableStmt stmt = new ShowTableStmt("testDb", null, false, TableType.VIEW, + null, null); + ShowExecutor executor = new ShowExecutor(ctx, stmt); + ShowResultSet resultSet = executor.execute(); + + Assert.assertFalse(resultSet.next()); + } + @Test public void testShowTableFromCatalog() throws AnalysisException { ShowTableStmt stmt = new ShowTableStmt("testDb", "internal", false, null); diff --git a/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalysisManagerTest.java b/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalysisManagerTest.java index f8a77fe06db754..674456b0b46891 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalysisManagerTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalysisManagerTest.java @@ -27,6 +27,7 @@ import org.apache.doris.catalog.PrimitiveType; import org.apache.doris.common.Config; import org.apache.doris.common.DdlException; +import org.apache.doris.common.Pair; import org.apache.doris.statistics.AnalysisInfo.AnalysisType; import org.apache.doris.statistics.AnalysisInfo.JobType; import org.apache.doris.statistics.AnalysisInfo.ScheduleType; @@ -45,9 +46,9 @@ import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; // CHECKSTYLE OFF public class AnalysisManagerTest { @@ -109,7 +110,7 @@ public String toString() { // test build sync job @Test public void testBuildAndAssignJob1() throws Exception { - AnalysisInfo analysisInfo = new AnalysisInfoBuilder().setColToPartitions(new HashMap<>()).build(); + AnalysisInfo analysisInfo = new AnalysisInfoBuilder().setJobColumns(new ArrayList<>()).build(); new MockUp() { @Mock @@ -167,12 +168,7 @@ public void updateTableStats(AnalysisInfo jobInfo) { AnalysisManager analysisManager = new AnalysisManager(); Assertions.assertNull(analysisManager.buildAndAssignJob(analyzeTblStmt)); - analysisInfo.colToPartitions.put("c1", new HashSet() { - { - add("p1"); - add("p2"); - } - }); + analysisInfo.jobColumns.add(Pair.of("index1", "c1")); analysisManager.buildAndAssignJob(analyzeTblStmt); new Expectations() { { @@ -191,7 +187,7 @@ public void updateTableStats(AnalysisInfo jobInfo) { // test build async job @Test public void testBuildAndAssignJob2(@Injectable OlapAnalysisTask analysisTask) throws Exception { - AnalysisInfo analysisInfo = new AnalysisInfoBuilder().setColToPartitions(new HashMap<>()) + AnalysisInfo analysisInfo = new AnalysisInfoBuilder().setJobColumns(new ArrayList<>()) .setScheduleType(ScheduleType.PERIOD) .build(); new MockUp() { @@ -255,12 +251,7 @@ public void logCreateAnalysisJob(AnalysisInfo analysisJob) { } })); AnalysisManager analysisManager = new AnalysisManager(); - analysisInfo.colToPartitions.put("c1", new HashSet() { - { - add("p1"); - add("p2"); - } - }); + analysisInfo.jobColumns.add(Pair.of("index1", "c1")); analysisManager.buildAndAssignJob(analyzeTblStmt); new Expectations() { { @@ -274,15 +265,7 @@ public void logCreateAnalysisJob(AnalysisInfo analysisJob) { public void testReAnalyze() { new MockUp() { - int count = 0; - int[] rowCount = new int[]{100, 100, 200, 200, 1, 1}; - final Column c = new Column("col1", PrimitiveType.INT); - @Mock - public long getRowCount() { - return rowCount[count++]; - } - @Mock public List getBaseSchema() { return Lists.newArrayList(c); @@ -291,22 +274,52 @@ public List getBaseSchema() { @Mock public List getColumns() { return Lists.newArrayList(c); } + @Mock + public List> getColumnIndexPairs(Set columns) { + List> jobList = Lists.newArrayList(); + jobList.add(Pair.of("1", "1")); + jobList.add(Pair.of("2", "2")); + jobList.add(Pair.of("3", "3")); + return jobList; + } }; OlapTable olapTable = new OlapTable(); + List> jobList = Lists.newArrayList(); + jobList.add(Pair.of("1", "1")); + jobList.add(Pair.of("2", "2")); + TableStatsMeta stats0 = new TableStatsMeta( + 0, new AnalysisInfoBuilder().setJobColumns(jobList) + .setColName("col1").build(), olapTable); + Assertions.assertTrue(olapTable.needReAnalyzeTable(stats0)); + + new MockUp() { + int count = 0; + int[] rowCount = new int[]{100, 100, 200, 200, 1, 1}; + + @Mock + public long getRowCount() { + return rowCount[count++]; + } + @Mock + public List> getColumnIndexPairs(Set columns) { + List> jobList = Lists.newArrayList(); + return jobList; + } + }; TableStatsMeta stats1 = new TableStatsMeta( - 50, new AnalysisInfoBuilder().setColToPartitions(new HashMap<>()) + 50, new AnalysisInfoBuilder().setJobColumns(new ArrayList<>()) .setColName("col1").build(), olapTable); stats1.updatedRows.addAndGet(50); Assertions.assertTrue(olapTable.needReAnalyzeTable(stats1)); TableStatsMeta stats2 = new TableStatsMeta( 190, new AnalysisInfoBuilder() - .setColToPartitions(new HashMap<>()).setColName("col1").build(), olapTable); + .setJobColumns(new ArrayList<>()).setColName("col1").build(), olapTable); stats2.updatedRows.addAndGet(20); Assertions.assertFalse(olapTable.needReAnalyzeTable(stats2)); TableStatsMeta stats3 = new TableStatsMeta(0, new AnalysisInfoBuilder() - .setColToPartitions(new HashMap<>()).setEmptyJob(true).setColName("col1").build(), olapTable); + .setJobColumns(new ArrayList<>()).setEmptyJob(true).setColName("col1").build(), olapTable); Assertions.assertTrue(olapTable.needReAnalyzeTable(stats3)); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalysisTaskExecutorTest.java b/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalysisTaskExecutorTest.java index b17ba3e68dbd7a..5698f0e9b20e63 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalysisTaskExecutorTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalysisTaskExecutorTest.java @@ -22,6 +22,7 @@ import org.apache.doris.catalog.InternalSchemaInitializer; import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.PrimitiveType; +import org.apache.doris.common.Pair; import org.apache.doris.common.jmockit.Deencapsulation; import org.apache.doris.datasource.InternalCatalog; import org.apache.doris.qe.StmtExecutor; @@ -33,7 +34,7 @@ import org.apache.doris.statistics.util.StatisticsUtil; import org.apache.doris.utframe.TestWithFeService; -import com.google.common.collect.Maps; +import com.google.common.collect.Lists; import mockit.Mock; import mockit.MockUp; import mockit.Mocked; @@ -41,10 +42,8 @@ import org.junit.jupiter.api.Test; import java.util.Collections; -import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.concurrent.BlockingQueue; import java.util.concurrent.atomic.AtomicBoolean; @@ -158,8 +157,8 @@ public void syncLoadColStats(long tableId, long idxId, String colName) { }; AnalysisTaskExecutor analysisTaskExecutor = new AnalysisTaskExecutor(1); - HashMap> colToPartitions = Maps.newHashMap(); - colToPartitions.put("col1", Collections.singleton("t1")); + List> columns = Lists.newArrayList(); + columns.add(Pair.of("col1", "t1")); AnalysisInfo analysisInfo = new AnalysisInfoBuilder().setJobId(0).setTaskId(0) .setCatalogId(0).setDBId(0).setTblId(0) .setColName("col1").setJobType(JobType.MANUAL) @@ -167,7 +166,7 @@ public void syncLoadColStats(long tableId, long idxId, String colName) { .setAnalysisMethod(AnalysisMethod.FULL) .setAnalysisType(AnalysisType.FUNDAMENTALS) .setState(AnalysisState.RUNNING) - .setColToPartitions(colToPartitions) + .setJobColumns(columns) .build(); OlapAnalysisTask task = new OlapAnalysisTask(analysisInfo); diff --git a/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalyzeTest.java b/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalyzeTest.java index 483cd3c03262c2..bf6ce32e155f42 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalyzeTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalyzeTest.java @@ -23,6 +23,7 @@ import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.PrimitiveType; import org.apache.doris.common.FeConstants; +import org.apache.doris.common.Pair; import org.apache.doris.datasource.InternalCatalog; import org.apache.doris.qe.AutoCloseConnectContext; import org.apache.doris.qe.ConnectContext; @@ -35,7 +36,7 @@ import org.apache.doris.statistics.util.StatisticsUtil; import org.apache.doris.utframe.TestWithFeService; -import com.google.common.collect.Maps; +import com.google.common.collect.Lists; import mockit.Expectations; import mockit.Mock; import mockit.MockUp; @@ -45,10 +46,8 @@ import java.util.ArrayList; import java.util.Collections; -import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Set; public class AnalyzeTest extends TestWithFeService { @@ -160,8 +159,8 @@ public void execSQLs(List partitionAnalysisSQLs, Map par @Mock protected void runQuery(String sql) {} }; - HashMap> colToPartitions = Maps.newHashMap(); - colToPartitions.put("col1", Collections.singleton("t1")); + List> colList = Lists.newArrayList(); + colList.add(Pair.of("col1", "index1")); AnalysisInfo analysisJobInfo = new AnalysisInfoBuilder().setJobId(0).setTaskId(0) .setCatalogId(0) .setDBId(0) @@ -170,7 +169,7 @@ protected void runQuery(String sql) {} .setAnalysisMode(AnalysisMode.FULL) .setAnalysisMethod(AnalysisMethod.FULL) .setAnalysisType(AnalysisType.FUNDAMENTALS) - .setColToPartitions(colToPartitions) + .setJobColumns(colList) .setState(AnalysisState.RUNNING) .build(); new OlapAnalysisTask(analysisJobInfo).doExecute(); diff --git a/fe/fe-core/src/test/java/org/apache/doris/statistics/HistogramTaskTest.java b/fe/fe-core/src/test/java/org/apache/doris/statistics/HistogramTaskTest.java index 4217fb5a0db397..09bf4dd94c6732 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/statistics/HistogramTaskTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/statistics/HistogramTaskTest.java @@ -90,7 +90,6 @@ public void test1TaskCreation() throws Exception { for (Entry infoEntry : taskInfo.entrySet()) { BaseAnalysisTask task = infoEntry.getValue(); - Assertions.assertEquals(AnalysisType.HISTOGRAM, task.info.analysisType); Assertions.assertEquals("col1", task.info.colName); } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/statistics/OlapAnalysisTaskTest.java b/fe/fe-core/src/test/java/org/apache/doris/statistics/OlapAnalysisTaskTest.java index 5d9d57406a3a10..75506b1c85a014 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/statistics/OlapAnalysisTaskTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/statistics/OlapAnalysisTaskTest.java @@ -159,11 +159,10 @@ public void runQuery(String sql) { + " IS NULL, `t1`.`count`, 0)), 0) * 5.0 as `null_count`, " + "SUBSTRING(CAST('1' AS STRING), 1, 1024) AS `min`," + " SUBSTRING(CAST('2' AS STRING), 1, 1024) AS `max`, " - + "SUM(LENGTH(`column_key`) * count) * 5.0 AS `data_size`, NOW() " + + "SUM(t1.count) * 4 * 5.0 AS `data_size`, NOW() " + "FROM ( SELECT t0.`${colName}` as `column_key`, COUNT(1) " - + "as `count` FROM (SELECT `${colName}` FROM " - + "`catalogName`.`${dbName}`.`${tblName}` " - + " limit 100) as `t0` GROUP BY `t0`.`${colName}` ) as `t1` ", sql); + + "as `count` FROM (SELECT `${colName}` FROM `catalogName`.`${dbName}`.`${tblName}`" + + " limit 100) as `t0` GROUP BY `t0`.`${colName}` ) as `t1` ", sql); return; } }; @@ -183,7 +182,7 @@ public Set getDistributionColumnNames() { }; OlapAnalysisTask olapAnalysisTask = new OlapAnalysisTask(); - olapAnalysisTask.col = new Column("test", PrimitiveType.STRING); + olapAnalysisTask.col = new Column("test", PrimitiveType.INT); olapAnalysisTask.tbl = tableIf; AnalysisInfoBuilder analysisInfoBuilder = new AnalysisInfoBuilder(); analysisInfoBuilder.setJobType(AnalysisInfo.JobType.MANUAL); @@ -322,7 +321,8 @@ public void runQuery(String sql) { + "SUBSTRING(CAST('1' AS STRING), 1, 1024) AS `min`, " + "SUBSTRING(CAST('2' AS STRING), 1, 1024) AS `max`, " + "SUM(LENGTH(`column_key`) * count) * 5.0 AS `data_size`, NOW() " - + "FROM ( SELECT t0.`${colName}` as `column_key`, COUNT(1) as `count` FROM (SELECT `${colName}` FROM `catalogName`.`${dbName}`.`${tblName}` limit 100) as `t0` GROUP BY `t0`.`${colName}` ) as `t1` ", sql); + + "FROM ( SELECT t0.`colValue` as `column_key`, COUNT(1) as `count` FROM " + + "(SELECT SUBSTRING(CAST(`${colName}` AS STRING), 1, 1024) AS `colValue` FROM `catalogName`.`${dbName}`.`${tblName}` limit 100) as `t0` GROUP BY `t0`.`colValue` ) as `t1` ", sql); return; } }; diff --git a/fe/fe-core/src/test/java/org/apache/doris/statistics/StatisticsAutoCollectorTest.java b/fe/fe-core/src/test/java/org/apache/doris/statistics/StatisticsAutoCollectorTest.java index 678e7580f8eefb..f7b75261cc54fa 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/statistics/StatisticsAutoCollectorTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/statistics/StatisticsAutoCollectorTest.java @@ -31,16 +31,12 @@ import org.apache.doris.common.Config; import org.apache.doris.common.DdlException; import org.apache.doris.common.FeConstants; +import org.apache.doris.common.Pair; import org.apache.doris.datasource.CatalogIf; import org.apache.doris.datasource.InternalCatalog; -import org.apache.doris.statistics.AnalysisInfo.AnalysisMethod; -import org.apache.doris.statistics.AnalysisInfo.AnalysisType; -import org.apache.doris.statistics.AnalysisInfo.JobType; import org.apache.doris.statistics.util.StatisticsUtil; import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import com.google.common.collect.Sets; import mockit.Expectations; import mockit.Injectable; import mockit.Mock; @@ -54,7 +50,6 @@ import java.util.Arrays; import java.util.Collection; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; @@ -144,97 +139,32 @@ public List getSchemaAllIndexes(boolean full) { StatisticsAutoCollector saa = new StatisticsAutoCollector(); List analysisInfoList = saa.constructAnalysisInfo(new Database(1, "anydb")); Assertions.assertEquals(1, analysisInfoList.size()); - Assertions.assertEquals("c1", analysisInfoList.get(0).colName.split(",")[0]); + Assertions.assertNull(analysisInfoList.get(0).colName); } @Test - public void testGetReAnalyzeRequiredPart0() { + public void testSkipWideTable() { TableIf tableIf = new OlapTable(); new MockUp() { - @Mock - protected Map> findReAnalyzeNeededPartitions() { - Set partitionNames = new HashSet<>(); - partitionNames.add("p1"); - partitionNames.add("p2"); - Map> map = new HashMap<>(); - map.put("col1", partitionNames); - return map; - } - - @Mock - public long getRowCount() { - return 100; - } - @Mock public List getBaseSchema() { return Lists.newArrayList(new Column("col1", Type.INT), new Column("col2", Type.INT)); } - }; - - new MockUp() { - @Mock - public TableIf findTable(long catalogName, long dbName, long tblName) { - return tableIf; - } - }; - AnalysisInfo analysisInfo = new AnalysisInfoBuilder().setAnalysisMethod(AnalysisMethod.FULL) - .setColToPartitions(new HashMap<>()).setAnalysisType( - AnalysisType.FUNDAMENTALS).setColName("col1").setJobType(JobType.SYSTEM).build(); - new MockUp() { - - int count = 0; - - TableStatsMeta[] tableStatsArr = - new TableStatsMeta[] {new TableStatsMeta(0, analysisInfo, tableIf), - new TableStatsMeta(0, analysisInfo, tableIf), null}; - - { - tableStatsArr[0].updatedRows.addAndGet(100); - tableStatsArr[1].updatedRows.addAndGet(0); - } - - @Mock - public TableStatsMeta findTableStatsStatus(long tblId) { - return tableStatsArr[count++]; - } - }; - - new MockUp() { - @Mock - public AnalysisInfo getAnalysisJobInfo(AnalysisInfo jobInfo, TableIf table, - Set needRunPartitions) { - return new AnalysisInfoBuilder().build(); - } - }; - StatisticsAutoCollector statisticsAutoCollector = new StatisticsAutoCollector(); - AnalysisInfo analysisInfo2 = new AnalysisInfoBuilder() - .setCatalogId(0) - .setDBId(0) - .setTblId(0).build(); - Assertions.assertNotNull(statisticsAutoCollector.getReAnalyzeRequiredPart(analysisInfo2)); - // uncomment it when updatedRows gets ready - // Assertions.assertNull(statisticsAutoCollector.getReAnalyzeRequiredPart(analysisInfo2)); - Assertions.assertNotNull(statisticsAutoCollector.getReAnalyzeRequiredPart(analysisInfo2)); - } - - @Test - public void testSkipWideTable() { - - TableIf tableIf = new OlapTable(); - new MockUp() { @Mock - public List getBaseSchema() { - return Lists.newArrayList(new Column("col1", Type.INT), new Column("col2", Type.INT)); + public List> getColumnIndexPairs(Set columns) { + ArrayList> list = Lists.newArrayList(); + list.add(Pair.of("1", "1")); + return list; } }; new MockUp() { int count = 0; - int [] thresholds = {1, 10}; + int[] thresholds = {1, 10}; + @Mock public TableIf findTable(long catalogName, long dbName, long tblName) { return tableIf; @@ -246,19 +176,10 @@ public int getAutoAnalyzeTableWidthThreshold() { } }; - new MockUp() { - @Mock - public Map> findReAnalyzeNeededPartitions() { - HashMap> ret = Maps.newHashMap(); - ret.put("key1", Sets.newHashSet()); - return ret; - } - }; - AnalysisInfo analysisInfo = new AnalysisInfoBuilder().build(); StatisticsAutoCollector statisticsAutoCollector = new StatisticsAutoCollector(); - Assertions.assertNull(statisticsAutoCollector.getReAnalyzeRequiredPart(analysisInfo)); - Assertions.assertNotNull(statisticsAutoCollector.getReAnalyzeRequiredPart(analysisInfo)); + Assertions.assertNull(statisticsAutoCollector.getNeedAnalyzeColumns(analysisInfo)); + Assertions.assertNotNull(statisticsAutoCollector.getNeedAnalyzeColumns(analysisInfo)); } @Test @@ -400,13 +321,9 @@ public TableIf findTable(long catalogId, long dbId, long tblId) { List jobInfos = new ArrayList<>(); sac.createAnalyzeJobForTbl(db, jobInfos, t1); AnalysisInfo jobInfo = jobInfos.get(0); - Map> colToPartitions = new HashMap<>(); - colToPartitions.put("test", new HashSet() { - { - add("p1"); - } - }); - jobInfo = new AnalysisInfoBuilder(jobInfo).setColToPartitions(colToPartitions).build(); + List> columnNames = Lists.newArrayList(); + columnNames.add(Pair.of("test", "t1")); + jobInfo = new AnalysisInfoBuilder(jobInfo).setJobColumns(columnNames).build(); Map analysisTasks = new HashMap<>(); AnalysisManager analysisManager = Env.getCurrentEnv().getAnalysisManager(); analysisManager.createTaskForEachColumns(jobInfo, analysisTasks, false); @@ -472,13 +389,9 @@ public TableIf findTable(long catalogId, long dbId, long tblId) { List jobInfos = new ArrayList<>(); sac.createAnalyzeJobForTbl(db, jobInfos, t1); AnalysisInfo jobInfo = jobInfos.get(0); - Map> colToPartitions = new HashMap<>(); - colToPartitions.put("test", new HashSet() { - { - add("p1"); - } - }); - jobInfo = new AnalysisInfoBuilder(jobInfo).setColToPartitions(colToPartitions).build(); + List> colNames = Lists.newArrayList(); + colNames.add(Pair.of("test", "1")); + jobInfo = new AnalysisInfoBuilder(jobInfo).setJobColumns(colNames).build(); Map analysisTasks = new HashMap<>(); AnalysisManager analysisManager = Env.getCurrentEnv().getAnalysisManager(); analysisManager.createTaskForEachColumns(jobInfo, analysisTasks, false); diff --git a/fe/fe-core/src/test/java/org/apache/doris/statistics/TableStatsMetaTest.java b/fe/fe-core/src/test/java/org/apache/doris/statistics/TableStatsMetaTest.java index b5e73ba09da728..94eab9e00cc501 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/statistics/TableStatsMetaTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/statistics/TableStatsMetaTest.java @@ -25,7 +25,7 @@ import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; -import java.util.HashMap; +import java.util.ArrayList; class TableStatsMetaTest { @@ -38,7 +38,7 @@ public long getRowCount() { } }; TableStatsMeta tableStatsMeta = new TableStatsMeta(); - AnalysisInfo jobInfo = new AnalysisInfoBuilder().setColToPartitions(new HashMap<>()) + AnalysisInfo jobInfo = new AnalysisInfoBuilder().setJobColumns(new ArrayList<>()) .setColName("col1").build(); tableStatsMeta.update(jobInfo, table); Assertions.assertEquals(4, tableStatsMeta.rowCount); diff --git a/fe/pom.xml b/fe/pom.xml index 8c7af48aff7c16..159fbbed41f241 100644 --- a/fe/pom.xml +++ b/fe/pom.xml @@ -206,6 +206,13 @@ under the License. + + + kr.motd.maven + os-maven-plugin + 1.7.0 + + fe-common @@ -305,7 +312,7 @@ under the License. 1.4.3 0.45.2-public 1.11.3 - 15.0.0 + 15.0.2 0.14.1 2.7.4-11 diff --git a/gensrc/proto/cloud.proto b/gensrc/proto/cloud.proto index 2db7dd1a5ef557..8e674f64b1f5af 100644 --- a/gensrc/proto/cloud.proto +++ b/gensrc/proto/cloud.proto @@ -718,6 +718,9 @@ message GetVersionRequest { repeated int64 db_ids = 6; repeated int64 table_ids = 7; repeated int64 partition_ids = 8; + + // True if get table version + optional bool is_table_version = 9; }; message GetVersionResponse { @@ -853,6 +856,8 @@ message IndexRequest { repeated int64 index_ids = 2; optional int64 table_id = 3; optional int64 expiration = 4; + optional int64 db_id = 5; + optional bool is_new_table = 6; } message IndexResponse { diff --git a/gensrc/thrift/FrontendService.thrift b/gensrc/thrift/FrontendService.thrift index 568b174c11c496..ef90e97b41b5ec 100644 --- a/gensrc/thrift/FrontendService.thrift +++ b/gensrc/thrift/FrontendService.thrift @@ -925,6 +925,7 @@ enum TSchemaTableName { METADATA_TABLE = 1, // tvf ACTIVE_QUERIES = 2, // db information_schema's table WORKLOAD_GROUPS = 3, // db information_schema's table + ROUTINES_INFO = 4, // db information_schema's table } struct TMetadataTableRequestParams { diff --git a/regression-test/data/correctness_p0/test_probe_clean.out b/regression-test/data/correctness_p0/test_probe_clean.out new file mode 100644 index 00000000000000..78ab5a2b890be1 --- /dev/null +++ b/regression-test/data/correctness_p0/test_probe_clean.out @@ -0,0 +1,10 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !select_pipelineX -- +2020 -5.2 + +-- !select_pipeline -- +2020 -5.2 + +-- !select_non_pipeline -- +2020 -5.2 + diff --git a/regression-test/data/external_table_p2/hive/test_complex_types.out b/regression-test/data/external_table_p2/hive/test_complex_types.out index fb712c63d30bcf..31cff0e45b2521 100644 --- a/regression-test/data/external_table_p2/hive/test_complex_types.out +++ b/regression-test/data/external_table_p2/hive/test_complex_types.out @@ -9,10 +9,10 @@ ["9wXr9n-TBm9Wyt-r8H-SkAq", "CPDH4G-ZXGPkku-3wY-ktaQ", "RvNlMt-HHjHN5M-VjP-xHAI", "qKIhKy-Ws344os-haX-2pmT", "DOJJ5l-UEkwVMs-x9F-HifD", "m871g8-1eFi7jt-oBq-S0yc", "wXugVP-v2fc6IF-DeU-On3T", "B0mXFX-QvgUgo7-Dih-6rDu", "E9zv3F-xMqSbMa-il4-FuDg", "msuFIN-ZkKO8TY-tu4-veH0", "0rSUyl-Un07aIW-KAx-WHnX", "XvbmO8-WA6oAqc-ihc-s8IL", "G6B6RD-AicAlZb-16u-Pn1I", "coDK0Q-tMg1294-JMQ-ZWQu", "4c0aWh-yhL6BOX-rRu-1n0r", "G4iUcG-ZhWw62v-VLt-n6lH", "IIB7qD-WQistwT-Vux-0c9B", "7cTyuR-5ssXm2S-sJR-JTIZ", "3KPhSW-FICEImf-bba-PCiQ", "qQ7Yup-XBeQGFz-3EP-q0vd", "gjRxRo-Af9Oqx5-IzN-3B9d", "1zSj57-nNZpZ0b-ZKn-BeY0", "sTK0mn-wkp1Xp5-PRS-txVM", "sLrM0s-1KnXLb6-1A3-Z1vJ", "UkYdkP-k7YKiKS-Fxp-qAcI", "v8p0YV-R5pAKZ8-UMr-P1bQ", "RJdTav-jk3os9Z-yRk-WhwV", "lB91ic-pNFZkE4-hBx-e104", "gmRV6e-GKJUg0L-ok7-J6Lz", "o3LUyz-7Toh54O-czG-Xep8", "8fzHhM-4otPAss-qTm-phg8", "kZsHhe-vfClpAR-b3H-7aHl", "TdZnlG-BUgMs7Z-iBM-9c3v", "RipJXn-p4gZkyy-1ZY-xkWe", "ke730M-LmMjGdc-EFy-0LUK", "jBSExJ-GXTc5TB-NSa-xBEd", "kI7Cc8-DSg5RdF-qLo-2bhe", "bAn3VI-x6xXWpB-zWe-G5CJ", "jAil30-kbt6K6z-kbr-8foB", "IHIwNs-1QGqy8l-i8i-vu4G", "p0IbZr-tHCtwiV-0hq-NtIt", "iggdij-M3YNBpd-yiD-a8Ro", "BrJEww-C4LpgaS-AeB-So4U", "xnO3Fi-8rXcpgj-zpm-EmuX", "5w57da-phYtDUx-px2-6frG", "31MfFs-1WyUAr6-gQ0-xLxY", "ryBl2p-rSoPhwd-WPv-NCAU", "KN5TEt-gOfJ4Hy-3pp-HiBa", "ytqxb8-utXXjUf-m41-i6ir", "WhGUGz-zzyvEpD-9BM-2bVf", "dE1tFe-zHClt4u-0cY-TQnC", "MveBhC-g29c0dU-tCT-R6nC", "JTpxue-xSqAhGo-AZk-zB1t", "92TVdU-qDJesPN-0lb-JOd3", "0PODnh-IciBdOZ-0CS-oNeL", "KkkW6x-TiemXQw-OiH-dZ9s", "PIs5Aj-g02HRXw-957-GD2z", "yJIzuw-au6460e-0Tl-XYEJ", "KHvMCD-OQDL0eX-nqK-TmEt", "6QJJgV-Z3IZ1Rf-wyv-rIJ6", "qA9ycc-sR2qm6P-PtB-AIax", "uDeuEb-B0t0Ljr-dWk-jkC4", "5vPy52-ygN0MMH-UB4-nZQL", "zbbmrQ-pT3uAuU-Kae-HjM5", "3QShHS-7RwUB10-0W2-H4Qy", "PMc4QI-5lNajXU-f8m-RGIi", "O9t3dl-q8YHozj-saR-A3Jm", "k4eH3O-aHnTKY7-ADp-4Vsi", "RA4epe-lWWnOff-bpM-bSR4", "6ysu2R-gSc5dwU-cv0-LqCJ", "tVl3TY-o42NMVO-k3S-iqOY", "NMgTrr-W1RrCvP-Zaf-paL7", "d1CJmF-CeG5asM-xms-1dwN", "N1D30g-zFjiGzI-eHC-Sof4", "tOhfKu-Gdtf9Ne-KwA-JdHV", "XLzwK0-6ocGDrS-TtU-wlEI", "XDgZfb-Sxc45Zn-mVO-S2QO", "GQD7a0-fnt9BZs-Kvh-dPbJ", "9dJxj9-HFwEQMY-6p9-s8Vt", "1qU9pA-QJGAna9-JoG-H7GS", "rKIkxA-UnGWYSn-0li-ziuB", "tbPazx-IjUrQ8J-NZe-VOPL", "xBpSIv-U6ojkK7-9p5-LviD", "88bnWI-pxrKa7T-n2d-tXk9", "0XviXp-9ksT8s0-fDy-35SW", "e0XauA-GNRALmd-SM2-Y4Gf", "kyvYBk-Bk5M4Xq-gxX-kE1B", "dIiQzS-5sT4ogL-6IV-tLmb", "OlGOyH-dyL1nzj-B2M-z8ir", "zC9Gtn-x8hpfPD-KOu-k31W", "qSq3z2-Lpv0YcB-hBq-Sabd", "LSyNyi-tBZUx1l-hAj-mwsx", "2c9aTP-hXloMK7-ufH-dgq6", "aXksHO-zARQxfo-sgS-8Bf4", "ioOXAL-eVUF0W8-vZx-ZeYX", "DXUkAP-A7SqnHj-V4U-PJfz", "cnzZXk-AOMepfN-hym-qbDH", "CMlAd6-8FF1yXs-fae-Izfv", "qiXnUv-e2PsJWm-tLF-KpjE", "Gfx3k9-JvXa7Wd-rI1-1e1E"] -- !map_values -- -[0.98055020292316664, 0.53302915957540542, 0.30024744873379805, 0.48563601750302665, 0.76871064251586241, 0.69935066449251015, 0.28493548088258069, 0.34734174551861408, 0.13500129443045072, 0.97081321037009394, 0.18583042639943448, 0.48863372645520731, 0.36354741695157655, 0.56408452689711752, 0.1374134087807577, 0.77665476474516226, 0.58353232966683177, 0.36544595471103491, 0.54797767099937644, 0.83799325421171922, 0.15665046278350814, 0.03371222042250388, 0.1699781825927229, 0.35796304950750779, 0.028092531855977265, 0.72042470298400274, 0.27604992564232056, 0.676890893219096, 0.035298786567000251, 0.022765783510278581, 0.097949917306254686, 0.52780628846133515, 0.13704041811391021, 0.54403524765808564, 0.72055406294199287, 0.13508529841959427, 0.41609464004318619, 0.29722954545629288, 0.92174265035856928, 0.58103998733474, 0.88454274363774732, 0.10179282672994228, 0.95471869739438919, 0.16801027847083416, 0.00084877454219867143, 0.16952415411069888, 0.67839217494332915, 0.71938183869710837, 0.930443435029246, 0.48466654693905176, 0.9924998940864419, 0.72382884810791481, 0.70535638177590088, 0.97351607727767553, 0.77824997878692337, 0.74133042805481741, 0.75509839260333067, 0.87136604463221856, 0.92052096787926374, 0.34197248989722773, 0.36968069857555563, 0.030232598171523017, 0.024774526048626844, 0.97641291575255884, 0.59330575594702828, 0.76125115548318434, 0.37875822703363504, 0.93127304595441207, 0.67120835078024121, 0.16508080008436798, 0.22928664639590624, 0.37366653502681058, 0.20480644640806578, 0.083943559374968335, 0.84949796967318236, 0.43215562556626219, 0.35346682671980267, 0.87917004341027716, 0.22745275830152578, 0.048869685073594016, 0.79365981101741634, 0.54497173434159185, 0.76359394459683483, 0.085055861839866242, 0.35091150265891446, 0.96331917452389082, 0.39725339103896173, 0.46597592499192675, 0.15790512463284645, 0.78535655781075941, 0.98949199397456544, 0.93953657306559291, 0.20226076738266596, 0.16196368561927676, 0.51055695298416159, 0.4531109229280732, 0.25791342685970842, 0.79621090899157465, 0.27729692295394215, 0.93159020376070611] +[0.9805502029231666, 0.5330291595754054, 0.30024744873379805, 0.48563601750302665, 0.7687106425158624, 0.6993506644925102, 0.2849354808825807, 0.3473417455186141, 0.13500129443045072, 0.9708132103700939, 0.18583042639943448, 0.4886337264552073, 0.36354741695157655, 0.5640845268971175, 0.1374134087807577, 0.7766547647451623, 0.5835323296668318, 0.3654459547110349, 0.5479776709993764, 0.8379932542117192, 0.15665046278350814, 0.03371222042250388, 0.1699781825927229, 0.3579630495075078, 0.028092531855977265, 0.7204247029840027, 0.27604992564232056, 0.676890893219096, 0.03529878656700025, 0.02276578351027858, 0.09794991730625469, 0.5278062884613351, 0.1370404181139102, 0.5440352476580856, 0.7205540629419929, 0.13508529841959427, 0.4160946400431862, 0.2972295454562929, 0.9217426503585693, 0.58103998733474, 0.8845427436377473, 0.10179282672994228, 0.9547186973943892, 0.16801027847083416, 0.0008487745421986714, 0.16952415411069888, 0.6783921749433292, 0.7193818386971084, 0.930443435029246, 0.48466654693905176, 0.9924998940864419, 0.7238288481079148, 0.7053563817759009, 0.9735160772776755, 0.7782499787869234, 0.7413304280548174, 0.7550983926033307, 0.8713660446322186, 0.9205209678792637, 0.3419724898972277, 0.36968069857555563, 0.030232598171523017, 0.024774526048626844, 0.9764129157525588, 0.5933057559470283, 0.7612511554831843, 0.37875822703363504, 0.9312730459544121, 0.6712083507802412, 0.16508080008436798, 0.22928664639590624, 0.3736665350268106, 0.20480644640806578, 0.08394355937496834, 0.8494979696731824, 0.4321556255662622, 0.3534668267198027, 0.8791700434102772, 0.22745275830152578, 0.048869685073594016, 0.7936598110174163, 0.5449717343415919, 0.7635939445968348, 0.08505586183986624, 0.35091150265891446, 0.9633191745238908, 0.39725339103896173, 0.46597592499192675, 0.15790512463284645, 0.7853565578107594, 0.9894919939745654, 0.9395365730655929, 0.20226076738266596, 0.16196368561927676, 0.5105569529841616, 0.4531109229280732, 0.2579134268597084, 0.7962109089915747, 0.27729692295394215, 0.9315902037607061] -- !map_contains_key -- -1077 [0.78055609958738448, 0.93034890022695593, 0.25295229975218769, 0.662270811026298, 0.664725297532439, 0.10194410917644769, 0.96140593006881736, 0.52781260099838434, 0.52875058412167075, 0.426116738236779, 0.42300502393871175, 0.53270263300536513, 0.60254817779426029, 0.27107336472576271, 0.613792118138183, 0.0021003027835629906, 0.32006750487285818, 0.54856110146602044, 0.51215105813137074, 0.51451366528053577] {"9wXr9n-TBm9Wyt-r8H-SkAq":0.93383290104809946, "CPDH4G-ZXGPkku-3wY-ktaQ":0.43552569633508809, "RvNlMt-HHjHN5M-VjP-xHAI":0.32634746118047819, "qKIhKy-Ws344os-haX-2pmT":0.565450203625137, "DOJJ5l-UEkwVMs-x9F-HifD":0.093756220108222377, "m871g8-1eFi7jt-oBq-S0yc":0.8819687247951038, "wXugVP-v2fc6IF-DeU-On3T":0.34482334864473108, "B0mXFX-QvgUgo7-Dih-6rDu":0.1914040395475467, "E9zv3F-xMqSbMa-il4-FuDg":0.38570218910843357, "msuFIN-ZkKO8TY-tu4-veH0":0.66461726530746279, "0rSUyl-Un07aIW-KAx-WHnX":0.35580099104309737, "XvbmO8-WA6oAqc-ihc-s8IL":0.40582064344114233, "G6B6RD-AicAlZb-16u-Pn1I":0.72035549468957494, "coDK0Q-tMg1294-JMQ-ZWQu":0.82363286277431857, "4c0aWh-yhL6BOX-rRu-1n0r":0.13980911842304278, "G4iUcG-ZhWw62v-VLt-n6lH":0.18382889782542144, "IIB7qD-WQistwT-Vux-0c9B":0.91743891443094583, "7cTyuR-5ssXm2S-sJR-JTIZ":0.81322372426728373, "3KPhSW-FICEImf-bba-PCiQ":0.63026435799435532, "qQ7Yup-XBeQGFz-3EP-q0vd":0.6109025726752364, "gjRxRo-Af9Oqx5-IzN-3B9d":0.92514684903269162, "1zSj57-nNZpZ0b-ZKn-BeY0":0.56284631091071435, "sTK0mn-wkp1Xp5-PRS-txVM":0.79058081295599958, "sLrM0s-1KnXLb6-1A3-Z1vJ":0.42345986776701572, "UkYdkP-k7YKiKS-Fxp-qAcI":0.75414012666798691, "v8p0YV-R5pAKZ8-UMr-P1bQ":0.29311525651106829, "RJdTav-jk3os9Z-yRk-WhwV":0.52638113097388772, "lB91ic-pNFZkE4-hBx-e104":0.6692292834321788, "gmRV6e-GKJUg0L-ok7-J6Lz":0.059247669596643515, "o3LUyz-7Toh54O-czG-Xep8":0.62841938211272641, "8fzHhM-4otPAss-qTm-phg8":0.89530024415370124, "kZsHhe-vfClpAR-b3H-7aHl":0.17750156127473993, "TdZnlG-BUgMs7Z-iBM-9c3v":0.27498394395046333, "RipJXn-p4gZkyy-1ZY-xkWe":0.054616268950389735, "ke730M-LmMjGdc-EFy-0LUK":0.30781761836448285, "jBSExJ-GXTc5TB-NSa-xBEd":0.66178278500540244, "kI7Cc8-DSg5RdF-qLo-2bhe":0.98357074613234885, "bAn3VI-x6xXWpB-zWe-G5CJ":0.21798212299794562, "jAil30-kbt6K6z-kbr-8foB":0.97880669772451379, "IHIwNs-1QGqy8l-i8i-vu4G":0.49679397412450588, "p0IbZr-tHCtwiV-0hq-NtIt":0.050183795109057017, "iggdij-M3YNBpd-yiD-a8Ro":0.982385582884686, "BrJEww-C4LpgaS-AeB-So4U":0.90248554155536553, "xnO3Fi-8rXcpgj-zpm-EmuX":0.20529118817468572, "5w57da-phYtDUx-px2-6frG":0.29690638791565971, "31MfFs-1WyUAr6-gQ0-xLxY":0.48795551283135086, "ryBl2p-rSoPhwd-WPv-NCAU":0.79544854844958068, "KN5TEt-gOfJ4Hy-3pp-HiBa":0.1533389643648807, "ytqxb8-utXXjUf-m41-i6ir":0.61502086737193573, "WhGUGz-zzyvEpD-9BM-2bVf":0.581040090228354, "dE1tFe-zHClt4u-0cY-TQnC":0.76089996323694564, "MveBhC-g29c0dU-tCT-R6nC":0.33457340282218506, "JTpxue-xSqAhGo-AZk-zB1t":0.35040302774880538, "92TVdU-qDJesPN-0lb-JOd3":0.73876949983198048, "0PODnh-IciBdOZ-0CS-oNeL":0.95159059657696443, "KkkW6x-TiemXQw-OiH-dZ9s":0.40824123319990813, "PIs5Aj-g02HRXw-957-GD2z":0.641526116451016, "yJIzuw-au6460e-0Tl-XYEJ":0.75219285303562355, "KHvMCD-OQDL0eX-nqK-TmEt":0.13096167278968263, "6QJJgV-Z3IZ1Rf-wyv-rIJ6":0.70071103877259622, "qA9ycc-sR2qm6P-PtB-AIax":0.44629776556459089, "uDeuEb-B0t0Ljr-dWk-jkC4":0.69046727674079578, "5vPy52-ygN0MMH-UB4-nZQL":0.60575965422000211, "zbbmrQ-pT3uAuU-Kae-HjM5":0.98126574986862791, "3QShHS-7RwUB10-0W2-H4Qy":0.41557608488608533, "PMc4QI-5lNajXU-f8m-RGIi":0.70464209768002883, "O9t3dl-q8YHozj-saR-A3Jm":0.85433449541965856, "k4eH3O-aHnTKY7-ADp-4Vsi":0.26558324547185574, "RA4epe-lWWnOff-bpM-bSR4":0.75232522102228827, "6ysu2R-gSc5dwU-cv0-LqCJ":0.78308993227167323, "tVl3TY-o42NMVO-k3S-iqOY":0.79238234012157993, "NMgTrr-W1RrCvP-Zaf-paL7":0.46869286547569355, "d1CJmF-CeG5asM-xms-1dwN":0.76229087810764928, "N1D30g-zFjiGzI-eHC-Sof4":0.847542878440137, "tOhfKu-Gdtf9Ne-KwA-JdHV":0.49992852174451541, "XLzwK0-6ocGDrS-TtU-wlEI":0.39853544027050947, "XDgZfb-Sxc45Zn-mVO-S2QO":0.057915803376441866, "GQD7a0-fnt9BZs-Kvh-dPbJ":0.663903859916476, "9dJxj9-HFwEQMY-6p9-s8Vt":0.21944075953054343, "1qU9pA-QJGAna9-JoG-H7GS":0.88774019472953825, "rKIkxA-UnGWYSn-0li-ziuB":0.16079062750364659, "tbPazx-IjUrQ8J-NZe-VOPL":0.68091669167975932, "xBpSIv-U6ojkK7-9p5-LviD":0.11956726473799006, "88bnWI-pxrKa7T-n2d-tXk9":0.19560689517877206, "0XviXp-9ksT8s0-fDy-35SW":0.8690659418822626, "e0XauA-GNRALmd-SM2-Y4Gf":0.68408168887520893, "kyvYBk-Bk5M4Xq-gxX-kE1B":0.7744771682336401, "dIiQzS-5sT4ogL-6IV-tLmb":0.0340772833497166, "OlGOyH-dyL1nzj-B2M-z8ir":0.3765608037933722, "zC9Gtn-x8hpfPD-KOu-k31W":0.864392047887076, "qSq3z2-Lpv0YcB-hBq-Sabd":0.15428476092466781, "LSyNyi-tBZUx1l-hAj-mwsx":0.30403432829870103, "2c9aTP-hXloMK7-ufH-dgq6":0.10168525529531069, "aXksHO-zARQxfo-sgS-8Bf4":0.54905330820199594, "ioOXAL-eVUF0W8-vZx-ZeYX":0.45281640384817845, "DXUkAP-A7SqnHj-V4U-PJfz":0.36074074474259388, "cnzZXk-AOMepfN-hym-qbDH":0.45873615005925683, "CMlAd6-8FF1yXs-fae-Izfv":0.075550197208259173, "qiXnUv-e2PsJWm-tLF-KpjE":0.94096810653636875, "Gfx3k9-JvXa7Wd-rI1-1e1E":0.74927933121782264} {"name": "r8HXXQM4XHoI", "age": 238221053, "tip": "2023-07-26 15:40:37.694000"} +1077 [0.7805560995873845, 0.9303489002269559, 0.2529522997521877, 0.662270811026298, 0.664725297532439, 0.1019441091764477, 0.9614059300688174, 0.5278126009983843, 0.5287505841216708, 0.426116738236779, 0.42300502393871175, 0.5327026330053651, 0.6025481777942603, 0.2710733647257627, 0.613792118138183, 0.0021003027835629906, 0.3200675048728582, 0.5485611014660204, 0.5121510581313707, 0.5145136652805358] {"9wXr9n-TBm9Wyt-r8H-SkAq":0.9338329010480995, "CPDH4G-ZXGPkku-3wY-ktaQ":0.4355256963350881, "RvNlMt-HHjHN5M-VjP-xHAI":0.3263474611804782, "qKIhKy-Ws344os-haX-2pmT":0.565450203625137, "DOJJ5l-UEkwVMs-x9F-HifD":0.09375622010822238, "m871g8-1eFi7jt-oBq-S0yc":0.8819687247951038, "wXugVP-v2fc6IF-DeU-On3T":0.3448233486447311, "B0mXFX-QvgUgo7-Dih-6rDu":0.1914040395475467, "E9zv3F-xMqSbMa-il4-FuDg":0.38570218910843357, "msuFIN-ZkKO8TY-tu4-veH0":0.6646172653074628, "0rSUyl-Un07aIW-KAx-WHnX":0.35580099104309737, "XvbmO8-WA6oAqc-ihc-s8IL":0.4058206434411423, "G6B6RD-AicAlZb-16u-Pn1I":0.7203554946895749, "coDK0Q-tMg1294-JMQ-ZWQu":0.8236328627743186, "4c0aWh-yhL6BOX-rRu-1n0r":0.13980911842304278, "G4iUcG-ZhWw62v-VLt-n6lH":0.18382889782542144, "IIB7qD-WQistwT-Vux-0c9B":0.9174389144309458, "7cTyuR-5ssXm2S-sJR-JTIZ":0.8132237242672837, "3KPhSW-FICEImf-bba-PCiQ":0.6302643579943553, "qQ7Yup-XBeQGFz-3EP-q0vd":0.6109025726752364, "gjRxRo-Af9Oqx5-IzN-3B9d":0.9251468490326916, "1zSj57-nNZpZ0b-ZKn-BeY0":0.5628463109107144, "sTK0mn-wkp1Xp5-PRS-txVM":0.7905808129559996, "sLrM0s-1KnXLb6-1A3-Z1vJ":0.4234598677670157, "UkYdkP-k7YKiKS-Fxp-qAcI":0.7541401266679869, "v8p0YV-R5pAKZ8-UMr-P1bQ":0.2931152565110683, "RJdTav-jk3os9Z-yRk-WhwV":0.5263811309738877, "lB91ic-pNFZkE4-hBx-e104":0.6692292834321788, "gmRV6e-GKJUg0L-ok7-J6Lz":0.059247669596643515, "o3LUyz-7Toh54O-czG-Xep8":0.6284193821127264, "8fzHhM-4otPAss-qTm-phg8":0.8953002441537012, "kZsHhe-vfClpAR-b3H-7aHl":0.17750156127473993, "TdZnlG-BUgMs7Z-iBM-9c3v":0.2749839439504633, "RipJXn-p4gZkyy-1ZY-xkWe":0.054616268950389735, "ke730M-LmMjGdc-EFy-0LUK":0.30781761836448285, "jBSExJ-GXTc5TB-NSa-xBEd":0.6617827850054024, "kI7Cc8-DSg5RdF-qLo-2bhe":0.9835707461323488, "bAn3VI-x6xXWpB-zWe-G5CJ":0.21798212299794562, "jAil30-kbt6K6z-kbr-8foB":0.9788066977245138, "IHIwNs-1QGqy8l-i8i-vu4G":0.4967939741245059, "p0IbZr-tHCtwiV-0hq-NtIt":0.05018379510905702, "iggdij-M3YNBpd-yiD-a8Ro":0.982385582884686, "BrJEww-C4LpgaS-AeB-So4U":0.9024855415553655, "xnO3Fi-8rXcpgj-zpm-EmuX":0.20529118817468572, "5w57da-phYtDUx-px2-6frG":0.2969063879156597, "31MfFs-1WyUAr6-gQ0-xLxY":0.48795551283135086, "ryBl2p-rSoPhwd-WPv-NCAU":0.7954485484495807, "KN5TEt-gOfJ4Hy-3pp-HiBa":0.1533389643648807, "ytqxb8-utXXjUf-m41-i6ir":0.6150208673719357, "WhGUGz-zzyvEpD-9BM-2bVf":0.581040090228354, "dE1tFe-zHClt4u-0cY-TQnC":0.7608999632369456, "MveBhC-g29c0dU-tCT-R6nC":0.33457340282218506, "JTpxue-xSqAhGo-AZk-zB1t":0.3504030277488054, "92TVdU-qDJesPN-0lb-JOd3":0.7387694998319805, "0PODnh-IciBdOZ-0CS-oNeL":0.9515905965769644, "KkkW6x-TiemXQw-OiH-dZ9s":0.4082412331999081, "PIs5Aj-g02HRXw-957-GD2z":0.641526116451016, "yJIzuw-au6460e-0Tl-XYEJ":0.7521928530356236, "KHvMCD-OQDL0eX-nqK-TmEt":0.13096167278968263, "6QJJgV-Z3IZ1Rf-wyv-rIJ6":0.7007110387725962, "qA9ycc-sR2qm6P-PtB-AIax":0.4462977655645909, "uDeuEb-B0t0Ljr-dWk-jkC4":0.6904672767407958, "5vPy52-ygN0MMH-UB4-nZQL":0.6057596542200021, "zbbmrQ-pT3uAuU-Kae-HjM5":0.9812657498686279, "3QShHS-7RwUB10-0W2-H4Qy":0.41557608488608533, "PMc4QI-5lNajXU-f8m-RGIi":0.7046420976800288, "O9t3dl-q8YHozj-saR-A3Jm":0.8543344954196586, "k4eH3O-aHnTKY7-ADp-4Vsi":0.26558324547185574, "RA4epe-lWWnOff-bpM-bSR4":0.7523252210222883, "6ysu2R-gSc5dwU-cv0-LqCJ":0.7830899322716732, "tVl3TY-o42NMVO-k3S-iqOY":0.7923823401215799, "NMgTrr-W1RrCvP-Zaf-paL7":0.46869286547569355, "d1CJmF-CeG5asM-xms-1dwN":0.7622908781076493, "N1D30g-zFjiGzI-eHC-Sof4":0.847542878440137, "tOhfKu-Gdtf9Ne-KwA-JdHV":0.4999285217445154, "XLzwK0-6ocGDrS-TtU-wlEI":0.39853544027050947, "XDgZfb-Sxc45Zn-mVO-S2QO":0.057915803376441866, "GQD7a0-fnt9BZs-Kvh-dPbJ":0.663903859916476, "9dJxj9-HFwEQMY-6p9-s8Vt":0.21944075953054343, "1qU9pA-QJGAna9-JoG-H7GS":0.8877401947295382, "rKIkxA-UnGWYSn-0li-ziuB":0.1607906275036466, "tbPazx-IjUrQ8J-NZe-VOPL":0.6809166916797593, "xBpSIv-U6ojkK7-9p5-LviD":0.11956726473799006, "88bnWI-pxrKa7T-n2d-tXk9":0.19560689517877206, "0XviXp-9ksT8s0-fDy-35SW":0.8690659418822626, "e0XauA-GNRALmd-SM2-Y4Gf":0.6840816888752089, "kyvYBk-Bk5M4Xq-gxX-kE1B":0.7744771682336401, "dIiQzS-5sT4ogL-6IV-tLmb":0.0340772833497166, "OlGOyH-dyL1nzj-B2M-z8ir":0.3765608037933722, "zC9Gtn-x8hpfPD-KOu-k31W":0.864392047887076, "qSq3z2-Lpv0YcB-hBq-Sabd":0.1542847609246678, "LSyNyi-tBZUx1l-hAj-mwsx":0.30403432829870103, "2c9aTP-hXloMK7-ufH-dgq6":0.1016852552953107, "aXksHO-zARQxfo-sgS-8Bf4":0.5490533082019959, "ioOXAL-eVUF0W8-vZx-ZeYX":0.45281640384817845, "DXUkAP-A7SqnHj-V4U-PJfz":0.3607407447425939, "cnzZXk-AOMepfN-hym-qbDH":0.45873615005925683, "CMlAd6-8FF1yXs-fae-Izfv":0.07555019720825917, "qiXnUv-e2PsJWm-tLF-KpjE":0.9409681065363688, "Gfx3k9-JvXa7Wd-rI1-1e1E":0.7492793312178226} {"name": "r8HXXQM4XHoI", "age": 238221053, "tip": "2023-07-26 15:40:37.694000"} -- !array_max -- 11028 @@ -33,10 +33,10 @@ ["9wXr9n-TBm9Wyt-r8H-SkAq", "CPDH4G-ZXGPkku-3wY-ktaQ", "RvNlMt-HHjHN5M-VjP-xHAI", "qKIhKy-Ws344os-haX-2pmT", "DOJJ5l-UEkwVMs-x9F-HifD", "m871g8-1eFi7jt-oBq-S0yc", "wXugVP-v2fc6IF-DeU-On3T", "B0mXFX-QvgUgo7-Dih-6rDu", "E9zv3F-xMqSbMa-il4-FuDg", "msuFIN-ZkKO8TY-tu4-veH0", "0rSUyl-Un07aIW-KAx-WHnX", "XvbmO8-WA6oAqc-ihc-s8IL", "G6B6RD-AicAlZb-16u-Pn1I", "coDK0Q-tMg1294-JMQ-ZWQu", "4c0aWh-yhL6BOX-rRu-1n0r", "G4iUcG-ZhWw62v-VLt-n6lH", "IIB7qD-WQistwT-Vux-0c9B", "7cTyuR-5ssXm2S-sJR-JTIZ", "3KPhSW-FICEImf-bba-PCiQ", "qQ7Yup-XBeQGFz-3EP-q0vd", "gjRxRo-Af9Oqx5-IzN-3B9d", "1zSj57-nNZpZ0b-ZKn-BeY0", "sTK0mn-wkp1Xp5-PRS-txVM", "sLrM0s-1KnXLb6-1A3-Z1vJ", "UkYdkP-k7YKiKS-Fxp-qAcI", "v8p0YV-R5pAKZ8-UMr-P1bQ", "RJdTav-jk3os9Z-yRk-WhwV", "lB91ic-pNFZkE4-hBx-e104", "gmRV6e-GKJUg0L-ok7-J6Lz", "o3LUyz-7Toh54O-czG-Xep8", "8fzHhM-4otPAss-qTm-phg8", "kZsHhe-vfClpAR-b3H-7aHl", "TdZnlG-BUgMs7Z-iBM-9c3v", "RipJXn-p4gZkyy-1ZY-xkWe", "ke730M-LmMjGdc-EFy-0LUK", "jBSExJ-GXTc5TB-NSa-xBEd", "kI7Cc8-DSg5RdF-qLo-2bhe", "bAn3VI-x6xXWpB-zWe-G5CJ", "jAil30-kbt6K6z-kbr-8foB", "IHIwNs-1QGqy8l-i8i-vu4G", "p0IbZr-tHCtwiV-0hq-NtIt", "iggdij-M3YNBpd-yiD-a8Ro", "BrJEww-C4LpgaS-AeB-So4U", "xnO3Fi-8rXcpgj-zpm-EmuX", "5w57da-phYtDUx-px2-6frG", "31MfFs-1WyUAr6-gQ0-xLxY", "ryBl2p-rSoPhwd-WPv-NCAU", "KN5TEt-gOfJ4Hy-3pp-HiBa", "ytqxb8-utXXjUf-m41-i6ir", "WhGUGz-zzyvEpD-9BM-2bVf", "dE1tFe-zHClt4u-0cY-TQnC", "MveBhC-g29c0dU-tCT-R6nC", "JTpxue-xSqAhGo-AZk-zB1t", "92TVdU-qDJesPN-0lb-JOd3", "0PODnh-IciBdOZ-0CS-oNeL", "KkkW6x-TiemXQw-OiH-dZ9s", "PIs5Aj-g02HRXw-957-GD2z", "yJIzuw-au6460e-0Tl-XYEJ", "KHvMCD-OQDL0eX-nqK-TmEt", "6QJJgV-Z3IZ1Rf-wyv-rIJ6", "qA9ycc-sR2qm6P-PtB-AIax", "uDeuEb-B0t0Ljr-dWk-jkC4", "5vPy52-ygN0MMH-UB4-nZQL", "zbbmrQ-pT3uAuU-Kae-HjM5", "3QShHS-7RwUB10-0W2-H4Qy", "PMc4QI-5lNajXU-f8m-RGIi", "O9t3dl-q8YHozj-saR-A3Jm", "k4eH3O-aHnTKY7-ADp-4Vsi", "RA4epe-lWWnOff-bpM-bSR4", "6ysu2R-gSc5dwU-cv0-LqCJ", "tVl3TY-o42NMVO-k3S-iqOY", "NMgTrr-W1RrCvP-Zaf-paL7", "d1CJmF-CeG5asM-xms-1dwN", "N1D30g-zFjiGzI-eHC-Sof4", "tOhfKu-Gdtf9Ne-KwA-JdHV", "XLzwK0-6ocGDrS-TtU-wlEI", "XDgZfb-Sxc45Zn-mVO-S2QO", "GQD7a0-fnt9BZs-Kvh-dPbJ", "9dJxj9-HFwEQMY-6p9-s8Vt", "1qU9pA-QJGAna9-JoG-H7GS", "rKIkxA-UnGWYSn-0li-ziuB", "tbPazx-IjUrQ8J-NZe-VOPL", "xBpSIv-U6ojkK7-9p5-LviD", "88bnWI-pxrKa7T-n2d-tXk9", "0XviXp-9ksT8s0-fDy-35SW", "e0XauA-GNRALmd-SM2-Y4Gf", "kyvYBk-Bk5M4Xq-gxX-kE1B", "dIiQzS-5sT4ogL-6IV-tLmb", "OlGOyH-dyL1nzj-B2M-z8ir", "zC9Gtn-x8hpfPD-KOu-k31W", "qSq3z2-Lpv0YcB-hBq-Sabd", "LSyNyi-tBZUx1l-hAj-mwsx", "2c9aTP-hXloMK7-ufH-dgq6", "aXksHO-zARQxfo-sgS-8Bf4", "ioOXAL-eVUF0W8-vZx-ZeYX", "DXUkAP-A7SqnHj-V4U-PJfz", "cnzZXk-AOMepfN-hym-qbDH", "CMlAd6-8FF1yXs-fae-Izfv", "qiXnUv-e2PsJWm-tLF-KpjE", "Gfx3k9-JvXa7Wd-rI1-1e1E"] -- !map_values_orc -- -[0.98055020292316664, 0.53302915957540542, 0.30024744873379805, 0.48563601750302665, 0.76871064251586241, 0.69935066449251015, 0.28493548088258069, 0.34734174551861408, 0.13500129443045072, 0.97081321037009394, 0.18583042639943448, 0.48863372645520731, 0.36354741695157655, 0.56408452689711752, 0.1374134087807577, 0.77665476474516226, 0.58353232966683177, 0.36544595471103491, 0.54797767099937644, 0.83799325421171922, 0.15665046278350814, 0.03371222042250388, 0.1699781825927229, 0.35796304950750779, 0.028092531855977265, 0.72042470298400274, 0.27604992564232056, 0.676890893219096, 0.035298786567000251, 0.022765783510278581, 0.097949917306254686, 0.52780628846133515, 0.13704041811391021, 0.54403524765808564, 0.72055406294199287, 0.13508529841959427, 0.41609464004318619, 0.29722954545629288, 0.92174265035856928, 0.58103998733474, 0.88454274363774732, 0.10179282672994228, 0.95471869739438919, 0.16801027847083416, 0.00084877454219867143, 0.16952415411069888, 0.67839217494332915, 0.71938183869710837, 0.930443435029246, 0.48466654693905176, 0.9924998940864419, 0.72382884810791481, 0.70535638177590088, 0.97351607727767553, 0.77824997878692337, 0.74133042805481741, 0.75509839260333067, 0.87136604463221856, 0.92052096787926374, 0.34197248989722773, 0.36968069857555563, 0.030232598171523017, 0.024774526048626844, 0.97641291575255884, 0.59330575594702828, 0.76125115548318434, 0.37875822703363504, 0.93127304595441207, 0.67120835078024121, 0.16508080008436798, 0.22928664639590624, 0.37366653502681058, 0.20480644640806578, 0.083943559374968335, 0.84949796967318236, 0.43215562556626219, 0.35346682671980267, 0.87917004341027716, 0.22745275830152578, 0.048869685073594016, 0.79365981101741634, 0.54497173434159185, 0.76359394459683483, 0.085055861839866242, 0.35091150265891446, 0.96331917452389082, 0.39725339103896173, 0.46597592499192675, 0.15790512463284645, 0.78535655781075941, 0.98949199397456544, 0.93953657306559291, 0.20226076738266596, 0.16196368561927676, 0.51055695298416159, 0.4531109229280732, 0.25791342685970842, 0.79621090899157465, 0.27729692295394215, 0.93159020376070611] +[0.9805502029231666, 0.5330291595754054, 0.30024744873379805, 0.48563601750302665, 0.7687106425158624, 0.6993506644925102, 0.2849354808825807, 0.3473417455186141, 0.13500129443045072, 0.9708132103700939, 0.18583042639943448, 0.4886337264552073, 0.36354741695157655, 0.5640845268971175, 0.1374134087807577, 0.7766547647451623, 0.5835323296668318, 0.3654459547110349, 0.5479776709993764, 0.8379932542117192, 0.15665046278350814, 0.03371222042250388, 0.1699781825927229, 0.3579630495075078, 0.028092531855977265, 0.7204247029840027, 0.27604992564232056, 0.676890893219096, 0.03529878656700025, 0.02276578351027858, 0.09794991730625469, 0.5278062884613351, 0.1370404181139102, 0.5440352476580856, 0.7205540629419929, 0.13508529841959427, 0.4160946400431862, 0.2972295454562929, 0.9217426503585693, 0.58103998733474, 0.8845427436377473, 0.10179282672994228, 0.9547186973943892, 0.16801027847083416, 0.0008487745421986714, 0.16952415411069888, 0.6783921749433292, 0.7193818386971084, 0.930443435029246, 0.48466654693905176, 0.9924998940864419, 0.7238288481079148, 0.7053563817759009, 0.9735160772776755, 0.7782499787869234, 0.7413304280548174, 0.7550983926033307, 0.8713660446322186, 0.9205209678792637, 0.3419724898972277, 0.36968069857555563, 0.030232598171523017, 0.024774526048626844, 0.9764129157525588, 0.5933057559470283, 0.7612511554831843, 0.37875822703363504, 0.9312730459544121, 0.6712083507802412, 0.16508080008436798, 0.22928664639590624, 0.3736665350268106, 0.20480644640806578, 0.08394355937496834, 0.8494979696731824, 0.4321556255662622, 0.3534668267198027, 0.8791700434102772, 0.22745275830152578, 0.048869685073594016, 0.7936598110174163, 0.5449717343415919, 0.7635939445968348, 0.08505586183986624, 0.35091150265891446, 0.9633191745238908, 0.39725339103896173, 0.46597592499192675, 0.15790512463284645, 0.7853565578107594, 0.9894919939745654, 0.9395365730655929, 0.20226076738266596, 0.16196368561927676, 0.5105569529841616, 0.4531109229280732, 0.2579134268597084, 0.7962109089915747, 0.27729692295394215, 0.9315902037607061] -- !map_contains_key_orc -- -1077 [0.78055609958738448, 0.93034890022695593, 0.25295229975218769, 0.662270811026298, 0.664725297532439, 0.10194410917644769, 0.96140593006881736, 0.52781260099838434, 0.52875058412167075, 0.426116738236779, 0.42300502393871175, 0.53270263300536513, 0.60254817779426029, 0.27107336472576271, 0.613792118138183, 0.0021003027835629906, 0.32006750487285818, 0.54856110146602044, 0.51215105813137074, 0.51451366528053577] {"9wXr9n-TBm9Wyt-r8H-SkAq":0.93383290104809946, "CPDH4G-ZXGPkku-3wY-ktaQ":0.43552569633508809, "RvNlMt-HHjHN5M-VjP-xHAI":0.32634746118047819, "qKIhKy-Ws344os-haX-2pmT":0.565450203625137, "DOJJ5l-UEkwVMs-x9F-HifD":0.093756220108222377, "m871g8-1eFi7jt-oBq-S0yc":0.8819687247951038, "wXugVP-v2fc6IF-DeU-On3T":0.34482334864473108, "B0mXFX-QvgUgo7-Dih-6rDu":0.1914040395475467, "E9zv3F-xMqSbMa-il4-FuDg":0.38570218910843357, "msuFIN-ZkKO8TY-tu4-veH0":0.66461726530746279, "0rSUyl-Un07aIW-KAx-WHnX":0.35580099104309737, "XvbmO8-WA6oAqc-ihc-s8IL":0.40582064344114233, "G6B6RD-AicAlZb-16u-Pn1I":0.72035549468957494, "coDK0Q-tMg1294-JMQ-ZWQu":0.82363286277431857, "4c0aWh-yhL6BOX-rRu-1n0r":0.13980911842304278, "G4iUcG-ZhWw62v-VLt-n6lH":0.18382889782542144, "IIB7qD-WQistwT-Vux-0c9B":0.91743891443094583, "7cTyuR-5ssXm2S-sJR-JTIZ":0.81322372426728373, "3KPhSW-FICEImf-bba-PCiQ":0.63026435799435532, "qQ7Yup-XBeQGFz-3EP-q0vd":0.6109025726752364, "gjRxRo-Af9Oqx5-IzN-3B9d":0.92514684903269162, "1zSj57-nNZpZ0b-ZKn-BeY0":0.56284631091071435, "sTK0mn-wkp1Xp5-PRS-txVM":0.79058081295599958, "sLrM0s-1KnXLb6-1A3-Z1vJ":0.42345986776701572, "UkYdkP-k7YKiKS-Fxp-qAcI":0.75414012666798691, "v8p0YV-R5pAKZ8-UMr-P1bQ":0.29311525651106829, "RJdTav-jk3os9Z-yRk-WhwV":0.52638113097388772, "lB91ic-pNFZkE4-hBx-e104":0.6692292834321788, "gmRV6e-GKJUg0L-ok7-J6Lz":0.059247669596643515, "o3LUyz-7Toh54O-czG-Xep8":0.62841938211272641, "8fzHhM-4otPAss-qTm-phg8":0.89530024415370124, "kZsHhe-vfClpAR-b3H-7aHl":0.17750156127473993, "TdZnlG-BUgMs7Z-iBM-9c3v":0.27498394395046333, "RipJXn-p4gZkyy-1ZY-xkWe":0.054616268950389735, "ke730M-LmMjGdc-EFy-0LUK":0.30781761836448285, "jBSExJ-GXTc5TB-NSa-xBEd":0.66178278500540244, "kI7Cc8-DSg5RdF-qLo-2bhe":0.98357074613234885, "bAn3VI-x6xXWpB-zWe-G5CJ":0.21798212299794562, "jAil30-kbt6K6z-kbr-8foB":0.97880669772451379, "IHIwNs-1QGqy8l-i8i-vu4G":0.49679397412450588, "p0IbZr-tHCtwiV-0hq-NtIt":0.050183795109057017, "iggdij-M3YNBpd-yiD-a8Ro":0.982385582884686, "BrJEww-C4LpgaS-AeB-So4U":0.90248554155536553, "xnO3Fi-8rXcpgj-zpm-EmuX":0.20529118817468572, "5w57da-phYtDUx-px2-6frG":0.29690638791565971, "31MfFs-1WyUAr6-gQ0-xLxY":0.48795551283135086, "ryBl2p-rSoPhwd-WPv-NCAU":0.79544854844958068, "KN5TEt-gOfJ4Hy-3pp-HiBa":0.1533389643648807, "ytqxb8-utXXjUf-m41-i6ir":0.61502086737193573, "WhGUGz-zzyvEpD-9BM-2bVf":0.581040090228354, "dE1tFe-zHClt4u-0cY-TQnC":0.76089996323694564, "MveBhC-g29c0dU-tCT-R6nC":0.33457340282218506, "JTpxue-xSqAhGo-AZk-zB1t":0.35040302774880538, "92TVdU-qDJesPN-0lb-JOd3":0.73876949983198048, "0PODnh-IciBdOZ-0CS-oNeL":0.95159059657696443, "KkkW6x-TiemXQw-OiH-dZ9s":0.40824123319990813, "PIs5Aj-g02HRXw-957-GD2z":0.641526116451016, "yJIzuw-au6460e-0Tl-XYEJ":0.75219285303562355, "KHvMCD-OQDL0eX-nqK-TmEt":0.13096167278968263, "6QJJgV-Z3IZ1Rf-wyv-rIJ6":0.70071103877259622, "qA9ycc-sR2qm6P-PtB-AIax":0.44629776556459089, "uDeuEb-B0t0Ljr-dWk-jkC4":0.69046727674079578, "5vPy52-ygN0MMH-UB4-nZQL":0.60575965422000211, "zbbmrQ-pT3uAuU-Kae-HjM5":0.98126574986862791, "3QShHS-7RwUB10-0W2-H4Qy":0.41557608488608533, "PMc4QI-5lNajXU-f8m-RGIi":0.70464209768002883, "O9t3dl-q8YHozj-saR-A3Jm":0.85433449541965856, "k4eH3O-aHnTKY7-ADp-4Vsi":0.26558324547185574, "RA4epe-lWWnOff-bpM-bSR4":0.75232522102228827, "6ysu2R-gSc5dwU-cv0-LqCJ":0.78308993227167323, "tVl3TY-o42NMVO-k3S-iqOY":0.79238234012157993, "NMgTrr-W1RrCvP-Zaf-paL7":0.46869286547569355, "d1CJmF-CeG5asM-xms-1dwN":0.76229087810764928, "N1D30g-zFjiGzI-eHC-Sof4":0.847542878440137, "tOhfKu-Gdtf9Ne-KwA-JdHV":0.49992852174451541, "XLzwK0-6ocGDrS-TtU-wlEI":0.39853544027050947, "XDgZfb-Sxc45Zn-mVO-S2QO":0.057915803376441866, "GQD7a0-fnt9BZs-Kvh-dPbJ":0.663903859916476, "9dJxj9-HFwEQMY-6p9-s8Vt":0.21944075953054343, "1qU9pA-QJGAna9-JoG-H7GS":0.88774019472953825, "rKIkxA-UnGWYSn-0li-ziuB":0.16079062750364659, "tbPazx-IjUrQ8J-NZe-VOPL":0.68091669167975932, "xBpSIv-U6ojkK7-9p5-LviD":0.11956726473799006, "88bnWI-pxrKa7T-n2d-tXk9":0.19560689517877206, "0XviXp-9ksT8s0-fDy-35SW":0.8690659418822626, "e0XauA-GNRALmd-SM2-Y4Gf":0.68408168887520893, "kyvYBk-Bk5M4Xq-gxX-kE1B":0.7744771682336401, "dIiQzS-5sT4ogL-6IV-tLmb":0.0340772833497166, "OlGOyH-dyL1nzj-B2M-z8ir":0.3765608037933722, "zC9Gtn-x8hpfPD-KOu-k31W":0.864392047887076, "qSq3z2-Lpv0YcB-hBq-Sabd":0.15428476092466781, "LSyNyi-tBZUx1l-hAj-mwsx":0.30403432829870103, "2c9aTP-hXloMK7-ufH-dgq6":0.10168525529531069, "aXksHO-zARQxfo-sgS-8Bf4":0.54905330820199594, "ioOXAL-eVUF0W8-vZx-ZeYX":0.45281640384817845, "DXUkAP-A7SqnHj-V4U-PJfz":0.36074074474259388, "cnzZXk-AOMepfN-hym-qbDH":0.45873615005925683, "CMlAd6-8FF1yXs-fae-Izfv":0.075550197208259173, "qiXnUv-e2PsJWm-tLF-KpjE":0.94096810653636875, "Gfx3k9-JvXa7Wd-rI1-1e1E":0.74927933121782264} {"name": "r8HXXQM4XHoI", "age": 238221053, "tip": "2023-07-26 15:40:37.694000"} +1077 [0.7805560995873845, 0.9303489002269559, 0.2529522997521877, 0.662270811026298, 0.664725297532439, 0.1019441091764477, 0.9614059300688174, 0.5278126009983843, 0.5287505841216708, 0.426116738236779, 0.42300502393871175, 0.5327026330053651, 0.6025481777942603, 0.2710733647257627, 0.613792118138183, 0.0021003027835629906, 0.3200675048728582, 0.5485611014660204, 0.5121510581313707, 0.5145136652805358] {"9wXr9n-TBm9Wyt-r8H-SkAq":0.9338329010480995, "CPDH4G-ZXGPkku-3wY-ktaQ":0.4355256963350881, "RvNlMt-HHjHN5M-VjP-xHAI":0.3263474611804782, "qKIhKy-Ws344os-haX-2pmT":0.565450203625137, "DOJJ5l-UEkwVMs-x9F-HifD":0.09375622010822238, "m871g8-1eFi7jt-oBq-S0yc":0.8819687247951038, "wXugVP-v2fc6IF-DeU-On3T":0.3448233486447311, "B0mXFX-QvgUgo7-Dih-6rDu":0.1914040395475467, "E9zv3F-xMqSbMa-il4-FuDg":0.38570218910843357, "msuFIN-ZkKO8TY-tu4-veH0":0.6646172653074628, "0rSUyl-Un07aIW-KAx-WHnX":0.35580099104309737, "XvbmO8-WA6oAqc-ihc-s8IL":0.4058206434411423, "G6B6RD-AicAlZb-16u-Pn1I":0.7203554946895749, "coDK0Q-tMg1294-JMQ-ZWQu":0.8236328627743186, "4c0aWh-yhL6BOX-rRu-1n0r":0.13980911842304278, "G4iUcG-ZhWw62v-VLt-n6lH":0.18382889782542144, "IIB7qD-WQistwT-Vux-0c9B":0.9174389144309458, "7cTyuR-5ssXm2S-sJR-JTIZ":0.8132237242672837, "3KPhSW-FICEImf-bba-PCiQ":0.6302643579943553, "qQ7Yup-XBeQGFz-3EP-q0vd":0.6109025726752364, "gjRxRo-Af9Oqx5-IzN-3B9d":0.9251468490326916, "1zSj57-nNZpZ0b-ZKn-BeY0":0.5628463109107144, "sTK0mn-wkp1Xp5-PRS-txVM":0.7905808129559996, "sLrM0s-1KnXLb6-1A3-Z1vJ":0.4234598677670157, "UkYdkP-k7YKiKS-Fxp-qAcI":0.7541401266679869, "v8p0YV-R5pAKZ8-UMr-P1bQ":0.2931152565110683, "RJdTav-jk3os9Z-yRk-WhwV":0.5263811309738877, "lB91ic-pNFZkE4-hBx-e104":0.6692292834321788, "gmRV6e-GKJUg0L-ok7-J6Lz":0.059247669596643515, "o3LUyz-7Toh54O-czG-Xep8":0.6284193821127264, "8fzHhM-4otPAss-qTm-phg8":0.8953002441537012, "kZsHhe-vfClpAR-b3H-7aHl":0.17750156127473993, "TdZnlG-BUgMs7Z-iBM-9c3v":0.2749839439504633, "RipJXn-p4gZkyy-1ZY-xkWe":0.054616268950389735, "ke730M-LmMjGdc-EFy-0LUK":0.30781761836448285, "jBSExJ-GXTc5TB-NSa-xBEd":0.6617827850054024, "kI7Cc8-DSg5RdF-qLo-2bhe":0.9835707461323488, "bAn3VI-x6xXWpB-zWe-G5CJ":0.21798212299794562, "jAil30-kbt6K6z-kbr-8foB":0.9788066977245138, "IHIwNs-1QGqy8l-i8i-vu4G":0.4967939741245059, "p0IbZr-tHCtwiV-0hq-NtIt":0.05018379510905702, "iggdij-M3YNBpd-yiD-a8Ro":0.982385582884686, "BrJEww-C4LpgaS-AeB-So4U":0.9024855415553655, "xnO3Fi-8rXcpgj-zpm-EmuX":0.20529118817468572, "5w57da-phYtDUx-px2-6frG":0.2969063879156597, "31MfFs-1WyUAr6-gQ0-xLxY":0.48795551283135086, "ryBl2p-rSoPhwd-WPv-NCAU":0.7954485484495807, "KN5TEt-gOfJ4Hy-3pp-HiBa":0.1533389643648807, "ytqxb8-utXXjUf-m41-i6ir":0.6150208673719357, "WhGUGz-zzyvEpD-9BM-2bVf":0.581040090228354, "dE1tFe-zHClt4u-0cY-TQnC":0.7608999632369456, "MveBhC-g29c0dU-tCT-R6nC":0.33457340282218506, "JTpxue-xSqAhGo-AZk-zB1t":0.3504030277488054, "92TVdU-qDJesPN-0lb-JOd3":0.7387694998319805, "0PODnh-IciBdOZ-0CS-oNeL":0.9515905965769644, "KkkW6x-TiemXQw-OiH-dZ9s":0.4082412331999081, "PIs5Aj-g02HRXw-957-GD2z":0.641526116451016, "yJIzuw-au6460e-0Tl-XYEJ":0.7521928530356236, "KHvMCD-OQDL0eX-nqK-TmEt":0.13096167278968263, "6QJJgV-Z3IZ1Rf-wyv-rIJ6":0.7007110387725962, "qA9ycc-sR2qm6P-PtB-AIax":0.4462977655645909, "uDeuEb-B0t0Ljr-dWk-jkC4":0.6904672767407958, "5vPy52-ygN0MMH-UB4-nZQL":0.6057596542200021, "zbbmrQ-pT3uAuU-Kae-HjM5":0.9812657498686279, "3QShHS-7RwUB10-0W2-H4Qy":0.41557608488608533, "PMc4QI-5lNajXU-f8m-RGIi":0.7046420976800288, "O9t3dl-q8YHozj-saR-A3Jm":0.8543344954196586, "k4eH3O-aHnTKY7-ADp-4Vsi":0.26558324547185574, "RA4epe-lWWnOff-bpM-bSR4":0.7523252210222883, "6ysu2R-gSc5dwU-cv0-LqCJ":0.7830899322716732, "tVl3TY-o42NMVO-k3S-iqOY":0.7923823401215799, "NMgTrr-W1RrCvP-Zaf-paL7":0.46869286547569355, "d1CJmF-CeG5asM-xms-1dwN":0.7622908781076493, "N1D30g-zFjiGzI-eHC-Sof4":0.847542878440137, "tOhfKu-Gdtf9Ne-KwA-JdHV":0.4999285217445154, "XLzwK0-6ocGDrS-TtU-wlEI":0.39853544027050947, "XDgZfb-Sxc45Zn-mVO-S2QO":0.057915803376441866, "GQD7a0-fnt9BZs-Kvh-dPbJ":0.663903859916476, "9dJxj9-HFwEQMY-6p9-s8Vt":0.21944075953054343, "1qU9pA-QJGAna9-JoG-H7GS":0.8877401947295382, "rKIkxA-UnGWYSn-0li-ziuB":0.1607906275036466, "tbPazx-IjUrQ8J-NZe-VOPL":0.6809166916797593, "xBpSIv-U6ojkK7-9p5-LviD":0.11956726473799006, "88bnWI-pxrKa7T-n2d-tXk9":0.19560689517877206, "0XviXp-9ksT8s0-fDy-35SW":0.8690659418822626, "e0XauA-GNRALmd-SM2-Y4Gf":0.6840816888752089, "kyvYBk-Bk5M4Xq-gxX-kE1B":0.7744771682336401, "dIiQzS-5sT4ogL-6IV-tLmb":0.0340772833497166, "OlGOyH-dyL1nzj-B2M-z8ir":0.3765608037933722, "zC9Gtn-x8hpfPD-KOu-k31W":0.864392047887076, "qSq3z2-Lpv0YcB-hBq-Sabd":0.1542847609246678, "LSyNyi-tBZUx1l-hAj-mwsx":0.30403432829870103, "2c9aTP-hXloMK7-ufH-dgq6":0.1016852552953107, "aXksHO-zARQxfo-sgS-8Bf4":0.5490533082019959, "ioOXAL-eVUF0W8-vZx-ZeYX":0.45281640384817845, "DXUkAP-A7SqnHj-V4U-PJfz":0.3607407447425939, "cnzZXk-AOMepfN-hym-qbDH":0.45873615005925683, "CMlAd6-8FF1yXs-fae-Izfv":0.07555019720825917, "qiXnUv-e2PsJWm-tLF-KpjE":0.9409681065363688, "Gfx3k9-JvXa7Wd-rI1-1e1E":0.7492793312178226} {"name": "r8HXXQM4XHoI", "age": 238221053, "tip": "2023-07-26 15:40:37.694000"} -- !array_max_orc -- 11028 diff --git a/regression-test/data/external_table_p2/hive/test_external_credit_data.out b/regression-test/data/external_table_p2/hive/test_external_credit_data.out index 3f5163310e3cc2..a80520330f6fb9 100644 --- a/regression-test/data/external_table_p2/hive/test_external_credit_data.out +++ b/regression-test/data/external_table_p2/hive/test_external_credit_data.out @@ -1,13 +1,13 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !credit_data -- +2023-03 0 0 4 0 0 0 No Loan bal 0 1 2023-03 0 0 4 0 3 3 25%-50% 0 1 2023-03 0 0 4 2 3 3 80%-90% 6 1 -2023-04 0 1 4 0 2 3 25%-50% 0 1 2023-04 0 0 3 1 0 0 No Loan bal 2 1 +2023-04 0 1 4 0 2 3 25%-50% 0 1 +2023-05 0 0 4 0 1 1 0%-10% 0 1 2023-05 0 0 4 0 1 1 25%-50% 0 1 +2023-05 0 0 4 0 1 1 50%-80% 0 1 2023-05 0 0 4 0 2 2 25%-50% 0 1 2023-05 1 0 3 1 0 0 No Loan bal 0 1 -2023-05 0 0 4 0 1 1 50%-80% 0 1 -2023-05 0 0 4 0 1 1 0%-10% 0 1 -2023-03 0 0 4 0 0 0 No Loan bal 0 1 diff --git a/regression-test/data/external_table_p2/hive/test_hive_to_array.out b/regression-test/data/external_table_p2/hive/test_hive_to_array.out index be2f15df94a466..0965e8d8be4fb5 100644 --- a/regression-test/data/external_table_p2/hive/test_hive_to_array.out +++ b/regression-test/data/external_table_p2/hive/test_hive_to_array.out @@ -3,8 +3,8 @@ [1, 2, 3, 4] [10, 20, 30, 40] [100, 200, 300] [100000000000000, 20000000000000, 30000000000000, 40000000000000] [1, 0, 1] [1.23, 4.56, 7.89] [10.1, 20.2, 30.3] ["apple", "banana", "orange"] ["2023-07-04 12:00:00.000000", "2023-07-05 12:00:00.000000", "2023-07-06 12:00:00.000000"] ["2023-07-04", "2023-07-05", "2023-07-06"] [1111111.111, 2111111.111, 3111111.111] ["a", "b", "c"] ["aa", "bb", "cc"] [10, 20, 30] [100, 200, 300, 400] [1000, 2000, 3000] [1000000000000000, 200000000000000, 300000000000000, 400000000000000] [1, 1, 1, 1] [12.3, 45.6, 78.9] [100.1, 200.2, 300.3] ["grapes", "watermelon", "kiwi"] ["2023-07-03 12:00:00.000000", "2023-07-03 12:00:00.000000", "2023-07-03 12:00:00.000000"] ["2021-07-05", "2021-07-05", "2021-07-05"] [2222222.111, 2222222.112, 2222222.113] \N \N [20, 30, 40, 50] [200, 300, 400, 500] [10000, 20000, 30000] [100000000000000, 20000000000000, 30000000000000] [1, 1, 1, 1, 0, 0] [120.3, 450.6, 780.9] [100.001, 200.002, 300.003] ["melon", "strawberry", "blueberry"] ["2023-07-02 12:00:00.000000", "2023-07-02 12:00:00.000000", "2023-07-02 12:00:00.000000"] ["2021-07-06", "2021-07-06", "2021-07-06"] [1111111.111, 2111111.111, 3111111.111] \N \N -[40, 50, 60, 70] [210, 310, 410, 510] [110000, 210000, 310000] [400000000000000, 50000000000000, 60000000000000] [1] [120.301, 450.602, 780.90009] [100.0000001, 200.0000002, 300.0000003] ["hello", "world"] ["2023-07-02 12:00:00.000000", "2023-07-02 12:00:00.000000", "2023-07-02 12:00:00.000000"] ["2021-07-06", "2021-07-06", "2021-07-06"] [3311111.111, 2211111.111, 3121111.111] ["1"] ["hello", "world"] -[4, 5, 6, 7] [2100, 3100, 4100, 5100] [110000, 220000, 330000] [60000000000000, 60000000000000, 60000000000000] [1] [120.301, 450.602, 780.90009] [100.0000001, 200.0000002, 300.0000003] ["hive", "text", "file", "format"] ["2023-07-09 12:00:00.000000", "2023-07-09 12:00:00.000000", "2023-07-09 12:00:00.000000"] ["2021-07-09", "2021-07-09", "2021-07-09"] [3311111.111, 2211111.111, 3121111.111] ["d", "d", "d", "d"] ["ffffffff"] +[4, 5, 6, 7] [2100, 3100, 4100, 5100] [110000, 220000, 330000] [60000000000000, 60000000000000, 60000000000000] [1] [120.301, 450.602, 780.9001] [100.0000001, 200.0000002, 300.0000003] ["hive", "text", "file", "format"] ["2023-07-09 12:00:00.000000", "2023-07-09 12:00:00.000000", "2023-07-09 12:00:00.000000"] ["2021-07-09", "2021-07-09", "2021-07-09"] [3311111.111, 2211111.111, 3121111.111] ["d", "d", "d", "d"] ["ffffffff"] +[40, 50, 60, 70] [210, 310, 410, 510] [110000, 210000, 310000] [400000000000000, 50000000000000, 60000000000000] [1] [120.301, 450.602, 780.9001] [100.0000001, 200.0000002, 300.0000003] ["hello", "world"] ["2023-07-02 12:00:00.000000", "2023-07-02 12:00:00.000000", "2023-07-02 12:00:00.000000"] ["2021-07-06", "2021-07-06", "2021-07-06"] [3311111.111, 2211111.111, 3121111.111] ["1"] ["hello", "world"] -- !array -- 1 [[[1, 2, 3], [4, 5, 6], [7, 8, 9]], [[10, 11, 12], [13, 14, 15], [16, 17, 18]]] @@ -17,6 +17,6 @@ [1, 2, 3, 4] [10, 20, 30, 40] [100, 200, 300] [100000000000000, 20000000000000, 30000000000000, 40000000000000] [1, 0, 1] [1.23, 4.56, 7.89] [10.1, 20.2, 30.3] ["apple", "banana", "orange"] ["2023-07-04 12:00:00.000000", "2023-07-05 12:00:00.000000", "2023-07-06 12:00:00.000000"] ["2023-07-04", "2023-07-05", "2023-07-06"] [1111111.111, 2111111.111, 3111111.111] 1 [[[1, 2, 3], [4, 5, 6], [7, 8, 9]], [[10, 11, 12], [13, 14, 15], [16, 17, 18]]] [10, 20, 30] [100, 200, 300, 400] [1000, 2000, 3000] [1000000000000000, 200000000000000, 300000000000000, 400000000000000] [1, 1, 1, 1] [12.3, 45.6, 78.9] [100.1, 200.2, 300.3] ["grapes", "watermelon", "kiwi"] ["2023-07-03 12:00:00.000000", "2023-07-03 12:00:00.000000", "2023-07-03 12:00:00.000000"] ["2021-07-05", "2021-07-05", "2021-07-05"] [2222222.111, 2222222.112, 2222222.113] 2 [[[19, 20, 21], [22, 23, 24], [25, 26, 27]], [[28], [31], [34]], [[28, 29], [31, 32], [34, 35]]] [20, 30, 40, 50] [200, 300, 400, 500] [10000, 20000, 30000] [100000000000000, 20000000000000, 30000000000000] [1, 1, 1, 1, 0, 0] [120.3, 450.6, 780.9] [100.001, 200.002, 300.003] ["melon", "strawberry", "blueberry"] ["2023-07-02 12:00:00.000000", "2023-07-02 12:00:00.000000", "2023-07-02 12:00:00.000000"] ["2021-07-06", "2021-07-06", "2021-07-06"] [1111111.111, 2111111.111, 3111111.111] 3 [[[1, 2, 3], [4, 5, 6], [7, 8, 9]]] -[40, 50, 60, 70] [210, 310, 410, 510] [110000, 210000, 310000] [400000000000000, 50000000000000, 60000000000000] [1] [120.301, 450.602, 780.90009] [100.0000001, 200.0000002, 300.0000003] ["hello", "world"] ["2023-07-02 12:00:00.000000", "2023-07-02 12:00:00.000000", "2023-07-02 12:00:00.000000"] ["2021-07-06", "2021-07-06", "2021-07-06"] [3311111.111, 2211111.111, 3121111.111] 4 [[[1]]] -[4, 5, 6, 7] [2100, 3100, 4100, 5100] [110000, 220000, 330000] [60000000000000, 60000000000000, 60000000000000] [1] [120.301, 450.602, 780.90009] [100.0000001, 200.0000002, 300.0000003] ["hive", "text", "file", "format"] ["2023-07-09 12:00:00.000000", "2023-07-09 12:00:00.000000", "2023-07-09 12:00:00.000000"] ["2021-07-09", "2021-07-09", "2021-07-09"] [3311111.111, 2211111.111, 3121111.111] 5 [[[1, 2, 3], [4, 5, 6], [7, 8, 9]], [[10, 11, 12], [13, 14, 15], [16, 17, 18]], [[1, 2, 3], [4, 5, 6], [7, 8, 9]], [[10, 11, 12], [13, 14, 15], [16, 17, 18]]] +[4, 5, 6, 7] [2100, 3100, 4100, 5100] [110000, 220000, 330000] [60000000000000, 60000000000000, 60000000000000] [1] [120.301, 450.602, 780.9001] [100.0000001, 200.0000002, 300.0000003] ["hive", "text", "file", "format"] ["2023-07-09 12:00:00.000000", "2023-07-09 12:00:00.000000", "2023-07-09 12:00:00.000000"] ["2021-07-09", "2021-07-09", "2021-07-09"] [3311111.111, 2211111.111, 3121111.111] 5 [[[1, 2, 3], [4, 5, 6], [7, 8, 9]], [[10, 11, 12], [13, 14, 15], [16, 17, 18]], [[1, 2, 3], [4, 5, 6], [7, 8, 9]], [[10, 11, 12], [13, 14, 15], [16, 17, 18]]] +[40, 50, 60, 70] [210, 310, 410, 510] [110000, 210000, 310000] [400000000000000, 50000000000000, 60000000000000] [1] [120.301, 450.602, 780.9001] [100.0000001, 200.0000002, 300.0000003] ["hello", "world"] ["2023-07-02 12:00:00.000000", "2023-07-02 12:00:00.000000", "2023-07-02 12:00:00.000000"] ["2021-07-06", "2021-07-06", "2021-07-06"] [3311111.111, 2211111.111, 3121111.111] 4 [[[1]]] diff --git a/regression-test/data/insert_overwrite_p0/insert_overwrite_auto_detect.out b/regression-test/data/insert_overwrite_p0/insert_overwrite_auto_detect.out index 3cde86880473dc..51e06f3f7bfe85 100644 --- a/regression-test/data/insert_overwrite_p0/insert_overwrite_auto_detect.out +++ b/regression-test/data/insert_overwrite_p0/insert_overwrite_auto_detect.out @@ -75,6 +75,19 @@ LIST SHANGHAI XXX +-- !sql -- +7654321 +BEIJING +LIST +SHANGHAI +XXX + +-- !sql -- +Beijing + +-- !sql -- +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa + -- !sql -- 2008-01-01 2008-02-02 diff --git a/regression-test/data/insert_overwrite_p1/test_iot_auto_detect_concurrent.out b/regression-test/data/insert_overwrite_p1/test_iot_auto_detect_concurrent.out new file mode 100644 index 00000000000000..3c8a25336daf5d --- /dev/null +++ b/regression-test/data/insert_overwrite_p1/test_iot_auto_detect_concurrent.out @@ -0,0 +1,10 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !sql3 -- +100 + +-- !sql4 -- +100 + +-- !sql5 -- +0 + diff --git a/regression-test/data/mtmv_p0/test_null_partition_mtmv.out b/regression-test/data/mtmv_p0/test_null_partition_mtmv.out new file mode 100644 index 00000000000000..f341905773fc60 --- /dev/null +++ b/regression-test/data/mtmv_p0/test_null_partition_mtmv.out @@ -0,0 +1,19 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !list_null -- +2 \N + +-- !list_1 -- +1 1 + +-- !range_p_32768_10 -- +1 \N + +-- !range_p_10_20 -- +2 15 + +-- !range_p_00000101_20201111 -- +1 \N + +-- !range_p_20201111_20211111 -- +2 2021-01-01 + diff --git a/regression-test/data/nereids_function_p0/agg_function/agg.out b/regression-test/data/nereids_function_p0/agg_function/agg.out index bfd27bf07d95ca..fc4dd3c6118c8d 100644 --- a/regression-test/data/nereids_function_p0/agg_function/agg.out +++ b/regression-test/data/nereids_function_p0/agg_function/agg.out @@ -265,7 +265,7 @@ -- !sql_avg_Double_gb -- \N -0.39999999999999997 +0.4 1.0 -- !sql_avg_Double -- @@ -298,7 +298,7 @@ 12 0.65 -- !sql_avg_Double_gb_notnull -- -0.39999999999999997 +0.4 1.0 -- !sql_avg_Double_notnull -- @@ -323,7 +323,7 @@ -- !sql_avg_Double_agg_phase_3_notnull -- 0 \N -7 0.39999999999999997 +7 0.4 5 1.0 -- !sql_avg_Double_agg_phase_4_notnull -- @@ -397,11 +397,11 @@ -- !sql_avg_weighted_TinyInt_Double_gb -- \N -5.0 +5.000000000000001 10.2 -- !sql_avg_weighted_TinyInt_Double -- -8.333333333333332 +8.333333333333334 -- !sql_avg_weighted_TinyInt_Double_agg_phase_1 -- 0 \N @@ -419,22 +419,22 @@ 1 12.0 -- !sql_avg_weighted_TinyInt_Double_agg_phase_2 -- -12 8.333333333333334 +12 8.333333333333332 -- !sql_avg_weighted_TinyInt_Double_agg_phase_3 -- 0 \N -7 5.0 +7 4.999999999999999 5 10.2 -- !sql_avg_weighted_TinyInt_Double_agg_phase_4 -- -12 8.333333333333334 +12 8.333333333333332 -- !sql_avg_weighted_TinyInt_Double_gb_notnull -- -5.0 -10.200000000000001 +4.999999999999999 +10.2 -- !sql_avg_weighted_TinyInt_Double_notnull -- -8.333333333333334 +8.333333333333332 -- !sql_avg_weighted_TinyInt_Double_agg_phase_1_notnull -- 1 1.0 @@ -451,19 +451,19 @@ 1 12.0 -- !sql_avg_weighted_TinyInt_Double_agg_phase_2_notnull -- -12 8.333333333333332 +12 8.333333333333334 -- !sql_avg_weighted_TinyInt_Double_agg_phase_3_notnull -- 0 \N -7 5.0 +7 4.999999999999999 5 10.2 -- !sql_avg_weighted_TinyInt_Double_agg_phase_4_notnull -- -12 8.333333333333332 +12 8.333333333333334 -- !sql_avg_weighted_SmallInt_Double_gb -- \N -4.999999999999999 +5.0 10.2 -- !sql_avg_weighted_SmallInt_Double -- @@ -485,18 +485,18 @@ 1 12.0 -- !sql_avg_weighted_SmallInt_Double_agg_phase_2 -- -12 8.333333333333332 +12 8.333333333333334 -- !sql_avg_weighted_SmallInt_Double_agg_phase_3 -- 0 \N -7 5.0 +7 4.999999999999999 5 10.2 -- !sql_avg_weighted_SmallInt_Double_agg_phase_4 -- -12 8.333333333333334 +12 8.333333333333332 -- !sql_avg_weighted_SmallInt_Double_gb_notnull -- -5.0 +4.999999999999999 10.2 -- !sql_avg_weighted_SmallInt_Double_notnull -- @@ -517,19 +517,19 @@ 1 12.0 -- !sql_avg_weighted_SmallInt_Double_agg_phase_2_notnull -- -12 8.333333333333332 +12 8.333333333333334 -- !sql_avg_weighted_SmallInt_Double_agg_phase_3_notnull -- 0 \N -7 5.0 +7 4.999999999999999 5 10.2 -- !sql_avg_weighted_SmallInt_Double_agg_phase_4_notnull -- -12 8.333333333333332 +12 8.333333333333334 -- !sql_avg_weighted_Integer_Double_gb -- \N -4.999999999999999 +5.000000000000001 10.2 -- !sql_avg_weighted_Integer_Double -- @@ -551,7 +551,7 @@ 1 12.0 -- !sql_avg_weighted_Integer_Double_agg_phase_2 -- -12 8.333333333333334 +12 8.333333333333332 -- !sql_avg_weighted_Integer_Double_agg_phase_3 -- 0 \N @@ -559,14 +559,14 @@ 5 10.2 -- !sql_avg_weighted_Integer_Double_agg_phase_4 -- -12 8.333333333333334 +12 8.333333333333332 -- !sql_avg_weighted_Integer_Double_gb_notnull -- -4.999999999999999 -10.2 +5.0 +10.199999999999998 -- !sql_avg_weighted_Integer_Double_notnull -- -8.333333333333332 +8.333333333333334 -- !sql_avg_weighted_Integer_Double_agg_phase_1_notnull -- 1 1.0 @@ -583,12 +583,12 @@ 1 12.0 -- !sql_avg_weighted_Integer_Double_agg_phase_2_notnull -- -12 8.333333333333334 +12 8.333333333333332 -- !sql_avg_weighted_Integer_Double_agg_phase_3_notnull -- 0 \N 7 5.0 -5 10.199999999999998 +5 10.2 -- !sql_avg_weighted_Integer_Double_agg_phase_4_notnull -- 12 8.333333333333332 @@ -617,7 +617,7 @@ 1 12.0 -- !sql_avg_weighted_BigInt_Double_agg_phase_2 -- -12 8.333333333333332 +12 8.333333333333334 -- !sql_avg_weighted_BigInt_Double_agg_phase_3 -- 0 \N @@ -628,11 +628,11 @@ 12 8.333333333333334 -- !sql_avg_weighted_BigInt_Double_gb_notnull -- -5.000000000000001 +4.999999999999999 10.2 -- !sql_avg_weighted_BigInt_Double_notnull -- -8.333333333333332 +8.333333333333334 -- !sql_avg_weighted_BigInt_Double_agg_phase_1_notnull -- 1 1.0 @@ -661,11 +661,11 @@ -- !sql_avg_weighted_Float_Double_gb -- \N -0.5000000045235669 +0.5000000045235667 1.0200000143051147 -- !sql_avg_weighted_Float_Double -- -0.833333344127123 +0.8333333441271231 -- !sql_avg_weighted_Float_Double_agg_phase_1 -- 0 \N @@ -683,7 +683,7 @@ 1 1.2000000476837158 -- !sql_avg_weighted_Float_Double_agg_phase_2 -- -12 0.8333333441271231 +12 0.833333344127123 -- !sql_avg_weighted_Float_Double_agg_phase_3 -- 0 \N @@ -691,14 +691,14 @@ 5 1.0200000143051147 -- !sql_avg_weighted_Float_Double_agg_phase_4 -- -12 0.8333333441271233 +12 0.833333344127123 -- !sql_avg_weighted_Float_Double_gb_notnull -- -0.5000000045235666 +0.5000000045235669 1.0200000143051147 -- !sql_avg_weighted_Float_Double_notnull -- -0.8333333441271233 +0.833333344127123 -- !sql_avg_weighted_Float_Double_agg_phase_1_notnull -- 1 0.10000000149011612 @@ -727,11 +727,11 @@ -- !sql_avg_weighted_Double_Double_gb -- \N -0.5 +0.49999999999999994 1.02 -- !sql_avg_weighted_Double_Double -- -0.8333333333333333 +0.8333333333333334 -- !sql_avg_weighted_Double_Double_agg_phase_1 -- 0 \N @@ -749,22 +749,22 @@ 1 1.2 -- !sql_avg_weighted_Double_Double_agg_phase_2 -- -12 0.8333333333333334 +12 0.8333333333333335 -- !sql_avg_weighted_Double_Double_agg_phase_3 -- 0 \N 7 0.5000000000000001 -5 1.0199999999999998 +5 1.02 -- !sql_avg_weighted_Double_Double_agg_phase_4 -- -12 0.8333333333333333 +12 0.8333333333333334 -- !sql_avg_weighted_Double_Double_gb_notnull -- 0.5 -1.0199999999999998 +1.02 -- !sql_avg_weighted_Double_Double_notnull -- -0.8333333333333335 +0.8333333333333331 -- !sql_avg_weighted_Double_Double_agg_phase_1_notnull -- 1 0.10000000000000002 @@ -781,23 +781,23 @@ 1 1.2 -- !sql_avg_weighted_Double_Double_agg_phase_2_notnull -- -12 0.8333333333333333 +12 0.8333333333333334 -- !sql_avg_weighted_Double_Double_agg_phase_3_notnull -- 0 \N -7 0.49999999999999994 +7 0.5000000000000001 5 1.02 -- !sql_avg_weighted_Double_Double_agg_phase_4_notnull -- -12 0.8333333333333335 +12 0.8333333333333333 -- !sql_avg_weighted_DecimalV2_Double_gb -- \N -0.5000000000000001 +0.5 1.02 -- !sql_avg_weighted_DecimalV2_Double -- -0.8333333333333334 +0.8333333333333333 -- !sql_avg_weighted_DecimalV2_Double_agg_phase_1 -- 0 \N @@ -819,7 +819,7 @@ -- !sql_avg_weighted_DecimalV2_Double_agg_phase_3 -- 0 \N -7 0.5000000000000001 +7 0.49999999999999994 5 1.02 -- !sql_avg_weighted_DecimalV2_Double_agg_phase_4 -- @@ -827,7 +827,7 @@ -- !sql_avg_weighted_DecimalV2_Double_gb_notnull -- 0.5 -1.02 +1.0199999999999998 -- !sql_avg_weighted_DecimalV2_Double_notnull -- 0.8333333333333333 @@ -852,10 +852,10 @@ -- !sql_avg_weighted_DecimalV2_Double_agg_phase_3_notnull -- 0 \N 7 0.5000000000000001 -5 1.02 +5 1.0199999999999998 -- !sql_avg_weighted_DecimalV2_Double_agg_phase_4_notnull -- -12 0.8333333333333333 +12 0.8333333333333334 -- !sql_bitmap_intersect_Bitmap_gb -- \N @@ -3327,7 +3327,7 @@ true -- !sql_stddev_TinyInt_agg_phase_3 -- 0 \N -7 2.0 +7 1.9999999999999998 5 1.4142135623730951 -- !sql_stddev_TinyInt_agg_phase_4 -- @@ -3367,7 +3367,7 @@ true -- !sql_stddev_SmallInt_gb -- \N -1.9999999999999998 +2.0 1.4142135623730951 -- !sql_stddev_SmallInt -- @@ -3393,7 +3393,7 @@ true -- !sql_stddev_SmallInt_agg_phase_3 -- 0 \N -7 2.0 +7 1.9999999999999998 5 1.4142135623730951 -- !sql_stddev_SmallInt_agg_phase_4 -- @@ -3426,7 +3426,7 @@ true -- !sql_stddev_SmallInt_agg_phase_3_notnull -- 0 \N 7 2.0 -5 1.4142135623730951 +5 1.4142135623730954 -- !sql_stddev_SmallInt_agg_phase_4_notnull -- 12 3.452052529534663 @@ -3434,10 +3434,10 @@ true -- !sql_stddev_Integer_gb -- \N 1.9999999999999998 -1.414213562373095 +1.4142135623730951 -- !sql_stddev_Integer -- -3.452052529534664 +3.452052529534663 -- !sql_stddev_Integer_agg_phase_1 -- 0 \N @@ -3459,7 +3459,7 @@ true -- !sql_stddev_Integer_agg_phase_3 -- 0 \N -7 1.9999999999999998 +7 2.0 5 1.4142135623730951 -- !sql_stddev_Integer_agg_phase_4 -- @@ -3491,7 +3491,7 @@ true -- !sql_stddev_Integer_agg_phase_3_notnull -- 0 \N -7 2.0 +7 1.9999999999999998 5 1.4142135623730951 -- !sql_stddev_Integer_agg_phase_4_notnull -- @@ -3503,7 +3503,7 @@ true 1.4142135623730951 -- !sql_stddev_BigInt -- -3.452052529534664 +3.452052529534663 -- !sql_stddev_BigInt_agg_phase_1 -- 0 \N @@ -3587,7 +3587,7 @@ true 1 0.0 -- !sql_stddev_Float_agg_phase_2 -- -12 0.3452052585470726 +12 0.34520525854707257 -- !sql_stddev_Float_agg_phase_3 -- 0 \N @@ -3595,7 +3595,7 @@ true 5 0.1414213730960499 -- !sql_stddev_Float_agg_phase_4 -- -12 0.3452052585470726 +12 0.34520525854707257 -- !sql_stddev_Float_gb_notnull -- 0.1999999992549422 @@ -3631,7 +3631,7 @@ true -- !sql_stddev_Double_gb -- \N -0.19999999999999998 +0.2 0.1414213562373095 -- !sql_stddev_Double -- @@ -3653,7 +3653,7 @@ true 1 0.0 -- !sql_stddev_Double_agg_phase_2 -- -12 0.34520525295346627 +12 0.3452052529534663 -- !sql_stddev_Double_agg_phase_3 -- 0 \N @@ -3685,11 +3685,11 @@ true 1 0.0 -- !sql_stddev_Double_agg_phase_2_notnull -- -12 0.34520525295346627 +12 0.3452052529534663 -- !sql_stddev_Double_agg_phase_3_notnull -- 0 \N -7 0.19999999999999996 +7 0.2 5 0.1414213562373095 -- !sql_stddev_Double_agg_phase_4_notnull -- @@ -3716,18 +3716,18 @@ true 1 0.0 -- !sql_stddev_DecimalV2_agg_phase_2 -- -12 0.34520525295346627 +12 0.3452052529534663 -- !sql_stddev_DecimalV2_agg_phase_3 -- 0 \N -7 0.19999999999999996 +7 0.19999999999999998 5 0.1414213562373095 -- !sql_stddev_DecimalV2_agg_phase_4 -- 12 0.3452052529534663 -- !sql_stddev_DecimalV2_gb_notnull -- -0.19999999999999998 +0.19999999999999996 0.1414213562373095 -- !sql_stddev_DecimalV2_agg_phase_1_notnull -- @@ -3745,11 +3745,11 @@ true 1 0.0 -- !sql_stddev_DecimalV2_agg_phase_2_notnull -- -12 0.34520525295346627 +12 0.3452052529534663 -- !sql_stddev_DecimalV2_agg_phase_3_notnull -- 0 \N -7 0.19999999999999998 +7 0.19999999999999996 5 0.1414213562373095 -- !sql_stddev_DecimalV2_agg_phase_4_notnull -- @@ -3784,7 +3784,7 @@ true -- !sql_stddev_samp_TinyInt_agg_phase_3 -- 0 \N 7 2.160246899469287 -5 1.5811388300841898 +5 1.58113883008419 -- !sql_stddev_samp_TinyInt_agg_phase_4 -- 12 3.605551275463989 @@ -3794,7 +3794,7 @@ true 1.5811388300841898 -- !sql_stddev_samp_TinyInt_notnull -- -3.605551275463989 +3.6055512754639896 -- !sql_stddev_samp_TinyInt_agg_phase_1_notnull -- 1 \N @@ -3815,7 +3815,7 @@ true -- !sql_stddev_samp_TinyInt_agg_phase_3_notnull -- 0 \N -7 2.160246899469287 +7 2.1602468994692865 5 1.5811388300841898 -- !sql_stddev_samp_TinyInt_agg_phase_4_notnull -- @@ -3823,7 +3823,7 @@ true -- !sql_stddev_samp_SmallInt_gb -- \N -2.160246899469287 +2.1602468994692865 1.5811388300841898 -- !sql_stddev_samp_SmallInt -- @@ -3849,7 +3849,7 @@ true -- !sql_stddev_samp_SmallInt_agg_phase_3 -- 0 \N -7 2.1602468994692865 +7 2.160246899469287 5 1.5811388300841898 -- !sql_stddev_samp_SmallInt_agg_phase_4 -- @@ -3881,7 +3881,7 @@ true -- !sql_stddev_samp_SmallInt_agg_phase_3_notnull -- 0 \N -7 2.160246899469287 +7 2.1602468994692865 5 1.5811388300841898 -- !sql_stddev_samp_SmallInt_agg_phase_4_notnull -- @@ -3915,14 +3915,14 @@ true -- !sql_stddev_samp_Integer_agg_phase_3 -- 0 \N -7 2.160246899469287 +7 2.1602468994692865 5 1.5811388300841898 -- !sql_stddev_samp_Integer_agg_phase_4 -- 12 3.605551275463989 -- !sql_stddev_samp_Integer_gb_notnull -- -2.1602468994692865 +2.160246899469287 1.5811388300841898 -- !sql_stddev_samp_Integer_notnull -- @@ -3947,8 +3947,8 @@ true -- !sql_stddev_samp_Integer_agg_phase_3_notnull -- 0 \N -7 2.1602468994692865 -5 1.5811388300841898 +7 2.160246899469287 +5 1.58113883008419 -- !sql_stddev_samp_Integer_agg_phase_4_notnull -- 12 3.605551275463989 @@ -3956,10 +3956,10 @@ true -- !sql_stddev_samp_BigInt_gb -- \N 2.1602468994692865 -1.5811388300841893 +1.5811388300841898 -- !sql_stddev_samp_BigInt -- -3.605551275463989 +3.6055512754639896 -- !sql_stddev_samp_BigInt_agg_phase_1 -- 0 \N @@ -3981,8 +3981,8 @@ true -- !sql_stddev_samp_BigInt_agg_phase_3 -- 0 \N -7 2.160246899469287 -5 1.58113883008419 +7 2.1602468994692865 +5 1.5811388300841898 -- !sql_stddev_samp_BigInt_agg_phase_4 -- 12 3.605551275463989 @@ -3992,7 +3992,7 @@ true 1.5811388300841898 -- !sql_stddev_samp_BigInt_notnull -- -3.605551275463989 +3.6055512754639896 -- !sql_stddev_samp_BigInt_agg_phase_1_notnull -- 1 \N @@ -4054,7 +4054,7 @@ true 12 0.36055513338873013 -- !sql_stddev_samp_Float_gb_notnull -- -0.2160246891421743 +0.21602468914217424 0.15811390185706375 -- !sql_stddev_samp_Float_notnull -- @@ -4088,7 +4088,7 @@ true -- !sql_stddev_samp_Double_gb -- \N 0.21602468994692867 -0.15811388300841897 +0.15811388300841894 -- !sql_stddev_samp_Double -- 0.36055512754639896 @@ -4109,19 +4109,19 @@ true 1 \N -- !sql_stddev_samp_Double_agg_phase_2 -- -12 0.36055512754639896 +12 0.3605551275463989 -- !sql_stddev_samp_Double_agg_phase_3 -- 0 \N 7 0.21602468994692867 -5 0.15811388300841894 +5 0.15811388300841897 -- !sql_stddev_samp_Double_agg_phase_4 -- 12 0.36055512754639896 -- !sql_stddev_samp_Double_gb_notnull -- -0.21602468994692864 -0.15811388300841897 +0.21602468994692867 +0.15811388300841894 -- !sql_stddev_samp_Double_notnull -- 0.36055512754639896 @@ -4146,10 +4146,10 @@ true -- !sql_stddev_samp_Double_agg_phase_3_notnull -- 0 \N 7 0.21602468994692864 -5 0.15811388300841894 +5 0.15811388300841897 -- !sql_stddev_samp_Double_agg_phase_4_notnull -- -12 0.36055512754639896 +12 0.3605551275463989 -- !sql_stddev_samp_DecimalV2_agg_phase_1 -- 0 \N @@ -4172,10 +4172,10 @@ true -- !sql_stddev_samp_DecimalV2_agg_phase_3 -- 0 \N 7 0.21602468994692864 -5 0.15811388300841897 +5 0.15811388300841894 -- !sql_stddev_samp_DecimalV2_agg_phase_4 -- -12 0.36055512754639896 +12 0.3605551275463989 -- !sql_stddev_samp_DecimalV2_agg_phase_1_notnull -- 1 \N @@ -4196,8 +4196,8 @@ true -- !sql_stddev_samp_DecimalV2_agg_phase_3_notnull -- 0 \N -7 0.21602468994692864 -5 0.15811388300841894 +7 0.21602468994692867 +5 0.15811388300841897 -- !sql_stddev_samp_DecimalV2_agg_phase_4_notnull -- 12 0.36055512754639896 @@ -4486,11 +4486,11 @@ true -- !sql_sum_Double_gb -- \N -2.8 +2.8000000000000003 5.0 -- !sql_sum_Double -- -7.8 +7.800000000000001 -- !sql_sum_Double_agg_phase_1 -- 0 \N @@ -4508,7 +4508,7 @@ true 1 1.2 -- !sql_sum_Double_agg_phase_2 -- -12 7.800000000000001 +12 7.8 -- !sql_sum_Double_agg_phase_3 -- 0 \N @@ -4516,14 +4516,14 @@ true 5 5.0 -- !sql_sum_Double_agg_phase_4 -- -12 7.8 +12 7.800000000000001 -- !sql_sum_Double_gb_notnull -- -2.8000000000000003 +2.8 5.0 -- !sql_sum_Double_notnull -- -7.800000000000001 +7.8 -- !sql_sum_Double_agg_phase_1_notnull -- 1 0.1 @@ -4540,7 +4540,7 @@ true 1 1.2 -- !sql_sum_Double_agg_phase_2_notnull -- -12 7.800000000000001 +12 7.8 -- !sql_sum_Double_agg_phase_3_notnull -- 0 \N @@ -4548,7 +4548,7 @@ true 5 5.0 -- !sql_sum_Double_agg_phase_4_notnull -- -12 7.800000000000001 +12 7.799999999999999 -- !sql_sum_DecimalV2_gb -- \N @@ -4682,53 +4682,533 @@ true -- !sql_sum_LargeInt_agg_phase_4_notnull -- 12 78 --- !sql_topn_Varchar_Integer_gb -- -{"null":1} -{"varchar11":3,"varchar13":2,"varchar12":2} -{"varchar13":2,"varchar12":2,"varchar11":1} +-- !sql_sum0_Boolean -- +5 --- !sql_topn_Varchar_Integer -- -{"varchar13":4,"varchar12":4,"varchar11":4} +-- !sql_sum0_Boolean_gb -- +0 +0 +0 +0 +0 +0 +0 +0 +1 +1 +1 +1 +1 --- !sql_topn_Varchar_Integer_agg_phase_1 -- -0 {"null":1} -1 {"varchar11":1} -1 {"varchar12":1} -1 {"varchar13":1} -1 {"varchar11":1} -1 {"varchar12":1} -1 {"varchar13":1} -1 {"varchar11":1} -1 {"varchar12":1} -1 {"varchar13":1} -1 {"varchar11":1} -1 {"varchar12":1} -1 {"varchar13":1} +-- !sql_sum0_TinyInt_gb -- +0 +28 +50 --- !sql_topn_Varchar_Integer_agg_phase_2 -- -12 {"varchar13":4,"varchar12":4,"varchar11":4} +-- !sql_sum0_TinyInt -- +78 --- !sql_topn_Varchar_Integer_agg_phase_3 -- -0 {"null":1} -7 {"varchar11":3,"varchar13":2,"varchar12":2} -5 {"varchar13":2,"varchar12":2,"varchar11":1} +-- !sql_sum0_TinyInt_agg_phase_1 -- +0 0 +1 1 +1 2 +1 3 +1 4 +1 5 +1 6 +1 7 +1 8 +1 9 +1 10 +1 11 +1 12 --- !sql_topn_Varchar_Integer_agg_phase_4 -- -12 {"varchar13":4,"varchar12":4,"varchar11":4} +-- !sql_sum0_TinyInt_agg_phase_2 -- +12 78 --- !sql_topn_Varchar_Integer_gb_notnull -- -{"varchar11":3,"varchar13":2,"varchar12":2} -{"varchar13":2,"varchar12":2,"varchar11":1} +-- !sql_sum0_TinyInt_agg_phase_3 -- +0 0 +7 28 +5 50 --- !sql_topn_Varchar_Integer_notnull -- -{"varchar13":4,"varchar12":4,"varchar11":4} +-- !sql_sum0_TinyInt_agg_phase_4 -- +12 78 --- !sql_topn_Varchar_Integer_agg_phase_1_notnull -- -1 {"varchar11":1} -1 {"varchar12":1} -1 {"varchar13":1} -1 {"varchar11":1} -1 {"varchar12":1} +-- !sql_sum0_TinyInt_gb_notnull -- +28 +50 + +-- !sql_sum0_TinyInt_notnull -- +78 + +-- !sql_sum0_TinyInt_agg_phase_1_notnull -- +1 1 +1 2 +1 3 +1 4 +1 5 +1 6 +1 7 +1 8 +1 9 +1 10 +1 11 +1 12 + +-- !sql_sum0_TinyInt_agg_phase_2_notnull -- +12 78 + +-- !sql_sum0_TinyInt_agg_phase_3_notnull -- +0 0 +7 28 +5 50 + +-- !sql_sum0_TinyInt_agg_phase_4_notnull -- +12 78 + +-- !sql_sum0_SmallInt_gb -- +0 +28 +50 + +-- !sql_sum0_SmallInt -- +78 + +-- !sql_sum0_SmallInt_agg_phase_1 -- +0 0 +1 1 +1 2 +1 3 +1 4 +1 5 +1 6 +1 7 +1 8 +1 9 +1 10 +1 11 +1 12 + +-- !sql_sum0_SmallInt_agg_phase_2 -- +12 78 + +-- !sql_sum0_SmallInt_agg_phase_3 -- +0 0 +7 28 +5 50 + +-- !sql_sum0_SmallInt_agg_phase_4 -- +12 78 + +-- !sql_sum0_SmallInt_gb_notnull -- +28 +50 + +-- !sql_sum0_SmallInt_notnull -- +78 + +-- !sql_sum0_SmallInt_agg_phase_1_notnull -- +1 1 +1 2 +1 3 +1 4 +1 5 +1 6 +1 7 +1 8 +1 9 +1 10 +1 11 +1 12 + +-- !sql_sum0_SmallInt_agg_phase_2_notnull -- +12 78 + +-- !sql_sum0_SmallInt_agg_phase_3_notnull -- +0 0 +7 28 +5 50 + +-- !sql_sum0_SmallInt_agg_phase_4_notnull -- +12 78 + +-- !sql_sum0_Integer_gb -- +0 +28 +50 + +-- !sql_sum0_Integer -- +78 + +-- !sql_sum0_Integer_agg_phase_1 -- +0 0 +1 1 +1 2 +1 3 +1 4 +1 5 +1 6 +1 7 +1 8 +1 9 +1 10 +1 11 +1 12 + +-- !sql_sum0_Integer_agg_phase_2 -- +12 78 + +-- !sql_sum0_Integer_agg_phase_3 -- +0 0 +7 28 +5 50 + +-- !sql_sum0_Integer_agg_phase_4 -- +12 78 + +-- !sql_sum0_Integer_gb_notnull -- +28 +50 + +-- !sql_sum0_Integer_notnull -- +78 + +-- !sql_sum0_Integer_agg_phase_1_notnull -- +1 1 +1 2 +1 3 +1 4 +1 5 +1 6 +1 7 +1 8 +1 9 +1 10 +1 11 +1 12 + +-- !sql_sum0_Integer_agg_phase_2_notnull -- +12 78 + +-- !sql_sum0_Integer_agg_phase_3_notnull -- +0 0 +7 28 +5 50 + +-- !sql_sum0_Integer_agg_phase_4_notnull -- +12 78 + +-- !sql_sum0_BigInt_gb -- +0 +28 +50 + +-- !sql_sum0_BigInt -- +78 + +-- !sql_sum0_BigInt_agg_phase_1 -- +0 0 +1 1 +1 2 +1 3 +1 4 +1 5 +1 6 +1 7 +1 8 +1 9 +1 10 +1 11 +1 12 + +-- !sql_sum0_BigInt_agg_phase_2 -- +12 78 + +-- !sql_sum0_BigInt_agg_phase_3 -- +0 0 +7 28 +5 50 + +-- !sql_sum0_BigInt_agg_phase_4 -- +12 78 + +-- !sql_sum0_BigInt_gb_notnull -- +28 +50 + +-- !sql_sum0_BigInt_notnull -- +78 + +-- !sql_sum0_BigInt_agg_phase_1_notnull -- +1 1 +1 2 +1 3 +1 4 +1 5 +1 6 +1 7 +1 8 +1 9 +1 10 +1 11 +1 12 + +-- !sql_sum0_BigInt_agg_phase_2_notnull -- +12 78 + +-- !sql_sum0_BigInt_agg_phase_3_notnull -- +0 0 +7 28 +5 50 + +-- !sql_sum0_BigInt_agg_phase_4_notnull -- +12 78 + +-- !sql_sum0_Double_gb -- +0.0 +2.8 +5.0 + +-- !sql_sum0_Double -- +7.800000000000001 + +-- !sql_sum0_Double_agg_phase_1 -- +0 0.0 +1 0.1 +1 0.2 +1 0.3 +1 0.4 +1 0.5 +1 0.6 +1 0.7 +1 0.8 +1 0.9 +1 1.0 +1 1.1 +1 1.2 + +-- !sql_sum0_Double_agg_phase_2 -- +12 7.800000000000001 + +-- !sql_sum0_Double_agg_phase_3 -- +0 0.0 +7 2.8000000000000003 +5 5.0 + +-- !sql_sum0_Double_agg_phase_4 -- +12 7.8 + +-- !sql_sum0_Double_gb_notnull -- +2.8 +5.0 + +-- !sql_sum0_Double_notnull -- +7.8 + +-- !sql_sum0_Double_agg_phase_1_notnull -- +1 0.1 +1 0.2 +1 0.3 +1 0.4 +1 0.5 +1 0.6 +1 0.7 +1 0.8 +1 0.9 +1 1.0 +1 1.1 +1 1.2 + +-- !sql_sum0_Double_agg_phase_2_notnull -- +12 7.8 + +-- !sql_sum0_Double_agg_phase_3_notnull -- +0 0.0 +7 2.8 +5 5.0 + +-- !sql_sum0_Double_agg_phase_4_notnull -- +12 7.800000000000001 + +-- !sql_sum0_DecimalV2_gb -- +0.000 +2.800 +5.000 + +-- !sql_sum0_DecimalV2 -- +7.800 + +-- !sql_sum0_DecimalV2_agg_phase_1 -- +0 0.000 +1 0.100 +1 0.200 +1 0.300 +1 0.400 +1 0.500 +1 0.600 +1 0.700 +1 0.800 +1 0.900 +1 1.000 +1 1.100 +1 1.200 + +-- !sql_sum0_DecimalV2_agg_phase_2 -- +12 7.800 + +-- !sql_sum0_DecimalV2_agg_phase_3 -- +0 0.000 +7 2.800 +5 5.000 + +-- !sql_sum0_DecimalV2_agg_phase_4 -- +12 7.800 + +-- !sql_sum0_DecimalV2_gb_notnull -- +2.800 +5.000 + +-- !sql_sum0_DecimalV2_notnull -- +7.800 + +-- !sql_sum0_DecimalV2_agg_phase_1_notnull -- +1 0.100 +1 0.200 +1 0.300 +1 0.400 +1 0.500 +1 0.600 +1 0.700 +1 0.800 +1 0.900 +1 1.000 +1 1.100 +1 1.200 + +-- !sql_sum0_DecimalV2_agg_phase_2_notnull -- +12 7.800 + +-- !sql_sum0_DecimalV2_agg_phase_3_notnull -- +0 0.000 +7 2.800 +5 5.000 + +-- !sql_sum0_DecimalV2_agg_phase_4_notnull -- +12 7.800 + +-- !sql_sum0_LargeInt_gb -- +0 +28 +50 + +-- !sql_sum0_LargeInt -- +78 + +-- !sql_sum0_LargeInt_agg_phase_1 -- +0 0 +1 1 +1 2 +1 3 +1 4 +1 5 +1 6 +1 7 +1 8 +1 9 +1 10 +1 11 +1 12 + +-- !sql_sum0_LargeInt_agg_phase_2 -- +12 78 + +-- !sql_sum0_LargeInt_agg_phase_3 -- +0 0 +7 28 +5 50 + +-- !sql_sum0_LargeInt_agg_phase_4 -- +12 78 + +-- !sql_sum0_LargeInt_gb_notnull -- +28 +50 + +-- !sql_sum0_LargeInt_notnull -- +78 + +-- !sql_sum0_LargeInt_agg_phase_1_notnull -- +1 1 +1 2 +1 3 +1 4 +1 5 +1 6 +1 7 +1 8 +1 9 +1 10 +1 11 +1 12 + +-- !sql_sum0_LargeInt_agg_phase_2_notnull -- +12 78 + +-- !sql_sum0_LargeInt_agg_phase_3_notnull -- +0 0 +7 28 +5 50 + +-- !sql_sum0_LargeInt_agg_phase_4_notnull -- +12 78 + +-- !sql_topn_Varchar_Integer_gb -- +{"null":1} +{"varchar11":3,"varchar13":2,"varchar12":2} +{"varchar13":2,"varchar12":2,"varchar11":1} + +-- !sql_topn_Varchar_Integer -- +{"varchar13":4,"varchar12":4,"varchar11":4} + +-- !sql_topn_Varchar_Integer_agg_phase_1 -- +0 {"null":1} +1 {"varchar11":1} +1 {"varchar12":1} +1 {"varchar13":1} +1 {"varchar11":1} +1 {"varchar12":1} +1 {"varchar13":1} +1 {"varchar11":1} +1 {"varchar12":1} +1 {"varchar13":1} +1 {"varchar11":1} +1 {"varchar12":1} +1 {"varchar13":1} + +-- !sql_topn_Varchar_Integer_agg_phase_2 -- +12 {"varchar13":4,"varchar12":4,"varchar11":4} + +-- !sql_topn_Varchar_Integer_agg_phase_3 -- +0 {"null":1} +7 {"varchar11":3,"varchar13":2,"varchar12":2} +5 {"varchar13":2,"varchar12":2,"varchar11":1} + +-- !sql_topn_Varchar_Integer_agg_phase_4 -- +12 {"varchar13":4,"varchar12":4,"varchar11":4} + +-- !sql_topn_Varchar_Integer_gb_notnull -- +{"varchar11":3,"varchar13":2,"varchar12":2} +{"varchar13":2,"varchar12":2,"varchar11":1} + +-- !sql_topn_Varchar_Integer_notnull -- +{"varchar13":4,"varchar12":4,"varchar11":4} + +-- !sql_topn_Varchar_Integer_agg_phase_1_notnull -- +1 {"varchar11":1} +1 {"varchar12":1} +1 {"varchar13":1} +1 {"varchar11":1} +1 {"varchar12":1} 1 {"varchar13":1} 1 {"varchar11":1} 1 {"varchar12":1} @@ -4949,7 +5429,7 @@ true -- !sql_variance_TinyInt_gb -- \N 3.9999999999999996 -2.0000000000000004 +1.9999999999999993 -- !sql_variance_TinyInt -- 11.916666666666666 @@ -4975,14 +5455,14 @@ true -- !sql_variance_TinyInt_agg_phase_3 -- 0 \N 7 4.0 -5 2.0 +5 2.000000000000001 -- !sql_variance_TinyInt_agg_phase_4 -- 12 11.916666666666666 -- !sql_variance_TinyInt_gb_notnull -- 4.0 -2.0 +2.0000000000000004 -- !sql_variance_TinyInt_notnull -- 11.916666666666666 @@ -5007,7 +5487,7 @@ true -- !sql_variance_TinyInt_agg_phase_3_notnull -- 0 \N 7 4.0 -5 2.0 +5 2.000000000000001 -- !sql_variance_TinyInt_agg_phase_4_notnull -- 12 11.916666666666666 @@ -5015,7 +5495,7 @@ true -- !sql_variance_SmallInt_gb -- \N 3.9999999999999996 -2.0 +1.9999999999999993 -- !sql_variance_SmallInt -- 11.916666666666666 @@ -5040,18 +5520,18 @@ true -- !sql_variance_SmallInt_agg_phase_3 -- 0 \N -7 3.9999999999999996 -5 2.0 +7 4.0 +5 2.000000000000001 -- !sql_variance_SmallInt_agg_phase_4 -- 12 11.916666666666666 -- !sql_variance_SmallInt_gb_notnull -- -4.0 +3.9999999999999996 2.0 -- !sql_variance_SmallInt_notnull -- -11.91666666666667 +11.916666666666666 -- !sql_variance_SmallInt_agg_phase_1_notnull -- 1 0.0 @@ -5072,16 +5552,16 @@ true -- !sql_variance_SmallInt_agg_phase_3_notnull -- 0 \N -7 3.9999999999999996 -5 2.0 +7 4.0 +5 2.000000000000001 -- !sql_variance_SmallInt_agg_phase_4_notnull -- 12 11.916666666666666 -- !sql_variance_Integer_gb -- \N -4.0 -2.0000000000000004 +3.9999999999999996 +2.0 -- !sql_variance_Integer -- 11.916666666666666 @@ -5106,15 +5586,15 @@ true -- !sql_variance_Integer_agg_phase_3 -- 0 \N -7 4.0 -5 2.000000000000001 +7 3.9999999999999996 +5 2.0000000000000004 -- !sql_variance_Integer_agg_phase_4 -- 12 11.916666666666666 -- !sql_variance_Integer_gb_notnull -- -3.9999999999999996 -2.0 +4.0 +2.0000000000000004 -- !sql_variance_Integer_notnull -- 11.916666666666666 @@ -5138,15 +5618,15 @@ true -- !sql_variance_Integer_agg_phase_3_notnull -- 0 \N -7 4.0 -5 2.0 +7 3.9999999999999996 +5 1.9999999999999993 -- !sql_variance_Integer_agg_phase_4_notnull -- 12 11.916666666666666 -- !sql_variance_BigInt_gb -- \N -4.0 +3.999999999999999 2.0 -- !sql_variance_BigInt -- @@ -5172,14 +5652,14 @@ true -- !sql_variance_BigInt_agg_phase_3 -- 0 \N -7 4.000000000000001 +7 3.9999999999999996 5 2.0 -- !sql_variance_BigInt_agg_phase_4 -- 12 11.916666666666666 -- !sql_variance_BigInt_gb_notnull -- -3.9999999999999996 +4.0 2.0 -- !sql_variance_BigInt_notnull -- @@ -5204,16 +5684,16 @@ true -- !sql_variance_BigInt_agg_phase_3_notnull -- 0 \N -7 4.0 -5 2.0 +7 3.9999999999999996 +5 2.0000000000000004 -- !sql_variance_BigInt_agg_phase_4_notnull -- 12 11.916666666666666 -- !sql_variance_Float_gb -- \N -0.039999999701976874 -0.020000004768372152 +0.03999999970197688 +0.02000000476837215 -- !sql_variance_Float -- 0.11916667052855125 @@ -5245,7 +5725,7 @@ true 12 0.11916667052855125 -- !sql_variance_Float_gb_notnull -- -0.039999999701976874 +0.03999999970197688 0.020000004768372152 -- !sql_variance_Float_notnull -- @@ -5270,19 +5750,19 @@ true -- !sql_variance_Float_agg_phase_3_notnull -- 0 \N -7 0.039999999701976874 +7 0.03999999970197688 5 0.020000004768372152 -- !sql_variance_Float_agg_phase_4_notnull -- -12 0.11916667052855123 +12 0.11916667052855125 -- !sql_variance_Double_gb -- \N 0.039999999999999994 -0.019999999999999997 +0.019999999999999993 -- !sql_variance_Double -- -0.11916666666666668 +0.11916666666666666 -- !sql_variance_Double_agg_phase_1 -- 0 \N @@ -5300,15 +5780,15 @@ true 1 0.0 -- !sql_variance_Double_agg_phase_2 -- -12 0.11916666666666666 +12 0.11916666666666668 -- !sql_variance_Double_agg_phase_3 -- 0 \N 7 0.03999999999999999 -5 0.019999999999999993 +5 0.019999999999999997 -- !sql_variance_Double_agg_phase_4 -- -12 0.11916666666666666 +12 0.11916666666666668 -- !sql_variance_Double_gb_notnull -- 0.039999999999999994 @@ -5332,23 +5812,23 @@ true 1 0.0 -- !sql_variance_Double_agg_phase_2_notnull -- -12 0.11916666666666666 +12 0.11916666666666668 -- !sql_variance_Double_agg_phase_3_notnull -- 0 \N -7 0.039999999999999994 +7 0.03999999999999999 5 0.019999999999999997 -- !sql_variance_Double_agg_phase_4_notnull -- -12 0.11916666666666664 +12 0.11916666666666666 -- !sql_variance_DecimalV2_gb -- \N -0.039999999999999994 +0.04 0.02 -- !sql_variance_DecimalV2 -- -0.11916666666666668 +0.11916666666666666 -- !sql_variance_DecimalV2_agg_phase_1 -- 0 \N @@ -5366,11 +5846,11 @@ true 1 0.0 -- !sql_variance_DecimalV2_agg_phase_2 -- -12 0.11916666666666666 +12 0.11916666666666664 -- !sql_variance_DecimalV2_agg_phase_3 -- 0 \N -7 0.039999999999999994 +7 0.04 5 0.02 -- !sql_variance_DecimalV2_agg_phase_4 -- @@ -5378,7 +5858,7 @@ true -- !sql_variance_DecimalV2_gb_notnull -- 0.039999999999999994 -0.01999999999999999 +0.02 -- !sql_variance_DecimalV2_notnull -- 0.11916666666666666 @@ -5398,20 +5878,20 @@ true 1 0.0 -- !sql_variance_DecimalV2_agg_phase_2_notnull -- -12 0.11916666666666668 +12 0.11916666666666666 -- !sql_variance_DecimalV2_agg_phase_3_notnull -- 0 \N 7 0.039999999999999994 -5 0.02 +5 0.019999999999999987 -- !sql_variance_DecimalV2_agg_phase_4_notnull -- 12 0.11916666666666666 -- !sql_variance_samp_TinyInt_gb -- \N -4.666666666666667 -2.5 +4.666666666666666 +2.499999999999999 -- !sql_variance_samp_TinyInt -- 13.0 @@ -5436,14 +5916,14 @@ true -- !sql_variance_samp_TinyInt_agg_phase_3 -- 0 \N -7 4.666666666666667 +7 4.666666666666666 5 2.5 -- !sql_variance_samp_TinyInt_agg_phase_4 -- 12 13.0 -- !sql_variance_samp_TinyInt_gb_notnull -- -4.666666666666667 +4.666666666666666 2.5 -- !sql_variance_samp_TinyInt_notnull -- @@ -5468,8 +5948,8 @@ true -- !sql_variance_samp_TinyInt_agg_phase_3_notnull -- 0 \N -7 4.666666666666668 -5 2.5 +7 4.666666666666667 +5 2.500000000000001 -- !sql_variance_samp_TinyInt_agg_phase_4_notnull -- 12 13.0 @@ -5477,7 +5957,7 @@ true -- !sql_variance_samp_SmallInt_gb -- \N 4.666666666666667 -2.5000000000000004 +2.5 -- !sql_variance_samp_SmallInt -- 13.0 @@ -5502,7 +5982,7 @@ true -- !sql_variance_samp_SmallInt_agg_phase_3 -- 0 \N -7 4.666666666666667 +7 4.666666666666666 5 2.5 -- !sql_variance_samp_SmallInt_agg_phase_4 -- @@ -5510,10 +5990,10 @@ true -- !sql_variance_samp_SmallInt_gb_notnull -- 4.666666666666666 -2.499999999999999 +2.5 -- !sql_variance_samp_SmallInt_notnull -- -13.000000000000002 +13.0 -- !sql_variance_samp_SmallInt_agg_phase_1_notnull -- 1 \N @@ -5534,8 +6014,8 @@ true -- !sql_variance_samp_SmallInt_agg_phase_3_notnull -- 0 \N -7 4.666666666666667 -5 2.5 +7 4.666666666666666 +5 2.499999999999999 -- !sql_variance_samp_SmallInt_agg_phase_4_notnull -- 12 13.0 @@ -5543,7 +6023,7 @@ true -- !sql_variance_samp_Integer_gb -- \N 4.666666666666667 -2.5 +2.5000000000000004 -- !sql_variance_samp_Integer -- 13.0 @@ -5568,8 +6048,8 @@ true -- !sql_variance_samp_Integer_agg_phase_3 -- 0 \N -7 4.666666666666667 -5 2.5 +7 4.666666666666666 +5 2.5000000000000004 -- !sql_variance_samp_Integer_agg_phase_4 -- 12 13.0 @@ -5600,7 +6080,7 @@ true -- !sql_variance_samp_Integer_agg_phase_3_notnull -- 0 \N -7 4.666666666666666 +7 4.666666666666668 5 2.5 -- !sql_variance_samp_Integer_agg_phase_4_notnull -- @@ -5641,7 +6121,7 @@ true 12 13.0 -- !sql_variance_samp_BigInt_gb_notnull -- -4.666666666666667 +4.666666666666666 2.5 -- !sql_variance_samp_BigInt_notnull -- @@ -5666,7 +6146,7 @@ true -- !sql_variance_samp_BigInt_agg_phase_3_notnull -- 0 \N -7 4.666666666666666 +7 4.666666666666668 5 2.5 -- !sql_variance_samp_BigInt_agg_phase_4_notnull -- @@ -5674,8 +6154,8 @@ true -- !sql_variance_samp_Float_gb -- \N -0.04666666631897304 -0.025000005960465192 +0.04666666631897303 +0.02500000596046519 -- !sql_variance_samp_Float -- 0.130000004212965 @@ -5696,7 +6176,7 @@ true 1 \N -- !sql_variance_samp_Float_agg_phase_2 -- -12 0.130000004212965 +12 0.13000000421296498 -- !sql_variance_samp_Float_agg_phase_3 -- 0 \N @@ -5736,12 +6216,12 @@ true 5 0.02500000596046519 -- !sql_variance_samp_Float_agg_phase_4_notnull -- -12 0.130000004212965 +12 0.13000000421296498 -- !sql_variance_samp_Double_gb -- \N 0.04666666666666666 -0.024999999999999988 +0.024999999999999998 -- !sql_variance_samp_Double -- 0.13 @@ -5767,14 +6247,14 @@ true -- !sql_variance_samp_Double_agg_phase_3 -- 0 \N 7 0.04666666666666666 -5 0.024999999999999984 +5 0.024999999999999998 -- !sql_variance_samp_Double_agg_phase_4 -- 12 0.13 -- !sql_variance_samp_Double_gb_notnull -- 0.04666666666666666 -0.024999999999999994 +0.025 -- !sql_variance_samp_Double_notnull -- 0.13 @@ -5794,12 +6274,12 @@ true 1 \N -- !sql_variance_samp_Double_agg_phase_2_notnull -- -12 0.13 +12 0.12999999999999998 -- !sql_variance_samp_Double_agg_phase_3_notnull -- 0 \N 7 0.046666666666666655 -5 0.024999999999999988 +5 0.024999999999999994 -- !sql_variance_samp_Double_agg_phase_4_notnull -- 12 0.13 @@ -5825,10 +6305,10 @@ true -- !sql_variance_samp_DecimalV2_agg_phase_3 -- 0 \N 7 0.04666666666666666 -5 0.02499999999999999 +5 0.024999999999999994 -- !sql_variance_samp_DecimalV2_agg_phase_4 -- -12 0.13 +12 0.12999999999999998 -- !sql_variance_samp_DecimalV2_agg_phase_1_notnull -- 1 \N @@ -5849,7 +6329,7 @@ true -- !sql_variance_samp_DecimalV2_agg_phase_3_notnull -- 0 \N -7 0.046666666666666655 +7 0.04666666666666666 5 0.024999999999999988 -- !sql_variance_samp_DecimalV2_agg_phase_4_notnull -- diff --git a/regression-test/data/nereids_hint_tpch_p0/shape/q15.out b/regression-test/data/nereids_hint_tpch_p0/shape/q15.out index 87cf9364346dbe..c059df5018f342 100644 --- a/regression-test/data/nereids_hint_tpch_p0/shape/q15.out +++ b/regression-test/data/nereids_hint_tpch_p0/shape/q15.out @@ -9,25 +9,26 @@ PhysicalResultSink ------------PhysicalProject --------------PhysicalOlapScan[supplier] ------------PhysicalDistribute[DistributionSpecHash] ---------------hashJoin[INNER_JOIN] hashCondition=((revenue0.total_revenue = max(total_revenue))) otherCondition=() -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecGather] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter((lineitem.l_shipdate < '1996-04-01') and (lineitem.l_shipdate >= '1996-01-01')) -----------------------------------PhysicalOlapScan[lineitem] -----------------PhysicalDistribute[DistributionSpecReplicated] -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------filter((lineitem.l_shipdate < '1996-04-01') and (lineitem.l_shipdate >= '1996-01-01')) -------------------------------PhysicalOlapScan[lineitem] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN] hashCondition=((revenue0.total_revenue = max(total_revenue))) otherCondition=() +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecGather] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashAgg[GLOBAL] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------hashAgg[LOCAL] +--------------------------------PhysicalProject +----------------------------------filter((lineitem.l_shipdate < '1996-04-01') and (lineitem.l_shipdate >= '1996-01-01')) +------------------------------------PhysicalOlapScan[lineitem] +------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------filter((lineitem.l_shipdate < '1996-04-01') and (lineitem.l_shipdate >= '1996-01-01')) +--------------------------------PhysicalOlapScan[lineitem] Hint log: Used: leading(revenue0 supplier ) diff --git a/regression-test/data/nereids_p0/hint/test_distribute.out b/regression-test/data/nereids_p0/hint/test_distribute.out index a14b6696a8dacb..072026c3a13abf 100644 --- a/regression-test/data/nereids_p0/hint/test_distribute.out +++ b/regression-test/data/nereids_p0/hint/test_distribute.out @@ -453,12 +453,13 @@ PhysicalResultSink ------hashAgg[LOCAL] --------PhysicalProject ----------NestedLoopJoin[INNER_JOIN](t1.c1 > t2.c2) -------------NestedLoopJoin[INNER_JOIN](t2.c2 > t3.c3) ---------------PhysicalProject -----------------PhysicalOlapScan[t2] ---------------PhysicalDistribute[DistributionSpecReplicated] +------------PhysicalProject +--------------NestedLoopJoin[INNER_JOIN](t2.c2 > t3.c3) ----------------PhysicalProject -------------------PhysicalOlapScan[t3] +------------------PhysicalOlapScan[t2] +----------------PhysicalDistribute[DistributionSpecReplicated] +------------------PhysicalProject +--------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] --------------PhysicalProject ----------------filter((t1.c1 < 100)) @@ -621,14 +622,15 @@ PhysicalResultSink ------hashAgg[LOCAL] --------PhysicalProject ----------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() -------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() ---------------PhysicalProject -----------------filter((t1.c1 <= 300) and (t1.c1 >= 100)) -------------------PhysicalOlapScan[t1] ---------------PhysicalDistribute[DistributionSpecHash] +------------PhysicalProject +--------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() ----------------PhysicalProject -------------------filter((t3.c3 <= 300) and (t3.c3 >= 100)) ---------------------PhysicalOlapScan[t3] +------------------filter((t1.c1 <= 300) and (t1.c1 >= 100)) +--------------------PhysicalOlapScan[t1] +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------filter((t3.c3 <= 300) and (t3.c3 >= 100)) +----------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecHash] --------------PhysicalProject ----------------filter((t2.c2 <= 300) and (t2.c2 >= 100)) @@ -641,14 +643,15 @@ PhysicalResultSink ------hashAgg[LOCAL] --------PhysicalProject ----------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() -------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() ---------------PhysicalProject -----------------filter((t1.c1 <= 300) and (t1.c1 >= 100)) -------------------PhysicalOlapScan[t1] ---------------PhysicalDistribute[DistributionSpecHash] +------------PhysicalProject +--------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() ----------------PhysicalProject -------------------filter((t3.c3 <= 300) and (t3.c3 >= 100)) ---------------------PhysicalOlapScan[t3] +------------------filter((t1.c1 <= 300) and (t1.c1 >= 100)) +--------------------PhysicalOlapScan[t1] +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------filter((t3.c3 <= 300) and (t3.c3 >= 100)) +----------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecHash] --------------PhysicalProject ----------------filter((t2.c2 <= 300) and (t2.c2 >= 100)) @@ -3717,12 +3720,13 @@ PhysicalResultSink ------hashAgg[LOCAL] --------PhysicalProject ----------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() ---------------PhysicalProject -----------------PhysicalOlapScan[t3] ---------------PhysicalDistribute[DistributionSpecHash] +------------PhysicalProject +--------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() ----------------PhysicalProject -------------------PhysicalOlapScan[t4] +------------------PhysicalOlapScan[t3] +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t4] ------------PhysicalDistribute[DistributionSpecHash] --------------PhysicalProject ----------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() @@ -4087,16 +4091,17 @@ PhysicalResultSink ------hashAgg[LOCAL] --------PhysicalProject ----------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t4.c4)) otherCondition=() ---------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() -----------------PhysicalProject -------------------PhysicalOlapScan[t1] +------------PhysicalProject +--------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t4.c4)) otherCondition=() +----------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +------------------PhysicalProject +--------------------PhysicalOlapScan[t1] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------PhysicalProject +----------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecHash] ------------------PhysicalProject ---------------------PhysicalOlapScan[t2] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t4] +--------------------PhysicalOlapScan[t4] ------------PhysicalDistribute[DistributionSpecHash] --------------PhysicalProject ----------------PhysicalOlapScan[t3] @@ -4439,16 +4444,17 @@ PhysicalResultSink ------hashAgg[LOCAL] --------PhysicalProject ----------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t4.c4)) otherCondition=() ---------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() -----------------PhysicalProject -------------------PhysicalOlapScan[t1] +------------PhysicalProject +--------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t4.c4)) otherCondition=() +----------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +------------------PhysicalProject +--------------------PhysicalOlapScan[t1] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------PhysicalProject +----------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecHash] ------------------PhysicalProject ---------------------PhysicalOlapScan[t2] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t4] +--------------------PhysicalOlapScan[t4] ------------PhysicalDistribute[DistributionSpecReplicated] --------------PhysicalProject ----------------PhysicalOlapScan[t3] @@ -4793,16 +4799,17 @@ PhysicalResultSink ------hashAgg[LOCAL] --------PhysicalProject ----------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t4.c4)) otherCondition=() ---------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() -----------------PhysicalProject -------------------PhysicalOlapScan[t1] +------------PhysicalProject +--------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t4.c4)) otherCondition=() +----------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +------------------PhysicalProject +--------------------PhysicalOlapScan[t1] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------PhysicalProject +----------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecHash] ------------------PhysicalProject ---------------------PhysicalOlapScan[t2] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t4] +--------------------PhysicalOlapScan[t4] ------------PhysicalDistribute[DistributionSpecHash] --------------PhysicalProject ----------------PhysicalOlapScan[t3] @@ -5145,16 +5152,17 @@ PhysicalResultSink ------hashAgg[LOCAL] --------PhysicalProject ----------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t4.c4)) otherCondition=() ---------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() -----------------PhysicalProject -------------------PhysicalOlapScan[t1] +------------PhysicalProject +--------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t4.c4)) otherCondition=() +----------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +------------------PhysicalProject +--------------------PhysicalOlapScan[t1] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------PhysicalProject +----------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecHash] ------------------PhysicalProject ---------------------PhysicalOlapScan[t2] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t4] +--------------------PhysicalOlapScan[t4] ------------PhysicalDistribute[DistributionSpecHash] --------------PhysicalProject ----------------PhysicalOlapScan[t3] diff --git a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q4.3.out b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q4.3.out index 38e0dd0adb1ab2..052e864e9be9a0 100644 --- a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q4.3.out +++ b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q4.3.out @@ -14,18 +14,19 @@ PhysicalResultSink ------------------PhysicalDistribute[DistributionSpecHash] --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF2 d_datekey->[lo_orderdate] -------------------------hashJoin[INNER_JOIN] hashCondition=((lineorder.lo_partkey = part.p_partkey)) otherCondition=() build RFs:RF1 p_partkey->[lo_partkey] ---------------------------hashJoin[INNER_JOIN] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF0 s_suppkey->[lo_suppkey] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN] hashCondition=((lineorder.lo_partkey = part.p_partkey)) otherCondition=() build RFs:RF1 p_partkey->[lo_partkey] +----------------------------hashJoin[INNER_JOIN] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF0 s_suppkey->[lo_suppkey] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 +------------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------------PhysicalProject +----------------------------------filter((supplier.s_nation = 'UNITED STATES')) +------------------------------------PhysicalOlapScan[supplier] ----------------------------PhysicalDistribute[DistributionSpecReplicated] ------------------------------PhysicalProject ---------------------------------filter((supplier.s_nation = 'UNITED STATES')) -----------------------------------PhysicalOlapScan[supplier] ---------------------------PhysicalDistribute[DistributionSpecReplicated] -----------------------------PhysicalProject -------------------------------filter((part.p_category = 'MFGR#14')) ---------------------------------PhysicalOlapScan[part] +--------------------------------filter((part.p_category = 'MFGR#14')) +----------------------------------PhysicalOlapScan[part] ------------------------PhysicalDistribute[DistributionSpecReplicated] --------------------------PhysicalProject ----------------------------filter(d_year IN (1997, 1998)) diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query1.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query1.out index 2fabf81b637cb9..b2b5a87ac2bc84 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query1.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query1.out @@ -24,20 +24,21 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------PhysicalProject ------------------PhysicalOlapScan[customer] --------------PhysicalDistribute[DistributionSpecHash] -----------------hashJoin[INNER_JOIN] hashCondition=((ctr1.ctr_store_sk = ctr2.ctr_store_sk)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN] hashCondition=((store.s_store_sk = ctr1.ctr_store_sk)) otherCondition=() build RFs:RF1 s_store_sk->[ctr_store_sk] -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF1 -----------------------PhysicalDistribute[DistributionSpecReplicated] -------------------------PhysicalProject ---------------------------filter((store.s_state = 'TN')) -----------------------------PhysicalOlapScan[store] -------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalDistribute[DistributionSpecExecutionAny] -----------------------------PhysicalProject -------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------PhysicalProject +------------------hashJoin[INNER_JOIN] hashCondition=((ctr1.ctr_store_sk = ctr2.ctr_store_sk)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN] hashCondition=((store.s_store_sk = ctr1.ctr_store_sk)) otherCondition=() build RFs:RF1 s_store_sk->[ctr_store_sk] +------------------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF1 +------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------PhysicalProject +----------------------------filter((store.s_state = 'TN')) +------------------------------PhysicalOlapScan[store] +--------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------------PhysicalProject +--------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query11.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query11.out index f9e24f7051ac51..895b8b5926777b 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query11.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query11.out @@ -44,18 +44,19 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------PhysicalProject ------------------filter((t_w_firstyear.dyear = 1998) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.00)) --------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------hashJoin[INNER_JOIN] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() -----------------PhysicalDistribute[DistributionSpecHash] -------------------PhysicalProject ---------------------filter((t_s_secyear.dyear = 1999) and (t_s_secyear.sale_type = 's')) -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -----------------hashJoin[INNER_JOIN] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=() -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalProject -----------------------filter((t_w_secyear.dyear = 1999) and (t_w_secyear.sale_type = 'w')) -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------PhysicalProject +----------------hashJoin[INNER_JOIN] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() ------------------PhysicalDistribute[DistributionSpecHash] --------------------PhysicalProject -----------------------filter((t_s_firstyear.dyear = 1998) and (t_s_firstyear.sale_type = 's') and (t_s_firstyear.year_total > 0.00)) +----------------------filter((t_s_secyear.dyear = 1999) and (t_s_secyear.sale_type = 's')) ------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------hashJoin[INNER_JOIN] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=() +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------filter((t_w_secyear.dyear = 1999) and (t_w_secyear.sale_type = 'w')) +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------filter((t_s_firstyear.dyear = 1998) and (t_s_firstyear.sale_type = 's') and (t_s_firstyear.year_total > 0.00)) +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) 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 6d388f54994014..0f5afbeafa7129 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 @@ -10,13 +10,14 @@ PhysicalResultSink --------------PhysicalProject ----------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=(((ca_state IN ('CA', 'GA', 'WA') OR substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274')) OR (catalog_sales.cs_sales_price > 500.00))) build RFs:RF2 c_customer_sk->[cs_bill_customer_sk] ------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF2 -----------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] ------------------------PhysicalProject ---------------------------filter((date_dim.d_qoy = 2) and (date_dim.d_year = 2001)) -----------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF2 +------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------PhysicalProject +----------------------------filter((date_dim.d_qoy = 2) and (date_dim.d_year = 2001)) +------------------------------PhysicalOlapScan[date_dim] ------------------PhysicalDistribute[DistributionSpecHash] --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query16.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query16.out index e51af6659181e1..7351b064d78504 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query16.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query16.out @@ -13,26 +13,27 @@ PhysicalResultSink --------------------PhysicalDistribute[DistributionSpecHash] ----------------------PhysicalProject ------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 ---------------------hashJoin[RIGHT_ANTI_JOIN] hashCondition=((cs1.cs_order_number = cr1.cr_order_number)) otherCondition=() build RFs:RF3 cs_order_number->[cr_order_number] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF3 -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashJoin[INNER_JOIN] hashCondition=((cs1.cs_call_center_sk = call_center.cc_call_center_sk)) otherCondition=() build RFs:RF2 cc_call_center_sk->[cs_call_center_sk] ---------------------------hashJoin[INNER_JOIN] hashCondition=((cs1.cs_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_ship_date_sk] -----------------------------hashJoin[INNER_JOIN] hashCondition=((cs1.cs_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[cs_ship_addr_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 +--------------------PhysicalProject +----------------------hashJoin[RIGHT_ANTI_JOIN] hashCondition=((cs1.cs_order_number = cr1.cr_order_number)) otherCondition=() build RFs:RF3 cs_order_number->[cr_order_number] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF3 +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashJoin[INNER_JOIN] hashCondition=((cs1.cs_call_center_sk = call_center.cc_call_center_sk)) otherCondition=() build RFs:RF2 cc_call_center_sk->[cs_call_center_sk] +----------------------------hashJoin[INNER_JOIN] hashCondition=((cs1.cs_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_ship_date_sk] +------------------------------hashJoin[INNER_JOIN] hashCondition=((cs1.cs_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[cs_ship_addr_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 +--------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------PhysicalProject +------------------------------------filter((customer_address.ca_state = 'PA')) +--------------------------------------PhysicalOlapScan[customer_address] ------------------------------PhysicalDistribute[DistributionSpecReplicated] --------------------------------PhysicalProject -----------------------------------filter((customer_address.ca_state = 'PA')) -------------------------------------PhysicalOlapScan[customer_address] +----------------------------------filter((date_dim.d_date <= '2002-05-31') and (date_dim.d_date >= '2002-04-01')) +------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalDistribute[DistributionSpecReplicated] ------------------------------PhysicalProject ---------------------------------filter((date_dim.d_date <= '2002-05-31') and (date_dim.d_date >= '2002-04-01')) -----------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute[DistributionSpecReplicated] -----------------------------PhysicalProject -------------------------------filter((call_center.cc_county = 'Williamson County')) ---------------------------------PhysicalOlapScan[call_center] +--------------------------------filter((call_center.cc_county = 'Williamson County')) +----------------------------------PhysicalOlapScan[call_center] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query21.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query21.out index 404d277586a152..45b33fc1bdf5d5 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query21.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query21.out @@ -12,12 +12,13 @@ PhysicalResultSink ------------------hashJoin[INNER_JOIN] hashCondition=((inventory.inv_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF2 w_warehouse_sk->[inv_warehouse_sk] --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN] hashCondition=((inventory.inv_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[inv_date_sk] -------------------------hashJoin[INNER_JOIN] hashCondition=((item.i_item_sk = inventory.inv_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[inv_item_sk] ---------------------------PhysicalOlapScan[inventory] apply RFs: RF0 RF1 RF2 ---------------------------PhysicalDistribute[DistributionSpecReplicated] -----------------------------PhysicalProject -------------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) ---------------------------------PhysicalOlapScan[item] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN] hashCondition=((item.i_item_sk = inventory.inv_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[inv_item_sk] +----------------------------PhysicalOlapScan[inventory] apply RFs: RF0 RF1 RF2 +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) +----------------------------------PhysicalOlapScan[item] ------------------------PhysicalDistribute[DistributionSpecReplicated] --------------------------PhysicalProject ----------------------------filter((date_dim.d_date <= '1999-07-22') and (date_dim.d_date >= '1999-05-23')) 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 e65c48acf44cef..c560a05f544fc7 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 @@ -10,25 +10,26 @@ PhysicalResultSink --------------PhysicalProject ----------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[cs_item_sk] ------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_promo_sk = promotion.p_promo_sk)) otherCondition=() build RFs:RF2 p_promo_sk->[cs_promo_sk] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_bill_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[cs_bill_cdemo_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 RF3 -------------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------------PhysicalProject -----------------------------------filter((customer_demographics.cd_education_status = 'Unknown') and (customer_demographics.cd_gender = 'M') and (customer_demographics.cd_marital_status = 'W')) -------------------------------------PhysicalOlapScan[customer_demographics] ---------------------------PhysicalDistribute[DistributionSpecReplicated] -----------------------------PhysicalProject -------------------------------filter((date_dim.d_year = 2002)) ---------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_promo_sk = promotion.p_promo_sk)) otherCondition=() build RFs:RF2 p_promo_sk->[cs_promo_sk] ------------------------PhysicalProject ---------------------------filter(((promotion.p_channel_email = 'N') OR (promotion.p_channel_event = 'N'))) -----------------------------PhysicalOlapScan[promotion] +--------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_bill_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[cs_bill_cdemo_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 RF3 +--------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------PhysicalProject +------------------------------------filter((customer_demographics.cd_education_status = 'Unknown') and (customer_demographics.cd_gender = 'M') and (customer_demographics.cd_marital_status = 'W')) +--------------------------------------PhysicalOlapScan[customer_demographics] +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_year = 2002)) +----------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------PhysicalProject +----------------------------filter(((promotion.p_channel_email = 'N') OR (promotion.p_channel_event = 'N'))) +------------------------------PhysicalOlapScan[promotion] ------------------PhysicalDistribute[DistributionSpecHash] --------------------PhysicalProject ----------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query29.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query29.out index 51e3aaec48acc8..bfc4d4cc6b72e5 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query29.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query29.out @@ -14,31 +14,34 @@ PhysicalResultSink ----------------------PhysicalProject ------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF7 RF8 RF9 ----------------------PhysicalDistribute[DistributionSpecReplicated] -------------------------hashJoin[INNER_JOIN] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF6 s_store_sk->[ss_store_sk] ---------------------------hashJoin[INNER_JOIN] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() build RFs:RF5 i_item_sk->[ss_item_sk,sr_item_sk] -----------------------------PhysicalDistribute[DistributionSpecHash] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF2 sr_customer_sk->[ss_customer_sk];RF3 sr_item_sk->[ss_item_sk];RF4 sr_ticket_number->[ss_ticket_number] -----------------------------------hashJoin[INNER_JOIN] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF6 s_store_sk->[ss_store_sk] +----------------------------hashJoin[INNER_JOIN] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() build RFs:RF5 i_item_sk->[ss_item_sk,sr_item_sk] +------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF2 sr_customer_sk->[ss_customer_sk];RF3 sr_item_sk->[ss_item_sk];RF4 sr_ticket_number->[ss_ticket_number] ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 RF4 RF5 RF6 -------------------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------------------PhysicalProject -----------------------------------------filter((d1.d_moy = 4) and (d1.d_year = 1998)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[sr_returned_date_sk] +--------------------------------------hashJoin[INNER_JOIN] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 RF4 RF5 RF6 +----------------------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------------------PhysicalProject +--------------------------------------------filter((d1.d_moy = 4) and (d1.d_year = 1998)) +----------------------------------------------PhysicalOlapScan[date_dim] ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF0 RF5 -------------------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------------------PhysicalProject -----------------------------------------filter((d2.d_moy <= 7) and (d2.d_moy >= 4) and (d2.d_year = 1998)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[sr_returned_date_sk] +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF0 RF5 +----------------------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------------------PhysicalProject +--------------------------------------------filter((d2.d_moy <= 7) and (d2.d_moy >= 4) and (d2.d_year = 1998)) +----------------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] +----------------------------PhysicalDistribute[DistributionSpecReplicated] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[item] ---------------------------PhysicalDistribute[DistributionSpecReplicated] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[store] +--------------------------------PhysicalOlapScan[store] ------------------PhysicalDistribute[DistributionSpecReplicated] --------------------PhysicalProject ----------------------filter(d_year IN (1998, 1999, 2000)) diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query4.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query4.out index 678e8bf71abac4..3c3a2095b93778 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query4.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query4.out @@ -56,29 +56,31 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------PhysicalProject ------------------filter((t_w_firstyear.dyear = 1999) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.000000)) --------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------hashJoin[INNER_JOIN] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=() -----------------PhysicalDistribute[DistributionSpecHash] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=() +------------------PhysicalDistribute[DistributionSpecHash] +--------------------PhysicalProject +----------------------filter((t_w_secyear.dyear = 2000) and (t_w_secyear.sale_type = 'w')) +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ------------------PhysicalProject ---------------------filter((t_w_secyear.dyear = 2000) and (t_w_secyear.sale_type = 'w')) -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -----------------PhysicalProject -------------------hashJoin[INNER_JOIN] hashCondition=((t_s_firstyear.customer_id = t_c_firstyear.customer_id)) otherCondition=((if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL) > if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL))) ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------PhysicalProject -------------------------filter((t_c_firstyear.dyear = 1999) and (t_c_firstyear.sale_type = 'c') and (t_c_firstyear.year_total > 0.000000)) ---------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------------hashJoin[INNER_JOIN] hashCondition=((t_s_firstyear.customer_id = t_c_secyear.customer_id)) otherCondition=() -----------------------hashJoin[INNER_JOIN] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------filter((t_s_firstyear.dyear = 1999) and (t_s_firstyear.sale_type = 's') and (t_s_firstyear.year_total > 0.000000)) -------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------filter((t_s_secyear.dyear = 2000) and (t_s_secyear.sale_type = 's')) -------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------hashJoin[INNER_JOIN] hashCondition=((t_s_firstyear.customer_id = t_c_firstyear.customer_id)) otherCondition=((if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL) > if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL))) ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------PhysicalProject ---------------------------filter((t_c_secyear.dyear = 2000) and (t_c_secyear.sale_type = 'c')) +--------------------------filter((t_c_firstyear.dyear = 1999) and (t_c_firstyear.sale_type = 'c') and (t_c_firstyear.year_total > 0.000000)) ----------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN] hashCondition=((t_s_firstyear.customer_id = t_c_secyear.customer_id)) otherCondition=() +--------------------------hashJoin[INNER_JOIN] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------filter((t_s_firstyear.dyear = 1999) and (t_s_firstyear.sale_type = 's') and (t_s_firstyear.year_total > 0.000000)) +----------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------filter((t_s_secyear.dyear = 2000) and (t_s_secyear.sale_type = 's')) +----------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------PhysicalProject +------------------------------filter((t_c_secyear.dyear = 2000) and (t_c_secyear.sale_type = 'c')) +--------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query40.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query40.out index ebeb4e5e09aa6f..77696a636590a3 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query40.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query40.out @@ -13,18 +13,19 @@ PhysicalResultSink --------------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() build RFs:RF2 cs_order_number->[cr_order_number];RF3 cs_item_sk->[cr_item_sk] ----------------------PhysicalProject ------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF2 RF3 -----------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] -------------------------hashJoin[INNER_JOIN] 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 RF4 +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +--------------------------hashJoin[INNER_JOIN] 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 RF4 +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) +----------------------------------PhysicalOlapScan[item] --------------------------PhysicalDistribute[DistributionSpecReplicated] ----------------------------PhysicalProject -------------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) ---------------------------------PhysicalOlapScan[item] -------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------PhysicalProject -----------------------------filter((date_dim.d_date <= '2001-06-01') and (date_dim.d_date >= '2001-04-02')) -------------------------------PhysicalOlapScan[date_dim] +------------------------------filter((date_dim.d_date <= '2001-06-01') and (date_dim.d_date >= '2001-04-02')) +--------------------------------PhysicalOlapScan[date_dim] ------------------PhysicalDistribute[DistributionSpecReplicated] --------------------PhysicalProject ----------------------PhysicalOlapScan[warehouse] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query42.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query42.out index 3b2b282a6b0e0e..93f77299d582b0 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query42.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query42.out @@ -9,13 +9,14 @@ PhysicalResultSink ------------hashAgg[LOCAL] --------------PhysicalProject ----------------hashJoin[INNER_JOIN] hashCondition=((dt.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] -------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] ---------------------PhysicalProject -----------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 ---------------------PhysicalDistribute[DistributionSpecReplicated] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] ----------------------PhysicalProject -------------------------filter((item.i_manager_id = 1)) ---------------------------PhysicalOlapScan[item] +------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 +----------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------PhysicalProject +--------------------------filter((item.i_manager_id = 1)) +----------------------------PhysicalOlapScan[item] ------------------PhysicalDistribute[DistributionSpecReplicated] --------------------PhysicalProject ----------------------filter((dt.d_moy = 11) and (dt.d_year = 1998)) diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query44.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query44.out index 5dddbe8e54335c..f37d21eef25020 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query44.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query44.out @@ -39,34 +39,35 @@ PhysicalResultSink ------------------------------------------------------filter((store_sales.ss_store_sk = 4) and ss_hdemo_sk IS NULL) --------------------------------------------------------PhysicalOlapScan[store_sales] ------------PhysicalDistribute[DistributionSpecHash] ---------------hashJoin[INNER_JOIN] hashCondition=((i2.i_item_sk = descending.item_sk)) otherCondition=() build RFs:RF0 item_sk->[i_item_sk] -----------------PhysicalProject -------------------PhysicalOlapScan[item] apply RFs: RF0 -----------------PhysicalDistribute[DistributionSpecHash] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN] hashCondition=((i2.i_item_sk = descending.item_sk)) otherCondition=() build RFs:RF0 item_sk->[i_item_sk] ------------------PhysicalProject ---------------------filter((rnk < 11)) -----------------------PhysicalWindow -------------------------PhysicalQuickSort[MERGE_SORT] ---------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------PhysicalQuickSort[LOCAL_SORT] -------------------------------PhysicalPartitionTopN ---------------------------------PhysicalProject -----------------------------------NestedLoopJoin[INNER_JOIN](cast(rank_col as DOUBLE) > cast((0.9 * rank_col) as DOUBLE)) -------------------------------------PhysicalProject ---------------------------------------hashAgg[GLOBAL] -----------------------------------------PhysicalDistribute[DistributionSpecHash] -------------------------------------------hashAgg[LOCAL] +--------------------PhysicalOlapScan[item] apply RFs: RF0 +------------------PhysicalDistribute[DistributionSpecHash] +--------------------PhysicalProject +----------------------filter((rnk < 11)) +------------------------PhysicalWindow +--------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------PhysicalDistribute[DistributionSpecGather] +------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------PhysicalPartitionTopN +----------------------------------PhysicalProject +------------------------------------NestedLoopJoin[INNER_JOIN](cast(rank_col as DOUBLE) > cast((0.9 * rank_col) as DOUBLE)) +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] +----------------------------------------------PhysicalProject +------------------------------------------------filter((ss1.ss_store_sk = 4)) +--------------------------------------------------PhysicalOlapScan[store_sales] +--------------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------------PhysicalAssertNumRows +------------------------------------------PhysicalDistribute[DistributionSpecGather] --------------------------------------------PhysicalProject -----------------------------------------------filter((ss1.ss_store_sk = 4)) -------------------------------------------------PhysicalOlapScan[store_sales] -------------------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------------------PhysicalAssertNumRows -----------------------------------------PhysicalDistribute[DistributionSpecGather] -------------------------------------------PhysicalProject ---------------------------------------------hashAgg[GLOBAL] -----------------------------------------------PhysicalDistribute[DistributionSpecHash] -------------------------------------------------hashAgg[LOCAL] ---------------------------------------------------PhysicalProject -----------------------------------------------------filter((store_sales.ss_store_sk = 4) and ss_hdemo_sk IS NULL) -------------------------------------------------------PhysicalOlapScan[store_sales] +----------------------------------------------hashAgg[GLOBAL] +------------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------------hashAgg[LOCAL] +----------------------------------------------------PhysicalProject +------------------------------------------------------filter((store_sales.ss_store_sk = 4) and ss_hdemo_sk IS NULL) +--------------------------------------------------------PhysicalOlapScan[store_sales] 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 4069222cacbfa9..6ed83d820ce9a4 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 @@ -11,14 +11,15 @@ PhysicalResultSink ----------------PhysicalProject ------------------hashJoin[INNER_JOIN] 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] --------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashJoin[INNER_JOIN] 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] -------------------------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 -------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN] 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] --------------------------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')))) -------------------------------PhysicalOlapScan[customer_demographics] +----------------------------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 +--------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------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')))) +--------------------------------PhysicalOlapScan[customer_demographics] --------------------PhysicalDistribute[DistributionSpecHash] ----------------------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/query58.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query58.out index 0d58db1ab5c111..0ac0f7a1c0a30e 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query58.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query58.out @@ -35,63 +35,64 @@ PhysicalResultSink ------------------------PhysicalDistribute[DistributionSpecHash] --------------------------PhysicalProject ----------------------------PhysicalOlapScan[item] apply RFs: RF13 -------------hashJoin[INNER_JOIN] hashCondition=((ss_items.item_id = ws_items.item_id)) otherCondition=((cast(ss_item_rev as DOUBLE) <= cast((1.1 * ws_item_rev) as DOUBLE)) and (cast(ss_item_rev as DOUBLE) >= cast((0.9 * ws_item_rev) as DOUBLE)) and (cast(ws_item_rev as DOUBLE) <= cast((1.1 * ss_item_rev) as DOUBLE)) and (cast(ws_item_rev as DOUBLE) >= cast((0.9 * ss_item_rev) as DOUBLE))) build RFs:RF8 item_id->[i_item_id] ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF7 i_item_sk->[ss_item_sk] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ss_sold_date_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 RF7 -------------------------------PhysicalDistribute[DistributionSpecReplicated] +------------PhysicalProject +--------------hashJoin[INNER_JOIN] hashCondition=((ss_items.item_id = ws_items.item_id)) otherCondition=((cast(ss_item_rev as DOUBLE) <= cast((1.1 * ws_item_rev) as DOUBLE)) and (cast(ss_item_rev as DOUBLE) >= cast((0.9 * ws_item_rev) as DOUBLE)) and (cast(ws_item_rev as DOUBLE) <= cast((1.1 * ss_item_rev) as DOUBLE)) and (cast(ws_item_rev as DOUBLE) >= cast((0.9 * ss_item_rev) as DOUBLE))) build RFs:RF8 item_id->[i_item_id] +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF7 i_item_sk->[ss_item_sk] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ss_sold_date_sk] --------------------------------PhysicalProject -----------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_date = date_dim.d_date)) otherCondition=() build RFs:RF5 d_date->[d_date] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF5 -------------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 RF7 +--------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------PhysicalProject +------------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_date = date_dim.d_date)) otherCondition=() build RFs:RF5 d_date->[d_date] --------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN] hashCondition=((date_dim.d_week_seq = date_dim.d_week_seq)) otherCondition=() build RFs:RF4 d_week_seq->[d_week_seq] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF4 -------------------------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------------------------PhysicalAssertNumRows -----------------------------------------------PhysicalDistribute[DistributionSpecGather] -------------------------------------------------PhysicalProject ---------------------------------------------------filter((date_dim.d_date = '2001-06-16')) -----------------------------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] apply RFs: RF8 ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] +----------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF5 +--------------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN] hashCondition=((date_dim.d_week_seq = date_dim.d_week_seq)) otherCondition=() build RFs:RF4 d_week_seq->[d_week_seq] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF4 +--------------------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------------------PhysicalAssertNumRows +------------------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_date = '2001-06-16')) +------------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------PhysicalDistribute[DistributionSpecHash] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[web_sales] apply RFs: RF2 RF3 -------------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------------PhysicalOlapScan[item] apply RFs: RF8 +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] --------------------------------PhysicalProject -----------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_date = date_dim.d_date)) otherCondition=() build RFs:RF1 d_date->[d_date] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF1 -------------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF2 RF3 +--------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------PhysicalProject +------------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_date = date_dim.d_date)) otherCondition=() build RFs:RF1 d_date->[d_date] --------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN] hashCondition=((date_dim.d_week_seq = date_dim.d_week_seq)) otherCondition=() build RFs:RF0 d_week_seq->[d_week_seq] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF0 -------------------------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------------------------PhysicalAssertNumRows -----------------------------------------------PhysicalDistribute[DistributionSpecGather] -------------------------------------------------PhysicalProject ---------------------------------------------------filter((date_dim.d_date = '2001-06-16')) -----------------------------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] +----------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF1 +--------------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN] hashCondition=((date_dim.d_week_seq = date_dim.d_week_seq)) otherCondition=() build RFs:RF0 d_week_seq->[d_week_seq] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF0 +--------------------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------------------PhysicalAssertNumRows +------------------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_date = '2001-06-16')) +------------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query62.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query62.out index 1a451677dc00f0..5797571c093881 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query62.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query62.out @@ -11,17 +11,18 @@ PhysicalResultSink ----------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF3 w_warehouse_sk->[ws_warehouse_sk] ------------------PhysicalProject --------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF2 sm_ship_mode_sk->[ws_ship_mode_sk] -----------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF1 web_site_sk->[ws_web_site_sk] -------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_ship_date_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 RF3 +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF1 web_site_sk->[ws_web_site_sk] +--------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_ship_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 RF3 +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_month_seq <= 1234) and (date_dim.d_month_seq >= 1223)) +----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalDistribute[DistributionSpecReplicated] ----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq <= 1234) and (date_dim.d_month_seq >= 1223)) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[web_site] +------------------------------PhysicalOlapScan[web_site] ----------------------PhysicalDistribute[DistributionSpecReplicated] ------------------------PhysicalProject --------------------------PhysicalOlapScan[ship_mode] 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 11d575eff53104..75e48c17970ff0 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 @@ -10,25 +10,26 @@ PhysicalResultSink --------------PhysicalProject ----------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] ------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_promo_sk = promotion.p_promo_sk)) otherCondition=() build RFs:RF2 p_promo_sk->[ss_promo_sk] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN] 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] hashCondition=((store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 -------------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------------PhysicalProject -----------------------------------filter((customer_demographics.cd_education_status = 'College') and (customer_demographics.cd_gender = 'F') and (customer_demographics.cd_marital_status = 'W')) -------------------------------------PhysicalOlapScan[customer_demographics] ---------------------------PhysicalDistribute[DistributionSpecReplicated] -----------------------------PhysicalProject -------------------------------filter((date_dim.d_year = 2001)) ---------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_promo_sk = promotion.p_promo_sk)) otherCondition=() build RFs:RF2 p_promo_sk->[ss_promo_sk] ------------------------PhysicalProject ---------------------------filter(((promotion.p_channel_email = 'N') OR (promotion.p_channel_event = 'N'))) -----------------------------PhysicalOlapScan[promotion] +--------------------------hashJoin[INNER_JOIN] 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] hashCondition=((store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 +--------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------PhysicalProject +------------------------------------filter((customer_demographics.cd_education_status = 'College') and (customer_demographics.cd_gender = 'F') and (customer_demographics.cd_marital_status = 'W')) +--------------------------------------PhysicalOlapScan[customer_demographics] +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_year = 2001)) +----------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------PhysicalProject +----------------------------filter(((promotion.p_channel_email = 'N') OR (promotion.p_channel_event = 'N'))) +------------------------------PhysicalOlapScan[promotion] ------------------PhysicalDistribute[DistributionSpecHash] --------------------PhysicalProject ----------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query72.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query72.out index 6ec1d4a1517445..2e113a8486414b 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query72.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query72.out @@ -16,46 +16,47 @@ PhysicalResultSink ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------PhysicalProject --------------------------hashJoin[INNER_JOIN] hashCondition=((d1.d_week_seq = d2.d_week_seq)) otherCondition=() build RFs:RF7 d_week_seq->[d_week_seq] -----------------------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((catalog_returns.cr_item_sk = catalog_sales.cs_item_sk) and (catalog_returns.cr_order_number = catalog_sales.cs_order_number)) otherCondition=() build RFs:RF5 cs_item_sk->[cr_item_sk];RF6 cs_order_number->[cr_order_number] -------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF5 RF6 -------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF4 i_item_sk->[cs_item_sk] -------------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------------PhysicalProject -----------------------------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((catalog_sales.cs_promo_sk = promotion.p_promo_sk)) otherCondition=() -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_bill_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF3 cd_demo_sk->[cs_bill_cdemo_sk] -----------------------------------------------PhysicalDistribute[DistributionSpecHash] -------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_bill_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF2 hd_demo_sk->[cs_bill_hdemo_sk] ---------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_ship_date_sk = d3.d_date_sk) and (catalog_sales.cs_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_ship_date_sk];RF1 d_date_sk->[cs_sold_date_sk] -----------------------------------------------------PhysicalDistribute[DistributionSpecHash] -------------------------------------------------------PhysicalProject ---------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 RF3 RF4 -----------------------------------------------------PhysicalDistribute[DistributionSpecHash] -------------------------------------------------------NestedLoopJoin[INNER_JOIN](d3.d_date > days_add(d_date, INTERVAL 5 DAY)) +----------------------------PhysicalProject +------------------------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((catalog_returns.cr_item_sk = catalog_sales.cs_item_sk) and (catalog_returns.cr_order_number = catalog_sales.cs_order_number)) otherCondition=() build RFs:RF5 cs_item_sk->[cr_item_sk];RF6 cs_order_number->[cr_order_number] +--------------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF5 RF6 +--------------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF4 i_item_sk->[cs_item_sk] +--------------------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((catalog_sales.cs_promo_sk = promotion.p_promo_sk)) otherCondition=() +--------------------------------------------PhysicalProject +----------------------------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_bill_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF3 cd_demo_sk->[cs_bill_cdemo_sk] +------------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_bill_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF2 hd_demo_sk->[cs_bill_hdemo_sk] +----------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_ship_date_sk = d3.d_date_sk) and (catalog_sales.cs_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_ship_date_sk];RF1 d_date_sk->[cs_sold_date_sk] +------------------------------------------------------PhysicalDistribute[DistributionSpecHash] --------------------------------------------------------PhysicalProject -----------------------------------------------------------PhysicalOlapScan[date_dim] ---------------------------------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 RF3 RF4 +------------------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------------------NestedLoopJoin[INNER_JOIN](d3.d_date > days_add(d_date, INTERVAL 5 DAY)) ----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((d1.d_year = 1998)) ---------------------------------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF7 ---------------------------------------------------PhysicalDistribute[DistributionSpecReplicated] -----------------------------------------------------PhysicalProject -------------------------------------------------------filter((household_demographics.hd_buy_potential = '1001-5000')) ---------------------------------------------------------PhysicalOlapScan[household_demographics] -----------------------------------------------PhysicalDistribute[DistributionSpecHash] -------------------------------------------------PhysicalProject ---------------------------------------------------filter((customer_demographics.cd_marital_status = 'S')) -----------------------------------------------------PhysicalOlapScan[customer_demographics] -------------------------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[promotion] -------------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[item] +------------------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------filter((d1.d_year = 1998)) +----------------------------------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF7 +----------------------------------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((household_demographics.hd_buy_potential = '1001-5000')) +----------------------------------------------------------PhysicalOlapScan[household_demographics] +------------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((customer_demographics.cd_marital_status = 'S')) +------------------------------------------------------PhysicalOlapScan[customer_demographics] +--------------------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------------------PhysicalProject +------------------------------------------------PhysicalOlapScan[promotion] +--------------------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[item] ----------------------------PhysicalDistribute[DistributionSpecReplicated] ------------------------------PhysicalProject --------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query74.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query74.out index 058f4eafb7caf4..6a1bf01f3920fa 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query74.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query74.out @@ -44,18 +44,19 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------PhysicalProject ------------------filter((t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year = 1999) and (t_w_firstyear.year_total > 0.00)) --------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------hashJoin[INNER_JOIN] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=() -----------------hashJoin[INNER_JOIN] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalProject -----------------------filter((t_s_firstyear.sale_type = 's') and (t_s_firstyear.year = 1999) and (t_s_firstyear.year_total > 0.00)) -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------PhysicalProject +----------------hashJoin[INNER_JOIN] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=() +------------------hashJoin[INNER_JOIN] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------filter((t_s_firstyear.sale_type = 's') and (t_s_firstyear.year = 1999) and (t_s_firstyear.year_total > 0.00)) +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------filter((t_s_secyear.sale_type = 's') and (t_s_secyear.year = 2000)) +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ------------------PhysicalDistribute[DistributionSpecHash] --------------------PhysicalProject -----------------------filter((t_s_secyear.sale_type = 's') and (t_s_secyear.year = 2000)) +----------------------filter((t_w_secyear.sale_type = 'w') and (t_w_secyear.year = 2000)) ------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -----------------PhysicalDistribute[DistributionSpecHash] -------------------PhysicalProject ---------------------filter((t_w_secyear.sale_type = 'w') and (t_w_secyear.year = 2000)) -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query76.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query76.out index 23da0604355465..4c120b380e004d 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query76.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query76.out @@ -11,13 +11,14 @@ PhysicalResultSink ----------------PhysicalDistribute[DistributionSpecExecutionAny] ------------------PhysicalProject --------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[ss_item_sk] -----------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] -------------------------PhysicalProject ---------------------------filter(ss_customer_sk IS NULL) -----------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 -------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN] 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[date_dim] +----------------------------filter(ss_customer_sk IS NULL) +------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 +--------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalDistribute[DistributionSpecReplicated] ------------------------PhysicalProject --------------------------PhysicalOlapScan[item] @@ -27,25 +28,27 @@ PhysicalResultSink ----------------------PhysicalProject ------------------------PhysicalOlapScan[item] apply RFs: RF3 ----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------filter(ws_promo_sk IS NULL) +----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF2 +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[date_dim] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF5 i_item_sk->[cs_item_sk] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[cs_sold_date_sk] --------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------PhysicalProject -------------------------------filter(ws_promo_sk IS NULL) ---------------------------------PhysicalOlapScan[web_sales] apply RFs: RF2 +------------------------------filter(cs_bill_customer_sk IS NULL) +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 RF5 --------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalProject -------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF5 i_item_sk->[cs_item_sk] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[cs_sold_date_sk] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------filter(cs_bill_customer_sk IS NULL) -------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 RF5 -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[date_dim] --------------------PhysicalDistribute[DistributionSpecHash] ----------------------PhysicalProject ------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query83.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query83.out index 98c0d70b202a65..764365271a1ec7 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query83.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query83.out @@ -33,59 +33,60 @@ PhysicalResultSink ------------------------PhysicalDistribute[DistributionSpecHash] --------------------------PhysicalProject ----------------------------PhysicalOlapScan[item] apply RFs: RF13 -------------hashJoin[INNER_JOIN] hashCondition=((sr_items.item_id = wr_items.item_id)) otherCondition=() build RFs:RF8 item_id->[i_item_id] ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF7 i_item_sk->[sr_item_sk] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_returned_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[sr_returned_date_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_returns] apply RFs: RF6 RF7 -------------------------------PhysicalDistribute[DistributionSpecReplicated] +------------PhysicalProject +--------------hashJoin[INNER_JOIN] hashCondition=((sr_items.item_id = wr_items.item_id)) otherCondition=() build RFs:RF8 item_id->[i_item_id] +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF7 i_item_sk->[sr_item_sk] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_returned_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[sr_returned_date_sk] --------------------------------PhysicalProject -----------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_date = date_dim.d_date)) otherCondition=() build RFs:RF5 d_date->[d_date] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF5 -------------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------PhysicalOlapScan[store_returns] apply RFs: RF6 RF7 +--------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------PhysicalProject +------------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_date = date_dim.d_date)) otherCondition=() build RFs:RF5 d_date->[d_date] --------------------------------------PhysicalProject -----------------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_week_seq = date_dim.d_week_seq)) otherCondition=() build RFs:RF4 d_week_seq->[d_week_seq] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF4 -------------------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF5 +--------------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_week_seq = date_dim.d_week_seq)) otherCondition=() build RFs:RF4 d_week_seq->[d_week_seq] --------------------------------------------PhysicalProject -----------------------------------------------filter(d_date IN ('2001-07-13', '2001-09-10', '2001-11-16')) -------------------------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] apply RFs: RF8 ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN] hashCondition=((web_returns.wr_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[wr_item_sk] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashJoin[INNER_JOIN] hashCondition=((web_returns.wr_returned_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[wr_returned_date_sk] +----------------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF4 +--------------------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------------------PhysicalProject +------------------------------------------------filter(d_date IN ('2001-07-13', '2001-09-10', '2001-11-16')) +--------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------PhysicalDistribute[DistributionSpecHash] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[web_returns] apply RFs: RF2 RF3 -------------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------------PhysicalOlapScan[item] apply RFs: RF8 +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN] hashCondition=((web_returns.wr_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[wr_item_sk] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------hashJoin[INNER_JOIN] hashCondition=((web_returns.wr_returned_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[wr_returned_date_sk] --------------------------------PhysicalProject -----------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_date = date_dim.d_date)) otherCondition=() build RFs:RF1 d_date->[d_date] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF1 -------------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------PhysicalOlapScan[web_returns] apply RFs: RF2 RF3 +--------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------PhysicalProject +------------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_date = date_dim.d_date)) otherCondition=() build RFs:RF1 d_date->[d_date] --------------------------------------PhysicalProject -----------------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_week_seq = date_dim.d_week_seq)) otherCondition=() build RFs:RF0 d_week_seq->[d_week_seq] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF0 -------------------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF1 +--------------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_week_seq = date_dim.d_week_seq)) otherCondition=() build RFs:RF0 d_week_seq->[d_week_seq] --------------------------------------------PhysicalProject -----------------------------------------------filter(d_date IN ('2001-07-13', '2001-09-10', '2001-11-16')) -------------------------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] +----------------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF0 +--------------------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------------------PhysicalProject +------------------------------------------------filter(d_date IN ('2001-07-13', '2001-09-10', '2001-11-16')) +--------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query93.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query93.out index 68052ea4a3f3a3..45ec7cf03cd327 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query93.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query93.out @@ -11,11 +11,12 @@ PhysicalResultSink ----------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_item_sk = store_sales.ss_item_sk) and (store_returns.sr_ticket_number = store_sales.ss_ticket_number)) otherCondition=() build RFs:RF1 sr_item_sk->[ss_item_sk];RF2 sr_ticket_number->[ss_ticket_number] ------------------PhysicalProject --------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 -------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_reason_sk = reason.r_reason_sk)) otherCondition=() build RFs:RF0 r_reason_sk->[sr_reason_sk] ---------------------PhysicalProject -----------------------PhysicalOlapScan[store_returns] apply RFs: RF0 ---------------------PhysicalDistribute[DistributionSpecReplicated] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_reason_sk = reason.r_reason_sk)) otherCondition=() build RFs:RF0 r_reason_sk->[sr_reason_sk] ----------------------PhysicalProject -------------------------filter((reason.r_reason_desc = 'reason 58')) ---------------------------PhysicalOlapScan[reason] +------------------------PhysicalOlapScan[store_returns] apply RFs: RF0 +----------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------PhysicalProject +--------------------------filter((reason.r_reason_desc = 'reason 58')) +----------------------------PhysicalOlapScan[reason] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query94.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query94.out index 4868b244dcaedd..c78c35b5af25b2 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query94.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query94.out @@ -13,26 +13,27 @@ PhysicalResultSink --------------------PhysicalDistribute[DistributionSpecHash] ----------------------PhysicalProject ------------------------PhysicalOlapScan[web_sales] apply RFs: RF4 ---------------------hashJoin[RIGHT_ANTI_JOIN] hashCondition=((ws1.ws_order_number = wr1.wr_order_number)) otherCondition=() build RFs:RF3 ws_order_number->[wr_order_number] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[web_returns] apply RFs: RF3 -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF2 web_site_sk->[ws_web_site_sk] ---------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_ship_date_sk] -----------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[ws_ship_addr_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 +--------------------PhysicalProject +----------------------hashJoin[RIGHT_ANTI_JOIN] hashCondition=((ws1.ws_order_number = wr1.wr_order_number)) otherCondition=() build RFs:RF3 ws_order_number->[wr_order_number] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[web_returns] apply RFs: RF3 +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF2 web_site_sk->[ws_web_site_sk] +----------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_ship_date_sk] +------------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[ws_ship_addr_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 +--------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------PhysicalProject +------------------------------------filter((customer_address.ca_state = 'OK')) +--------------------------------------PhysicalOlapScan[customer_address] ------------------------------PhysicalDistribute[DistributionSpecReplicated] --------------------------------PhysicalProject -----------------------------------filter((customer_address.ca_state = 'OK')) -------------------------------------PhysicalOlapScan[customer_address] +----------------------------------filter((date_dim.d_date <= '2002-06-30') and (date_dim.d_date >= '2002-05-01')) +------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalDistribute[DistributionSpecReplicated] ------------------------------PhysicalProject ---------------------------------filter((date_dim.d_date <= '2002-06-30') and (date_dim.d_date >= '2002-05-01')) -----------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute[DistributionSpecReplicated] -----------------------------PhysicalProject -------------------------------filter((web_site.web_company_name = 'pri')) ---------------------------------PhysicalOlapScan[web_site] +--------------------------------filter((web_site.web_company_name = 'pri')) +----------------------------------PhysicalOlapScan[web_site] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query95.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query95.out index bb7864faabb5f9..c9c64a64e35fe7 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query95.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query95.out @@ -28,26 +28,27 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[web_returns] apply RFs: RF6 -----------------------hashJoin[RIGHT_SEMI_JOIN] hashCondition=((ws1.ws_order_number = ws_wh.ws_order_number)) otherCondition=() build RFs:RF7 ws_order_number->[ws_order_number,ws_order_number] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF3 web_site_sk->[ws_web_site_sk] -----------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_ship_date_sk] -------------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF1 ca_address_sk->[ws_ship_addr_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF2 RF3 +----------------------PhysicalProject +------------------------hashJoin[RIGHT_SEMI_JOIN] hashCondition=((ws1.ws_order_number = ws_wh.ws_order_number)) otherCondition=() build RFs:RF7 ws_order_number->[ws_order_number,ws_order_number] +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------PhysicalProject +------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF3 web_site_sk->[ws_web_site_sk] +------------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_ship_date_sk] +--------------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF1 ca_address_sk->[ws_ship_addr_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF2 RF3 +----------------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------------PhysicalProject +--------------------------------------filter((customer_address.ca_state = 'VA')) +----------------------------------------PhysicalOlapScan[customer_address] --------------------------------PhysicalDistribute[DistributionSpecReplicated] ----------------------------------PhysicalProject -------------------------------------filter((customer_address.ca_state = 'VA')) ---------------------------------------PhysicalOlapScan[customer_address] +------------------------------------filter((date_dim.d_date <= '2001-05-31') and (date_dim.d_date >= '2001-04-01')) +--------------------------------------PhysicalOlapScan[date_dim] ------------------------------PhysicalDistribute[DistributionSpecReplicated] --------------------------------PhysicalProject -----------------------------------filter((date_dim.d_date <= '2001-05-31') and (date_dim.d_date >= '2001-04-01')) -------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalDistribute[DistributionSpecReplicated] -------------------------------PhysicalProject ---------------------------------filter((web_site.web_company_name = 'pri')) -----------------------------------PhysicalOlapScan[web_site] +----------------------------------filter((web_site.web_company_name = 'pri')) +------------------------------------PhysicalOlapScan[web_site] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query99.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query99.out index b8d1afb99a9f66..ee7ff237ab80eb 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query99.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query99.out @@ -11,17 +11,18 @@ PhysicalResultSink ----------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF3 w_warehouse_sk->[cs_warehouse_sk] ------------------PhysicalProject --------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF2 sm_ship_mode_sk->[cs_ship_mode_sk] -----------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_call_center_sk = call_center.cc_call_center_sk)) otherCondition=() build RFs:RF1 cc_call_center_sk->[cs_call_center_sk] -------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_ship_date_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 RF3 +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_call_center_sk = call_center.cc_call_center_sk)) otherCondition=() build RFs:RF1 cc_call_center_sk->[cs_call_center_sk] +--------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_ship_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 RF3 +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_month_seq <= 1205) and (date_dim.d_month_seq >= 1194)) +----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalDistribute[DistributionSpecReplicated] ----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq <= 1205) and (date_dim.d_month_seq >= 1194)) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[call_center] +------------------------------PhysicalOlapScan[call_center] ----------------------PhysicalDistribute[DistributionSpecReplicated] ------------------------PhysicalProject --------------------------PhysicalOlapScan[ship_mode] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query16.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query16.out index 4e0cd864a37abe..8e5966e76f7751 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query16.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query16.out @@ -10,28 +10,29 @@ PhysicalResultSink --------------hashAgg[LOCAL] ----------------PhysicalProject ------------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((cs1.cs_order_number = cr1.cr_order_number)) otherCondition=() ---------------------hashJoin[INNER_JOIN] hashCondition=((cs1.cs_call_center_sk = call_center.cc_call_center_sk)) otherCondition=() build RFs:RF3 cc_call_center_sk->[cs_call_center_sk] -----------------------hashJoin[RIGHT_SEMI_JOIN] hashCondition=((cs1.cs_order_number = cs2.cs_order_number)) otherCondition=(( not (cs_warehouse_sk = cs_warehouse_sk))) build RFs:RF2 cs_order_number->[cs_order_number] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------hashJoin[INNER_JOIN] hashCondition=((cs1.cs_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF1 ca_address_sk->[cs_ship_addr_sk] -----------------------------hashJoin[INNER_JOIN] hashCondition=((cs1.cs_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_ship_date_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF3 +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN] hashCondition=((cs1.cs_call_center_sk = call_center.cc_call_center_sk)) otherCondition=() build RFs:RF3 cc_call_center_sk->[cs_call_center_sk] +------------------------hashJoin[RIGHT_SEMI_JOIN] hashCondition=((cs1.cs_order_number = cs2.cs_order_number)) otherCondition=(( not (cs_warehouse_sk = cs_warehouse_sk))) build RFs:RF2 cs_order_number->[cs_order_number] +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------hashJoin[INNER_JOIN] hashCondition=((cs1.cs_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF1 ca_address_sk->[cs_ship_addr_sk] +------------------------------hashJoin[INNER_JOIN] hashCondition=((cs1.cs_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_ship_date_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF3 +--------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------PhysicalProject +------------------------------------filter((date_dim.d_date <= '2002-05-31') and (date_dim.d_date >= '2002-04-01')) +--------------------------------------PhysicalOlapScan[date_dim] ------------------------------PhysicalDistribute[DistributionSpecReplicated] --------------------------------PhysicalProject -----------------------------------filter((date_dim.d_date <= '2002-05-31') and (date_dim.d_date >= '2002-04-01')) -------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalDistribute[DistributionSpecReplicated] -------------------------------PhysicalProject ---------------------------------filter((customer_address.ca_state = 'WV')) -----------------------------------PhysicalOlapScan[customer_address] -----------------------PhysicalDistribute[DistributionSpecReplicated] -------------------------PhysicalProject ---------------------------filter(cc_county IN ('Barrow County', 'Daviess County', 'Luce County', 'Richland County', 'Ziebach County')) -----------------------------PhysicalOlapScan[call_center] +----------------------------------filter((customer_address.ca_state = 'WV')) +------------------------------------PhysicalOlapScan[customer_address] +------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------PhysicalProject +----------------------------filter(cc_county IN ('Barrow County', 'Daviess County', 'Luce County', 'Richland County', 'Ziebach County')) +------------------------------PhysicalOlapScan[call_center] --------------------PhysicalDistribute[DistributionSpecHash] ----------------------PhysicalProject ------------------------PhysicalOlapScan[catalog_returns] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query2.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query2.out index d5d35f846a560c..390dc119a1e1ee 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query2.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query2.out @@ -23,20 +23,21 @@ PhysicalCteAnchor ( cteId=CTEId#1 ) --------PhysicalQuickSort[LOCAL_SORT] ----------PhysicalProject ------------hashJoin[INNER_JOIN] hashCondition=((date_dim.d_week_seq = d_week_seq2)) otherCondition=() build RFs:RF2 d_week_seq->[d_week_seq] ---------------hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_week_seq1 as BIGINT) = expr_(d_week_seq2 - 53))) otherCondition=() -----------------PhysicalDistribute[DistributionSpecHash] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN] hashCondition=((date_dim.d_week_seq = d_week_seq1)) otherCondition=() build RFs:RF1 d_week_seq->[d_week_seq] -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) apply RFs: RF1 -----------------------PhysicalDistribute[DistributionSpecReplicated] -------------------------PhysicalProject ---------------------------filter((date_dim.d_year = 1998)) -----------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecHash] -------------------PhysicalProject ---------------------PhysicalCteConsumer ( cteId=CTEId#1 ) apply RFs: RF2 +--------------PhysicalProject +----------------hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_week_seq1 as BIGINT) = expr_(d_week_seq2 - 53))) otherCondition=() +------------------PhysicalDistribute[DistributionSpecHash] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN] hashCondition=((date_dim.d_week_seq = d_week_seq1)) otherCondition=() build RFs:RF1 d_week_seq->[d_week_seq] +------------------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------------PhysicalProject +----------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) apply RFs: RF1 +------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------PhysicalProject +----------------------------filter((date_dim.d_year = 1998)) +------------------------------PhysicalOlapScan[date_dim] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------PhysicalProject +----------------------PhysicalCteConsumer ( cteId=CTEId#1 ) apply RFs: RF2 --------------PhysicalDistribute[DistributionSpecReplicated] ----------------PhysicalProject ------------------filter((date_dim.d_year = 1999)) 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 d8aa5338964dbf..a357addc100c44 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 @@ -57,36 +57,38 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------PhysicalProject --------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((catalog_sales.cs_item_sk = frequent_ss_items.item_sk)) otherCondition=() ----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] ---------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((catalog_sales.cs_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() -----------------------------PhysicalDistribute[DistributionSpecHash] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 -----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] +----------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((catalog_sales.cs_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() +------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 +------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------PhysicalProject +----------------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) +----------------------------PhysicalDistribute[DistributionSpecReplicated] ------------------------------PhysicalProject ---------------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) ---------------------------PhysicalDistribute[DistributionSpecReplicated] -----------------------------PhysicalProject -------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) ---------------------------------PhysicalOlapScan[date_dim] +--------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) +----------------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------PhysicalProject --------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ------------------PhysicalProject --------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((web_sales.ws_item_sk = frequent_ss_items.item_sk)) otherCondition=() ----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ws_sold_date_sk] ---------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((web_sales.ws_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() -----------------------------PhysicalDistribute[DistributionSpecHash] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[web_sales] apply RFs: RF4 -----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ws_sold_date_sk] +----------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((web_sales.ws_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() +------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF4 +------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------PhysicalProject +----------------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) +----------------------------PhysicalDistribute[DistributionSpecReplicated] ------------------------------PhysicalProject ---------------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) ---------------------------PhysicalDistribute[DistributionSpecReplicated] -----------------------------PhysicalProject -------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) ---------------------------------PhysicalOlapScan[date_dim] +--------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) +----------------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------PhysicalProject --------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query25.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query25.out index 2cbe6131465cc9..7aad2def1929a7 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query25.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query25.out @@ -9,40 +9,41 @@ PhysicalResultSink ------------hashAgg[LOCAL] --------------PhysicalProject ----------------hashJoin[INNER_JOIN] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() -------------------hashJoin[INNER_JOIN] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[cs_sold_date_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[sr_returned_date_sk] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[ss_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) and (store_returns.sr_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF3 cs_bill_customer_sk->[ss_customer_sk,sr_customer_sk];RF4 cs_item_sk->[ss_item_sk,sr_item_sk] -------------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF3 RF4 RF5 -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF3 RF4 RF6 -------------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF7 ---------------------------------PhysicalDistribute[DistributionSpecReplicated] -----------------------------------PhysicalProject -------------------------------------filter((d1.d_moy = 4) and (d1.d_year = 2000)) ---------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalDistribute[DistributionSpecReplicated] -------------------------------PhysicalProject ---------------------------------filter((d2.d_moy <= 10) and (d2.d_moy >= 4) and (d2.d_year = 2000)) -----------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------PhysicalProject -----------------------------filter((d3.d_moy <= 10) and (d3.d_moy >= 4) and (d3.d_year = 2000)) -------------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalDistribute[DistributionSpecReplicated] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() ----------------------PhysicalProject -------------------------PhysicalOlapScan[item] +------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[cs_sold_date_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[sr_returned_date_sk] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[ss_sold_date_sk] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) and (store_returns.sr_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF3 cs_bill_customer_sk->[ss_customer_sk,sr_customer_sk];RF4 cs_item_sk->[ss_item_sk,sr_item_sk] +--------------------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF3 RF4 RF5 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF3 RF4 RF6 +--------------------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF7 +----------------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------------PhysicalProject +--------------------------------------filter((d1.d_moy = 4) and (d1.d_year = 2000)) +----------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------------PhysicalProject +----------------------------------filter((d2.d_moy <= 10) and (d2.d_moy >= 4) and (d2.d_year = 2000)) +------------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------PhysicalProject +------------------------------filter((d3.d_moy <= 10) and (d3.d_moy >= 4) and (d3.d_year = 2000)) +--------------------------------PhysicalOlapScan[date_dim] +----------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[item] ------------------PhysicalDistribute[DistributionSpecReplicated] --------------------PhysicalProject ----------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query29.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query29.out index 812e5126e88480..132428dfe8594d 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query29.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query29.out @@ -9,39 +9,41 @@ PhysicalResultSink ------------hashAgg[LOCAL] --------------PhysicalProject ----------------hashJoin[INNER_JOIN] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() -------------------hashJoin[INNER_JOIN] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[sr_returned_date_sk] -------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[cs_sold_date_sk] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[sr_returned_date_sk] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[ss_sold_date_sk] +----------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[cs_sold_date_sk] ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) and (store_returns.sr_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF3 cs_bill_customer_sk->[ss_customer_sk,sr_customer_sk];RF4 cs_item_sk->[ss_item_sk,sr_item_sk] -----------------------------------PhysicalDistribute[DistributionSpecHash] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() +--------------------------------hashJoin[INNER_JOIN] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[ss_sold_date_sk] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) and (store_returns.sr_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF3 cs_bill_customer_sk->[ss_customer_sk,sr_customer_sk];RF4 cs_item_sk->[ss_item_sk,sr_item_sk] +--------------------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF3 RF4 RF5 +------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF3 RF4 RF5 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF3 RF4 RF7 +--------------------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF3 RF4 RF7 -----------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF6 +----------------------------------PhysicalDistribute[DistributionSpecReplicated] ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF6 +--------------------------------------filter((d1.d_moy = 4) and (d1.d_year = 1999)) +----------------------------------------PhysicalOlapScan[date_dim] ------------------------------PhysicalDistribute[DistributionSpecReplicated] --------------------------------PhysicalProject -----------------------------------filter((d1.d_moy = 4) and (d1.d_year = 1999)) +----------------------------------filter(d_year IN (1999, 2000, 2001)) ------------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalDistribute[DistributionSpecReplicated] ----------------------------PhysicalProject -------------------------------filter(d_year IN (1999, 2000, 2001)) +------------------------------filter((d2.d_moy <= 7) and (d2.d_moy >= 4) and (d2.d_year = 1999)) --------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------PhysicalProject -----------------------------filter((d2.d_moy <= 7) and (d2.d_moy >= 4) and (d2.d_year = 1999)) -------------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalDistribute[DistributionSpecReplicated] -----------------------PhysicalProject -------------------------PhysicalOlapScan[item] +----------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[item] ------------------PhysicalDistribute[DistributionSpecReplicated] --------------------PhysicalProject ----------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query30.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query30.out index 6dd7c6aecc8a06..985c714b0ab82e 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query30.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query30.out @@ -25,21 +25,22 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------PhysicalTopN[LOCAL_SORT] ----------PhysicalProject ------------hashJoin[INNER_JOIN] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] ---------------hashJoin[INNER_JOIN] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) -----------------PhysicalDistribute[DistributionSpecHash] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ctr_customer_sk] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF2 -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[customer] apply RFs: RF3 -----------------hashAgg[GLOBAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) ------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashAgg[LOCAL] -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ctr_customer_sk] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF2 +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[customer] apply RFs: RF3 +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------------PhysicalProject +----------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) --------------PhysicalDistribute[DistributionSpecReplicated] ----------------PhysicalProject ------------------filter((customer_address.ca_state = 'IN')) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query40.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query40.out index 01b875d6f12ad0..7b345807e0f93c 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query40.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query40.out @@ -11,18 +11,19 @@ PhysicalResultSink ----------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() ------------------PhysicalProject --------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() -----------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] -------------------------hashJoin[INNER_JOIN] 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 +------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +--------------------------hashJoin[INNER_JOIN] 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 +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) +----------------------------------PhysicalOlapScan[item] --------------------------PhysicalDistribute[DistributionSpecReplicated] ----------------------------PhysicalProject -------------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) ---------------------------------PhysicalOlapScan[item] -------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------PhysicalProject -----------------------------filter((date_dim.d_date <= '2001-05-02') and (date_dim.d_date >= '2001-03-03')) -------------------------------PhysicalOlapScan[date_dim] +------------------------------filter((date_dim.d_date <= '2001-05-02') and (date_dim.d_date >= '2001-03-03')) +--------------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalProject ------------------------PhysicalOlapScan[catalog_returns] ------------------PhysicalDistribute[DistributionSpecReplicated] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query42.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query42.out index fe908b68c8f3a9..f167fc2b10a4c8 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query42.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query42.out @@ -9,13 +9,14 @@ PhysicalResultSink ------------hashAgg[LOCAL] --------------PhysicalProject ----------------hashJoin[INNER_JOIN] hashCondition=((dt.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] -------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] ---------------------PhysicalProject -----------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 ---------------------PhysicalDistribute[DistributionSpecReplicated] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] ----------------------PhysicalProject -------------------------filter((item.i_manager_id = 1)) ---------------------------PhysicalOlapScan[item] +------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 +----------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------PhysicalProject +--------------------------filter((item.i_manager_id = 1)) +----------------------------PhysicalOlapScan[item] ------------------PhysicalDistribute[DistributionSpecReplicated] --------------------PhysicalProject ----------------------filter((dt.d_moy = 11) and (dt.d_year = 2002)) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query62.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query62.out index 6a5afb5e8ac582..00cb6c408c16dc 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query62.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query62.out @@ -11,17 +11,18 @@ PhysicalResultSink ----------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() ------------------PhysicalProject --------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() -----------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_ship_date_sk] -------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_ship_date_sk] +--------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[web_site] --------------------------PhysicalDistribute[DistributionSpecReplicated] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_site] -------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------PhysicalProject -----------------------------filter((date_dim.d_month_seq <= 1205) and (date_dim.d_month_seq >= 1194)) -------------------------------PhysicalOlapScan[date_dim] +------------------------------filter((date_dim.d_month_seq <= 1205) and (date_dim.d_month_seq >= 1194)) +--------------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalDistribute[DistributionSpecReplicated] ------------------------PhysicalProject --------------------------PhysicalOlapScan[ship_mode] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query75.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query75.out index 6203d20841799b..a5bd3c2a3895a3 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query75.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query75.out @@ -12,17 +12,18 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------PhysicalDistribute[DistributionSpecExecutionAny] --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] -------------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN] 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 -------------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------PhysicalProject +--------------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] --------------------------------PhysicalProject -----------------------------------filter((item.i_category = 'Home')) -------------------------------------PhysicalOlapScan[item] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_returns] +----------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 +--------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------PhysicalProject +------------------------------------filter((item.i_category = 'Home')) +--------------------------------------PhysicalOlapScan[item] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_returns] ------------------------PhysicalDistribute[DistributionSpecReplicated] --------------------------PhysicalProject ----------------------------filter(d_year IN (1998, 1999)) @@ -30,17 +31,18 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------PhysicalDistribute[DistributionSpecExecutionAny] --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ss_sold_date_sk] -------------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() build RFs:RF2 i_item_sk->[ss_item_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 -------------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------PhysicalProject +--------------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() build RFs:RF2 i_item_sk->[ss_item_sk] --------------------------------PhysicalProject -----------------------------------filter((item.i_category = 'Home')) -------------------------------------PhysicalOlapScan[item] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[store_returns] +----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +--------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------PhysicalProject +------------------------------------filter((item.i_category = 'Home')) +--------------------------------------PhysicalOlapScan[item] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[store_returns] ------------------------PhysicalDistribute[DistributionSpecReplicated] --------------------------PhysicalProject ----------------------------filter(d_year IN (1998, 1999)) @@ -48,17 +50,18 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------PhysicalDistribute[DistributionSpecExecutionAny] --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN] hashCondition=((date_dim.d_date_sk = web_sales.ws_sold_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[ws_sold_date_sk] -------------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((web_sales.ws_item_sk = web_returns.wr_item_sk) and (web_sales.ws_order_number = web_returns.wr_order_number)) otherCondition=() ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN] hashCondition=((item.i_item_sk = web_sales.ws_item_sk)) otherCondition=() build RFs:RF4 i_item_sk->[ws_item_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[web_sales] apply RFs: RF4 RF5 -------------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------PhysicalProject +--------------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((web_sales.ws_item_sk = web_returns.wr_item_sk) and (web_sales.ws_order_number = web_returns.wr_order_number)) otherCondition=() +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN] hashCondition=((item.i_item_sk = web_sales.ws_item_sk)) otherCondition=() build RFs:RF4 i_item_sk->[ws_item_sk] --------------------------------PhysicalProject -----------------------------------filter((item.i_category = 'Home')) -------------------------------------PhysicalOlapScan[item] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[web_returns] +----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF4 RF5 +--------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------PhysicalProject +------------------------------------filter((item.i_category = 'Home')) +--------------------------------------PhysicalOlapScan[item] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[web_returns] ------------------------PhysicalDistribute[DistributionSpecReplicated] --------------------------PhysicalProject ----------------------------filter(d_year IN (1998, 1999)) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query81.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query81.out index a2307e48d0f4be..ac9cf29ee123d4 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query81.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query81.out @@ -26,21 +26,22 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------PhysicalProject ------------hashJoin[INNER_JOIN] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] --------------PhysicalDistribute[DistributionSpecHash] -----------------hashJoin[INNER_JOIN] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ctr_customer_sk] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF2 -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[customer] apply RFs: RF3 -------------------hashAgg[GLOBAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) --------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashAgg[LOCAL] -------------------------PhysicalDistribute[DistributionSpecExecutionAny] ---------------------------PhysicalProject -----------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ctr_customer_sk] +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF2 +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[customer] apply RFs: RF3 +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------hashAgg[LOCAL] +--------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------------PhysicalProject +------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) --------------PhysicalDistribute[DistributionSpecHash] ----------------PhysicalProject ------------------filter((customer_address.ca_state = 'CA')) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query94.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query94.out index ceb9643e66a4ab..96c56c1f6e9a08 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query94.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query94.out @@ -10,28 +10,29 @@ PhysicalResultSink --------------hashAgg[LOCAL] ----------------PhysicalProject ------------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((ws1.ws_order_number = wr1.wr_order_number)) otherCondition=() ---------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF3 web_site_sk->[ws_web_site_sk] -----------------------hashJoin[RIGHT_SEMI_JOIN] hashCondition=((ws1.ws_order_number = ws2.ws_order_number)) otherCondition=(( not (ws_warehouse_sk = ws_warehouse_sk))) build RFs:RF2 ws_order_number->[ws_order_number] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[web_sales] apply RFs: RF2 -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF1 ca_address_sk->[ws_ship_addr_sk] -----------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_ship_date_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF3 +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF3 web_site_sk->[ws_web_site_sk] +------------------------hashJoin[RIGHT_SEMI_JOIN] hashCondition=((ws1.ws_order_number = ws2.ws_order_number)) otherCondition=(( not (ws_warehouse_sk = ws_warehouse_sk))) build RFs:RF2 ws_order_number->[ws_order_number] +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[web_sales] apply RFs: RF2 +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF1 ca_address_sk->[ws_ship_addr_sk] +------------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_ship_date_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF3 +--------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------PhysicalProject +------------------------------------filter((date_dim.d_date <= '2000-04-01') and (date_dim.d_date >= '2000-02-01')) +--------------------------------------PhysicalOlapScan[date_dim] ------------------------------PhysicalDistribute[DistributionSpecReplicated] --------------------------------PhysicalProject -----------------------------------filter((date_dim.d_date <= '2000-04-01') and (date_dim.d_date >= '2000-02-01')) -------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalDistribute[DistributionSpecReplicated] -------------------------------PhysicalProject ---------------------------------filter((customer_address.ca_state = 'OK')) -----------------------------------PhysicalOlapScan[customer_address] -----------------------PhysicalDistribute[DistributionSpecReplicated] -------------------------PhysicalProject ---------------------------filter((web_site.web_company_name = 'pri')) -----------------------------PhysicalOlapScan[web_site] +----------------------------------filter((customer_address.ca_state = 'OK')) +------------------------------------PhysicalOlapScan[customer_address] +------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------PhysicalProject +----------------------------filter((web_site.web_company_name = 'pri')) +------------------------------PhysicalOlapScan[web_site] --------------------PhysicalDistribute[DistributionSpecHash] ----------------------PhysicalProject ------------------------PhysicalOlapScan[web_returns] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query95.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query95.out index 82a42a4795c3a4..b042e3531742ae 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query95.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query95.out @@ -23,31 +23,32 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------PhysicalProject --------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF6 -----------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF5 web_site_sk->[ws_web_site_sk] -------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF4 ca_address_sk->[ws_ship_addr_sk] ---------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ws_ship_date_sk] -----------------------------hashJoin[RIGHT_SEMI_JOIN] hashCondition=((ws1.ws_order_number = web_returns.wr_order_number)) otherCondition=() build RFs:RF2 ws_order_number->[wr_order_number];RF7 ws_order_number->[ws_order_number,ws_order_number] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN] hashCondition=((web_returns.wr_order_number = ws_wh.ws_order_number)) otherCondition=() -----------------------------------PhysicalDistribute[DistributionSpecHash] -------------------------------------PhysicalProject ---------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -----------------------------------PhysicalDistribute[DistributionSpecHash] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[web_returns] apply RFs: RF2 -------------------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF5 web_site_sk->[ws_web_site_sk] +--------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF4 ca_address_sk->[ws_ship_addr_sk] +----------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ws_ship_date_sk] +------------------------------hashJoin[RIGHT_SEMI_JOIN] hashCondition=((ws1.ws_order_number = web_returns.wr_order_number)) otherCondition=() build RFs:RF2 ws_order_number->[wr_order_number];RF7 ws_order_number->[ws_order_number,ws_order_number] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN] hashCondition=((web_returns.wr_order_number = ws_wh.ws_order_number)) otherCondition=() +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------PhysicalProject +----------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[web_returns] apply RFs: RF2 +--------------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF3 RF4 RF5 +------------------------------PhysicalDistribute[DistributionSpecReplicated] --------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF3 RF4 RF5 +----------------------------------filter((date_dim.d_date <= '1999-04-02') and (date_dim.d_date >= '1999-02-01')) +------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalDistribute[DistributionSpecReplicated] ------------------------------PhysicalProject ---------------------------------filter((date_dim.d_date <= '1999-04-02') and (date_dim.d_date >= '1999-02-01')) -----------------------------------PhysicalOlapScan[date_dim] +--------------------------------filter((customer_address.ca_state = 'NC')) +----------------------------------PhysicalOlapScan[customer_address] --------------------------PhysicalDistribute[DistributionSpecReplicated] ----------------------------PhysicalProject -------------------------------filter((customer_address.ca_state = 'NC')) ---------------------------------PhysicalOlapScan[customer_address] -------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------PhysicalProject -----------------------------filter((web_site.web_company_name = 'pri')) -------------------------------PhysicalOlapScan[web_site] +------------------------------filter((web_site.web_company_name = 'pri')) +--------------------------------PhysicalOlapScan[web_site] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query99.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query99.out index 9f6ce5d0e1c0fa..628908e83a3bb4 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query99.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query99.out @@ -11,17 +11,18 @@ PhysicalResultSink ----------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() ------------------PhysicalProject --------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() -----------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_ship_date_sk] -------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_call_center_sk = call_center.cc_call_center_sk)) otherCondition=() ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_ship_date_sk] +--------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_call_center_sk = call_center.cc_call_center_sk)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[call_center] --------------------------PhysicalDistribute[DistributionSpecReplicated] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[call_center] -------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------PhysicalProject -----------------------------filter((date_dim.d_month_seq <= 1235) and (date_dim.d_month_seq >= 1224)) -------------------------------PhysicalOlapScan[date_dim] +------------------------------filter((date_dim.d_month_seq <= 1235) and (date_dim.d_month_seq >= 1224)) +--------------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalDistribute[DistributionSpecReplicated] ------------------------PhysicalProject --------------------------PhysicalOlapScan[ship_mode] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query16.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query16.out index 4e0cd864a37abe..8e5966e76f7751 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query16.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query16.out @@ -10,28 +10,29 @@ PhysicalResultSink --------------hashAgg[LOCAL] ----------------PhysicalProject ------------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((cs1.cs_order_number = cr1.cr_order_number)) otherCondition=() ---------------------hashJoin[INNER_JOIN] hashCondition=((cs1.cs_call_center_sk = call_center.cc_call_center_sk)) otherCondition=() build RFs:RF3 cc_call_center_sk->[cs_call_center_sk] -----------------------hashJoin[RIGHT_SEMI_JOIN] hashCondition=((cs1.cs_order_number = cs2.cs_order_number)) otherCondition=(( not (cs_warehouse_sk = cs_warehouse_sk))) build RFs:RF2 cs_order_number->[cs_order_number] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------hashJoin[INNER_JOIN] hashCondition=((cs1.cs_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF1 ca_address_sk->[cs_ship_addr_sk] -----------------------------hashJoin[INNER_JOIN] hashCondition=((cs1.cs_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_ship_date_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF3 +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN] hashCondition=((cs1.cs_call_center_sk = call_center.cc_call_center_sk)) otherCondition=() build RFs:RF3 cc_call_center_sk->[cs_call_center_sk] +------------------------hashJoin[RIGHT_SEMI_JOIN] hashCondition=((cs1.cs_order_number = cs2.cs_order_number)) otherCondition=(( not (cs_warehouse_sk = cs_warehouse_sk))) build RFs:RF2 cs_order_number->[cs_order_number] +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------hashJoin[INNER_JOIN] hashCondition=((cs1.cs_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF1 ca_address_sk->[cs_ship_addr_sk] +------------------------------hashJoin[INNER_JOIN] hashCondition=((cs1.cs_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_ship_date_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF3 +--------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------PhysicalProject +------------------------------------filter((date_dim.d_date <= '2002-05-31') and (date_dim.d_date >= '2002-04-01')) +--------------------------------------PhysicalOlapScan[date_dim] ------------------------------PhysicalDistribute[DistributionSpecReplicated] --------------------------------PhysicalProject -----------------------------------filter((date_dim.d_date <= '2002-05-31') and (date_dim.d_date >= '2002-04-01')) -------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalDistribute[DistributionSpecReplicated] -------------------------------PhysicalProject ---------------------------------filter((customer_address.ca_state = 'WV')) -----------------------------------PhysicalOlapScan[customer_address] -----------------------PhysicalDistribute[DistributionSpecReplicated] -------------------------PhysicalProject ---------------------------filter(cc_county IN ('Barrow County', 'Daviess County', 'Luce County', 'Richland County', 'Ziebach County')) -----------------------------PhysicalOlapScan[call_center] +----------------------------------filter((customer_address.ca_state = 'WV')) +------------------------------------PhysicalOlapScan[customer_address] +------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------PhysicalProject +----------------------------filter(cc_county IN ('Barrow County', 'Daviess County', 'Luce County', 'Richland County', 'Ziebach County')) +------------------------------PhysicalOlapScan[call_center] --------------------PhysicalDistribute[DistributionSpecHash] ----------------------PhysicalProject ------------------------PhysicalOlapScan[catalog_returns] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query2.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query2.out index 5d8e996d89882e..ecdbe2e1939ffb 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query2.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query2.out @@ -23,20 +23,21 @@ PhysicalCteAnchor ( cteId=CTEId#1 ) --------PhysicalQuickSort[LOCAL_SORT] ----------PhysicalProject ------------hashJoin[INNER_JOIN] hashCondition=((date_dim.d_week_seq = d_week_seq2)) otherCondition=() build RFs:RF2 d_week_seq->[d_week_seq] ---------------hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_week_seq1 as BIGINT) = expr_(d_week_seq2 - 53))) otherCondition=() -----------------PhysicalDistribute[DistributionSpecHash] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN] hashCondition=((date_dim.d_week_seq = d_week_seq1)) otherCondition=() build RFs:RF1 d_week_seq->[d_week_seq] -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) apply RFs: RF1 -----------------------PhysicalDistribute[DistributionSpecReplicated] -------------------------PhysicalProject ---------------------------filter((date_dim.d_year = 1998)) -----------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecHash] -------------------PhysicalProject ---------------------PhysicalCteConsumer ( cteId=CTEId#1 ) apply RFs: RF2 +--------------PhysicalProject +----------------hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_week_seq1 as BIGINT) = expr_(d_week_seq2 - 53))) otherCondition=() +------------------PhysicalDistribute[DistributionSpecHash] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN] hashCondition=((date_dim.d_week_seq = d_week_seq1)) otherCondition=() build RFs:RF1 d_week_seq->[d_week_seq] +------------------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------------PhysicalProject +----------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) apply RFs: RF1 +------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------PhysicalProject +----------------------------filter((date_dim.d_year = 1998)) +------------------------------PhysicalOlapScan[date_dim] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------PhysicalProject +----------------------PhysicalCteConsumer ( cteId=CTEId#1 ) apply RFs: RF2 --------------PhysicalDistribute[DistributionSpecReplicated] ----------------PhysicalProject ------------------filter((date_dim.d_year = 1999)) 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 6e97a9eafc3e5b..4a4d01a429a13d 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 @@ -57,36 +57,38 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------PhysicalProject --------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((catalog_sales.cs_item_sk = frequent_ss_items.item_sk)) otherCondition=() ----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] ---------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((catalog_sales.cs_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() -----------------------------PhysicalDistribute[DistributionSpecHash] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 -----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] +----------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((catalog_sales.cs_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() +------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 +------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------PhysicalProject +----------------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) +----------------------------PhysicalDistribute[DistributionSpecReplicated] ------------------------------PhysicalProject ---------------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) ---------------------------PhysicalDistribute[DistributionSpecReplicated] -----------------------------PhysicalProject -------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) ---------------------------------PhysicalOlapScan[date_dim] +--------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) +----------------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------PhysicalProject --------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ------------------PhysicalProject --------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((web_sales.ws_item_sk = frequent_ss_items.item_sk)) otherCondition=() ----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ws_sold_date_sk] ---------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((web_sales.ws_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() -----------------------------PhysicalDistribute[DistributionSpecHash] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[web_sales] apply RFs: RF4 -----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ws_sold_date_sk] +----------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((web_sales.ws_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() +------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF4 +------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------PhysicalProject +----------------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) +----------------------------PhysicalDistribute[DistributionSpecReplicated] ------------------------------PhysicalProject ---------------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) ---------------------------PhysicalDistribute[DistributionSpecReplicated] -----------------------------PhysicalProject -------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) ---------------------------------PhysicalOlapScan[date_dim] +--------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) +----------------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------PhysicalProject --------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query25.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query25.out index ecfc4cf3f19ade..efdd5a0c6641e1 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query25.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query25.out @@ -9,40 +9,41 @@ PhysicalResultSink ------------hashAgg[LOCAL] --------------PhysicalProject ----------------hashJoin[INNER_JOIN] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF9 s_store_sk->[ss_store_sk] -------------------hashJoin[INNER_JOIN] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() build RFs:RF8 i_item_sk->[ss_item_sk,sr_item_sk] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[cs_sold_date_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[sr_returned_date_sk] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[ss_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) and (store_returns.sr_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF3 cs_bill_customer_sk->[ss_customer_sk,sr_customer_sk];RF4 cs_item_sk->[ss_item_sk,sr_item_sk] -------------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF0 sr_customer_sk->[ss_customer_sk];RF1 sr_item_sk->[ss_item_sk];RF2 sr_ticket_number->[ss_ticket_number] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 RF4 RF5 RF8 RF9 -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF3 RF4 RF6 RF8 -------------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF7 ---------------------------------PhysicalDistribute[DistributionSpecReplicated] -----------------------------------PhysicalProject -------------------------------------filter((d1.d_moy = 4) and (d1.d_year = 2000)) ---------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalDistribute[DistributionSpecReplicated] -------------------------------PhysicalProject ---------------------------------filter((d2.d_moy <= 10) and (d2.d_moy >= 4) and (d2.d_year = 2000)) -----------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------PhysicalProject -----------------------------filter((d3.d_moy <= 10) and (d3.d_moy >= 4) and (d3.d_year = 2000)) -------------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalDistribute[DistributionSpecReplicated] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() build RFs:RF8 i_item_sk->[ss_item_sk,sr_item_sk] ----------------------PhysicalProject -------------------------PhysicalOlapScan[item] +------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[cs_sold_date_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[sr_returned_date_sk] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[ss_sold_date_sk] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) and (store_returns.sr_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF3 cs_bill_customer_sk->[ss_customer_sk,sr_customer_sk];RF4 cs_item_sk->[ss_item_sk,sr_item_sk] +--------------------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF0 sr_customer_sk->[ss_customer_sk];RF1 sr_item_sk->[ss_item_sk];RF2 sr_ticket_number->[ss_ticket_number] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 RF4 RF5 RF8 RF9 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF3 RF4 RF6 RF8 +--------------------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF7 +----------------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------------PhysicalProject +--------------------------------------filter((d1.d_moy = 4) and (d1.d_year = 2000)) +----------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------------PhysicalProject +----------------------------------filter((d2.d_moy <= 10) and (d2.d_moy >= 4) and (d2.d_year = 2000)) +------------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------PhysicalProject +------------------------------filter((d3.d_moy <= 10) and (d3.d_moy >= 4) and (d3.d_year = 2000)) +--------------------------------PhysicalOlapScan[date_dim] +----------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[item] ------------------PhysicalDistribute[DistributionSpecReplicated] --------------------PhysicalProject ----------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query29.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query29.out index f792e648e87598..f90b4bf6f81714 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query29.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query29.out @@ -9,39 +9,41 @@ PhysicalResultSink ------------hashAgg[LOCAL] --------------PhysicalProject ----------------hashJoin[INNER_JOIN] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF9 s_store_sk->[ss_store_sk] -------------------hashJoin[INNER_JOIN] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() build RFs:RF8 i_item_sk->[ss_item_sk,sr_item_sk] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[sr_returned_date_sk] -------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[cs_sold_date_sk] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() build RFs:RF8 i_item_sk->[ss_item_sk,sr_item_sk] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[sr_returned_date_sk] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[ss_sold_date_sk] +----------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[cs_sold_date_sk] ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) and (store_returns.sr_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF3 cs_bill_customer_sk->[ss_customer_sk,sr_customer_sk];RF4 cs_item_sk->[ss_item_sk,sr_item_sk] -----------------------------------PhysicalDistribute[DistributionSpecHash] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF0 sr_customer_sk->[ss_customer_sk];RF1 sr_item_sk->[ss_item_sk];RF2 sr_ticket_number->[ss_ticket_number] +--------------------------------hashJoin[INNER_JOIN] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[ss_sold_date_sk] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) and (store_returns.sr_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF3 cs_bill_customer_sk->[ss_customer_sk,sr_customer_sk];RF4 cs_item_sk->[ss_item_sk,sr_item_sk] +--------------------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 RF4 RF5 RF8 RF9 +------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF0 sr_customer_sk->[ss_customer_sk];RF1 sr_item_sk->[ss_item_sk];RF2 sr_ticket_number->[ss_ticket_number] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 RF4 RF5 RF8 RF9 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF3 RF4 RF7 RF8 +--------------------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF3 RF4 RF7 RF8 -----------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF6 +----------------------------------PhysicalDistribute[DistributionSpecReplicated] ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF6 +--------------------------------------filter((d1.d_moy = 4) and (d1.d_year = 1999)) +----------------------------------------PhysicalOlapScan[date_dim] ------------------------------PhysicalDistribute[DistributionSpecReplicated] --------------------------------PhysicalProject -----------------------------------filter((d1.d_moy = 4) and (d1.d_year = 1999)) +----------------------------------filter(d_year IN (1999, 2000, 2001)) ------------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalDistribute[DistributionSpecReplicated] ----------------------------PhysicalProject -------------------------------filter(d_year IN (1999, 2000, 2001)) +------------------------------filter((d2.d_moy <= 7) and (d2.d_moy >= 4) and (d2.d_year = 1999)) --------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------PhysicalProject -----------------------------filter((d2.d_moy <= 7) and (d2.d_moy >= 4) and (d2.d_year = 1999)) -------------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalDistribute[DistributionSpecReplicated] -----------------------PhysicalProject -------------------------PhysicalOlapScan[item] +----------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[item] ------------------PhysicalDistribute[DistributionSpecReplicated] --------------------PhysicalProject ----------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query30.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query30.out index 4190bf7300dc26..bfa6bc4fe650b8 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query30.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query30.out @@ -25,21 +25,22 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------PhysicalTopN[LOCAL_SORT] ----------PhysicalProject ------------hashJoin[INNER_JOIN] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] ---------------hashJoin[INNER_JOIN] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) -----------------PhysicalDistribute[DistributionSpecHash] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ctr_customer_sk] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF2 -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[customer] apply RFs: RF3 -----------------hashAgg[GLOBAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) ------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashAgg[LOCAL] -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ctr_customer_sk] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF2 +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[customer] apply RFs: RF3 +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------------PhysicalProject +----------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) --------------PhysicalDistribute[DistributionSpecReplicated] ----------------PhysicalProject ------------------filter((customer_address.ca_state = 'IN')) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query40.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query40.out index fe24268b63cb86..35c582a3534661 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query40.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query40.out @@ -11,18 +11,19 @@ PhysicalResultSink ----------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF2 w_warehouse_sk->[cs_warehouse_sk] ------------------PhysicalProject --------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() -----------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] -------------------------hashJoin[INNER_JOIN] 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 RF2 +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +--------------------------hashJoin[INNER_JOIN] 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 RF2 +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) +----------------------------------PhysicalOlapScan[item] --------------------------PhysicalDistribute[DistributionSpecReplicated] ----------------------------PhysicalProject -------------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) ---------------------------------PhysicalOlapScan[item] -------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------PhysicalProject -----------------------------filter((date_dim.d_date <= '2001-05-02') and (date_dim.d_date >= '2001-03-03')) -------------------------------PhysicalOlapScan[date_dim] +------------------------------filter((date_dim.d_date <= '2001-05-02') and (date_dim.d_date >= '2001-03-03')) +--------------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalProject ------------------------PhysicalOlapScan[catalog_returns] ------------------PhysicalDistribute[DistributionSpecReplicated] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query42.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query42.out index fe908b68c8f3a9..f167fc2b10a4c8 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query42.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query42.out @@ -9,13 +9,14 @@ PhysicalResultSink ------------hashAgg[LOCAL] --------------PhysicalProject ----------------hashJoin[INNER_JOIN] hashCondition=((dt.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] -------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] ---------------------PhysicalProject -----------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 ---------------------PhysicalDistribute[DistributionSpecReplicated] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] ----------------------PhysicalProject -------------------------filter((item.i_manager_id = 1)) ---------------------------PhysicalOlapScan[item] +------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 +----------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------PhysicalProject +--------------------------filter((item.i_manager_id = 1)) +----------------------------PhysicalOlapScan[item] ------------------PhysicalDistribute[DistributionSpecReplicated] --------------------PhysicalProject ----------------------filter((dt.d_moy = 11) and (dt.d_year = 2002)) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query62.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query62.out index 27d9bede78b1c1..3bcfe99bdd20fe 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query62.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query62.out @@ -11,17 +11,18 @@ PhysicalResultSink ----------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF3 w_warehouse_sk->[ws_warehouse_sk] ------------------PhysicalProject --------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF2 sm_ship_mode_sk->[ws_ship_mode_sk] -----------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_ship_date_sk] -------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF0 web_site_sk->[ws_web_site_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 RF3 +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_ship_date_sk] +--------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF0 web_site_sk->[ws_web_site_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 RF3 +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[web_site] --------------------------PhysicalDistribute[DistributionSpecReplicated] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_site] -------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------PhysicalProject -----------------------------filter((date_dim.d_month_seq <= 1205) and (date_dim.d_month_seq >= 1194)) -------------------------------PhysicalOlapScan[date_dim] +------------------------------filter((date_dim.d_month_seq <= 1205) and (date_dim.d_month_seq >= 1194)) +--------------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalDistribute[DistributionSpecReplicated] ------------------------PhysicalProject --------------------------PhysicalOlapScan[ship_mode] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query75.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query75.out index 6203d20841799b..a5bd3c2a3895a3 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query75.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query75.out @@ -12,17 +12,18 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------PhysicalDistribute[DistributionSpecExecutionAny] --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] -------------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN] 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 -------------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------PhysicalProject +--------------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] --------------------------------PhysicalProject -----------------------------------filter((item.i_category = 'Home')) -------------------------------------PhysicalOlapScan[item] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_returns] +----------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 +--------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------PhysicalProject +------------------------------------filter((item.i_category = 'Home')) +--------------------------------------PhysicalOlapScan[item] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_returns] ------------------------PhysicalDistribute[DistributionSpecReplicated] --------------------------PhysicalProject ----------------------------filter(d_year IN (1998, 1999)) @@ -30,17 +31,18 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------PhysicalDistribute[DistributionSpecExecutionAny] --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ss_sold_date_sk] -------------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() build RFs:RF2 i_item_sk->[ss_item_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 -------------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------PhysicalProject +--------------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() build RFs:RF2 i_item_sk->[ss_item_sk] --------------------------------PhysicalProject -----------------------------------filter((item.i_category = 'Home')) -------------------------------------PhysicalOlapScan[item] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[store_returns] +----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +--------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------PhysicalProject +------------------------------------filter((item.i_category = 'Home')) +--------------------------------------PhysicalOlapScan[item] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[store_returns] ------------------------PhysicalDistribute[DistributionSpecReplicated] --------------------------PhysicalProject ----------------------------filter(d_year IN (1998, 1999)) @@ -48,17 +50,18 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------PhysicalDistribute[DistributionSpecExecutionAny] --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN] hashCondition=((date_dim.d_date_sk = web_sales.ws_sold_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[ws_sold_date_sk] -------------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((web_sales.ws_item_sk = web_returns.wr_item_sk) and (web_sales.ws_order_number = web_returns.wr_order_number)) otherCondition=() ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN] hashCondition=((item.i_item_sk = web_sales.ws_item_sk)) otherCondition=() build RFs:RF4 i_item_sk->[ws_item_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[web_sales] apply RFs: RF4 RF5 -------------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------PhysicalProject +--------------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((web_sales.ws_item_sk = web_returns.wr_item_sk) and (web_sales.ws_order_number = web_returns.wr_order_number)) otherCondition=() +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN] hashCondition=((item.i_item_sk = web_sales.ws_item_sk)) otherCondition=() build RFs:RF4 i_item_sk->[ws_item_sk] --------------------------------PhysicalProject -----------------------------------filter((item.i_category = 'Home')) -------------------------------------PhysicalOlapScan[item] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[web_returns] +----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF4 RF5 +--------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------PhysicalProject +------------------------------------filter((item.i_category = 'Home')) +--------------------------------------PhysicalOlapScan[item] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[web_returns] ------------------------PhysicalDistribute[DistributionSpecReplicated] --------------------------PhysicalProject ----------------------------filter(d_year IN (1998, 1999)) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query81.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query81.out index 601cdffde57ed9..6463028a8fec4c 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query81.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query81.out @@ -26,21 +26,22 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------PhysicalProject ------------hashJoin[INNER_JOIN] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] --------------PhysicalDistribute[DistributionSpecHash] -----------------hashJoin[INNER_JOIN] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ctr_customer_sk] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF2 -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[customer] apply RFs: RF3 -------------------hashAgg[GLOBAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) --------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashAgg[LOCAL] -------------------------PhysicalDistribute[DistributionSpecExecutionAny] ---------------------------PhysicalProject -----------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ctr_customer_sk] +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF2 +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[customer] apply RFs: RF3 +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------hashAgg[LOCAL] +--------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------------PhysicalProject +------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) --------------PhysicalDistribute[DistributionSpecHash] ----------------PhysicalProject ------------------filter((customer_address.ca_state = 'CA')) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query94.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query94.out index ceb9643e66a4ab..96c56c1f6e9a08 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query94.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query94.out @@ -10,28 +10,29 @@ PhysicalResultSink --------------hashAgg[LOCAL] ----------------PhysicalProject ------------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((ws1.ws_order_number = wr1.wr_order_number)) otherCondition=() ---------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF3 web_site_sk->[ws_web_site_sk] -----------------------hashJoin[RIGHT_SEMI_JOIN] hashCondition=((ws1.ws_order_number = ws2.ws_order_number)) otherCondition=(( not (ws_warehouse_sk = ws_warehouse_sk))) build RFs:RF2 ws_order_number->[ws_order_number] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[web_sales] apply RFs: RF2 -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF1 ca_address_sk->[ws_ship_addr_sk] -----------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_ship_date_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF3 +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF3 web_site_sk->[ws_web_site_sk] +------------------------hashJoin[RIGHT_SEMI_JOIN] hashCondition=((ws1.ws_order_number = ws2.ws_order_number)) otherCondition=(( not (ws_warehouse_sk = ws_warehouse_sk))) build RFs:RF2 ws_order_number->[ws_order_number] +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[web_sales] apply RFs: RF2 +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF1 ca_address_sk->[ws_ship_addr_sk] +------------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_ship_date_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF3 +--------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------PhysicalProject +------------------------------------filter((date_dim.d_date <= '2000-04-01') and (date_dim.d_date >= '2000-02-01')) +--------------------------------------PhysicalOlapScan[date_dim] ------------------------------PhysicalDistribute[DistributionSpecReplicated] --------------------------------PhysicalProject -----------------------------------filter((date_dim.d_date <= '2000-04-01') and (date_dim.d_date >= '2000-02-01')) -------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalDistribute[DistributionSpecReplicated] -------------------------------PhysicalProject ---------------------------------filter((customer_address.ca_state = 'OK')) -----------------------------------PhysicalOlapScan[customer_address] -----------------------PhysicalDistribute[DistributionSpecReplicated] -------------------------PhysicalProject ---------------------------filter((web_site.web_company_name = 'pri')) -----------------------------PhysicalOlapScan[web_site] +----------------------------------filter((customer_address.ca_state = 'OK')) +------------------------------------PhysicalOlapScan[customer_address] +------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------PhysicalProject +----------------------------filter((web_site.web_company_name = 'pri')) +------------------------------PhysicalOlapScan[web_site] --------------------PhysicalDistribute[DistributionSpecHash] ----------------------PhysicalProject ------------------------PhysicalOlapScan[web_returns] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query95.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query95.out index 765c019e5d4e90..9e96715c5e4990 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query95.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query95.out @@ -23,31 +23,32 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------PhysicalProject --------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF12 RF13 -----------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF10 web_site_sk->[ws_web_site_sk];RF11 web_site_sk->[ws_web_site_sk] -------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF8 ca_address_sk->[ws_ship_addr_sk];RF9 ca_address_sk->[ws_ship_addr_sk] ---------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ws_ship_date_sk];RF7 d_date_sk->[ws_ship_date_sk] -----------------------------hashJoin[RIGHT_SEMI_JOIN] hashCondition=((ws1.ws_order_number = web_returns.wr_order_number)) otherCondition=() build RFs:RF4 ws_order_number->[wr_order_number];RF5 ws_order_number->[wr_order_number];RF14 ws_order_number->[ws_order_number,ws_order_number];RF15 ws_order_number->[ws_order_number,ws_order_number] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN] hashCondition=((web_returns.wr_order_number = ws_wh.ws_order_number)) otherCondition=() build RFs:RF2 wr_order_number->[ws_order_number];RF3 wr_order_number->[ws_order_number] -----------------------------------PhysicalDistribute[DistributionSpecHash] -------------------------------------PhysicalProject ---------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF2 RF3 -----------------------------------PhysicalDistribute[DistributionSpecHash] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[web_returns] apply RFs: RF4 RF5 -------------------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF10 web_site_sk->[ws_web_site_sk];RF11 web_site_sk->[ws_web_site_sk] +--------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF8 ca_address_sk->[ws_ship_addr_sk];RF9 ca_address_sk->[ws_ship_addr_sk] +----------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ws_ship_date_sk];RF7 d_date_sk->[ws_ship_date_sk] +------------------------------hashJoin[RIGHT_SEMI_JOIN] hashCondition=((ws1.ws_order_number = web_returns.wr_order_number)) otherCondition=() build RFs:RF4 ws_order_number->[wr_order_number];RF5 ws_order_number->[wr_order_number];RF14 ws_order_number->[ws_order_number,ws_order_number];RF15 ws_order_number->[ws_order_number,ws_order_number] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN] hashCondition=((web_returns.wr_order_number = ws_wh.ws_order_number)) otherCondition=() build RFs:RF2 wr_order_number->[ws_order_number];RF3 wr_order_number->[ws_order_number] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------PhysicalProject +----------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF2 RF3 +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[web_returns] apply RFs: RF4 RF5 +--------------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF6 RF7 RF8 RF9 RF10 RF11 +------------------------------PhysicalDistribute[DistributionSpecReplicated] --------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF6 RF7 RF8 RF9 RF10 RF11 +----------------------------------filter((date_dim.d_date <= '1999-04-02') and (date_dim.d_date >= '1999-02-01')) +------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalDistribute[DistributionSpecReplicated] ------------------------------PhysicalProject ---------------------------------filter((date_dim.d_date <= '1999-04-02') and (date_dim.d_date >= '1999-02-01')) -----------------------------------PhysicalOlapScan[date_dim] +--------------------------------filter((customer_address.ca_state = 'NC')) +----------------------------------PhysicalOlapScan[customer_address] --------------------------PhysicalDistribute[DistributionSpecReplicated] ----------------------------PhysicalProject -------------------------------filter((customer_address.ca_state = 'NC')) ---------------------------------PhysicalOlapScan[customer_address] -------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------PhysicalProject -----------------------------filter((web_site.web_company_name = 'pri')) -------------------------------PhysicalOlapScan[web_site] +------------------------------filter((web_site.web_company_name = 'pri')) +--------------------------------PhysicalOlapScan[web_site] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query99.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query99.out index 65b23223566e6e..14f2338fe922db 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query99.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query99.out @@ -11,17 +11,18 @@ PhysicalResultSink ----------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF3 w_warehouse_sk->[cs_warehouse_sk] ------------------PhysicalProject --------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF2 sm_ship_mode_sk->[cs_ship_mode_sk] -----------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_ship_date_sk] -------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_call_center_sk = call_center.cc_call_center_sk)) otherCondition=() build RFs:RF0 cc_call_center_sk->[cs_call_center_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 RF3 +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_ship_date_sk] +--------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_call_center_sk = call_center.cc_call_center_sk)) otherCondition=() build RFs:RF0 cc_call_center_sk->[cs_call_center_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 RF3 +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[call_center] --------------------------PhysicalDistribute[DistributionSpecReplicated] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[call_center] -------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------PhysicalProject -----------------------------filter((date_dim.d_month_seq <= 1235) and (date_dim.d_month_seq >= 1224)) -------------------------------PhysicalOlapScan[date_dim] +------------------------------filter((date_dim.d_month_seq <= 1235) and (date_dim.d_month_seq >= 1224)) +--------------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalDistribute[DistributionSpecReplicated] ------------------------PhysicalProject --------------------------PhysicalOlapScan[ship_mode] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query1.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query1.out index 73dea99cdf9090..eca7d46a8f3ef1 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query1.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query1.out @@ -24,20 +24,21 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------PhysicalProject ------------------PhysicalOlapScan[customer] --------------PhysicalDistribute[DistributionSpecHash] -----------------hashJoin[INNER_JOIN] hashCondition=((ctr1.ctr_store_sk = ctr2.ctr_store_sk)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN] hashCondition=((store.s_store_sk = ctr1.ctr_store_sk)) otherCondition=() build RFs:RF1 s_store_sk->[ctr_store_sk] -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF1 -----------------------PhysicalDistribute[DistributionSpecReplicated] -------------------------PhysicalProject ---------------------------filter((store.s_state = 'SD')) -----------------------------PhysicalOlapScan[store] -------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalDistribute[DistributionSpecExecutionAny] -----------------------------PhysicalProject -------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------PhysicalProject +------------------hashJoin[INNER_JOIN] hashCondition=((ctr1.ctr_store_sk = ctr2.ctr_store_sk)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN] hashCondition=((store.s_store_sk = ctr1.ctr_store_sk)) otherCondition=() build RFs:RF1 s_store_sk->[ctr_store_sk] +------------------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF1 +------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------PhysicalProject +----------------------------filter((store.s_state = 'SD')) +------------------------------PhysicalOlapScan[store] +--------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------------PhysicalProject +--------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query11.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query11.out index 20e3fd86528acc..d5b8b6f8dba20a 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query11.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query11.out @@ -44,18 +44,19 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------PhysicalProject ------------------filter((t_w_firstyear.dyear = 2001) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.00)) --------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------hashJoin[INNER_JOIN] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() -----------------PhysicalDistribute[DistributionSpecHash] -------------------PhysicalProject ---------------------filter((t_s_secyear.dyear = 2002) and (t_s_secyear.sale_type = 's')) -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -----------------hashJoin[INNER_JOIN] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=() -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalProject -----------------------filter((t_w_secyear.dyear = 2002) and (t_w_secyear.sale_type = 'w')) -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------PhysicalProject +----------------hashJoin[INNER_JOIN] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() ------------------PhysicalDistribute[DistributionSpecHash] --------------------PhysicalProject -----------------------filter((t_s_firstyear.dyear = 2001) and (t_s_firstyear.sale_type = 's') and (t_s_firstyear.year_total > 0.00)) +----------------------filter((t_s_secyear.dyear = 2002) and (t_s_secyear.sale_type = 's')) ------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------hashJoin[INNER_JOIN] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=() +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------filter((t_w_secyear.dyear = 2002) and (t_w_secyear.sale_type = 'w')) +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------filter((t_s_firstyear.dyear = 2001) and (t_s_firstyear.sale_type = 's') and (t_s_firstyear.year_total > 0.00)) +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) 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 f2d4abf7f789eb..09f6bc5b473123 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 @@ -10,13 +10,14 @@ PhysicalResultSink --------------PhysicalProject ----------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=(((ca_state IN ('CA', 'GA', 'WA') OR substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274')) OR (catalog_sales.cs_sales_price > 500.00))) ------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 -----------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] ------------------------PhysicalProject ---------------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2001)) -----------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 +------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------PhysicalProject +----------------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2001)) +------------------------------PhysicalOlapScan[date_dim] ------------------PhysicalDistribute[DistributionSpecHash] --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query16.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query16.out index 869cd7397751db..c867983673a566 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query16.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query16.out @@ -13,26 +13,27 @@ PhysicalResultSink --------------------PhysicalDistribute[DistributionSpecHash] ----------------------PhysicalProject ------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 ---------------------hashJoin[INNER_JOIN] hashCondition=((cs1.cs_call_center_sk = call_center.cc_call_center_sk)) otherCondition=() build RFs:RF3 cc_call_center_sk->[cs_call_center_sk] -----------------------hashJoin[RIGHT_ANTI_JOIN] hashCondition=((cs1.cs_order_number = cr1.cr_order_number)) otherCondition=() build RFs:RF2 cs_order_number->[cr_order_number] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF2 -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------hashJoin[INNER_JOIN] hashCondition=((cs1.cs_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_ship_date_sk] -----------------------------hashJoin[INNER_JOIN] hashCondition=((cs1.cs_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[cs_ship_addr_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF3 +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN] hashCondition=((cs1.cs_call_center_sk = call_center.cc_call_center_sk)) otherCondition=() build RFs:RF3 cc_call_center_sk->[cs_call_center_sk] +------------------------hashJoin[RIGHT_ANTI_JOIN] hashCondition=((cs1.cs_order_number = cr1.cr_order_number)) otherCondition=() build RFs:RF2 cs_order_number->[cr_order_number] +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF2 +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------hashJoin[INNER_JOIN] hashCondition=((cs1.cs_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_ship_date_sk] +------------------------------hashJoin[INNER_JOIN] hashCondition=((cs1.cs_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[cs_ship_addr_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF3 +--------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------PhysicalProject +------------------------------------filter((customer_address.ca_state = 'WV')) +--------------------------------------PhysicalOlapScan[customer_address] ------------------------------PhysicalDistribute[DistributionSpecReplicated] --------------------------------PhysicalProject -----------------------------------filter((customer_address.ca_state = 'WV')) -------------------------------------PhysicalOlapScan[customer_address] -----------------------------PhysicalDistribute[DistributionSpecReplicated] -------------------------------PhysicalProject ---------------------------------filter((date_dim.d_date <= '2002-05-31') and (date_dim.d_date >= '2002-04-01')) -----------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalDistribute[DistributionSpecReplicated] -------------------------PhysicalProject ---------------------------filter(cc_county IN ('Barrow County', 'Daviess County', 'Luce County', 'Richland County', 'Ziebach County')) -----------------------------PhysicalOlapScan[call_center] +----------------------------------filter((date_dim.d_date <= '2002-05-31') and (date_dim.d_date >= '2002-04-01')) +------------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------PhysicalProject +----------------------------filter(cc_county IN ('Barrow County', 'Daviess County', 'Luce County', 'Richland County', 'Ziebach County')) +------------------------------PhysicalOlapScan[call_center] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query21.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query21.out index 448937184bb099..2f42f35ccc8d51 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query21.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query21.out @@ -12,12 +12,13 @@ PhysicalResultSink ------------------hashJoin[INNER_JOIN] hashCondition=((inventory.inv_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN] hashCondition=((inventory.inv_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[inv_date_sk] -------------------------hashJoin[INNER_JOIN] hashCondition=((item.i_item_sk = inventory.inv_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[inv_item_sk] ---------------------------PhysicalOlapScan[inventory] apply RFs: RF0 RF1 ---------------------------PhysicalDistribute[DistributionSpecReplicated] -----------------------------PhysicalProject -------------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) ---------------------------------PhysicalOlapScan[item] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN] hashCondition=((item.i_item_sk = inventory.inv_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[inv_item_sk] +----------------------------PhysicalOlapScan[inventory] apply RFs: RF0 RF1 +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) +----------------------------------PhysicalOlapScan[item] ------------------------PhysicalDistribute[DistributionSpecReplicated] --------------------------PhysicalProject ----------------------------filter((date_dim.d_date <= '2002-03-29') and (date_dim.d_date >= '2002-01-28')) 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 a83a3b91eb6470..a7f123bca8be7f 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 @@ -10,25 +10,26 @@ PhysicalResultSink --------------PhysicalProject ----------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() ------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_promo_sk = promotion.p_promo_sk)) otherCondition=() build RFs:RF2 p_promo_sk->[cs_promo_sk] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_bill_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[cs_bill_cdemo_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 -------------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------------PhysicalProject -----------------------------------filter((customer_demographics.cd_education_status = 'Unknown') and (customer_demographics.cd_gender = 'M') and (customer_demographics.cd_marital_status = 'S')) -------------------------------------PhysicalOlapScan[customer_demographics] ---------------------------PhysicalDistribute[DistributionSpecReplicated] -----------------------------PhysicalProject -------------------------------filter((date_dim.d_year = 2001)) ---------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_promo_sk = promotion.p_promo_sk)) otherCondition=() build RFs:RF2 p_promo_sk->[cs_promo_sk] ------------------------PhysicalProject ---------------------------filter(((promotion.p_channel_email = 'N') OR (promotion.p_channel_event = 'N'))) -----------------------------PhysicalOlapScan[promotion] +--------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_bill_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[cs_bill_cdemo_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 +--------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------PhysicalProject +------------------------------------filter((customer_demographics.cd_education_status = 'Unknown') and (customer_demographics.cd_gender = 'M') and (customer_demographics.cd_marital_status = 'S')) +--------------------------------------PhysicalOlapScan[customer_demographics] +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_year = 2001)) +----------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------PhysicalProject +----------------------------filter(((promotion.p_channel_email = 'N') OR (promotion.p_channel_event = 'N'))) +------------------------------PhysicalOlapScan[promotion] ------------------PhysicalDistribute[DistributionSpecHash] --------------------PhysicalProject ----------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query29.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query29.out index 02d504f24abf83..85dcd01d97761a 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query29.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query29.out @@ -14,31 +14,34 @@ PhysicalResultSink ----------------------PhysicalProject ------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF7 RF8 RF9 ----------------------PhysicalDistribute[DistributionSpecReplicated] -------------------------hashJoin[INNER_JOIN] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() ---------------------------hashJoin[INNER_JOIN] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() -----------------------------PhysicalDistribute[DistributionSpecHash] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF2 sr_customer_sk->[ss_customer_sk];RF3 sr_item_sk->[ss_item_sk];RF4 sr_ticket_number->[ss_ticket_number] -----------------------------------hashJoin[INNER_JOIN] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() +----------------------------hashJoin[INNER_JOIN] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() +------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF2 sr_customer_sk->[ss_customer_sk];RF3 sr_item_sk->[ss_item_sk];RF4 sr_ticket_number->[ss_ticket_number] ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 RF4 -------------------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------------------PhysicalProject -----------------------------------------filter((d1.d_moy = 4) and (d1.d_year = 1999)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[sr_returned_date_sk] +--------------------------------------hashJoin[INNER_JOIN] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 RF4 +----------------------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------------------PhysicalProject +--------------------------------------------filter((d1.d_moy = 4) and (d1.d_year = 1999)) +----------------------------------------------PhysicalOlapScan[date_dim] ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF0 -------------------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------------------PhysicalProject -----------------------------------------filter((d2.d_moy <= 7) and (d2.d_moy >= 4) and (d2.d_year = 1999)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[sr_returned_date_sk] +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF0 +----------------------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------------------PhysicalProject +--------------------------------------------filter((d2.d_moy <= 7) and (d2.d_moy >= 4) and (d2.d_year = 1999)) +----------------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] +----------------------------PhysicalDistribute[DistributionSpecReplicated] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[item] ---------------------------PhysicalDistribute[DistributionSpecReplicated] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[store] +--------------------------------PhysicalOlapScan[store] ------------------PhysicalDistribute[DistributionSpecReplicated] --------------------PhysicalProject ----------------------filter(d_year IN (1999, 2000, 2001)) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query4.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query4.out index a2d3494e5203e1..7dc2ff7cfb5275 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query4.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query4.out @@ -56,29 +56,31 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------PhysicalProject ------------------filter((t_w_firstyear.dyear = 1999) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.000000)) --------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------hashJoin[INNER_JOIN] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=() -----------------PhysicalDistribute[DistributionSpecHash] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=() +------------------PhysicalDistribute[DistributionSpecHash] +--------------------PhysicalProject +----------------------filter((t_w_secyear.dyear = 2000) and (t_w_secyear.sale_type = 'w')) +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ------------------PhysicalProject ---------------------filter((t_w_secyear.dyear = 2000) and (t_w_secyear.sale_type = 'w')) -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -----------------PhysicalProject -------------------hashJoin[INNER_JOIN] hashCondition=((t_s_firstyear.customer_id = t_c_firstyear.customer_id)) otherCondition=((if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL) > if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL))) ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------PhysicalProject -------------------------filter((t_c_firstyear.dyear = 1999) and (t_c_firstyear.sale_type = 'c') and (t_c_firstyear.year_total > 0.000000)) ---------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------------hashJoin[INNER_JOIN] hashCondition=((t_s_firstyear.customer_id = t_c_secyear.customer_id)) otherCondition=() -----------------------hashJoin[INNER_JOIN] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------filter((t_s_firstyear.dyear = 1999) and (t_s_firstyear.sale_type = 's') and (t_s_firstyear.year_total > 0.000000)) -------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------filter((t_s_secyear.dyear = 2000) and (t_s_secyear.sale_type = 's')) -------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------hashJoin[INNER_JOIN] hashCondition=((t_s_firstyear.customer_id = t_c_firstyear.customer_id)) otherCondition=((if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL) > if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL))) ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------PhysicalProject ---------------------------filter((t_c_secyear.dyear = 2000) and (t_c_secyear.sale_type = 'c')) +--------------------------filter((t_c_firstyear.dyear = 1999) and (t_c_firstyear.sale_type = 'c') and (t_c_firstyear.year_total > 0.000000)) ----------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN] hashCondition=((t_s_firstyear.customer_id = t_c_secyear.customer_id)) otherCondition=() +--------------------------hashJoin[INNER_JOIN] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------filter((t_s_firstyear.dyear = 1999) and (t_s_firstyear.sale_type = 's') and (t_s_firstyear.year_total > 0.000000)) +----------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------filter((t_s_secyear.dyear = 2000) and (t_s_secyear.sale_type = 's')) +----------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------PhysicalProject +------------------------------filter((t_c_secyear.dyear = 2000) and (t_c_secyear.sale_type = 'c')) +--------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query40.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query40.out index 4cf35773697542..2b506dcf8ff9f0 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query40.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query40.out @@ -13,18 +13,19 @@ PhysicalResultSink --------------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() build RFs:RF2 cs_order_number->[cr_order_number];RF3 cs_item_sk->[cr_item_sk] ----------------------PhysicalProject ------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF2 RF3 -----------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] -------------------------hashJoin[INNER_JOIN] 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 +------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +--------------------------hashJoin[INNER_JOIN] 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 +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) +----------------------------------PhysicalOlapScan[item] --------------------------PhysicalDistribute[DistributionSpecReplicated] ----------------------------PhysicalProject -------------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) ---------------------------------PhysicalOlapScan[item] -------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------PhysicalProject -----------------------------filter((date_dim.d_date <= '2001-05-02') and (date_dim.d_date >= '2001-03-03')) -------------------------------PhysicalOlapScan[date_dim] +------------------------------filter((date_dim.d_date <= '2001-05-02') and (date_dim.d_date >= '2001-03-03')) +--------------------------------PhysicalOlapScan[date_dim] ------------------PhysicalDistribute[DistributionSpecReplicated] --------------------PhysicalProject ----------------------PhysicalOlapScan[warehouse] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query42.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query42.out index fe908b68c8f3a9..f167fc2b10a4c8 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query42.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query42.out @@ -9,13 +9,14 @@ PhysicalResultSink ------------hashAgg[LOCAL] --------------PhysicalProject ----------------hashJoin[INNER_JOIN] hashCondition=((dt.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] -------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] ---------------------PhysicalProject -----------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 ---------------------PhysicalDistribute[DistributionSpecReplicated] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] ----------------------PhysicalProject -------------------------filter((item.i_manager_id = 1)) ---------------------------PhysicalOlapScan[item] +------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 +----------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------PhysicalProject +--------------------------filter((item.i_manager_id = 1)) +----------------------------PhysicalOlapScan[item] ------------------PhysicalDistribute[DistributionSpecReplicated] --------------------PhysicalProject ----------------------filter((dt.d_moy = 11) and (dt.d_year = 2002)) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query44.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query44.out index 78d9a82df01d4c..a29f21336e4c80 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query44.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query44.out @@ -39,34 +39,35 @@ PhysicalResultSink ------------------------------------------------------filter((store_sales.ss_store_sk = 146) and ss_addr_sk IS NULL) --------------------------------------------------------PhysicalOlapScan[store_sales] ------------PhysicalDistribute[DistributionSpecHash] ---------------hashJoin[INNER_JOIN] hashCondition=((i2.i_item_sk = descending.item_sk)) otherCondition=() build RFs:RF0 item_sk->[i_item_sk] -----------------PhysicalProject -------------------PhysicalOlapScan[item] apply RFs: RF0 -----------------PhysicalDistribute[DistributionSpecHash] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN] hashCondition=((i2.i_item_sk = descending.item_sk)) otherCondition=() build RFs:RF0 item_sk->[i_item_sk] ------------------PhysicalProject ---------------------filter((rnk < 11)) -----------------------PhysicalWindow -------------------------PhysicalQuickSort[MERGE_SORT] ---------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------PhysicalQuickSort[LOCAL_SORT] -------------------------------PhysicalPartitionTopN ---------------------------------PhysicalProject -----------------------------------NestedLoopJoin[INNER_JOIN](cast(rank_col as DOUBLE) > cast((0.9 * rank_col) as DOUBLE)) -------------------------------------PhysicalProject ---------------------------------------hashAgg[GLOBAL] -----------------------------------------PhysicalDistribute[DistributionSpecHash] -------------------------------------------hashAgg[LOCAL] +--------------------PhysicalOlapScan[item] apply RFs: RF0 +------------------PhysicalDistribute[DistributionSpecHash] +--------------------PhysicalProject +----------------------filter((rnk < 11)) +------------------------PhysicalWindow +--------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------PhysicalDistribute[DistributionSpecGather] +------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------PhysicalPartitionTopN +----------------------------------PhysicalProject +------------------------------------NestedLoopJoin[INNER_JOIN](cast(rank_col as DOUBLE) > cast((0.9 * rank_col) as DOUBLE)) +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] +----------------------------------------------PhysicalProject +------------------------------------------------filter((ss1.ss_store_sk = 146)) +--------------------------------------------------PhysicalOlapScan[store_sales] +--------------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------------PhysicalAssertNumRows +------------------------------------------PhysicalDistribute[DistributionSpecGather] --------------------------------------------PhysicalProject -----------------------------------------------filter((ss1.ss_store_sk = 146)) -------------------------------------------------PhysicalOlapScan[store_sales] -------------------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------------------PhysicalAssertNumRows -----------------------------------------PhysicalDistribute[DistributionSpecGather] -------------------------------------------PhysicalProject ---------------------------------------------hashAgg[GLOBAL] -----------------------------------------------PhysicalDistribute[DistributionSpecHash] -------------------------------------------------hashAgg[LOCAL] ---------------------------------------------------PhysicalProject -----------------------------------------------------filter((store_sales.ss_store_sk = 146) and ss_addr_sk IS NULL) -------------------------------------------------------PhysicalOlapScan[store_sales] +----------------------------------------------hashAgg[GLOBAL] +------------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------------hashAgg[LOCAL] +----------------------------------------------------PhysicalProject +------------------------------------------------------filter((store_sales.ss_store_sk = 146) and ss_addr_sk IS NULL) +--------------------------------------------------------PhysicalOlapScan[store_sales] 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 3aefef56ee9702..ecc76cb2850337 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 @@ -35,11 +35,12 @@ PhysicalResultSink ------------------------PhysicalProject --------------------------PhysicalOlapScan[customer_address] ------------PhysicalDistribute[DistributionSpecHash] ---------------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_addr_sk = current_addr.ca_address_sk)) otherCondition=() -----------------PhysicalDistribute[DistributionSpecHash] -------------------PhysicalProject ---------------------PhysicalOlapScan[customer] -----------------PhysicalDistribute[DistributionSpecHash] -------------------PhysicalProject ---------------------PhysicalOlapScan[customer_address] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_addr_sk = current_addr.ca_address_sk)) otherCondition=() +------------------PhysicalDistribute[DistributionSpecHash] +--------------------PhysicalProject +----------------------PhysicalOlapScan[customer] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------PhysicalProject +----------------------PhysicalOlapScan[customer_address] 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 ad7e5318fda922..a955f07d550130 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 @@ -11,14 +11,15 @@ PhysicalResultSink ----------------PhysicalProject ------------------hashJoin[INNER_JOIN] 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] --------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashJoin[INNER_JOIN] 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] -------------------------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 -------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN] 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] --------------------------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')))) -------------------------------PhysicalOlapScan[customer_demographics] +----------------------------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 +--------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------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')))) +--------------------------------PhysicalOlapScan[customer_demographics] --------------------PhysicalDistribute[DistributionSpecHash] ----------------------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/query58.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query58.out index bd785307372b59..16da25a33a92a8 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query58.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query58.out @@ -35,63 +35,64 @@ PhysicalResultSink ------------------------PhysicalDistribute[DistributionSpecHash] --------------------------PhysicalProject ----------------------------PhysicalOlapScan[item] apply RFs: RF13 -------------hashJoin[INNER_JOIN] hashCondition=((ss_items.item_id = ws_items.item_id)) otherCondition=((cast(ss_item_rev as DOUBLE) <= cast((1.1 * ws_item_rev) as DOUBLE)) and (cast(ss_item_rev as DOUBLE) >= cast((0.9 * ws_item_rev) as DOUBLE)) and (cast(ws_item_rev as DOUBLE) <= cast((1.1 * ss_item_rev) as DOUBLE)) and (cast(ws_item_rev as DOUBLE) >= cast((0.9 * ss_item_rev) as DOUBLE))) build RFs:RF8 item_id->[i_item_id] ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF7 i_item_sk->[ss_item_sk] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ss_sold_date_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 RF7 -------------------------------PhysicalDistribute[DistributionSpecReplicated] +------------PhysicalProject +--------------hashJoin[INNER_JOIN] hashCondition=((ss_items.item_id = ws_items.item_id)) otherCondition=((cast(ss_item_rev as DOUBLE) <= cast((1.1 * ws_item_rev) as DOUBLE)) and (cast(ss_item_rev as DOUBLE) >= cast((0.9 * ws_item_rev) as DOUBLE)) and (cast(ws_item_rev as DOUBLE) <= cast((1.1 * ss_item_rev) as DOUBLE)) and (cast(ws_item_rev as DOUBLE) >= cast((0.9 * ss_item_rev) as DOUBLE))) build RFs:RF8 item_id->[i_item_id] +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF7 i_item_sk->[ss_item_sk] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ss_sold_date_sk] --------------------------------PhysicalProject -----------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_date = date_dim.d_date)) otherCondition=() build RFs:RF5 d_date->[d_date] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF5 -------------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 RF7 +--------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------PhysicalProject +------------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_date = date_dim.d_date)) otherCondition=() build RFs:RF5 d_date->[d_date] --------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN] hashCondition=((date_dim.d_week_seq = date_dim.d_week_seq)) otherCondition=() build RFs:RF4 d_week_seq->[d_week_seq] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF4 -------------------------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------------------------PhysicalAssertNumRows -----------------------------------------------PhysicalDistribute[DistributionSpecGather] -------------------------------------------------PhysicalProject ---------------------------------------------------filter((date_dim.d_date = '2001-03-24')) -----------------------------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] apply RFs: RF8 ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] +----------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF5 +--------------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN] hashCondition=((date_dim.d_week_seq = date_dim.d_week_seq)) otherCondition=() build RFs:RF4 d_week_seq->[d_week_seq] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF4 +--------------------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------------------PhysicalAssertNumRows +------------------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_date = '2001-03-24')) +------------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------PhysicalDistribute[DistributionSpecHash] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[web_sales] apply RFs: RF2 -------------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------------PhysicalOlapScan[item] apply RFs: RF8 +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] --------------------------------PhysicalProject -----------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_date = date_dim.d_date)) otherCondition=() build RFs:RF1 d_date->[d_date] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF1 -------------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF2 +--------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------PhysicalProject +------------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_date = date_dim.d_date)) otherCondition=() build RFs:RF1 d_date->[d_date] --------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN] hashCondition=((date_dim.d_week_seq = date_dim.d_week_seq)) otherCondition=() build RFs:RF0 d_week_seq->[d_week_seq] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF0 -------------------------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------------------------PhysicalAssertNumRows -----------------------------------------------PhysicalDistribute[DistributionSpecGather] -------------------------------------------------PhysicalProject ---------------------------------------------------filter((date_dim.d_date = '2001-03-24')) -----------------------------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] +----------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF1 +--------------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN] hashCondition=((date_dim.d_week_seq = date_dim.d_week_seq)) otherCondition=() build RFs:RF0 d_week_seq->[d_week_seq] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF0 +--------------------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------------------PhysicalAssertNumRows +------------------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_date = '2001-03-24')) +------------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query62.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query62.out index ee40a79bf9598c..027778d4a235d3 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query62.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query62.out @@ -11,17 +11,18 @@ PhysicalResultSink ----------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() ------------------PhysicalProject --------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() -----------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() -------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_ship_date_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() +--------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_ship_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_month_seq <= 1205) and (date_dim.d_month_seq >= 1194)) +----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalDistribute[DistributionSpecReplicated] ----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq <= 1205) and (date_dim.d_month_seq >= 1194)) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[web_site] +------------------------------PhysicalOlapScan[web_site] ----------------------PhysicalDistribute[DistributionSpecReplicated] ------------------------PhysicalProject --------------------------PhysicalOlapScan[ship_mode] 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 e5bbc4df3dc54b..85a8486f84450d 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 @@ -10,25 +10,26 @@ PhysicalResultSink --------------PhysicalProject ----------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() ------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_promo_sk = promotion.p_promo_sk)) otherCondition=() build RFs:RF2 p_promo_sk->[ss_promo_sk] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN] 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] hashCondition=((store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 -------------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------------PhysicalProject -----------------------------------filter((customer_demographics.cd_education_status = 'College') and (customer_demographics.cd_gender = 'F') and (customer_demographics.cd_marital_status = 'W')) -------------------------------------PhysicalOlapScan[customer_demographics] ---------------------------PhysicalDistribute[DistributionSpecReplicated] -----------------------------PhysicalProject -------------------------------filter((date_dim.d_year = 2001)) ---------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_promo_sk = promotion.p_promo_sk)) otherCondition=() build RFs:RF2 p_promo_sk->[ss_promo_sk] ------------------------PhysicalProject ---------------------------filter(((promotion.p_channel_email = 'N') OR (promotion.p_channel_event = 'N'))) -----------------------------PhysicalOlapScan[promotion] +--------------------------hashJoin[INNER_JOIN] 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] hashCondition=((store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 +--------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------PhysicalProject +------------------------------------filter((customer_demographics.cd_education_status = 'College') and (customer_demographics.cd_gender = 'F') and (customer_demographics.cd_marital_status = 'W')) +--------------------------------------PhysicalOlapScan[customer_demographics] +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_year = 2001)) +----------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------PhysicalProject +----------------------------filter(((promotion.p_channel_email = 'N') OR (promotion.p_channel_event = 'N'))) +------------------------------PhysicalOlapScan[promotion] ------------------PhysicalDistribute[DistributionSpecHash] --------------------PhysicalProject ----------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query74.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query74.out index 27ae6ad0b2206c..88963927ca31e9 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query74.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query74.out @@ -44,18 +44,19 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------PhysicalProject ------------------filter((t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year = 1999) and (t_w_firstyear.year_total > 0.0)) --------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------hashJoin[INNER_JOIN] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=() -----------------hashJoin[INNER_JOIN] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalProject -----------------------filter((t_s_firstyear.sale_type = 's') and (t_s_firstyear.year = 1999) and (t_s_firstyear.year_total > 0.0)) -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------PhysicalProject +----------------hashJoin[INNER_JOIN] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=() +------------------hashJoin[INNER_JOIN] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------filter((t_s_firstyear.sale_type = 's') and (t_s_firstyear.year = 1999) and (t_s_firstyear.year_total > 0.0)) +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------filter((t_s_secyear.sale_type = 's') and (t_s_secyear.year = 2000)) +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ------------------PhysicalDistribute[DistributionSpecHash] --------------------PhysicalProject -----------------------filter((t_s_secyear.sale_type = 's') and (t_s_secyear.year = 2000)) +----------------------filter((t_w_secyear.sale_type = 'w') and (t_w_secyear.year = 2000)) ------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -----------------PhysicalDistribute[DistributionSpecHash] -------------------PhysicalProject ---------------------filter((t_w_secyear.sale_type = 'w') and (t_w_secyear.year = 2000)) -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query76.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query76.out index bbb934bf88ac68..bdd871b35d5fd6 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query76.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query76.out @@ -14,37 +14,40 @@ PhysicalResultSink ----------------------PhysicalProject ------------------------PhysicalOlapScan[item] apply RFs: RF1 ----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 ss_sold_date_sk->[d_date_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[date_dim] apply RFs: RF0 ---------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 ss_sold_date_sk->[d_date_sk] ----------------------------PhysicalProject -------------------------------filter(ss_hdemo_sk IS NULL) ---------------------------------PhysicalOlapScan[store_sales] +------------------------------PhysicalOlapScan[date_dim] apply RFs: RF0 +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------filter(ss_hdemo_sk IS NULL) +----------------------------------PhysicalOlapScan[store_sales] ----------------PhysicalDistribute[DistributionSpecExecutionAny] ------------------PhysicalProject --------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 ws_item_sk->[i_item_sk] ----------------------PhysicalProject ------------------------PhysicalOlapScan[item] apply RFs: RF3 ----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 ws_sold_date_sk->[d_date_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[date_dim] apply RFs: RF2 ---------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 ws_sold_date_sk->[d_date_sk] ----------------------------PhysicalProject -------------------------------filter(ws_bill_addr_sk IS NULL) ---------------------------------PhysicalOlapScan[web_sales] +------------------------------PhysicalOlapScan[date_dim] apply RFs: RF2 +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------filter(ws_bill_addr_sk IS NULL) +----------------------------------PhysicalOlapScan[web_sales] ----------------PhysicalDistribute[DistributionSpecExecutionAny] ------------------PhysicalProject --------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF5 cs_item_sk->[i_item_sk] ----------------------PhysicalProject ------------------------PhysicalOlapScan[item] apply RFs: RF5 ----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 cs_sold_date_sk->[d_date_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[date_dim] apply RFs: RF4 ---------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 cs_sold_date_sk->[d_date_sk] ----------------------------PhysicalProject -------------------------------filter(cs_warehouse_sk IS NULL) ---------------------------------PhysicalOlapScan[catalog_sales] +------------------------------PhysicalOlapScan[date_dim] apply RFs: RF4 +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------filter(cs_warehouse_sk IS NULL) +----------------------------------PhysicalOlapScan[catalog_sales] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query83.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query83.out index bd813f63a54151..8d75439401df13 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query83.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query83.out @@ -32,58 +32,59 @@ PhysicalResultSink ------------------------------------------PhysicalProject --------------------------------------------filter(d_date IN ('2001-06-06', '2001-09-02', '2001-11-11')) ----------------------------------------------PhysicalOlapScan[date_dim] -------------hashJoin[INNER_JOIN] hashCondition=((sr_items.item_id = wr_items.item_id)) otherCondition=() build RFs:RF8 item_id->[i_item_id] ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF7 i_item_sk->[sr_item_sk] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_returned_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[sr_returned_date_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_returns] apply RFs: RF6 RF7 -------------------------------PhysicalDistribute[DistributionSpecReplicated] +------------PhysicalProject +--------------hashJoin[INNER_JOIN] hashCondition=((sr_items.item_id = wr_items.item_id)) otherCondition=() build RFs:RF8 item_id->[i_item_id] +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF7 i_item_sk->[sr_item_sk] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_returned_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[sr_returned_date_sk] --------------------------------PhysicalProject -----------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_date = date_dim.d_date)) otherCondition=() build RFs:RF5 d_date->[d_date] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF5 -------------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------PhysicalOlapScan[store_returns] apply RFs: RF6 RF7 +--------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------PhysicalProject +------------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_date = date_dim.d_date)) otherCondition=() build RFs:RF5 d_date->[d_date] --------------------------------------PhysicalProject -----------------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_week_seq = date_dim.d_week_seq)) otherCondition=() build RFs:RF4 d_week_seq->[d_week_seq] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF4 -------------------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF5 +--------------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_week_seq = date_dim.d_week_seq)) otherCondition=() build RFs:RF4 d_week_seq->[d_week_seq] --------------------------------------------PhysicalProject -----------------------------------------------filter(d_date IN ('2001-06-06', '2001-09-02', '2001-11-11')) -------------------------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] apply RFs: RF8 ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN] hashCondition=((web_returns.wr_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 wr_item_sk->[i_item_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] apply RFs: RF3 ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashJoin[INNER_JOIN] hashCondition=((web_returns.wr_returned_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[wr_returned_date_sk] +----------------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF4 +--------------------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------------------PhysicalProject +------------------------------------------------filter(d_date IN ('2001-06-06', '2001-09-02', '2001-11-11')) +--------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------PhysicalDistribute[DistributionSpecHash] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[web_returns] apply RFs: RF2 -------------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------------PhysicalOlapScan[item] apply RFs: RF8 +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN] hashCondition=((web_returns.wr_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 wr_item_sk->[i_item_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[item] apply RFs: RF3 +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------hashJoin[INNER_JOIN] hashCondition=((web_returns.wr_returned_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[wr_returned_date_sk] --------------------------------PhysicalProject -----------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_date = date_dim.d_date)) otherCondition=() build RFs:RF1 d_date->[d_date] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF1 -------------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------PhysicalOlapScan[web_returns] apply RFs: RF2 +--------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------PhysicalProject +------------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_date = date_dim.d_date)) otherCondition=() build RFs:RF1 d_date->[d_date] --------------------------------------PhysicalProject -----------------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_week_seq = date_dim.d_week_seq)) otherCondition=() build RFs:RF0 d_week_seq->[d_week_seq] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF0 -------------------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF1 +--------------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_week_seq = date_dim.d_week_seq)) otherCondition=() build RFs:RF0 d_week_seq->[d_week_seq] --------------------------------------------PhysicalProject -----------------------------------------------filter(d_date IN ('2001-06-06', '2001-09-02', '2001-11-11')) -------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF0 +--------------------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------------------PhysicalProject +------------------------------------------------filter(d_date IN ('2001-06-06', '2001-09-02', '2001-11-11')) +--------------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query93.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query93.out index e3e4a1f5b8db91..f24fba1c86e27c 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query93.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query93.out @@ -11,11 +11,12 @@ PhysicalResultSink ----------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_item_sk = store_sales.ss_item_sk) and (store_returns.sr_ticket_number = store_sales.ss_ticket_number)) otherCondition=() build RFs:RF1 sr_item_sk->[ss_item_sk];RF2 sr_ticket_number->[ss_ticket_number] ------------------PhysicalProject --------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 -------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_reason_sk = reason.r_reason_sk)) otherCondition=() build RFs:RF0 r_reason_sk->[sr_reason_sk] ---------------------PhysicalProject -----------------------PhysicalOlapScan[store_returns] apply RFs: RF0 ---------------------PhysicalDistribute[DistributionSpecReplicated] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_reason_sk = reason.r_reason_sk)) otherCondition=() build RFs:RF0 r_reason_sk->[sr_reason_sk] ----------------------PhysicalProject -------------------------filter((reason.r_reason_desc = 'duplicate purchase')) ---------------------------PhysicalOlapScan[reason] +------------------------PhysicalOlapScan[store_returns] apply RFs: RF0 +----------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------PhysicalProject +--------------------------filter((reason.r_reason_desc = 'duplicate purchase')) +----------------------------PhysicalOlapScan[reason] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query94.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query94.out index 05fda5a6545ec7..c2b1f42cc8d246 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query94.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query94.out @@ -13,26 +13,27 @@ PhysicalResultSink --------------------PhysicalDistribute[DistributionSpecHash] ----------------------PhysicalProject ------------------------PhysicalOlapScan[web_sales] apply RFs: RF4 ---------------------hashJoin[RIGHT_ANTI_JOIN] hashCondition=((ws1.ws_order_number = wr1.wr_order_number)) otherCondition=() build RFs:RF3 ws_order_number->[wr_order_number] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[web_returns] apply RFs: RF3 -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF2 web_site_sk->[ws_web_site_sk] ---------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_ship_date_sk] -----------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[ws_ship_addr_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 +--------------------PhysicalProject +----------------------hashJoin[RIGHT_ANTI_JOIN] hashCondition=((ws1.ws_order_number = wr1.wr_order_number)) otherCondition=() build RFs:RF3 ws_order_number->[wr_order_number] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[web_returns] apply RFs: RF3 +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF2 web_site_sk->[ws_web_site_sk] +----------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_ship_date_sk] +------------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[ws_ship_addr_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 +--------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------PhysicalProject +------------------------------------filter((customer_address.ca_state = 'OK')) +--------------------------------------PhysicalOlapScan[customer_address] ------------------------------PhysicalDistribute[DistributionSpecReplicated] --------------------------------PhysicalProject -----------------------------------filter((customer_address.ca_state = 'OK')) -------------------------------------PhysicalOlapScan[customer_address] +----------------------------------filter((date_dim.d_date <= '2000-04-01') and (date_dim.d_date >= '2000-02-01')) +------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalDistribute[DistributionSpecReplicated] ------------------------------PhysicalProject ---------------------------------filter((date_dim.d_date <= '2000-04-01') and (date_dim.d_date >= '2000-02-01')) -----------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute[DistributionSpecReplicated] -----------------------------PhysicalProject -------------------------------filter((web_site.web_company_name = 'pri')) ---------------------------------PhysicalOlapScan[web_site] +--------------------------------filter((web_site.web_company_name = 'pri')) +----------------------------------PhysicalOlapScan[web_site] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query95.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query95.out index f4c9f48d1728d5..976a8937349c99 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query95.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query95.out @@ -28,26 +28,27 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[web_returns] apply RFs: RF6 -----------------------hashJoin[RIGHT_SEMI_JOIN] hashCondition=((ws1.ws_order_number = ws_wh.ws_order_number)) otherCondition=() build RFs:RF7 ws_order_number->[ws_order_number,ws_order_number] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF3 web_site_sk->[ws_web_site_sk] -----------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_ship_date_sk] -------------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF1 ca_address_sk->[ws_ship_addr_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF2 RF3 +----------------------PhysicalProject +------------------------hashJoin[RIGHT_SEMI_JOIN] hashCondition=((ws1.ws_order_number = ws_wh.ws_order_number)) otherCondition=() build RFs:RF7 ws_order_number->[ws_order_number,ws_order_number] +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------PhysicalProject +------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF3 web_site_sk->[ws_web_site_sk] +------------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_ship_date_sk] +--------------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF1 ca_address_sk->[ws_ship_addr_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF2 RF3 +----------------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------------PhysicalProject +--------------------------------------filter((customer_address.ca_state = 'NC')) +----------------------------------------PhysicalOlapScan[customer_address] --------------------------------PhysicalDistribute[DistributionSpecReplicated] ----------------------------------PhysicalProject -------------------------------------filter((customer_address.ca_state = 'NC')) ---------------------------------------PhysicalOlapScan[customer_address] +------------------------------------filter((date_dim.d_date <= '1999-04-02') and (date_dim.d_date >= '1999-02-01')) +--------------------------------------PhysicalOlapScan[date_dim] ------------------------------PhysicalDistribute[DistributionSpecReplicated] --------------------------------PhysicalProject -----------------------------------filter((date_dim.d_date <= '1999-04-02') and (date_dim.d_date >= '1999-02-01')) -------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalDistribute[DistributionSpecReplicated] -------------------------------PhysicalProject ---------------------------------filter((web_site.web_company_name = 'pri')) -----------------------------------PhysicalOlapScan[web_site] +----------------------------------filter((web_site.web_company_name = 'pri')) +------------------------------------PhysicalOlapScan[web_site] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query99.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query99.out index 921f227b023f73..783ff1229eec41 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query99.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query99.out @@ -11,17 +11,18 @@ PhysicalResultSink ----------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() ------------------PhysicalProject --------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() -----------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_call_center_sk = call_center.cc_call_center_sk)) otherCondition=() -------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_ship_date_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_call_center_sk = call_center.cc_call_center_sk)) otherCondition=() +--------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_ship_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_month_seq <= 1235) and (date_dim.d_month_seq >= 1224)) +----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalDistribute[DistributionSpecReplicated] ----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq <= 1235) and (date_dim.d_month_seq >= 1224)) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[call_center] +------------------------------PhysicalOlapScan[call_center] ----------------------PhysicalDistribute[DistributionSpecReplicated] ------------------------PhysicalProject --------------------------PhysicalOlapScan[ship_mode] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query1.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query1.out index 73dea99cdf9090..eca7d46a8f3ef1 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query1.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query1.out @@ -24,20 +24,21 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------PhysicalProject ------------------PhysicalOlapScan[customer] --------------PhysicalDistribute[DistributionSpecHash] -----------------hashJoin[INNER_JOIN] hashCondition=((ctr1.ctr_store_sk = ctr2.ctr_store_sk)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN] hashCondition=((store.s_store_sk = ctr1.ctr_store_sk)) otherCondition=() build RFs:RF1 s_store_sk->[ctr_store_sk] -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF1 -----------------------PhysicalDistribute[DistributionSpecReplicated] -------------------------PhysicalProject ---------------------------filter((store.s_state = 'SD')) -----------------------------PhysicalOlapScan[store] -------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalDistribute[DistributionSpecExecutionAny] -----------------------------PhysicalProject -------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------PhysicalProject +------------------hashJoin[INNER_JOIN] hashCondition=((ctr1.ctr_store_sk = ctr2.ctr_store_sk)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN] hashCondition=((store.s_store_sk = ctr1.ctr_store_sk)) otherCondition=() build RFs:RF1 s_store_sk->[ctr_store_sk] +------------------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF1 +------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------PhysicalProject +----------------------------filter((store.s_state = 'SD')) +------------------------------PhysicalOlapScan[store] +--------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------------PhysicalProject +--------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query11.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query11.out index 0b0e59d34d4ac0..c1302b449170a5 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query11.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query11.out @@ -44,18 +44,19 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------PhysicalProject ------------------filter((t_w_firstyear.dyear = 2001) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.00)) --------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------hashJoin[INNER_JOIN] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() -----------------PhysicalDistribute[DistributionSpecHash] -------------------PhysicalProject ---------------------filter((t_s_secyear.dyear = 2002) and (t_s_secyear.sale_type = 's')) -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -----------------hashJoin[INNER_JOIN] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=() -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalProject -----------------------filter((t_w_secyear.dyear = 2002) and (t_w_secyear.sale_type = 'w')) -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------PhysicalProject +----------------hashJoin[INNER_JOIN] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() ------------------PhysicalDistribute[DistributionSpecHash] --------------------PhysicalProject -----------------------filter((t_s_firstyear.dyear = 2001) and (t_s_firstyear.sale_type = 's') and (t_s_firstyear.year_total > 0.00)) +----------------------filter((t_s_secyear.dyear = 2002) and (t_s_secyear.sale_type = 's')) ------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------hashJoin[INNER_JOIN] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=() +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------filter((t_w_secyear.dyear = 2002) and (t_w_secyear.sale_type = 'w')) +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------filter((t_s_firstyear.dyear = 2001) and (t_s_firstyear.sale_type = 's') and (t_s_firstyear.year_total > 0.00)) +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) 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 e89101870b893d..f039fb0a521724 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 @@ -10,13 +10,14 @@ PhysicalResultSink --------------PhysicalProject ----------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=(((ca_state IN ('CA', 'GA', 'WA') OR substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274')) OR (catalog_sales.cs_sales_price > 500.00))) build RFs:RF2 c_customer_sk->[cs_bill_customer_sk] ------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF2 -----------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] ------------------------PhysicalProject ---------------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2001)) -----------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF2 +------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------PhysicalProject +----------------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2001)) +------------------------------PhysicalOlapScan[date_dim] ------------------PhysicalDistribute[DistributionSpecHash] --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query16.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query16.out index 869cd7397751db..c867983673a566 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query16.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query16.out @@ -13,26 +13,27 @@ PhysicalResultSink --------------------PhysicalDistribute[DistributionSpecHash] ----------------------PhysicalProject ------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 ---------------------hashJoin[INNER_JOIN] hashCondition=((cs1.cs_call_center_sk = call_center.cc_call_center_sk)) otherCondition=() build RFs:RF3 cc_call_center_sk->[cs_call_center_sk] -----------------------hashJoin[RIGHT_ANTI_JOIN] hashCondition=((cs1.cs_order_number = cr1.cr_order_number)) otherCondition=() build RFs:RF2 cs_order_number->[cr_order_number] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF2 -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------hashJoin[INNER_JOIN] hashCondition=((cs1.cs_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_ship_date_sk] -----------------------------hashJoin[INNER_JOIN] hashCondition=((cs1.cs_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[cs_ship_addr_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF3 +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN] hashCondition=((cs1.cs_call_center_sk = call_center.cc_call_center_sk)) otherCondition=() build RFs:RF3 cc_call_center_sk->[cs_call_center_sk] +------------------------hashJoin[RIGHT_ANTI_JOIN] hashCondition=((cs1.cs_order_number = cr1.cr_order_number)) otherCondition=() build RFs:RF2 cs_order_number->[cr_order_number] +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF2 +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------hashJoin[INNER_JOIN] hashCondition=((cs1.cs_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_ship_date_sk] +------------------------------hashJoin[INNER_JOIN] hashCondition=((cs1.cs_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[cs_ship_addr_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF3 +--------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------PhysicalProject +------------------------------------filter((customer_address.ca_state = 'WV')) +--------------------------------------PhysicalOlapScan[customer_address] ------------------------------PhysicalDistribute[DistributionSpecReplicated] --------------------------------PhysicalProject -----------------------------------filter((customer_address.ca_state = 'WV')) -------------------------------------PhysicalOlapScan[customer_address] -----------------------------PhysicalDistribute[DistributionSpecReplicated] -------------------------------PhysicalProject ---------------------------------filter((date_dim.d_date <= '2002-05-31') and (date_dim.d_date >= '2002-04-01')) -----------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalDistribute[DistributionSpecReplicated] -------------------------PhysicalProject ---------------------------filter(cc_county IN ('Barrow County', 'Daviess County', 'Luce County', 'Richland County', 'Ziebach County')) -----------------------------PhysicalOlapScan[call_center] +----------------------------------filter((date_dim.d_date <= '2002-05-31') and (date_dim.d_date >= '2002-04-01')) +------------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------PhysicalProject +----------------------------filter(cc_county IN ('Barrow County', 'Daviess County', 'Luce County', 'Richland County', 'Ziebach County')) +------------------------------PhysicalOlapScan[call_center] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query21.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query21.out index 7d4c0362945cf7..de0825ccdb1e9d 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query21.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query21.out @@ -12,12 +12,13 @@ PhysicalResultSink ------------------hashJoin[INNER_JOIN] hashCondition=((inventory.inv_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF2 w_warehouse_sk->[inv_warehouse_sk] --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN] hashCondition=((inventory.inv_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[inv_date_sk] -------------------------hashJoin[INNER_JOIN] hashCondition=((item.i_item_sk = inventory.inv_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[inv_item_sk] ---------------------------PhysicalOlapScan[inventory] apply RFs: RF0 RF1 RF2 ---------------------------PhysicalDistribute[DistributionSpecReplicated] -----------------------------PhysicalProject -------------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) ---------------------------------PhysicalOlapScan[item] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN] hashCondition=((item.i_item_sk = inventory.inv_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[inv_item_sk] +----------------------------PhysicalOlapScan[inventory] apply RFs: RF0 RF1 RF2 +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) +----------------------------------PhysicalOlapScan[item] ------------------------PhysicalDistribute[DistributionSpecReplicated] --------------------------PhysicalProject ----------------------------filter((date_dim.d_date <= '2002-03-29') and (date_dim.d_date >= '2002-01-28')) 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 86200679785901..06209793f9e62a 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 @@ -10,25 +10,26 @@ PhysicalResultSink --------------PhysicalProject ----------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[cs_item_sk] ------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_promo_sk = promotion.p_promo_sk)) otherCondition=() build RFs:RF2 p_promo_sk->[cs_promo_sk] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_bill_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[cs_bill_cdemo_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 RF3 -------------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------------PhysicalProject -----------------------------------filter((customer_demographics.cd_education_status = 'Unknown') and (customer_demographics.cd_gender = 'M') and (customer_demographics.cd_marital_status = 'S')) -------------------------------------PhysicalOlapScan[customer_demographics] ---------------------------PhysicalDistribute[DistributionSpecReplicated] -----------------------------PhysicalProject -------------------------------filter((date_dim.d_year = 2001)) ---------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_promo_sk = promotion.p_promo_sk)) otherCondition=() build RFs:RF2 p_promo_sk->[cs_promo_sk] ------------------------PhysicalProject ---------------------------filter(((promotion.p_channel_email = 'N') OR (promotion.p_channel_event = 'N'))) -----------------------------PhysicalOlapScan[promotion] +--------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_bill_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[cs_bill_cdemo_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 RF3 +--------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------PhysicalProject +------------------------------------filter((customer_demographics.cd_education_status = 'Unknown') and (customer_demographics.cd_gender = 'M') and (customer_demographics.cd_marital_status = 'S')) +--------------------------------------PhysicalOlapScan[customer_demographics] +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_year = 2001)) +----------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------PhysicalProject +----------------------------filter(((promotion.p_channel_email = 'N') OR (promotion.p_channel_event = 'N'))) +------------------------------PhysicalOlapScan[promotion] ------------------PhysicalDistribute[DistributionSpecHash] --------------------PhysicalProject ----------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query29.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query29.out index 85996806881158..e0b47aa943df1e 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query29.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query29.out @@ -14,31 +14,34 @@ PhysicalResultSink ----------------------PhysicalProject ------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF7 RF8 RF9 ----------------------PhysicalDistribute[DistributionSpecReplicated] -------------------------hashJoin[INNER_JOIN] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF6 s_store_sk->[ss_store_sk] ---------------------------hashJoin[INNER_JOIN] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() build RFs:RF5 i_item_sk->[ss_item_sk,sr_item_sk] -----------------------------PhysicalDistribute[DistributionSpecHash] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF2 sr_customer_sk->[ss_customer_sk];RF3 sr_item_sk->[ss_item_sk];RF4 sr_ticket_number->[ss_ticket_number] -----------------------------------hashJoin[INNER_JOIN] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF6 s_store_sk->[ss_store_sk] +----------------------------hashJoin[INNER_JOIN] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() build RFs:RF5 i_item_sk->[ss_item_sk,sr_item_sk] +------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF2 sr_customer_sk->[ss_customer_sk];RF3 sr_item_sk->[ss_item_sk];RF4 sr_ticket_number->[ss_ticket_number] ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 RF4 RF5 RF6 -------------------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------------------PhysicalProject -----------------------------------------filter((d1.d_moy = 4) and (d1.d_year = 1999)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[sr_returned_date_sk] +--------------------------------------hashJoin[INNER_JOIN] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 RF4 RF5 RF6 +----------------------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------------------PhysicalProject +--------------------------------------------filter((d1.d_moy = 4) and (d1.d_year = 1999)) +----------------------------------------------PhysicalOlapScan[date_dim] ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF0 RF5 -------------------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------------------PhysicalProject -----------------------------------------filter((d2.d_moy <= 7) and (d2.d_moy >= 4) and (d2.d_year = 1999)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[sr_returned_date_sk] +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF0 RF5 +----------------------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------------------PhysicalProject +--------------------------------------------filter((d2.d_moy <= 7) and (d2.d_moy >= 4) and (d2.d_year = 1999)) +----------------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] +----------------------------PhysicalDistribute[DistributionSpecReplicated] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[item] ---------------------------PhysicalDistribute[DistributionSpecReplicated] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[store] +--------------------------------PhysicalOlapScan[store] ------------------PhysicalDistribute[DistributionSpecReplicated] --------------------PhysicalProject ----------------------filter(d_year IN (1999, 2000, 2001)) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query4.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query4.out index 678e8bf71abac4..3c3a2095b93778 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query4.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query4.out @@ -56,29 +56,31 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------PhysicalProject ------------------filter((t_w_firstyear.dyear = 1999) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.000000)) --------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------hashJoin[INNER_JOIN] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=() -----------------PhysicalDistribute[DistributionSpecHash] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=() +------------------PhysicalDistribute[DistributionSpecHash] +--------------------PhysicalProject +----------------------filter((t_w_secyear.dyear = 2000) and (t_w_secyear.sale_type = 'w')) +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ------------------PhysicalProject ---------------------filter((t_w_secyear.dyear = 2000) and (t_w_secyear.sale_type = 'w')) -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -----------------PhysicalProject -------------------hashJoin[INNER_JOIN] hashCondition=((t_s_firstyear.customer_id = t_c_firstyear.customer_id)) otherCondition=((if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL) > if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL))) ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------PhysicalProject -------------------------filter((t_c_firstyear.dyear = 1999) and (t_c_firstyear.sale_type = 'c') and (t_c_firstyear.year_total > 0.000000)) ---------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------------hashJoin[INNER_JOIN] hashCondition=((t_s_firstyear.customer_id = t_c_secyear.customer_id)) otherCondition=() -----------------------hashJoin[INNER_JOIN] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------filter((t_s_firstyear.dyear = 1999) and (t_s_firstyear.sale_type = 's') and (t_s_firstyear.year_total > 0.000000)) -------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------filter((t_s_secyear.dyear = 2000) and (t_s_secyear.sale_type = 's')) -------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------hashJoin[INNER_JOIN] hashCondition=((t_s_firstyear.customer_id = t_c_firstyear.customer_id)) otherCondition=((if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL) > if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL))) ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------PhysicalProject ---------------------------filter((t_c_secyear.dyear = 2000) and (t_c_secyear.sale_type = 'c')) +--------------------------filter((t_c_firstyear.dyear = 1999) and (t_c_firstyear.sale_type = 'c') and (t_c_firstyear.year_total > 0.000000)) ----------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN] hashCondition=((t_s_firstyear.customer_id = t_c_secyear.customer_id)) otherCondition=() +--------------------------hashJoin[INNER_JOIN] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------filter((t_s_firstyear.dyear = 1999) and (t_s_firstyear.sale_type = 's') and (t_s_firstyear.year_total > 0.000000)) +----------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------filter((t_s_secyear.dyear = 2000) and (t_s_secyear.sale_type = 's')) +----------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------PhysicalProject +------------------------------filter((t_c_secyear.dyear = 2000) and (t_c_secyear.sale_type = 'c')) +--------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query40.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query40.out index 67242e90a55477..981fc65b527055 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query40.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query40.out @@ -13,18 +13,19 @@ PhysicalResultSink --------------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() build RFs:RF2 cs_order_number->[cr_order_number];RF3 cs_item_sk->[cr_item_sk] ----------------------PhysicalProject ------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF2 RF3 -----------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] -------------------------hashJoin[INNER_JOIN] 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 RF4 +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +--------------------------hashJoin[INNER_JOIN] 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 RF4 +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) +----------------------------------PhysicalOlapScan[item] --------------------------PhysicalDistribute[DistributionSpecReplicated] ----------------------------PhysicalProject -------------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) ---------------------------------PhysicalOlapScan[item] -------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------PhysicalProject -----------------------------filter((date_dim.d_date <= '2001-05-02') and (date_dim.d_date >= '2001-03-03')) -------------------------------PhysicalOlapScan[date_dim] +------------------------------filter((date_dim.d_date <= '2001-05-02') and (date_dim.d_date >= '2001-03-03')) +--------------------------------PhysicalOlapScan[date_dim] ------------------PhysicalDistribute[DistributionSpecReplicated] --------------------PhysicalProject ----------------------PhysicalOlapScan[warehouse] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query42.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query42.out index fe908b68c8f3a9..f167fc2b10a4c8 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query42.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query42.out @@ -9,13 +9,14 @@ PhysicalResultSink ------------hashAgg[LOCAL] --------------PhysicalProject ----------------hashJoin[INNER_JOIN] hashCondition=((dt.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] -------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] ---------------------PhysicalProject -----------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 ---------------------PhysicalDistribute[DistributionSpecReplicated] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] ----------------------PhysicalProject -------------------------filter((item.i_manager_id = 1)) ---------------------------PhysicalOlapScan[item] +------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 +----------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------PhysicalProject +--------------------------filter((item.i_manager_id = 1)) +----------------------------PhysicalOlapScan[item] ------------------PhysicalDistribute[DistributionSpecReplicated] --------------------PhysicalProject ----------------------filter((dt.d_moy = 11) and (dt.d_year = 2002)) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query44.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query44.out index 78d9a82df01d4c..a29f21336e4c80 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query44.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query44.out @@ -39,34 +39,35 @@ PhysicalResultSink ------------------------------------------------------filter((store_sales.ss_store_sk = 146) and ss_addr_sk IS NULL) --------------------------------------------------------PhysicalOlapScan[store_sales] ------------PhysicalDistribute[DistributionSpecHash] ---------------hashJoin[INNER_JOIN] hashCondition=((i2.i_item_sk = descending.item_sk)) otherCondition=() build RFs:RF0 item_sk->[i_item_sk] -----------------PhysicalProject -------------------PhysicalOlapScan[item] apply RFs: RF0 -----------------PhysicalDistribute[DistributionSpecHash] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN] hashCondition=((i2.i_item_sk = descending.item_sk)) otherCondition=() build RFs:RF0 item_sk->[i_item_sk] ------------------PhysicalProject ---------------------filter((rnk < 11)) -----------------------PhysicalWindow -------------------------PhysicalQuickSort[MERGE_SORT] ---------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------PhysicalQuickSort[LOCAL_SORT] -------------------------------PhysicalPartitionTopN ---------------------------------PhysicalProject -----------------------------------NestedLoopJoin[INNER_JOIN](cast(rank_col as DOUBLE) > cast((0.9 * rank_col) as DOUBLE)) -------------------------------------PhysicalProject ---------------------------------------hashAgg[GLOBAL] -----------------------------------------PhysicalDistribute[DistributionSpecHash] -------------------------------------------hashAgg[LOCAL] +--------------------PhysicalOlapScan[item] apply RFs: RF0 +------------------PhysicalDistribute[DistributionSpecHash] +--------------------PhysicalProject +----------------------filter((rnk < 11)) +------------------------PhysicalWindow +--------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------PhysicalDistribute[DistributionSpecGather] +------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------PhysicalPartitionTopN +----------------------------------PhysicalProject +------------------------------------NestedLoopJoin[INNER_JOIN](cast(rank_col as DOUBLE) > cast((0.9 * rank_col) as DOUBLE)) +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] +----------------------------------------------PhysicalProject +------------------------------------------------filter((ss1.ss_store_sk = 146)) +--------------------------------------------------PhysicalOlapScan[store_sales] +--------------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------------PhysicalAssertNumRows +------------------------------------------PhysicalDistribute[DistributionSpecGather] --------------------------------------------PhysicalProject -----------------------------------------------filter((ss1.ss_store_sk = 146)) -------------------------------------------------PhysicalOlapScan[store_sales] -------------------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------------------PhysicalAssertNumRows -----------------------------------------PhysicalDistribute[DistributionSpecGather] -------------------------------------------PhysicalProject ---------------------------------------------hashAgg[GLOBAL] -----------------------------------------------PhysicalDistribute[DistributionSpecHash] -------------------------------------------------hashAgg[LOCAL] ---------------------------------------------------PhysicalProject -----------------------------------------------------filter((store_sales.ss_store_sk = 146) and ss_addr_sk IS NULL) -------------------------------------------------------PhysicalOlapScan[store_sales] +----------------------------------------------hashAgg[GLOBAL] +------------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------------hashAgg[LOCAL] +----------------------------------------------------PhysicalProject +------------------------------------------------------filter((store_sales.ss_store_sk = 146) and ss_addr_sk IS NULL) +--------------------------------------------------------PhysicalOlapScan[store_sales] 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 968a23d06260a8..6d639b96748025 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 @@ -35,11 +35,12 @@ PhysicalResultSink ------------------------PhysicalProject --------------------------PhysicalOlapScan[customer_address] ------------PhysicalDistribute[DistributionSpecHash] ---------------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_addr_sk = current_addr.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] -----------------PhysicalDistribute[DistributionSpecHash] -------------------PhysicalProject ---------------------PhysicalOlapScan[customer] apply RFs: RF0 -----------------PhysicalDistribute[DistributionSpecHash] -------------------PhysicalProject ---------------------PhysicalOlapScan[customer_address] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_addr_sk = current_addr.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------PhysicalProject +----------------------PhysicalOlapScan[customer] apply RFs: RF0 +------------------PhysicalDistribute[DistributionSpecHash] +--------------------PhysicalProject +----------------------PhysicalOlapScan[customer_address] 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 fc7edfee4de422..341df6f374e79e 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 @@ -11,14 +11,15 @@ PhysicalResultSink ----------------PhysicalProject ------------------hashJoin[INNER_JOIN] 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] --------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashJoin[INNER_JOIN] 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] -------------------------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 -------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN] 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] --------------------------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')))) -------------------------------PhysicalOlapScan[customer_demographics] +----------------------------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 +--------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------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')))) +--------------------------------PhysicalOlapScan[customer_demographics] --------------------PhysicalDistribute[DistributionSpecHash] ----------------------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/query58.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query58.out index ceda5070451ed2..8eb2246fac8918 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query58.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query58.out @@ -35,63 +35,64 @@ PhysicalResultSink ------------------------PhysicalDistribute[DistributionSpecHash] --------------------------PhysicalProject ----------------------------PhysicalOlapScan[item] apply RFs: RF13 -------------hashJoin[INNER_JOIN] hashCondition=((ss_items.item_id = ws_items.item_id)) otherCondition=((cast(ss_item_rev as DOUBLE) <= cast((1.1 * ws_item_rev) as DOUBLE)) and (cast(ss_item_rev as DOUBLE) >= cast((0.9 * ws_item_rev) as DOUBLE)) and (cast(ws_item_rev as DOUBLE) <= cast((1.1 * ss_item_rev) as DOUBLE)) and (cast(ws_item_rev as DOUBLE) >= cast((0.9 * ss_item_rev) as DOUBLE))) build RFs:RF8 item_id->[i_item_id] ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF7 i_item_sk->[ss_item_sk] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ss_sold_date_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 RF7 -------------------------------PhysicalDistribute[DistributionSpecReplicated] +------------PhysicalProject +--------------hashJoin[INNER_JOIN] hashCondition=((ss_items.item_id = ws_items.item_id)) otherCondition=((cast(ss_item_rev as DOUBLE) <= cast((1.1 * ws_item_rev) as DOUBLE)) and (cast(ss_item_rev as DOUBLE) >= cast((0.9 * ws_item_rev) as DOUBLE)) and (cast(ws_item_rev as DOUBLE) <= cast((1.1 * ss_item_rev) as DOUBLE)) and (cast(ws_item_rev as DOUBLE) >= cast((0.9 * ss_item_rev) as DOUBLE))) build RFs:RF8 item_id->[i_item_id] +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF7 i_item_sk->[ss_item_sk] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ss_sold_date_sk] --------------------------------PhysicalProject -----------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_date = date_dim.d_date)) otherCondition=() build RFs:RF5 d_date->[d_date] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF5 -------------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 RF7 +--------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------PhysicalProject +------------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_date = date_dim.d_date)) otherCondition=() build RFs:RF5 d_date->[d_date] --------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN] hashCondition=((date_dim.d_week_seq = date_dim.d_week_seq)) otherCondition=() build RFs:RF4 d_week_seq->[d_week_seq] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF4 -------------------------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------------------------PhysicalAssertNumRows -----------------------------------------------PhysicalDistribute[DistributionSpecGather] -------------------------------------------------PhysicalProject ---------------------------------------------------filter((date_dim.d_date = '2001-03-24')) -----------------------------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] apply RFs: RF8 ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] +----------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF5 +--------------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN] hashCondition=((date_dim.d_week_seq = date_dim.d_week_seq)) otherCondition=() build RFs:RF4 d_week_seq->[d_week_seq] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF4 +--------------------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------------------PhysicalAssertNumRows +------------------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_date = '2001-03-24')) +------------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------PhysicalDistribute[DistributionSpecHash] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[web_sales] apply RFs: RF2 RF3 -------------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------------PhysicalOlapScan[item] apply RFs: RF8 +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] --------------------------------PhysicalProject -----------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_date = date_dim.d_date)) otherCondition=() build RFs:RF1 d_date->[d_date] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF1 -------------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF2 RF3 +--------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------PhysicalProject +------------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_date = date_dim.d_date)) otherCondition=() build RFs:RF1 d_date->[d_date] --------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN] hashCondition=((date_dim.d_week_seq = date_dim.d_week_seq)) otherCondition=() build RFs:RF0 d_week_seq->[d_week_seq] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF0 -------------------------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------------------------PhysicalAssertNumRows -----------------------------------------------PhysicalDistribute[DistributionSpecGather] -------------------------------------------------PhysicalProject ---------------------------------------------------filter((date_dim.d_date = '2001-03-24')) -----------------------------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] +----------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF1 +--------------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN] hashCondition=((date_dim.d_week_seq = date_dim.d_week_seq)) otherCondition=() build RFs:RF0 d_week_seq->[d_week_seq] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF0 +--------------------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------------------PhysicalAssertNumRows +------------------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_date = '2001-03-24')) +------------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query62.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query62.out index 82115bef298fe9..595dfbe8f1070c 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query62.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query62.out @@ -11,17 +11,18 @@ PhysicalResultSink ----------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF3 w_warehouse_sk->[ws_warehouse_sk] ------------------PhysicalProject --------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF2 sm_ship_mode_sk->[ws_ship_mode_sk] -----------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF1 web_site_sk->[ws_web_site_sk] -------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_ship_date_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 RF3 +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF1 web_site_sk->[ws_web_site_sk] +--------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_ship_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 RF3 +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_month_seq <= 1205) and (date_dim.d_month_seq >= 1194)) +----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalDistribute[DistributionSpecReplicated] ----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq <= 1205) and (date_dim.d_month_seq >= 1194)) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[web_site] +------------------------------PhysicalOlapScan[web_site] ----------------------PhysicalDistribute[DistributionSpecReplicated] ------------------------PhysicalProject --------------------------PhysicalOlapScan[ship_mode] 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 11d575eff53104..75e48c17970ff0 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 @@ -10,25 +10,26 @@ PhysicalResultSink --------------PhysicalProject ----------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] ------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_promo_sk = promotion.p_promo_sk)) otherCondition=() build RFs:RF2 p_promo_sk->[ss_promo_sk] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN] 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] hashCondition=((store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 -------------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------------PhysicalProject -----------------------------------filter((customer_demographics.cd_education_status = 'College') and (customer_demographics.cd_gender = 'F') and (customer_demographics.cd_marital_status = 'W')) -------------------------------------PhysicalOlapScan[customer_demographics] ---------------------------PhysicalDistribute[DistributionSpecReplicated] -----------------------------PhysicalProject -------------------------------filter((date_dim.d_year = 2001)) ---------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_promo_sk = promotion.p_promo_sk)) otherCondition=() build RFs:RF2 p_promo_sk->[ss_promo_sk] ------------------------PhysicalProject ---------------------------filter(((promotion.p_channel_email = 'N') OR (promotion.p_channel_event = 'N'))) -----------------------------PhysicalOlapScan[promotion] +--------------------------hashJoin[INNER_JOIN] 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] hashCondition=((store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 +--------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------PhysicalProject +------------------------------------filter((customer_demographics.cd_education_status = 'College') and (customer_demographics.cd_gender = 'F') and (customer_demographics.cd_marital_status = 'W')) +--------------------------------------PhysicalOlapScan[customer_demographics] +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_year = 2001)) +----------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------PhysicalProject +----------------------------filter(((promotion.p_channel_email = 'N') OR (promotion.p_channel_event = 'N'))) +------------------------------PhysicalOlapScan[promotion] ------------------PhysicalDistribute[DistributionSpecHash] --------------------PhysicalProject ----------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query74.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query74.out index 1d3336d590d3cc..05d89d51e4b799 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query74.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query74.out @@ -44,18 +44,19 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------PhysicalProject ------------------filter((t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year = 1999) and (t_w_firstyear.year_total > 0.0)) --------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------hashJoin[INNER_JOIN] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=() -----------------hashJoin[INNER_JOIN] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalProject -----------------------filter((t_s_firstyear.sale_type = 's') and (t_s_firstyear.year = 1999) and (t_s_firstyear.year_total > 0.0)) -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------PhysicalProject +----------------hashJoin[INNER_JOIN] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=() +------------------hashJoin[INNER_JOIN] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------filter((t_s_firstyear.sale_type = 's') and (t_s_firstyear.year = 1999) and (t_s_firstyear.year_total > 0.0)) +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------filter((t_s_secyear.sale_type = 's') and (t_s_secyear.year = 2000)) +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ------------------PhysicalDistribute[DistributionSpecHash] --------------------PhysicalProject -----------------------filter((t_s_secyear.sale_type = 's') and (t_s_secyear.year = 2000)) +----------------------filter((t_w_secyear.sale_type = 'w') and (t_w_secyear.year = 2000)) ------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -----------------PhysicalDistribute[DistributionSpecHash] -------------------PhysicalProject ---------------------filter((t_w_secyear.sale_type = 'w') and (t_w_secyear.year = 2000)) -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query76.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query76.out index bbb934bf88ac68..bdd871b35d5fd6 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query76.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query76.out @@ -14,37 +14,40 @@ PhysicalResultSink ----------------------PhysicalProject ------------------------PhysicalOlapScan[item] apply RFs: RF1 ----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 ss_sold_date_sk->[d_date_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[date_dim] apply RFs: RF0 ---------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 ss_sold_date_sk->[d_date_sk] ----------------------------PhysicalProject -------------------------------filter(ss_hdemo_sk IS NULL) ---------------------------------PhysicalOlapScan[store_sales] +------------------------------PhysicalOlapScan[date_dim] apply RFs: RF0 +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------filter(ss_hdemo_sk IS NULL) +----------------------------------PhysicalOlapScan[store_sales] ----------------PhysicalDistribute[DistributionSpecExecutionAny] ------------------PhysicalProject --------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 ws_item_sk->[i_item_sk] ----------------------PhysicalProject ------------------------PhysicalOlapScan[item] apply RFs: RF3 ----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 ws_sold_date_sk->[d_date_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[date_dim] apply RFs: RF2 ---------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 ws_sold_date_sk->[d_date_sk] ----------------------------PhysicalProject -------------------------------filter(ws_bill_addr_sk IS NULL) ---------------------------------PhysicalOlapScan[web_sales] +------------------------------PhysicalOlapScan[date_dim] apply RFs: RF2 +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------filter(ws_bill_addr_sk IS NULL) +----------------------------------PhysicalOlapScan[web_sales] ----------------PhysicalDistribute[DistributionSpecExecutionAny] ------------------PhysicalProject --------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF5 cs_item_sk->[i_item_sk] ----------------------PhysicalProject ------------------------PhysicalOlapScan[item] apply RFs: RF5 ----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 cs_sold_date_sk->[d_date_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[date_dim] apply RFs: RF4 ---------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 cs_sold_date_sk->[d_date_sk] ----------------------------PhysicalProject -------------------------------filter(cs_warehouse_sk IS NULL) ---------------------------------PhysicalOlapScan[catalog_sales] +------------------------------PhysicalOlapScan[date_dim] apply RFs: RF4 +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------filter(cs_warehouse_sk IS NULL) +----------------------------------PhysicalOlapScan[catalog_sales] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query83.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query83.out index bd813f63a54151..8d75439401df13 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query83.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query83.out @@ -32,58 +32,59 @@ PhysicalResultSink ------------------------------------------PhysicalProject --------------------------------------------filter(d_date IN ('2001-06-06', '2001-09-02', '2001-11-11')) ----------------------------------------------PhysicalOlapScan[date_dim] -------------hashJoin[INNER_JOIN] hashCondition=((sr_items.item_id = wr_items.item_id)) otherCondition=() build RFs:RF8 item_id->[i_item_id] ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF7 i_item_sk->[sr_item_sk] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_returned_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[sr_returned_date_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_returns] apply RFs: RF6 RF7 -------------------------------PhysicalDistribute[DistributionSpecReplicated] +------------PhysicalProject +--------------hashJoin[INNER_JOIN] hashCondition=((sr_items.item_id = wr_items.item_id)) otherCondition=() build RFs:RF8 item_id->[i_item_id] +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF7 i_item_sk->[sr_item_sk] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_returned_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[sr_returned_date_sk] --------------------------------PhysicalProject -----------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_date = date_dim.d_date)) otherCondition=() build RFs:RF5 d_date->[d_date] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF5 -------------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------PhysicalOlapScan[store_returns] apply RFs: RF6 RF7 +--------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------PhysicalProject +------------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_date = date_dim.d_date)) otherCondition=() build RFs:RF5 d_date->[d_date] --------------------------------------PhysicalProject -----------------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_week_seq = date_dim.d_week_seq)) otherCondition=() build RFs:RF4 d_week_seq->[d_week_seq] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF4 -------------------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF5 +--------------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_week_seq = date_dim.d_week_seq)) otherCondition=() build RFs:RF4 d_week_seq->[d_week_seq] --------------------------------------------PhysicalProject -----------------------------------------------filter(d_date IN ('2001-06-06', '2001-09-02', '2001-11-11')) -------------------------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] apply RFs: RF8 ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN] hashCondition=((web_returns.wr_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 wr_item_sk->[i_item_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] apply RFs: RF3 ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashJoin[INNER_JOIN] hashCondition=((web_returns.wr_returned_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[wr_returned_date_sk] +----------------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF4 +--------------------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------------------PhysicalProject +------------------------------------------------filter(d_date IN ('2001-06-06', '2001-09-02', '2001-11-11')) +--------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------PhysicalDistribute[DistributionSpecHash] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[web_returns] apply RFs: RF2 -------------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------------PhysicalOlapScan[item] apply RFs: RF8 +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN] hashCondition=((web_returns.wr_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 wr_item_sk->[i_item_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[item] apply RFs: RF3 +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------hashJoin[INNER_JOIN] hashCondition=((web_returns.wr_returned_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[wr_returned_date_sk] --------------------------------PhysicalProject -----------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_date = date_dim.d_date)) otherCondition=() build RFs:RF1 d_date->[d_date] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF1 -------------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------PhysicalOlapScan[web_returns] apply RFs: RF2 +--------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------PhysicalProject +------------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_date = date_dim.d_date)) otherCondition=() build RFs:RF1 d_date->[d_date] --------------------------------------PhysicalProject -----------------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_week_seq = date_dim.d_week_seq)) otherCondition=() build RFs:RF0 d_week_seq->[d_week_seq] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF0 -------------------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF1 +--------------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_week_seq = date_dim.d_week_seq)) otherCondition=() build RFs:RF0 d_week_seq->[d_week_seq] --------------------------------------------PhysicalProject -----------------------------------------------filter(d_date IN ('2001-06-06', '2001-09-02', '2001-11-11')) -------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF0 +--------------------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------------------PhysicalProject +------------------------------------------------filter(d_date IN ('2001-06-06', '2001-09-02', '2001-11-11')) +--------------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query93.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query93.out index e3e4a1f5b8db91..f24fba1c86e27c 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query93.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query93.out @@ -11,11 +11,12 @@ PhysicalResultSink ----------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_item_sk = store_sales.ss_item_sk) and (store_returns.sr_ticket_number = store_sales.ss_ticket_number)) otherCondition=() build RFs:RF1 sr_item_sk->[ss_item_sk];RF2 sr_ticket_number->[ss_ticket_number] ------------------PhysicalProject --------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 -------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_reason_sk = reason.r_reason_sk)) otherCondition=() build RFs:RF0 r_reason_sk->[sr_reason_sk] ---------------------PhysicalProject -----------------------PhysicalOlapScan[store_returns] apply RFs: RF0 ---------------------PhysicalDistribute[DistributionSpecReplicated] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_reason_sk = reason.r_reason_sk)) otherCondition=() build RFs:RF0 r_reason_sk->[sr_reason_sk] ----------------------PhysicalProject -------------------------filter((reason.r_reason_desc = 'duplicate purchase')) ---------------------------PhysicalOlapScan[reason] +------------------------PhysicalOlapScan[store_returns] apply RFs: RF0 +----------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------PhysicalProject +--------------------------filter((reason.r_reason_desc = 'duplicate purchase')) +----------------------------PhysicalOlapScan[reason] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query94.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query94.out index 05fda5a6545ec7..c2b1f42cc8d246 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query94.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query94.out @@ -13,26 +13,27 @@ PhysicalResultSink --------------------PhysicalDistribute[DistributionSpecHash] ----------------------PhysicalProject ------------------------PhysicalOlapScan[web_sales] apply RFs: RF4 ---------------------hashJoin[RIGHT_ANTI_JOIN] hashCondition=((ws1.ws_order_number = wr1.wr_order_number)) otherCondition=() build RFs:RF3 ws_order_number->[wr_order_number] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[web_returns] apply RFs: RF3 -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF2 web_site_sk->[ws_web_site_sk] ---------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_ship_date_sk] -----------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[ws_ship_addr_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 +--------------------PhysicalProject +----------------------hashJoin[RIGHT_ANTI_JOIN] hashCondition=((ws1.ws_order_number = wr1.wr_order_number)) otherCondition=() build RFs:RF3 ws_order_number->[wr_order_number] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[web_returns] apply RFs: RF3 +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF2 web_site_sk->[ws_web_site_sk] +----------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_ship_date_sk] +------------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[ws_ship_addr_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 +--------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------PhysicalProject +------------------------------------filter((customer_address.ca_state = 'OK')) +--------------------------------------PhysicalOlapScan[customer_address] ------------------------------PhysicalDistribute[DistributionSpecReplicated] --------------------------------PhysicalProject -----------------------------------filter((customer_address.ca_state = 'OK')) -------------------------------------PhysicalOlapScan[customer_address] +----------------------------------filter((date_dim.d_date <= '2000-04-01') and (date_dim.d_date >= '2000-02-01')) +------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalDistribute[DistributionSpecReplicated] ------------------------------PhysicalProject ---------------------------------filter((date_dim.d_date <= '2000-04-01') and (date_dim.d_date >= '2000-02-01')) -----------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute[DistributionSpecReplicated] -----------------------------PhysicalProject -------------------------------filter((web_site.web_company_name = 'pri')) ---------------------------------PhysicalOlapScan[web_site] +--------------------------------filter((web_site.web_company_name = 'pri')) +----------------------------------PhysicalOlapScan[web_site] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query95.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query95.out index 17c36c6120e58b..7477231b372203 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query95.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query95.out @@ -28,26 +28,27 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[web_returns] apply RFs: RF12 RF13 -----------------------hashJoin[RIGHT_SEMI_JOIN] hashCondition=((ws1.ws_order_number = ws_wh.ws_order_number)) otherCondition=() build RFs:RF14 ws_order_number->[ws_order_number,ws_order_number];RF15 ws_order_number->[ws_order_number,ws_order_number] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF6 web_site_sk->[ws_web_site_sk];RF7 web_site_sk->[ws_web_site_sk] -----------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ws_ship_date_sk];RF5 d_date_sk->[ws_ship_date_sk] -------------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[ws_ship_addr_sk];RF3 ca_address_sk->[ws_ship_addr_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF2 RF3 RF4 RF5 RF6 RF7 +----------------------PhysicalProject +------------------------hashJoin[RIGHT_SEMI_JOIN] hashCondition=((ws1.ws_order_number = ws_wh.ws_order_number)) otherCondition=() build RFs:RF14 ws_order_number->[ws_order_number,ws_order_number];RF15 ws_order_number->[ws_order_number,ws_order_number] +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------PhysicalProject +------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF6 web_site_sk->[ws_web_site_sk];RF7 web_site_sk->[ws_web_site_sk] +------------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ws_ship_date_sk];RF5 d_date_sk->[ws_ship_date_sk] +--------------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[ws_ship_addr_sk];RF3 ca_address_sk->[ws_ship_addr_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF2 RF3 RF4 RF5 RF6 RF7 +----------------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------------PhysicalProject +--------------------------------------filter((customer_address.ca_state = 'NC')) +----------------------------------------PhysicalOlapScan[customer_address] --------------------------------PhysicalDistribute[DistributionSpecReplicated] ----------------------------------PhysicalProject -------------------------------------filter((customer_address.ca_state = 'NC')) ---------------------------------------PhysicalOlapScan[customer_address] +------------------------------------filter((date_dim.d_date <= '1999-04-02') and (date_dim.d_date >= '1999-02-01')) +--------------------------------------PhysicalOlapScan[date_dim] ------------------------------PhysicalDistribute[DistributionSpecReplicated] --------------------------------PhysicalProject -----------------------------------filter((date_dim.d_date <= '1999-04-02') and (date_dim.d_date >= '1999-02-01')) -------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalDistribute[DistributionSpecReplicated] -------------------------------PhysicalProject ---------------------------------filter((web_site.web_company_name = 'pri')) -----------------------------------PhysicalOlapScan[web_site] +----------------------------------filter((web_site.web_company_name = 'pri')) +------------------------------------PhysicalOlapScan[web_site] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query99.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query99.out index 46f26c2f1f1b55..7fb723236cdfa9 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query99.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query99.out @@ -11,17 +11,18 @@ PhysicalResultSink ----------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF3 w_warehouse_sk->[cs_warehouse_sk] ------------------PhysicalProject --------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF2 sm_ship_mode_sk->[cs_ship_mode_sk] -----------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_call_center_sk = call_center.cc_call_center_sk)) otherCondition=() build RFs:RF1 cc_call_center_sk->[cs_call_center_sk] -------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_ship_date_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 RF3 +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_call_center_sk = call_center.cc_call_center_sk)) otherCondition=() build RFs:RF1 cc_call_center_sk->[cs_call_center_sk] +--------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_ship_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 RF3 +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_month_seq <= 1235) and (date_dim.d_month_seq >= 1224)) +----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalDistribute[DistributionSpecReplicated] ----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq <= 1235) and (date_dim.d_month_seq >= 1224)) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[call_center] +------------------------------PhysicalOlapScan[call_center] ----------------------PhysicalDistribute[DistributionSpecReplicated] ------------------------PhysicalProject --------------------------PhysicalOlapScan[ship_mode] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/nostats_rf_prune/q15.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/nostats_rf_prune/q15.out index 66fefac51975dd..108af45c2035f8 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/nostats_rf_prune/q15.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/nostats_rf_prune/q15.out @@ -6,25 +6,26 @@ PhysicalResultSink ------PhysicalQuickSort[LOCAL_SORT] --------PhysicalProject ----------hashJoin[INNER_JOIN] hashCondition=((supplier.s_suppkey = revenue0.supplier_no)) otherCondition=() -------------hashJoin[INNER_JOIN] hashCondition=((revenue0.total_revenue = max(total_revenue))) otherCondition=() ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------filter((lineitem.l_shipdate < '1996-04-01') and (lineitem.l_shipdate >= '1996-01-01')) ---------------------------PhysicalOlapScan[lineitem] ---------------PhysicalDistribute[DistributionSpecReplicated] -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecGather] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter((lineitem.l_shipdate < '1996-04-01') and (lineitem.l_shipdate >= '1996-01-01')) -----------------------------------PhysicalOlapScan[lineitem] +------------PhysicalProject +--------------hashJoin[INNER_JOIN] hashCondition=((revenue0.total_revenue = max(total_revenue))) otherCondition=() +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------filter((lineitem.l_shipdate < '1996-04-01') and (lineitem.l_shipdate >= '1996-01-01')) +----------------------------PhysicalOlapScan[lineitem] +----------------PhysicalDistribute[DistributionSpecReplicated] +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecGather] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashAgg[GLOBAL] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------hashAgg[LOCAL] +--------------------------------PhysicalProject +----------------------------------filter((lineitem.l_shipdate < '1996-04-01') and (lineitem.l_shipdate >= '1996-01-01')) +------------------------------------PhysicalOlapScan[lineitem] ------------PhysicalDistribute[DistributionSpecHash] --------------PhysicalProject ----------------PhysicalOlapScan[supplier] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/nostats_rf_prune/q22.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/nostats_rf_prune/q22.out index 69c64e0e5b6096..768320b234b649 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/nostats_rf_prune/q22.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/nostats_rf_prune/q22.out @@ -13,15 +13,16 @@ PhysicalResultSink --------------------PhysicalProject ----------------------PhysicalOlapScan[orders] apply RFs: RF0 ------------------PhysicalDistribute[DistributionSpecHash] ---------------------NestedLoopJoin[INNER_JOIN](cast(c_acctbal as DECIMALV3(38, 4)) > avg(cast(c_acctbal as DECIMALV3(17, 4)))) -----------------------PhysicalProject -------------------------filter(substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) ---------------------------PhysicalOlapScan[customer] -----------------------PhysicalDistribute[DistributionSpecReplicated] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter((customer.c_acctbal > 0.00) and substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) -----------------------------------PhysicalOlapScan[customer] +--------------------PhysicalProject +----------------------NestedLoopJoin[INNER_JOIN](cast(c_acctbal as DECIMALV3(38, 4)) > avg(cast(c_acctbal as DECIMALV3(17, 4)))) +------------------------PhysicalProject +--------------------------filter(substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) +----------------------------PhysicalOlapScan[customer] +------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------hashAgg[GLOBAL] +----------------------------PhysicalDistribute[DistributionSpecGather] +------------------------------hashAgg[LOCAL] +--------------------------------PhysicalProject +----------------------------------filter((customer.c_acctbal > 0.00) and substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) +------------------------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q15.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q15.out index 02dd9a5d7bcdca..1203c328d87044 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q15.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q15.out @@ -9,23 +9,24 @@ PhysicalResultSink ------------PhysicalProject --------------PhysicalOlapScan[supplier] apply RFs: RF0 ------------PhysicalDistribute[DistributionSpecHash] ---------------hashJoin[INNER_JOIN] hashCondition=((revenue0.total_revenue = max(total_revenue))) otherCondition=() -----------------PhysicalProject -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------filter((lineitem.l_shipdate < '1996-04-01') and (lineitem.l_shipdate >= '1996-01-01')) -----------------------------PhysicalOlapScan[lineitem] -----------------PhysicalDistribute[DistributionSpecReplicated] -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashAgg[GLOBAL] -----------------------------PhysicalDistribute[DistributionSpecHash] -------------------------------hashAgg[LOCAL] ---------------------------------PhysicalProject -----------------------------------filter((lineitem.l_shipdate < '1996-04-01') and (lineitem.l_shipdate >= '1996-01-01')) -------------------------------------PhysicalOlapScan[lineitem] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN] hashCondition=((revenue0.total_revenue = max(total_revenue))) otherCondition=() +------------------PhysicalProject +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------filter((lineitem.l_shipdate < '1996-04-01') and (lineitem.l_shipdate >= '1996-01-01')) +------------------------------PhysicalOlapScan[lineitem] +------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute[DistributionSpecGather] +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------hashAgg[GLOBAL] +------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------hashAgg[LOCAL] +----------------------------------PhysicalProject +------------------------------------filter((lineitem.l_shipdate < '1996-04-01') and (lineitem.l_shipdate >= '1996-01-01')) +--------------------------------------PhysicalOlapScan[lineitem] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q16.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q16.out index 719e9b99f86abc..e26fad635f7687 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q16.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q16.out @@ -10,12 +10,13 @@ PhysicalResultSink --------------PhysicalProject ----------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((partsupp.ps_suppkey = supplier.s_suppkey)) otherCondition=() ------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashJoin[INNER_JOIN] hashCondition=((part.p_partkey = partsupp.ps_partkey)) otherCondition=() build RFs:RF0 p_partkey->[ps_partkey] -----------------------PhysicalProject -------------------------PhysicalOlapScan[partsupp] apply RFs: RF0 -----------------------PhysicalProject -------------------------filter(( not (p_brand = 'Brand#45')) and ( not (p_type like 'MEDIUM POLISHED%')) and p_size IN (14, 19, 23, 3, 36, 45, 49, 9)) ---------------------------PhysicalOlapScan[part] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN] hashCondition=((part.p_partkey = partsupp.ps_partkey)) otherCondition=() build RFs:RF0 p_partkey->[ps_partkey] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[partsupp] apply RFs: RF0 +------------------------PhysicalProject +--------------------------filter(( not (p_brand = 'Brand#45')) and ( not (p_type like 'MEDIUM POLISHED%')) and p_size IN (14, 19, 23, 3, 36, 45, 49, 9)) +----------------------------PhysicalOlapScan[part] ------------------PhysicalDistribute[DistributionSpecHash] --------------------PhysicalProject ----------------------filter((s_comment like '%Customer%Complaints%')) diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q20-rewrite.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q20-rewrite.out index 4856ba9c8749e6..037a45f3d83720 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q20-rewrite.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q20-rewrite.out @@ -24,11 +24,12 @@ PhysicalResultSink ------------------------filter((p_name like 'forest%')) --------------------------PhysicalOlapScan[part] ------------PhysicalDistribute[DistributionSpecHash] ---------------hashJoin[INNER_JOIN] hashCondition=((supplier.s_nationkey = nation.n_nationkey)) otherCondition=() build RFs:RF0 n_nationkey->[s_nationkey] -----------------PhysicalProject -------------------PhysicalOlapScan[supplier] apply RFs: RF0 -----------------PhysicalDistribute[DistributionSpecReplicated] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN] hashCondition=((supplier.s_nationkey = nation.n_nationkey)) otherCondition=() build RFs:RF0 n_nationkey->[s_nationkey] ------------------PhysicalProject ---------------------filter((nation.n_name = 'CANADA')) -----------------------PhysicalOlapScan[nation] +--------------------PhysicalOlapScan[supplier] apply RFs: RF0 +------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------PhysicalProject +----------------------filter((nation.n_name = 'CANADA')) +------------------------PhysicalOlapScan[nation] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q20.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q20.out index bbfe15d9102de9..e4a0293c22177d 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q20.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q20.out @@ -23,11 +23,12 @@ PhysicalResultSink ------------------------filter((p_name like 'forest%')) --------------------------PhysicalOlapScan[part] ------------PhysicalDistribute[DistributionSpecHash] ---------------hashJoin[INNER_JOIN] hashCondition=((supplier.s_nationkey = nation.n_nationkey)) otherCondition=() build RFs:RF0 n_nationkey->[s_nationkey] -----------------PhysicalProject -------------------PhysicalOlapScan[supplier] apply RFs: RF0 -----------------PhysicalDistribute[DistributionSpecReplicated] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN] hashCondition=((supplier.s_nationkey = nation.n_nationkey)) otherCondition=() build RFs:RF0 n_nationkey->[s_nationkey] ------------------PhysicalProject ---------------------filter((nation.n_name = 'CANADA')) -----------------------PhysicalOlapScan[nation] +--------------------PhysicalOlapScan[supplier] apply RFs: RF0 +------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------PhysicalProject +----------------------filter((nation.n_name = 'CANADA')) +------------------------PhysicalOlapScan[nation] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q21.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q21.out index 815ebd60e97e72..1ceb96da996e05 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q21.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q21.out @@ -11,24 +11,25 @@ PhysicalResultSink ----------------hashJoin[RIGHT_SEMI_JOIN] hashCondition=((l2.l_orderkey = l1.l_orderkey)) otherCondition=(( not (l_suppkey = l_suppkey))) build RFs:RF4 l_orderkey->[l_orderkey] ------------------PhysicalProject --------------------PhysicalOlapScan[lineitem] apply RFs: RF4 -------------------hashJoin[INNER_JOIN] hashCondition=((orders.o_orderkey = l1.l_orderkey)) otherCondition=() build RFs:RF3 l_orderkey->[o_orderkey] ---------------------PhysicalProject -----------------------filter((orders.o_orderstatus = 'F')) -------------------------PhysicalOlapScan[orders] apply RFs: RF3 ---------------------hashJoin[RIGHT_ANTI_JOIN] hashCondition=((l3.l_orderkey = l1.l_orderkey)) otherCondition=(( not (l_suppkey = l_suppkey))) build RFs:RF2 l_orderkey->[l_orderkey] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN] hashCondition=((orders.o_orderkey = l1.l_orderkey)) otherCondition=() build RFs:RF3 l_orderkey->[o_orderkey] ----------------------PhysicalProject -------------------------filter((l3.l_receiptdate > l3.l_commitdate)) ---------------------------PhysicalOlapScan[lineitem] apply RFs: RF2 -----------------------hashJoin[INNER_JOIN] hashCondition=((supplier.s_suppkey = l1.l_suppkey)) otherCondition=() build RFs:RF1 s_suppkey->[l_suppkey] +------------------------filter((orders.o_orderstatus = 'F')) +--------------------------PhysicalOlapScan[orders] apply RFs: RF3 +----------------------hashJoin[RIGHT_ANTI_JOIN] hashCondition=((l3.l_orderkey = l1.l_orderkey)) otherCondition=(( not (l_suppkey = l_suppkey))) build RFs:RF2 l_orderkey->[l_orderkey] ------------------------PhysicalProject ---------------------------filter((l1.l_receiptdate > l1.l_commitdate)) -----------------------------PhysicalOlapScan[lineitem] apply RFs: RF1 -------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------hashJoin[INNER_JOIN] hashCondition=((supplier.s_nationkey = nation.n_nationkey)) otherCondition=() build RFs:RF0 n_nationkey->[s_nationkey] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[supplier] apply RFs: RF0 -----------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------filter((l3.l_receiptdate > l3.l_commitdate)) +----------------------------PhysicalOlapScan[lineitem] apply RFs: RF2 +------------------------hashJoin[INNER_JOIN] hashCondition=((supplier.s_suppkey = l1.l_suppkey)) otherCondition=() build RFs:RF1 s_suppkey->[l_suppkey] +--------------------------PhysicalProject +----------------------------filter((l1.l_receiptdate > l1.l_commitdate)) +------------------------------PhysicalOlapScan[lineitem] apply RFs: RF1 +--------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------hashJoin[INNER_JOIN] hashCondition=((supplier.s_nationkey = nation.n_nationkey)) otherCondition=() build RFs:RF0 n_nationkey->[s_nationkey] ------------------------------PhysicalProject ---------------------------------filter((nation.n_name = 'SAUDI ARABIA')) -----------------------------------PhysicalOlapScan[nation] +--------------------------------PhysicalOlapScan[supplier] apply RFs: RF0 +------------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------------PhysicalProject +----------------------------------filter((nation.n_name = 'SAUDI ARABIA')) +------------------------------------PhysicalOlapScan[nation] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q22.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q22.out index 69c64e0e5b6096..768320b234b649 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q22.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q22.out @@ -13,15 +13,16 @@ PhysicalResultSink --------------------PhysicalProject ----------------------PhysicalOlapScan[orders] apply RFs: RF0 ------------------PhysicalDistribute[DistributionSpecHash] ---------------------NestedLoopJoin[INNER_JOIN](cast(c_acctbal as DECIMALV3(38, 4)) > avg(cast(c_acctbal as DECIMALV3(17, 4)))) -----------------------PhysicalProject -------------------------filter(substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) ---------------------------PhysicalOlapScan[customer] -----------------------PhysicalDistribute[DistributionSpecReplicated] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter((customer.c_acctbal > 0.00) and substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) -----------------------------------PhysicalOlapScan[customer] +--------------------PhysicalProject +----------------------NestedLoopJoin[INNER_JOIN](cast(c_acctbal as DECIMALV3(38, 4)) > avg(cast(c_acctbal as DECIMALV3(17, 4)))) +------------------------PhysicalProject +--------------------------filter(substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) +----------------------------PhysicalOlapScan[customer] +------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------hashAgg[GLOBAL] +----------------------------PhysicalDistribute[DistributionSpecGather] +------------------------------hashAgg[LOCAL] +--------------------------------PhysicalProject +----------------------------------filter((customer.c_acctbal > 0.00) and substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) +------------------------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q5.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q5.out index 0c5e71bdc26b32..e037476380b7ac 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q5.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q5.out @@ -20,17 +20,18 @@ PhysicalResultSink ------------------------------filter((orders.o_orderdate < '1995-01-01') and (orders.o_orderdate >= '1994-01-01')) --------------------------------PhysicalOlapScan[orders] ------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------hashJoin[INNER_JOIN] hashCondition=((supplier.s_nationkey = nation.n_nationkey)) otherCondition=() build RFs:RF1 n_nationkey->[s_nationkey] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[supplier] apply RFs: RF1 -----------------------------PhysicalDistribute[DistributionSpecReplicated] -------------------------------hashJoin[INNER_JOIN] hashCondition=((nation.n_regionkey = region.r_regionkey)) otherCondition=() build RFs:RF0 r_regionkey->[n_regionkey] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[nation] apply RFs: RF0 ---------------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN] hashCondition=((supplier.s_nationkey = nation.n_nationkey)) otherCondition=() build RFs:RF1 n_nationkey->[s_nationkey] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[supplier] apply RFs: RF1 +------------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------------hashJoin[INNER_JOIN] hashCondition=((nation.n_regionkey = region.r_regionkey)) otherCondition=() build RFs:RF0 r_regionkey->[n_regionkey] ----------------------------------PhysicalProject -------------------------------------filter((region.r_name = 'ASIA')) ---------------------------------------PhysicalOlapScan[region] +------------------------------------PhysicalOlapScan[nation] apply RFs: RF0 +----------------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------------PhysicalProject +--------------------------------------filter((region.r_name = 'ASIA')) +----------------------------------------PhysicalOlapScan[region] ------------------PhysicalDistribute[DistributionSpecHash] --------------------PhysicalProject ----------------------PhysicalOlapScan[customer] 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 0f11a38ddee401..5dc5bfcc1e9cb3 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 @@ -15,13 +15,14 @@ PhysicalResultSink ------------------------filter((lineitem.l_shipdate <= '1996-12-31') and (lineitem.l_shipdate >= '1995-01-01')) --------------------------PhysicalOlapScan[lineitem] apply RFs: RF3 RF4 ----------------------PhysicalDistribute[DistributionSpecReplicated] -------------------------hashJoin[INNER_JOIN] hashCondition=((supplier.s_nationkey = n1.n_nationkey)) otherCondition=() build RFs:RF2 n_nationkey->[s_nationkey] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[supplier] apply RFs: RF2 ---------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN] hashCondition=((supplier.s_nationkey = n1.n_nationkey)) otherCondition=() build RFs:RF2 n_nationkey->[s_nationkey] ----------------------------PhysicalProject -------------------------------filter(n_name IN ('FRANCE', 'GERMANY')) ---------------------------------PhysicalOlapScan[nation] +------------------------------PhysicalOlapScan[supplier] apply RFs: RF2 +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------filter(n_name IN ('FRANCE', 'GERMANY')) +----------------------------------PhysicalOlapScan[nation] ------------------PhysicalDistribute[DistributionSpecHash] --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_custkey = orders.o_custkey)) otherCondition=() build RFs:RF1 c_custkey->[o_custkey] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q9.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q9.out index 63d585c9c77d98..24e4fdec458c62 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q9.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q9.out @@ -12,19 +12,20 @@ PhysicalResultSink ------------------PhysicalProject --------------------hashJoin[INNER_JOIN] hashCondition=((supplier.s_suppkey = lineitem.l_suppkey)) otherCondition=() ----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashJoin[INNER_JOIN] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=() build RFs:RF2 l_orderkey->[o_orderkey] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[orders] apply RFs: RF2 ---------------------------PhysicalDistribute[DistributionSpecHash] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=() build RFs:RF2 l_orderkey->[o_orderkey] ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN] hashCondition=((part.p_partkey = lineitem.l_partkey)) otherCondition=() build RFs:RF1 p_partkey->[l_partkey] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[lineitem] apply RFs: RF1 ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------PhysicalProject -------------------------------------filter((p_name like '%green%')) ---------------------------------------PhysicalOlapScan[part] +------------------------------PhysicalOlapScan[orders] apply RFs: RF2 +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN] hashCondition=((part.p_partkey = lineitem.l_partkey)) otherCondition=() build RFs:RF1 p_partkey->[l_partkey] +----------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[lineitem] apply RFs: RF1 +----------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------PhysicalProject +--------------------------------------filter((p_name like '%green%')) +----------------------------------------PhysicalOlapScan[part] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------hashJoin[INNER_JOIN] hashCondition=((supplier.s_nationkey = nation.n_nationkey)) otherCondition=() --------------------------PhysicalProject diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q15.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q15.out index 02dd9a5d7bcdca..1203c328d87044 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q15.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q15.out @@ -9,23 +9,24 @@ PhysicalResultSink ------------PhysicalProject --------------PhysicalOlapScan[supplier] apply RFs: RF0 ------------PhysicalDistribute[DistributionSpecHash] ---------------hashJoin[INNER_JOIN] hashCondition=((revenue0.total_revenue = max(total_revenue))) otherCondition=() -----------------PhysicalProject -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------filter((lineitem.l_shipdate < '1996-04-01') and (lineitem.l_shipdate >= '1996-01-01')) -----------------------------PhysicalOlapScan[lineitem] -----------------PhysicalDistribute[DistributionSpecReplicated] -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashAgg[GLOBAL] -----------------------------PhysicalDistribute[DistributionSpecHash] -------------------------------hashAgg[LOCAL] ---------------------------------PhysicalProject -----------------------------------filter((lineitem.l_shipdate < '1996-04-01') and (lineitem.l_shipdate >= '1996-01-01')) -------------------------------------PhysicalOlapScan[lineitem] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN] hashCondition=((revenue0.total_revenue = max(total_revenue))) otherCondition=() +------------------PhysicalProject +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------filter((lineitem.l_shipdate < '1996-04-01') and (lineitem.l_shipdate >= '1996-01-01')) +------------------------------PhysicalOlapScan[lineitem] +------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute[DistributionSpecGather] +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------hashAgg[GLOBAL] +------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------hashAgg[LOCAL] +----------------------------------PhysicalProject +------------------------------------filter((lineitem.l_shipdate < '1996-04-01') and (lineitem.l_shipdate >= '1996-01-01')) +--------------------------------------PhysicalOlapScan[lineitem] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q16.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q16.out index 719e9b99f86abc..e26fad635f7687 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q16.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q16.out @@ -10,12 +10,13 @@ PhysicalResultSink --------------PhysicalProject ----------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((partsupp.ps_suppkey = supplier.s_suppkey)) otherCondition=() ------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashJoin[INNER_JOIN] hashCondition=((part.p_partkey = partsupp.ps_partkey)) otherCondition=() build RFs:RF0 p_partkey->[ps_partkey] -----------------------PhysicalProject -------------------------PhysicalOlapScan[partsupp] apply RFs: RF0 -----------------------PhysicalProject -------------------------filter(( not (p_brand = 'Brand#45')) and ( not (p_type like 'MEDIUM POLISHED%')) and p_size IN (14, 19, 23, 3, 36, 45, 49, 9)) ---------------------------PhysicalOlapScan[part] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN] hashCondition=((part.p_partkey = partsupp.ps_partkey)) otherCondition=() build RFs:RF0 p_partkey->[ps_partkey] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[partsupp] apply RFs: RF0 +------------------------PhysicalProject +--------------------------filter(( not (p_brand = 'Brand#45')) and ( not (p_type like 'MEDIUM POLISHED%')) and p_size IN (14, 19, 23, 3, 36, 45, 49, 9)) +----------------------------PhysicalOlapScan[part] ------------------PhysicalDistribute[DistributionSpecHash] --------------------PhysicalProject ----------------------filter((s_comment like '%Customer%Complaints%')) diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q20-rewrite.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q20-rewrite.out index 4856ba9c8749e6..037a45f3d83720 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q20-rewrite.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q20-rewrite.out @@ -24,11 +24,12 @@ PhysicalResultSink ------------------------filter((p_name like 'forest%')) --------------------------PhysicalOlapScan[part] ------------PhysicalDistribute[DistributionSpecHash] ---------------hashJoin[INNER_JOIN] hashCondition=((supplier.s_nationkey = nation.n_nationkey)) otherCondition=() build RFs:RF0 n_nationkey->[s_nationkey] -----------------PhysicalProject -------------------PhysicalOlapScan[supplier] apply RFs: RF0 -----------------PhysicalDistribute[DistributionSpecReplicated] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN] hashCondition=((supplier.s_nationkey = nation.n_nationkey)) otherCondition=() build RFs:RF0 n_nationkey->[s_nationkey] ------------------PhysicalProject ---------------------filter((nation.n_name = 'CANADA')) -----------------------PhysicalOlapScan[nation] +--------------------PhysicalOlapScan[supplier] apply RFs: RF0 +------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------PhysicalProject +----------------------filter((nation.n_name = 'CANADA')) +------------------------PhysicalOlapScan[nation] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q20.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q20.out index bbfe15d9102de9..e4a0293c22177d 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q20.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q20.out @@ -23,11 +23,12 @@ PhysicalResultSink ------------------------filter((p_name like 'forest%')) --------------------------PhysicalOlapScan[part] ------------PhysicalDistribute[DistributionSpecHash] ---------------hashJoin[INNER_JOIN] hashCondition=((supplier.s_nationkey = nation.n_nationkey)) otherCondition=() build RFs:RF0 n_nationkey->[s_nationkey] -----------------PhysicalProject -------------------PhysicalOlapScan[supplier] apply RFs: RF0 -----------------PhysicalDistribute[DistributionSpecReplicated] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN] hashCondition=((supplier.s_nationkey = nation.n_nationkey)) otherCondition=() build RFs:RF0 n_nationkey->[s_nationkey] ------------------PhysicalProject ---------------------filter((nation.n_name = 'CANADA')) -----------------------PhysicalOlapScan[nation] +--------------------PhysicalOlapScan[supplier] apply RFs: RF0 +------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------PhysicalProject +----------------------filter((nation.n_name = 'CANADA')) +------------------------PhysicalOlapScan[nation] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q21.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q21.out index 815ebd60e97e72..1ceb96da996e05 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q21.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q21.out @@ -11,24 +11,25 @@ PhysicalResultSink ----------------hashJoin[RIGHT_SEMI_JOIN] hashCondition=((l2.l_orderkey = l1.l_orderkey)) otherCondition=(( not (l_suppkey = l_suppkey))) build RFs:RF4 l_orderkey->[l_orderkey] ------------------PhysicalProject --------------------PhysicalOlapScan[lineitem] apply RFs: RF4 -------------------hashJoin[INNER_JOIN] hashCondition=((orders.o_orderkey = l1.l_orderkey)) otherCondition=() build RFs:RF3 l_orderkey->[o_orderkey] ---------------------PhysicalProject -----------------------filter((orders.o_orderstatus = 'F')) -------------------------PhysicalOlapScan[orders] apply RFs: RF3 ---------------------hashJoin[RIGHT_ANTI_JOIN] hashCondition=((l3.l_orderkey = l1.l_orderkey)) otherCondition=(( not (l_suppkey = l_suppkey))) build RFs:RF2 l_orderkey->[l_orderkey] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN] hashCondition=((orders.o_orderkey = l1.l_orderkey)) otherCondition=() build RFs:RF3 l_orderkey->[o_orderkey] ----------------------PhysicalProject -------------------------filter((l3.l_receiptdate > l3.l_commitdate)) ---------------------------PhysicalOlapScan[lineitem] apply RFs: RF2 -----------------------hashJoin[INNER_JOIN] hashCondition=((supplier.s_suppkey = l1.l_suppkey)) otherCondition=() build RFs:RF1 s_suppkey->[l_suppkey] +------------------------filter((orders.o_orderstatus = 'F')) +--------------------------PhysicalOlapScan[orders] apply RFs: RF3 +----------------------hashJoin[RIGHT_ANTI_JOIN] hashCondition=((l3.l_orderkey = l1.l_orderkey)) otherCondition=(( not (l_suppkey = l_suppkey))) build RFs:RF2 l_orderkey->[l_orderkey] ------------------------PhysicalProject ---------------------------filter((l1.l_receiptdate > l1.l_commitdate)) -----------------------------PhysicalOlapScan[lineitem] apply RFs: RF1 -------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------hashJoin[INNER_JOIN] hashCondition=((supplier.s_nationkey = nation.n_nationkey)) otherCondition=() build RFs:RF0 n_nationkey->[s_nationkey] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[supplier] apply RFs: RF0 -----------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------filter((l3.l_receiptdate > l3.l_commitdate)) +----------------------------PhysicalOlapScan[lineitem] apply RFs: RF2 +------------------------hashJoin[INNER_JOIN] hashCondition=((supplier.s_suppkey = l1.l_suppkey)) otherCondition=() build RFs:RF1 s_suppkey->[l_suppkey] +--------------------------PhysicalProject +----------------------------filter((l1.l_receiptdate > l1.l_commitdate)) +------------------------------PhysicalOlapScan[lineitem] apply RFs: RF1 +--------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------hashJoin[INNER_JOIN] hashCondition=((supplier.s_nationkey = nation.n_nationkey)) otherCondition=() build RFs:RF0 n_nationkey->[s_nationkey] ------------------------------PhysicalProject ---------------------------------filter((nation.n_name = 'SAUDI ARABIA')) -----------------------------------PhysicalOlapScan[nation] +--------------------------------PhysicalOlapScan[supplier] apply RFs: RF0 +------------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------------PhysicalProject +----------------------------------filter((nation.n_name = 'SAUDI ARABIA')) +------------------------------------PhysicalOlapScan[nation] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q22.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q22.out index 69c64e0e5b6096..768320b234b649 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q22.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q22.out @@ -13,15 +13,16 @@ PhysicalResultSink --------------------PhysicalProject ----------------------PhysicalOlapScan[orders] apply RFs: RF0 ------------------PhysicalDistribute[DistributionSpecHash] ---------------------NestedLoopJoin[INNER_JOIN](cast(c_acctbal as DECIMALV3(38, 4)) > avg(cast(c_acctbal as DECIMALV3(17, 4)))) -----------------------PhysicalProject -------------------------filter(substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) ---------------------------PhysicalOlapScan[customer] -----------------------PhysicalDistribute[DistributionSpecReplicated] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter((customer.c_acctbal > 0.00) and substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) -----------------------------------PhysicalOlapScan[customer] +--------------------PhysicalProject +----------------------NestedLoopJoin[INNER_JOIN](cast(c_acctbal as DECIMALV3(38, 4)) > avg(cast(c_acctbal as DECIMALV3(17, 4)))) +------------------------PhysicalProject +--------------------------filter(substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) +----------------------------PhysicalOlapScan[customer] +------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------hashAgg[GLOBAL] +----------------------------PhysicalDistribute[DistributionSpecGather] +------------------------------hashAgg[LOCAL] +--------------------------------PhysicalProject +----------------------------------filter((customer.c_acctbal > 0.00) and substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) +------------------------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q5.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q5.out index 0963b0a556a20e..dc1adcfafab09a 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q5.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q5.out @@ -20,17 +20,18 @@ PhysicalResultSink ------------------------------filter((orders.o_orderdate < '1995-01-01') and (orders.o_orderdate >= '1994-01-01')) --------------------------------PhysicalOlapScan[orders] apply RFs: RF5 ------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------hashJoin[INNER_JOIN] hashCondition=((supplier.s_nationkey = nation.n_nationkey)) otherCondition=() build RFs:RF1 n_nationkey->[s_nationkey] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[supplier] apply RFs: RF1 RF4 -----------------------------PhysicalDistribute[DistributionSpecReplicated] -------------------------------hashJoin[INNER_JOIN] hashCondition=((nation.n_regionkey = region.r_regionkey)) otherCondition=() build RFs:RF0 r_regionkey->[n_regionkey] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[nation] apply RFs: RF0 RF4 ---------------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN] hashCondition=((supplier.s_nationkey = nation.n_nationkey)) otherCondition=() build RFs:RF1 n_nationkey->[s_nationkey] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[supplier] apply RFs: RF1 RF4 +------------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------------hashJoin[INNER_JOIN] hashCondition=((nation.n_regionkey = region.r_regionkey)) otherCondition=() build RFs:RF0 r_regionkey->[n_regionkey] ----------------------------------PhysicalProject -------------------------------------filter((region.r_name = 'ASIA')) ---------------------------------------PhysicalOlapScan[region] +------------------------------------PhysicalOlapScan[nation] apply RFs: RF0 RF4 +----------------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------------PhysicalProject +--------------------------------------filter((region.r_name = 'ASIA')) +----------------------------------------PhysicalOlapScan[region] ------------------PhysicalDistribute[DistributionSpecHash] --------------------PhysicalProject ----------------------PhysicalOlapScan[customer] 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 0f11a38ddee401..5dc5bfcc1e9cb3 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 @@ -15,13 +15,14 @@ PhysicalResultSink ------------------------filter((lineitem.l_shipdate <= '1996-12-31') and (lineitem.l_shipdate >= '1995-01-01')) --------------------------PhysicalOlapScan[lineitem] apply RFs: RF3 RF4 ----------------------PhysicalDistribute[DistributionSpecReplicated] -------------------------hashJoin[INNER_JOIN] hashCondition=((supplier.s_nationkey = n1.n_nationkey)) otherCondition=() build RFs:RF2 n_nationkey->[s_nationkey] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[supplier] apply RFs: RF2 ---------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN] hashCondition=((supplier.s_nationkey = n1.n_nationkey)) otherCondition=() build RFs:RF2 n_nationkey->[s_nationkey] ----------------------------PhysicalProject -------------------------------filter(n_name IN ('FRANCE', 'GERMANY')) ---------------------------------PhysicalOlapScan[nation] +------------------------------PhysicalOlapScan[supplier] apply RFs: RF2 +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------filter(n_name IN ('FRANCE', 'GERMANY')) +----------------------------------PhysicalOlapScan[nation] ------------------PhysicalDistribute[DistributionSpecHash] --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_custkey = orders.o_custkey)) otherCondition=() build RFs:RF1 c_custkey->[o_custkey] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q9.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q9.out index af4927d5f0a80b..f94b72a9700813 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q9.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q9.out @@ -12,19 +12,20 @@ PhysicalResultSink ------------------PhysicalProject --------------------hashJoin[INNER_JOIN] hashCondition=((supplier.s_suppkey = lineitem.l_suppkey)) otherCondition=() build RFs:RF3 s_suppkey->[l_suppkey] ----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashJoin[INNER_JOIN] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=() build RFs:RF2 l_orderkey->[o_orderkey] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[orders] apply RFs: RF2 ---------------------------PhysicalDistribute[DistributionSpecHash] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=() build RFs:RF2 l_orderkey->[o_orderkey] ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN] hashCondition=((part.p_partkey = lineitem.l_partkey)) otherCondition=() build RFs:RF1 p_partkey->[l_partkey] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[lineitem] apply RFs: RF1 RF3 RF4 RF5 ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------PhysicalProject -------------------------------------filter((p_name like '%green%')) ---------------------------------------PhysicalOlapScan[part] apply RFs: RF5 +------------------------------PhysicalOlapScan[orders] apply RFs: RF2 +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN] hashCondition=((part.p_partkey = lineitem.l_partkey)) otherCondition=() build RFs:RF1 p_partkey->[l_partkey] +----------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[lineitem] apply RFs: RF1 RF3 RF4 RF5 +----------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------PhysicalProject +--------------------------------------filter((p_name like '%green%')) +----------------------------------------PhysicalOlapScan[part] apply RFs: RF5 ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------hashJoin[INNER_JOIN] hashCondition=((supplier.s_nationkey = nation.n_nationkey)) otherCondition=() build RFs:RF0 n_nationkey->[s_nationkey] --------------------------PhysicalProject diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q15.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q15.out index c24e199e98f3f2..eb5a495c767cc6 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q15.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q15.out @@ -6,25 +6,26 @@ PhysicalResultSink ------PhysicalQuickSort[LOCAL_SORT] --------PhysicalProject ----------hashJoin[INNER_JOIN] hashCondition=((supplier.s_suppkey = revenue0.supplier_no)) otherCondition=() build RFs:RF0 s_suppkey->[l_suppkey] -------------hashJoin[INNER_JOIN] hashCondition=((revenue0.total_revenue = max(total_revenue))) otherCondition=() ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------filter((lineitem.l_shipdate < '1996-04-01') and (lineitem.l_shipdate >= '1996-01-01')) ---------------------------PhysicalOlapScan[lineitem] apply RFs: RF0 ---------------PhysicalDistribute[DistributionSpecReplicated] -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecGather] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter((lineitem.l_shipdate < '1996-04-01') and (lineitem.l_shipdate >= '1996-01-01')) -----------------------------------PhysicalOlapScan[lineitem] +------------PhysicalProject +--------------hashJoin[INNER_JOIN] hashCondition=((revenue0.total_revenue = max(total_revenue))) otherCondition=() +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------filter((lineitem.l_shipdate < '1996-04-01') and (lineitem.l_shipdate >= '1996-01-01')) +----------------------------PhysicalOlapScan[lineitem] apply RFs: RF0 +----------------PhysicalDistribute[DistributionSpecReplicated] +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecGather] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashAgg[GLOBAL] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------hashAgg[LOCAL] +--------------------------------PhysicalProject +----------------------------------filter((lineitem.l_shipdate < '1996-04-01') and (lineitem.l_shipdate >= '1996-01-01')) +------------------------------------PhysicalOlapScan[lineitem] ------------PhysicalDistribute[DistributionSpecHash] --------------PhysicalProject ----------------PhysicalOlapScan[supplier] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q22.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q22.out index 69c64e0e5b6096..768320b234b649 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q22.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q22.out @@ -13,15 +13,16 @@ PhysicalResultSink --------------------PhysicalProject ----------------------PhysicalOlapScan[orders] apply RFs: RF0 ------------------PhysicalDistribute[DistributionSpecHash] ---------------------NestedLoopJoin[INNER_JOIN](cast(c_acctbal as DECIMALV3(38, 4)) > avg(cast(c_acctbal as DECIMALV3(17, 4)))) -----------------------PhysicalProject -------------------------filter(substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) ---------------------------PhysicalOlapScan[customer] -----------------------PhysicalDistribute[DistributionSpecReplicated] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter((customer.c_acctbal > 0.00) and substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) -----------------------------------PhysicalOlapScan[customer] +--------------------PhysicalProject +----------------------NestedLoopJoin[INNER_JOIN](cast(c_acctbal as DECIMALV3(38, 4)) > avg(cast(c_acctbal as DECIMALV3(17, 4)))) +------------------------PhysicalProject +--------------------------filter(substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) +----------------------------PhysicalOlapScan[customer] +------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------hashAgg[GLOBAL] +----------------------------PhysicalDistribute[DistributionSpecGather] +------------------------------hashAgg[LOCAL] +--------------------------------PhysicalProject +----------------------------------filter((customer.c_acctbal > 0.00) and substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) +------------------------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/workload_manager_p0/test_curd_wlg.out b/regression-test/data/workload_manager_p0/test_curd_wlg.out index 13b33fd9b839a7..fca16d077e4095 100644 --- a/regression-test/data/workload_manager_p0/test_curd_wlg.out +++ b/regression-test/data/workload_manager_p0/test_curd_wlg.out @@ -6,15 +6,15 @@ 2 -- !show_1 -- -normal 20 50% true 2147483647 0 0 1% 16 -test_group 10 10% true 2147483647 0 0 0% -1 +normal 20 50% true 2147483647 0 0 1% 16 +test_group 10 10% true 2147483647 0 0 -1 -1 -- !mem_limit_1 -- 2 -- !mem_limit_2 -- normal 20 50% true 2147483647 0 0 1% 16 -test_group 10 11% true 2147483647 0 0 0% -1 +test_group 10 11% true 2147483647 0 0 -1 -1 -- !mem_overcommit_1 -- 2 @@ -24,7 +24,7 @@ test_group 10 11% true 2147483647 0 0 0% -1 -- !mem_overcommit_3 -- normal 20 50% true 2147483647 0 0 1% 16 -test_group 10 11% false 2147483647 0 0 0% -1 +test_group 10 11% false 2147483647 0 0 -1 -1 -- !cpu_hard_limit_1 -- 2 @@ -45,14 +45,22 @@ normal 20 50% true 2147483647 0 0 1% 16 test_group 10 11% false 100 0 0 20% -1 -- !show_spill_1 -- -spill_group_test 1024 0% true 2147483647 0 0 0% -1 10% 10% +spill_group_test 1024 0% true 2147483647 0 0 -1 -1 10% 10% -- !show_spill_1 -- -spill_group_test 1024 0% true 2147483647 0 0 0% -1 -1 10% +spill_group_test 1024 0% true 2147483647 0 0 -1 -1 -1 10% -- !show_spill_2 -- -spill_group_test 1024 0% true 2147483647 0 0 0% -1 5% 10% +spill_group_test 1024 0% true 2147483647 0 0 -1 -1 5% 10% -- !show_spill_3 -- -spill_group_test 1024 0% true 2147483647 0 0 0% -1 5% 40% +spill_group_test 1024 0% true 2147483647 0 0 -1 -1 5% 40% + +-- !show_wg_tag -- +tag1_mem_wg1 50% -1 mem_tag1 +tag1_mem_wg2 49% -1 mem_tag1 +tag1_mem_wg3 1% -1 mem_tag1 +tag1_wg1 0% 10% tag1 +tag1_wg2 0% 10% tag1 +tag1_wg3 0% 80% tag1 diff --git a/regression-test/framework/pom.xml b/regression-test/framework/pom.xml index 6447218f0298ec..9cf3b55409228f 100644 --- a/regression-test/framework/pom.xml +++ b/regression-test/framework/pom.xml @@ -75,7 +75,7 @@ under the License. 4.9.3 2.8.0 1.11.95 - 15.0.0 + 15.0.2 diff --git a/regression-test/framework/src/main/groovy/org/apache/doris/regression/action/StreamLoadAction.groovy b/regression-test/framework/src/main/groovy/org/apache/doris/regression/action/StreamLoadAction.groovy index 56c80e88a40857..606b9bc4ac8922 100644 --- a/regression-test/framework/src/main/groovy/org/apache/doris/regression/action/StreamLoadAction.groovy +++ b/regression-test/framework/src/main/groovy/org/apache/doris/regression/action/StreamLoadAction.groovy @@ -54,6 +54,7 @@ class StreamLoadAction implements SuiteAction { Map headers SuiteContext context boolean directToBe = false + boolean twoPhaseCommit = false StreamLoadAction(SuiteContext context) { this.address = context.getFeHttpAddress() @@ -137,6 +138,22 @@ class StreamLoadAction implements SuiteAction { this.time = time.call() } + void twoPhaseCommit(boolean twoPhaseCommit) { + this.twoPhaseCommit = twoPhaseCommit; + } + + void twoPhaseCommit(Closure twoPhaseCommit) { + this.twoPhaseCommit = twoPhaseCommit.call(); + } + + // compatible with selectdb case + void isCloud(boolean isCloud) { + } + + // compatible with selectdb case + void isCloud(Closure isCloud) { + } + void check(@ClosureParams(value = FromString, options = ["String,Throwable,Long,Long"]) Closure check) { this.check = check } @@ -156,8 +173,14 @@ class StreamLoadAction implements SuiteAction { long startTime = System.currentTimeMillis() def isHttpStream = headers.containsKey("version") try { - def uri = isHttpStream ? "http://${address.hostString}:${address.port}/api/_http_stream" - : "http://${address.hostString}:${address.port}/api/${db}/${table}/_stream_load" + def uri = "" + if (isHttpStream) { + uri = "http://${address.hostString}:${address.port}/api/_http_stream" + } else if (twoPhaseCommit) { + uri = "http://${address.hostString}:${address.port}/api/${db}/_stream_load_2pc" + } else { + uri = "http://${address.hostString}:${address.port}/api/${db}/${table}/_stream_load" + } HttpClients.createDefault().withCloseable { client -> RequestBuilder requestBuilder = prepareRequestHeader(RequestBuilder.put(uri)) HttpEntity httpEntity = prepareHttpEntity(client) @@ -362,6 +385,10 @@ class StreamLoadAction implements SuiteAction { def jsonSlurper = new JsonSlurper() def parsed = jsonSlurper.parseText(responseText) String status = parsed.Status + if (twoPhaseCommit) { + status = parsed.status + return status; + } long txnId = parsed.TxnId if (!status.equalsIgnoreCase("Publish Timeout")) { return status; diff --git a/regression-test/pipeline/cloud_p0/conf/regression-conf-custom.groovy b/regression-test/pipeline/cloud_p0/conf/regression-conf-custom.groovy index 5c7162e1eef197..cdd49d76183137 100644 --- a/regression-test/pipeline/cloud_p0/conf/regression-conf-custom.groovy +++ b/regression-test/pipeline/cloud_p0/conf/regression-conf-custom.groovy @@ -54,7 +54,6 @@ excludeDirectories = "000_the_start_sentinel_do_not_touch," + // keep this line "unique_with_mow_p0/cluster_key," + "unique_with_mow_p0/ssb_unique_sql_zstd_cluster," + "unique_with_mow_p0/ssb_unique_load_zstd_c," + - "nereids_rules_p0/mv," + "backup_restore," + // not a case for cloud mode, no need to run "cold_heat_separation," + "storage_medium_p0," + diff --git a/regression-test/pipeline/external/conf/regression-conf.groovy b/regression-test/pipeline/external/conf/regression-conf.groovy index f04ea8574127b6..43994cccd8eb91 100644 --- a/regression-test/pipeline/external/conf/regression-conf.groovy +++ b/regression-test/pipeline/external/conf/regression-conf.groovy @@ -63,16 +63,11 @@ excludeSuites = "000_the_start_sentinel_do_not_touch," + // keep this line as th "test_cast_string_to_array," + "test_refresh_mtmv," + "test_spark_load," + - "test_trino_hive_orc," + - "test_trino_hive_other," + "test_hive_write_insert," + "test_broker_load_func," + "test_hive_write_partitions," + "zzz_the_end_sentinel_do_not_touch" // keep this line as the last line -// this directories will not be executed -excludeDirectories = "" - customConf1 = "test_custom_conf_value" // for test csv with header diff --git a/regression-test/pipeline/p0/conf/regression-conf.groovy b/regression-test/pipeline/p0/conf/regression-conf.groovy index c5d9a57a0b2f19..20238b26312afc 100644 --- a/regression-test/pipeline/p0/conf/regression-conf.groovy +++ b/regression-test/pipeline/p0/conf/regression-conf.groovy @@ -70,8 +70,6 @@ excludeSuites = "000_the_start_sentinel_do_not_touch," + // keep this line as th "test_profile," + "test_refresh_mtmv," + "test_spark_load," + - "test_trino_hive_orc," + - "test_trino_hive_other," + "test_hive_write_insert," + "test_hive_write_partitions," + "test_broker_load_func," + diff --git a/regression-test/suites/account_p0/test_nereids_row_policy.groovy b/regression-test/suites/account_p0/test_nereids_row_policy.groovy index c071d88306c8fe..6ae858997b11a0 100644 --- a/regression-test/suites/account_p0/test_nereids_row_policy.groovy +++ b/regression-test/suites/account_p0/test_nereids_row_policy.groovy @@ -103,6 +103,11 @@ suite("test_nereids_row_policy") { sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}"""; } + dropPolciy "policy0" + dropPolciy "policy1" + dropPolciy "policy2" + dropPolciy "policy3" + // no policy assertQueryResult 3 @@ -126,4 +131,8 @@ suite("test_nereids_row_policy") { createPolicy"policy3", "k = 2", "PERMISSIVE" assertQueryResult 2 + dropPolciy "policy0" + dropPolciy "policy1" + dropPolciy "policy2" + dropPolciy "policy3" } diff --git a/regression-test/suites/alter_p0/test_decommission_with_replica_num_fail.groovy b/regression-test/suites/alter_p0/test_decommission_with_replica_num_fail.groovy new file mode 100644 index 00000000000000..ff19adae27dad2 --- /dev/null +++ b/regression-test/suites/alter_p0/test_decommission_with_replica_num_fail.groovy @@ -0,0 +1,59 @@ +// 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_decommission_with_replica_num_fail') { + if (isCloudMode()) { + return + } + + def tbl = 'test_decommission_with_replica_num_fail' + def backends = sql_return_maparray('show backends') + def replicaNum = 0 + def targetBackend = null + for (def be : backends) { + def alive = be.Alive.toBoolean() + def decommissioned = be.SystemDecommissioned.toBoolean() + if (alive && !decommissioned) { + replicaNum++ + targetBackend = be + } + } + assertTrue(replicaNum > 0) + + sql "DROP TABLE IF EXISTS ${tbl} FORCE" + sql """ + CREATE TABLE ${tbl} + ( + k1 int, + k2 int + ) + DISTRIBUTED BY HASH(k1) BUCKETS 6 + PROPERTIES + ( + "replication_num" = "${replicaNum}" + ); + """ + try { + test { + sql "ALTER SYSTEM DECOMMISSION BACKEND '${targetBackend.Host}:${targetBackend.HeartbeatPort}'" + exception "otherwise need to decrease the partition's replication num" + } + } finally { + sql "CANCEL DECOMMISSION BACKEND '${targetBackend.Host}:${targetBackend.HeartbeatPort}'" + } + sql "DROP TABLE IF EXISTS ${tbl} FORCE" +} diff --git a/regression-test/suites/catalog_recycle_bin_p0/recover_with_schema_change.groovy b/regression-test/suites/catalog_recycle_bin_p0/recover_with_schema_change.groovy index a23c449ab95c05..a6f494999246a4 100644 --- a/regression-test/suites/catalog_recycle_bin_p0/recover_with_schema_change.groovy +++ b/regression-test/suites/catalog_recycle_bin_p0/recover_with_schema_change.groovy @@ -116,7 +116,7 @@ suite("recover_with_schema_change") { } else { test { sql """ insert into ${table} values(4, 'b', '2022-01-02'); """ - exception "Insert has filtered data in strict mode" + exception "" } order_qt_sql """ select name from ${table}; """ } diff --git a/regression-test/suites/correctness_p0/test_probe_clean.groovy b/regression-test/suites/correctness_p0/test_probe_clean.groovy new file mode 100644 index 00000000000000..febc05f66fbe02 --- /dev/null +++ b/regression-test/suites/correctness_p0/test_probe_clean.groovy @@ -0,0 +1,95 @@ +// 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. + +// The cases is copied from https://github.com/trinodb/trino/tree/master +// /testing/trino-product-tests/src/main/resources/sql-tests/testcases/aggregate +// and modified by Doris. + +suite("test_probe_clean") { + +sql """ drop table IF EXISTS clearblocktable1; """ +sql """ + CREATE TABLE IF NOT EXISTS clearblocktable1 ( + `col_int_undef_signed` INT NULL COMMENT "", + `col_int_undef_signed_not_null` INT NOT NULL COMMENT "", + `col_date_undef_signed_not_null` date(11) NOT NULL COMMENT "", + + ) ENGINE=OLAP + DUPLICATE KEY(`col_int_undef_signed`) + DISTRIBUTED BY HASH(`col_int_undef_signed`) BUCKETS 1 + PROPERTIES ( + 'replication_num' = '1' +); +""" + + +sql """ +insert into clearblocktable1 values(1,1,'2020-01-01'); +""" +sql """ +drop table IF EXISTS clearblocktable2; +""" +sql """ +CREATE TABLE IF NOT EXISTS clearblocktable2 ( + `col_int_undef_signed` INT NULL COMMENT "", + `col_int_undef_signed_not_null` INT NOT NULL COMMENT "", + `col_date_undef_signed_not_null` date(11) NOT NULL COMMENT "", + + ) ENGINE=OLAP + DUPLICATE KEY(`col_int_undef_signed`) + DISTRIBUTED BY HASH(`col_int_undef_signed`) BUCKETS 1 + PROPERTIES ( + 'replication_num' = '1' +); +""" + +sql """ +insert into clearblocktable2 values(1,1,'2020-01-01'); +""" + +sql """ +set enable_pipeline_x_engine=true, enable_pipeline_engine=true; +""" +qt_select_pipelineX """ + +SELECT YEAR(ifnull(clearblocktable1.`col_date_undef_signed_not_null`, clearblocktable1.`col_date_undef_signed_not_null`)) AS field1 , +CASE WHEN clearblocktable1.`col_int_undef_signed` != clearblocktable1.`col_int_undef_signed` * (8 + 1) THEN -5.2 ELSE clearblocktable1.`col_int_undef_signed` END AS field2 +FROM clearblocktable1 INNER JOIN clearblocktable2 ON clearblocktable2.`col_int_undef_signed` = clearblocktable1.`col_int_undef_signed` WHERE clearblocktable1.`col_int_undef_signed_not_null` <> 7; + +""" + +sql """ +set enable_pipeline_x_engine=false,enable_pipeline_engine=true; +""" +qt_select_pipeline """ + +SELECT YEAR(ifnull(clearblocktable1.`col_date_undef_signed_not_null`, clearblocktable1.`col_date_undef_signed_not_null`)) AS field1 , +CASE WHEN clearblocktable1.`col_int_undef_signed` != clearblocktable1.`col_int_undef_signed` * (8 + 1) THEN -5.2 ELSE clearblocktable1.`col_int_undef_signed` END AS field2 +FROM clearblocktable1 INNER JOIN clearblocktable2 ON clearblocktable2.`col_int_undef_signed` = clearblocktable1.`col_int_undef_signed` WHERE clearblocktable1.`col_int_undef_signed_not_null` <> 7; + +""" + +sql """ +set enable_pipeline_x_engine=false, enable_pipeline_engine=false; +""" +qt_select_non_pipeline """ + +SELECT YEAR(ifnull(clearblocktable1.`col_date_undef_signed_not_null`, clearblocktable1.`col_date_undef_signed_not_null`)) AS field1 , +CASE WHEN clearblocktable1.`col_int_undef_signed` != clearblocktable1.`col_int_undef_signed` * (8 + 1) THEN -5.2 ELSE clearblocktable1.`col_int_undef_signed` END AS field2 +FROM clearblocktable1 INNER JOIN clearblocktable2 ON clearblocktable2.`col_int_undef_signed` = clearblocktable1.`col_int_undef_signed` WHERE clearblocktable1.`col_int_undef_signed_not_null` <> 7; +""" +} diff --git a/regression-test/suites/external_table_p2/hive/test_external_credit_data.groovy b/regression-test/suites/external_table_p2/hive/test_external_credit_data.groovy index 96478e45ecb108..00f379c222878d 100644 --- a/regression-test/suites/external_table_p2/hive/test_external_credit_data.groovy +++ b/regression-test/suites/external_table_p2/hive/test_external_credit_data.groovy @@ -37,7 +37,7 @@ suite("test_external_credit_data", "p2,external,hive,external_remote,external_re // Table crdmm_data shows the credit card data after desensitization. // The query includes the conversion of string types to other types, // and the processing of materialized columns for nested subqueries. - qt_credit_data """ + order_qt_credit_data """ select substring(begin_time, 1, 7) as dt, case diff --git a/regression-test/suites/external_table_p2/hive/test_hive_to_array.groovy b/regression-test/suites/external_table_p2/hive/test_hive_to_array.groovy index 013d5f1cd2c0db..b03a83a158e801 100644 --- a/regression-test/suites/external_table_p2/hive/test_hive_to_array.groovy +++ b/regression-test/suites/external_table_p2/hive/test_hive_to_array.groovy @@ -35,11 +35,11 @@ suite("test_hive_to_array", "p2,external,hive,external_remote,external_remote_hi sql """ use multi_catalog """ - qt_types """ select * from hive_textfile_array_all_types""" + order_qt_types """ select * from hive_textfile_array_all_types""" - qt_array """ select * from hive_textfile_nestedarray""" + order_qt_array """ select * from hive_textfile_nestedarray""" - qt_delimiter """ select * from hive_textfile_array_delimiter""" + order_qt_delimiter """ select * from hive_textfile_array_delimiter""" } } diff --git a/regression-test/suites/insert_overwrite_p0/insert_overwrite_auto_detect.groovy b/regression-test/suites/insert_overwrite_p0/insert_overwrite_auto_detect.groovy index 367aaa9d536ec5..4cf5f28e15333f 100644 --- a/regression-test/suites/insert_overwrite_p0/insert_overwrite_auto_detect.groovy +++ b/regression-test/suites/insert_overwrite_p0/insert_overwrite_auto_detect.groovy @@ -16,10 +16,12 @@ // under the License. suite("test_iot_auto_detect") { + // only nereids now sql """set enable_nereids_planner = true""" sql """set enable_fallback_to_original_planner = false""" sql """set enable_nereids_dml = true""" + // range sql " drop table if exists range1; " sql """ create table range1( @@ -46,6 +48,7 @@ suite("test_iot_auto_detect") { sql " insert overwrite table range1 partition(*) values (-100), (-100), (333), (444), (555); " qt_sql " select * from range1 order by k0; " + // list sql " drop table if exists list1; " sql """ create table list1( @@ -72,6 +75,50 @@ suite("test_iot_auto_detect") { qt_sql " select * from list1 order by k0; " sql """ insert overwrite table list1 partition(*) values ("BEIJING"), ("SHANGHAI"), ("XXX"), ("LIST"), ("7654321"); """ qt_sql " select * from list1 order by k0; " + + // with label - transactions + sql """ insert overwrite table list1 partition(*) with label `txn1` values ("BEIJING"), ("7654321"); """ + sql """ insert overwrite table list1 partition(*) with label `txn2` values ("SHANGHAI"), ("LIST"); """ + sql """ insert overwrite table list1 partition(*) with label `txn3` values ("XXX"); """ + + def max_try_milli_secs = 10000 + while(max_try_milli_secs) { + def result = sql " show load where label like 'txn_' " + if(result[0][2] == "FINISHED" && result[1][2] == "FINISHED" && result[2][2] == "FINISHED" ) { + break + } else { + sleep(1000) // wait 1 second every time + max_try_milli_secs -= 1000 + if(max_try_milli_secs <= 0) { + log.info("result: ${result[0][2]}, ${result[1][2]}, ${result[2][2]}") + fail() + } + } + } + + qt_sql " select * from list1 order by k0; " + + // long partition value + sql " drop table if exists list_long; " + sql """ + create table list_long( + k0 varchar null + ) + partition by list (k0) + ( + PARTITION p1 values in (("Beijing"), ("aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa")), + PARTITION p2 values in (("nonono")) + ) + DISTRIBUTED BY HASH(`k0`) BUCKETS 1 + properties("replication_num" = "1"); + """ + sql """ insert into list_long values ("aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa"); """ + sql """ insert overwrite table list_long partition(*) values ("Beijing"); """ + qt_sql " select * from list_long order by k0; " + sql """ insert overwrite table list_long partition(*) values ("aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa"); """ + qt_sql " select * from list_long order by k0; " + + // miss partitions try { sql """ insert overwrite table list1 partition(*) values ("BEIJING"), ("invalid"); """ } catch (Exception e) { diff --git a/regression-test/suites/insert_overwrite_p1/ddl/test_iot_auto_detect_concurrent.sql b/regression-test/suites/insert_overwrite_p1/ddl/test_iot_auto_detect_concurrent.sql new file mode 100644 index 00000000000000..02c3b9037b401e --- /dev/null +++ b/regression-test/suites/insert_overwrite_p1/ddl/test_iot_auto_detect_concurrent.sql @@ -0,0 +1,108 @@ +create table test_concurrent_write( + k0 int null +) +partition by range (k0) +( + PARTITION p10 values less than (10), + PARTITION p20 values less than (20), + PARTITION p30 values less than (30), + PARTITION p40 values less than (40), + PARTITION p50 values less than (50), + PARTITION p60 values less than (60), + PARTITION p70 values less than (70), + PARTITION p80 values less than (80), + PARTITION p90 values less than (90), + PARTITION p100 values less than (100), + PARTITION p110 values less than (110), + PARTITION p120 values less than (120), + PARTITION p130 values less than (130), + PARTITION p140 values less than (140), + PARTITION p150 values less than (150), + PARTITION p160 values less than (160), + PARTITION p170 values less than (170), + PARTITION p180 values less than (180), + PARTITION p190 values less than (190), + PARTITION p200 values less than (200), + PARTITION p210 values less than (210), + PARTITION p220 values less than (220), + PARTITION p230 values less than (230), + PARTITION p240 values less than (240), + PARTITION p250 values less than (250), + PARTITION p260 values less than (260), + PARTITION p270 values less than (270), + PARTITION p280 values less than (280), + PARTITION p290 values less than (290), + PARTITION p300 values less than (300), + PARTITION p310 values less than (310), + PARTITION p320 values less than (320), + PARTITION p330 values less than (330), + PARTITION p340 values less than (340), + PARTITION p350 values less than (350), + PARTITION p360 values less than (360), + PARTITION p370 values less than (370), + PARTITION p380 values less than (380), + PARTITION p390 values less than (390), + PARTITION p400 values less than (400), + PARTITION p410 values less than (410), + PARTITION p420 values less than (420), + PARTITION p430 values less than (430), + PARTITION p440 values less than (440), + PARTITION p450 values less than (450), + PARTITION p460 values less than (460), + PARTITION p470 values less than (470), + PARTITION p480 values less than (480), + PARTITION p490 values less than (490), + PARTITION p500 values less than (500), + PARTITION p510 values less than (510), + PARTITION p520 values less than (520), + PARTITION p530 values less than (530), + PARTITION p540 values less than (540), + PARTITION p550 values less than (550), + PARTITION p560 values less than (560), + PARTITION p570 values less than (570), + PARTITION p580 values less than (580), + PARTITION p590 values less than (590), + PARTITION p600 values less than (600), + PARTITION p610 values less than (610), + PARTITION p620 values less than (620), + PARTITION p630 values less than (630), + PARTITION p640 values less than (640), + PARTITION p650 values less than (650), + PARTITION p660 values less than (660), + PARTITION p670 values less than (670), + PARTITION p680 values less than (680), + PARTITION p690 values less than (690), + PARTITION p700 values less than (700), + PARTITION p710 values less than (710), + PARTITION p720 values less than (720), + PARTITION p730 values less than (730), + PARTITION p740 values less than (740), + PARTITION p750 values less than (750), + PARTITION p760 values less than (760), + PARTITION p770 values less than (770), + PARTITION p780 values less than (780), + PARTITION p790 values less than (790), + PARTITION p800 values less than (800), + PARTITION p810 values less than (810), + PARTITION p820 values less than (820), + PARTITION p830 values less than (830), + PARTITION p840 values less than (840), + PARTITION p850 values less than (850), + PARTITION p860 values less than (860), + PARTITION p870 values less than (870), + PARTITION p880 values less than (880), + PARTITION p890 values less than (890), + PARTITION p900 values less than (900), + PARTITION p910 values less than (910), + PARTITION p920 values less than (920), + PARTITION p930 values less than (930), + PARTITION p940 values less than (940), + PARTITION p950 values less than (950), + PARTITION p960 values less than (960), + PARTITION p970 values less than (970), + PARTITION p980 values less than (980), + PARTITION p990 values less than (990), + PARTITION p1000 values less than (1000) +) +DISTRIBUTED BY HASH(`k0`) BUCKETS 1 +properties("replication_num" = "1"); diff --git a/regression-test/suites/insert_overwrite_p1/test_iot_auto_detect_concurrent.groovy b/regression-test/suites/insert_overwrite_p1/test_iot_auto_detect_concurrent.groovy new file mode 100644 index 00000000000000..200dd874df9540 --- /dev/null +++ b/regression-test/suites/insert_overwrite_p1/test_iot_auto_detect_concurrent.groovy @@ -0,0 +1,124 @@ +// 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_iot_auto_detect_concurrent") { + // only nereids now + sql """set enable_nereids_planner = true""" + sql """set enable_fallback_to_original_planner = false""" + sql """set enable_nereids_dml = true""" + + def db_name = "test_iot_auto_detect_concurrent" + def table_name = "test_concurrent_write" + + sql " create database if not exists test_iot_auto_detect_concurrent; " + sql " use test_iot_auto_detect_concurrent; " + sql " drop table if exists test_concurrent_write; " + sql new File("""${context.file.parent}/ddl/test_iot_auto_detect_concurrent.sql""").text + + def success_status = true + def load_data = { range, offset, expect_success -> + try { + sql " use test_iot_auto_detect_concurrent; " + sql """set enable_nereids_planner = true""" + sql """set enable_fallback_to_original_planner = false""" + sql """set enable_nereids_dml = true""" + sql """ insert overwrite table test_concurrent_write partition(*) + select number*10+${offset} from numbers("number" = "${range}"); + """ + } catch (Exception e) { + if (expect_success) { + success_status = false + log.info("fails one") + } + log.info("successfully catch the failed insert") + return + } + if (!expect_success) { + success_status = false + } + } + + def dropping = true + def drop_partition = { + sql " use test_iot_auto_detect_concurrent; " + while (dropping) { + try { + sql """ alter table test_concurrent_write + drop partition p10, drop partition p20, drop partition p30, drop partition p40, drop partition p50, + drop partition p60, drop partition p70, drop partition p80, drop partition p90, drop partition p100; + """ + } catch (Exception e) {} + } + } + + def result + + + /// same data and partitions + success_status = true + sql """ insert into test_concurrent_write select * from numbers("number" = "1000"); """ + def thread1 = Thread.start { load_data(100, 0, false) } + def thread2 = Thread.start { load_data(100, 0, false) } + def thread3 = Thread.start { load_data(100, 0, false) } + def thread4 = Thread.start { load_data(100, 0, false) } + def thread5 = Thread.start { load_data(100, 0, false) } + thread1.join() + thread2.join() + thread3.join() + thread4.join() + thread5.join() + // suppose result: success zero or one + if (success_status) { // success zero + result = sql " select count(k0) from test_concurrent_write; " + assertEquals(result[0][0], 1000) + result = sql " select count(distinct k0) from test_concurrent_write; " + assertEquals(result[0][0], 1000) + } else { // success one + result = sql " select count(k0) from test_concurrent_write; " + assertEquals(result[0][0], 100) + result = sql " select count(distinct k0) from test_concurrent_write; " + assertEquals(result[0][0], 100) + } + + + /// not same data/partitions + success_status = true + sql """ insert overwrite table test_concurrent_write select * from numbers("number" = "1000"); """ + def thread6 = Thread.start { load_data(50, 0, true) } // 0, 10 ... 490 + def thread7 = Thread.start { load_data(50, 500, true) } // 500, 10 ... 990 + thread6.join() + thread7.join() + // suppose result: Success to overwrite with a multiple of ten values + assertTrue(success_status) + qt_sql3 " select count(k0) from test_concurrent_write; " + qt_sql4 " select count(distinct k0) from test_concurrent_write; " + + + /// with drop partition concurrently + success_status = true + sql """ truncate table test_concurrent_write; """ + def thread10 = Thread.start { drop_partition() } + def thread8 = Thread.start { load_data(100, 0, false) } + def thread9 = Thread.start { load_data(100, 0, false) } + thread8.join() + thread9.join() + dropping = false // stop dropping + thread10.join() + // no success insert occur + assertTrue(success_status) // we concerned about this. no + qt_sql5 " select count(k0) from test_concurrent_write; " +} \ No newline at end of file diff --git a/regression-test/suites/mtmv_p0/test_null_partition_mtmv.groovy b/regression-test/suites/mtmv_p0/test_null_partition_mtmv.groovy new file mode 100644 index 00000000000000..a6a7890180ecc2 --- /dev/null +++ b/regression-test/suites/mtmv_p0/test_null_partition_mtmv.groovy @@ -0,0 +1,169 @@ +// 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_null_partition_mtmv") { + def tableName = "t_test_null_partition_mtmv_user" + def mvName = "t_test_null_partition_mtmv" + def dbName = "regression_test_mtmv_p0" + sql """drop table if exists `${tableName}`""" + sql """drop materialized view if exists ${mvName};""" + + // list table + sql """ + CREATE TABLE `${tableName}` ( + `user_id` LARGEINT NOT NULL COMMENT '\"用户id\"', + `num` SMALLINT COMMENT '\"数量\"' + ) ENGINE=OLAP + DUPLICATE KEY(`user_id`, `num`) + COMMENT 'OLAP' + PARTITION BY list(`num`) + ( + PARTITION p_1 VALUES IN (1), + PARTITION p_null VALUES IN (null) + ) + DISTRIBUTED BY HASH(`user_id`) BUCKETS 2 + PROPERTIES ('replication_num' = '1') ; + """ + sql """ + insert into ${tableName} values(1,1),(2,null); + """ + + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD DEFERRED REFRESH AUTO ON MANUAL + partition by(`num`) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS + SELECT * FROM ${tableName}; + """ + def showPartitionsResult = sql """show partitions from ${mvName}""" + logger.info("showPartitionsResult: " + showPartitionsResult.toString()) + assertTrue(showPartitionsResult.toString().contains("p_1")) + assertTrue(showPartitionsResult.toString().contains("p_NULL")) + + sql """ + REFRESH MATERIALIZED VIEW ${mvName} AUTO + """ + def jobName = getJobName(dbName, mvName); + log.info(jobName) + waitingMTMVTaskFinished(jobName) + + order_qt_list_null "SELECT * FROM ${mvName} partitions(p_NULL) order by user_id,num" + order_qt_list_1 "SELECT * FROM ${mvName} partitions(p_1) order by user_id,num" + + sql """drop table if exists `${tableName}`""" + sql """drop materialized view if exists ${mvName};""" + + + // range table + sql """ + CREATE TABLE `${tableName}` ( + `user_id` LARGEINT NOT NULL COMMENT '\"用户id\"', + `num` SMALLINT COMMENT '\"数量\"' + ) ENGINE=OLAP + DUPLICATE KEY(`user_id`, `num`) + COMMENT 'OLAP' + PARTITION BY range(`num`) + ( + PARTITION p_10 VALUES LESS THAN (10), + PARTITION p_20 VALUES LESS THAN (20) + ) + DISTRIBUTED BY HASH(`user_id`) BUCKETS 2 + PROPERTIES ('replication_num' = '1') ; + """ + sql """ + insert into ${tableName} values(1,null),(2,15); + """ + + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD DEFERRED REFRESH AUTO ON MANUAL + partition by(`num`) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS + SELECT * FROM ${tableName}; + """ + showPartitionsResult = sql """show partitions from ${mvName}""" + logger.info("showPartitionsResult: " + showPartitionsResult.toString()) + assertTrue(showPartitionsResult.toString().contains("p_32768_10")) + assertTrue(showPartitionsResult.toString().contains("p_10_20")) + + sql """ + REFRESH MATERIALIZED VIEW ${mvName} AUTO + """ + jobName = getJobName(dbName, mvName); + log.info(jobName) + waitingMTMVTaskFinished(jobName) + + order_qt_range_p_32768_10 "SELECT * FROM ${mvName} partitions(p_32768_10) order by user_id,num" + order_qt_range_p_10_20 "SELECT * FROM ${mvName} partitions(p_10_20) order by user_id,num" + + sql """drop table if exists `${tableName}`""" + sql """drop materialized view if exists ${mvName};""" + + // range table + sql """ + CREATE TABLE `${tableName}` ( + `user_id` LARGEINT NOT NULL COMMENT '\"用户id\"', + `create_date` DATE + ) ENGINE=OLAP + DUPLICATE KEY(`user_id`, `create_date`) + COMMENT 'OLAP' + PARTITION BY range(`create_date`) + ( + PARTITION p_10 VALUES LESS THAN ("2020-11-11"), + PARTITION p_20 VALUES LESS THAN ("2021-11-11") + ) + DISTRIBUTED BY HASH(`user_id`) BUCKETS 2 + PROPERTIES ('replication_num' = '1') ; + """ + sql """ + insert into ${tableName} values(1,null),(2,"2021-01-01"); + """ + + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD DEFERRED REFRESH AUTO ON MANUAL + partition by(`create_date`) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS + SELECT * FROM ${tableName}; + """ + showPartitionsResult = sql """show partitions from ${mvName}""" + logger.info("showPartitionsResult: " + showPartitionsResult.toString()) + assertTrue(showPartitionsResult.toString().contains("p_00000101_20201111")) + assertTrue(showPartitionsResult.toString().contains("p_20201111_20211111")) + + sql """ + REFRESH MATERIALIZED VIEW ${mvName} AUTO + """ + jobName = getJobName(dbName, mvName); + log.info(jobName) + waitingMTMVTaskFinished(jobName) + + order_qt_range_p_00000101_20201111 "SELECT * FROM ${mvName} partitions(p_00000101_20201111) order by user_id,create_date" + order_qt_range_p_20201111_20211111 "SELECT * FROM ${mvName} partitions(p_20201111_20211111) order by user_id,create_date" + + sql """drop table if exists `${tableName}`""" + sql """drop materialized view if exists ${mvName};""" + +} diff --git a/regression-test/suites/nereids_function_p0/agg_function/agg.groovy b/regression-test/suites/nereids_function_p0/agg_function/agg.groovy index 81c84ad32d8f7e..087ca2cf9f1ca6 100644 --- a/regression-test/suites/nereids_function_p0/agg_function/agg.groovy +++ b/regression-test/suites/nereids_function_p0/agg_function/agg.groovy @@ -2269,6 +2269,191 @@ suite("nereids_agg_fn") { qt_sql_sum_LargeInt_agg_phase_4_notnull ''' select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), sum(klint) from fn_test''' + qt_sql_sum0_Boolean ''' + select sum0(kbool) from fn_test''' + qt_sql_sum0_Boolean_gb ''' + select sum0(kbool) from fn_test group by id order by id''' + qt_sql_sum0_TinyInt_gb ''' + select sum0(ktint) from fn_test group by kbool order by kbool''' + qt_sql_sum0_TinyInt ''' + select sum0(ktint) from fn_test''' + qt_sql_sum0_TinyInt_agg_phase_1 ''' + select count(id), sum0(ktint) from fn_test group by id order by id''' + qt_sql_sum0_TinyInt_agg_phase_2 ''' + select count(distinct id), sum0(ktint) from fn_test''' + qt_sql_sum0_TinyInt_agg_phase_3 ''' + select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), sum0(ktint) from fn_test group by kbool order by kbool''' + qt_sql_sum0_TinyInt_agg_phase_4 ''' + select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), sum0(ktint) from fn_test''' + qt_sql_sum0_TinyInt_gb_notnull ''' + select sum0(ktint) from fn_test_not_nullable group by kbool order by kbool''' + qt_sql_sum0_TinyInt_notnull ''' + select sum0(ktint) from fn_test_not_nullable''' + qt_sql_sum0_TinyInt_agg_phase_1_notnull ''' + select count(id), sum0(ktint) from fn_test_not_nullable group by id order by id''' + qt_sql_sum0_TinyInt_agg_phase_2_notnull ''' + select count(distinct id), sum0(ktint) from fn_test_not_nullable''' + qt_sql_sum0_TinyInt_agg_phase_3_notnull ''' + select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), sum0(ktint) from fn_test group by kbool order by kbool''' + qt_sql_sum0_TinyInt_agg_phase_4_notnull ''' + select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), sum0(ktint) from fn_test''' + + qt_sql_sum0_SmallInt_gb ''' + select sum0(ksint) from fn_test group by kbool order by kbool''' + qt_sql_sum0_SmallInt ''' + select sum0(ksint) from fn_test''' + qt_sql_sum0_SmallInt_agg_phase_1 ''' + select count(id), sum0(ksint) from fn_test group by id order by id''' + qt_sql_sum0_SmallInt_agg_phase_2 ''' + select count(distinct id), sum0(ksint) from fn_test''' + qt_sql_sum0_SmallInt_agg_phase_3 ''' + select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), sum0(ksint) from fn_test group by kbool order by kbool''' + qt_sql_sum0_SmallInt_agg_phase_4 ''' + select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), sum0(ksint) from fn_test''' + qt_sql_sum0_SmallInt_gb_notnull ''' + select sum0(ksint) from fn_test_not_nullable group by kbool order by kbool''' + qt_sql_sum0_SmallInt_notnull ''' + select sum0(ksint) from fn_test_not_nullable''' + qt_sql_sum0_SmallInt_agg_phase_1_notnull ''' + select count(id), sum0(ksint) from fn_test_not_nullable group by id order by id''' + qt_sql_sum0_SmallInt_agg_phase_2_notnull ''' + select count(distinct id), sum0(ksint) from fn_test_not_nullable''' + qt_sql_sum0_SmallInt_agg_phase_3_notnull ''' + select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), sum0(ksint) from fn_test group by kbool order by kbool''' + qt_sql_sum0_SmallInt_agg_phase_4_notnull ''' + select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), sum0(ksint) from fn_test''' + + qt_sql_sum0_Integer_gb ''' + select sum0(kint) from fn_test group by kbool order by kbool''' + qt_sql_sum0_Integer ''' + select sum0(kint) from fn_test''' + qt_sql_sum0_Integer_agg_phase_1 ''' + select count(id), sum0(kint) from fn_test group by id order by id''' + qt_sql_sum0_Integer_agg_phase_2 ''' + select count(distinct id), sum0(kint) from fn_test''' + qt_sql_sum0_Integer_agg_phase_3 ''' + select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), sum0(kint) from fn_test group by kbool order by kbool''' + qt_sql_sum0_Integer_agg_phase_4 ''' + select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), sum0(kint) from fn_test''' + qt_sql_sum0_Integer_gb_notnull ''' + select sum0(kint) from fn_test_not_nullable group by kbool order by kbool''' + qt_sql_sum0_Integer_notnull ''' + select sum0(kint) from fn_test_not_nullable''' + qt_sql_sum0_Integer_agg_phase_1_notnull ''' + select count(id), sum0(kint) from fn_test_not_nullable group by id order by id''' + qt_sql_sum0_Integer_agg_phase_2_notnull ''' + select count(distinct id), sum0(kint) from fn_test_not_nullable''' + qt_sql_sum0_Integer_agg_phase_3_notnull ''' + select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), sum0(kint) from fn_test group by kbool order by kbool''' + qt_sql_sum0_Integer_agg_phase_4_notnull ''' + select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), sum0(kint) from fn_test''' + + qt_sql_sum0_BigInt_gb ''' + select sum0(kbint) from fn_test group by kbool order by kbool''' + qt_sql_sum0_BigInt ''' + select sum0(kbint) from fn_test''' + qt_sql_sum0_BigInt_agg_phase_1 ''' + select count(id), sum0(kbint) from fn_test group by id order by id''' + qt_sql_sum0_BigInt_agg_phase_2 ''' + select count(distinct id), sum0(kbint) from fn_test''' + qt_sql_sum0_BigInt_agg_phase_3 ''' + select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), sum0(kbint) from fn_test group by kbool order by kbool''' + qt_sql_sum0_BigInt_agg_phase_4 ''' + select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), sum0(kbint) from fn_test''' + qt_sql_sum0_BigInt_gb_notnull ''' + select sum0(kbint) from fn_test_not_nullable group by kbool order by kbool''' + qt_sql_sum0_BigInt_notnull ''' + select sum0(kbint) from fn_test_not_nullable''' + qt_sql_sum0_BigInt_agg_phase_1_notnull ''' + select count(id), sum0(kbint) from fn_test_not_nullable group by id order by id''' + qt_sql_sum0_BigInt_agg_phase_2_notnull ''' + select count(distinct id), sum0(kbint) from fn_test_not_nullable''' + qt_sql_sum0_BigInt_agg_phase_3_notnull ''' + select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), sum0(kbint) from fn_test group by kbool order by kbool''' + qt_sql_sum0_BigInt_agg_phase_4_notnull ''' + select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), sum0(kbint) from fn_test''' + + //not cast float to double + explain { + sql("select sum0(kfloat) from fn_test;") + contains "partial_sum0(kfloat" + } + + qt_sql_sum0_Double_gb ''' + select sum0(kdbl) from fn_test group by kbool order by kbool''' + qt_sql_sum0_Double ''' + select sum0(kdbl) from fn_test''' + qt_sql_sum0_Double_agg_phase_1 ''' + select count(id), sum0(kdbl) from fn_test group by id order by id''' + qt_sql_sum0_Double_agg_phase_2 ''' + select count(distinct id), sum0(kdbl) from fn_test''' + qt_sql_sum0_Double_agg_phase_3 ''' + select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), sum0(kdbl) from fn_test group by kbool order by kbool''' + qt_sql_sum0_Double_agg_phase_4 ''' + select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), sum0(kdbl) from fn_test''' + qt_sql_sum0_Double_gb_notnull ''' + select sum0(kdbl) from fn_test_not_nullable group by kbool order by kbool''' + qt_sql_sum0_Double_notnull ''' + select sum0(kdbl) from fn_test_not_nullable''' + qt_sql_sum0_Double_agg_phase_1_notnull ''' + select count(id), sum0(kdbl) from fn_test_not_nullable group by id order by id''' + qt_sql_sum0_Double_agg_phase_2_notnull ''' + select count(distinct id), sum0(kdbl) from fn_test_not_nullable''' + qt_sql_sum0_Double_agg_phase_3_notnull ''' + select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), sum0(kdbl) from fn_test group by kbool order by kbool''' + qt_sql_sum0_Double_agg_phase_4_notnull ''' + select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), sum0(kdbl) from fn_test''' + + qt_sql_sum0_DecimalV2_gb ''' + select sum0(kdcmls1) from fn_test group by kbool order by kbool''' + qt_sql_sum0_DecimalV2 ''' + select sum0(kdcmls1) from fn_test''' + qt_sql_sum0_DecimalV2_agg_phase_1 ''' + select count(id), sum0(kdcmls1) from fn_test group by id order by id''' + qt_sql_sum0_DecimalV2_agg_phase_2 ''' + select count(distinct id), sum0(kdcmls1) from fn_test''' + qt_sql_sum0_DecimalV2_agg_phase_3 ''' + select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), sum0(kdcmls1) from fn_test group by kbool order by kbool''' + qt_sql_sum0_DecimalV2_agg_phase_4 ''' + select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), sum0(kdcmls1) from fn_test''' + qt_sql_sum0_DecimalV2_gb_notnull ''' + select sum0(kdcmls1) from fn_test_not_nullable group by kbool order by kbool''' + qt_sql_sum0_DecimalV2_notnull ''' + select sum0(kdcmls1) from fn_test_not_nullable''' + qt_sql_sum0_DecimalV2_agg_phase_1_notnull ''' + select count(id), sum0(kdcmls1) from fn_test_not_nullable group by id order by id''' + qt_sql_sum0_DecimalV2_agg_phase_2_notnull ''' + select count(distinct id), sum0(kdcmls1) from fn_test_not_nullable''' + qt_sql_sum0_DecimalV2_agg_phase_3_notnull ''' + select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), sum0(kdcmls1) from fn_test group by kbool order by kbool''' + qt_sql_sum0_DecimalV2_agg_phase_4_notnull ''' + select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), sum0(kdcmls1) from fn_test''' + + qt_sql_sum0_LargeInt_gb ''' + select sum0(klint) from fn_test group by kbool order by kbool''' + qt_sql_sum0_LargeInt ''' + select sum0(klint) from fn_test''' + qt_sql_sum0_LargeInt_agg_phase_1 ''' + select count(id), sum0(klint) from fn_test group by id order by id''' + qt_sql_sum0_LargeInt_agg_phase_2 ''' + select count(distinct id), sum0(klint) from fn_test''' + qt_sql_sum0_LargeInt_agg_phase_3 ''' + select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), sum0(klint) from fn_test group by kbool order by kbool''' + qt_sql_sum0_LargeInt_agg_phase_4 ''' + select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), sum0(klint) from fn_test''' + qt_sql_sum0_LargeInt_gb_notnull ''' + select sum0(klint) from fn_test_not_nullable group by kbool order by kbool''' + qt_sql_sum0_LargeInt_notnull ''' + select sum0(klint) from fn_test_not_nullable''' + qt_sql_sum0_LargeInt_agg_phase_1_notnull ''' + select count(id), sum0(klint) from fn_test_not_nullable group by id order by id''' + qt_sql_sum0_LargeInt_agg_phase_2_notnull ''' + select count(distinct id), sum0(klint) from fn_test_not_nullable''' + qt_sql_sum0_LargeInt_agg_phase_3_notnull ''' + select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), sum0(klint) from fn_test group by kbool order by kbool''' + qt_sql_sum0_LargeInt_agg_phase_4_notnull ''' + select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), sum0(klint) from fn_test''' + qt_sql_topn_Varchar_Integer_gb ''' select topn(kvchrs1, 3) from fn_test group by kbool order by kbool''' qt_sql_topn_Varchar_Integer ''' diff --git a/regression-test/suites/nereids_p0/aggregate/aggregate_groupby_constant.groovy b/regression-test/suites/nereids_p0/aggregate/aggregate_groupby_constant.groovy new file mode 100644 index 00000000000000..ffd6c9b382e2e3 --- /dev/null +++ b/regression-test/suites/nereids_p0/aggregate/aggregate_groupby_constant.groovy @@ -0,0 +1,50 @@ +// 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("aggregate_groupby_constant") { + sql "SET enable_nereids_planner=true" + sql "SET enable_fallback_to_original_planner=false" + + sql """ DROP TABLE IF EXISTS table_500_undef_partitions2_keys3_properties4_distributed_by5; """ + sql """ + create table table_500_undef_partitions2_keys3_properties4_distributed_by5 ( + col_date_undef_signed date , + pk int, + col_int_undef_signed int , + col_int_undef_signed2 int , + col_date_undef_signed2 date , + col_varchar_10__undef_signed varchar(10) , + col_varchar_1024__undef_signed varchar(1024) + ) engine=olap + DUPLICATE KEY(col_date_undef_signed, pk, col_int_undef_signed) + PARTITION BY RANGE(col_date_undef_signed) ( FROM ('2023-12-09') TO ('2023-12-12') INTERVAL 1 DAY ) + distributed by hash(pk) buckets 10 + properties("replication_num" = "1"); + """ + sql """SELECT table1 . col_int_undef_signed2 AS field1, + ( TO_DATE (CASE + WHEN ( '2024-01-08' < '2024-02-18' ) THEN + '2023-12-19' + WHEN ( table1 . `col_date_undef_signed2` < DATE_ADD ( table1 . `col_date_undef_signed` , INTERVAL 6 DAY ) ) THEN + '2026-02-18' + ELSE DATE_ADD( table1 . `col_date_undef_signed` , INTERVAL 365 DAY ) END)) AS field2, MAX( DISTINCT table1 . `col_varchar_10__undef_signed` ) AS field3 + FROM table_500_undef_partitions2_keys3_properties4_distributed_by5 AS table1 + WHERE ( ( table1 . `col_date_undef_signed` is NOT NULL ) + OR table1 . `col_int_undef_signed` <> NULL ) + GROUP BY field1,field2 + ORDER BY field1,field2 LIMIT 10000;""" +} 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 2b81dfd8aead29..3a60f5e90e4159 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 @@ -135,6 +135,19 @@ suite("partition_mv_rewrite_dimension_1") { """ } + def create_mv_all = { mv_name, mv_sql -> + sql """DROP MATERIALIZED VIEW IF EXISTS ${mv_name};""" + sql """DROP TABLE IF EXISTS ${mv_name}""" + sql""" + CREATE MATERIALIZED VIEW ${mv_name} + BUILD IMMEDIATE REFRESH AUTO ON MANUAL + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS + ${mv_sql} + """ + } + def compare_res = { def stmt -> sql "SET enable_materialized_view_rewrite=false" def origin_res = sql stmt @@ -155,9 +168,9 @@ suite("partition_mv_rewrite_dimension_1") { // join direction def mv_name_1 = "mv_join_1" def join_direction_mv_1 = """ - select l_shipdate, o_orderdate, l_partkey, l_suppkey - from lineitem_1 - left join orders_1 + select l_shipdate, o_orderdate, l_partkey, l_suppkey + from lineitem_1 + left join orders_1 on lineitem_1.l_orderkey = orders_1.o_orderkey """ @@ -166,15 +179,15 @@ suite("partition_mv_rewrite_dimension_1") { waitingMTMVTaskFinished(job_name_1) def join_direction_sql_1 = """ - select l_shipdate - from lineitem_1 - left join orders_1 + select l_shipdate + from lineitem_1 + left join orders_1 on lineitem_1.l_orderkey = orders_1.o_orderkey """ def join_direction_sql_2 = """ - select l_shipdate - from orders_1 - left join lineitem_1 + select l_shipdate + from orders_1 + left join lineitem_1 on orders_1.o_orderkey = lineitem_1.l_orderkey """ explain { @@ -191,9 +204,9 @@ suite("partition_mv_rewrite_dimension_1") { def mv_name_2 = "mv_join_2" def join_direction_mv_2 = """ - select l_shipdate, o_orderdate, l_partkey, l_suppkey - from lineitem_1 - inner join orders_1 + select l_shipdate, o_orderdate, l_partkey, l_suppkey + from lineitem_1 + inner join orders_1 on lineitem_1.l_orderkey = orders_1.o_orderkey """ @@ -202,15 +215,15 @@ suite("partition_mv_rewrite_dimension_1") { waitingMTMVTaskFinished(job_name_2) def join_direction_sql_3 = """ - select l_shipdate - from lineitem_1 - inner join orders_1 + select l_shipdate + from lineitem_1 + inner join orders_1 on lineitem_1.l_orderkey = orders_1.o_orderkey """ def join_direction_sql_4 = """ - select l_shipdate - from orders_1 - inner join lineitem_1 + select l_shipdate + from orders_1 + inner join lineitem_1 on orders_1.o_orderkey = lineitem_1.l_orderkey """ explain { @@ -227,50 +240,50 @@ suite("partition_mv_rewrite_dimension_1") { // join filter position def join_filter_stmt_1 = """ - select l_shipdate, o_orderdate, l_partkey, l_suppkey, o_orderkey - from lineitem_1 - left join orders_1 + select l_shipdate, o_orderdate, l_partkey, l_suppkey, o_orderkey + from lineitem_1 + left join orders_1 on lineitem_1.l_orderkey = orders_1.o_orderkey""" def join_filter_stmt_2 = """ - select l_shipdate, o_orderdate, l_partkey, l_suppkey, o_orderkey - from (select * from lineitem_1 where l_shipdate = '2023-10-17' ) t1 - left join orders_1 + select l_shipdate, o_orderdate, l_partkey, l_suppkey, o_orderkey + from (select * from lineitem_1 where l_shipdate = '2023-10-17' ) t1 + left join orders_1 on t1.l_orderkey = orders_1.o_orderkey""" def join_filter_stmt_3 = """ - select l_shipdate, o_orderdate, l_partkey, l_suppkey, o_orderkey - from lineitem_1 - left join (select * from orders_1 where o_orderdate = '2023-10-17' ) t2 + select l_shipdate, o_orderdate, l_partkey, l_suppkey, o_orderkey + from lineitem_1 + left join (select * from orders_1 where o_orderdate = '2023-10-17' ) t2 on lineitem_1.l_orderkey = t2.o_orderkey""" def join_filter_stmt_4 = """ - select l_shipdate, o_orderdate, l_partkey, l_suppkey, o_orderkey - from lineitem_1 - left join orders_1 - on lineitem_1.l_orderkey = orders_1.o_orderkey + select l_shipdate, o_orderdate, l_partkey, l_suppkey, o_orderkey + from lineitem_1 + left join orders_1 + on lineitem_1.l_orderkey = orders_1.o_orderkey where l_shipdate = '2023-10-17' and o_orderdate = '2023-10-17'""" def join_filter_stmt_5 = """ - select l_shipdate, o_orderdate, l_partkey, l_suppkey, o_orderkey - from lineitem_1 - left join orders_1 - on lineitem_1.l_orderkey = orders_1.o_orderkey + select l_shipdate, o_orderdate, l_partkey, l_suppkey, o_orderkey + from lineitem_1 + left join orders_1 + on lineitem_1.l_orderkey = orders_1.o_orderkey where l_shipdate = '2023-10-17'""" def join_filter_stmt_6 = """ - select l_shipdate, o_orderdate, l_partkey, l_suppkey, o_orderkey - from lineitem_1 - left join orders_1 - on lineitem_1.l_orderkey = orders_1.o_orderkey + select l_shipdate, o_orderdate, l_partkey, l_suppkey, o_orderkey + from lineitem_1 + left join orders_1 + on lineitem_1.l_orderkey = orders_1.o_orderkey where o_orderdate = '2023-10-17'""" def join_filter_stmt_7 = """ - select l_shipdate, o_orderdate, l_partkey, l_suppkey, o_orderkey - from lineitem_1 - left join orders_1 - on lineitem_1.l_orderkey = orders_1.o_orderkey + select l_shipdate, o_orderdate, l_partkey, l_suppkey, o_orderkey + from lineitem_1 + left join orders_1 + on lineitem_1.l_orderkey = orders_1.o_orderkey where orders_1.o_orderkey=1""" def mv_list = [ join_filter_stmt_1, join_filter_stmt_2, join_filter_stmt_3, join_filter_stmt_4, join_filter_stmt_5, join_filter_stmt_6, join_filter_stmt_7] - for (int i = 0; i < mv_list.size(); i++) { + for (int i =0; i < mv_list.size(); i++) { logger.info("i:" + i) def join_filter_mv = """join_filter_mv_${i}""" create_mv_lineitem(join_filter_mv, mv_list[i]) @@ -403,39 +416,49 @@ suite("partition_mv_rewrite_dimension_1") { from lineitem_1 inner join orders_1 on lineitem_1.l_orderkey = orders_1.o_orderkey""" - - // Todo: right/cross/full/semi/anti join - // Currently, only left join and inner join are supported. -// def join_type_stmt_3 = """ -// select l_shipdate, o_orderdate, l_partkey, l_suppkey -// from lineitem_1 -// right join orders_1 -// on lineitem_1.l_orderkey = orders_1.o_orderkey""" + def join_type_stmt_3 = """ + select l_shipdate, o_orderdate, l_partkey, l_suppkey + from lineitem_1 + right join orders_1 + on lineitem_1.l_orderkey = orders_1.o_orderkey""" // def join_type_stmt_4 = """ // select l_shipdate, o_orderdate, l_partkey, l_suppkey // from lineitem_1 // cross join orders_1""" -// def join_type_stmt_5 = """ -// select l_shipdate, o_orderdate, l_partkey, l_suppkey -// from lineitem_1 -// full join orders_1 -// on lineitem_1.l_orderkey = orders_1.o_orderkey""" -// def join_type_stmt_6 = """ -// select l_shipdate, o_orderdate, l_partkey, l_suppkey -// from lineitem_1 -// semi join orders_1 -// on lineitem_1.l_orderkey = orders_1.o_orderkey""" -// def join_type_stmt_7 = """ -// select l_shipdate, o_orderdate, l_partkey, l_suppkey -// from lineitem_1 -// anti join orders_1 -// on lineitem_1.l_orderkey = orders_1.o_orderkey""" - def join_type_stmt_list = [join_type_stmt_1, join_type_stmt_2] + def join_type_stmt_5 = """ + select l_shipdate, o_orderdate, l_partkey, l_suppkey + from lineitem_1 + full join orders_1 + on lineitem_1.l_orderkey = orders_1.o_orderkey""" + def join_type_stmt_6 = """ + select l_shipdate, l_partkey, l_suppkey, l_linenumber + from lineitem_1 + left semi join orders_1 + on lineitem_1.l_orderkey = orders_1.o_orderkey""" + def join_type_stmt_7 = """ + select o_orderdate, o_orderkey, o_custkey, o_orderstatus + from lineitem_1 + right semi join orders_1 + on lineitem_1.l_orderkey = orders_1.o_orderkey""" + def join_type_stmt_8 = """ + select l_shipdate, l_partkey, l_suppkey, l_linenumber + from lineitem_1 + left anti join orders_1 + on lineitem_1.l_orderkey = orders_1.o_orderkey""" + def join_type_stmt_9 = """ + select o_orderdate, o_orderkey, o_custkey, o_orderstatus + from lineitem_1 + right anti join orders_1 + on lineitem_1.l_orderkey = orders_1.o_orderkey""" + def join_type_stmt_list = [join_type_stmt_1, join_type_stmt_2, join_type_stmt_3, join_type_stmt_5, + join_type_stmt_6, join_type_stmt_7, join_type_stmt_8, join_type_stmt_9] for (int i = 0; i < join_type_stmt_list.size(); i++) { logger.info("i:" + i) String join_type_mv = """join_type_mv_${i}""" - if (i == 2) { + if (i == 2 || i == 5 || i == 7) { create_mv_orders(join_type_mv, join_type_stmt_list[i]) + } else if (i == 3) { + create_mv_all(join_type_mv, join_type_stmt_list[i]) } else { create_mv_lineitem(join_type_mv, join_type_stmt_list[i]) } @@ -465,30 +488,30 @@ suite("partition_mv_rewrite_dimension_1") { sql """DROP MATERIALIZED VIEW IF EXISTS ${agg_mv_name_1};""" sql """DROP TABLE IF EXISTS ${agg_mv_name_1}""" sql """ - CREATE MATERIALIZED VIEW ${agg_mv_name_1} - BUILD IMMEDIATE REFRESH AUTO ON MANUAL - DISTRIBUTED BY RANDOM BUCKETS 2 - PROPERTIES ('replication_num' = '1') - AS + CREATE MATERIALIZED VIEW ${agg_mv_name_1} + BUILD IMMEDIATE REFRESH AUTO ON MANUAL + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS select - 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 + 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 orders_1 """ def agg_job_name_1 = getJobName(db, agg_mv_name_1) waitingMTMVTaskFinished(agg_job_name_1) - def agg_sql_1 = """select - count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) as cnt_1, - count(distinct case when O_SHIPPRIORITY > 2 and o_orderkey IN (2) then o_custkey else null end) as cnt_2, - sum(o_totalprice), - max(o_totalprice), - min(o_totalprice), - count(*) + def agg_sql_1 = """select + count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) as cnt_1, + count(distinct case when O_SHIPPRIORITY > 2 and o_orderkey IN (2) then o_custkey else null end) as cnt_2, + sum(o_totalprice), + max(o_totalprice), + min(o_totalprice), + count(*) from orders_1 """ explain { @@ -501,23 +524,23 @@ suite("partition_mv_rewrite_dimension_1") { // agg + with group by + without agg function def agg_mv_name_2 = "agg_mv_name_2" def agg_mv_stmt_2 = """ - select o_orderdate, o_shippriority, o_comment - from orders_1 - group by - o_orderdate, - o_shippriority, - o_comment + select o_orderdate, o_shippriority, o_comment + from orders_1 + group by + o_orderdate, + o_shippriority, + o_comment """ create_mv_orders(agg_mv_name_2, agg_mv_stmt_2) def agg_job_name_2 = getJobName(db, agg_mv_name_2) waitingMTMVTaskFinished(agg_job_name_2) sql """analyze table ${agg_mv_name_2} with sync;""" - def agg_sql_2 = """select o_shippriority, o_comment - from orders_1 - group by - o_shippriority, - o_comment + def agg_sql_2 = """select o_shippriority, o_comment + from orders_1 + group by + o_shippriority, + o_comment """ def agg_sql_explain_2 = sql """explain ${agg_sql_2};""" def mv_index_1 = agg_sql_explain_2.toString().indexOf("MaterializedViewRewriteFail:") @@ -528,35 +551,35 @@ suite("partition_mv_rewrite_dimension_1") { // agg + with group by + with agg function def agg_mv_name_3 = "agg_mv_name_3" def agg_mv_stmt_3 = """ - select o_orderdate, o_shippriority, o_comment, - 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 orders_1 - group by - o_orderdate, - o_shippriority, - o_comment + select o_orderdate, o_shippriority, o_comment, + 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 orders_1 + group by + o_orderdate, + o_shippriority, + o_comment """ create_mv_orders(agg_mv_name_3, agg_mv_stmt_3) def agg_job_name_3 = getJobName(db, agg_mv_name_3) waitingMTMVTaskFinished(agg_job_name_3) sql """analyze table ${agg_mv_name_3} with sync;""" - def agg_sql_3 = """select o_shippriority, o_comment, + def agg_sql_3 = """select o_shippriority, o_comment, count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) as cnt_1, - count(distinct case when O_SHIPPRIORITY > 2 and o_orderkey IN (2) then o_custkey else null end) as cnt_2, - sum(o_totalprice), - max(o_totalprice), - min(o_totalprice), - count(*) - from orders_1 - group by - o_shippriority, - o_comment + count(distinct case when O_SHIPPRIORITY > 2 and o_orderkey IN (2) then o_custkey else null end) as cnt_2, + sum(o_totalprice), + max(o_totalprice), + min(o_totalprice), + count(*) + from orders_1 + group by + o_shippriority, + o_comment """ def agg_sql_explain_3 = sql """explain ${agg_sql_3};""" def mv_index_2 = agg_sql_explain_3.toString().indexOf("MaterializedViewRewriteFail:") @@ -686,4 +709,5 @@ suite("partition_mv_rewrite_dimension_1") { // contains "${rewriting_mv_name_1}(${rewriting_mv_name_1})" // } // sql """DROP MATERIALIZED VIEW IF EXISTS ${rewriting_mv_name_1};""" + } 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 new file mode 100644 index 00000000000000..82f3c0b0de8719 --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_full_join.groovy @@ -0,0 +1,441 @@ +// 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 suite is a two dimensional test case file. +It mainly tests the full join and filter positions. + */ +suite("partition_mv_rewrite_dimension_2_full_join") { + String db = context.config.getDbNameByFile(context.file) + sql "use ${db}" + sql "SET enable_nereids_planner=true" + sql "SET enable_fallback_to_original_planner=false" + sql "SET enable_materialized_view_rewrite=true" + sql "SET enable_nereids_timeout = false" + + sql """ + drop table if exists orders_2_full_join + """ + + sql """CREATE TABLE `orders_2_full_join` ( + `o_orderkey` BIGINT 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 not NULL + ) ENGINE=OLAP + DUPLICATE KEY(`o_orderkey`, `o_custkey`) + COMMENT 'OLAP' + AUTO PARTITION BY range date_trunc(`o_orderdate`, 'day') () + DISTRIBUTED BY HASH(`o_orderkey`) BUCKETS 96 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + );""" + + sql """ + drop table if exists lineitem_2_full_join + """ + + sql """CREATE TABLE `lineitem_2_full_join` ( + `l_orderkey` BIGINT 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 not NULL + ) ENGINE=OLAP + DUPLICATE KEY(l_orderkey, l_linenumber, l_partkey, l_suppkey ) + COMMENT 'OLAP' + AUTO PARTITION BY range date_trunc(`l_shipdate`, 'day') () + DISTRIBUTED BY HASH(`l_orderkey`) BUCKETS 96 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + );""" + + sql """ + insert into orders_2_full_join values + (null, 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_2_full_join 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'), + (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'); + """ + + sql """analyze table orders_2_full_join with sync;""" + sql """analyze table lineitem_2_full_join with sync;""" + + def create_mv_lineitem = { mv_name, mv_sql -> + sql """DROP MATERIALIZED VIEW IF EXISTS ${mv_name};""" + sql """DROP TABLE IF EXISTS ${mv_name}""" + sql""" + CREATE MATERIALIZED VIEW ${mv_name} + BUILD IMMEDIATE REFRESH AUTO ON MANUAL + partition by(l_shipdate) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS + ${mv_sql} + """ + } + + def create_mv_orders = { mv_name, mv_sql -> + sql """DROP MATERIALIZED VIEW IF EXISTS ${mv_name};""" + sql """DROP TABLE IF EXISTS ${mv_name}""" + sql""" + CREATE MATERIALIZED VIEW ${mv_name} + BUILD IMMEDIATE REFRESH AUTO ON MANUAL + partition by(o_orderdate) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS + ${mv_sql} + """ + } + + def create_mv_all = { mv_name, mv_sql -> + sql """DROP MATERIALIZED VIEW IF EXISTS ${mv_name};""" + sql """DROP TABLE IF EXISTS ${mv_name}""" + sql""" + CREATE MATERIALIZED VIEW ${mv_name} + BUILD IMMEDIATE REFRESH AUTO ON MANUAL + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS + ${mv_sql} + """ + } + + def compare_res = { def stmt -> + sql "SET enable_materialized_view_rewrite=false" + def origin_res = sql stmt + logger.info("origin_res: " + origin_res) + sql "SET enable_materialized_view_rewrite=true" + def mv_origin_res = sql stmt + logger.info("mv_origin_res: " + mv_origin_res) + assertTrue((mv_origin_res == [] && origin_res == []) || (mv_origin_res.size() == origin_res.size())) + for (int row = 0; row < mv_origin_res.size(); row++) { + assertTrue(mv_origin_res[row].size() == origin_res[row].size()) + for (int col = 0; col < mv_origin_res[row].size(); col++) { + assertTrue(mv_origin_res[row][col] == origin_res[row][col]) + } + } + } + + // full join + filter on different position + def mv_stmt_0 = """select t.l_shipdate, o_orderdate, t.l_partkey, t.l_suppkey, orders_2_full_join.o_orderkey + from (select l_shipdate, l_partkey, l_suppkey, l_orderkey from lineitem_2_full_join where l_shipdate = '2023-10-17') t + full join orders_2_full_join + on t.l_orderkey = orders_2_full_join.o_orderkey""" + + def mv_stmt_1 = """select l_shipdate, t.o_orderdate, l_partkey, l_suppkey, t.o_orderkey + from lineitem_2_full_join + full join (select o_orderdate,o_orderkey from orders_2_full_join where o_orderdate = '2023-10-17' ) t + on lineitem_2_full_join.l_orderkey = t.o_orderkey""" + + def mv_stmt_2 = """select l_shipdate, o_orderdate, l_partkey, l_suppkey, orders_2_full_join.o_orderkey + from lineitem_2_full_join + full join orders_2_full_join + on lineitem_2_full_join.l_orderkey = orders_2_full_join.o_orderkey + where l_shipdate = '2023-10-17'""" + + def mv_stmt_3 = """select l_shipdate, o_orderdate, l_partkey, l_suppkey, orders_2_full_join.o_orderkey + from lineitem_2_full_join + full join orders_2_full_join + on lineitem_2_full_join.l_orderkey = orders_2_full_join.o_orderkey + where o_orderdate = '2023-10-17'""" + + def mv_stmt_4 = """select l_shipdate, o_orderdate, l_partkey, l_suppkey, orders_2_full_join.o_orderkey + from lineitem_2_full_join + full join orders_2_full_join + on lineitem_2_full_join.l_orderkey = orders_2_full_join.o_orderkey + where l_shipdate = '2023-10-17' and o_orderdate = '2023-10-17'""" + + def mv_stmt_5 = """select l_shipdate, o_orderdate, l_partkey, l_suppkey, orders_2_full_join.o_orderkey + from lineitem_2_full_join + full join orders_2_full_join + on lineitem_2_full_join.l_orderkey = orders_2_full_join.o_orderkey + where l_shipdate = '2023-10-17' and o_orderdate = '2023-10-17' + and o_orderkey = 1""" + + def mv_stmt_6 = """select t.l_shipdate, o_orderdate, t.l_partkey, t.l_suppkey, orders_2_full_join.o_orderkey + from orders_2_full_join + full join (select l_shipdate, l_orderkey, l_partkey, l_suppkey from lineitem_2_full_join where l_shipdate = '2023-10-17') t + on t.l_orderkey = orders_2_full_join.o_orderkey""" + + def mv_stmt_7 = """select l_shipdate, t.o_orderdate, l_partkey, l_suppkey, t.o_orderkey + from (select o_orderdate, o_orderkey from orders_2_full_join where o_orderdate = '2023-10-17' ) t + full join lineitem_2_full_join + on lineitem_2_full_join.l_orderkey = t.o_orderkey""" + + def mv_stmt_8 = """select l_shipdate, o_orderdate, l_partkey, l_suppkey, orders_2_full_join.o_orderkey + from orders_2_full_join + full join lineitem_2_full_join + on lineitem_2_full_join.l_orderkey = orders_2_full_join.o_orderkey + where l_shipdate = '2023-10-17' """ + + def mv_stmt_9 = """select l_shipdate, o_orderdate, l_partkey, l_suppkey, orders_2_full_join.o_orderkey + from orders_2_full_join + full join lineitem_2_full_join + on lineitem_2_full_join.l_orderkey = orders_2_full_join.o_orderkey + where o_orderdate = '2023-10-17' """ + + def mv_stmt_10 = """select l_shipdate, o_orderdate, l_partkey, l_suppkey, orders_2_full_join.o_orderkey + from orders_2_full_join + full join lineitem_2_full_join + on lineitem_2_full_join.l_orderkey = orders_2_full_join.o_orderkey + where l_shipdate = '2023-10-17' and o_orderdate = '2023-10-17' """ + + def mv_stmt_11 = """select l_shipdate, o_orderdate, l_partkey, l_suppkey, orders_2_full_join.o_orderkey + from orders_2_full_join + full join lineitem_2_full_join + on lineitem_2_full_join.l_orderkey = orders_2_full_join.o_orderkey + where l_shipdate = '2023-10-17' and o_orderdate = '2023-10-17' + and o_orderkey = 1""" + def mv_list_1 = [mv_stmt_0, mv_stmt_1, mv_stmt_2, mv_stmt_3, mv_stmt_4, mv_stmt_5, mv_stmt_6, + mv_stmt_7, mv_stmt_8, mv_stmt_9, mv_stmt_10, mv_stmt_11] + for (int i = 0; i < mv_list_1.size(); i++) { + logger.info("i:" + i) + def mv_name = """mv_name_2_full_join_${i}""" + create_mv_all(mv_name, mv_list_1[i]) + def job_name = getJobName(db, mv_name) + waitingMTMVTaskFinished(job_name) + if (i == 0) { + for (int j = 0; j < mv_list_1.size(); j++) { + logger.info("current index j:" + j) + if (j in [0, 2, 4, 5, 6, 8, 10, 11]) { + explain { + sql("${mv_list_1[j]}") + contains "${mv_name}(${mv_name})" + } + compare_res(mv_list_1[j] + " order by 1,2,3,4,5") + } else { + explain { + sql("${mv_list_1[j]}") + notContains "${mv_name}(${mv_name})" + } + } + } + } else if (i == 1) { + for (int j = 0; j < mv_list_1.size(); j++) { + logger.info("j:" + j) + if (j in [1, 3, 4, 5, 7, 9, 10, 11]) { + explain { + sql("${mv_list_1[j]}") + contains "${mv_name}(${mv_name})" + } + compare_res(mv_list_1[j] + " order by 1,2,3,4,5") + } else { + explain { + sql("${mv_list_1[j]}") + notContains "${mv_name}(${mv_name})" + } + } + } + } else if (i == 2) { + for (int j = 0; j < mv_list_1.size(); j++) { + logger.info("j:" + j) + if (j in [2, 4, 5, 8, 10, 11]) { + explain { + sql("${mv_list_1[j]}") + contains "${mv_name}(${mv_name})" + } + compare_res(mv_list_1[j] + " order by 1,2,3,4,5") + } else { + explain { + sql("${mv_list_1[j]}") + notContains "${mv_name}(${mv_name})" + } + } + } + } else if (i == 3) { + for (int j = 0; j < mv_list_1.size(); j++) { + logger.info("j:" + j) + if (j in [3, 4, 5, 9, 10, 11]) { + explain { + sql("${mv_list_1[j]}") + contains "${mv_name}(${mv_name})" + } + compare_res(mv_list_1[j] + " order by 1,2,3,4,5") + } else { + explain { + sql("${mv_list_1[j]}") + notContains "${mv_name}(${mv_name})" + } + } + } + } else if (i == 4) { + for (int j = 0; j < mv_list_1.size(); j++) { + logger.info("j:" + j) + if (j in [4, 5, 10, 11]) { + explain { + sql("${mv_list_1[j]}") + contains "${mv_name}(${mv_name})" + } + compare_res(mv_list_1[j] + " order by 1,2,3,4,5") + } else { + explain { + sql("${mv_list_1[j]}") + notContains "${mv_name}(${mv_name})" + } + } + } + } else if (i == 5) { + for (int j = 0; j < mv_list_1.size(); j++) { + logger.info("j:" + j) + if (j in [5, 11]) { + explain { + sql("${mv_list_1[j]}") + contains "${mv_name}(${mv_name})" + } + compare_res(mv_list_1[j] + " order by 1,2,3,4,5") + } else { + explain { + sql("${mv_list_1[j]}") + notContains "${mv_name}(${mv_name})" + } + } + } + } else if (i == 6) { + for (int j = 0; j < mv_list_1.size(); j++) { + logger.info("j:" + j) + if (j in [0, 2, 4, 5, 6, 8, 10, 11]) { + explain { + sql("${mv_list_1[j]}") + contains "${mv_name}(${mv_name})" + } + compare_res(mv_list_1[j] + " order by 1,2,3,4,5") + } else { + explain { + sql("${mv_list_1[j]}") + notContains "${mv_name}(${mv_name})" + } + } + } + } else if (i == 7) { + for (int j = 0; j < mv_list_1.size(); j++) { + logger.info("j:" + j) + if (j in [1, 3, 4, 5, 7, 9, 10, 11]) { + explain { + sql("${mv_list_1[j]}") + contains "${mv_name}(${mv_name})" + } + compare_res(mv_list_1[j] + " order by 1,2,3,4,5") + } else { + explain { + sql("${mv_list_1[j]}") + notContains "${mv_name}(${mv_name})" + } + } + } + } else if (i == 8) { + for (int j = 0; j < mv_list_1.size(); j++) { + logger.info("j:" + j) + if (j in [2, 4, 5, 8, 10, 11]) { + explain { + sql("${mv_list_1[j]}") + contains "${mv_name}(${mv_name})" + } + compare_res(mv_list_1[j] + " order by 1,2,3,4,5") + } else { + explain { + sql("${mv_list_1[j]}") + notContains "${mv_name}(${mv_name})" + } + } + } + } else if (i == 9) { + for (int j = 0; j < mv_list_1.size(); j++) { + logger.info("j:" + j) + if (j in [3, 4, 5, 9, 10, 11]) { + explain { + sql("${mv_list_1[j]}") + contains "${mv_name}(${mv_name})" + } + compare_res(mv_list_1[j] + " order by 1,2,3,4,5") + } else { + explain { + sql("${mv_list_1[j]}") + notContains "${mv_name}(${mv_name})" + } + } + } + } else if (i == 10) { + for (int j = 0; j < mv_list_1.size(); j++) { + logger.info("j:" + j) + if (j in [4, 5, 10, 11]) { + explain { + sql("${mv_list_1[j]}") + contains "${mv_name}(${mv_name})" + } + compare_res(mv_list_1[j] + " order by 1,2,3,4,5") + } else { + explain { + sql("${mv_list_1[j]}") + notContains "${mv_name}(${mv_name})" + } + } + } + } else if (i == 11) { + for (int j = 0; j < mv_list_1.size(); j++) { + logger.info("j:" + j) + if (j in [5, 11]) { + explain { + sql("${mv_list_1[j]}") + contains "${mv_name}(${mv_name})" + } + compare_res(mv_list_1[j] + " order by 1,2,3,4,5") + } else { + explain { + sql("${mv_list_1[j]}") + notContains "${mv_name}(${mv_name})" + } + } + } + } + sql """DROP MATERIALIZED VIEW IF EXISTS ${mv_name};""" + } +} diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_2.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_inner_join.groovy similarity index 100% rename from regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_2.groovy rename to regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_inner_join.groovy 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 new file mode 100644 index 00000000000000..13c6bc9c82c319 --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_left_anti_join.groovy @@ -0,0 +1,299 @@ +// 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 suite is a two dimensional test case file. +It mainly tests the left anti join and filter positions. + */ +suite("partition_mv_rewrite_dimension_2_left_anti_join") { + String db = context.config.getDbNameByFile(context.file) + sql "use ${db}" + sql "SET enable_nereids_planner=true" + sql "SET enable_fallback_to_original_planner=false" + sql "SET enable_materialized_view_rewrite=true" + sql "SET enable_nereids_timeout = false" + + sql """ + drop table if exists orders_2_left_anti_join + """ + + sql """CREATE TABLE `orders_2_left_anti_join` ( + `o_orderkey` BIGINT 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 not NULL + ) ENGINE=OLAP + DUPLICATE KEY(`o_orderkey`, `o_custkey`) + COMMENT 'OLAP' + AUTO PARTITION BY range date_trunc(`o_orderdate`, 'day') () + DISTRIBUTED BY HASH(`o_orderkey`) BUCKETS 96 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + );""" + + sql """ + drop table if exists lineitem_2_left_anti_join + """ + + sql """CREATE TABLE `lineitem_2_left_anti_join` ( + `l_orderkey` BIGINT 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 not NULL + ) ENGINE=OLAP + DUPLICATE KEY(l_orderkey, l_linenumber, l_partkey, l_suppkey ) + COMMENT 'OLAP' + AUTO PARTITION BY range date_trunc(`l_shipdate`, 'day') () + DISTRIBUTED BY HASH(`l_orderkey`) BUCKETS 96 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + );""" + + sql """ + insert into orders_2_left_anti_join values + (null, 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, 'o', 99.5, 'a', 'b', null, 'yy', '2023-10-19'), + (2, 1, 'k', 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_2_left_anti_join 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'), + (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'); + """ + + sql """analyze table orders_2_left_anti_join with sync;""" + sql """analyze table lineitem_2_left_anti_join with sync;""" + + def create_mv_lineitem = { mv_name, mv_sql -> + sql """DROP MATERIALIZED VIEW IF EXISTS ${mv_name};""" + sql """DROP TABLE IF EXISTS ${mv_name}""" + sql""" + CREATE MATERIALIZED VIEW ${mv_name} + BUILD IMMEDIATE REFRESH AUTO ON MANUAL + partition by(l_shipdate) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS + ${mv_sql} + """ + } + + def create_mv_orders = { mv_name, mv_sql -> + sql """DROP MATERIALIZED VIEW IF EXISTS ${mv_name};""" + sql """DROP TABLE IF EXISTS ${mv_name}""" + sql""" + CREATE MATERIALIZED VIEW ${mv_name} + BUILD IMMEDIATE REFRESH AUTO ON MANUAL + partition by(o_orderdate) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS + ${mv_sql} + """ + } + + def compare_res = { def stmt -> + sql "SET enable_materialized_view_rewrite=false" + def origin_res = sql stmt + logger.info("origin_res: " + origin_res) + sql "SET enable_materialized_view_rewrite=true" + def mv_origin_res = sql stmt + logger.info("mv_origin_res: " + mv_origin_res) + assertTrue((mv_origin_res == [] && origin_res == []) || (mv_origin_res.size() == origin_res.size())) + for (int row = 0; row < mv_origin_res.size(); row++) { + assertTrue(mv_origin_res[row].size() == origin_res[row].size()) + for (int col = 0; col < mv_origin_res[row].size(); col++) { + assertTrue(mv_origin_res[row][col] == origin_res[row][col]) + } + } + } + + // left anti join + filter on different position + def mv_stmt_0 = """select t.l_shipdate, t.l_partkey, t.l_suppkey + from (select l_shipdate, l_partkey, l_suppkey, l_orderkey from lineitem_2_left_anti_join where l_shipdate = '2023-10-17') t + left anti join orders_2_left_anti_join + on t.l_orderkey = orders_2_left_anti_join.o_orderkey""" + + def mv_stmt_1 = """select l_shipdate, l_partkey, l_suppkey + from lineitem_2_left_anti_join + left anti join (select o_orderdate,o_orderkey from orders_2_left_anti_join where o_orderdate = '2023-10-17' ) t + on lineitem_2_left_anti_join.l_orderkey = t.o_orderkey""" + + def mv_stmt_2 = """select l_shipdate, l_partkey, l_suppkey + from lineitem_2_left_anti_join + left anti join orders_2_left_anti_join + on lineitem_2_left_anti_join.l_orderkey = orders_2_left_anti_join.o_orderkey + where l_shipdate = '2023-10-17'""" + + def mv_stmt_3 = """select o_orderkey, o_orderdate, o_custkey + from orders_2_left_anti_join + left anti join (select l_shipdate, l_orderkey, l_partkey, l_suppkey from lineitem_2_left_anti_join where l_shipdate = '2023-10-17') t + on t.l_orderkey = orders_2_left_anti_join.o_orderkey""" + + def mv_stmt_4 = """select t.o_orderkey, t.o_orderdate, t.o_custkey + from (select o_orderkey, o_orderdate, o_custkey from orders_2_left_anti_join where o_orderdate = '2023-10-17' ) t + left anti join lineitem_2_left_anti_join + on lineitem_2_left_anti_join.l_orderkey = t.o_orderkey""" + + def mv_stmt_5 = """select o_orderkey, o_orderdate, o_custkey + from orders_2_left_anti_join + left anti join lineitem_2_left_anti_join + on lineitem_2_left_anti_join.l_orderkey = orders_2_left_anti_join.o_orderkey + where o_orderdate = '2023-10-17' """ + + def mv_list_1 = [mv_stmt_0, mv_stmt_1, mv_stmt_2, mv_stmt_3, mv_stmt_4, mv_stmt_5] + def order_by_stmt = " order by 1,2,3" + for (int i = 0; i < mv_list_1.size(); i++) { + logger.info("i:" + i) + def mv_name = """mv_name_2_left_anti_join_${i}""" + if (i < 3) { + create_mv_lineitem(mv_name, mv_list_1[i]) + } else { + create_mv_orders(mv_name, mv_list_1[i]) + } + def job_name = getJobName(db, mv_name) + waitingMTMVTaskFinished(job_name) + if (i == 0) { + for (int j = 0; j < mv_list_1.size(); j++) { + logger.info("j:" + j) + if (j in [ 0, 2]) { + explain { + sql("${mv_list_1[j]}") + contains "${mv_name}(${mv_name})" + } + compare_res(mv_list_1[j] + order_by_stmt) + } else { + explain { + sql("${mv_list_1[j]}") + notContains "${mv_name}(${mv_name})" + } + } + } + } else if (i == 1) { + for (int j = 0; j < mv_list_1.size(); j++) { + logger.info("j:" + j) + if (j in [1]) { + explain { + sql("${mv_list_1[j]}") + contains "${mv_name}(${mv_name})" + } + compare_res(mv_list_1[j] + order_by_stmt) + } else { + explain { + sql("${mv_list_1[j]}") + notContains "${mv_name}(${mv_name})" + } + } + } + } else if (i == 2) { + for (int j = 0; j < mv_list_1.size(); j++) { + logger.info("j:" + j) + if (j in [0, 2]) { + explain { + sql("${mv_list_1[j]}") + contains "${mv_name}(${mv_name})" + } + compare_res(mv_list_1[j] + order_by_stmt) + } else { + explain { + sql("${mv_list_1[j]}") + notContains "${mv_name}(${mv_name})" + } + } + } + } else if (i == 3) { + for (int j = 0; j < mv_list_1.size(); j++) { + logger.info("j:" + j) + if (j in [3]) { + explain { + sql("${mv_list_1[j]}") + contains "${mv_name}(${mv_name})" + } + compare_res(mv_list_1[j] + order_by_stmt) + } else { + explain { + sql("${mv_list_1[j]}") + notContains "${mv_name}(${mv_name})" + } + } + } + } else if (i == 4) { + for (int j = 0; j < mv_list_1.size(); j++) { + logger.info("j:" + j) + if (j in [4, 5]) { + explain { + sql("${mv_list_1[j]}") + contains "${mv_name}(${mv_name})" + } + compare_res(mv_list_1[j] + order_by_stmt) + } else { + explain { + sql("${mv_list_1[j]}") + notContains "${mv_name}(${mv_name})" + } + } + } + } else if (i == 5) { + for (int j = 0; j < mv_list_1.size(); j++) { + logger.info("j:" + j) + if (j in [4, 5]) { + explain { + sql("${mv_list_1[j]}") + contains "${mv_name}(${mv_name})" + } + compare_res(mv_list_1[j] + order_by_stmt) + } else { + explain { + sql("${mv_list_1[j]}") + notContains "${mv_name}(${mv_name})" + } + } + } + } + sql """DROP MATERIALIZED VIEW IF EXISTS ${mv_name};""" + } +} diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_1.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_left_join.groovy similarity index 99% rename from regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_1.groovy rename to regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_left_join.groovy index 328bf7925d13d2..cc3591c6f44417 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_1.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_left_join.groovy @@ -227,7 +227,7 @@ suite("partition_mv_rewrite_dimension_2_1") { for (int i = 0; i < mv_list_1.size(); i++) { logger.info("i:" + i) def mv_name = """mv_name_2_1_${i}""" - if (i < 6) { + if (i < 3) { create_mv_lineitem(mv_name, mv_list_1[i]) } else { create_mv_orders(mv_name, mv_list_1[i]) @@ -334,7 +334,7 @@ suite("partition_mv_rewrite_dimension_2_1") { for (int j = 0; j < mv_list_1.size(); j++) { logger.info("j:" + j) // 5, 11 should be success but not now, should support in the future by equivalence class - if (j in [4, 6, 8, 10]) { + if (j in [4, 5, 6, 8, 10, 11]) { explain { sql("${mv_list_1[j]}") contains "${mv_name}(${mv_name})" 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 new file mode 100644 index 00000000000000..e018aab7d799b5 --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_left_semi_join.groovy @@ -0,0 +1,299 @@ +// 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 suite is a two dimensional test case file. +It mainly tests the left semi join and filter positions. + */ +suite("partition_mv_rewrite_dimension_2_left_semi_join") { + String db = context.config.getDbNameByFile(context.file) + sql "use ${db}" + sql "SET enable_nereids_planner=true" + sql "SET enable_fallback_to_original_planner=false" + sql "SET enable_materialized_view_rewrite=true" + sql "SET enable_nereids_timeout = false" + + sql """ + drop table if exists orders_2_left_semi_join + """ + + sql """CREATE TABLE `orders_2_left_semi_join` ( + `o_orderkey` BIGINT 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 not NULL + ) ENGINE=OLAP + DUPLICATE KEY(`o_orderkey`, `o_custkey`) + COMMENT 'OLAP' + AUTO PARTITION BY range date_trunc(`o_orderdate`, 'day') () + DISTRIBUTED BY HASH(`o_orderkey`) BUCKETS 96 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + );""" + + sql """ + drop table if exists lineitem_2_left_semi_join + """ + + sql """CREATE TABLE `lineitem_2_left_semi_join` ( + `l_orderkey` BIGINT 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 not NULL + ) ENGINE=OLAP + DUPLICATE KEY(l_orderkey, l_linenumber, l_partkey, l_suppkey ) + COMMENT 'OLAP' + AUTO PARTITION BY range date_trunc(`l_shipdate`, 'day') () + DISTRIBUTED BY HASH(`l_orderkey`) BUCKETS 96 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + );""" + + sql """ + insert into orders_2_left_semi_join values + (null, 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, 'o', 99.5, 'a', 'b', null, 'yy', '2023-10-19'), + (2, 1, 'k', 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_2_left_semi_join 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'), + (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'); + """ + + sql """analyze table orders_2_left_semi_join with sync;""" + sql """analyze table lineitem_2_left_semi_join with sync;""" + + def create_mv_lineitem = { mv_name, mv_sql -> + sql """DROP MATERIALIZED VIEW IF EXISTS ${mv_name};""" + sql """DROP TABLE IF EXISTS ${mv_name}""" + sql""" + CREATE MATERIALIZED VIEW ${mv_name} + BUILD IMMEDIATE REFRESH AUTO ON MANUAL + partition by(l_shipdate) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS + ${mv_sql} + """ + } + + def create_mv_orders = { mv_name, mv_sql -> + sql """DROP MATERIALIZED VIEW IF EXISTS ${mv_name};""" + sql """DROP TABLE IF EXISTS ${mv_name}""" + sql""" + CREATE MATERIALIZED VIEW ${mv_name} + BUILD IMMEDIATE REFRESH AUTO ON MANUAL + partition by(o_orderdate) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS + ${mv_sql} + """ + } + + def compare_res = { def stmt -> + sql "SET enable_materialized_view_rewrite=false" + def origin_res = sql stmt + logger.info("origin_res: " + origin_res) + sql "SET enable_materialized_view_rewrite=true" + def mv_origin_res = sql stmt + logger.info("mv_origin_res: " + mv_origin_res) + assertTrue((mv_origin_res == [] && origin_res == []) || (mv_origin_res.size() == origin_res.size())) + for (int row = 0; row < mv_origin_res.size(); row++) { + assertTrue(mv_origin_res[row].size() == origin_res[row].size()) + for (int col = 0; col < mv_origin_res[row].size(); col++) { + assertTrue(mv_origin_res[row][col] == origin_res[row][col]) + } + } + } + + // left semi join + filter on different position + def mv_stmt_0 = """select t.l_shipdate, t.l_partkey, t.l_suppkey + from (select l_shipdate, l_partkey, l_suppkey, l_orderkey from lineitem_2_left_semi_join where l_shipdate = '2023-10-17') t + left semi join orders_2_left_semi_join + on t.l_orderkey = orders_2_left_semi_join.o_orderkey""" + + def mv_stmt_1 = """select l_shipdate, l_partkey, l_suppkey + from lineitem_2_left_semi_join + left semi join (select o_orderdate,o_orderkey from orders_2_left_semi_join where o_orderdate = '2023-10-17' ) t + on lineitem_2_left_semi_join.l_orderkey = t.o_orderkey""" + + def mv_stmt_2 = """select l_shipdate, l_partkey, l_suppkey + from lineitem_2_left_semi_join + left semi join orders_2_left_semi_join + on lineitem_2_left_semi_join.l_orderkey = orders_2_left_semi_join.o_orderkey + where l_shipdate = '2023-10-17'""" + + def mv_stmt_3 = """select o_orderkey, o_orderdate, o_custkey + from orders_2_left_semi_join + left semi join (select l_shipdate, l_orderkey, l_partkey, l_suppkey from lineitem_2_left_semi_join where l_shipdate = '2023-10-17') t + on t.l_orderkey = orders_2_left_semi_join.o_orderkey""" + + def mv_stmt_4 = """select t.o_orderkey, t.o_orderdate, t.o_custkey + from (select o_orderkey, o_orderdate, o_custkey from orders_2_left_semi_join where o_orderdate = '2023-10-17' ) t + left semi join lineitem_2_left_semi_join + on lineitem_2_left_semi_join.l_orderkey = t.o_orderkey""" + + def mv_stmt_5 = """select o_orderkey, o_orderdate, o_custkey + from orders_2_left_semi_join + left semi join lineitem_2_left_semi_join + on lineitem_2_left_semi_join.l_orderkey = orders_2_left_semi_join.o_orderkey + where o_orderdate = '2023-10-17' """ + + def mv_list_1 = [mv_stmt_0, mv_stmt_1, mv_stmt_2, mv_stmt_3, mv_stmt_4, mv_stmt_5] + def order_by_stmt = " order by 1,2,3" + for (int i = 0; i < mv_list_1.size(); i++) { + logger.info("i:" + i) + def mv_name = """mv_name_2_left_semi_join_${i}""" + if (i < 3) { + create_mv_lineitem(mv_name, mv_list_1[i]) + } else { + create_mv_orders(mv_name, mv_list_1[i]) + } + def job_name = getJobName(db, mv_name) + waitingMTMVTaskFinished(job_name) + if (i == 0) { + for (int j = 0; j < mv_list_1.size(); j++) { + logger.info("j:" + j) + if (j in [ 0, 2]) { + explain { + sql("${mv_list_1[j]}") + contains "${mv_name}(${mv_name})" + } + compare_res(mv_list_1[j] + order_by_stmt) + } else { + explain { + sql("${mv_list_1[j]}") + notContains "${mv_name}(${mv_name})" + } + } + } + } else if (i == 1) { + for (int j = 0; j < mv_list_1.size(); j++) { + logger.info("j:" + j) + if (j in [1]) { + explain { + sql("${mv_list_1[j]}") + contains "${mv_name}(${mv_name})" + } + compare_res(mv_list_1[j] + order_by_stmt) + } else { + explain { + sql("${mv_list_1[j]}") + notContains "${mv_name}(${mv_name})" + } + } + } + } else if (i == 2) { + for (int j = 0; j < mv_list_1.size(); j++) { + logger.info("j:" + j) + if (j in [0, 2]) { + explain { + sql("${mv_list_1[j]}") + contains "${mv_name}(${mv_name})" + } + compare_res(mv_list_1[j] + order_by_stmt) + } else { + explain { + sql("${mv_list_1[j]}") + notContains "${mv_name}(${mv_name})" + } + } + } + } else if (i == 3) { + for (int j = 0; j < mv_list_1.size(); j++) { + logger.info("j:" + j) + if (j in [3]) { + explain { + sql("${mv_list_1[j]}") + contains "${mv_name}(${mv_name})" + } + compare_res(mv_list_1[j] + order_by_stmt) + } else { + explain { + sql("${mv_list_1[j]}") + notContains "${mv_name}(${mv_name})" + } + } + } + } else if (i == 4) { + for (int j = 0; j < mv_list_1.size(); j++) { + logger.info("j:" + j) + if (j in [4, 5]) { + explain { + sql("${mv_list_1[j]}") + contains "${mv_name}(${mv_name})" + } + compare_res(mv_list_1[j] + order_by_stmt) + } else { + explain { + sql("${mv_list_1[j]}") + notContains "${mv_name}(${mv_name})" + } + } + } + } else if (i == 5) { + for (int j = 0; j < mv_list_1.size(); j++) { + logger.info("j:" + j) + if (j in [4, 5]) { + explain { + sql("${mv_list_1[j]}") + contains "${mv_name}(${mv_name})" + } + compare_res(mv_list_1[j] + order_by_stmt) + } else { + explain { + sql("${mv_list_1[j]}") + notContains "${mv_name}(${mv_name})" + } + } + } + } + sql """DROP MATERIALIZED VIEW IF EXISTS ${mv_name};""" + } +} 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 new file mode 100644 index 00000000000000..b6b576b1926ac9 --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_right_anti_join.groovy @@ -0,0 +1,299 @@ +// 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 suite is a two dimensional test case file. +It mainly tests the right anti join and filter positions. + */ +suite("partition_mv_rewrite_dimension_2_right_anti_join") { + String db = context.config.getDbNameByFile(context.file) + sql "use ${db}" + sql "SET enable_nereids_planner=true" + sql "SET enable_fallback_to_original_planner=false" + sql "SET enable_materialized_view_rewrite=true" + sql "SET enable_nereids_timeout = false" + + sql """ + drop table if exists orders_2_right_anti_join + """ + + sql """CREATE TABLE `orders_2_right_anti_join` ( + `o_orderkey` BIGINT 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 not NULL + ) ENGINE=OLAP + DUPLICATE KEY(`o_orderkey`, `o_custkey`) + COMMENT 'OLAP' + AUTO PARTITION BY range date_trunc(`o_orderdate`, 'day') () + DISTRIBUTED BY HASH(`o_orderkey`) BUCKETS 96 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + );""" + + sql """ + drop table if exists lineitem_2_right_anti_join + """ + + sql """CREATE TABLE `lineitem_2_right_anti_join` ( + `l_orderkey` BIGINT 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 not NULL + ) ENGINE=OLAP + DUPLICATE KEY(l_orderkey, l_linenumber, l_partkey, l_suppkey ) + COMMENT 'OLAP' + AUTO PARTITION BY range date_trunc(`l_shipdate`, 'day') () + DISTRIBUTED BY HASH(`l_orderkey`) BUCKETS 96 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + );""" + + sql """ + insert into orders_2_right_anti_join values + (null, 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, 'o', 99.5, 'a', 'b', null, 'yy', '2023-10-19'), + (2, 1, 'k', 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_2_right_anti_join 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'), + (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'); + """ + + sql """analyze table orders_2_right_anti_join with sync;""" + sql """analyze table lineitem_2_right_anti_join with sync;""" + + def create_mv_lineitem = { mv_name, mv_sql -> + sql """DROP MATERIALIZED VIEW IF EXISTS ${mv_name};""" + sql """DROP TABLE IF EXISTS ${mv_name}""" + sql""" + CREATE MATERIALIZED VIEW ${mv_name} + BUILD IMMEDIATE REFRESH AUTO ON MANUAL + partition by(l_shipdate) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS + ${mv_sql} + """ + } + + def create_mv_orders = { mv_name, mv_sql -> + sql """DROP MATERIALIZED VIEW IF EXISTS ${mv_name};""" + sql """DROP TABLE IF EXISTS ${mv_name}""" + sql""" + CREATE MATERIALIZED VIEW ${mv_name} + BUILD IMMEDIATE REFRESH AUTO ON MANUAL + partition by(o_orderdate) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS + ${mv_sql} + """ + } + + def compare_res = { def stmt -> + sql "SET enable_materialized_view_rewrite=false" + def origin_res = sql stmt + logger.info("origin_res: " + origin_res) + sql "SET enable_materialized_view_rewrite=true" + def mv_origin_res = sql stmt + logger.info("mv_origin_res: " + mv_origin_res) + assertTrue((mv_origin_res == [] && origin_res == []) || (mv_origin_res.size() == origin_res.size())) + for (int row = 0; row < mv_origin_res.size(); row++) { + assertTrue(mv_origin_res[row].size() == origin_res[row].size()) + for (int col = 0; col < mv_origin_res[row].size(); col++) { + assertTrue(mv_origin_res[row][col] == origin_res[row][col]) + } + } + } + + // right anti join + filter on different position + def mv_stmt_0 = """select o_orderkey, o_orderdate, o_custkey + from (select l_shipdate, l_partkey, l_suppkey, l_orderkey from lineitem_2_right_anti_join where l_shipdate = '2023-10-17') t + right anti join orders_2_right_anti_join + on t.l_orderkey = orders_2_right_anti_join.o_orderkey""" + + def mv_stmt_1 = """select t.o_orderkey, t.o_orderdate, t.o_custkey + from lineitem_2_right_anti_join + right anti join (select o_orderdate,o_orderkey, o_custkey from orders_2_right_anti_join where o_orderdate = '2023-10-17' ) t + on lineitem_2_right_anti_join.l_orderkey = t.o_orderkey""" + + def mv_stmt_2 = """select o_orderkey, o_orderdate, o_custkey + from lineitem_2_right_anti_join + right anti join orders_2_right_anti_join + on lineitem_2_right_anti_join.l_orderkey = orders_2_right_anti_join.o_orderkey + where o_orderdate = '2023-10-17'""" + + def mv_stmt_3 = """select t.l_shipdate, t.l_partkey, t.l_suppkey + from orders_2_right_anti_join + right anti join (select l_shipdate, l_orderkey, l_partkey, l_suppkey from lineitem_2_right_anti_join where l_shipdate = '2023-10-17') t + on t.l_orderkey = orders_2_right_anti_join.o_orderkey""" + + def mv_stmt_4 = """select l_shipdate, l_partkey, l_suppkey + from (select o_orderkey, o_orderdate, o_custkey from orders_2_right_anti_join where o_orderdate = '2023-10-17' ) t + right anti join lineitem_2_right_anti_join + on lineitem_2_right_anti_join.l_orderkey = t.o_orderkey""" + + def mv_stmt_5 = """select l_shipdate, l_partkey, l_suppkey + from orders_2_right_anti_join + right anti join lineitem_2_right_anti_join + on lineitem_2_right_anti_join.l_orderkey = orders_2_right_anti_join.o_orderkey + where l_shipdate = '2023-10-17' """ + + def mv_list_1 = [mv_stmt_0, mv_stmt_1, mv_stmt_2, mv_stmt_3, mv_stmt_4, mv_stmt_5] + def order_by_stmt = " order by 1,2,3" + for (int i = 0; i < mv_list_1.size(); i++) { + logger.info("i:" + i) + def mv_name = """mv_name_2_right_anti_join_${i}""" + if (i < 3) { + create_mv_orders(mv_name, mv_list_1[i]) + } else { + create_mv_lineitem(mv_name, mv_list_1[i]) + } + def job_name = getJobName(db, mv_name) + waitingMTMVTaskFinished(job_name) + if (i == 0) { + for (int j = 0; j < mv_list_1.size(); j++) { + logger.info("j:" + j) + if (j in [0]) { + explain { + sql("${mv_list_1[j]}") + contains "${mv_name}(${mv_name})" + } + compare_res(mv_list_1[j] + order_by_stmt) + } else { + explain { + sql("${mv_list_1[j]}") + notContains "${mv_name}(${mv_name})" + } + } + } + } else if (i == 1) { + for (int j = 0; j < mv_list_1.size(); j++) { + logger.info("j:" + j) + if (j in [1, 2]) { + explain { + sql("${mv_list_1[j]}") + contains "${mv_name}(${mv_name})" + } + compare_res(mv_list_1[j] + order_by_stmt) + } else { + explain { + sql("${mv_list_1[j]}") + notContains "${mv_name}(${mv_name})" + } + } + } + } else if (i == 2) { + for (int j = 0; j < mv_list_1.size(); j++) { + logger.info("j:" + j) + if (j in [1, 2]) { + explain { + sql("${mv_list_1[j]}") + contains "${mv_name}(${mv_name})" + } + compare_res(mv_list_1[j] + order_by_stmt) + } else { + explain { + sql("${mv_list_1[j]}") + notContains "${mv_name}(${mv_name})" + } + } + } + } else if (i == 3) { + for (int j = 0; j < mv_list_1.size(); j++) { + logger.info("j:" + j) + if (j in [3, 5]) { + explain { + sql("${mv_list_1[j]}") + contains "${mv_name}(${mv_name})" + } + compare_res(mv_list_1[j] + order_by_stmt) + } else { + explain { + sql("${mv_list_1[j]}") + notContains "${mv_name}(${mv_name})" + } + } + } + } else if (i == 4) { + for (int j = 0; j < mv_list_1.size(); j++) { + logger.info("j:" + j) + if (j in [4]) { + explain { + sql("${mv_list_1[j]}") + contains "${mv_name}(${mv_name})" + } + compare_res(mv_list_1[j] + order_by_stmt) + } else { + explain { + sql("${mv_list_1[j]}") + notContains "${mv_name}(${mv_name})" + } + } + } + } else if (i == 5) { + for (int j = 0; j < mv_list_1.size(); j++) { + logger.info("j:" + j) + if (j in [3, 5]) { + explain { + sql("${mv_list_1[j]}") + contains "${mv_name}(${mv_name})" + } + compare_res(mv_list_1[j] + order_by_stmt) + } else { + explain { + sql("${mv_list_1[j]}") + notContains "${mv_name}(${mv_name})" + } + } + } + } + sql """DROP MATERIALIZED VIEW IF EXISTS ${mv_name};""" + } +} 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 new file mode 100644 index 00000000000000..996ca5ae78b24f --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_right_join.groovy @@ -0,0 +1,434 @@ +// 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 suite is a two dimensional test case file. +It mainly tests the right join and filter positions. + */ +suite("partition_mv_rewrite_dimension_2_right_join") { + String db = context.config.getDbNameByFile(context.file) + sql "use ${db}" + sql "SET enable_nereids_planner=true" + sql "SET enable_fallback_to_original_planner=false" + sql "SET enable_materialized_view_rewrite=true" + sql "SET enable_nereids_timeout = false" + + sql """ + drop table if exists orders_2_right_join + """ + + sql """CREATE TABLE `orders_2_right_join` ( + `o_orderkey` BIGINT 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 not NULL + ) ENGINE=OLAP + DUPLICATE KEY(`o_orderkey`, `o_custkey`) + COMMENT 'OLAP' + AUTO PARTITION BY range date_trunc(`o_orderdate`, 'day') () + DISTRIBUTED BY HASH(`o_orderkey`) BUCKETS 96 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + );""" + + sql """ + drop table if exists lineitem_2_right_join + """ + + sql """CREATE TABLE `lineitem_2_right_join` ( + `l_orderkey` BIGINT 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 not NULL + ) ENGINE=OLAP + DUPLICATE KEY(l_orderkey, l_linenumber, l_partkey, l_suppkey ) + COMMENT 'OLAP' + AUTO PARTITION BY range date_trunc(`l_shipdate`, 'day') () + DISTRIBUTED BY HASH(`l_orderkey`) BUCKETS 96 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + );""" + + sql """ + insert into orders_2_right_join values + (null, 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, 'o', 99.5, 'a', 'b', null, 'yy', '2023-10-19'), + (2, 1, 'k', 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_2_right_join 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'), + (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'); + """ + + sql """analyze table orders_2_right_join with sync;""" + sql """analyze table lineitem_2_right_join with sync;""" + + def create_mv_lineitem = { mv_name, mv_sql -> + sql """DROP MATERIALIZED VIEW IF EXISTS ${mv_name};""" + sql """DROP TABLE IF EXISTS ${mv_name}""" + sql""" + CREATE MATERIALIZED VIEW ${mv_name} + BUILD IMMEDIATE REFRESH AUTO ON MANUAL + partition by(l_shipdate) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS + ${mv_sql} + """ + } + + def create_mv_orders = { mv_name, mv_sql -> + sql """DROP MATERIALIZED VIEW IF EXISTS ${mv_name};""" + sql """DROP TABLE IF EXISTS ${mv_name}""" + sql""" + CREATE MATERIALIZED VIEW ${mv_name} + BUILD IMMEDIATE REFRESH AUTO ON MANUAL + partition by(o_orderdate) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS + ${mv_sql} + """ + } + + def compare_res = { def stmt -> + sql "SET enable_materialized_view_rewrite=false" + def origin_res = sql stmt + logger.info("origin_res: " + origin_res) + sql "SET enable_materialized_view_rewrite=true" + def mv_origin_res = sql stmt + logger.info("mv_origin_res: " + mv_origin_res) + assertTrue((mv_origin_res == [] && origin_res == []) || (mv_origin_res.size() == origin_res.size())) + for (int row = 0; row < mv_origin_res.size(); row++) { + assertTrue(mv_origin_res[row].size() == origin_res[row].size()) + for (int col = 0; col < mv_origin_res[row].size(); col++) { + assertTrue(mv_origin_res[row][col] == origin_res[row][col]) + } + } + } + + def mv_stmt_0 = """select t.l_shipdate, o_orderdate, t.l_partkey, t.l_suppkey, orders_2_right_join.o_orderkey + from orders_2_right_join + right join (select l_shipdate, l_orderkey, l_partkey, l_suppkey from lineitem_2_right_join where l_shipdate = '2023-10-17') t + on t.l_orderkey = orders_2_right_join.o_orderkey""" + + def mv_stmt_1 = """select l_shipdate, t.o_orderdate, l_partkey, l_suppkey, t.o_orderkey + from (select o_orderdate, o_orderkey from orders_2_right_join where o_orderdate = '2023-10-17' ) t + right join lineitem_2_right_join + on lineitem_2_right_join.l_orderkey = t.o_orderkey""" + + def mv_stmt_2 = """select l_shipdate, o_orderdate, l_partkey, l_suppkey, orders_2_right_join.o_orderkey + from orders_2_right_join + right join lineitem_2_right_join + on lineitem_2_right_join.l_orderkey = orders_2_right_join.o_orderkey + where l_shipdate = '2023-10-17' """ + + def mv_stmt_3 = """select l_shipdate, o_orderdate, l_partkey, l_suppkey, orders_2_right_join.o_orderkey + from orders_2_right_join + right join lineitem_2_right_join + on lineitem_2_right_join.l_orderkey = orders_2_right_join.o_orderkey + where o_orderdate = '2023-10-17' """ + + def mv_stmt_4 = """select l_shipdate, o_orderdate, l_partkey, l_suppkey, orders_2_right_join.o_orderkey + from orders_2_right_join + right join lineitem_2_right_join + on lineitem_2_right_join.l_orderkey = orders_2_right_join.o_orderkey + where l_shipdate = '2023-10-17' and o_orderdate = '2023-10-17' """ + + def mv_stmt_5 = """select l_shipdate, o_orderdate, l_partkey, l_suppkey, orders_2_right_join.o_orderkey + from orders_2_right_join + right join lineitem_2_right_join + on lineitem_2_right_join.l_orderkey = orders_2_right_join.o_orderkey + where l_shipdate = '2023-10-17' and o_orderdate = '2023-10-17' + and o_orderkey = 1""" + + // right join + filter on different position + def mv_stmt_6 = """select t.l_shipdate, o_orderdate, t.l_partkey, t.l_suppkey, orders_2_right_join.o_orderkey + from (select l_shipdate, l_partkey, l_suppkey, l_orderkey from lineitem_2_right_join where l_shipdate = '2023-10-17') t + right join orders_2_right_join + on t.l_orderkey = orders_2_right_join.o_orderkey""" + + def mv_stmt_7 = """select l_shipdate, t.o_orderdate, l_partkey, l_suppkey, t.o_orderkey + from lineitem_2_right_join + right join (select o_orderdate,o_orderkey from orders_2_right_join where o_orderdate = '2023-10-17' ) t + on lineitem_2_right_join.l_orderkey = t.o_orderkey""" + + def mv_stmt_8 = """select l_shipdate, o_orderdate, l_partkey, l_suppkey, orders_2_right_join.o_orderkey + from lineitem_2_right_join + right join orders_2_right_join + on lineitem_2_right_join.l_orderkey = orders_2_right_join.o_orderkey + where l_shipdate = '2023-10-17'""" + + def mv_stmt_9 = """select l_shipdate, o_orderdate, l_partkey, l_suppkey, orders_2_right_join.o_orderkey + from lineitem_2_right_join + right join orders_2_right_join + on lineitem_2_right_join.l_orderkey = orders_2_right_join.o_orderkey + where o_orderdate = '2023-10-17'""" + + def mv_stmt_10 = """select l_shipdate, o_orderdate, l_partkey, l_suppkey, orders_2_right_join.o_orderkey + from lineitem_2_right_join + right join orders_2_right_join + on lineitem_2_right_join.l_orderkey = orders_2_right_join.o_orderkey + where l_shipdate = '2023-10-17' and o_orderdate = '2023-10-17'""" + + def mv_stmt_11 = """select l_shipdate, o_orderdate, l_partkey, l_suppkey, orders_2_right_join.o_orderkey + from lineitem_2_right_join + right join orders_2_right_join + on lineitem_2_right_join.l_orderkey = orders_2_right_join.o_orderkey + where l_shipdate = '2023-10-17' and o_orderdate = '2023-10-17' + and o_orderkey = 1""" + + def mv_list_1 = [mv_stmt_0, mv_stmt_1, mv_stmt_2, mv_stmt_3, mv_stmt_4, mv_stmt_5, mv_stmt_6, + mv_stmt_7, mv_stmt_8, mv_stmt_9, mv_stmt_10, mv_stmt_11] + for (int i = 0; i < mv_list_1.size(); i++) { + logger.info("i:" + i) + def mv_name = """mv_name_2_right_join_${i}""" + if (i < 6) { + create_mv_lineitem(mv_name, mv_list_1[i]) + } else { + create_mv_orders(mv_name, mv_list_1[i]) + } + def job_name = getJobName(db, mv_name) + waitingMTMVTaskFinished(job_name) + if (i == 0) { + for (int j = 0; j < mv_list_1.size(); j++) { + logger.info("j:" + j) + if (j in [ 0, 2, 4, 5, 10, 11]) { + explain { + sql("${mv_list_1[j]}") + contains "${mv_name}(${mv_name})" + } + compare_res(mv_list_1[j] + " order by 1,2,3,4,5") + } else { + explain { + sql("${mv_list_1[j]}") + notContains "${mv_name}(${mv_name})" + } + } + } + } else if (i == 1) { + for (int j = 0; j < mv_list_1.size(); j++) { + logger.info("j:" + j) + if (j in [1, 3, 4, 5, 10, 11]) { + explain { + sql("${mv_list_1[j]}") + contains "${mv_name}(${mv_name})" + } + compare_res(mv_list_1[j] + " order by 1,2,3,4,5") + } else { + explain { + sql("${mv_list_1[j]}") + notContains "${mv_name}(${mv_name})" + } + } + } + } else if (i == 2) { + for (int j = 0; j < mv_list_1.size(); j++) { + logger.info("j:" + j) + if (j in [0, 2, 4, 5, 10, 11]) { + explain { + sql("${mv_list_1[j]}") + contains "${mv_name}(${mv_name})" + } + compare_res(mv_list_1[j] + " order by 1,2,3,4,5") + } else { + explain { + sql("${mv_list_1[j]}") + notContains "${mv_name}(${mv_name})" + } + } + } + } else if (i == 3) { + for (int j = 0; j < mv_list_1.size(); j++) { + logger.info("j:" + j) + if (j in [3, 4, 5, 10, 11]) { + explain { + sql("${mv_list_1[j]}") + contains "${mv_name}(${mv_name})" + } + compare_res(mv_list_1[j] + " order by 1,2,3,4,5") + } else { + explain { + sql("${mv_list_1[j]}") + notContains "${mv_name}(${mv_name})" + } + } + } + } else if (i == 4) { + for (int j = 0; j < mv_list_1.size(); j++) { + logger.info("j:" + j) + if (j in [4, 5, 10, 11]) { + explain { + sql("${mv_list_1[j]}") + contains "${mv_name}(${mv_name})" + } + compare_res(mv_list_1[j] + " order by 1,2,3,4,5") + } else { + explain { + sql("${mv_list_1[j]}") + notContains "${mv_name}(${mv_name})" + } + } + } + } else if (i == 5) { + for (int j = 0; j < mv_list_1.size(); j++) { + logger.info("j:" + j) + if (j in [5, 11]) { + explain { + sql("${mv_list_1[j]}") + contains "${mv_name}(${mv_name})" + } + compare_res(mv_list_1[j] + " order by 1,2,3,4,5") + } else { + explain { + sql("${mv_list_1[j]}") + notContains "${mv_name}(${mv_name})" + } + } + } + } else if (i == 6) { + for (int j = 0; j < mv_list_1.size(); j++) { + logger.info("j:" + j) + // 5, 11 should be success but not now, should support in the future by equivalence class + if (j in [4, 5, 6, 8, 10, 11]) { + explain { + sql("${mv_list_1[j]}") + contains "${mv_name}(${mv_name})" + } + compare_res(mv_list_1[j] + " order by 1,2,3,4,5") + } else { + explain { + sql("${mv_list_1[j]}") + notContains "${mv_name}(${mv_name})" + } + } + } + } else if (i == 7) { + for (int j = 0; j < mv_list_1.size(); j++) { + logger.info("j:" + j) + if (j in [4, 5, 7, 9, 10, 11]) { + explain { + sql("${mv_list_1[j]}") + contains "${mv_name}(${mv_name})" + } + compare_res(mv_list_1[j] + " order by 1,2,3,4,5") + } else { + explain { + sql("${mv_list_1[j]}") + notContains "${mv_name}(${mv_name})" + } + } + } + } else if (i == 8) { + for (int j = 0; j < mv_list_1.size(); j++) { + logger.info("j:" + j) + if (j in [4, 5, 8, 10, 11]) { + explain { + sql("${mv_list_1[j]}") + contains "${mv_name}(${mv_name})" + } + compare_res(mv_list_1[j] + " order by 1,2,3,4,5") + } else { + explain { + sql("${mv_list_1[j]}") + notContains "${mv_name}(${mv_name})" + } + } + } + } else if (i == 9) { + for (int j = 0; j < mv_list_1.size(); j++) { + logger.info("j:" + j) + if (j in [4, 5, 7, 9, 10, 11]) { + explain { + sql("${mv_list_1[j]}") + contains "${mv_name}(${mv_name})" + } + compare_res(mv_list_1[j] + " order by 1,2,3,4,5") + } else { + explain { + sql("${mv_list_1[j]}") + notContains "${mv_name}(${mv_name})" + } + } + } + } else if (i == 10) { + for (int j = 0; j < mv_list_1.size(); j++) { + logger.info("j:" + j) + if (j in [4, 5, 10, 11]) { + explain { + sql("${mv_list_1[j]}") + contains "${mv_name}(${mv_name})" + } + compare_res(mv_list_1[j] + " order by 1,2,3,4,5") + } else { + explain { + sql("${mv_list_1[j]}") + notContains "${mv_name}(${mv_name})" + } + } + } + } else if (i == 11) { + for (int j = 0; j < mv_list_1.size(); j++) { + logger.info("j:" + j) + if (j in [5, 11]) { + explain { + sql("${mv_list_1[j]}") + contains "${mv_name}(${mv_name})" + } + compare_res(mv_list_1[j] + " order by 1,2,3,4,5") + } else { + explain { + sql("${mv_list_1[j]}") + notContains "${mv_name}(${mv_name})" + } + } + } + } + sql """DROP MATERIALIZED VIEW IF EXISTS ${mv_name};""" + } +} 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 new file mode 100644 index 00000000000000..39b9497610693c --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_right_semi_join.groovy @@ -0,0 +1,299 @@ +// 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 suite is a two dimensional test case file. +It mainly tests the right semi join and filter positions. + */ +suite("partition_mv_rewrite_dimension_2_right_semi_join") { + String db = context.config.getDbNameByFile(context.file) + sql "use ${db}" + sql "SET enable_nereids_planner=true" + sql "SET enable_fallback_to_original_planner=false" + sql "SET enable_materialized_view_rewrite=true" + sql "SET enable_nereids_timeout = false" + + sql """ + drop table if exists orders_2_right_semi_join + """ + + sql """CREATE TABLE `orders_2_right_semi_join` ( + `o_orderkey` BIGINT 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 not NULL + ) ENGINE=OLAP + DUPLICATE KEY(`o_orderkey`, `o_custkey`) + COMMENT 'OLAP' + AUTO PARTITION BY range date_trunc(`o_orderdate`, 'day') () + DISTRIBUTED BY HASH(`o_orderkey`) BUCKETS 96 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + );""" + + sql """ + drop table if exists lineitem_2_right_semi_join + """ + + sql """CREATE TABLE `lineitem_2_right_semi_join` ( + `l_orderkey` BIGINT 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 not NULL + ) ENGINE=OLAP + DUPLICATE KEY(l_orderkey, l_linenumber, l_partkey, l_suppkey ) + COMMENT 'OLAP' + AUTO PARTITION BY range date_trunc(`l_shipdate`, 'day') () + DISTRIBUTED BY HASH(`l_orderkey`) BUCKETS 96 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + );""" + + sql """ + insert into orders_2_right_semi_join values + (null, 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, 'o', 99.5, 'a', 'b', null, 'yy', '2023-10-19'), + (2, 1, 'k', 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_2_right_semi_join 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'), + (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'); + """ + + sql """analyze table orders_2_right_semi_join with sync;""" + sql """analyze table lineitem_2_right_semi_join with sync;""" + + def create_mv_lineitem = { mv_name, mv_sql -> + sql """DROP MATERIALIZED VIEW IF EXISTS ${mv_name};""" + sql """DROP TABLE IF EXISTS ${mv_name}""" + sql""" + CREATE MATERIALIZED VIEW ${mv_name} + BUILD IMMEDIATE REFRESH AUTO ON MANUAL + partition by(l_shipdate) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS + ${mv_sql} + """ + } + + def create_mv_orders = { mv_name, mv_sql -> + sql """DROP MATERIALIZED VIEW IF EXISTS ${mv_name};""" + sql """DROP TABLE IF EXISTS ${mv_name}""" + sql""" + CREATE MATERIALIZED VIEW ${mv_name} + BUILD IMMEDIATE REFRESH AUTO ON MANUAL + partition by(o_orderdate) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS + ${mv_sql} + """ + } + + def compare_res = { def stmt -> + sql "SET enable_materialized_view_rewrite=false" + def origin_res = sql stmt + logger.info("origin_res: " + origin_res) + sql "SET enable_materialized_view_rewrite=true" + def mv_origin_res = sql stmt + logger.info("mv_origin_res: " + mv_origin_res) + assertTrue((mv_origin_res == [] && origin_res == []) || (mv_origin_res.size() == origin_res.size())) + for (int row = 0; row < mv_origin_res.size(); row++) { + assertTrue(mv_origin_res[row].size() == origin_res[row].size()) + for (int col = 0; col < mv_origin_res[row].size(); col++) { + assertTrue(mv_origin_res[row][col] == origin_res[row][col]) + } + } + } + + // right semi join + filter on different position + def mv_stmt_0 = """select o_orderkey, o_orderdate, o_custkey + from (select l_shipdate, l_partkey, l_suppkey, l_orderkey from lineitem_2_right_semi_join where l_shipdate = '2023-10-17') t + right semi join orders_2_right_semi_join + on t.l_orderkey = orders_2_right_semi_join.o_orderkey""" + + def mv_stmt_1 = """select t.o_orderkey, t.o_orderdate, t.o_custkey + from lineitem_2_right_semi_join + right semi join (select o_orderdate, o_orderkey, o_custkey from orders_2_right_semi_join where o_orderdate = '2023-10-17' ) t + on lineitem_2_right_semi_join.l_orderkey = t.o_orderkey""" + + def mv_stmt_2 = """select o_orderkey, o_orderdate, o_custkey + from lineitem_2_right_semi_join + right semi join orders_2_right_semi_join + on lineitem_2_right_semi_join.l_orderkey = orders_2_right_semi_join.o_orderkey + where o_orderdate = '2023-10-17'""" + + def mv_stmt_3 = """select t.l_shipdate, t.l_partkey, t.l_suppkey + from orders_2_right_semi_join + right semi join (select l_shipdate, l_orderkey, l_partkey, l_suppkey from lineitem_2_right_semi_join where l_shipdate = '2023-10-17') t + on t.l_orderkey = orders_2_right_semi_join.o_orderkey""" + + def mv_stmt_4 = """select l_shipdate, l_partkey, l_suppkey + from (select o_orderkey, o_orderdate, o_custkey from orders_2_right_semi_join where o_orderdate = '2023-10-17' ) t + right semi join lineitem_2_right_semi_join + on lineitem_2_right_semi_join.l_orderkey = t.o_orderkey""" + + def mv_stmt_5 = """select l_shipdate, l_partkey, l_suppkey + from orders_2_right_semi_join + right semi join lineitem_2_right_semi_join + on lineitem_2_right_semi_join.l_orderkey = orders_2_right_semi_join.o_orderkey + where l_shipdate = '2023-10-17' """ + + def mv_list_1 = [mv_stmt_0, mv_stmt_1, mv_stmt_2, mv_stmt_3, mv_stmt_4, mv_stmt_5] + def order_by_stmt = " order by 1,2,3" + for (int i = 0; i < mv_list_1.size(); i++) { + logger.info("i:" + i) + def mv_name = """mv_name_2_right_semi_join_${i}""" + if (i < 3) { + create_mv_orders(mv_name, mv_list_1[i]) + } else { + create_mv_lineitem(mv_name, mv_list_1[i]) + } + def job_name = getJobName(db, mv_name) + waitingMTMVTaskFinished(job_name) + if (i == 0) { + for (int j = 0; j < mv_list_1.size(); j++) { + logger.info("j:" + j) + if (j in [0]) { + explain { + sql("${mv_list_1[j]}") + contains "${mv_name}(${mv_name})" + } + compare_res(mv_list_1[j] + order_by_stmt) + } else { + explain { + sql("${mv_list_1[j]}") + notContains "${mv_name}(${mv_name})" + } + } + } + } else if (i == 1) { + for (int j = 0; j < mv_list_1.size(); j++) { + logger.info("j:" + j) + if (j in [1, 2]) { + explain { + sql("${mv_list_1[j]}") + contains "${mv_name}(${mv_name})" + } + compare_res(mv_list_1[j] + order_by_stmt) + } else { + explain { + sql("${mv_list_1[j]}") + notContains "${mv_name}(${mv_name})" + } + } + } + } else if (i == 2) { + for (int j = 0; j < mv_list_1.size(); j++) { + logger.info("j:" + j) + if (j in [1, 2]) { + explain { + sql("${mv_list_1[j]}") + contains "${mv_name}(${mv_name})" + } + compare_res(mv_list_1[j] + order_by_stmt) + } else { + explain { + sql("${mv_list_1[j]}") + notContains "${mv_name}(${mv_name})" + } + } + } + } else if (i == 3) { + for (int j = 0; j < mv_list_1.size(); j++) { + logger.info("j:" + j) + if (j in [3, 5]) { + explain { + sql("${mv_list_1[j]}") + contains "${mv_name}(${mv_name})" + } + compare_res(mv_list_1[j] + order_by_stmt) + } else { + explain { + sql("${mv_list_1[j]}") + notContains "${mv_name}(${mv_name})" + } + } + } + } else if (i == 4) { + for (int j = 0; j < mv_list_1.size(); j++) { + logger.info("j:" + j) + if (j in [4]) { + explain { + sql("${mv_list_1[j]}") + contains "${mv_name}(${mv_name})" + } + compare_res(mv_list_1[j] + order_by_stmt) + } else { + explain { + sql("${mv_list_1[j]}") + notContains "${mv_name}(${mv_name})" + } + } + } + } else if (i == 5) { + for (int j = 0; j < mv_list_1.size(); j++) { + logger.info("j:" + j) + if (j in [3, 5]) { + explain { + sql("${mv_list_1[j]}") + contains "${mv_name}(${mv_name})" + } + compare_res(mv_list_1[j] + order_by_stmt) + } else { + explain { + sql("${mv_list_1[j]}") + notContains "${mv_name}(${mv_name})" + } + } + } + } + sql """DROP MATERIALIZED VIEW IF EXISTS ${mv_name};""" + } +} 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 a6702710a14016..055861fd344b7d 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 @@ -883,17 +883,21 @@ suite("dimension_2_join_agg_replenish") { t1.sum_total, col3, count_all """ - def sql_list = [left_mv_stmt_1,left_mv_stmt_2,left_mv_stmt_3,left_mv_stmt_4,left_mv_stmt_5,left_mv_stmt_6,left_mv_stmt_7,left_mv_stmt_8,left_mv_stmt_9,left_mv_stmt_10, - right_mv_stmt_1,right_mv_stmt_2,right_mv_stmt_3,right_mv_stmt_4,right_mv_stmt_5,right_mv_stmt_6,right_mv_stmt_7,right_mv_stmt_8,right_mv_stmt_9,right_mv_stmt_10, - inner_mv_stmt_1,inner_mv_stmt_2,inner_mv_stmt_3,inner_mv_stmt_4,inner_mv_stmt_5,inner_mv_stmt_6,inner_mv_stmt_7,inner_mv_stmt_8,inner_mv_stmt_9,inner_mv_stmt_10] -// query rewrite by materialzied view only support inner join, left outer join and right outer join(which can be converted to left outer join) -// full_mv_stmt_1,full_mv_stmt_2,full_mv_stmt_3,full_mv_stmt_4,full_mv_stmt_5,full_mv_stmt_6,full_mv_stmt_7,full_mv_stmt_8,full_mv_stmt_9,full_mv_stmt_10, -// cross_mv_stmt_1,cross_mv_stmt_2,cross_mv_stmt_3,cross_mv_stmt_4,cross_mv_stmt_5,cross_mv_stmt_6,cross_mv_stmt_7,cross_mv_stmt_8,cross_mv_stmt_9,cross_mv_stmt_10, -// left_semi_mv_stmt_1,left_semi_mv_stmt_2,left_semi_mv_stmt_3,left_semi_mv_stmt_4,left_semi_mv_stmt_5,left_semi_mv_stmt_6,left_semi_mv_stmt_7,left_semi_mv_stmt_8,left_semi_mv_stmt_9,left_semi_mv_stmt_10, -// left_anti_mv_stmt_1,left_anti_mv_stmt_2,left_anti_mv_stmt_3,left_anti_mv_stmt_4,left_anti_mv_stmt_5,left_anti_mv_stmt_6,left_anti_mv_stmt_7,left_anti_mv_stmt_8,left_anti_mv_stmt_9,left_anti_mv_stmt_10, -// right_semi_mv_stmt_1,right_semi_mv_stmt_2,right_semi_mv_stmt_3,right_semi_mv_stmt_4,right_semi_mv_stmt_5,right_semi_mv_stmt_6,right_semi_mv_stmt_7,right_semi_mv_stmt_8,right_semi_mv_stmt_9,right_semi_mv_stmt_10, -// right_anti_mv_stmt_1,right_anti_mv_stmt_2,right_anti_mv_stmt_3,right_anti_mv_stmt_4,right_anti_mv_stmt_5,right_anti_mv_stmt_6,right_anti_mv_stmt_7,right_anti_mv_stmt_8,right_anti_mv_stmt_9,right_anti_mv_stmt_10] - + def sql_list = [ + left_mv_stmt_1,left_mv_stmt_2,left_mv_stmt_3,left_mv_stmt_4,left_mv_stmt_5,left_mv_stmt_6,left_mv_stmt_7,left_mv_stmt_8,left_mv_stmt_9,left_mv_stmt_10, + right_mv_stmt_1,right_mv_stmt_2,right_mv_stmt_3,right_mv_stmt_4,right_mv_stmt_5,right_mv_stmt_6,right_mv_stmt_7,right_mv_stmt_8,right_mv_stmt_9,right_mv_stmt_10, + inner_mv_stmt_1,inner_mv_stmt_2,inner_mv_stmt_3,inner_mv_stmt_4,inner_mv_stmt_5,inner_mv_stmt_6,inner_mv_stmt_7,inner_mv_stmt_8,inner_mv_stmt_9,inner_mv_stmt_10, + full_mv_stmt_1,full_mv_stmt_2,full_mv_stmt_3,full_mv_stmt_4,full_mv_stmt_5,full_mv_stmt_6,full_mv_stmt_7,full_mv_stmt_8,full_mv_stmt_9,full_mv_stmt_10, + + // agg pulled up, and the struct info is invalid. need to support aggregate merge then support following query rewriting + // left_semi_mv_stmt_1,left_semi_mv_stmt_2,left_semi_mv_stmt_3,left_semi_mv_stmt_4,left_semi_mv_stmt_5,left_semi_mv_stmt_6,left_semi_mv_stmt_7,left_semi_mv_stmt_8,left_semi_mv_stmt_9,left_semi_mv_stmt_10, + // left_anti_mv_stmt_1,left_anti_mv_stmt_2,left_anti_mv_stmt_3,left_anti_mv_stmt_4,left_anti_mv_stmt_5,left_anti_mv_stmt_6,left_anti_mv_stmt_7,left_anti_mv_stmt_8,left_anti_mv_stmt_9,left_anti_mv_stmt_10, + // right_semi_mv_stmt_1,right_semi_mv_stmt_2,right_semi_mv_stmt_3,right_semi_mv_stmt_4,right_semi_mv_stmt_5,right_semi_mv_stmt_6,right_semi_mv_stmt_7,right_semi_mv_stmt_8,right_semi_mv_stmt_9,right_semi_mv_stmt_10, + // right_anti_mv_stmt_1,right_anti_mv_stmt_2,right_anti_mv_stmt_3,right_anti_mv_stmt_4,right_anti_mv_stmt_5,right_anti_mv_stmt_6,right_anti_mv_stmt_7,right_anti_mv_stmt_8,right_anti_mv_stmt_9,right_anti_mv_stmt_10 + + // query rewrite by materialized view doesn't support cross join currently + // cross_mv_stmt_1,cross_mv_stmt_2,cross_mv_stmt_3,cross_mv_stmt_4,cross_mv_stmt_5,cross_mv_stmt_6,cross_mv_stmt_7,cross_mv_stmt_8,cross_mv_stmt_9,cross_mv_stmt_10, + ] for (int i = 0; i < sql_list.size(); i++) { def origin_res = sql sql_list[i] assert (origin_res.size() > 0) diff --git a/regression-test/suites/nereids_rules_p0/mv/tpch/mv_tpch_test.groovy b/regression-test/suites/nereids_rules_p0/mv/tpch/mv_tpch_test.groovy index 2956654a6250a2..15b9ebf9f23f3a 100644 --- a/regression-test/suites/nereids_rules_p0/mv/tpch/mv_tpch_test.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/tpch/mv_tpch_test.groovy @@ -336,7 +336,7 @@ suite("mv_tpch_test") { """ // contains subquery, doesn't support now order_qt_query4_before "${query4}" - check_mv_rewrite_fail(db, mv4, query4, "mv4") + check_mv_rewrite_success(db, mv4, query4, "mv4") order_qt_query4_after "${query4}" sql """ DROP MATERIALIZED VIEW IF EXISTS mv4""" @@ -1070,7 +1070,7 @@ suite("mv_tpch_test") { """ // contains subquery, doesn't support now order_qt_query16_before "${query16}" - check_mv_rewrite_fail(db, mv16, query16, "mv16") + check_mv_rewrite_success(db, mv16, query16, "mv16") order_qt_query16_after "${query16}" sql """ DROP MATERIALIZED VIEW IF EXISTS mv16""" diff --git a/regression-test/suites/node_p0/test_backend.groovy b/regression-test/suites/node_p0/test_backend.groovy index 1fe6f802e907bf..cce111b0a19076 100644 --- a/regression-test/suites/node_p0/test_backend.groovy +++ b/regression-test/suites/node_p0/test_backend.groovy @@ -41,11 +41,12 @@ suite("test_backend", "nonConcurrent") { } if (context.config.jdbcUser.equals("root")) { + def beId1 = null try { + GetDebugPoint().enableDebugPointForAllFEs("SystemHandler.decommission_no_check_replica_num"); try_sql """admin set frontend config("drop_backend_after_decommission" = "false")""" def result = sql_return_maparray """SHOW BACKENDS;""" logger.info("show backends result:${result}") - def beId1 = null for (def res : result) { beId1 = res.BackendId break @@ -58,16 +59,23 @@ suite("test_backend", "nonConcurrent") { assertTrue(res.SystemDecommissioned.toBoolean()) } } - result = sql """CANCEL DECOMMISSION BACKEND "${beId1}" """ - logger.info("CANCEL DECOMMISSION BACKEND ${result}") - result = sql_return_maparray """SHOW BACKENDS;""" - for (def res : result) { - if (res.BackendId == "${beId1}") { - assertFalse(res.SystemDecommissioned.toBoolean()) + } finally { + try { + if (beId1 != null) { + def result = sql """CANCEL DECOMMISSION BACKEND "${beId1}" """ + logger.info("CANCEL DECOMMISSION BACKEND ${result}") + + result = sql_return_maparray """SHOW BACKENDS;""" + for (def res : result) { + if (res.BackendId == "${beId1}") { + assertFalse(res.SystemDecommissioned.toBoolean()) + } + } } + } finally { + GetDebugPoint().disableDebugPointForAllFEs('SystemHandler.decommission_no_check_replica_num'); + try_sql """admin set frontend config("drop_backend_after_decommission" = "true")""" } - } finally { - try_sql """admin set frontend config("drop_backend_after_decommission" = "true")""" } } } diff --git a/regression-test/suites/query_p0/schema_table/test_active_queries.groovy b/regression-test/suites/query_p0/schema_table/test_active_queries.groovy index 64c344deec7d3d..24bc177173e9c3 100644 --- a/regression-test/suites/query_p0/schema_table/test_active_queries.groovy +++ b/regression-test/suites/query_p0/schema_table/test_active_queries.groovy @@ -17,7 +17,7 @@ suite("test_active_queries") { def thread1 = new Thread({ - while(true) { + for (int i = 0; i <= 300; i++) { // non-pipeline sql "set experimental_enable_pipeline_engine=false" sql "set experimental_enable_pipeline_x_engine=false" diff --git a/regression-test/suites/query_p0/schema_table/test_backend_active_tasks.groovy b/regression-test/suites/query_p0/schema_table/test_backend_active_tasks.groovy index 172a7bbf8505fe..f0d93fc43c967f 100644 --- a/regression-test/suites/query_p0/schema_table/test_backend_active_tasks.groovy +++ b/regression-test/suites/query_p0/schema_table/test_backend_active_tasks.groovy @@ -17,7 +17,7 @@ suite("test_backend_active_tasks") { def thread1 = new Thread({ - while(true) { + for (int i = 0; i <= 300; i++) { // non-pipeline sql "set experimental_enable_pipeline_engine=false" sql "set experimental_enable_pipeline_x_engine=false" diff --git a/regression-test/suites/query_p0/sql_functions/search_functions/test_multi_string_position.groovy b/regression-test/suites/query_p0/sql_functions/search_functions/test_multi_string_position.groovy index 1c82fc7c6e9717..c844424b189e6a 100644 --- a/regression-test/suites/query_p0/sql_functions/search_functions/test_multi_string_position.groovy +++ b/regression-test/suites/query_p0/sql_functions/search_functions/test_multi_string_position.groovy @@ -58,24 +58,24 @@ suite("test_multi_string_position") { try { sql "select multi_search_all_positions(content, 'hello') from ${table_name} order by col1" } catch (Exception ex) { - assert("${ex}".contains("errCode = 2, detailMessage = No matching function with signature: multi_search_all_positions")) + assert("${ex}".contains("multi_search_all_positions")) } try { sql "select multi_search_all_positions(content, 'hello, !, world, Hello, World') from ${table_name} order by col1" } catch (Exception ex) { - assert("${ex}".contains("errCode = 2, detailMessage = No matching function with signature: multi_search_all_positions")) + assert("${ex}".contains("multi_search_all_positions")) } try { sql "select multi_search_all_positions(content, '[hello]') from ${table_name} order by col1" } catch (Exception ex) { - assert("${ex}".contains("errCode = 2, detailMessage = No matching function with signature: multi_search_all_positions")) + assert("${ex}".contains("multi_search_all_positions")) } try { sql "select multi_search_all_positions(content, '[hello, !, world, Hello, World]') from ${table_name} order by col1" } catch (Exception ex) { - assert("${ex}".contains("errCode = 2, detailMessage = No matching function with signature: multi_search_all_positions")) + assert("${ex}".contains("multi_search_all_positions")) } } 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 55a9f6894fd725..f1487d283dfcdf 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 @@ -75,24 +75,24 @@ suite("test_multi_string_search", "arrow_flight_sql") { try { sql "select multi_match_any(content, 'hello') from ${table_name} order by col1" } catch (Exception ex) { - assert("${ex}".contains("errCode = 2, detailMessage = No matching function with signature: multi_match_any")) + assert("${ex}".contains("multi_match_any")) } try { sql "select multi_match_any(content, 'hello, !, world, Hello, World') from ${table_name} order by col1" } catch (Exception ex) { - assert("${ex}".contains("errCode = 2, detailMessage = No matching function with signature: multi_match_any")) + assert("${ex}".contains("multi_match_any")) } try { sql "select multi_match_any(content, '[hello]') from ${table_name} order by col1" } catch (Exception ex) { - assert("${ex}".contains("errCode = 2, detailMessage = No matching function with signature: multi_match_any")) + assert("${ex}".contains("multi_match_any")) } try { sql "select multi_match_any(content, '[hello, !, world, Hello, World]') from ${table_name} order by col1" } catch (Exception ex) { - assert("${ex}".contains("errCode = 2, detailMessage = No matching function with signature: multi_match_any")) + assert("${ex}".contains("multi_match_any")) } } diff --git a/regression-test/suites/query_p0/sql_functions/test_in_expr.groovy b/regression-test/suites/query_p0/sql_functions/test_in_expr.groovy index 9829c9d61a791d..b6955e8df127a8 100644 --- a/regression-test/suites/query_p0/sql_functions/test_in_expr.groovy +++ b/regression-test/suites/query_p0/sql_functions/test_in_expr.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_in_expr", "query,arrow_flight_sql") { +suite("test_in_expr", "query") { // "arrow_flight_sql", groovy not support print arrow array type, throw IndexOutOfBoundsException. def nullTableName = "in_expr_test_null" def notNullTableName = "in_expr_test_not_null" diff --git a/regression-test/suites/query_p0/sql_functions/window_functions/test_window_fn.groovy b/regression-test/suites/query_p0/sql_functions/window_functions/test_window_fn.groovy index 165d5cf1d425a5..66bbab89d24971 100644 --- a/regression-test/suites/query_p0/sql_functions/window_functions/test_window_fn.groovy +++ b/regression-test/suites/query_p0/sql_functions/window_functions/test_window_fn.groovy @@ -14,7 +14,9 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations // under the License. -suite("test_window_fn", "arrow_flight_sql") { +suite("test_window_fn") { + // "arrow_flight_sql", groovy use flight sql connection to execute query `SUM(MAX(c1) OVER (PARTITION BY))` report error: + // `AGGREGATE clause must not contain analytic expressions`, but no problem in Java execute it with `jdbc::arrow-flight-sql`. def tbName1 = "empsalary" def tbName2 = "tenk1" sql """ DROP TABLE IF EXISTS ${tbName1} """ diff --git a/regression-test/suites/query_p0/test_two_phase_read_with_having.groovy b/regression-test/suites/query_p0/test_two_phase_read_with_having.groovy index d0207960db8bff..28e5e0405e5d5a 100644 --- a/regression-test/suites/query_p0/test_two_phase_read_with_having.groovy +++ b/regression-test/suites/query_p0/test_two_phase_read_with_having.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_two_phase_read_with_having", "arrow_flight_sql") { +suite("test_two_phase_read_with_having") { // "arrow_flight_sql" not support two phase read sql """ set enable_partition_topn = 1; """ sql """ set topn_opt_limit_threshold = 1024; """ diff --git a/regression-test/suites/schema_change_p0/test_alter_table_replace.groovy b/regression-test/suites/schema_change_p0/test_alter_table_replace.groovy index b07a54c528865a..e261725c4394cd 100644 --- a/regression-test/suites/schema_change_p0/test_alter_table_replace.groovy +++ b/regression-test/suites/schema_change_p0/test_alter_table_replace.groovy @@ -96,7 +96,7 @@ suite("test_alter_table_replace") { test { sql """ select * from ${tbNameB} order by user_id""" // check exception message contains - exception "Unknown table '${tbNameB}'" + exception "'${tbNameB}'" } sql "DROP TABLE IF EXISTS ${tbNameA} FORCE;" diff --git a/regression-test/suites/show_p0/test_show_data_skew.groovy b/regression-test/suites/show_p0/test_show_data_skew.groovy new file mode 100644 index 00000000000000..f9a65dbbb3eb02 --- /dev/null +++ b/regression-test/suites/show_p0/test_show_data_skew.groovy @@ -0,0 +1,37 @@ +// 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_data_skew") { + sql """ + CREATE TABLE test_show_data_skew ( + id int, + name string, + pdate DATETIME) + PARTITION BY RANGE(pdate) ( + FROM ("2023-04-16") TO ("2023-04-20") INTERVAL 1 DAY + ) DISTRIBUTED BY HASH(id) BUCKETS 5 + properties("replication_num" = "1"); + """ + def result = sql """show data skew from test_show_data_skew;""" + assertTrue(result.size() == 20) + + def result2 = sql """show data skew from test_show_data_skew partition(p_20230416);""" + assertTrue(result2.size() == 5) + + def result3 = sql """show data skew from test_show_data_skew partition(p_20230416, p_20230418);""" + assertTrue(result3.size() == 10) +} diff --git a/regression-test/suites/statistics/analyze_stats.groovy b/regression-test/suites/statistics/analyze_stats.groovy index 699e595df598cb..8cfec96623399a 100644 --- a/regression-test/suites/statistics/analyze_stats.groovy +++ b/regression-test/suites/statistics/analyze_stats.groovy @@ -1122,10 +1122,10 @@ PARTITION `p599` VALUES IN (599) System.out.println(actual_result) return expected_result.containsAll(actual_result) && actual_result.containsAll(expected_result) } - assert check_column(afterDropped, "[col2, col3]") + assert check_column(afterDropped, "[test_meta_management:col2, test_meta_management:col3]") sql """ANALYZE TABLE test_meta_management WITH SYNC""" afterDropped = sql """SHOW TABLE STATS test_meta_management""" - assert check_column(afterDropped, "[col1, col2, col3]") + assert check_column(afterDropped, "[test_meta_management:col1, test_meta_management:col2, test_meta_management:col3]") sql """ DROP TABLE IF EXISTS test_updated_rows """ sql """ diff --git a/regression-test/suites/statistics/test_analyze_mtmv.groovy b/regression-test/suites/statistics/test_analyze_mtmv.groovy index 7662fd1fbbe32e..3655a35390bbb3 100644 --- a/regression-test/suites/statistics/test_analyze_mtmv.groovy +++ b/regression-test/suites/statistics/test_analyze_mtmv.groovy @@ -143,7 +143,7 @@ suite("test_analyze_mtmv") { result_sample = sql """show column stats mv1(l_shipdate)""" assertEquals(1, result_sample.size()) assertEquals("l_shipdate", result_sample[0][0]) - assertEquals("N/A", result_sample[0][1]) + assertEquals("mv1", result_sample[0][1]) assertEquals("3.0", result_sample[0][2]) assertEquals("3.0", result_sample[0][3]) assertEquals("0.0", result_sample[0][4]) @@ -157,7 +157,7 @@ suite("test_analyze_mtmv") { result_sample = sql """show column cached stats mv1(l_shipdate)""" assertEquals(1, result_sample.size()) assertEquals("l_shipdate", result_sample[0][0]) - assertEquals("N/A", result_sample[0][1]) + assertEquals("mv1", result_sample[0][1]) assertEquals("3.0", result_sample[0][2]) assertEquals("3.0", result_sample[0][3]) assertEquals("0.0", result_sample[0][4]) @@ -171,7 +171,7 @@ suite("test_analyze_mtmv") { result_sample = sql """show column stats mv1(o_orderdate)""" assertEquals(1, result_sample.size()) assertEquals("o_orderdate", result_sample[0][0]) - assertEquals("N/A", result_sample[0][1]) + assertEquals("mv1", result_sample[0][1]) assertEquals("3.0", result_sample[0][2]) assertEquals("3.0", result_sample[0][3]) assertEquals("0.0", result_sample[0][4]) @@ -185,7 +185,7 @@ suite("test_analyze_mtmv") { result_sample = sql """show column cached stats mv1(o_orderdate)""" assertEquals(1, result_sample.size()) assertEquals("o_orderdate", result_sample[0][0]) - assertEquals("N/A", result_sample[0][1]) + assertEquals("mv1", result_sample[0][1]) assertEquals("3.0", result_sample[0][2]) assertEquals("3.0", result_sample[0][3]) assertEquals("0.0", result_sample[0][4]) @@ -199,7 +199,7 @@ suite("test_analyze_mtmv") { result_sample = sql """show column stats mv1(l_partkey)""" assertEquals(1, result_sample.size()) assertEquals("l_partkey", result_sample[0][0]) - assertEquals("N/A", result_sample[0][1]) + assertEquals("mv1", result_sample[0][1]) assertEquals("3.0", result_sample[0][2]) assertEquals("1.0", result_sample[0][3]) assertEquals("0.0", result_sample[0][4]) @@ -213,7 +213,7 @@ suite("test_analyze_mtmv") { result_sample = sql """show column cached stats mv1(l_partkey)""" assertEquals(1, result_sample.size()) assertEquals("l_partkey", result_sample[0][0]) - assertEquals("N/A", result_sample[0][1]) + assertEquals("mv1", result_sample[0][1]) assertEquals("3.0", result_sample[0][2]) assertEquals("1.0", result_sample[0][3]) assertEquals("0.0", result_sample[0][4]) @@ -227,7 +227,7 @@ suite("test_analyze_mtmv") { result_sample = sql """show column stats mv1(l_suppkey)""" assertEquals(1, result_sample.size()) assertEquals("l_suppkey", result_sample[0][0]) - assertEquals("N/A", result_sample[0][1]) + assertEquals("mv1", result_sample[0][1]) assertEquals("3.0", result_sample[0][2]) assertEquals("1.0", result_sample[0][3]) assertEquals("0.0", result_sample[0][4]) @@ -241,7 +241,7 @@ suite("test_analyze_mtmv") { result_sample = sql """show column cached stats mv1(l_suppkey)""" assertEquals(1, result_sample.size()) assertEquals("l_suppkey", result_sample[0][0]) - assertEquals("N/A", result_sample[0][1]) + assertEquals("mv1", result_sample[0][1]) assertEquals("3.0", result_sample[0][2]) assertEquals("1.0", result_sample[0][3]) assertEquals("0.0", result_sample[0][4]) @@ -255,7 +255,7 @@ suite("test_analyze_mtmv") { result_sample = sql """show column stats mv1(sum_total)""" assertEquals(1, result_sample.size()) assertEquals("sum_total", result_sample[0][0]) - assertEquals("N/A", result_sample[0][1]) + assertEquals("mv1", result_sample[0][1]) assertEquals("3.0", result_sample[0][2]) assertEquals("2.0", result_sample[0][3]) assertEquals("0.0", result_sample[0][4]) @@ -269,7 +269,7 @@ suite("test_analyze_mtmv") { result_sample = sql """show column cached stats mv1(sum_total)""" assertEquals(1, result_sample.size()) assertEquals("sum_total", result_sample[0][0]) - assertEquals("N/A", result_sample[0][1]) + assertEquals("mv1", result_sample[0][1]) assertEquals("3.0", result_sample[0][2]) assertEquals("2.0", result_sample[0][3]) assertEquals("0.0", result_sample[0][4]) @@ -298,7 +298,7 @@ suite("test_analyze_mtmv") { } assertEquals(1, result_sample.size()) assertEquals("l_shipdate", result_sample[0][0]) - assertEquals("N/A", result_sample[0][1]) + assertEquals("mv1", result_sample[0][1]) assertEquals("3.0", result_sample[0][2]) assertEquals("3.0", result_sample[0][3]) assertEquals("0.0", result_sample[0][4]) @@ -319,7 +319,7 @@ suite("test_analyze_mtmv") { } assertEquals(1, result_sample.size()) assertEquals("l_shipdate", result_sample[0][0]) - assertEquals("N/A", result_sample[0][1]) + assertEquals("mv1", result_sample[0][1]) assertEquals("3.0", result_sample[0][2]) assertEquals("3.0", result_sample[0][3]) assertEquals("0.0", result_sample[0][4]) @@ -340,7 +340,7 @@ suite("test_analyze_mtmv") { } assertEquals(1, result_sample.size()) assertEquals("o_orderdate", result_sample[0][0]) - assertEquals("N/A", result_sample[0][1]) + assertEquals("mv1", result_sample[0][1]) assertEquals("3.0", result_sample[0][2]) assertEquals("3.0", result_sample[0][3]) assertEquals("0.0", result_sample[0][4]) @@ -361,7 +361,7 @@ suite("test_analyze_mtmv") { } assertEquals(1, result_sample.size()) assertEquals("o_orderdate", result_sample[0][0]) - assertEquals("N/A", result_sample[0][1]) + assertEquals("mv1", result_sample[0][1]) assertEquals("3.0", result_sample[0][2]) assertEquals("3.0", result_sample[0][3]) assertEquals("0.0", result_sample[0][4]) @@ -382,7 +382,7 @@ suite("test_analyze_mtmv") { } assertEquals(1, result_sample.size()) assertEquals("l_partkey", result_sample[0][0]) - assertEquals("N/A", result_sample[0][1]) + assertEquals("mv1", result_sample[0][1]) assertEquals("3.0", result_sample[0][2]) assertEquals("1.0", result_sample[0][3]) assertEquals("0.0", result_sample[0][4]) @@ -403,7 +403,7 @@ suite("test_analyze_mtmv") { } assertEquals(1, result_sample.size()) assertEquals("l_partkey", result_sample[0][0]) - assertEquals("N/A", result_sample[0][1]) + assertEquals("mv1", result_sample[0][1]) assertEquals("3.0", result_sample[0][2]) assertEquals("1.0", result_sample[0][3]) assertEquals("0.0", result_sample[0][4]) @@ -424,7 +424,7 @@ suite("test_analyze_mtmv") { } assertEquals(1, result_sample.size()) assertEquals("l_suppkey", result_sample[0][0]) - assertEquals("N/A", result_sample[0][1]) + assertEquals("mv1", result_sample[0][1]) assertEquals("3.0", result_sample[0][2]) assertEquals("1.0", result_sample[0][3]) assertEquals("0.0", result_sample[0][4]) @@ -445,7 +445,7 @@ suite("test_analyze_mtmv") { } assertEquals(1, result_sample.size()) assertEquals("l_suppkey", result_sample[0][0]) - assertEquals("N/A", result_sample[0][1]) + assertEquals("mv1", result_sample[0][1]) assertEquals("3.0", result_sample[0][2]) assertEquals("1.0", result_sample[0][3]) assertEquals("0.0", result_sample[0][4]) @@ -466,7 +466,7 @@ suite("test_analyze_mtmv") { } assertEquals(1, result_sample.size()) assertEquals("sum_total", result_sample[0][0]) - assertEquals("N/A", result_sample[0][1]) + assertEquals("mv1", result_sample[0][1]) assertEquals("3.0", result_sample[0][2]) assertEquals("2.0", result_sample[0][3]) assertEquals("0.0", result_sample[0][4]) @@ -487,7 +487,7 @@ suite("test_analyze_mtmv") { } assertEquals(1, result_sample.size()) assertEquals("sum_total", result_sample[0][0]) - assertEquals("N/A", result_sample[0][1]) + assertEquals("mv1", result_sample[0][1]) assertEquals("3.0", result_sample[0][2]) assertEquals("2.0", result_sample[0][3]) assertEquals("0.0", result_sample[0][4]) diff --git a/regression-test/suites/statistics/test_analyze_mv.groovy b/regression-test/suites/statistics/test_analyze_mv.groovy index 635837e6c16400..3348623acaf7b2 100644 --- a/regression-test/suites/statistics/test_analyze_mv.groovy +++ b/regression-test/suites/statistics/test_analyze_mv.groovy @@ -145,7 +145,7 @@ suite("test_analyze_mv") { def result_sample = sql """show column stats mvTestDup(key1)""" assertEquals(1, result_sample.size()) assertEquals("key1", result_sample[0][0]) - assertEquals("N/A", result_sample[0][1]) + assertEquals("mvTestDup", result_sample[0][1]) assertEquals("6.0", result_sample[0][2]) assertEquals("4.0", result_sample[0][3]) assertEquals("1", result_sample[0][7]) @@ -157,7 +157,7 @@ suite("test_analyze_mv") { result_sample = sql """show column stats mvTestDup(value1)""" assertEquals(1, result_sample.size()) assertEquals("value1", result_sample[0][0]) - assertEquals("N/A", result_sample[0][1]) + assertEquals("mvTestDup", result_sample[0][1]) assertEquals("6.0", result_sample[0][2]) assertEquals("4.0", result_sample[0][3]) assertEquals("3", result_sample[0][7]) @@ -252,9 +252,9 @@ suite("test_analyze_mv") { result_sample = sql """show column stats mvTestAgg(key1)""" assertEquals(2, result_sample.size()) - if (result_sample[0][1] == "N/A") { + if (result_sample[0][1] == "mvTestAgg") { assertEquals("key1", result_sample[0][0]) - assertEquals("N/A", result_sample[0][1]) + assertEquals("mvTestAgg", result_sample[0][1]) assertEquals("5.0", result_sample[0][2]) assertEquals("4.0", result_sample[0][3]) assertEquals("1", result_sample[0][7]) @@ -267,7 +267,7 @@ suite("test_analyze_mv") { assertEquals("1001", result_sample[1][8]) } else { assertEquals("key1", result_sample[1][0]) - assertEquals("N/A", result_sample[1][1]) + assertEquals("mvTestAgg", result_sample[1][1]) assertEquals("5.0", result_sample[1][2]) assertEquals("4.0", result_sample[1][3]) assertEquals("1", result_sample[1][7]) @@ -282,9 +282,9 @@ suite("test_analyze_mv") { result_sample = sql """show column stats mvTestAgg(value1)""" assertEquals(2, result_sample.size()) - if (result_sample[0][1] == "N/A") { + if (result_sample[0][1] == "mvTestAgg") { assertEquals("value1", result_sample[0][0]) - assertEquals("N/A", result_sample[0][1]) + assertEquals("mvTestAgg", result_sample[0][1]) assertEquals("5.0", result_sample[0][2]) assertEquals("5.0", result_sample[0][3]) assertEquals("6", result_sample[0][7]) @@ -297,7 +297,7 @@ suite("test_analyze_mv") { assertEquals("3001", result_sample[1][8]) } else { assertEquals("value1", result_sample[1][0]) - assertEquals("N/A", result_sample[1][1]) + assertEquals("mvTestAgg", result_sample[1][1]) assertEquals("5.0", result_sample[1][2]) assertEquals("5.0", result_sample[1][3]) assertEquals("6", result_sample[1][7]) @@ -313,7 +313,7 @@ suite("test_analyze_mv") { result_sample = sql """show column stats mvTestAgg(key2)""" assertEquals(1, result_sample.size()) assertEquals("key2", result_sample[0][0]) - assertEquals("N/A", result_sample[0][1]) + assertEquals("mvTestAgg", result_sample[0][1]) assertEquals("5.0", result_sample[0][2]) assertEquals("5.0", result_sample[0][3]) assertEquals("2", result_sample[0][7]) @@ -323,7 +323,7 @@ suite("test_analyze_mv") { result_sample = sql """show column stats mvTestAgg(value2)""" assertEquals(1, result_sample.size()) assertEquals("value2", result_sample[0][0]) - assertEquals("N/A", result_sample[0][1]) + assertEquals("mvTestAgg", result_sample[0][1]) assertEquals("5.0", result_sample[0][2]) assertEquals("5.0", result_sample[0][3]) assertEquals("4", result_sample[0][7]) @@ -391,7 +391,7 @@ suite("test_analyze_mv") { result_sample = sql """show column stats mvTestUni(key1)""" assertEquals(1, result_sample.size()) assertEquals("key1", result_sample[0][0]) - assertEquals("N/A", result_sample[0][1]) + assertEquals("mvTestUni", result_sample[0][1]) assertEquals("5.0", result_sample[0][2]) assertEquals("4.0", result_sample[0][3]) assertEquals("1", result_sample[0][7]) @@ -444,7 +444,7 @@ suite("test_analyze_mv") { } assertEquals(1, result_sample.size()) assertEquals("key1", result_sample[0][0]) - assertEquals("N/A", result_sample[0][1]) + assertEquals("mvTestDup", result_sample[0][1]) assertEquals("6.0", result_sample[0][2]) assertEquals("4.0", result_sample[0][3]) assertEquals("1", result_sample[0][7]) @@ -462,7 +462,7 @@ suite("test_analyze_mv") { } assertEquals(1, result_sample.size()) assertEquals("value1", result_sample[0][0]) - assertEquals("N/A", result_sample[0][1]) + assertEquals("mvTestDup", result_sample[0][1]) assertEquals("6.0", result_sample[0][2]) assertEquals("4.0", result_sample[0][3]) assertEquals("3", result_sample[0][7]) @@ -558,11 +558,11 @@ suite("test_analyze_mv") { logger.info("col " + colName + " in index " + indexName + " found ? " + found) assertTrue(found) } - verifyTaskStatus(result_sample, "key1", "N/A") - verifyTaskStatus(result_sample, "key2", "N/A") - verifyTaskStatus(result_sample, "value1", "N/A") - verifyTaskStatus(result_sample, "value2", "N/A") - verifyTaskStatus(result_sample, "value3", "N/A") + verifyTaskStatus(result_sample, "key1", "mvTestDup") + verifyTaskStatus(result_sample, "key2", "mvTestDup") + verifyTaskStatus(result_sample, "value1", "mvTestDup") + verifyTaskStatus(result_sample, "value2", "mvTestDup") + verifyTaskStatus(result_sample, "value3", "mvTestDup") verifyTaskStatus(result_sample, "mv_key1", "mv1") verifyTaskStatus(result_sample, "mv_key1", "mv3") verifyTaskStatus(result_sample, "mv_key2", "mv2") @@ -580,7 +580,7 @@ suite("test_analyze_mv") { def result = sql """show column cached stats mvTestDup(key1)""" assertEquals(1, result.size()) assertEquals("key1", result[0][0]) - assertEquals("N/A", result[0][1]) + assertEquals("mvTestDup", result[0][1]) assertEquals("50.0", result[0][2]) assertEquals("1.0", result[0][3]) assertEquals("1.0", result[0][4]) diff --git a/regression-test/suites/unique_with_mow_p0/partial_update/test_partial_update_insert_light_schema_change.groovy b/regression-test/suites/unique_with_mow_p0/partial_update/test_partial_update_insert_light_schema_change.groovy index 22a063c6dcc5e8..6f05843138b905 100644 --- a/regression-test/suites/unique_with_mow_p0/partial_update/test_partial_update_insert_light_schema_change.groovy +++ b/regression-test/suites/unique_with_mow_p0/partial_update/test_partial_update_insert_light_schema_change.groovy @@ -132,7 +132,7 @@ suite("test_partial_update_insert_light_schema_change", "p0") { sql "set enable_unique_key_partial_update=true;" test { sql "insert into ${tableName}(c0,c1,c2,c8) values(1,1,1,10);" - exception "Unknown column 'c8' in 'test_partial_update_insert_light_schema_change_delete_column'" + exception "Unknown column 'c8' in" } sql "insert into ${tableName}(c0,c1,c2) values(1,1,1);" sql "set enable_unique_key_partial_update=false;" diff --git a/regression-test/suites/workload_manager_p0/test_curd_wlg.groovy b/regression-test/suites/workload_manager_p0/test_curd_wlg.groovy index a73801f8743c5e..b68aab16fff9b7 100644 --- a/regression-test/suites/workload_manager_p0/test_curd_wlg.groovy +++ b/regression-test/suites/workload_manager_p0/test_curd_wlg.groovy @@ -24,6 +24,14 @@ suite("test_crud_wlg") { sql "drop table if exists ${table_name2}" sql "drop table if exists ${table_name3}" + sql "drop workload group if exists tag1_wg1;" + sql "drop workload group if exists tag1_wg2;" + sql "drop workload group if exists tag2_wg1;" + sql "drop workload group if exists tag1_wg3;" + sql "drop workload group if exists tag1_mem_wg1;" + sql "drop workload group if exists tag1_mem_wg2;" + sql "drop workload group if exists tag1_mem_wg3;" + sql """ CREATE TABLE IF NOT EXISTS `${table_name}` ( `siteid` int(11) NOT NULL COMMENT "", @@ -117,7 +125,7 @@ suite("test_crud_wlg") { ");" sql "set workload_group=test_group;" - qt_show_1 "select name,cpu_share,memory_limit,enable_memory_overcommit,max_concurrency,max_queue_size,queue_timeout,cpu_hard_limit,scan_thread_num from information_schema.workload_groups where name in ('normal','test_group') order by name;" + qt_show_1 "select name,cpu_share,memory_limit,enable_memory_overcommit,max_concurrency,max_queue_size,queue_timeout,cpu_hard_limit,scan_thread_num,tag from information_schema.workload_groups where name in ('normal','test_group') order by name;" // test memory_limit test { @@ -147,7 +155,7 @@ suite("test_crud_wlg") { test { sql "alter workload group test_group properties ( 'cpu_hard_limit'='101%' );" - exception "can not be greater than 100%" + exception "must be a positive integer" } sql "alter workload group test_group properties ( 'cpu_hard_limit'='99%' );" @@ -245,7 +253,7 @@ suite("test_crud_wlg") { " 'cpu_hard_limit'='120%' " + ");" - exception "can not be greater than" + exception "must be a positive integer" } test { @@ -257,7 +265,7 @@ suite("test_crud_wlg") { " 'cpu_hard_limit'='99%' " + ");" - exception "can not be greater than" + exception "can not be greater than 100%" } // test show workload groups @@ -402,4 +410,80 @@ suite("test_crud_wlg") { sql "drop workload group test_group;" sql "drop workload group spill_group_test;" + + + // test workload group's tag property, cpu_hard_limit + test { + sql "create workload group if not exists tag1_wg1 properties ( 'cpu_hard_limit'='101%', 'tag'='tag1')" + exception "must be a positive integer" + } + + test { + sql "create workload group if not exists tag1_wg1 properties ( 'cpu_hard_limit'='-2%', 'tag'='tag1')" + exception "must be a positive integer" + } + + test { + sql "create workload group if not exists tag1_wg1 properties ( 'cpu_hard_limit'='-1%', 'tag'='tag1')" + exception "must be a positive integer" + } + + sql "create workload group if not exists tag1_wg1 properties ( 'cpu_hard_limit'='10%', 'tag'='tag1');" + + test { + sql "create workload group if not exists tag1_wg2 properties ( 'cpu_hard_limit'='91%', 'tag'='tag1');" + exception "can not be greater than 100%" + } + + sql "create workload group if not exists tag1_wg2 properties ( 'cpu_hard_limit'='10%', 'tag'='tag1');" + + sql "create workload group if not exists tag2_wg1 properties ( 'cpu_hard_limit'='91%', 'tag'='tag2');" + + test { + sql "alter workload group tag2_wg1 properties ( 'tag'='tag1' );" + exception "can not be greater than 100% " + } + + sql "alter workload group tag2_wg1 properties ( 'cpu_hard_limit'='10%' );" + sql "alter workload group tag2_wg1 properties ( 'tag'='tag1' );" + + test { + sql "create workload group if not exists tag1_wg3 properties ( 'cpu_hard_limit'='80%', 'tag'='tag1');" + exception "can not be greater than 100% " + } + + sql "drop workload group tag2_wg1;" + sql "create workload group if not exists tag1_wg3 properties ( 'cpu_hard_limit'='80%', 'tag'='tag1');" + + // test workload group's tag property, memory_limit + sql "create workload group if not exists tag1_mem_wg1 properties ( 'memory_limit'='50%', 'tag'='mem_tag1');" + + test { + sql "create workload group if not exists tag1_mem_wg2 properties ( 'memory_limit'='60%', 'tag'='mem_tag1');" + exception "cannot be greater than 100.0%" + } + + sql "create workload group if not exists tag1_mem_wg2 properties ('memory_limit'='49%', 'tag'='mem_tag1');" + + sql "create workload group if not exists tag1_mem_wg3 properties ( 'memory_limit'='2%');" + + test { + sql "alter workload group tag1_mem_wg3 properties ( 'tag'='mem_tag1' );" + exception "cannot be greater than 100.0%" + } + + sql "alter workload group tag1_mem_wg3 properties ( 'memory_limit'='1%' );" + + sql "alter workload group tag1_mem_wg3 properties ( 'tag'='mem_tag1' );" + + qt_show_wg_tag "select name,MEMORY_LIMIT,CPU_HARD_LIMIT,TAG from information_schema.workload_groups where name in('tag1_wg1','tag1_wg2','tag2_wg1','tag1_wg3','tag1_mem_wg1','tag1_mem_wg2','tag1_mem_wg3') order by tag,name;" + + sql "drop workload group tag1_wg1;" + sql "drop workload group tag1_wg2;" + sql "drop workload group if exists tag2_wg1;" + sql "drop workload group tag1_wg3;" + sql "drop workload group tag1_mem_wg1;" + sql "drop workload group tag1_mem_wg2;" + sql "drop workload group tag1_mem_wg3;" + } diff --git a/run-regression-test.sh b/run-regression-test.sh index 78523ea220018c..dc90f4b648d5dc 100755 --- a/run-regression-test.sh +++ b/run-regression-test.sh @@ -200,6 +200,7 @@ fi echo "===== Run Regression Test =====" +# if use jdk17, add java option "--add-opens=java.base/java.nio=ALL-UNNAMED" if [[ "${TEAMCITY}" -eq 1 ]]; then JAVA_OPTS="${JAVA_OPTS} -DstdoutAppenderType=teamcity -Xmx2048m" fi