From 3d984da89b9523cbbe5c75acff5f9c2963f93ba5 Mon Sep 17 00:00:00 2001 From: yanweiqi <592838129@qq.com> Date: Thu, 22 Sep 2022 14:13:03 +0800 Subject: [PATCH] Test: refine astToExecutor to ExecutorBinders (#5915) ref pingcap/tiflash#4609 --- .../Debug/MockExecutor/AggregationBinder.cpp | 280 +++ .../Debug/MockExecutor/AggregationBinder.h | 63 + dbms/src/Debug/MockExecutor/AstToPB.cpp | 684 ++++++ dbms/src/Debug/MockExecutor/AstToPB.h | 98 + ...stToExecutorUtils.cpp => AstToPBUtils.cpp} | 3 +- .../{astToExecutorUtils.h => AstToPBUtils.h} | 0 .../MockExecutor/ExchangeReceiverBinder.cpp | 61 + .../MockExecutor/ExchangeReceiverBinder.h | 39 + .../MockExecutor/ExchangeSenderBinder.cpp | 76 + .../Debug/MockExecutor/ExchangeSenderBinder.h | 43 + dbms/src/Debug/MockExecutor/ExecutorBinder.h | 63 + .../{funcSigs.h => FuncSigMap.cpp} | 3 - dbms/src/Debug/MockExecutor/FuncSigMap.h | 25 + dbms/src/Debug/MockExecutor/JoinBinder.cpp | 338 +++ dbms/src/Debug/MockExecutor/JoinBinder.h | 73 + dbms/src/Debug/MockExecutor/LimitBinder.cpp | 44 + dbms/src/Debug/MockExecutor/LimitBinder.h | 38 + dbms/src/Debug/MockExecutor/ProjectBinder.cpp | 121 ++ dbms/src/Debug/MockExecutor/ProjectBinder.h | 38 + .../Debug/MockExecutor/SelectionBinder.cpp | 50 + dbms/src/Debug/MockExecutor/SelectionBinder.h | 39 + dbms/src/Debug/MockExecutor/SortBinder.cpp | 62 + dbms/src/Debug/MockExecutor/SortBinder.h | 44 + .../Debug/MockExecutor/TableScanBinder.cpp | 123 ++ dbms/src/Debug/MockExecutor/TableScanBinder.h | 51 + dbms/src/Debug/MockExecutor/TopNBinder.cpp | 66 + dbms/src/Debug/MockExecutor/TopNBinder.h | 40 + dbms/src/Debug/MockExecutor/WindowBinder.cpp | 220 ++ dbms/src/Debug/MockExecutor/WindowBinder.h | 60 + dbms/src/Debug/MockExecutor/astToExecutor.cpp | 1843 ----------------- dbms/src/Debug/MockExecutor/astToExecutor.h | 362 ---- dbms/src/Debug/dbgFuncCoprocessor.cpp | 53 +- dbms/src/Debug/dbgFuncCoprocessor.h | 16 +- dbms/src/Server/FlashGrpcServerHolder.cpp | 1 + dbms/src/Server/FlashGrpcServerHolder.h | 4 +- dbms/src/TestUtils/ColumnsToTiPBExpr.cpp | 4 +- dbms/src/TestUtils/mockExecutor.cpp | 30 +- dbms/src/TestUtils/mockExecutor.h | 6 +- 38 files changed, 2906 insertions(+), 2258 deletions(-) create mode 100644 dbms/src/Debug/MockExecutor/AggregationBinder.cpp create mode 100644 dbms/src/Debug/MockExecutor/AggregationBinder.h create mode 100644 dbms/src/Debug/MockExecutor/AstToPB.cpp create mode 100644 dbms/src/Debug/MockExecutor/AstToPB.h rename dbms/src/Debug/MockExecutor/{astToExecutorUtils.cpp => AstToPBUtils.cpp} (97%) rename dbms/src/Debug/MockExecutor/{astToExecutorUtils.h => AstToPBUtils.h} (100%) create mode 100644 dbms/src/Debug/MockExecutor/ExchangeReceiverBinder.cpp create mode 100644 dbms/src/Debug/MockExecutor/ExchangeReceiverBinder.h create mode 100644 dbms/src/Debug/MockExecutor/ExchangeSenderBinder.cpp create mode 100644 dbms/src/Debug/MockExecutor/ExchangeSenderBinder.h create mode 100644 dbms/src/Debug/MockExecutor/ExecutorBinder.h rename dbms/src/Debug/MockExecutor/{funcSigs.h => FuncSigMap.cpp} (99%) create mode 100644 dbms/src/Debug/MockExecutor/FuncSigMap.h create mode 100644 dbms/src/Debug/MockExecutor/JoinBinder.cpp create mode 100644 dbms/src/Debug/MockExecutor/JoinBinder.h create mode 100644 dbms/src/Debug/MockExecutor/LimitBinder.cpp create mode 100644 dbms/src/Debug/MockExecutor/LimitBinder.h create mode 100644 dbms/src/Debug/MockExecutor/ProjectBinder.cpp create mode 100644 dbms/src/Debug/MockExecutor/ProjectBinder.h create mode 100644 dbms/src/Debug/MockExecutor/SelectionBinder.cpp create mode 100644 dbms/src/Debug/MockExecutor/SelectionBinder.h create mode 100644 dbms/src/Debug/MockExecutor/SortBinder.cpp create mode 100644 dbms/src/Debug/MockExecutor/SortBinder.h create mode 100644 dbms/src/Debug/MockExecutor/TableScanBinder.cpp create mode 100644 dbms/src/Debug/MockExecutor/TableScanBinder.h create mode 100644 dbms/src/Debug/MockExecutor/TopNBinder.cpp create mode 100644 dbms/src/Debug/MockExecutor/TopNBinder.h create mode 100644 dbms/src/Debug/MockExecutor/WindowBinder.cpp create mode 100644 dbms/src/Debug/MockExecutor/WindowBinder.h delete mode 100644 dbms/src/Debug/MockExecutor/astToExecutor.cpp delete mode 100644 dbms/src/Debug/MockExecutor/astToExecutor.h diff --git a/dbms/src/Debug/MockExecutor/AggregationBinder.cpp b/dbms/src/Debug/MockExecutor/AggregationBinder.cpp new file mode 100644 index 00000000000..a39f196a389 --- /dev/null +++ b/dbms/src/Debug/MockExecutor/AggregationBinder.cpp @@ -0,0 +1,280 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// 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 + +namespace DB::mock +{ +bool AggregationBinder::toTiPBExecutor(tipb::Executor * tipb_executor, int32_t collator_id, const MPPInfo & mpp_info, const Context & context) +{ + tipb_executor->set_tp(tipb::ExecType::TypeAggregation); + tipb_executor->set_executor_id(name); + auto * agg = tipb_executor->mutable_aggregation(); + buildAggExpr(agg, collator_id, context); + buildGroupBy(agg, collator_id, context); + auto * child_executor = agg->mutable_child(); + return children[0]->toTiPBExecutor(child_executor, collator_id, mpp_info, context); +} + +void AggregationBinder::columnPrune(std::unordered_set & used_columns) +{ + /// output schema for partial agg is the original agg's output schema + output_schema_for_partial_agg = output_schema; + output_schema.erase(std::remove_if(output_schema.begin(), output_schema.end(), [&](const auto & field) { return used_columns.count(field.first) == 0; }), + output_schema.end()); + std::unordered_set used_input_columns; + for (auto & func : agg_exprs) + { + if (used_columns.find(func->getColumnName()) != used_columns.end()) + { + const auto * agg_func = typeid_cast(func.get()); + if (agg_func != nullptr) + { + /// agg_func should not be nullptr, just double check + for (auto & child : agg_func->arguments->children) + collectUsedColumnsFromExpr(children[0]->output_schema, child, used_input_columns); + } + } + } + for (auto & gby_expr : gby_exprs) + { + collectUsedColumnsFromExpr(children[0]->output_schema, gby_expr, used_input_columns); + } + children[0]->columnPrune(used_input_columns); +} + +void AggregationBinder::toMPPSubPlan(size_t & executor_index, const DAGProperties & properties, std::unordered_map, std::shared_ptr>> & exchange_map) +{ + if (!is_final_mode) + { + children[0]->toMPPSubPlan(executor_index, properties, exchange_map); + return; + } + /// for aggregation, change aggregation to partial_aggregation => exchange_sender => exchange_receiver => final_aggregation + // todo support avg + if (has_uniq_raw_res) + throw Exception("uniq raw res not supported in mpp query"); + std::shared_ptr partial_agg = std::make_shared( + executor_index, + output_schema_for_partial_agg, + has_uniq_raw_res, + false, + std::move(agg_exprs), + std::move(gby_exprs), + false); + partial_agg->children.push_back(children[0]); + std::vector partition_keys; + size_t agg_func_num = partial_agg->agg_exprs.size(); + for (size_t i = 0; i < partial_agg->gby_exprs.size(); ++i) + { + partition_keys.push_back(i + agg_func_num); + } + + std::shared_ptr exchange_sender + = std::make_shared(executor_index, output_schema_for_partial_agg, partition_keys.empty() ? tipb::PassThrough : tipb::Hash, partition_keys); + exchange_sender->children.push_back(partial_agg); + + std::shared_ptr exchange_receiver + = std::make_shared(executor_index, output_schema_for_partial_agg); + exchange_map[exchange_receiver->name] = std::make_pair(exchange_receiver, exchange_sender); + + /// re-construct agg_exprs and gby_exprs in final_agg + for (size_t i = 0; i < partial_agg->agg_exprs.size(); ++i) + { + const auto * agg_func = typeid_cast(partial_agg->agg_exprs[i].get()); + ASTPtr update_agg_expr = agg_func->clone(); + auto * update_agg_func = typeid_cast(update_agg_expr.get()); + if (agg_func->name == "count") + update_agg_func->name = "sum"; + update_agg_func->arguments->children.clear(); + update_agg_func->arguments->children.push_back(std::make_shared(output_schema_for_partial_agg[i].first)); + agg_exprs.push_back(update_agg_expr); + } + for (size_t i = 0; i < partial_agg->gby_exprs.size(); ++i) + { + gby_exprs.push_back(std::make_shared(output_schema_for_partial_agg[agg_func_num + i].first)); + } + children[0] = exchange_receiver; +} + +bool AggregationBinder::needAppendProject() const +{ + return need_append_project; +} + +size_t AggregationBinder::exprSize() const +{ + return agg_exprs.size() + gby_exprs.size(); +} + +bool AggregationBinder::hasUniqRawRes() const +{ + return has_uniq_raw_res; +} + +void AggregationBinder::buildGroupBy(tipb::Aggregation * agg, int32_t collator_id, const Context & context) const +{ + auto & input_schema = children[0]->output_schema; + for (const auto & child : gby_exprs) + { + tipb::Expr * gby = agg->add_group_by(); + astToPB(input_schema, child, gby, collator_id, context); + } +} + +void AggregationBinder::buildAggExpr(tipb::Aggregation * agg, int32_t collator_id, const Context & context) const +{ + auto & input_schema = children[0]->output_schema; + + for (const auto & expr : agg_exprs) + { + const auto * func = typeid_cast(expr.get()); + if (!func || !AggregateFunctionFactory::instance().isAggregateFunctionName(func->name)) + throw Exception("Only agg function is allowed in select for a query with aggregation", ErrorCodes::LOGICAL_ERROR); + + tipb::Expr * agg_func = agg->add_agg_func(); + + for (const auto & arg : func->arguments->children) + { + tipb::Expr * arg_expr = agg_func->add_children(); + astToPB(input_schema, arg, arg_expr, collator_id, context); + } + + buildAggFunc(agg_func, func, collator_id); + } +} + +void AggregationBinder::buildAggFunc(tipb::Expr * agg_func, const ASTFunction * func, int32_t collator_id) const +{ + auto agg_sig_it = tests::agg_func_name_to_sig.find(func->name); + if (agg_sig_it == tests::agg_func_name_to_sig.end()) + throw Exception("Unsupported agg function: " + func->name, ErrorCodes::LOGICAL_ERROR); + + auto agg_sig = agg_sig_it->second; + agg_func->set_tp(agg_sig); + + if (agg_sig == tipb::ExprType::Count || agg_sig == tipb::ExprType::Sum) + { + auto * ft = agg_func->mutable_field_type(); + ft->set_tp(TiDB::TypeLongLong); + ft->set_flag(TiDB::ColumnFlagUnsigned | TiDB::ColumnFlagNotNull); + } + else if (agg_sig == tipb::ExprType::Min || agg_sig == tipb::ExprType::Max || agg_sig == tipb::ExprType::First) + { + if (agg_func->children_size() != 1) + throw Exception(fmt::format("Agg function({}) only accept 1 argument", func->name)); + + auto * ft = agg_func->mutable_field_type(); + ft->set_tp(agg_func->children(0).field_type().tp()); + ft->set_decimal(agg_func->children(0).field_type().decimal()); + ft->set_flag(agg_func->children(0).field_type().flag() & (~TiDB::ColumnFlagNotNull)); + ft->set_collate(collator_id); + } + else if (agg_sig == tipb::ExprType::ApproxCountDistinct) + { + auto * ft = agg_func->mutable_field_type(); + ft->set_tp(TiDB::TypeString); + ft->set_flag(1); + } + else if (agg_sig == tipb::ExprType::GroupConcat) + { + auto * ft = agg_func->mutable_field_type(); + ft->set_tp(TiDB::TypeString); + } + if (is_final_mode) + agg_func->set_aggfuncmode(tipb::AggFunctionMode::FinalMode); + else + agg_func->set_aggfuncmode(tipb::AggFunctionMode::Partial1Mode); +} + +ExecutorBinderPtr compileAggregation(ExecutorBinderPtr input, size_t & executor_index, ASTPtr agg_funcs, ASTPtr group_by_exprs) +{ + std::vector agg_exprs; + std::vector gby_exprs; + DAGSchema output_schema; + bool has_uniq_raw_res = false; + bool need_append_project = false; + if (agg_funcs != nullptr) + { + for (const auto & expr : agg_funcs->children) + { + const auto * func = typeid_cast(expr.get()); + if (!func || !AggregateFunctionFactory::instance().isAggregateFunctionName(func->name)) + { + need_append_project = true; + continue; + } + + agg_exprs.push_back(expr); + std::vector children_ci; + + for (const auto & arg : func->arguments->children) + { + children_ci.push_back(compileExpr(input->output_schema, arg)); + } + + TiDB::ColumnInfo ci; + if (func->name == "count") + { + ci.tp = TiDB::TypeLongLong; + ci.flag = TiDB::ColumnFlagUnsigned | TiDB::ColumnFlagNotNull; + } + else if (func->name == "max" || func->name == "min" || func->name == "first_row" || func->name == "sum") + { + ci = children_ci[0]; + ci.flag &= ~TiDB::ColumnFlagNotNull; + } + else if (func->name == uniq_raw_res_name) + { + has_uniq_raw_res = true; + ci.tp = TiDB::TypeString; + ci.flag = 1; + } + // TODO: Other agg func. + else + { + throw Exception("Unsupported agg function: " + func->name, ErrorCodes::LOGICAL_ERROR); + } + + output_schema.emplace_back(std::make_pair(func->getColumnName(), ci)); + } + } + + if (group_by_exprs != nullptr) + { + for (const auto & child : group_by_exprs->children) + { + gby_exprs.push_back(child); + auto ci = compileExpr(input->output_schema, child); + output_schema.emplace_back(std::make_pair(child->getColumnName(), ci)); + } + } + + auto aggregation = std::make_shared( + executor_index, + output_schema, + has_uniq_raw_res, + need_append_project, + std::move(agg_exprs), + std::move(gby_exprs), + true); + aggregation->children.push_back(input); + return aggregation; +} +} // namespace DB::mock diff --git a/dbms/src/Debug/MockExecutor/AggregationBinder.h b/dbms/src/Debug/MockExecutor/AggregationBinder.h new file mode 100644 index 00000000000..4ece3ff7838 --- /dev/null +++ b/dbms/src/Debug/MockExecutor/AggregationBinder.h @@ -0,0 +1,63 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#pragma once + +#include +#include +#include + +namespace DB::mock +{ +class AggregationBinder : public ExecutorBinder +{ +public: + AggregationBinder(size_t & index_, const DAGSchema & output_schema_, bool has_uniq_raw_res_, bool need_append_project_, ASTs && agg_exprs_, ASTs && gby_exprs_, bool is_final_mode_) + : ExecutorBinder(index_, "aggregation_" + std::to_string(index_), output_schema_) + , has_uniq_raw_res(has_uniq_raw_res_) + , need_append_project(need_append_project_) + , agg_exprs(std::move(agg_exprs_)) + , gby_exprs(std::move(gby_exprs_)) + , is_final_mode(is_final_mode_) + {} + + bool toTiPBExecutor(tipb::Executor * tipb_executor, int32_t collator_id, const MPPInfo & mpp_info, const Context & context) override; + + void columnPrune(std::unordered_set & used_columns) override; + + void toMPPSubPlan(size_t & executor_index, const DAGProperties & properties, std::unordered_map, std::shared_ptr>> & exchange_map) override; + + bool needAppendProject() const; + + size_t exprSize() const; + + bool hasUniqRawRes() const; + +protected: + bool has_uniq_raw_res; + bool need_append_project; + std::vector agg_exprs; + std::vector gby_exprs; + bool is_final_mode; + DAGSchema output_schema_for_partial_agg; + +private: + void buildGroupBy(tipb::Aggregation * agg, int32_t collator_id, const Context & context) const; + void buildAggExpr(tipb::Aggregation * agg, int32_t collator_id, const Context & context) const; + void buildAggFunc(tipb::Expr * agg_func, const ASTFunction * func, int32_t collator_id) const; +}; + +ExecutorBinderPtr compileAggregation(ExecutorBinderPtr input, size_t & executor_index, ASTPtr agg_funcs, ASTPtr group_by_exprs); + +} // namespace DB::mock diff --git a/dbms/src/Debug/MockExecutor/AstToPB.cpp b/dbms/src/Debug/MockExecutor/AstToPB.cpp new file mode 100644 index 00000000000..306d2c24813 --- /dev/null +++ b/dbms/src/Debug/MockExecutor/AstToPB.cpp @@ -0,0 +1,684 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// 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 + +namespace DB +{ +void literalFieldToTiPBExpr(const ColumnInfo & ci, const Field & val_field, tipb::Expr * expr, Int32 collator_id) +{ + *(expr->mutable_field_type()) = columnInfoToFieldType(ci); + expr->mutable_field_type()->set_collate(collator_id); + if (!val_field.isNull()) + { + WriteBufferFromOwnString ss; + switch (ci.tp) + { + case TiDB::TypeLongLong: + case TiDB::TypeLong: + case TiDB::TypeShort: + case TiDB::TypeTiny: + case TiDB::TypeInt24: + if (ci.hasUnsignedFlag()) + { + expr->set_tp(tipb::ExprType::Uint64); + UInt64 val = val_field.safeGet(); + encodeDAGUInt64(val, ss); + } + else + { + expr->set_tp(tipb::ExprType::Int64); + Int64 val = val_field.safeGet(); + encodeDAGInt64(val, ss); + } + break; + case TiDB::TypeFloat: + { + expr->set_tp(tipb::ExprType::Float32); + auto val = static_cast(val_field.safeGet()); + encodeDAGFloat32(val, ss); + break; + } + case TiDB::TypeDouble: + { + expr->set_tp(tipb::ExprType::Float64); + Float64 val = val_field.safeGet(); + encodeDAGFloat64(val, ss); + break; + } + case TiDB::TypeString: + { + expr->set_tp(tipb::ExprType::String); + const auto & val = val_field.safeGet(); + encodeDAGString(val, ss); + break; + } + case TiDB::TypeNewDecimal: + { + expr->set_tp(tipb::ExprType::MysqlDecimal); + encodeDAGDecimal(val_field, ss); + break; + } + case TiDB::TypeDate: + { + expr->set_tp(tipb::ExprType::MysqlTime); + UInt64 val = val_field.safeGet(); + encodeDAGUInt64(MyDate(val).toPackedUInt(), ss); + break; + } + case TiDB::TypeDatetime: + case TiDB::TypeTimestamp: + { + expr->set_tp(tipb::ExprType::MysqlTime); + UInt64 val = val_field.safeGet(); + encodeDAGUInt64(MyDateTime(val).toPackedUInt(), ss); + break; + } + case TiDB::TypeTime: + { + expr->set_tp(tipb::ExprType::MysqlDuration); + Int64 val = val_field.safeGet(); + encodeDAGInt64(val, ss); + break; + } + default: + throw Exception(fmt::format("Type {} does not support literal in function unit test", getDataTypeByColumnInfo(ci)->getName())); + } + expr->set_val(ss.releaseStr()); + } + else + { + expr->set_tp(tipb::ExprType::Null); + } +} + +void literalToPB(tipb::Expr * expr, const Field & value, int32_t collator_id) +{ + DataTypePtr type = applyVisitor(FieldToDataType(), value); + ColumnInfo ci = reverseGetColumnInfo({"", type}, 0, Field(), true); + literalFieldToTiPBExpr(ci, value, expr, collator_id); +} + +String getFunctionNameForConstantFolding(tipb::Expr * expr) +{ + // todo support more function for constant folding + switch (expr->sig()) + { + case tipb::ScalarFuncSig::CastStringAsTime: + return "toMyDateTimeOrNull"; + default: + return ""; + } +} + +void foldConstant(tipb::Expr * expr, int32_t collator_id, const Context & context) +{ + if (expr->tp() == tipb::ScalarFunc) + { + bool all_const = true; + for (const auto & c : expr->children()) + { + if (!isLiteralExpr(c)) + { + all_const = false; + break; + } + } + if (!all_const) + return; + DataTypes arguments_types; + ColumnsWithTypeAndName argument_columns; + for (const auto & c : expr->children()) + { + Field value = decodeLiteral(c); + DataTypePtr flash_type = applyVisitor(FieldToDataType(), value); + DataTypePtr target_type = inferDataType4Literal(c); + ColumnWithTypeAndName column; + column.column = target_type->createColumnConst(1, convertFieldToType(value, *target_type, flash_type.get())); + column.name = exprToString(c, {}) + "_" + target_type->getName(); + column.type = target_type; + arguments_types.emplace_back(target_type); + argument_columns.emplace_back(column); + } + auto func_name = getFunctionNameForConstantFolding(expr); + if (func_name.empty()) + return; + const auto & function_builder_ptr = FunctionFactory::instance().get(func_name, context); + auto function_ptr = function_builder_ptr->build(argument_columns); + if (function_ptr->isSuitableForConstantFolding()) + { + Block block_with_constants(argument_columns); + ColumnNumbers argument_numbers(arguments_types.size()); + for (size_t i = 0, size = arguments_types.size(); i < size; i++) + argument_numbers[i] = i; + size_t result_pos = argument_numbers.size(); + block_with_constants.insert({nullptr, function_ptr->getReturnType(), "result"}); + function_ptr->execute(block_with_constants, argument_numbers, result_pos); + const auto & result_column = block_with_constants.getByPosition(result_pos).column; + if (result_column->isColumnConst()) + { + auto updated_value = (*result_column)[0]; + tipb::FieldType orig_field_type = expr->field_type(); + expr->Clear(); + literalToPB(expr, updated_value, collator_id); + expr->clear_field_type(); + auto * field_type = expr->mutable_field_type(); + (*field_type) = orig_field_type; + } + } + } +} + +void astToPB(const DAGSchema & input, ASTPtr ast, tipb::Expr * expr, int32_t collator_id, const Context & context) +{ + if (auto * id = typeid_cast(ast.get())) + { + identifierToPB(input, id, expr, collator_id); + } + else if (auto * func = typeid_cast(ast.get())) + { + functionToPB(input, func, expr, collator_id, context); + } + else if (auto * lit = typeid_cast(ast.get())) + { + literalToPB(expr, lit->value, collator_id); + } + else + { + throw Exception("Unsupported expression: " + ast->getColumnName(), ErrorCodes::LOGICAL_ERROR); + } +} + +void functionToPB(const DAGSchema & input, ASTFunction * func, tipb::Expr * expr, int32_t collator_id, const Context & context) +{ + /// aggregation function is handled in Aggregation, so just treated as a column + auto ft = checkSchema(input, func->getColumnName()); + if (ft != input.end()) + { + expr->set_tp(tipb::ColumnRef); + *(expr->mutable_field_type()) = columnInfoToFieldType((*ft).second); + WriteBufferFromOwnString ss; + encodeDAGInt64(ft - input.begin(), ss); + expr->set_val(ss.releaseStr()); + return; + } + if (AggregateFunctionFactory::instance().isAggregateFunctionName(func->name)) + { + throw Exception("No such column: " + func->getColumnName(), ErrorCodes::NO_SUCH_COLUMN_IN_TABLE); + } + String func_name_lowercase = Poco::toLower(func->name); + // TODO: Support more functions. + // TODO: Support type inference. + + const auto it_sig = tests::func_name_to_sig.find(func_name_lowercase); + if (it_sig == tests::func_name_to_sig.end()) + { + throw Exception("Unsupported function: " + func_name_lowercase, ErrorCodes::LOGICAL_ERROR); + } + switch (it_sig->second) + { + case tipb::ScalarFuncSig::InInt: + { + tipb::Expr * in_expr = expr; + if (func_name_lowercase == "notin") + { + // notin is transformed into not(in()) by tidb + expr->set_sig(tipb::ScalarFuncSig::UnaryNotInt); + auto * ft = expr->mutable_field_type(); + ft->set_tp(TiDB::TypeLongLong); + ft->set_flag(TiDB::ColumnFlagUnsigned); + expr->set_tp(tipb::ExprType::ScalarFunc); + in_expr = expr->add_children(); + } + in_expr->set_sig(tipb::ScalarFuncSig::InInt); + auto * ft = in_expr->mutable_field_type(); + ft->set_tp(TiDB::TypeLongLong); + ft->set_flag(TiDB::ColumnFlagUnsigned); + ft->set_collate(collator_id); + in_expr->set_tp(tipb::ExprType::ScalarFunc); + for (const auto & child_ast : func->arguments->children) + { + auto * tuple_func = typeid_cast(child_ast.get()); + if (tuple_func != nullptr && tuple_func->name == "tuple") + { + // flatten tuple elements + for (const auto & c : tuple_func->arguments->children) + { + tipb::Expr * child = in_expr->add_children(); + astToPB(input, c, child, collator_id, context); + } + } + else + { + tipb::Expr * child = in_expr->add_children(); + astToPB(input, child_ast, child, collator_id, context); + } + } + return; + } + case tipb::ScalarFuncSig::IfInt: + case tipb::ScalarFuncSig::BitAndSig: + case tipb::ScalarFuncSig::BitOrSig: + case tipb::ScalarFuncSig::BitXorSig: + case tipb::ScalarFuncSig::BitNegSig: + expr->set_sig(it_sig->second); + expr->set_tp(tipb::ExprType::ScalarFunc); + for (size_t i = 0; i < func->arguments->children.size(); i++) + { + const auto & child_ast = func->arguments->children[i]; + tipb::Expr * child = expr->add_children(); + astToPB(input, child_ast, child, collator_id, context); + // todo should infer the return type based on all input types + if ((it_sig->second == tipb::ScalarFuncSig::IfInt && i == 1) + || (it_sig->second != tipb::ScalarFuncSig::IfInt && i == 0)) + *(expr->mutable_field_type()) = child->field_type(); + } + return; + case tipb::ScalarFuncSig::PlusInt: + case tipb::ScalarFuncSig::MinusInt: + { + for (const auto & child_ast : func->arguments->children) + { + tipb::Expr * child = expr->add_children(); + astToPB(input, child_ast, child, collator_id, context); + } + expr->set_sig(it_sig->second); + auto * ft = expr->mutable_field_type(); + ft->set_tp(expr->children(0).field_type().tp()); + ft->set_flag(expr->children(0).field_type().flag()); + ft->set_collate(collator_id); + expr->set_tp(tipb::ExprType::ScalarFunc); + return; + } + case tipb::ScalarFuncSig::LikeSig: + { + expr->set_sig(tipb::ScalarFuncSig::LikeSig); + auto * ft = expr->mutable_field_type(); + ft->set_tp(TiDB::TypeLongLong); + ft->set_flag(TiDB::ColumnFlagUnsigned); + ft->set_collate(collator_id); + expr->set_tp(tipb::ExprType::ScalarFunc); + for (const auto & child_ast : func->arguments->children) + { + tipb::Expr * child = expr->add_children(); + astToPB(input, child_ast, child, collator_id, context); + } + // for like need to add the third argument + *expr->add_children() = constructInt64LiteralTiExpr(92); + return; + } + case tipb::ScalarFuncSig::FromUnixTime2Arg: + if (func->arguments->children.size() == 1) + { + expr->set_sig(tipb::ScalarFuncSig::FromUnixTime1Arg); + auto * ft = expr->mutable_field_type(); + ft->set_tp(TiDB::TypeDatetime); + ft->set_decimal(6); + } + else + { + expr->set_sig(tipb::ScalarFuncSig::FromUnixTime2Arg); + auto * ft = expr->mutable_field_type(); + ft->set_tp(TiDB::TypeString); + } + break; + case tipb::ScalarFuncSig::DateFormatSig: + expr->set_sig(tipb::ScalarFuncSig::DateFormatSig); + expr->mutable_field_type()->set_tp(TiDB::TypeString); + break; + case tipb::ScalarFuncSig::CastIntAsTime: + case tipb::ScalarFuncSig::CastRealAsTime: + case tipb::ScalarFuncSig::CastTimeAsTime: + case tipb::ScalarFuncSig::CastDecimalAsTime: + case tipb::ScalarFuncSig::CastStringAsTime: + { + expr->set_sig(it_sig->second); + auto * ft = expr->mutable_field_type(); + if (it_sig->first.find("datetime")) + { + ft->set_tp(TiDB::TypeDatetime); + } + else + { + ft->set_tp(TiDB::TypeDate); + } + break; + } + case tipb::ScalarFuncSig::CastIntAsReal: + case tipb::ScalarFuncSig::CastRealAsReal: + { + expr->set_sig(it_sig->second); + auto * ft = expr->mutable_field_type(); + ft->set_tp(TiDB::TypeDouble); + ft->set_collate(collator_id); + break; + } + case tipb::ScalarFuncSig::Concat: + { + expr->set_sig(it_sig->second); + auto * ft = expr->mutable_field_type(); + ft->set_tp(TiDB::TypeString); + ft->set_collate(collator_id); + break; + } + case tipb::ScalarFuncSig::RoundInt: + case tipb::ScalarFuncSig::RoundWithFracInt: + { + expr->set_sig(it_sig->second); + auto * ft = expr->mutable_field_type(); + ft->set_tp(TiDB::TypeLongLong); + if (it_sig->first.find("uint") != std::string::npos) + ft->set_flag(TiDB::ColumnFlagUnsigned); + ft->set_collate(collator_id); + break; + } + case tipb::ScalarFuncSig::RoundDec: + case tipb::ScalarFuncSig::RoundWithFracDec: + { + expr->set_sig(it_sig->second); + auto * ft = expr->mutable_field_type(); + ft->set_tp(TiDB::TypeNewDecimal); + ft->set_collate(collator_id); + break; + } + case tipb::ScalarFuncSig::RoundReal: + case tipb::ScalarFuncSig::RoundWithFracReal: + { + expr->set_sig(it_sig->second); + auto * ft = expr->mutable_field_type(); + ft->set_tp(TiDB::TypeDouble); + ft->set_collate(collator_id); + break; + } + default: + { + expr->set_sig(it_sig->second); + auto * ft = expr->mutable_field_type(); + ft->set_tp(TiDB::TypeLongLong); + ft->set_flag(TiDB::ColumnFlagUnsigned); + ft->set_collate(collator_id); + break; + } + } + expr->set_tp(tipb::ExprType::ScalarFunc); + for (const auto & child_ast : func->arguments->children) + { + tipb::Expr * child = expr->add_children(); + astToPB(input, child_ast, child, collator_id, context); + } + foldConstant(expr, collator_id, context); +} + +void identifierToPB(const DAGSchema & input, ASTIdentifier * id, tipb::Expr * expr, int32_t collator_id) +{ + auto ft = checkSchema(input, id->getColumnName()); + if (ft == input.end()) + throw Exception("No such column: " + id->getColumnName(), ErrorCodes::NO_SUCH_COLUMN_IN_TABLE); + expr->set_tp(tipb::ColumnRef); + *(expr->mutable_field_type()) = columnInfoToFieldType((*ft).second); + expr->mutable_field_type()->set_collate(collator_id); + WriteBufferFromOwnString ss; + encodeDAGInt64(ft - input.begin(), ss); + expr->set_val(ss.releaseStr()); +} + +void collectUsedColumnsFromExpr(const DAGSchema & input, ASTPtr ast, std::unordered_set & used_columns) +{ + if (auto * id = typeid_cast(ast.get())) + { + auto [db_name, table_name, column_name] = splitQualifiedName(id->getColumnName()); + if (!table_name.empty()) + used_columns.emplace(id->getColumnName()); + else + { + bool found = false; + for (const auto & field : input) + { + if (splitQualifiedName(field.first).column_name == column_name) + { + if (found) + throw Exception("ambiguous column for " + column_name); + found = true; + used_columns.emplace(field.first); + } + } + } + } + else if (auto * func = typeid_cast(ast.get())) + { + if (AggregateFunctionFactory::instance().isAggregateFunctionName(func->name)) + { + used_columns.emplace(func->getColumnName()); + } + else + { + /// check function + auto ft = checkSchema(input, func->getColumnName()); + if (ft != input.end()) + { + used_columns.emplace(func->getColumnName()); + return; + } + for (const auto & child_ast : func->arguments->children) + { + collectUsedColumnsFromExpr(input, child_ast, used_columns); + } + } + } +} + +TiDB::ColumnInfo compileIdentifier(const DAGSchema & input, ASTIdentifier * id) +{ + TiDB::ColumnInfo ci; + + /// check column + auto ft = checkSchema(input, id->getColumnName()); + if (ft == input.end()) + throw Exception("No such column: " + id->getColumnName(), ErrorCodes::NO_SUCH_COLUMN_IN_TABLE); + ci = ft->second; + + return ci; +} + +TiDB::ColumnInfo compileFunction(const DAGSchema & input, ASTFunction * func) +{ + TiDB::ColumnInfo ci; + /// check function + String func_name_lowercase = Poco::toLower(func->name); + const auto it_sig = tests::func_name_to_sig.find(func_name_lowercase); + if (it_sig == tests::func_name_to_sig.end()) + { + throw Exception("Unsupported function: " + func_name_lowercase, ErrorCodes::LOGICAL_ERROR); + } + switch (it_sig->second) + { + case tipb::ScalarFuncSig::InInt: + ci.tp = TiDB::TypeLongLong; + ci.flag = TiDB::ColumnFlagUnsigned; + for (const auto & child_ast : func->arguments->children) + { + auto * tuple_func = typeid_cast(child_ast.get()); + if (tuple_func != nullptr && tuple_func->name == "tuple") + { + // flatten tuple elements + for (const auto & c : tuple_func->arguments->children) + { + compileExpr(input, c); + } + } + else + { + compileExpr(input, child_ast); + } + } + return ci; + case tipb::ScalarFuncSig::IfInt: + case tipb::ScalarFuncSig::BitAndSig: + case tipb::ScalarFuncSig::BitOrSig: + case tipb::ScalarFuncSig::BitXorSig: + case tipb::ScalarFuncSig::BitNegSig: + for (size_t i = 0; i < func->arguments->children.size(); i++) + { + const auto & child_ast = func->arguments->children[i]; + auto child_ci = compileExpr(input, child_ast); + // todo should infer the return type based on all input types + if ((it_sig->second == tipb::ScalarFuncSig::IfInt && i == 1) + || (it_sig->second != tipb::ScalarFuncSig::IfInt && i == 0)) + ci = child_ci; + } + return ci; + case tipb::ScalarFuncSig::PlusInt: + case tipb::ScalarFuncSig::MinusInt: + return compileExpr(input, func->arguments->children[0]); + case tipb::ScalarFuncSig::LikeSig: + ci.tp = TiDB::TypeLongLong; + ci.flag = TiDB::ColumnFlagUnsigned; + for (const auto & child_ast : func->arguments->children) + { + compileExpr(input, child_ast); + } + return ci; + case tipb::ScalarFuncSig::FromUnixTime2Arg: + if (func->arguments->children.size() == 1) + { + ci.tp = TiDB::TypeDatetime; + ci.decimal = 6; + } + else + { + ci.tp = TiDB::TypeString; + } + break; + case tipb::ScalarFuncSig::DateFormatSig: + ci.tp = TiDB::TypeString; + break; + case tipb::ScalarFuncSig::CastIntAsTime: + case tipb::ScalarFuncSig::CastRealAsTime: + case tipb::ScalarFuncSig::CastTimeAsTime: + case tipb::ScalarFuncSig::CastDecimalAsTime: + case tipb::ScalarFuncSig::CastStringAsTime: + if (it_sig->first.find("datetime")) + { + ci.tp = TiDB::TypeDatetime; + } + else + { + ci.tp = TiDB::TypeDate; + } + break; + case tipb::ScalarFuncSig::CastIntAsReal: + case tipb::ScalarFuncSig::CastRealAsReal: + { + ci.tp = TiDB::TypeDouble; + break; + } + case tipb::ScalarFuncSig::RoundInt: + case tipb::ScalarFuncSig::RoundWithFracInt: + { + ci.tp = TiDB::TypeLongLong; + if (it_sig->first.find("uint") != std::string::npos) + ci.flag = TiDB::ColumnFlagUnsigned; + break; + } + case tipb::ScalarFuncSig::RoundDec: + case tipb::ScalarFuncSig::RoundWithFracDec: + { + ci.tp = TiDB::TypeNewDecimal; + break; + } + case tipb::ScalarFuncSig::RoundReal: + case tipb::ScalarFuncSig::RoundWithFracReal: + { + ci.tp = TiDB::TypeDouble; + break; + } + default: + ci.tp = TiDB::TypeLongLong; + ci.flag = TiDB::ColumnFlagUnsigned; + break; + } + for (const auto & child_ast : func->arguments->children) + { + compileExpr(input, child_ast); + } + return ci; +} + +TiDB::ColumnInfo compileLiteral(ASTLiteral * lit) +{ + TiDB::ColumnInfo ci; + switch (lit->value.getType()) + { + case Field::Types::Which::Null: + ci.tp = TiDB::TypeNull; + // Null literal expr doesn't need value. + break; + case Field::Types::Which::UInt64: + ci.tp = TiDB::TypeLongLong; + ci.flag = TiDB::ColumnFlagUnsigned; + break; + case Field::Types::Which::Int64: + ci.tp = TiDB::TypeLongLong; + break; + case Field::Types::Which::Float64: + ci.tp = TiDB::TypeDouble; + break; + case Field::Types::Which::Decimal32: + case Field::Types::Which::Decimal64: + case Field::Types::Which::Decimal128: + case Field::Types::Which::Decimal256: + ci.tp = TiDB::TypeNewDecimal; + break; + case Field::Types::Which::String: + ci.tp = TiDB::TypeString; + break; + default: + throw Exception(String("Unsupported literal type: ") + lit->value.getTypeName(), ErrorCodes::LOGICAL_ERROR); + } + return ci; +} + +TiDB::ColumnInfo compileExpr(const DAGSchema & input, ASTPtr ast) +{ + if (auto * id = typeid_cast(ast.get())) + return compileIdentifier(input, id); + else if (auto * func = typeid_cast(ast.get())) + return compileFunction(input, func); + else if (auto * lit = typeid_cast(ast.get())) + return compileLiteral(lit); + else + { + /// not supported unless this is a literal + throw Exception("Unsupported expression: " + ast->getColumnName(), ErrorCodes::LOGICAL_ERROR); + } +} + +void compileFilter(const DAGSchema & input, ASTPtr ast, std::vector & conditions) +{ + if (auto * func = typeid_cast(ast.get())) + { + if (func->name == "and") + { + for (auto & child : func->arguments->children) + { + compileFilter(input, child, conditions); + } + return; + } + } + conditions.push_back(ast); + compileExpr(input, ast); +} + +} // namespace DB diff --git a/dbms/src/Debug/MockExecutor/AstToPB.h b/dbms/src/Debug/MockExecutor/AstToPB.h new file mode 100644 index 00000000000..518d04f89b9 --- /dev/null +++ b/dbms/src/Debug/MockExecutor/AstToPB.h @@ -0,0 +1,98 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ +extern const int BAD_ARGUMENTS; +extern const int LOGICAL_ERROR; +extern const int NO_SUCH_COLUMN_IN_TABLE; +} // namespace ErrorCodes +struct MPPCtx +{ + Timestamp start_ts; + Int64 next_task_id; + std::vector sender_target_task_ids; + explicit MPPCtx(Timestamp start_ts_) + : start_ts(start_ts_) + , next_task_id(1) + {} +}; + +using MPPCtxPtr = std::shared_ptr; + +struct MPPInfo +{ + Timestamp start_ts; + Int64 partition_id; + Int64 task_id; + const std::vector sender_target_task_ids; + const std::unordered_map> receiver_source_task_ids_map; + + MPPInfo( + Timestamp start_ts_, + Int64 partition_id_, + Int64 task_id_, + const std::vector & sender_target_task_ids_, + const std::unordered_map> & receiver_source_task_ids_map_) + : start_ts(start_ts_) + , partition_id(partition_id_) + , task_id(task_id_) + , sender_target_task_ids(sender_target_task_ids_) + , receiver_source_task_ids_map(receiver_source_task_ids_map_) + {} +}; + +struct TaskMeta +{ + UInt64 start_ts = 0; + Int64 task_id = 0; + Int64 partition_id = 0; +}; + +using TaskMetas = std::vector; +void literalFieldToTiPBExpr(const ColumnInfo & ci, const Field & field, tipb::Expr * expr, Int32 collator_id); +void literalToPB(tipb::Expr * expr, const Field & value, int32_t collator_id); +String getFunctionNameForConstantFolding(tipb::Expr * expr); +void foldConstant(tipb::Expr * expr, int32_t collator_id, const Context & context); +void functionToPB(const DAGSchema & input, ASTFunction * func, tipb::Expr * expr, int32_t collator_id, const Context & context); +void identifierToPB(const DAGSchema & input, ASTIdentifier * id, tipb::Expr * expr, int32_t collator_id); +void astToPB(const DAGSchema & input, ASTPtr ast, tipb::Expr * expr, int32_t collator_id, const Context & context); +void collectUsedColumnsFromExpr(const DAGSchema & input, ASTPtr ast, std::unordered_set & used_columns); +TiDB::ColumnInfo compileExpr(const DAGSchema & input, ASTPtr ast); +void compileFilter(const DAGSchema & input, ASTPtr ast, std::vector & conditions); + +} // namespace DB diff --git a/dbms/src/Debug/MockExecutor/astToExecutorUtils.cpp b/dbms/src/Debug/MockExecutor/AstToPBUtils.cpp similarity index 97% rename from dbms/src/Debug/MockExecutor/astToExecutorUtils.cpp rename to dbms/src/Debug/MockExecutor/AstToPBUtils.cpp index 1933486cf2b..869aa8b4d13 100644 --- a/dbms/src/Debug/MockExecutor/astToExecutorUtils.cpp +++ b/dbms/src/Debug/MockExecutor/AstToPBUtils.cpp @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include +#include namespace DB { @@ -41,7 +41,6 @@ ColumnName splitQualifiedName(const String & s) return ret; } - DAGSchema::const_iterator checkSchema(const DAGSchema & input, const String & checked_column) { auto ft = std::find_if(input.begin(), input.end(), [&checked_column](const auto & field) { diff --git a/dbms/src/Debug/MockExecutor/astToExecutorUtils.h b/dbms/src/Debug/MockExecutor/AstToPBUtils.h similarity index 100% rename from dbms/src/Debug/MockExecutor/astToExecutorUtils.h rename to dbms/src/Debug/MockExecutor/AstToPBUtils.h diff --git a/dbms/src/Debug/MockExecutor/ExchangeReceiverBinder.cpp b/dbms/src/Debug/MockExecutor/ExchangeReceiverBinder.cpp new file mode 100644 index 00000000000..e7f0491b74f --- /dev/null +++ b/dbms/src/Debug/MockExecutor/ExchangeReceiverBinder.cpp @@ -0,0 +1,61 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// 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 + +namespace DB::mock +{ +bool ExchangeReceiverBinder::toTiPBExecutor(tipb::Executor * tipb_executor, int32_t collator_id, const MPPInfo & mpp_info, const Context & context) +{ + tipb_executor->set_tp(tipb::ExecType::TypeExchangeReceiver); + tipb_executor->set_executor_id(name); + tipb_executor->set_fine_grained_shuffle_stream_count(fine_grained_shuffle_stream_count); + tipb::ExchangeReceiver * exchange_receiver = tipb_executor->mutable_exchange_receiver(); + + for (auto & field : output_schema) + { + auto tipb_type = TiDB::columnInfoToFieldType(field.second); + tipb_type.set_collate(collator_id); + + auto * field_type = exchange_receiver->add_field_types(); + *field_type = tipb_type; + } + + auto it = mpp_info.receiver_source_task_ids_map.find(name); + if (it == mpp_info.receiver_source_task_ids_map.end()) + throw Exception("Can not found mpp receiver info"); + + auto size = it->second.size(); + for (size_t i = 0; i < size; ++i) + { + mpp::TaskMeta meta; + meta.set_start_ts(mpp_info.start_ts); + meta.set_task_id(it->second[i]); + meta.set_partition_id(i); + auto addr = context.isMPPTest() ? tests::MockComputeServerManager::instance().getServerConfigMap()[i].addr : Debug::LOCAL_HOST; + meta.set_address(addr); + auto * meta_string = exchange_receiver->add_encoded_task_meta(); + meta.AppendToString(meta_string); + } + return true; +} + + +ExecutorBinderPtr compileExchangeReceiver(size_t & executor_index, DAGSchema schema, uint64_t fine_grained_shuffle_stream_count) +{ + ExecutorBinderPtr exchange_receiver = std::make_shared(executor_index, schema, fine_grained_shuffle_stream_count); + return exchange_receiver; +} +} // namespace DB::mock diff --git a/dbms/src/Debug/MockExecutor/ExchangeReceiverBinder.h b/dbms/src/Debug/MockExecutor/ExchangeReceiverBinder.h new file mode 100644 index 00000000000..2885dfd895d --- /dev/null +++ b/dbms/src/Debug/MockExecutor/ExchangeReceiverBinder.h @@ -0,0 +1,39 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// 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 + +namespace DB::mock +{ +class ExchangeReceiverBinder : public ExecutorBinder +{ +public: + ExchangeReceiverBinder(size_t & index, const DAGSchema & output, uint64_t fine_grained_shuffle_stream_count_ = 0) + : ExecutorBinder(index, "exchange_receiver_" + std::to_string(index), output) + , fine_grained_shuffle_stream_count(fine_grained_shuffle_stream_count_) + {} + + bool toTiPBExecutor(tipb::Executor * tipb_executor, int32_t collator_id, const MPPInfo & mpp_info, const Context &) override; + + void columnPrune(std::unordered_set &) override {} + +private: + TaskMetas task_metas; + uint64_t fine_grained_shuffle_stream_count; +}; + +ExecutorBinderPtr compileExchangeReceiver(size_t & executor_index, DAGSchema schema, uint64_t fine_grained_shuffle_stream_count); +} // namespace DB::mock diff --git a/dbms/src/Debug/MockExecutor/ExchangeSenderBinder.cpp b/dbms/src/Debug/MockExecutor/ExchangeSenderBinder.cpp new file mode 100644 index 00000000000..065d983cb60 --- /dev/null +++ b/dbms/src/Debug/MockExecutor/ExchangeSenderBinder.cpp @@ -0,0 +1,76 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// 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 + +namespace DB::mock +{ +bool ExchangeSenderBinder::toTiPBExecutor(tipb::Executor * tipb_executor, int32_t collator_id, const MPPInfo & mpp_info, const Context & context) +{ + tipb_executor->set_tp(tipb::ExecType::TypeExchangeSender); + tipb_executor->set_executor_id(name); + tipb::ExchangeSender * exchange_sender = tipb_executor->mutable_exchange_sender(); + exchange_sender->set_tp(type); + for (auto i : partition_keys) + { + auto * expr = exchange_sender->add_partition_keys(); + expr->set_tp(tipb::ColumnRef); + WriteBufferFromOwnString ss; + encodeDAGInt64(i, ss); + expr->set_val(ss.releaseStr()); + auto tipb_type = TiDB::columnInfoToFieldType(output_schema[i].second); + *expr->mutable_field_type() = tipb_type; + tipb_type.set_collate(collator_id); + *exchange_sender->add_types() = tipb_type; + } + + int i = 0; + for (auto task_id : mpp_info.sender_target_task_ids) + { + mpp::TaskMeta meta; + meta.set_start_ts(mpp_info.start_ts); + meta.set_task_id(task_id); + meta.set_partition_id(i); + auto addr = context.isMPPTest() ? tests::MockComputeServerManager::instance().getServerConfigMap()[i++].addr : Debug::LOCAL_HOST; + meta.set_address(addr); + + auto * meta_string = exchange_sender->add_encoded_task_meta(); + meta.AppendToString(meta_string); + } + + for (auto & field : output_schema) + { + auto tipb_type = TiDB::columnInfoToFieldType(field.second); + tipb_type.set_collate(collator_id); + auto * field_type = exchange_sender->add_all_field_types(); + *field_type = tipb_type; + } + + auto * child_executor = exchange_sender->mutable_child(); + return children[0]->toTiPBExecutor(child_executor, collator_id, mpp_info, context); +} + +tipb::ExchangeType ExchangeSenderBinder::getType() const +{ + return type; +} + +ExecutorBinderPtr compileExchangeSender(ExecutorBinderPtr input, size_t & executor_index, tipb::ExchangeType exchange_type) +{ + ExecutorBinderPtr exchange_sender = std::make_shared(executor_index, input->output_schema, exchange_type); + exchange_sender->children.push_back(input); + return exchange_sender; +} +} // namespace DB::mock diff --git a/dbms/src/Debug/MockExecutor/ExchangeSenderBinder.h b/dbms/src/Debug/MockExecutor/ExchangeSenderBinder.h new file mode 100644 index 00000000000..0b8b33821cf --- /dev/null +++ b/dbms/src/Debug/MockExecutor/ExchangeSenderBinder.h @@ -0,0 +1,43 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// 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 + +namespace DB::mock +{ +class ExchangeSenderBinder : public ExecutorBinder +{ +public: + ExchangeSenderBinder(size_t & index, const DAGSchema & output, tipb::ExchangeType type_, const std::vector & partition_keys_ = {}) + : ExecutorBinder(index, "exchange_sender_" + std::to_string(index), output) + , type(type_) + , partition_keys(partition_keys_) + {} + + bool toTiPBExecutor(tipb::Executor * tipb_executor, int32_t collator_id, const MPPInfo & mpp_info, const Context & context) override; + + void columnPrune(std::unordered_set &) override {} + + tipb::ExchangeType getType() const; + +private: + tipb::ExchangeType type; + TaskMetas task_metas; + std::vector partition_keys; +}; + +ExecutorBinderPtr compileExchangeSender(ExecutorBinderPtr input, size_t & executor_index, tipb::ExchangeType exchange_type); +} // namespace DB::mock diff --git a/dbms/src/Debug/MockExecutor/ExecutorBinder.h b/dbms/src/Debug/MockExecutor/ExecutorBinder.h new file mode 100644 index 00000000000..de8e3c9928c --- /dev/null +++ b/dbms/src/Debug/MockExecutor/ExecutorBinder.h @@ -0,0 +1,63 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#pragma once + +#include +#include +#include +#include + + +namespace DB::mock +{ +class ExchangeSenderBinder; +class ExchangeReceiverBinder; + + +// Convert CH AST to tipb::Executor +// Used in integration test framework and Unit test framework. +class ExecutorBinder +{ +public: + size_t index [[maybe_unused]]; + String name; + DB::DAGSchema output_schema; + std::vector> children; + +public: + ExecutorBinder(size_t & index_, String && name_, const DAGSchema & output_schema_) + : index(index_) + , name(std::move(name_)) + , output_schema(output_schema_) + { + index_++; + } + + std::vector> getChildren() + { + return children; + } + + virtual void columnPrune(std::unordered_set & used_columns) = 0; + virtual bool toTiPBExecutor(tipb::Executor * tipb_executor, int32_t collator_id, const MPPInfo & mpp_info, const Context & context) = 0; + virtual void toMPPSubPlan(size_t & executor_index, const DAGProperties & properties, std::unordered_map, std::shared_ptr>> & exchange_map) + { + children[0]->toMPPSubPlan(executor_index, properties, exchange_map); + } + virtual ~ExecutorBinder() = default; +}; + +using ExecutorBinderPtr = std::shared_ptr; +} // namespace DB::mock diff --git a/dbms/src/Debug/MockExecutor/funcSigs.h b/dbms/src/Debug/MockExecutor/FuncSigMap.cpp similarity index 99% rename from dbms/src/Debug/MockExecutor/funcSigs.h rename to dbms/src/Debug/MockExecutor/FuncSigMap.cpp index 4c45a4a5736..a6698cb3db9 100644 --- a/dbms/src/Debug/MockExecutor/funcSigs.h +++ b/dbms/src/Debug/MockExecutor/FuncSigMap.cpp @@ -11,9 +11,6 @@ // 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 diff --git a/dbms/src/Debug/MockExecutor/FuncSigMap.h b/dbms/src/Debug/MockExecutor/FuncSigMap.h new file mode 100644 index 00000000000..5d024a66f36 --- /dev/null +++ b/dbms/src/Debug/MockExecutor/FuncSigMap.h @@ -0,0 +1,25 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// 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 + +namespace DB::tests +{ +extern std::unordered_map func_name_to_sig; +extern std::unordered_map agg_func_name_to_sig; +extern std::unordered_map window_func_name_to_sig; +} // namespace DB::tests diff --git a/dbms/src/Debug/MockExecutor/JoinBinder.cpp b/dbms/src/Debug/MockExecutor/JoinBinder.cpp new file mode 100644 index 00000000000..92109b73f1b --- /dev/null +++ b/dbms/src/Debug/MockExecutor/JoinBinder.cpp @@ -0,0 +1,338 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// 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 + +namespace DB::mock +{ +void JoinBinder::columnPrune(std::unordered_set & used_columns) +{ + std::unordered_set left_columns; + std::unordered_set right_columns; + + for (auto & field : children[0]->output_schema) + { + auto [db_name, table_name, column_name] = splitQualifiedName(field.first); + left_columns.emplace(table_name + "." + column_name); + } + + for (auto & field : children[1]->output_schema) + { + auto [db_name, table_name, column_name] = splitQualifiedName(field.first); + right_columns.emplace(table_name + "." + column_name); + } + std::unordered_set left_used_columns; + std::unordered_set right_used_columns; + + for (const auto & s : used_columns) + { + auto [db_name, table_name, col_name] = splitQualifiedName(s); + auto t = table_name + "." + col_name; + if (left_columns.find(t) != left_columns.end()) + left_used_columns.emplace(t); + + if (right_columns.find(t) != right_columns.end()) + right_used_columns.emplace(t); + } + + for (const auto & child : join_cols) + { + if (auto * identifier = typeid_cast(child.get())) + { + auto col_name = identifier->getColumnName(); + for (auto & field : children[0]->output_schema) + { + auto [db_name, table_name, column_name] = splitQualifiedName(field.first); + if (col_name == column_name) + { + left_used_columns.emplace(table_name + "." + column_name); + break; + } + } + for (auto & field : children[1]->output_schema) + { + auto [db_name, table_name, column_name] = splitQualifiedName(field.first); + if (col_name == column_name) + { + right_used_columns.emplace(table_name + "." + column_name); + break; + } + } + } + else + { + throw Exception("Only support Join on columns"); + } + } + + children[0]->columnPrune(left_used_columns); + children[1]->columnPrune(right_used_columns); + + /// update output schema + output_schema.clear(); + + for (auto & field : children[0]->output_schema) + { + if (tp == tipb::TypeRightOuterJoin && field.second.hasNotNullFlag()) + output_schema.push_back(toNullableDAGColumnInfo(field)); + else + output_schema.push_back(field); + } + + for (auto & field : children[1]->output_schema) + { + if (tp == tipb::TypeLeftOuterJoin && field.second.hasNotNullFlag()) + output_schema.push_back(toNullableDAGColumnInfo(field)); + else + output_schema.push_back(field); + } +} + +void JoinBinder::fillJoinKeyAndFieldType( + ASTPtr key, + const DAGSchema & child_schema, + tipb::Expr * tipb_key, + tipb::FieldType * tipb_field_type, + int32_t collator_id) +{ + auto * identifier = typeid_cast(key.get()); + for (size_t index = 0; index < child_schema.size(); ++index) + { + const auto & [col_name, col_info] = child_schema[index]; + + if (splitQualifiedName(col_name).column_name == identifier->getColumnName()) + { + auto tipb_type = TiDB::columnInfoToFieldType(col_info); + tipb_type.set_collate(collator_id); + + tipb_key->set_tp(tipb::ColumnRef); + WriteBufferFromOwnString ss; + encodeDAGInt64(index, ss); + tipb_key->set_val(ss.releaseStr()); + *tipb_key->mutable_field_type() = tipb_type; + + *tipb_field_type = tipb_type; + break; + } + } +} + +bool JoinBinder::toTiPBExecutor(tipb::Executor * tipb_executor, int32_t collator_id, const MPPInfo & mpp_info, const Context & context) +{ + tipb_executor->set_tp(tipb::ExecType::TypeJoin); + tipb_executor->set_executor_id(name); + + tipb::Join * join = tipb_executor->mutable_join(); + + join->set_join_type(tp); + join->set_join_exec_type(tipb::JoinExecType::TypeHashJoin); + join->set_inner_idx(1); + + for (const auto & key : join_cols) + { + fillJoinKeyAndFieldType(key, children[0]->output_schema, join->add_left_join_keys(), join->add_probe_types(), collator_id); + fillJoinKeyAndFieldType(key, children[1]->output_schema, join->add_right_join_keys(), join->add_build_types(), collator_id); + } + + for (const auto & expr : left_conds) + { + tipb::Expr * cond = join->add_left_conditions(); + astToPB(children[0]->output_schema, expr, cond, collator_id, context); + } + + for (const auto & expr : right_conds) + { + tipb::Expr * cond = join->add_right_conditions(); + astToPB(children[1]->output_schema, expr, cond, collator_id, context); + } + + DAGSchema merged_children_schema{children[0]->output_schema}; + merged_children_schema.insert(merged_children_schema.end(), children[1]->output_schema.begin(), children[1]->output_schema.end()); + + for (const auto & expr : other_conds) + { + tipb::Expr * cond = join->add_other_conditions(); + astToPB(merged_children_schema, expr, cond, collator_id, context); + } + + for (const auto & expr : other_eq_conds_from_in) + { + tipb::Expr * cond = join->add_other_eq_conditions_from_in(); + astToPB(merged_children_schema, expr, cond, collator_id, context); + } + + auto * left_child_executor = join->add_children(); + children[0]->toTiPBExecutor(left_child_executor, collator_id, mpp_info, context); + auto * right_child_executor = join->add_children(); + return children[1]->toTiPBExecutor(right_child_executor, collator_id, mpp_info, context); +} + +void JoinBinder::toMPPSubPlan(size_t & executor_index, const DAGProperties & properties, std::unordered_map, std::shared_ptr>> & exchange_map) +{ + if (properties.use_broadcast_join) + { + /// for broadcast join, always use right side as the broadcast side + std::shared_ptr right_exchange_sender + = std::make_shared(executor_index, children[1]->output_schema, tipb::Broadcast); + right_exchange_sender->children.push_back(children[1]); + + std::shared_ptr right_exchange_receiver + = std::make_shared(executor_index, children[1]->output_schema); + children[1] = right_exchange_receiver; + exchange_map[right_exchange_receiver->name] = std::make_pair(right_exchange_receiver, right_exchange_sender); + return; + } + + std::vector left_partition_keys; + std::vector right_partition_keys; + + auto push_back_partition_key = [](auto & partition_keys, const auto & child_schema, const auto & key) { + for (size_t index = 0; index < child_schema.size(); ++index) + { + if (splitQualifiedName(child_schema[index].first).column_name == key->getColumnName()) + { + partition_keys.push_back(index); + break; + } + } + }; + + for (const auto & key : join_cols) + { + push_back_partition_key(left_partition_keys, children[0]->output_schema, key); + push_back_partition_key(right_partition_keys, children[1]->output_schema, key); + } + + std::shared_ptr left_exchange_sender + = std::make_shared(executor_index, children[0]->output_schema, tipb::Hash, left_partition_keys); + left_exchange_sender->children.push_back(children[0]); + std::shared_ptr right_exchange_sender + = std::make_shared(executor_index, children[1]->output_schema, tipb::Hash, right_partition_keys); + right_exchange_sender->children.push_back(children[1]); + + std::shared_ptr left_exchange_receiver + = std::make_shared(executor_index, children[0]->output_schema); + std::shared_ptr right_exchange_receiver + = std::make_shared(executor_index, children[1]->output_schema); + children[0] = left_exchange_receiver; + children[1] = right_exchange_receiver; + + exchange_map[left_exchange_receiver->name] = std::make_pair(left_exchange_receiver, left_exchange_sender); + exchange_map[right_exchange_receiver->name] = std::make_pair(right_exchange_receiver, right_exchange_sender); +} + +static void buildLeftSideJoinSchema(DAGSchema & schema, const DAGSchema & left_schema, tipb::JoinType tp) +{ + for (const auto & field : left_schema) + { + if (tp == tipb::JoinType::TypeRightOuterJoin && field.second.hasNotNullFlag()) + schema.push_back(toNullableDAGColumnInfo(field)); + else + schema.push_back(field); + } +} + +static void buildRightSideJoinSchema(DAGSchema & schema, const DAGSchema & right_schema, tipb::JoinType tp) +{ + /// Note: for semi join, the right table column is ignored + /// but for (anti) left outer semi join, a 1/0 (uint8) field is pushed back + /// indicating whether right table has matching row(s), see comment in ASTTableJoin::Kind for details. + if (tp == tipb::JoinType::TypeLeftOuterSemiJoin || tp == tipb::JoinType::TypeAntiLeftOuterSemiJoin) + { + tipb::FieldType field_type{}; + field_type.set_tp(TiDB::TypeTiny); + field_type.set_charset("binary"); + field_type.set_collate(TiDB::ITiDBCollator::BINARY); + field_type.set_flag(0); + field_type.set_flen(-1); + field_type.set_decimal(-1); + schema.push_back(std::make_pair("", TiDB::fieldTypeToColumnInfo(field_type))); + } + else if (tp != tipb::JoinType::TypeSemiJoin && tp != tipb::JoinType::TypeAntiSemiJoin) + { + for (const auto & field : right_schema) + { + if (tp == tipb::JoinType::TypeLeftOuterJoin && field.second.hasNotNullFlag()) + schema.push_back(toNullableDAGColumnInfo(field)); + else + schema.push_back(field); + } + } +} + +// compileJoin constructs a mocked Join executor node, note that all conditional expression params can be default +ExecutorBinderPtr compileJoin(size_t & executor_index, + ExecutorBinderPtr left, + ExecutorBinderPtr right, + tipb::JoinType tp, + const ASTs & join_cols, + const ASTs & left_conds, + const ASTs & right_conds, + const ASTs & other_conds, + const ASTs & other_eq_conds_from_in) +{ + DAGSchema output_schema; + + buildLeftSideJoinSchema(output_schema, left->output_schema, tp); + buildRightSideJoinSchema(output_schema, right->output_schema, tp); + + auto join = std::make_shared(executor_index, output_schema, tp, join_cols, left_conds, right_conds, other_conds, other_eq_conds_from_in); + join->children.push_back(left); + join->children.push_back(right); + + return join; +} + +/// Note: this api is only used by legacy test framework for compatibility purpose, which will be depracated soon, +/// so please avoid using it. +/// Old executor test framework bases on ch's parser to translate sql string to ast tree, then manually to DAGRequest. +/// However, as for join executor, this translation, from ASTTableJoin to tipb::Join, is not a one-to-one mapping +/// because of the different join classification model used by these two structures. Therefore, under old test framework, +/// it is hard to fully test join executor. New framework aims to directly construct DAGRequest, so new framework APIs for join should +/// avoid using ASTTableJoin. +ExecutorBinderPtr compileJoin(size_t & executor_index, ExecutorBinderPtr left, ExecutorBinderPtr right, ASTPtr params) +{ + tipb::JoinType tp; + const auto & ast_join = (static_cast(*params)); + switch (ast_join.kind) + { + case ASTTableJoin::Kind::Inner: + tp = tipb::JoinType::TypeInnerJoin; + break; + case ASTTableJoin::Kind::Left: + tp = tipb::JoinType::TypeLeftOuterJoin; + break; + case ASTTableJoin::Kind::Right: + tp = tipb::JoinType::TypeRightOuterJoin; + break; + default: + throw Exception("Unsupported join type"); + } + + // in legacy test framework, we only support using_expr of join + ASTs join_cols; + if (ast_join.using_expression_list) + { + for (const auto & key : ast_join.using_expression_list->children) + { + join_cols.push_back(key); + } + } + return compileJoin(executor_index, left, right, tp, join_cols); +} +} // namespace DB::mock diff --git a/dbms/src/Debug/MockExecutor/JoinBinder.h b/dbms/src/Debug/MockExecutor/JoinBinder.h new file mode 100644 index 00000000000..5ab1fb83f4b --- /dev/null +++ b/dbms/src/Debug/MockExecutor/JoinBinder.h @@ -0,0 +1,73 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#pragma once + +#include +#include +#include + +namespace DB::mock +{ +class JoinBinder : public ExecutorBinder +{ +public: + JoinBinder(size_t & index_, const DAGSchema & output_schema_, tipb::JoinType tp_, const ASTs & join_cols_, const ASTs & l_conds, const ASTs & r_conds, const ASTs & o_conds, const ASTs & o_eq_conds) + : ExecutorBinder(index_, "Join_" + std::to_string(index_), output_schema_) + , tp(tp_) + , join_cols(join_cols_) + , left_conds(l_conds) + , right_conds(r_conds) + , other_conds(o_conds) + , other_eq_conds_from_in(o_eq_conds) + { + if (!(join_cols.size() + left_conds.size() + right_conds.size() + other_conds.size() + other_eq_conds_from_in.size())) + throw Exception("No join condition found."); + } + + void columnPrune(std::unordered_set & used_columns) override; + + static void fillJoinKeyAndFieldType( + ASTPtr key, + const DAGSchema & schema, + tipb::Expr * tipb_key, + tipb::FieldType * tipb_field_type, + int32_t collator_id); + + bool toTiPBExecutor(tipb::Executor * tipb_executor, int32_t collator_id, const MPPInfo & mpp_info, const Context & context) override; + + void toMPPSubPlan(size_t & executor_index, const DAGProperties & properties, std::unordered_map, std::shared_ptr>> & exchange_map) override; + +protected: + tipb::JoinType tp; + + const ASTs join_cols{}; + const ASTs left_conds{}; + const ASTs right_conds{}; + const ASTs other_conds{}; + const ASTs other_eq_conds_from_in{}; +}; +// compileJoin constructs a mocked Join executor node, note that all conditional expression params can be default +ExecutorBinderPtr compileJoin(size_t & executor_index, ExecutorBinderPtr left, ExecutorBinderPtr right, tipb::JoinType tp, const ASTs & join_cols, const ASTs & left_conds = {}, const ASTs & right_conds = {}, const ASTs & other_conds = {}, const ASTs & other_eq_conds_from_in = {}); + + +/// Note: this api is only used by legacy test framework for compatibility purpose, which will be depracated soon, +/// so please avoid using it. +/// Old executor test framework bases on ch's parser to translate sql string to ast tree, then manually to DAGRequest. +/// However, as for join executor, this translation, from ASTTableJoin to tipb::Join, is not a one-to-one mapping +/// because of the different join classification model used by these two structures. Therefore, under old test framework, +/// it is hard to fully test join executor. New framework aims to directly construct DAGRequest, so new framework APIs for join should +/// avoid using ASTTableJoin. +ExecutorBinderPtr compileJoin(size_t & executor_index, ExecutorBinderPtr left, ExecutorBinderPtr right, ASTPtr params); +} // namespace DB::mock diff --git a/dbms/src/Debug/MockExecutor/LimitBinder.cpp b/dbms/src/Debug/MockExecutor/LimitBinder.cpp new file mode 100644 index 00000000000..c0a9bf17a82 --- /dev/null +++ b/dbms/src/Debug/MockExecutor/LimitBinder.cpp @@ -0,0 +1,44 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// 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 + +namespace DB::mock +{ +bool LimitBinder::toTiPBExecutor(tipb::Executor * tipb_executor, int32_t collator_id, const MPPInfo & mpp_info, const Context & context) +{ + tipb_executor->set_tp(tipb::ExecType::TypeLimit); + tipb_executor->set_executor_id(name); + tipb::Limit * lt = tipb_executor->mutable_limit(); + lt->set_limit(limit); + auto * child_executor = lt->mutable_child(); + return children[0]->toTiPBExecutor(child_executor, collator_id, mpp_info, context); +} + +void LimitBinder::columnPrune(std::unordered_set & used_columns) +{ + children[0]->columnPrune(used_columns); + /// update output schema after column prune + output_schema = children[0]->output_schema; +} + +ExecutorBinderPtr compileLimit(ExecutorBinderPtr input, size_t & executor_index, ASTPtr limit_expr) +{ + auto limit_length = safeGet(typeid_cast(*limit_expr).value); + auto limit = std::make_shared(executor_index, input->output_schema, limit_length); + limit->children.push_back(input); + return limit; +} +} // namespace DB::mock diff --git a/dbms/src/Debug/MockExecutor/LimitBinder.h b/dbms/src/Debug/MockExecutor/LimitBinder.h new file mode 100644 index 00000000000..51bef6c68e7 --- /dev/null +++ b/dbms/src/Debug/MockExecutor/LimitBinder.h @@ -0,0 +1,38 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// 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 + +namespace DB::mock +{ +class LimitBinder : public ExecutorBinder +{ +public: + LimitBinder(size_t & index_, const DAGSchema & output_schema_, size_t limit_) + : ExecutorBinder(index_, "limit_" + std::to_string(index_), output_schema_) + , limit(limit_) + {} + + bool toTiPBExecutor(tipb::Executor * tipb_executor, int32_t collator_id, const MPPInfo & mpp_info, const Context & context) override; + + void columnPrune(std::unordered_set & used_columns) override; + +private: + size_t limit; +}; + +ExecutorBinderPtr compileLimit(ExecutorBinderPtr input, size_t & executor_index, ASTPtr limit_expr); +} // namespace DB::mock diff --git a/dbms/src/Debug/MockExecutor/ProjectBinder.cpp b/dbms/src/Debug/MockExecutor/ProjectBinder.cpp new file mode 100644 index 00000000000..ebe8e5d8bde --- /dev/null +++ b/dbms/src/Debug/MockExecutor/ProjectBinder.cpp @@ -0,0 +1,121 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// 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 + +namespace DB::mock +{ +bool ProjectBinder::toTiPBExecutor(tipb::Executor * tipb_executor, int32_t collator_id, const MPPInfo & mpp_info, const Context & context) +{ + tipb_executor->set_tp(tipb::ExecType::TypeProjection); + tipb_executor->set_executor_id(name); + auto * proj = tipb_executor->mutable_projection(); + auto & input_schema = children[0]->output_schema; + for (const auto & child : exprs) + { + if (typeid_cast(child.get())) + { + /// special case, select * + for (size_t i = 0; i < input_schema.size(); ++i) + { + tipb::Expr * expr = proj->add_exprs(); + expr->set_tp(tipb::ColumnRef); + *(expr->mutable_field_type()) = columnInfoToFieldType(input_schema[i].second); + WriteBufferFromOwnString ss; + encodeDAGInt64(i, ss); + expr->set_val(ss.releaseStr()); + } + continue; + } + tipb::Expr * expr = proj->add_exprs(); + astToPB(input_schema, child, expr, collator_id, context); + } + auto * children_executor = proj->mutable_child(); + return children[0]->toTiPBExecutor(children_executor, collator_id, mpp_info, context); +} + +void ProjectBinder::columnPrune(std::unordered_set & used_columns) +{ + output_schema.erase(std::remove_if(output_schema.begin(), output_schema.end(), [&](const auto & field) { return used_columns.count(field.first) == 0; }), + output_schema.end()); + std::unordered_set used_input_columns; + for (auto & expr : exprs) + { + if (typeid_cast(expr.get())) + { + /// for select *, just add all its input columns, maybe + /// can do some optimization, but it is not worth for mock + /// tests + for (auto & field : children[0]->output_schema) + { + used_input_columns.emplace(field.first); + } + break; + } + if (used_columns.find(expr->getColumnName()) != used_columns.end()) + { + collectUsedColumnsFromExpr(children[0]->output_schema, expr, used_input_columns); + } + } + children[0]->columnPrune(used_input_columns); +} + +ExecutorBinderPtr compileProject(ExecutorBinderPtr input, size_t & executor_index, ASTPtr select_list) +{ + std::vector exprs; + DAGSchema output_schema; + for (const auto & expr : select_list->children) + { + if (typeid_cast(expr.get())) + { + /// special case, select * + exprs.push_back(expr); + const auto & last_output = input->output_schema; + for (const auto & field : last_output) + { + // todo need to use the subquery alias to reconstruct the field + // name if subquery is supported + output_schema.emplace_back(field.first, field.second); + } + } + else + { + exprs.push_back(expr); + auto ft = std::find_if(input->output_schema.begin(), input->output_schema.end(), [&](const auto & field) { return field.first == expr->getColumnName(); }); + if (ft != input->output_schema.end()) + { + output_schema.emplace_back(ft->first, ft->second); + continue; + } + const auto * func = typeid_cast(expr.get()); + if (func && AggregateFunctionFactory::instance().isAggregateFunctionName(func->name)) + { + throw Exception("No such agg " + func->getColumnName(), ErrorCodes::NO_SUCH_COLUMN_IN_TABLE); + } + else + { + auto ci = compileExpr(input->output_schema, expr); + // todo need to use the subquery alias to reconstruct the field + // name if subquery is supported + output_schema.emplace_back(std::make_pair(expr->getColumnName(), ci)); + } + } + } + auto project = std::make_shared(executor_index, output_schema, std::move(exprs)); + project->children.push_back(input); + return project; +} +} // namespace DB::mock diff --git a/dbms/src/Debug/MockExecutor/ProjectBinder.h b/dbms/src/Debug/MockExecutor/ProjectBinder.h new file mode 100644 index 00000000000..2ce35b75474 --- /dev/null +++ b/dbms/src/Debug/MockExecutor/ProjectBinder.h @@ -0,0 +1,38 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// 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 + +namespace DB::mock +{ + +class ProjectBinder : public ExecutorBinder +{ +public: + ProjectBinder(size_t & index_, const DAGSchema & output_schema_, ASTs && exprs_) + : ExecutorBinder(index_, "project_" + std::to_string(index_), output_schema_) + , exprs(std::move(exprs_)) + {} + + bool toTiPBExecutor(tipb::Executor * tipb_executor, int32_t collator_id, const MPPInfo & mpp_info, const Context & context) override; + + void columnPrune(std::unordered_set & used_columns) override; + +private: + std::vector exprs; +}; + +ExecutorBinderPtr compileProject(ExecutorBinderPtr input, size_t & executor_index, ASTPtr select_list); +} // namespace DB::mock diff --git a/dbms/src/Debug/MockExecutor/SelectionBinder.cpp b/dbms/src/Debug/MockExecutor/SelectionBinder.cpp new file mode 100644 index 00000000000..cea52b56922 --- /dev/null +++ b/dbms/src/Debug/MockExecutor/SelectionBinder.cpp @@ -0,0 +1,50 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// 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 + +namespace DB::mock +{ +bool SelectionBinder::toTiPBExecutor(tipb::Executor * tipb_executor, int32_t collator_id, const MPPInfo & mpp_info, const Context & context) +{ + tipb_executor->set_tp(tipb::ExecType::TypeSelection); + tipb_executor->set_executor_id(name); + auto * sel = tipb_executor->mutable_selection(); + for (auto & expr : conditions) + { + tipb::Expr * cond = sel->add_conditions(); + astToPB(children[0]->output_schema, expr, cond, collator_id, context); + } + auto * child_executor = sel->mutable_child(); + return children[0]->toTiPBExecutor(child_executor, collator_id, mpp_info, context); +} + +void SelectionBinder::columnPrune(std::unordered_set & used_columns) +{ + for (auto & expr : conditions) + collectUsedColumnsFromExpr(children[0]->output_schema, expr, used_columns); + children[0]->columnPrune(used_columns); + /// update output schema after column prune + output_schema = children[0]->output_schema; +} + +ExecutorBinderPtr compileSelection(ExecutorBinderPtr input, size_t & executor_index, ASTPtr filter) +{ + std::vector conditions; + compileFilter(input->output_schema, filter, conditions); + auto selection = std::make_shared(executor_index, input->output_schema, std::move(conditions)); + selection->children.push_back(input); + return selection; +} +} // namespace DB::mock diff --git a/dbms/src/Debug/MockExecutor/SelectionBinder.h b/dbms/src/Debug/MockExecutor/SelectionBinder.h new file mode 100644 index 00000000000..d4270ed5fac --- /dev/null +++ b/dbms/src/Debug/MockExecutor/SelectionBinder.h @@ -0,0 +1,39 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// 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 + +namespace DB::mock +{ +class SelectionBinder : public ExecutorBinder +{ +public: + SelectionBinder(size_t & index_, const DAGSchema & output_schema_, ASTs && conditions_) + : ExecutorBinder(index_, "selection_" + std::to_string(index_), output_schema_) + , conditions(std::move(conditions_)) + {} + + bool toTiPBExecutor(tipb::Executor * tipb_executor, int32_t collator_id, const MPPInfo & mpp_info, const Context & context) override; + + void columnPrune(std::unordered_set & used_columns) override; + +protected: + std::vector conditions; +}; + +ExecutorBinderPtr compileSelection(ExecutorBinderPtr input, size_t & executor_index, ASTPtr filter); +} // namespace DB::mock diff --git a/dbms/src/Debug/MockExecutor/SortBinder.cpp b/dbms/src/Debug/MockExecutor/SortBinder.cpp new file mode 100644 index 00000000000..80265448824 --- /dev/null +++ b/dbms/src/Debug/MockExecutor/SortBinder.cpp @@ -0,0 +1,62 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// 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 + +namespace DB::mock +{ +bool SortBinder::toTiPBExecutor(tipb::Executor * tipb_executor, int32_t collator_id, const MPPInfo & mpp_info, const Context & context) +{ + tipb_executor->set_tp(tipb::ExecType::TypeSort); + tipb_executor->set_executor_id(name); + tipb_executor->set_fine_grained_shuffle_stream_count(fine_grained_shuffle_stream_count); + tipb::Sort * sort = tipb_executor->mutable_sort(); + sort->set_ispartialsort(is_partial_sort); + + for (const auto & child : by_exprs) + { + auto * elem = typeid_cast(child.get()); + if (!elem) + throw Exception("Invalid order by element", ErrorCodes::LOGICAL_ERROR); + tipb::ByItem * by = sort->add_byitems(); + by->set_desc(elem->direction < 0); + tipb::Expr * expr = by->mutable_expr(); + astToPB(children[0]->output_schema, elem->children[0], expr, collator_id, context); + } + + auto * children_executor = sort->mutable_child(); + return children[0]->toTiPBExecutor(children_executor, collator_id, mpp_info, context); +} + +ExecutorBinderPtr compileSort(ExecutorBinderPtr input, size_t & executor_index, ASTPtr order_by_expr_list, bool is_partial_sort, uint64_t fine_grained_shuffle_stream_count) +{ + std::vector order_columns; + if (order_by_expr_list != nullptr) + { + for (const auto & child : order_by_expr_list->children) + { + auto * elem = typeid_cast(child.get()); + if (!elem) + throw Exception("Invalid order by element", ErrorCodes::LOGICAL_ERROR); + order_columns.push_back(child); + compileExpr(input->output_schema, elem->children[0]); + } + } + ExecutorBinderPtr sort = std::make_shared(executor_index, input->output_schema, std::move(order_columns), is_partial_sort, fine_grained_shuffle_stream_count); + sort->children.push_back(input); + return sort; +} +} // namespace DB::mock diff --git a/dbms/src/Debug/MockExecutor/SortBinder.h b/dbms/src/Debug/MockExecutor/SortBinder.h new file mode 100644 index 00000000000..72a5a08199d --- /dev/null +++ b/dbms/src/Debug/MockExecutor/SortBinder.h @@ -0,0 +1,44 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// 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 + +namespace DB::mock +{ +class SortBinder : public ExecutorBinder +{ +public: + SortBinder(size_t & index_, const DAGSchema & output_schema_, ASTs && by_exprs_, bool is_partial_sort_, uint64_t fine_grained_shuffle_stream_count_ = 0) + : ExecutorBinder(index_, "sort_" + std::to_string(index_), output_schema_) + , by_exprs(by_exprs_) + , is_partial_sort(is_partial_sort_) + , fine_grained_shuffle_stream_count(fine_grained_shuffle_stream_count_) + {} + + // Currently only use Sort Executor in Unit Test which don't call columnPrume. + // TODO: call columnPrune in unit test and further benchmark test to eliminate compute process. + void columnPrune(std::unordered_set &) override { throw Exception("Should not reach here"); } + + bool toTiPBExecutor(tipb::Executor * tipb_executor, int32_t collator_id, const MPPInfo & mpp_info, const Context & context) override; + +private: + std::vector by_exprs; + bool is_partial_sort; + uint64_t fine_grained_shuffle_stream_count; +}; + +ExecutorBinderPtr compileSort(ExecutorBinderPtr input, size_t & executor_index, ASTPtr order_by_expr_list, bool is_partial_sort, uint64_t fine_grained_shuffle_stream_count); +} // namespace DB::mock diff --git a/dbms/src/Debug/MockExecutor/TableScanBinder.cpp b/dbms/src/Debug/MockExecutor/TableScanBinder.cpp new file mode 100644 index 00000000000..e35a14e4269 --- /dev/null +++ b/dbms/src/Debug/MockExecutor/TableScanBinder.cpp @@ -0,0 +1,123 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// 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 + +namespace DB::mock +{ +bool TableScanBinder::toTiPBExecutor(tipb::Executor * tipb_executor, int32_t, const MPPInfo &, const Context &) +{ + if (table_info.is_partition_table) + buildPartionTable(tipb_executor); + else + buildTable(tipb_executor); + + return true; +} + +void TableScanBinder::columnPrune(std::unordered_set & used_columns) +{ + DAGSchema new_schema; + for (const auto & col : output_schema) + { + for (const auto & used_col : used_columns) + { + if (splitQualifiedName(used_col).column_name == splitQualifiedName(col.first).column_name && splitQualifiedName(used_col).table_name == splitQualifiedName(col.first).table_name) + { + new_schema.push_back({used_col, col.second}); + } + } + } + + output_schema = new_schema; +} + +TableID TableScanBinder::getTableId() const +{ + return table_info.id; +} + +void TableScanBinder::setTipbColumnInfo(tipb::ColumnInfo * ci, const DAGColumnInfo & dag_column_info) const +{ + auto names = splitQualifiedName(dag_column_info.first); + if (names.column_name == MutableSupport::tidb_pk_column_name) + ci->set_column_id(-1); + else + ci->set_column_id(table_info.getColumnID(names.column_name)); + ci->set_tp(dag_column_info.second.tp); + ci->set_flag(dag_column_info.second.flag); + ci->set_columnlen(dag_column_info.second.flen); + ci->set_decimal(dag_column_info.second.decimal); + if (!dag_column_info.second.elems.empty()) + { + for (const auto & pair : dag_column_info.second.elems) + { + ci->add_elems(pair.first); + } + } +} + +void TableScanBinder::buildPartionTable(tipb::Executor * tipb_executor) +{ + tipb_executor->set_tp(tipb::ExecType::TypePartitionTableScan); + tipb_executor->set_executor_id(name); + auto * partition_ts = tipb_executor->mutable_partition_table_scan(); + partition_ts->set_table_id(table_info.id); + for (const auto & info : output_schema) + setTipbColumnInfo(partition_ts->add_columns(), info); + for (const auto & partition : table_info.partition.definitions) + partition_ts->add_partition_ids(partition.id); +} + +void TableScanBinder::buildTable(tipb::Executor * tipb_executor) +{ + tipb_executor->set_tp(tipb::ExecType::TypeTableScan); + tipb_executor->set_executor_id(name); + auto * ts = tipb_executor->mutable_tbl_scan(); + ts->set_table_id(table_info.id); + for (const auto & info : output_schema) + setTipbColumnInfo(ts->add_columns(), info); +} + +ExecutorBinderPtr compileTableScan(size_t & executor_index, TableInfo & table_info, const String & db, const String & table_name, bool append_pk_column) +{ + DAGSchema ts_output; + for (const auto & column_info : table_info.columns) + { + ColumnInfo ci; + ci.tp = column_info.tp; + ci.flag = column_info.flag; + ci.flen = column_info.flen; + ci.decimal = column_info.decimal; + ci.elems = column_info.elems; + ci.default_value = column_info.default_value; + ci.origin_default_value = column_info.origin_default_value; + /// use qualified name as the column name to handle multiple table queries, not very + /// efficient but functionally enough for mock test + ts_output.emplace_back(std::make_pair(db + "." + table_name + "." + column_info.name, std::move(ci))); + } + if (append_pk_column) + { + ColumnInfo ci; + ci.tp = TiDB::TypeLongLong; + ci.setPriKeyFlag(); + ci.setNotNullFlag(); + ts_output.emplace_back(std::make_pair(MutableSupport::tidb_pk_column_name, std::move(ci))); + } + + return std::make_shared(executor_index, ts_output, table_info); +} +} // namespace DB::mock diff --git a/dbms/src/Debug/MockExecutor/TableScanBinder.h b/dbms/src/Debug/MockExecutor/TableScanBinder.h new file mode 100644 index 00000000000..6eaeae7a035 --- /dev/null +++ b/dbms/src/Debug/MockExecutor/TableScanBinder.h @@ -0,0 +1,51 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// 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 + +namespace DB::mock +{ +using TableInfo = TiDB::TableInfo; +class TableScanBinder : public ExecutorBinder +{ +public: + TableScanBinder(size_t & index_, const DAGSchema & output_schema_, const TableInfo & table_info_) + : ExecutorBinder(index_, "table_scan_" + std::to_string(index_), output_schema_) + , table_info(table_info_) + {} + + void columnPrune(std::unordered_set & used_columns) override; + + + bool toTiPBExecutor(tipb::Executor * tipb_executor, int32_t, const MPPInfo &, const Context &) override; + + void toMPPSubPlan(size_t &, const DAGProperties &, std::unordered_map, std::shared_ptr>> &) override + {} + + TableID getTableId() const; + +private: + TableInfo table_info; /// used by column pruner + +private: + void setTipbColumnInfo(tipb::ColumnInfo * ci, const DAGColumnInfo & dag_column_info) const; + void buildPartionTable(tipb::Executor * tipb_executor); + void buildTable(tipb::Executor * tipb_executor); +}; + +ExecutorBinderPtr compileTableScan(size_t & executor_index, TableInfo & table_info, const String & db, const String & table_name, bool append_pk_column); +} // namespace DB::mock diff --git a/dbms/src/Debug/MockExecutor/TopNBinder.cpp b/dbms/src/Debug/MockExecutor/TopNBinder.cpp new file mode 100644 index 00000000000..f8d7dd5f006 --- /dev/null +++ b/dbms/src/Debug/MockExecutor/TopNBinder.cpp @@ -0,0 +1,66 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// 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 + +namespace DB::mock +{ +bool TopNBinder::toTiPBExecutor(tipb::Executor * tipb_executor, int32_t collator_id, const MPPInfo & mpp_info, const Context & context) +{ + tipb_executor->set_tp(tipb::ExecType::TypeTopN); + tipb_executor->set_executor_id(name); + tipb::TopN * topn = tipb_executor->mutable_topn(); + for (const auto & child : order_columns) + { + auto * elem = typeid_cast(child.get()); + if (!elem) + throw Exception("Invalid order by element", ErrorCodes::LOGICAL_ERROR); + tipb::ByItem * by = topn->add_order_by(); + by->set_desc(elem->direction < 0); + tipb::Expr * expr = by->mutable_expr(); + astToPB(children[0]->output_schema, elem->children[0], expr, collator_id, context); + } + topn->set_limit(limit); + auto * child_executor = topn->mutable_child(); + return children[0]->toTiPBExecutor(child_executor, collator_id, mpp_info, context); +} + +void TopNBinder::columnPrune(std::unordered_set & used_columns) +{ + for (auto & expr : order_columns) + collectUsedColumnsFromExpr(children[0]->output_schema, expr, used_columns); + children[0]->columnPrune(used_columns); + /// update output schema after column prune + output_schema = children[0]->output_schema; +} + +ExecutorBinderPtr compileTopN(ExecutorBinderPtr input, size_t & executor_index, ASTPtr order_exprs, ASTPtr limit_expr) +{ + std::vector order_columns; + for (const auto & child : order_exprs->children) + { + auto * elem = typeid_cast(child.get()); + if (!elem) + throw Exception("Invalid order by element", ErrorCodes::LOGICAL_ERROR); + order_columns.push_back(child); + compileExpr(input->output_schema, elem->children[0]); + } + auto limit = safeGet(typeid_cast(*limit_expr).value); + auto top_n = std::make_shared(executor_index, input->output_schema, std::move(order_columns), limit); + top_n->children.push_back(input); + return top_n; +} +} // namespace DB::mock diff --git a/dbms/src/Debug/MockExecutor/TopNBinder.h b/dbms/src/Debug/MockExecutor/TopNBinder.h new file mode 100644 index 00000000000..c783d16d56e --- /dev/null +++ b/dbms/src/Debug/MockExecutor/TopNBinder.h @@ -0,0 +1,40 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// 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 + +namespace DB::mock +{ +class TopNBinder : public ExecutorBinder +{ +public: + TopNBinder(size_t & index_, const DAGSchema & output_schema_, ASTs && order_columns_, size_t limit_) + : ExecutorBinder(index_, "topn_" + std::to_string(index_), output_schema_) + , order_columns(std::move(order_columns_)) + , limit(limit_) + {} + + bool toTiPBExecutor(tipb::Executor * tipb_executor, int32_t collator_id, const MPPInfo & mpp_info, const Context & context) override; + + void columnPrune(std::unordered_set & used_columns) override; + +protected: + std::vector order_columns; + size_t limit; +}; + +ExecutorBinderPtr compileTopN(ExecutorBinderPtr input, size_t & executor_index, ASTPtr order_exprs, ASTPtr limit_expr); +} // namespace DB::mock diff --git a/dbms/src/Debug/MockExecutor/WindowBinder.cpp b/dbms/src/Debug/MockExecutor/WindowBinder.cpp new file mode 100644 index 00000000000..8da8ae5d8ef --- /dev/null +++ b/dbms/src/Debug/MockExecutor/WindowBinder.cpp @@ -0,0 +1,220 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// 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 + +namespace DB::mock +{ +using ASTPartitionByElement = ASTOrderByElement; + +bool WindowBinder::toTiPBExecutor(tipb::Executor * tipb_executor, int32_t collator_id, const MPPInfo & mpp_info, const Context & context) +{ + tipb_executor->set_tp(tipb::ExecType::TypeWindow); + tipb_executor->set_executor_id(name); + tipb_executor->set_fine_grained_shuffle_stream_count(fine_grained_shuffle_stream_count); + tipb::Window * window = tipb_executor->mutable_window(); + auto & input_schema = children[0]->output_schema; + for (const auto & expr : func_descs) + { + tipb::Expr * window_expr = window->add_func_desc(); + const auto * window_func = typeid_cast(expr.get()); + for (const auto & arg : window_func->arguments->children) + { + tipb::Expr * func = window_expr->add_children(); + astToPB(input_schema, arg, func, collator_id, context); + } + auto window_sig_it = tests::window_func_name_to_sig.find(window_func->name); + if (window_sig_it == tests::window_func_name_to_sig.end()) + throw Exception(fmt::format("Unsupported window function {}", window_func->name), ErrorCodes::LOGICAL_ERROR); + auto window_sig = window_sig_it->second; + window_expr->set_tp(window_sig); + auto * ft = window_expr->mutable_field_type(); + switch (window_sig) + { + case tipb::ExprType::Lead: + case tipb::ExprType::Lag: + { + // TODO handling complex situations + // like lead(col, offset, NULL), lead(data_type1, offset, data_type2) + assert(window_expr->children_size() >= 1 && window_expr->children_size() <= 3); + const auto first_arg_type = window_expr->children(0).field_type(); + ft->set_tp(first_arg_type.tp()); + if (window_expr->children_size() < 3) + { + auto field_type = TiDB::fieldTypeToColumnInfo(first_arg_type); + field_type.clearNotNullFlag(); + ft->set_flag(field_type.flag); + } + else + { + const auto third_arg_type = window_expr->children(2).field_type(); + assert(first_arg_type.tp() == third_arg_type.tp()); + ft->set_flag(TiDB::fieldTypeToColumnInfo(first_arg_type).hasNotNullFlag() + ? third_arg_type.flag() + : first_arg_type.flag()); + } + ft->set_collate(first_arg_type.collate()); + ft->set_flen(first_arg_type.flen()); + ft->set_decimal(first_arg_type.decimal()); + break; + } + default: + ft->set_tp(TiDB::TypeLongLong); + ft->set_flag(TiDB::ColumnFlagBinary); + ft->set_collate(collator_id); + ft->set_flen(21); + ft->set_decimal(-1); + } + } + + for (const auto & child : order_by_exprs) + { + auto * elem = typeid_cast(child.get()); + if (!elem) + throw Exception("Invalid order by element", ErrorCodes::LOGICAL_ERROR); + tipb::ByItem * by = window->add_order_by(); + by->set_desc(elem->direction < 0); + tipb::Expr * expr = by->mutable_expr(); + astToPB(children[0]->output_schema, elem->children[0], expr, collator_id, context); + } + + for (const auto & child : partition_by_exprs) + { + auto * elem = typeid_cast(child.get()); + if (!elem) + throw Exception("Invalid partition by element", ErrorCodes::LOGICAL_ERROR); + tipb::ByItem * by = window->add_partition_by(); + by->set_desc(elem->direction < 0); + tipb::Expr * expr = by->mutable_expr(); + astToPB(children[0]->output_schema, elem->children[0], expr, collator_id, context); + } + + if (frame.type.has_value()) + { + tipb::WindowFrame * mut_frame = window->mutable_frame(); + mut_frame->set_type(frame.type.value()); + if (frame.start.has_value()) + { + auto * start = mut_frame->mutable_start(); + start->set_offset(std::get<2>(frame.start.value())); + start->set_unbounded(std::get<1>(frame.start.value())); + start->set_type(std::get<0>(frame.start.value())); + } + + if (frame.end.has_value()) + { + auto * end = mut_frame->mutable_end(); + end->set_offset(std::get<2>(frame.end.value())); + end->set_unbounded(std::get<1>(frame.end.value())); + end->set_type(std::get<0>(frame.end.value())); + } + } + + auto * children_executor = window->mutable_child(); + return children[0]->toTiPBExecutor(children_executor, collator_id, mpp_info, context); +} + +ExecutorBinderPtr compileWindow(ExecutorBinderPtr input, size_t & executor_index, ASTPtr func_desc_list, ASTPtr partition_by_expr_list, ASTPtr order_by_expr_list, mock::MockWindowFrame frame, uint64_t fine_grained_shuffle_stream_count) +{ + std::vector partition_columns; + if (partition_by_expr_list != nullptr) + { + for (const auto & child : partition_by_expr_list->children) + { + auto * elem = typeid_cast(child.get()); + if (!elem) + throw Exception("Invalid partition by element", ErrorCodes::LOGICAL_ERROR); + partition_columns.push_back(child); + compileExpr(input->output_schema, elem->children[0]); + } + } + + std::vector order_columns; + if (order_by_expr_list != nullptr) + { + for (const auto & child : order_by_expr_list->children) + { + auto * elem = typeid_cast(child.get()); + if (!elem) + throw Exception("Invalid order by element", ErrorCodes::LOGICAL_ERROR); + order_columns.push_back(child); + compileExpr(input->output_schema, elem->children[0]); + } + } + + DAGSchema output_schema; + output_schema.insert(output_schema.end(), input->output_schema.begin(), input->output_schema.end()); + + std::vector window_exprs; + if (func_desc_list != nullptr) + { + for (const auto & expr : func_desc_list->children) + { + const auto * func = typeid_cast(expr.get()); + window_exprs.push_back(expr); + std::vector children_ci; + for (const auto & arg : func->arguments->children) + { + children_ci.push_back(compileExpr(input->output_schema, arg)); + } + // TODO: add more window functions + TiDB::ColumnInfo ci; + switch (tests::window_func_name_to_sig[func->name]) + { + case tipb::ExprType::RowNumber: + case tipb::ExprType::Rank: + case tipb::ExprType::DenseRank: + { + ci.tp = TiDB::TypeLongLong; + ci.flag = TiDB::ColumnFlagBinary; + break; + } + case tipb::ExprType::Lead: + case tipb::ExprType::Lag: + { + // TODO handling complex situations + // like lead(col, offset, NULL), lead(data_type1, offset, data_type2) + assert(!children_ci.empty() && children_ci.size() <= 3); + if (children_ci.size() < 3) + { + ci = children_ci[0]; + ci.clearNotNullFlag(); + } + else + { + assert(children_ci[0].tp == children_ci[2].tp); + ci = children_ci[0].hasNotNullFlag() ? children_ci[2] : children_ci[0]; + } + break; + } + default: + throw Exception(fmt::format("Unsupported window function {}", func->name), ErrorCodes::LOGICAL_ERROR); + } + output_schema.emplace_back(std::make_pair(func->getColumnName(), ci)); + } + } + + ExecutorBinderPtr window = std::make_shared( + executor_index, + output_schema, + std::move(window_exprs), + std::move(partition_columns), + std::move(order_columns), + frame, + fine_grained_shuffle_stream_count); + window->children.push_back(input); + return window; +} +} // namespace DB::mock diff --git a/dbms/src/Debug/MockExecutor/WindowBinder.h b/dbms/src/Debug/MockExecutor/WindowBinder.h new file mode 100644 index 00000000000..443506baa33 --- /dev/null +++ b/dbms/src/Debug/MockExecutor/WindowBinder.h @@ -0,0 +1,60 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// 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 + +namespace DB::mock +{ +using MockWindowFrameBound = std::tuple; +struct MockWindowFrame +{ + std::optional type; + std::optional start; + std::optional end; + // TODO: support calcFuncs +}; + +using ASTPartitionByElement = ASTOrderByElement; + +class WindowBinder : public ExecutorBinder +{ +public: + WindowBinder(size_t & index_, const DAGSchema & output_schema_, ASTs && func_descs_, ASTs && partition_by_exprs_, ASTs && order_by_exprs_, MockWindowFrame frame_, uint64_t fine_grained_shuffle_stream_count_ = 0) + : ExecutorBinder(index_, "window_" + std::to_string(index_), output_schema_) + , func_descs(std::move(func_descs_)) + , partition_by_exprs(std::move(partition_by_exprs_)) + , order_by_exprs(order_by_exprs_) + , frame(frame_) + , fine_grained_shuffle_stream_count(fine_grained_shuffle_stream_count_) + {} + + // Currently only use Window Executor in Unit Test which don't call columnPrume. + // TODO: call columnPrune in unit test and further benchmark test to eliminate compute process. + void columnPrune(std::unordered_set &) override { throw Exception("Should not reach here"); } + + bool toTiPBExecutor(tipb::Executor * tipb_executor, int32_t collator_id, const MPPInfo & mpp_info, const Context & context) override; + +private: + std::vector func_descs; + std::vector partition_by_exprs; + std::vector order_by_exprs; + MockWindowFrame frame; + uint64_t fine_grained_shuffle_stream_count; +}; + +ExecutorBinderPtr compileWindow(ExecutorBinderPtr input, size_t & executor_index, ASTPtr func_desc_list, ASTPtr partition_by_expr_list, ASTPtr order_by_expr_list, mock::MockWindowFrame frame, uint64_t fine_grained_shuffle_stream_count); +} // namespace DB::mock diff --git a/dbms/src/Debug/MockExecutor/astToExecutor.cpp b/dbms/src/Debug/MockExecutor/astToExecutor.cpp deleted file mode 100644 index 6a5f38de9e0..00000000000 --- a/dbms/src/Debug/MockExecutor/astToExecutor.cpp +++ /dev/null @@ -1,1843 +0,0 @@ -// Copyright 2022 PingCAP, Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// 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 -#include -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ -namespace ErrorCodes -{ -extern const int BAD_ARGUMENTS; -extern const int LOGICAL_ERROR; -extern const int NO_SUCH_COLUMN_IN_TABLE; -} // namespace ErrorCodes - -using ASTPartitionByElement = ASTOrderByElement; -using MockComputeServerManager = tests::MockComputeServerManager; - -void literalFieldToTiPBExpr(const ColumnInfo & ci, const Field & val_field, tipb::Expr * expr, Int32 collator_id) -{ - *(expr->mutable_field_type()) = columnInfoToFieldType(ci); - expr->mutable_field_type()->set_collate(collator_id); - if (!val_field.isNull()) - { - WriteBufferFromOwnString ss; - switch (ci.tp) - { - case TiDB::TypeLongLong: - case TiDB::TypeLong: - case TiDB::TypeShort: - case TiDB::TypeTiny: - case TiDB::TypeInt24: - if (ci.hasUnsignedFlag()) - { - expr->set_tp(tipb::ExprType::Uint64); - UInt64 val = val_field.safeGet(); - encodeDAGUInt64(val, ss); - } - else - { - expr->set_tp(tipb::ExprType::Int64); - Int64 val = val_field.safeGet(); - encodeDAGInt64(val, ss); - } - break; - case TiDB::TypeFloat: - { - expr->set_tp(tipb::ExprType::Float32); - auto val = static_cast(val_field.safeGet()); - encodeDAGFloat32(val, ss); - break; - } - case TiDB::TypeDouble: - { - expr->set_tp(tipb::ExprType::Float64); - Float64 val = val_field.safeGet(); - encodeDAGFloat64(val, ss); - break; - } - case TiDB::TypeString: - { - expr->set_tp(tipb::ExprType::String); - const auto & val = val_field.safeGet(); - encodeDAGString(val, ss); - break; - } - case TiDB::TypeNewDecimal: - { - expr->set_tp(tipb::ExprType::MysqlDecimal); - encodeDAGDecimal(val_field, ss); - break; - } - case TiDB::TypeDate: - { - expr->set_tp(tipb::ExprType::MysqlTime); - UInt64 val = val_field.safeGet(); - encodeDAGUInt64(MyDate(val).toPackedUInt(), ss); - break; - } - case TiDB::TypeDatetime: - case TiDB::TypeTimestamp: - { - expr->set_tp(tipb::ExprType::MysqlTime); - UInt64 val = val_field.safeGet(); - encodeDAGUInt64(MyDateTime(val).toPackedUInt(), ss); - break; - } - case TiDB::TypeTime: - { - expr->set_tp(tipb::ExprType::MysqlDuration); - Int64 val = val_field.safeGet(); - encodeDAGInt64(val, ss); - break; - } - default: - throw Exception(fmt::format("Type {} does not support literal in function unit test", getDataTypeByColumnInfo(ci)->getName())); - } - expr->set_val(ss.releaseStr()); - } - else - { - expr->set_tp(tipb::ExprType::Null); - } -} - -void literalToPB(tipb::Expr * expr, const Field & value, int32_t collator_id) -{ - DataTypePtr type = applyVisitor(FieldToDataType(), value); - ColumnInfo ci = reverseGetColumnInfo({"", type}, 0, Field(), true); - literalFieldToTiPBExpr(ci, value, expr, collator_id); -} - -String getFunctionNameForConstantFolding(tipb::Expr * expr) -{ - // todo support more function for constant folding - switch (expr->sig()) - { - case tipb::ScalarFuncSig::CastStringAsTime: - return "toMyDateTimeOrNull"; - default: - return ""; - } -} - -void foldConstant(tipb::Expr * expr, int32_t collator_id, const Context & context) -{ - if (expr->tp() == tipb::ScalarFunc) - { - bool all_const = true; - for (const auto & c : expr->children()) - { - if (!isLiteralExpr(c)) - { - all_const = false; - break; - } - } - if (!all_const) - return; - DataTypes arguments_types; - ColumnsWithTypeAndName argument_columns; - for (const auto & c : expr->children()) - { - Field value = decodeLiteral(c); - DataTypePtr flash_type = applyVisitor(FieldToDataType(), value); - DataTypePtr target_type = inferDataType4Literal(c); - ColumnWithTypeAndName column; - column.column = target_type->createColumnConst(1, convertFieldToType(value, *target_type, flash_type.get())); - column.name = exprToString(c, {}) + "_" + target_type->getName(); - column.type = target_type; - arguments_types.emplace_back(target_type); - argument_columns.emplace_back(column); - } - auto func_name = getFunctionNameForConstantFolding(expr); - if (func_name.empty()) - return; - const auto & function_builder_ptr = FunctionFactory::instance().get(func_name, context); - auto function_ptr = function_builder_ptr->build(argument_columns); - if (function_ptr->isSuitableForConstantFolding()) - { - Block block_with_constants(argument_columns); - ColumnNumbers argument_numbers(arguments_types.size()); - for (size_t i = 0, size = arguments_types.size(); i < size; i++) - argument_numbers[i] = i; - size_t result_pos = argument_numbers.size(); - block_with_constants.insert({nullptr, function_ptr->getReturnType(), "result"}); - function_ptr->execute(block_with_constants, argument_numbers, result_pos); - const auto & result_column = block_with_constants.getByPosition(result_pos).column; - if (result_column->isColumnConst()) - { - auto updated_value = (*result_column)[0]; - tipb::FieldType orig_field_type = expr->field_type(); - expr->Clear(); - literalToPB(expr, updated_value, collator_id); - expr->clear_field_type(); - auto * field_type = expr->mutable_field_type(); - (*field_type) = orig_field_type; - } - } - } -} - -void functionToPB(const DAGSchema & input, ASTFunction * func, tipb::Expr * expr, int32_t collator_id, const Context & context); - -void identifierToPB(const DAGSchema & input, ASTIdentifier * id, tipb::Expr * expr, int32_t collator_id); - -void astToPB(const DAGSchema & input, ASTPtr ast, tipb::Expr * expr, int32_t collator_id, const Context & context) -{ - if (auto * id = typeid_cast(ast.get())) - { - identifierToPB(input, id, expr, collator_id); - } - else if (auto * func = typeid_cast(ast.get())) - { - functionToPB(input, func, expr, collator_id, context); - } - else if (auto * lit = typeid_cast(ast.get())) - { - literalToPB(expr, lit->value, collator_id); - } - else - { - throw Exception("Unsupported expression " + ast->getColumnName(), ErrorCodes::LOGICAL_ERROR); - } -} - -void functionToPB(const DAGSchema & input, ASTFunction * func, tipb::Expr * expr, int32_t collator_id, const Context & context) -{ - /// aggregation function is handled in Aggregation, so just treated as a column - auto ft = checkSchema(input, func->getColumnName()); - if (ft != input.end()) - { - expr->set_tp(tipb::ColumnRef); - *(expr->mutable_field_type()) = columnInfoToFieldType((*ft).second); - WriteBufferFromOwnString ss; - encodeDAGInt64(ft - input.begin(), ss); - expr->set_val(ss.releaseStr()); - return; - } - if (AggregateFunctionFactory::instance().isAggregateFunctionName(func->name)) - { - throw Exception("No such column " + func->getColumnName(), ErrorCodes::NO_SUCH_COLUMN_IN_TABLE); - } - String func_name_lowercase = Poco::toLower(func->name); - // TODO: Support more functions. - // TODO: Support type inference. - - const auto it_sig = tests::func_name_to_sig.find(func_name_lowercase); - if (it_sig == tests::func_name_to_sig.end()) - { - throw Exception("Unsupported function: " + func_name_lowercase, ErrorCodes::LOGICAL_ERROR); - } - switch (it_sig->second) - { - case tipb::ScalarFuncSig::InInt: - { - tipb::Expr * in_expr = expr; - if (func_name_lowercase == "notin") - { - // notin is transformed into not(in()) by tidb - expr->set_sig(tipb::ScalarFuncSig::UnaryNotInt); - auto * ft = expr->mutable_field_type(); - ft->set_tp(TiDB::TypeLongLong); - ft->set_flag(TiDB::ColumnFlagUnsigned); - expr->set_tp(tipb::ExprType::ScalarFunc); - in_expr = expr->add_children(); - } - in_expr->set_sig(tipb::ScalarFuncSig::InInt); - auto * ft = in_expr->mutable_field_type(); - ft->set_tp(TiDB::TypeLongLong); - ft->set_flag(TiDB::ColumnFlagUnsigned); - ft->set_collate(collator_id); - in_expr->set_tp(tipb::ExprType::ScalarFunc); - for (const auto & child_ast : func->arguments->children) - { - auto * tuple_func = typeid_cast(child_ast.get()); - if (tuple_func != nullptr && tuple_func->name == "tuple") - { - // flatten tuple elements - for (const auto & c : tuple_func->arguments->children) - { - tipb::Expr * child = in_expr->add_children(); - astToPB(input, c, child, collator_id, context); - } - } - else - { - tipb::Expr * child = in_expr->add_children(); - astToPB(input, child_ast, child, collator_id, context); - } - } - return; - } - case tipb::ScalarFuncSig::IfInt: - case tipb::ScalarFuncSig::BitAndSig: - case tipb::ScalarFuncSig::BitOrSig: - case tipb::ScalarFuncSig::BitXorSig: - case tipb::ScalarFuncSig::BitNegSig: - expr->set_sig(it_sig->second); - expr->set_tp(tipb::ExprType::ScalarFunc); - for (size_t i = 0; i < func->arguments->children.size(); i++) - { - const auto & child_ast = func->arguments->children[i]; - tipb::Expr * child = expr->add_children(); - astToPB(input, child_ast, child, collator_id, context); - // todo should infer the return type based on all input types - if ((it_sig->second == tipb::ScalarFuncSig::IfInt && i == 1) - || (it_sig->second != tipb::ScalarFuncSig::IfInt && i == 0)) - *(expr->mutable_field_type()) = child->field_type(); - } - return; - case tipb::ScalarFuncSig::PlusInt: - case tipb::ScalarFuncSig::MinusInt: - { - for (const auto & child_ast : func->arguments->children) - { - tipb::Expr * child = expr->add_children(); - astToPB(input, child_ast, child, collator_id, context); - } - expr->set_sig(it_sig->second); - auto * ft = expr->mutable_field_type(); - ft->set_tp(expr->children(0).field_type().tp()); - ft->set_flag(expr->children(0).field_type().flag()); - ft->set_collate(collator_id); - expr->set_tp(tipb::ExprType::ScalarFunc); - return; - } - case tipb::ScalarFuncSig::LikeSig: - { - expr->set_sig(tipb::ScalarFuncSig::LikeSig); - auto * ft = expr->mutable_field_type(); - ft->set_tp(TiDB::TypeLongLong); - ft->set_flag(TiDB::ColumnFlagUnsigned); - ft->set_collate(collator_id); - expr->set_tp(tipb::ExprType::ScalarFunc); - for (const auto & child_ast : func->arguments->children) - { - tipb::Expr * child = expr->add_children(); - astToPB(input, child_ast, child, collator_id, context); - } - // for like need to add the third argument - *expr->add_children() = constructInt64LiteralTiExpr(92); - return; - } - case tipb::ScalarFuncSig::FromUnixTime2Arg: - if (func->arguments->children.size() == 1) - { - expr->set_sig(tipb::ScalarFuncSig::FromUnixTime1Arg); - auto * ft = expr->mutable_field_type(); - ft->set_tp(TiDB::TypeDatetime); - ft->set_decimal(6); - } - else - { - expr->set_sig(tipb::ScalarFuncSig::FromUnixTime2Arg); - auto * ft = expr->mutable_field_type(); - ft->set_tp(TiDB::TypeString); - } - break; - case tipb::ScalarFuncSig::DateFormatSig: - expr->set_sig(tipb::ScalarFuncSig::DateFormatSig); - expr->mutable_field_type()->set_tp(TiDB::TypeString); - break; - case tipb::ScalarFuncSig::CastIntAsTime: - case tipb::ScalarFuncSig::CastRealAsTime: - case tipb::ScalarFuncSig::CastTimeAsTime: - case tipb::ScalarFuncSig::CastDecimalAsTime: - case tipb::ScalarFuncSig::CastStringAsTime: - { - expr->set_sig(it_sig->second); - auto * ft = expr->mutable_field_type(); - if (it_sig->first.find("datetime")) - { - ft->set_tp(TiDB::TypeDatetime); - } - else - { - ft->set_tp(TiDB::TypeDate); - } - break; - } - case tipb::ScalarFuncSig::CastIntAsReal: - case tipb::ScalarFuncSig::CastRealAsReal: - { - expr->set_sig(it_sig->second); - auto * ft = expr->mutable_field_type(); - ft->set_tp(TiDB::TypeDouble); - ft->set_collate(collator_id); - break; - } - case tipb::ScalarFuncSig::Concat: - { - expr->set_sig(it_sig->second); - auto * ft = expr->mutable_field_type(); - ft->set_tp(TiDB::TypeString); - ft->set_collate(collator_id); - break; - } - case tipb::ScalarFuncSig::RoundInt: - case tipb::ScalarFuncSig::RoundWithFracInt: - { - expr->set_sig(it_sig->second); - auto * ft = expr->mutable_field_type(); - ft->set_tp(TiDB::TypeLongLong); - if (it_sig->first.find("uint") != std::string::npos) - ft->set_flag(TiDB::ColumnFlagUnsigned); - ft->set_collate(collator_id); - break; - } - case tipb::ScalarFuncSig::RoundDec: - case tipb::ScalarFuncSig::RoundWithFracDec: - { - expr->set_sig(it_sig->second); - auto * ft = expr->mutable_field_type(); - ft->set_tp(TiDB::TypeNewDecimal); - ft->set_collate(collator_id); - break; - } - case tipb::ScalarFuncSig::RoundReal: - case tipb::ScalarFuncSig::RoundWithFracReal: - { - expr->set_sig(it_sig->second); - auto * ft = expr->mutable_field_type(); - ft->set_tp(TiDB::TypeDouble); - ft->set_collate(collator_id); - break; - } - default: - { - expr->set_sig(it_sig->second); - auto * ft = expr->mutable_field_type(); - ft->set_tp(TiDB::TypeLongLong); - ft->set_flag(TiDB::ColumnFlagUnsigned); - ft->set_collate(collator_id); - break; - } - } - expr->set_tp(tipb::ExprType::ScalarFunc); - for (const auto & child_ast : func->arguments->children) - { - tipb::Expr * child = expr->add_children(); - astToPB(input, child_ast, child, collator_id, context); - } - foldConstant(expr, collator_id, context); -} - -void identifierToPB(const DAGSchema & input, ASTIdentifier * id, tipb::Expr * expr, int32_t collator_id) -{ - auto ft = checkSchema(input, id->getColumnName()); - if (ft == input.end()) - throw Exception("No such column " + id->getColumnName(), ErrorCodes::NO_SUCH_COLUMN_IN_TABLE); - expr->set_tp(tipb::ColumnRef); - *(expr->mutable_field_type()) = columnInfoToFieldType((*ft).second); - expr->mutable_field_type()->set_collate(collator_id); - WriteBufferFromOwnString ss; - encodeDAGInt64(ft - input.begin(), ss); - expr->set_val(ss.releaseStr()); -} - -void collectUsedColumnsFromExpr(const DAGSchema & input, ASTPtr ast, std::unordered_set & used_columns) -{ - if (auto * id = typeid_cast(ast.get())) - { - auto [db_name, table_name, column_name] = splitQualifiedName(id->getColumnName()); - if (!table_name.empty()) - used_columns.emplace(id->getColumnName()); - else - { - bool found = false; - for (const auto & field : input) - { - if (splitQualifiedName(field.first).column_name == column_name) - { - if (found) - throw Exception("ambiguous column for " + column_name); - found = true; - used_columns.emplace(field.first); - } - } - } - } - else if (auto * func = typeid_cast(ast.get())) - { - if (AggregateFunctionFactory::instance().isAggregateFunctionName(func->name)) - { - used_columns.emplace(func->getColumnName()); - } - else - { - /// check function - auto ft = checkSchema(input, func->getColumnName()); - if (ft != input.end()) - { - used_columns.emplace(func->getColumnName()); - return; - } - for (const auto & child_ast : func->arguments->children) - { - collectUsedColumnsFromExpr(input, child_ast, used_columns); - } - } - } -} - -TiDB::ColumnInfo compileExpr(const DAGSchema & input, ASTPtr ast) -{ - TiDB::ColumnInfo ci; - if (auto * id = typeid_cast(ast.get())) - { - /// check column - auto ft = checkSchema(input, id->getColumnName()); - if (ft == input.end()) - throw Exception("No such column " + id->getColumnName(), ErrorCodes::NO_SUCH_COLUMN_IN_TABLE); - ci = ft->second; - } - else if (auto * func = typeid_cast(ast.get())) - { - /// check function - String func_name_lowercase = Poco::toLower(func->name); - const auto it_sig = tests::func_name_to_sig.find(func_name_lowercase); - if (it_sig == tests::func_name_to_sig.end()) - { - throw Exception("Unsupported function: " + func_name_lowercase, ErrorCodes::LOGICAL_ERROR); - } - switch (it_sig->second) - { - case tipb::ScalarFuncSig::InInt: - ci.tp = TiDB::TypeLongLong; - ci.flag = TiDB::ColumnFlagUnsigned; - for (const auto & child_ast : func->arguments->children) - { - auto * tuple_func = typeid_cast(child_ast.get()); - if (tuple_func != nullptr && tuple_func->name == "tuple") - { - // flatten tuple elements - for (const auto & c : tuple_func->arguments->children) - { - compileExpr(input, c); - } - } - else - { - compileExpr(input, child_ast); - } - } - return ci; - case tipb::ScalarFuncSig::IfInt: - case tipb::ScalarFuncSig::BitAndSig: - case tipb::ScalarFuncSig::BitOrSig: - case tipb::ScalarFuncSig::BitXorSig: - case tipb::ScalarFuncSig::BitNegSig: - for (size_t i = 0; i < func->arguments->children.size(); i++) - { - const auto & child_ast = func->arguments->children[i]; - auto child_ci = compileExpr(input, child_ast); - // todo should infer the return type based on all input types - if ((it_sig->second == tipb::ScalarFuncSig::IfInt && i == 1) - || (it_sig->second != tipb::ScalarFuncSig::IfInt && i == 0)) - ci = child_ci; - } - return ci; - case tipb::ScalarFuncSig::PlusInt: - case tipb::ScalarFuncSig::MinusInt: - return compileExpr(input, func->arguments->children[0]); - case tipb::ScalarFuncSig::LikeSig: - ci.tp = TiDB::TypeLongLong; - ci.flag = TiDB::ColumnFlagUnsigned; - for (const auto & child_ast : func->arguments->children) - { - compileExpr(input, child_ast); - } - return ci; - case tipb::ScalarFuncSig::FromUnixTime2Arg: - if (func->arguments->children.size() == 1) - { - ci.tp = TiDB::TypeDatetime; - ci.decimal = 6; - } - else - { - ci.tp = TiDB::TypeString; - } - break; - case tipb::ScalarFuncSig::DateFormatSig: - ci.tp = TiDB::TypeString; - break; - case tipb::ScalarFuncSig::CastIntAsTime: - case tipb::ScalarFuncSig::CastRealAsTime: - case tipb::ScalarFuncSig::CastTimeAsTime: - case tipb::ScalarFuncSig::CastDecimalAsTime: - case tipb::ScalarFuncSig::CastStringAsTime: - if (it_sig->first.find("datetime")) - { - ci.tp = TiDB::TypeDatetime; - } - else - { - ci.tp = TiDB::TypeDate; - } - break; - case tipb::ScalarFuncSig::CastIntAsReal: - case tipb::ScalarFuncSig::CastRealAsReal: - { - ci.tp = TiDB::TypeDouble; - break; - } - case tipb::ScalarFuncSig::RoundInt: - case tipb::ScalarFuncSig::RoundWithFracInt: - { - ci.tp = TiDB::TypeLongLong; - if (it_sig->first.find("uint") != std::string::npos) - ci.flag = TiDB::ColumnFlagUnsigned; - break; - } - case tipb::ScalarFuncSig::RoundDec: - case tipb::ScalarFuncSig::RoundWithFracDec: - { - ci.tp = TiDB::TypeNewDecimal; - break; - } - case tipb::ScalarFuncSig::RoundReal: - case tipb::ScalarFuncSig::RoundWithFracReal: - { - ci.tp = TiDB::TypeDouble; - break; - } - default: - ci.tp = TiDB::TypeLongLong; - ci.flag = TiDB::ColumnFlagUnsigned; - break; - } - for (const auto & child_ast : func->arguments->children) - { - compileExpr(input, child_ast); - } - } - else if (auto * lit = typeid_cast(ast.get())) - { - switch (lit->value.getType()) - { - case Field::Types::Which::Null: - ci.tp = TiDB::TypeNull; - // Null literal expr doesn't need value. - break; - case Field::Types::Which::UInt64: - ci.tp = TiDB::TypeLongLong; - ci.flag = TiDB::ColumnFlagUnsigned; - break; - case Field::Types::Which::Int64: - ci.tp = TiDB::TypeLongLong; - break; - case Field::Types::Which::Float64: - ci.tp = TiDB::TypeDouble; - break; - case Field::Types::Which::Decimal32: - case Field::Types::Which::Decimal64: - case Field::Types::Which::Decimal128: - case Field::Types::Which::Decimal256: - ci.tp = TiDB::TypeNewDecimal; - break; - case Field::Types::Which::String: - ci.tp = TiDB::TypeString; - break; - default: - throw Exception(String("Unsupported literal type: ") + lit->value.getTypeName(), ErrorCodes::LOGICAL_ERROR); - } - } - else - { - /// not supported unless this is a literal - throw Exception("Unsupported expression " + ast->getColumnName(), ErrorCodes::LOGICAL_ERROR); - } - return ci; -} - -void compileFilter(const DAGSchema & input, ASTPtr ast, std::vector & conditions) -{ - if (auto * func = typeid_cast(ast.get())) - { - if (func->name == "and") - { - for (auto & child : func->arguments->children) - { - compileFilter(input, child, conditions); - } - return; - } - } - conditions.push_back(ast); - compileExpr(input, ast); -} - -namespace mock -{ -bool ExchangeSender::toTiPBExecutor(tipb::Executor * tipb_executor, int32_t collator_id, const MPPInfo & mpp_info, const Context & context) -{ - tipb_executor->set_tp(tipb::ExecType::TypeExchangeSender); - tipb_executor->set_executor_id(name); - tipb::ExchangeSender * exchange_sender = tipb_executor->mutable_exchange_sender(); - exchange_sender->set_tp(type); - for (auto i : partition_keys) - { - auto * expr = exchange_sender->add_partition_keys(); - expr->set_tp(tipb::ColumnRef); - WriteBufferFromOwnString ss; - encodeDAGInt64(i, ss); - expr->set_val(ss.releaseStr()); - auto tipb_type = TiDB::columnInfoToFieldType(output_schema[i].second); - *expr->mutable_field_type() = tipb_type; - tipb_type.set_collate(collator_id); - *exchange_sender->add_types() = tipb_type; - } - - int i = 0; - for (auto task_id : mpp_info.sender_target_task_ids) - { - mpp::TaskMeta meta; - meta.set_start_ts(mpp_info.start_ts); - meta.set_task_id(task_id); - meta.set_partition_id(i); - auto addr = context.isMPPTest() ? MockComputeServerManager::instance().getServerConfigMap()[i++].addr : Debug::LOCAL_HOST; - meta.set_address(addr); - - auto * meta_string = exchange_sender->add_encoded_task_meta(); - meta.AppendToString(meta_string); - } - - for (auto & field : output_schema) - { - auto tipb_type = TiDB::columnInfoToFieldType(field.second); - tipb_type.set_collate(collator_id); - auto * field_type = exchange_sender->add_all_field_types(); - *field_type = tipb_type; - } - - auto * child_executor = exchange_sender->mutable_child(); - return children[0]->toTiPBExecutor(child_executor, collator_id, mpp_info, context); -} - -bool ExchangeReceiver::toTiPBExecutor(tipb::Executor * tipb_executor, int32_t collator_id, const MPPInfo & mpp_info, const Context & context) -{ - tipb_executor->set_tp(tipb::ExecType::TypeExchangeReceiver); - tipb_executor->set_executor_id(name); - tipb_executor->set_fine_grained_shuffle_stream_count(fine_grained_shuffle_stream_count); - tipb::ExchangeReceiver * exchange_receiver = tipb_executor->mutable_exchange_receiver(); - - for (auto & field : output_schema) - { - auto tipb_type = TiDB::columnInfoToFieldType(field.second); - tipb_type.set_collate(collator_id); - - auto * field_type = exchange_receiver->add_field_types(); - *field_type = tipb_type; - } - - auto it = mpp_info.receiver_source_task_ids_map.find(name); - if (it == mpp_info.receiver_source_task_ids_map.end()) - throw Exception("Can not found mpp receiver info"); - - auto size = it->second.size(); - for (size_t i = 0; i < size; ++i) - { - mpp::TaskMeta meta; - meta.set_start_ts(mpp_info.start_ts); - meta.set_task_id(it->second[i]); - meta.set_partition_id(i); - auto addr = context.isMPPTest() ? MockComputeServerManager::instance().getServerConfigMap()[i].addr : Debug::LOCAL_HOST; - meta.set_address(addr); - auto * meta_string = exchange_receiver->add_encoded_task_meta(); - meta.AppendToString(meta_string); - } - return true; -} - -void TableScan::columnPrune(std::unordered_set & used_columns) -{ - DAGSchema new_schema; - for (const auto & col : output_schema) - { - for (const auto & used_col : used_columns) - { - if (splitQualifiedName(used_col).column_name == splitQualifiedName(col.first).column_name && splitQualifiedName(used_col).table_name == splitQualifiedName(col.first).table_name) - { - new_schema.push_back({used_col, col.second}); - } - } - } - - output_schema = new_schema; -} - -bool TableScan::toTiPBExecutor(tipb::Executor * tipb_executor, int32_t, const MPPInfo &, const Context &) -{ - if (table_info.is_partition_table) - { - tipb_executor->set_tp(tipb::ExecType::TypePartitionTableScan); - tipb_executor->set_executor_id(name); - auto * partition_ts = tipb_executor->mutable_partition_table_scan(); - partition_ts->set_table_id(table_info.id); - for (const auto & info : output_schema) - setTipbColumnInfo(partition_ts->add_columns(), info); - for (const auto & partition : table_info.partition.definitions) - partition_ts->add_partition_ids(partition.id); - } - else - { - tipb_executor->set_tp(tipb::ExecType::TypeTableScan); - tipb_executor->set_executor_id(name); - auto * ts = tipb_executor->mutable_tbl_scan(); - ts->set_table_id(table_info.id); - for (const auto & info : output_schema) - setTipbColumnInfo(ts->add_columns(), info); - } - return true; -} - -bool Selection::toTiPBExecutor(tipb::Executor * tipb_executor, int32_t collator_id, const MPPInfo & mpp_info, const Context & context) -{ - tipb_executor->set_tp(tipb::ExecType::TypeSelection); - tipb_executor->set_executor_id(name); - auto * sel = tipb_executor->mutable_selection(); - for (auto & expr : conditions) - { - tipb::Expr * cond = sel->add_conditions(); - astToPB(children[0]->output_schema, expr, cond, collator_id, context); - } - auto * child_executor = sel->mutable_child(); - return children[0]->toTiPBExecutor(child_executor, collator_id, mpp_info, context); -} - -void Selection::columnPrune(std::unordered_set & used_columns) -{ - for (auto & expr : conditions) - collectUsedColumnsFromExpr(children[0]->output_schema, expr, used_columns); - children[0]->columnPrune(used_columns); - /// update output schema after column prune - output_schema = children[0]->output_schema; -} - -bool TopN::toTiPBExecutor(tipb::Executor * tipb_executor, int32_t collator_id, const MPPInfo & mpp_info, const Context & context) -{ - tipb_executor->set_tp(tipb::ExecType::TypeTopN); - tipb_executor->set_executor_id(name); - tipb::TopN * topn = tipb_executor->mutable_topn(); - for (const auto & child : order_columns) - { - auto * elem = typeid_cast(child.get()); - if (!elem) - throw Exception("Invalid order by element", ErrorCodes::LOGICAL_ERROR); - tipb::ByItem * by = topn->add_order_by(); - by->set_desc(elem->direction < 0); - tipb::Expr * expr = by->mutable_expr(); - astToPB(children[0]->output_schema, elem->children[0], expr, collator_id, context); - } - topn->set_limit(limit); - auto * child_executor = topn->mutable_child(); - return children[0]->toTiPBExecutor(child_executor, collator_id, mpp_info, context); -} - -void TopN::columnPrune(std::unordered_set & used_columns) -{ - for (auto & expr : order_columns) - collectUsedColumnsFromExpr(children[0]->output_schema, expr, used_columns); - children[0]->columnPrune(used_columns); - /// update output schema after column prune - output_schema = children[0]->output_schema; -} - -bool Limit::toTiPBExecutor(tipb::Executor * tipb_executor, int32_t collator_id, const MPPInfo & mpp_info, const Context & context) -{ - tipb_executor->set_tp(tipb::ExecType::TypeLimit); - tipb_executor->set_executor_id(name); - tipb::Limit * lt = tipb_executor->mutable_limit(); - lt->set_limit(limit); - auto * child_executor = lt->mutable_child(); - return children[0]->toTiPBExecutor(child_executor, collator_id, mpp_info, context); -} - -void Limit::columnPrune(std::unordered_set & used_columns) -{ - children[0]->columnPrune(used_columns); - /// update output schema after column prune - output_schema = children[0]->output_schema; -} - -bool Aggregation::toTiPBExecutor(tipb::Executor * tipb_executor, int32_t collator_id, const MPPInfo & mpp_info, const Context & context) -{ - tipb_executor->set_tp(tipb::ExecType::TypeAggregation); - tipb_executor->set_executor_id(name); - auto * agg = tipb_executor->mutable_aggregation(); - auto & input_schema = children[0]->output_schema; - for (const auto & expr : agg_exprs) - { - const auto * func = typeid_cast(expr.get()); - if (!func || !AggregateFunctionFactory::instance().isAggregateFunctionName(func->name)) - throw Exception("Only agg function is allowed in select for a query with aggregation", ErrorCodes::LOGICAL_ERROR); - - tipb::Expr * agg_func = agg->add_agg_func(); - - for (const auto & arg : func->arguments->children) - { - tipb::Expr * arg_expr = agg_func->add_children(); - astToPB(input_schema, arg, arg_expr, collator_id, context); - } - auto agg_sig_it = tests::agg_func_name_to_sig.find(func->name); - if (agg_sig_it == tests::agg_func_name_to_sig.end()) - throw Exception("Unsupported agg function " + func->name, ErrorCodes::LOGICAL_ERROR); - auto agg_sig = agg_sig_it->second; - agg_func->set_tp(agg_sig); - - if (agg_sig == tipb::ExprType::Count || agg_sig == tipb::ExprType::Sum) - { - auto * ft = agg_func->mutable_field_type(); - ft->set_tp(TiDB::TypeLongLong); - ft->set_flag(TiDB::ColumnFlagUnsigned | TiDB::ColumnFlagNotNull); - } - else if (agg_sig == tipb::ExprType::Min || agg_sig == tipb::ExprType::Max || agg_sig == tipb::ExprType::First) - { - if (agg_func->children_size() != 1) - throw Exception("udaf " + func->name + " only accept 1 argument"); - auto * ft = agg_func->mutable_field_type(); - ft->set_tp(agg_func->children(0).field_type().tp()); - ft->set_decimal(agg_func->children(0).field_type().decimal()); - ft->set_flag(agg_func->children(0).field_type().flag() & (~TiDB::ColumnFlagNotNull)); - ft->set_collate(collator_id); - } - else if (agg_sig == tipb::ExprType::ApproxCountDistinct) - { - auto * ft = agg_func->mutable_field_type(); - ft->set_tp(TiDB::TypeString); - ft->set_flag(1); - } - else if (agg_sig == tipb::ExprType::GroupConcat) - { - auto * ft = agg_func->mutable_field_type(); - ft->set_tp(TiDB::TypeString); - } - if (is_final_mode) - agg_func->set_aggfuncmode(tipb::AggFunctionMode::FinalMode); - else - agg_func->set_aggfuncmode(tipb::AggFunctionMode::Partial1Mode); - } - - for (const auto & child : gby_exprs) - { - tipb::Expr * gby = agg->add_group_by(); - astToPB(input_schema, child, gby, collator_id, context); - } - - auto * child_executor = agg->mutable_child(); - return children[0]->toTiPBExecutor(child_executor, collator_id, mpp_info, context); -} - -void Aggregation::columnPrune(std::unordered_set & used_columns) -{ - /// output schema for partial agg is the original agg's output schema - output_schema_for_partial_agg = output_schema; - output_schema.erase(std::remove_if(output_schema.begin(), output_schema.end(), [&](const auto & field) { return used_columns.count(field.first) == 0; }), - output_schema.end()); - std::unordered_set used_input_columns; - for (auto & func : agg_exprs) - { - if (used_columns.find(func->getColumnName()) != used_columns.end()) - { - const auto * agg_func = typeid_cast(func.get()); - if (agg_func != nullptr) - { - /// agg_func should not be nullptr, just double check - for (auto & child : agg_func->arguments->children) - collectUsedColumnsFromExpr(children[0]->output_schema, child, used_input_columns); - } - } - } - for (auto & gby_expr : gby_exprs) - { - collectUsedColumnsFromExpr(children[0]->output_schema, gby_expr, used_input_columns); - } - children[0]->columnPrune(used_input_columns); -} - -void Aggregation::toMPPSubPlan(size_t & executor_index, const DAGProperties & properties, std::unordered_map, std::shared_ptr>> & exchange_map) -{ - if (!is_final_mode) - { - children[0]->toMPPSubPlan(executor_index, properties, exchange_map); - return; - } - /// for aggregation, change aggregation to partial_aggregation => exchange_sender => exchange_receiver => final_aggregation - // todo support avg - if (has_uniq_raw_res) - throw Exception("uniq raw res not supported in mpp query"); - std::shared_ptr partial_agg = std::make_shared( - executor_index, - output_schema_for_partial_agg, - has_uniq_raw_res, - false, - std::move(agg_exprs), - std::move(gby_exprs), - false); - partial_agg->children.push_back(children[0]); - std::vector partition_keys; - size_t agg_func_num = partial_agg->agg_exprs.size(); - for (size_t i = 0; i < partial_agg->gby_exprs.size(); i++) - { - partition_keys.push_back(i + agg_func_num); - } - std::shared_ptr exchange_sender - = std::make_shared(executor_index, output_schema_for_partial_agg, partition_keys.empty() ? tipb::PassThrough : tipb::Hash, partition_keys); - exchange_sender->children.push_back(partial_agg); - - std::shared_ptr exchange_receiver - = std::make_shared(executor_index, output_schema_for_partial_agg); - exchange_map[exchange_receiver->name] = std::make_pair(exchange_receiver, exchange_sender); - /// re-construct agg_exprs and gby_exprs in final_agg - for (size_t i = 0; i < partial_agg->agg_exprs.size(); i++) - { - const auto * agg_func = typeid_cast(partial_agg->agg_exprs[i].get()); - ASTPtr update_agg_expr = agg_func->clone(); - auto * update_agg_func = typeid_cast(update_agg_expr.get()); - if (agg_func->name == "count") - update_agg_func->name = "sum"; - update_agg_func->arguments->children.clear(); - update_agg_func->arguments->children.push_back(std::make_shared(output_schema_for_partial_agg[i].first)); - agg_exprs.push_back(update_agg_expr); - } - for (size_t i = 0; i < partial_agg->gby_exprs.size(); i++) - { - gby_exprs.push_back(std::make_shared(output_schema_for_partial_agg[agg_func_num + i].first)); - } - children[0] = exchange_receiver; -} - -bool Project::toTiPBExecutor(tipb::Executor * tipb_executor, int32_t collator_id, const MPPInfo & mpp_info, const Context & context) -{ - tipb_executor->set_tp(tipb::ExecType::TypeProjection); - tipb_executor->set_executor_id(name); - auto * proj = tipb_executor->mutable_projection(); - auto & input_schema = children[0]->output_schema; - for (const auto & child : exprs) - { - if (typeid_cast(child.get())) - { - /// special case, select * - for (size_t i = 0; i < input_schema.size(); i++) - { - tipb::Expr * expr = proj->add_exprs(); - expr->set_tp(tipb::ColumnRef); - *(expr->mutable_field_type()) = columnInfoToFieldType(input_schema[i].second); - WriteBufferFromOwnString ss; - encodeDAGInt64(i, ss); - expr->set_val(ss.releaseStr()); - } - continue; - } - tipb::Expr * expr = proj->add_exprs(); - astToPB(input_schema, child, expr, collator_id, context); - } - auto * children_executor = proj->mutable_child(); - return children[0]->toTiPBExecutor(children_executor, collator_id, mpp_info, context); -} - -void Project::columnPrune(std::unordered_set & used_columns) -{ - output_schema.erase(std::remove_if(output_schema.begin(), output_schema.end(), [&](const auto & field) { return used_columns.count(field.first) == 0; }), - output_schema.end()); - std::unordered_set used_input_columns; - for (auto & expr : exprs) - { - if (typeid_cast(expr.get())) - { - /// for select *, just add all its input columns, maybe - /// can do some optimization, but it is not worth for mock - /// tests - for (auto & field : children[0]->output_schema) - { - used_input_columns.emplace(field.first); - } - break; - } - if (used_columns.find(expr->getColumnName()) != used_columns.end()) - { - collectUsedColumnsFromExpr(children[0]->output_schema, expr, used_input_columns); - } - } - children[0]->columnPrune(used_input_columns); -} - -void Join::columnPrune(std::unordered_set & used_columns) -{ - std::unordered_set left_columns; - std::unordered_set right_columns; - - for (auto & field : children[0]->output_schema) - { - auto [db_name, table_name, column_name] = splitQualifiedName(field.first); - left_columns.emplace(table_name + "." + column_name); - } - - for (auto & field : children[1]->output_schema) - { - auto [db_name, table_name, column_name] = splitQualifiedName(field.first); - right_columns.emplace(table_name + "." + column_name); - } - std::unordered_set left_used_columns; - std::unordered_set right_used_columns; - - for (const auto & s : used_columns) - { - auto [db_name, table_name, col_name] = splitQualifiedName(s); - auto t = table_name + "." + col_name; - if (left_columns.find(t) != left_columns.end()) - left_used_columns.emplace(t); - - if (right_columns.find(t) != right_columns.end()) - right_used_columns.emplace(t); - } - - for (const auto & child : join_cols) - { - if (auto * identifier = typeid_cast(child.get())) - { - auto col_name = identifier->getColumnName(); - for (auto & field : children[0]->output_schema) - { - auto [db_name, table_name, column_name] = splitQualifiedName(field.first); - if (col_name == column_name) - { - left_used_columns.emplace(table_name + "." + column_name); - break; - } - } - for (auto & field : children[1]->output_schema) - { - auto [db_name, table_name, column_name] = splitQualifiedName(field.first); - if (col_name == column_name) - { - right_used_columns.emplace(table_name + "." + column_name); - break; - } - } - } - else - { - throw Exception("Only support Join on columns"); - } - } - - children[0]->columnPrune(left_used_columns); - children[1]->columnPrune(right_used_columns); - - /// update output schema - output_schema.clear(); - - for (auto & field : children[0]->output_schema) - { - if (tp == tipb::TypeRightOuterJoin && field.second.hasNotNullFlag()) - output_schema.push_back(toNullableDAGColumnInfo(field)); - else - output_schema.push_back(field); - } - - for (auto & field : children[1]->output_schema) - { - if (tp == tipb::TypeLeftOuterJoin && field.second.hasNotNullFlag()) - output_schema.push_back(toNullableDAGColumnInfo(field)); - else - output_schema.push_back(field); - } -} - -void Join::fillJoinKeyAndFieldType( - ASTPtr key, - const DAGSchema & child_schema, - tipb::Expr * tipb_key, - tipb::FieldType * tipb_field_type, - int32_t collator_id) -{ - auto * identifier = typeid_cast(key.get()); - for (size_t index = 0; index < child_schema.size(); ++index) - { - const auto & [col_name, col_info] = child_schema[index]; - - if (splitQualifiedName(col_name).column_name == identifier->getColumnName()) - { - auto tipb_type = TiDB::columnInfoToFieldType(col_info); - tipb_type.set_collate(collator_id); - - tipb_key->set_tp(tipb::ColumnRef); - WriteBufferFromOwnString ss; - encodeDAGInt64(index, ss); - tipb_key->set_val(ss.releaseStr()); - *tipb_key->mutable_field_type() = tipb_type; - - *tipb_field_type = tipb_type; - break; - } - } -} - -bool Join::toTiPBExecutor(tipb::Executor * tipb_executor, int32_t collator_id, const MPPInfo & mpp_info, const Context & context) -{ - tipb_executor->set_tp(tipb::ExecType::TypeJoin); - tipb_executor->set_executor_id(name); - - tipb::Join * join = tipb_executor->mutable_join(); - - join->set_join_type(tp); - join->set_join_exec_type(tipb::JoinExecType::TypeHashJoin); - join->set_inner_idx(1); - - for (const auto & key : join_cols) - { - fillJoinKeyAndFieldType(key, children[0]->output_schema, join->add_left_join_keys(), join->add_probe_types(), collator_id); - fillJoinKeyAndFieldType(key, children[1]->output_schema, join->add_right_join_keys(), join->add_build_types(), collator_id); - } - - for (const auto & expr : left_conds) - { - tipb::Expr * cond = join->add_left_conditions(); - astToPB(children[0]->output_schema, expr, cond, collator_id, context); - } - - for (const auto & expr : right_conds) - { - tipb::Expr * cond = join->add_right_conditions(); - astToPB(children[1]->output_schema, expr, cond, collator_id, context); - } - - DAGSchema merged_children_schema{children[0]->output_schema}; - merged_children_schema.insert(merged_children_schema.end(), children[1]->output_schema.begin(), children[1]->output_schema.end()); - - for (const auto & expr : other_conds) - { - tipb::Expr * cond = join->add_other_conditions(); - astToPB(merged_children_schema, expr, cond, collator_id, context); - } - - for (const auto & expr : other_eq_conds_from_in) - { - tipb::Expr * cond = join->add_other_eq_conditions_from_in(); - astToPB(merged_children_schema, expr, cond, collator_id, context); - } - - auto * left_child_executor = join->add_children(); - children[0]->toTiPBExecutor(left_child_executor, collator_id, mpp_info, context); - auto * right_child_executor = join->add_children(); - return children[1]->toTiPBExecutor(right_child_executor, collator_id, mpp_info, context); -} - -void Join::toMPPSubPlan(size_t & executor_index, const DAGProperties & properties, std::unordered_map, std::shared_ptr>> & exchange_map) -{ - if (properties.use_broadcast_join) - { - /// for broadcast join, always use right side as the broadcast side - std::shared_ptr right_exchange_sender - = std::make_shared(executor_index, children[1]->output_schema, tipb::Broadcast); - right_exchange_sender->children.push_back(children[1]); - - std::shared_ptr right_exchange_receiver - = std::make_shared(executor_index, children[1]->output_schema); - children[1] = right_exchange_receiver; - exchange_map[right_exchange_receiver->name] = std::make_pair(right_exchange_receiver, right_exchange_sender); - return; - } - - std::vector left_partition_keys; - std::vector right_partition_keys; - - auto push_back_partition_key = [](auto & partition_keys, const auto & child_schema, const auto & key) { - for (size_t index = 0; index < child_schema.size(); ++index) - { - if (splitQualifiedName(child_schema[index].first).column_name == key->getColumnName()) - { - partition_keys.push_back(index); - break; - } - } - }; - - for (const auto & key : join_cols) - { - push_back_partition_key(left_partition_keys, children[0]->output_schema, key); - push_back_partition_key(right_partition_keys, children[1]->output_schema, key); - } - - std::shared_ptr left_exchange_sender - = std::make_shared(executor_index, children[0]->output_schema, tipb::Hash, left_partition_keys); - left_exchange_sender->children.push_back(children[0]); - std::shared_ptr right_exchange_sender - = std::make_shared(executor_index, children[1]->output_schema, tipb::Hash, right_partition_keys); - right_exchange_sender->children.push_back(children[1]); - - std::shared_ptr left_exchange_receiver - = std::make_shared(executor_index, children[0]->output_schema); - std::shared_ptr right_exchange_receiver - = std::make_shared(executor_index, children[1]->output_schema); - children[0] = left_exchange_receiver; - children[1] = right_exchange_receiver; - - exchange_map[left_exchange_receiver->name] = std::make_pair(left_exchange_receiver, left_exchange_sender); - exchange_map[right_exchange_receiver->name] = std::make_pair(right_exchange_receiver, right_exchange_sender); -} - -bool Window::toTiPBExecutor(tipb::Executor * tipb_executor, int32_t collator_id, const MPPInfo & mpp_info, const Context & context) -{ - tipb_executor->set_tp(tipb::ExecType::TypeWindow); - tipb_executor->set_executor_id(name); - tipb_executor->set_fine_grained_shuffle_stream_count(fine_grained_shuffle_stream_count); - tipb::Window * window = tipb_executor->mutable_window(); - auto & input_schema = children[0]->output_schema; - for (const auto & expr : func_descs) - { - tipb::Expr * window_expr = window->add_func_desc(); - const auto * window_func = typeid_cast(expr.get()); - for (const auto & arg : window_func->arguments->children) - { - tipb::Expr * func = window_expr->add_children(); - astToPB(input_schema, arg, func, collator_id, context); - } - auto window_sig_it = tests::window_func_name_to_sig.find(window_func->name); - if (window_sig_it == tests::window_func_name_to_sig.end()) - throw Exception(fmt::format("Unsupported window function {}", window_func->name), ErrorCodes::LOGICAL_ERROR); - auto window_sig = window_sig_it->second; - window_expr->set_tp(window_sig); - auto * ft = window_expr->mutable_field_type(); - switch (window_sig) - { - case tipb::ExprType::Lead: - case tipb::ExprType::Lag: - { - // TODO handling complex situations - // like lead(col, offset, NULL), lead(data_type1, offset, data_type2) - assert(window_expr->children_size() >= 1 && window_expr->children_size() <= 3); - const auto first_arg_type = window_expr->children(0).field_type(); - ft->set_tp(first_arg_type.tp()); - if (window_expr->children_size() < 3) - { - auto field_type = TiDB::fieldTypeToColumnInfo(first_arg_type); - field_type.clearNotNullFlag(); - ft->set_flag(field_type.flag); - } - else - { - const auto third_arg_type = window_expr->children(2).field_type(); - assert(first_arg_type.tp() == third_arg_type.tp()); - ft->set_flag(TiDB::fieldTypeToColumnInfo(first_arg_type).hasNotNullFlag() - ? third_arg_type.flag() - : first_arg_type.flag()); - } - ft->set_collate(first_arg_type.collate()); - ft->set_flen(first_arg_type.flen()); - ft->set_decimal(first_arg_type.decimal()); - break; - } - default: - ft->set_tp(TiDB::TypeLongLong); - ft->set_flag(TiDB::ColumnFlagBinary); - ft->set_collate(collator_id); - ft->set_flen(21); - ft->set_decimal(-1); - } - } - - for (const auto & child : order_by_exprs) - { - auto * elem = typeid_cast(child.get()); - if (!elem) - throw Exception("Invalid order by element", ErrorCodes::LOGICAL_ERROR); - tipb::ByItem * by = window->add_order_by(); - by->set_desc(elem->direction < 0); - tipb::Expr * expr = by->mutable_expr(); - astToPB(children[0]->output_schema, elem->children[0], expr, collator_id, context); - } - - for (const auto & child : partition_by_exprs) - { - auto * elem = typeid_cast(child.get()); - if (!elem) - throw Exception("Invalid partition by element", ErrorCodes::LOGICAL_ERROR); - tipb::ByItem * by = window->add_partition_by(); - by->set_desc(elem->direction < 0); - tipb::Expr * expr = by->mutable_expr(); - astToPB(children[0]->output_schema, elem->children[0], expr, collator_id, context); - } - - if (frame.type.has_value()) - { - tipb::WindowFrame * mut_frame = window->mutable_frame(); - mut_frame->set_type(frame.type.value()); - if (frame.start.has_value()) - { - auto * start = mut_frame->mutable_start(); - start->set_offset(std::get<2>(frame.start.value())); - start->set_unbounded(std::get<1>(frame.start.value())); - start->set_type(std::get<0>(frame.start.value())); - } - - if (frame.end.has_value()) - { - auto * end = mut_frame->mutable_end(); - end->set_offset(std::get<2>(frame.end.value())); - end->set_unbounded(std::get<1>(frame.end.value())); - end->set_type(std::get<0>(frame.end.value())); - } - } - - auto * children_executor = window->mutable_child(); - return children[0]->toTiPBExecutor(children_executor, collator_id, mpp_info, context); -} - -bool Sort::toTiPBExecutor(tipb::Executor * tipb_executor, int32_t collator_id, const MPPInfo & mpp_info, const Context & context) -{ - tipb_executor->set_tp(tipb::ExecType::TypeSort); - tipb_executor->set_executor_id(name); - tipb_executor->set_fine_grained_shuffle_stream_count(fine_grained_shuffle_stream_count); - tipb::Sort * sort = tipb_executor->mutable_sort(); - sort->set_ispartialsort(is_partial_sort); - - for (const auto & child : by_exprs) - { - auto * elem = typeid_cast(child.get()); - if (!elem) - throw Exception("Invalid order by element", ErrorCodes::LOGICAL_ERROR); - tipb::ByItem * by = sort->add_byitems(); - by->set_desc(elem->direction < 0); - tipb::Expr * expr = by->mutable_expr(); - astToPB(children[0]->output_schema, elem->children[0], expr, collator_id, context); - } - - auto * children_executor = sort->mutable_child(); - return children[0]->toTiPBExecutor(children_executor, collator_id, mpp_info, context); -} - -} // namespace mock - -ExecutorPtr compileTableScan(size_t & executor_index, TableInfo & table_info, const String & db, const String & table_name, bool append_pk_column) -{ - DAGSchema ts_output; - for (const auto & column_info : table_info.columns) - { - ColumnInfo ci; - ci.tp = column_info.tp; - ci.flag = column_info.flag; - ci.flen = column_info.flen; - ci.decimal = column_info.decimal; - ci.elems = column_info.elems; - ci.default_value = column_info.default_value; - ci.origin_default_value = column_info.origin_default_value; - /// use qualified name as the column name to handle multiple table queries, not very - /// efficient but functionally enough for mock test - ts_output.emplace_back(std::make_pair(db + "." + table_name + "." + column_info.name, std::move(ci))); - } - if (append_pk_column) - { - ColumnInfo ci; - ci.tp = TiDB::TypeLongLong; - ci.setPriKeyFlag(); - ci.setNotNullFlag(); - ts_output.emplace_back(std::make_pair(MutableSupport::tidb_pk_column_name, std::move(ci))); - } - - return std::make_shared(executor_index, ts_output, table_info); -} - -ExecutorPtr compileSelection(ExecutorPtr input, size_t & executor_index, ASTPtr filter) -{ - std::vector conditions; - compileFilter(input->output_schema, filter, conditions); - auto selection = std::make_shared(executor_index, input->output_schema, std::move(conditions)); - selection->children.push_back(input); - return selection; -} - -ExecutorPtr compileTopN(ExecutorPtr input, size_t & executor_index, ASTPtr order_exprs, ASTPtr limit_expr) -{ - std::vector order_columns; - for (const auto & child : order_exprs->children) - { - auto * elem = typeid_cast(child.get()); - if (!elem) - throw Exception("Invalid order by element", ErrorCodes::LOGICAL_ERROR); - order_columns.push_back(child); - compileExpr(input->output_schema, elem->children[0]); - } - auto limit = safeGet(typeid_cast(*limit_expr).value); - auto top_n = std::make_shared(executor_index, input->output_schema, std::move(order_columns), limit); - top_n->children.push_back(input); - return top_n; -} - -ExecutorPtr compileLimit(ExecutorPtr input, size_t & executor_index, ASTPtr limit_expr) -{ - auto limit_length = safeGet(typeid_cast(*limit_expr).value); - auto limit = std::make_shared(executor_index, input->output_schema, limit_length); - limit->children.push_back(input); - return limit; -} - -ExecutorPtr compileAggregation(ExecutorPtr input, size_t & executor_index, ASTPtr agg_funcs, ASTPtr group_by_exprs) -{ - std::vector agg_exprs; - std::vector gby_exprs; - DAGSchema output_schema; - bool has_uniq_raw_res = false; - bool need_append_project = false; - if (agg_funcs != nullptr) - { - for (const auto & expr : agg_funcs->children) - { - const auto * func = typeid_cast(expr.get()); - if (!func || !AggregateFunctionFactory::instance().isAggregateFunctionName(func->name)) - { - need_append_project = true; - continue; - } - - agg_exprs.push_back(expr); - std::vector children_ci; - - for (const auto & arg : func->arguments->children) - { - children_ci.push_back(compileExpr(input->output_schema, arg)); - } - - TiDB::ColumnInfo ci; - if (func->name == "count") - { - ci.tp = TiDB::TypeLongLong; - ci.flag = TiDB::ColumnFlagUnsigned | TiDB::ColumnFlagNotNull; - } - else if (func->name == "max" || func->name == "min" || func->name == "first_row" || func->name == "sum") - { - ci = children_ci[0]; - ci.flag &= ~TiDB::ColumnFlagNotNull; - } - else if (func->name == uniq_raw_res_name) - { - has_uniq_raw_res = true; - ci.tp = TiDB::TypeString; - ci.flag = 1; - } - // TODO: Other agg func. - else - { - throw Exception("Unsupported agg function " + func->name, ErrorCodes::LOGICAL_ERROR); - } - - output_schema.emplace_back(std::make_pair(func->getColumnName(), ci)); - } - } - - if (group_by_exprs != nullptr) - { - for (const auto & child : group_by_exprs->children) - { - gby_exprs.push_back(child); - auto ci = compileExpr(input->output_schema, child); - output_schema.emplace_back(std::make_pair(child->getColumnName(), ci)); - } - } - - auto aggregation = std::make_shared( - executor_index, - output_schema, - has_uniq_raw_res, - need_append_project, - std::move(agg_exprs), - std::move(gby_exprs), - true); - aggregation->children.push_back(input); - return aggregation; -} - -ExecutorPtr compileProject(ExecutorPtr input, size_t & executor_index, ASTPtr select_list) -{ - std::vector exprs; - DAGSchema output_schema; - for (const auto & expr : select_list->children) - { - if (typeid_cast(expr.get())) - { - /// special case, select * - exprs.push_back(expr); - const auto & last_output = input->output_schema; - for (const auto & field : last_output) - { - // todo need to use the subquery alias to reconstruct the field - // name if subquery is supported - output_schema.emplace_back(field.first, field.second); - } - } - else - { - exprs.push_back(expr); - auto ft = std::find_if(input->output_schema.begin(), input->output_schema.end(), [&](const auto & field) { return field.first == expr->getColumnName(); }); - if (ft != input->output_schema.end()) - { - output_schema.emplace_back(ft->first, ft->second); - continue; - } - const auto * func = typeid_cast(expr.get()); - if (func && AggregateFunctionFactory::instance().isAggregateFunctionName(func->name)) - { - throw Exception("No such agg " + func->getColumnName(), ErrorCodes::NO_SUCH_COLUMN_IN_TABLE); - } - else - { - auto ci = compileExpr(input->output_schema, expr); - // todo need to use the subquery alias to reconstruct the field - // name if subquery is supported - output_schema.emplace_back(std::make_pair(expr->getColumnName(), ci)); - } - } - } - auto project = std::make_shared(executor_index, output_schema, std::move(exprs)); - project->children.push_back(input); - return project; -} - -static void buildLeftSideJoinSchema(DAGSchema & schema, const DAGSchema & left_schema, tipb::JoinType tp) -{ - for (const auto & field : left_schema) - { - if (tp == tipb::JoinType::TypeRightOuterJoin && field.second.hasNotNullFlag()) - schema.push_back(toNullableDAGColumnInfo(field)); - else - schema.push_back(field); - } -} - -static void buildRightSideJoinSchema(DAGSchema & schema, const DAGSchema & right_schema, tipb::JoinType tp) -{ - /// Note: for semi join, the right table column is ignored - /// but for (anti) left outer semi join, a 1/0 (uint8) field is pushed back - /// indicating whether right table has matching row(s), see comment in ASTTableJoin::Kind for details. - if (tp == tipb::JoinType::TypeLeftOuterSemiJoin || tp == tipb::JoinType::TypeAntiLeftOuterSemiJoin) - { - tipb::FieldType field_type{}; - field_type.set_tp(TiDB::TypeTiny); - field_type.set_charset("binary"); - field_type.set_collate(TiDB::ITiDBCollator::BINARY); - field_type.set_flag(0); - field_type.set_flen(-1); - field_type.set_decimal(-1); - schema.push_back(std::make_pair("", TiDB::fieldTypeToColumnInfo(field_type))); - } - else if (tp != tipb::JoinType::TypeSemiJoin && tp != tipb::JoinType::TypeAntiSemiJoin) - { - for (const auto & field : right_schema) - { - if (tp == tipb::JoinType::TypeLeftOuterJoin && field.second.hasNotNullFlag()) - schema.push_back(toNullableDAGColumnInfo(field)); - else - schema.push_back(field); - } - } -} - -// compileJoin constructs a mocked Join executor node, note that all conditional expression params can be default -ExecutorPtr compileJoin(size_t & executor_index, - ExecutorPtr left, - ExecutorPtr right, - tipb::JoinType tp, - const ASTs & join_cols, - const ASTs & left_conds, - const ASTs & right_conds, - const ASTs & other_conds, - const ASTs & other_eq_conds_from_in) -{ - DAGSchema output_schema; - - buildLeftSideJoinSchema(output_schema, left->output_schema, tp); - buildRightSideJoinSchema(output_schema, right->output_schema, tp); - - auto join = std::make_shared(executor_index, output_schema, tp, join_cols, left_conds, right_conds, other_conds, other_eq_conds_from_in); - join->children.push_back(left); - join->children.push_back(right); - - return join; -} - -ExecutorPtr compileJoin(size_t & executor_index, ExecutorPtr left, ExecutorPtr right, ASTPtr params) -{ - tipb::JoinType tp; - const auto & ast_join = (static_cast(*params)); - switch (ast_join.kind) - { - case ASTTableJoin::Kind::Inner: - tp = tipb::JoinType::TypeInnerJoin; - break; - case ASTTableJoin::Kind::Left: - tp = tipb::JoinType::TypeLeftOuterJoin; - break; - case ASTTableJoin::Kind::Right: - tp = tipb::JoinType::TypeRightOuterJoin; - break; - default: - throw Exception("Unsupported join type"); - } - - // in legacy test framework, we only support using_expr of join - ASTs join_cols; - if (ast_join.using_expression_list) - { - for (const auto & key : ast_join.using_expression_list->children) - { - join_cols.push_back(key); - } - } - return compileJoin(executor_index, left, right, tp, join_cols); -} - - -ExecutorPtr compileExchangeSender(ExecutorPtr input, size_t & executor_index, tipb::ExchangeType exchange_type) -{ - ExecutorPtr exchange_sender = std::make_shared(executor_index, input->output_schema, exchange_type); - exchange_sender->children.push_back(input); - return exchange_sender; -} - -ExecutorPtr compileExchangeReceiver(size_t & executor_index, DAGSchema schema, uint64_t fine_grained_shuffle_stream_count) -{ - ExecutorPtr exchange_receiver = std::make_shared(executor_index, schema, fine_grained_shuffle_stream_count); - return exchange_receiver; -} - -ExecutorPtr compileWindow(ExecutorPtr input, size_t & executor_index, ASTPtr func_desc_list, ASTPtr partition_by_expr_list, ASTPtr order_by_expr_list, mock::MockWindowFrame frame, uint64_t fine_grained_shuffle_stream_count) -{ - std::vector partition_columns; - if (partition_by_expr_list != nullptr) - { - for (const auto & child : partition_by_expr_list->children) - { - auto * elem = typeid_cast(child.get()); - if (!elem) - throw Exception("Invalid partition by element", ErrorCodes::LOGICAL_ERROR); - partition_columns.push_back(child); - compileExpr(input->output_schema, elem->children[0]); - } - } - - std::vector order_columns; - if (order_by_expr_list != nullptr) - { - for (const auto & child : order_by_expr_list->children) - { - auto * elem = typeid_cast(child.get()); - if (!elem) - throw Exception("Invalid order by element", ErrorCodes::LOGICAL_ERROR); - order_columns.push_back(child); - compileExpr(input->output_schema, elem->children[0]); - } - } - - DAGSchema output_schema; - output_schema.insert(output_schema.end(), input->output_schema.begin(), input->output_schema.end()); - - std::vector window_exprs; - if (func_desc_list != nullptr) - { - for (const auto & expr : func_desc_list->children) - { - const auto * func = typeid_cast(expr.get()); - window_exprs.push_back(expr); - std::vector children_ci; - for (const auto & arg : func->arguments->children) - { - children_ci.push_back(compileExpr(input->output_schema, arg)); - } - // TODO: add more window functions - TiDB::ColumnInfo ci; - switch (tests::window_func_name_to_sig[func->name]) - { - case tipb::ExprType::RowNumber: - case tipb::ExprType::Rank: - case tipb::ExprType::DenseRank: - { - ci.tp = TiDB::TypeLongLong; - ci.flag = TiDB::ColumnFlagBinary; - break; - } - case tipb::ExprType::Lead: - case tipb::ExprType::Lag: - { - // TODO handling complex situations - // like lead(col, offset, NULL), lead(data_type1, offset, data_type2) - assert(children_ci.size() >= 1 && children_ci.size() <= 3); - if (children_ci.size() < 3) - { - ci = children_ci[0]; - ci.clearNotNullFlag(); - } - else - { - assert(children_ci[0].tp == children_ci[2].tp); - ci = children_ci[0].hasNotNullFlag() ? children_ci[2] : children_ci[0]; - } - break; - } - default: - throw Exception(fmt::format("Unsupported window function {}", func->name), ErrorCodes::LOGICAL_ERROR); - } - output_schema.emplace_back(std::make_pair(func->getColumnName(), ci)); - } - } - - ExecutorPtr window = std::make_shared( - executor_index, - output_schema, - window_exprs, - std::move(partition_columns), - std::move(order_columns), - frame, - fine_grained_shuffle_stream_count); - window->children.push_back(input); - return window; -} - -ExecutorPtr compileSort(ExecutorPtr input, size_t & executor_index, ASTPtr order_by_expr_list, bool is_partial_sort, uint64_t fine_grained_shuffle_stream_count) -{ - std::vector order_columns; - if (order_by_expr_list != nullptr) - { - for (const auto & child : order_by_expr_list->children) - { - auto * elem = typeid_cast(child.get()); - if (!elem) - throw Exception("Invalid order by element", ErrorCodes::LOGICAL_ERROR); - order_columns.push_back(child); - compileExpr(input->output_schema, elem->children[0]); - } - } - ExecutorPtr sort = std::make_shared(executor_index, input->output_schema, std::move(order_columns), is_partial_sort, fine_grained_shuffle_stream_count); - sort->children.push_back(input); - return sort; -} -} // namespace DB diff --git a/dbms/src/Debug/MockExecutor/astToExecutor.h b/dbms/src/Debug/MockExecutor/astToExecutor.h deleted file mode 100644 index aa82121a707..00000000000 --- a/dbms/src/Debug/MockExecutor/astToExecutor.h +++ /dev/null @@ -1,362 +0,0 @@ -// Copyright 2022 PingCAP, Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -#pragma once - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include - -namespace DB -{ -struct MPPCtx -{ - Timestamp start_ts; - Int64 next_task_id; - std::vector sender_target_task_ids; - explicit MPPCtx(Timestamp start_ts_) - : start_ts(start_ts_) - , next_task_id(1) - {} -}; - -using MPPCtxPtr = std::shared_ptr; - -struct MPPInfo -{ - Timestamp start_ts; - Int64 partition_id; - Int64 task_id; - const std::vector sender_target_task_ids; - const std::unordered_map> receiver_source_task_ids_map; - - MPPInfo( - Timestamp start_ts_, - Int64 partition_id_, - Int64 task_id_, - const std::vector & sender_target_task_ids_, - const std::unordered_map> & receiver_source_task_ids_map_) - : start_ts(start_ts_) - , partition_id(partition_id_) - , task_id(task_id_) - , sender_target_task_ids(sender_target_task_ids_) - , receiver_source_task_ids_map(receiver_source_task_ids_map_) - {} -}; - -struct TaskMeta -{ - UInt64 start_ts = 0; - Int64 task_id = 0; - Int64 partition_id = 0; -}; - -using TaskMetas = std::vector; - -namespace mock -{ -struct ExchangeSender; -struct ExchangeReceiver; -struct Executor -{ - size_t index; - String name; - DAGSchema output_schema; - std::vector> children; - virtual void columnPrune(std::unordered_set & used_columns) = 0; - Executor(size_t & index_, String && name_, const DAGSchema & output_schema_) - : index(index_) - , name(std::move(name_)) - , output_schema(output_schema_) - { - index_++; - } - virtual bool toTiPBExecutor(tipb::Executor * tipb_executor, int32_t collator_id, const MPPInfo & mpp_info, const Context & context) - = 0; - virtual void toMPPSubPlan(size_t & executor_index, const DAGProperties & properties, std::unordered_map, std::shared_ptr>> & exchange_map) - { - children[0]->toMPPSubPlan(executor_index, properties, exchange_map); - } - virtual ~Executor() = default; -}; - -struct ExchangeSender : Executor -{ - tipb::ExchangeType type; - TaskMetas task_metas; - std::vector partition_keys; - ExchangeSender(size_t & index, const DAGSchema & output, tipb::ExchangeType type_, const std::vector & partition_keys_ = {}) - : Executor(index, "exchange_sender_" + std::to_string(index), output) - , type(type_) - , partition_keys(partition_keys_) - {} - void columnPrune(std::unordered_set &) override { throw Exception("Should not reach here"); } - bool toTiPBExecutor(tipb::Executor * tipb_executor, int32_t collator_id, const MPPInfo & mpp_info, const Context & context) override; -}; - -struct ExchangeReceiver : Executor -{ - TaskMetas task_metas; - uint64_t fine_grained_shuffle_stream_count; - - ExchangeReceiver(size_t & index, const DAGSchema & output, uint64_t fine_grained_shuffle_stream_count_ = 0) - : Executor(index, "exchange_receiver_" + std::to_string(index), output) - , fine_grained_shuffle_stream_count(fine_grained_shuffle_stream_count_) - {} - void columnPrune(std::unordered_set &) override { throw Exception("Should not reach here"); } - bool toTiPBExecutor(tipb::Executor * tipb_executor, int32_t collator_id, const MPPInfo & mpp_info, const Context &) override; -}; - -struct TableScan : public Executor -{ - TableInfo table_info; - /// used by column pruner - TableScan(size_t & index_, const DAGSchema & output_schema_, const TableInfo & table_info_) - : Executor(index_, "table_scan_" + std::to_string(index_), output_schema_) - , table_info(table_info_) - {} - void columnPrune(std::unordered_set & used_columns) override; - bool toTiPBExecutor(tipb::Executor * tipb_executor, int32_t, const MPPInfo &, const Context &) override; - void toMPPSubPlan(size_t &, const DAGProperties &, std::unordered_map, std::shared_ptr>> &) override - {} - - void setTipbColumnInfo(tipb::ColumnInfo * ci, const DAGColumnInfo & dag_column_info) const - { - auto names = splitQualifiedName(dag_column_info.first); - if (names.column_name == MutableSupport::tidb_pk_column_name) - ci->set_column_id(-1); - else - ci->set_column_id(table_info.getColumnID(names.column_name)); - ci->set_tp(dag_column_info.second.tp); - ci->set_flag(dag_column_info.second.flag); - ci->set_columnlen(dag_column_info.second.flen); - ci->set_decimal(dag_column_info.second.decimal); - if (!dag_column_info.second.elems.empty()) - { - for (const auto & pair : dag_column_info.second.elems) - { - ci->add_elems(pair.first); - } - } - } -}; - -struct Selection : public Executor -{ - std::vector conditions; - Selection(size_t & index_, const DAGSchema & output_schema_, std::vector conditions_) - : Executor(index_, "selection_" + std::to_string(index_), output_schema_) - , conditions(std::move(conditions_)) - {} - bool toTiPBExecutor(tipb::Executor * tipb_executor, int32_t collator_id, const MPPInfo & mpp_info, const Context & context) override; - void columnPrune(std::unordered_set & used_columns) override; -}; - -struct TopN : public Executor -{ - std::vector order_columns; - size_t limit; - TopN(size_t & index_, const DAGSchema & output_schema_, std::vector order_columns_, size_t limit_) - : Executor(index_, "topn_" + std::to_string(index_), output_schema_) - , order_columns(std::move(order_columns_)) - , limit(limit_) - {} - bool toTiPBExecutor(tipb::Executor * tipb_executor, int32_t collator_id, const MPPInfo & mpp_info, const Context & context) override; - void columnPrune(std::unordered_set & used_columns) override; -}; - -struct Limit : public Executor -{ - size_t limit; - Limit(size_t & index_, const DAGSchema & output_schema_, size_t limit_) - : Executor(index_, "limit_" + std::to_string(index_), output_schema_) - , limit(limit_) - {} - bool toTiPBExecutor(tipb::Executor * tipb_executor, int32_t collator_id, const MPPInfo & mpp_info, const Context & context) override; - void columnPrune(std::unordered_set & used_columns) override; -}; - -struct Aggregation : public Executor -{ - bool has_uniq_raw_res; - bool need_append_project; - std::vector agg_exprs; - std::vector gby_exprs; - bool is_final_mode; - DAGSchema output_schema_for_partial_agg; - Aggregation(size_t & index_, const DAGSchema & output_schema_, bool has_uniq_raw_res_, bool need_append_project_, std::vector agg_exprs_, std::vector gby_exprs_, bool is_final_mode_) - : Executor(index_, "aggregation_" + std::to_string(index_), output_schema_) - , has_uniq_raw_res(has_uniq_raw_res_) - , need_append_project(need_append_project_) - , agg_exprs(std::move(agg_exprs_)) - , gby_exprs(std::move(gby_exprs_)) - , is_final_mode(is_final_mode_) - {} - bool toTiPBExecutor(tipb::Executor * tipb_executor, int32_t collator_id, const MPPInfo & mpp_info, const Context & context) override; - void columnPrune(std::unordered_set & used_columns) override; - void toMPPSubPlan(size_t & executor_index, const DAGProperties & properties, std::unordered_map, std::shared_ptr>> & exchange_map) override; -}; - -struct Project : public Executor -{ - std::vector exprs; - Project(size_t & index_, const DAGSchema & output_schema_, std::vector && exprs_) - : Executor(index_, "project_" + std::to_string(index_), output_schema_) - , exprs(std::move(exprs_)) - {} - bool toTiPBExecutor(tipb::Executor * tipb_executor, int32_t collator_id, const MPPInfo & mpp_info, const Context & context) override; - void columnPrune(std::unordered_set & used_columns) override; -}; - -struct Join : Executor -{ - tipb::JoinType tp; - - const ASTs join_cols{}; - const ASTs left_conds{}; - const ASTs right_conds{}; - const ASTs other_conds{}; - const ASTs other_eq_conds_from_in{}; - - Join(size_t & index_, const DAGSchema & output_schema_, tipb::JoinType tp_, const ASTs & join_cols_, const ASTs & l_conds, const ASTs & r_conds, const ASTs & o_conds, const ASTs & o_eq_conds) - : Executor(index_, "Join_" + std::to_string(index_), output_schema_) - , tp(tp_) - , join_cols(join_cols_) - , left_conds(l_conds) - , right_conds(r_conds) - , other_conds(o_conds) - , other_eq_conds_from_in(o_eq_conds) - { - if (!(join_cols.size() + left_conds.size() + right_conds.size() + other_conds.size() + other_eq_conds_from_in.size())) - throw Exception("No join condition found."); - } - - void columnPrune(std::unordered_set & used_columns) override; - - static void fillJoinKeyAndFieldType( - ASTPtr key, - const DAGSchema & schema, - tipb::Expr * tipb_key, - tipb::FieldType * tipb_field_type, - int32_t collator_id); - - bool toTiPBExecutor(tipb::Executor * tipb_executor, int32_t collator_id, const MPPInfo & mpp_info, const Context & context) override; - - void toMPPSubPlan(size_t & executor_index, const DAGProperties & properties, std::unordered_map, std::shared_ptr>> & exchange_map) override; -}; - -using MockWindowFrameBound = std::tuple; - -struct MockWindowFrame -{ - std::optional type; - std::optional start; - std::optional end; - // TODO: support calcFuncs -}; - -struct Window : Executor -{ - std::vector func_descs; - std::vector partition_by_exprs; - std::vector order_by_exprs; - MockWindowFrame frame; - uint64_t fine_grained_shuffle_stream_count; - - Window(size_t & index_, const DAGSchema & output_schema_, std::vector func_descs_, std::vector partition_by_exprs_, std::vector order_by_exprs_, MockWindowFrame frame_, uint64_t fine_grained_shuffle_stream_count_ = 0) - : Executor(index_, "window_" + std::to_string(index_), output_schema_) - , func_descs(std::move(func_descs_)) - , partition_by_exprs(std::move(partition_by_exprs_)) - , order_by_exprs(order_by_exprs_) - , frame(frame_) - , fine_grained_shuffle_stream_count(fine_grained_shuffle_stream_count_) - { - } - // Currently only use Window Executor in Unit Test which don't call columnPrume. - // TODO: call columnPrune in unit test and further benchmark test to eliminate compute process. - void columnPrune(std::unordered_set &) override { throw Exception("Should not reach here"); } - bool toTiPBExecutor(tipb::Executor * tipb_executor, int32_t collator_id, const MPPInfo & mpp_info, const Context & context) override; -}; - -struct Sort : Executor -{ - std::vector by_exprs; - bool is_partial_sort; - uint64_t fine_grained_shuffle_stream_count; - - Sort(size_t & index_, const DAGSchema & output_schema_, std::vector by_exprs_, bool is_partial_sort_, uint64_t fine_grained_shuffle_stream_count_ = 0) - : Executor(index_, "sort_" + std::to_string(index_), output_schema_) - , by_exprs(by_exprs_) - , is_partial_sort(is_partial_sort_) - , fine_grained_shuffle_stream_count(fine_grained_shuffle_stream_count_) - { - } - // Currently only use Sort Executor in Unit Test which don't call columnPrume. - // TODO: call columnPrune in unit test and further benchmark test to eliminate compute process. - void columnPrune(std::unordered_set &) override { throw Exception("Should not reach here"); } - bool toTiPBExecutor(tipb::Executor * tipb_executor, int32_t collator_id, const MPPInfo & mpp_info, const Context & context) override; -}; -} // namespace mock - -using ExecutorPtr = std::shared_ptr; - -ExecutorPtr compileTableScan(size_t & executor_index, TableInfo & table_info, const String & db, const String & table_name, bool append_pk_column); - -ExecutorPtr compileSelection(ExecutorPtr input, size_t & executor_index, ASTPtr filter); - -ExecutorPtr compileTopN(ExecutorPtr input, size_t & executor_index, ASTPtr order_exprs, ASTPtr limit_expr); - -ExecutorPtr compileLimit(ExecutorPtr input, size_t & executor_index, ASTPtr limit_expr); - -ExecutorPtr compileAggregation(ExecutorPtr input, size_t & executor_index, ASTPtr agg_funcs, ASTPtr group_by_exprs); - -ExecutorPtr compileProject(ExecutorPtr input, size_t & executor_index, ASTPtr select_list); - -/// Note: this api is only used by legacy test framework for compatibility purpose, which will be depracated soon, -/// so please avoid using it. -/// Old executor test framework bases on ch's parser to translate sql string to ast tree, then manually to DAGRequest. -/// However, as for join executor, this translation, from ASTTableJoin to tipb::Join, is not a one-to-one mapping -/// because of the different join classification model used by these two structures. Therefore, under old test framework, -/// it is hard to fully test join executor. New framework aims to directly construct DAGRequest, so new framework APIs for join should -/// avoid using ASTTableJoin. -ExecutorPtr compileJoin(size_t & executor_index, ExecutorPtr left, ExecutorPtr right, ASTPtr params); - -ExecutorPtr compileJoin(size_t & executor_index, ExecutorPtr left, ExecutorPtr right, tipb::JoinType tp, const ASTs & join_cols, const ASTs & left_conds = {}, const ASTs & right_conds = {}, const ASTs & other_conds = {}, const ASTs & other_eq_conds_from_in = {}); - -ExecutorPtr compileExchangeSender(ExecutorPtr input, size_t & executor_index, tipb::ExchangeType exchange_type); - -ExecutorPtr compileExchangeReceiver(size_t & executor_index, DAGSchema schema, uint64_t fine_grained_shuffle_stream_count = 0); - -ExecutorPtr compileWindow(ExecutorPtr input, size_t & executor_index, ASTPtr func_desc_list, ASTPtr partition_by_expr_list, ASTPtr order_by_expr_list, mock::MockWindowFrame frame, uint64_t fine_grained_shuffle_stream_count = 0); - -ExecutorPtr compileSort(ExecutorPtr input, size_t & executor_index, ASTPtr order_by_expr_list, bool is_partial_sort, uint64_t fine_grained_shuffle_stream_count = 0); - -void literalFieldToTiPBExpr(const ColumnInfo & ci, const Field & field, tipb::Expr * expr, Int32 collator_id); -} // namespace DB diff --git a/dbms/src/Debug/dbgFuncCoprocessor.cpp b/dbms/src/Debug/dbgFuncCoprocessor.cpp index c0eca662887..013d24fa23e 100644 --- a/dbms/src/Debug/dbgFuncCoprocessor.cpp +++ b/dbms/src/Debug/dbgFuncCoprocessor.cpp @@ -20,7 +20,6 @@ #include #include #include -#include #include #include #include @@ -57,7 +56,6 @@ #include #include - namespace DB { namespace ErrorCodes @@ -70,6 +68,7 @@ extern const int NO_SUCH_COLUMN_IN_TABLE; using DAGColumnInfo = std::pair; using DAGSchema = std::vector; using TiFlashTestEnv = tests::TiFlashTestEnv; +using ExecutorBinderPtr = mock::ExecutorBinderPtr; static const String ENCODE_TYPE_NAME = "encode_type"; static const String TZ_OFFSET_NAME = "tz_offset"; static const String TZ_NAME_NAME = "tz_name"; @@ -590,13 +589,13 @@ BlockInputStreamPtr dbgFuncMockTiDBQuery(Context & context, const ASTs & args) struct QueryFragment { - ExecutorPtr root_executor; + ExecutorBinderPtr root_executor; TableID table_id; bool is_top_fragment; std::vector sender_target_task_ids; std::unordered_map> receiver_source_task_ids_map; std::vector task_ids; - QueryFragment(ExecutorPtr root_executor_, TableID table_id_, bool is_top_fragment_, std::vector && sender_target_task_ids_ = {}, std::unordered_map> && receiver_source_task_ids_map_ = {}, std::vector && task_ids_ = {}) + QueryFragment(ExecutorBinderPtr root_executor_, TableID table_id_, bool is_top_fragment_, std::vector && sender_target_task_ids_ = {}, std::unordered_map> && receiver_source_task_ids_map_ = {}, std::vector && task_ids_ = {}) : root_executor(std::move(root_executor_)) , table_id(table_id_) , is_top_fragment(is_top_fragment_) @@ -660,15 +659,15 @@ struct QueryFragment using QueryFragments = std::vector; -TableID findTableIdForQueryFragment(ExecutorPtr root_executor, bool must_have_table_id) +TableID findTableIdForQueryFragment(ExecutorBinderPtr root_executor, bool must_have_table_id) { - ExecutorPtr current_executor = root_executor; + ExecutorBinderPtr current_executor = root_executor; while (!current_executor->children.empty()) { - ExecutorPtr non_exchange_child; + ExecutorBinderPtr non_exchange_child; for (const auto & c : current_executor->children) { - if (dynamic_cast(c.get())) + if (dynamic_cast(c.get())) continue; if (non_exchange_child != nullptr) throw Exception("More than one non-exchange child, should not happen"); @@ -682,25 +681,25 @@ TableID findTableIdForQueryFragment(ExecutorPtr root_executor, bool must_have_ta } current_executor = non_exchange_child; } - auto * ts = dynamic_cast(current_executor.get()); + auto * ts = dynamic_cast(current_executor.get()); if (ts == nullptr) { if (must_have_table_id) throw Exception("Table scan not found"); return -1; } - return ts->table_info.id; + return ts->getTableId(); } QueryFragments mppQueryToQueryFragments( - ExecutorPtr root_executor, + ExecutorBinderPtr root_executor, size_t & executor_index, const DAGProperties & properties, bool for_root_fragment, MPPCtxPtr mpp_ctx) { QueryFragments fragments; - std::unordered_map, std::shared_ptr>> exchange_map; + std::unordered_map, std::shared_ptr>> exchange_map; root_executor->toMPPSubPlan(executor_index, properties, exchange_map); TableID table_id = findTableIdForQueryFragment(root_executor, exchange_map.empty()); std::vector sender_target_task_ids = mpp_ctx->sender_target_task_ids; @@ -708,7 +707,7 @@ QueryFragments mppQueryToQueryFragments( size_t current_task_num = properties.mpp_partition_num; for (auto & exchange : exchange_map) { - if (exchange.second.second->type == tipb::ExchangeType::PassThrough) + if (exchange.second.second->getType() == tipb::ExchangeType::PassThrough) { current_task_num = 1; break; @@ -728,12 +727,12 @@ QueryFragments mppQueryToQueryFragments( return fragments; } -QueryFragments queryPlanToQueryFragments(const DAGProperties & properties, ExecutorPtr root_executor, size_t & executor_index) +QueryFragments queryPlanToQueryFragments(const DAGProperties & properties, ExecutorBinderPtr root_executor, size_t & executor_index) { if (properties.is_mpp_query) { - ExecutorPtr root_exchange_sender - = std::make_shared(executor_index, root_executor->output_schema, tipb::PassThrough); + ExecutorBinderPtr root_exchange_sender + = std::make_shared(executor_index, root_executor->output_schema, tipb::PassThrough); root_exchange_sender->children.push_back(root_executor); root_executor = root_exchange_sender; MPPCtxPtr mpp_ctx = std::make_shared(properties.start_ts); @@ -750,7 +749,7 @@ QueryFragments queryPlanToQueryFragments(const DAGProperties & properties, Execu QueryTasks queryPlanToQueryTasks( const DAGProperties & properties, - ExecutorPtr root_executor, + ExecutorBinderPtr root_executor, size_t & executor_index, const Context & context) { @@ -788,7 +787,7 @@ const ASTTablesInSelectQueryElement * getJoin(ASTSelectQuery & ast_query) return joined_table; } -std::pair compileQueryBlock( +std::pair compileQueryBlock( Context & context, size_t & executor_index, SchemaFetcher schema_fetcher, @@ -800,7 +799,7 @@ std::pair compileQueryBlock( /// the return value of `ApproxCountDistinct` is just the raw result, we need to convert it to a readable /// value when decoding the result(using `UniqRawResReformatBlockOutputStream`) bool has_uniq_raw_res = false; - ExecutorPtr root_executor = nullptr; + ExecutorBinderPtr root_executor = nullptr; TableInfo table_info; String table_alias; @@ -843,7 +842,7 @@ std::pair compileQueryBlock( } } } - root_executor = compileTableScan(executor_index, table_info, "", table_alias, append_pk_column); + root_executor = mock::compileTableScan(executor_index, table_info, "", table_alias, append_pk_column); } } else @@ -904,9 +903,9 @@ std::pair compileQueryBlock( } } } - auto left_ts = compileTableScan(executor_index, left_table_info, "", left_table_alias, left_append_pk_column); - auto right_ts = compileTableScan(executor_index, right_table_info, "", right_table_alias, right_append_pk_column); - root_executor = compileJoin(executor_index, left_ts, right_ts, joined_table->table_join); + auto left_ts = mock::compileTableScan(executor_index, left_table_info, "", left_table_alias, left_append_pk_column); + auto right_ts = mock::compileTableScan(executor_index, right_table_info, "", right_table_alias, right_append_pk_column); + root_executor = mock::compileJoin(executor_index, left_ts, right_ts, joined_table->table_join); } /// Filter. @@ -942,7 +941,7 @@ std::pair compileQueryBlock( if (has_gby || has_agg_func) { if (!properties.is_mpp_query - && (dynamic_cast(root_executor.get()) != nullptr || dynamic_cast(root_executor.get()) != nullptr)) + && (dynamic_cast(root_executor.get()) != nullptr || dynamic_cast(root_executor.get()) != nullptr)) throw Exception("Limit/TopN and Agg cannot co-exist in non-mpp mode.", ErrorCodes::LOGICAL_ERROR); root_executor = compileAggregation( @@ -951,7 +950,7 @@ std::pair compileQueryBlock( ast_query.select_expression_list, has_gby ? ast_query.group_expression_list : nullptr); - if (dynamic_cast(root_executor.get())->has_uniq_raw_res) + if (dynamic_cast(root_executor.get())->hasUniqRawRes()) { // todo support uniq_raw in mpp mode if (properties.is_mpp_query) @@ -960,8 +959,8 @@ std::pair compileQueryBlock( has_uniq_raw_res = true; } - auto * agg = dynamic_cast(root_executor.get()); - if (agg->need_append_project || ast_query.select_expression_list->children.size() != agg->agg_exprs.size() + agg->gby_exprs.size()) + auto * agg = dynamic_cast(root_executor.get()); + if (agg->needAppendProject() || ast_query.select_expression_list->children.size() != agg->exprSize()) { /// Project if needed root_executor = compileProject(root_executor, executor_index, ast_query.select_expression_list); diff --git a/dbms/src/Debug/dbgFuncCoprocessor.h b/dbms/src/Debug/dbgFuncCoprocessor.h index efbf70f743f..e87f7f1a032 100644 --- a/dbms/src/Debug/dbgFuncCoprocessor.h +++ b/dbms/src/Debug/dbgFuncCoprocessor.h @@ -16,7 +16,19 @@ #include #include -#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include #include #include #include @@ -80,7 +92,7 @@ std::tuple compileQuery( QueryTasks queryPlanToQueryTasks( const DAGProperties & properties, - ExecutorPtr root_executor, + mock::ExecutorBinderPtr root_executor, size_t & executor_index, const Context & context); diff --git a/dbms/src/Server/FlashGrpcServerHolder.cpp b/dbms/src/Server/FlashGrpcServerHolder.cpp index f36a23353f9..cdd42a9288d 100644 --- a/dbms/src/Server/FlashGrpcServerHolder.cpp +++ b/dbms/src/Server/FlashGrpcServerHolder.cpp @@ -14,6 +14,7 @@ #include #include + namespace DB { namespace ErrorCodes diff --git a/dbms/src/Server/FlashGrpcServerHolder.h b/dbms/src/Server/FlashGrpcServerHolder.h index ae1edddca40..f8f425efff2 100644 --- a/dbms/src/Server/FlashGrpcServerHolder.h +++ b/dbms/src/Server/FlashGrpcServerHolder.h @@ -16,11 +16,13 @@ #include #include #include -#include +#include #include #include #include #include +#include + namespace DB { diff --git a/dbms/src/TestUtils/ColumnsToTiPBExpr.cpp b/dbms/src/TestUtils/ColumnsToTiPBExpr.cpp index 73168ba2cec..af8c8bed4ba 100644 --- a/dbms/src/TestUtils/ColumnsToTiPBExpr.cpp +++ b/dbms/src/TestUtils/ColumnsToTiPBExpr.cpp @@ -13,11 +13,13 @@ // limitations under the License. #include -#include +#include +#include #include #include #include + namespace DB { namespace tests diff --git a/dbms/src/TestUtils/mockExecutor.cpp b/dbms/src/TestUtils/mockExecutor.cpp index 0e4232ac7e7..19e12060447 100644 --- a/dbms/src/TestUtils/mockExecutor.cpp +++ b/dbms/src/TestUtils/mockExecutor.cpp @@ -13,7 +13,6 @@ // limitations under the License. #include -#include #include #include #include @@ -22,6 +21,7 @@ #include #include #include +#include #include #include #include @@ -30,6 +30,8 @@ namespace DB::tests { +using TableInfo = TiDB::TableInfo; + ASTPtr buildColumn(const String & column_name) { return std::make_shared(column_name); @@ -115,7 +117,7 @@ std::shared_ptr DAGRequestBuilder::build(MockDAGRequestContext // Currently Sort and Window Executors don't support columnPrune. // TODO: support columnPrume for Sort and Window. -void columnPrune(ExecutorPtr executor) +void columnPrune(mock::ExecutorBinderPtr executor) { std::unordered_set used_columns; for (auto & schema : executor->output_schema) @@ -167,7 +169,7 @@ DAGRequestBuilder & DAGRequestBuilder::mockTable(const String & db, const String ret.id = i++; table_info.columns.push_back(std::move(ret)); } - root = compileTableScan(getExecutorIndex(), table_info, db, table, false); + root = mock::compileTableScan(getExecutorIndex(), table_info, db, table, false); return *this; } @@ -192,42 +194,42 @@ DAGRequestBuilder & DAGRequestBuilder::buildExchangeReceiver(const MockColumnInf schema.push_back({column.first, info}); } - root = compileExchangeReceiver(getExecutorIndex(), schema, fine_grained_shuffle_stream_count); + root = mock::compileExchangeReceiver(getExecutorIndex(), schema, fine_grained_shuffle_stream_count); return *this; } DAGRequestBuilder & DAGRequestBuilder::filter(ASTPtr filter_expr) { assert(root); - root = compileSelection(root, getExecutorIndex(), filter_expr); + root = mock::compileSelection(root, getExecutorIndex(), filter_expr); return *this; } DAGRequestBuilder & DAGRequestBuilder::limit(int limit) { assert(root); - root = compileLimit(root, getExecutorIndex(), buildLiteral(Field(static_cast(limit)))); + root = mock::compileLimit(root, getExecutorIndex(), buildLiteral(Field(static_cast(limit)))); return *this; } DAGRequestBuilder & DAGRequestBuilder::limit(ASTPtr limit_expr) { assert(root); - root = compileLimit(root, getExecutorIndex(), limit_expr); + root = mock::compileLimit(root, getExecutorIndex(), limit_expr); return *this; } DAGRequestBuilder & DAGRequestBuilder::topN(ASTPtr order_exprs, ASTPtr limit_expr) { assert(root); - root = compileTopN(root, getExecutorIndex(), order_exprs, limit_expr); + root = mock::compileTopN(root, getExecutorIndex(), order_exprs, limit_expr); return *this; } DAGRequestBuilder & DAGRequestBuilder::topN(const String & col_name, bool desc, int limit) { assert(root); - root = compileTopN(root, getExecutorIndex(), buildOrderByItemVec({{col_name, desc}}), buildLiteral(Field(static_cast(limit)))); + root = mock::compileTopN(root, getExecutorIndex(), buildOrderByItemVec({{col_name, desc}}), buildLiteral(Field(static_cast(limit)))); return *this; } @@ -239,7 +241,7 @@ DAGRequestBuilder & DAGRequestBuilder::topN(MockOrderByItemVec order_by_items, i DAGRequestBuilder & DAGRequestBuilder::topN(MockOrderByItemVec order_by_items, ASTPtr limit_expr) { assert(root); - root = compileTopN(root, getExecutorIndex(), buildOrderByItemVec(order_by_items), limit_expr); + root = mock::compileTopN(root, getExecutorIndex(), buildOrderByItemVec(order_by_items), limit_expr); return *this; } @@ -251,7 +253,7 @@ DAGRequestBuilder & DAGRequestBuilder::project(MockAstVec exprs) { exp_list->children.push_back(expr); } - root = compileProject(root, getExecutorIndex(), exp_list); + root = mock::compileProject(root, getExecutorIndex(), exp_list); return *this; } @@ -263,14 +265,14 @@ DAGRequestBuilder & DAGRequestBuilder::project(MockColumnNameVec col_names) { exp_list->children.push_back(col(name)); } - root = compileProject(root, getExecutorIndex(), exp_list); + root = mock::compileProject(root, getExecutorIndex(), exp_list); return *this; } DAGRequestBuilder & DAGRequestBuilder::exchangeSender(tipb::ExchangeType exchange_type) { assert(root); - root = compileExchangeSender(root, getExecutorIndex(), exchange_type); + root = mock::compileExchangeSender(root, getExecutorIndex(), exchange_type); return *this; } @@ -285,7 +287,7 @@ DAGRequestBuilder & DAGRequestBuilder::join(const DAGRequestBuilder & right, assert(root); assert(right.root); - root = compileJoin(getExecutorIndex(), root, right.root, tp, join_cols, left_conds, right_conds, other_conds, other_eq_conds_from_in); + root = mock::compileJoin(getExecutorIndex(), root, right.root, tp, join_cols, left_conds, right_conds, other_conds, other_eq_conds_from_in); return *this; } diff --git a/dbms/src/TestUtils/mockExecutor.h b/dbms/src/TestUtils/mockExecutor.h index 55ad992ca06..1160d35f411 100644 --- a/dbms/src/TestUtils/mockExecutor.h +++ b/dbms/src/TestUtils/mockExecutor.h @@ -15,7 +15,7 @@ #pragma once #include -#include +#include #include #include #include @@ -71,7 +71,7 @@ class DAGRequestBuilder properties.collator = -abs(collator); } - ExecutorPtr getRoot() + mock::ExecutorBinderPtr getRoot() { return root; } @@ -147,7 +147,7 @@ class DAGRequestBuilder DAGRequestBuilder & buildAggregation(ASTPtr agg_funcs, ASTPtr group_by_exprs); DAGRequestBuilder & buildExchangeReceiver(const MockColumnInfoVec & columns, uint64_t fine_grained_shuffle_stream_count = 0); - ExecutorPtr root; + mock::ExecutorBinderPtr root; DAGProperties properties; };