Merge branch 'master' of github.com:ClickHouse/ClickHouse into tighten-limits-functional-tests

This commit is contained in:
Alexey Milovidov 2024-07-29 21:17:37 +02:00
commit 9e1663df1e
290 changed files with 5106 additions and 1748 deletions

View File

@ -93,21 +93,21 @@ jobs:
with:
stage: Builds_2
data: ${{ needs.RunConfig.outputs.data }}
Tests_2:
Tests_2_ww:
needs: [RunConfig, Builds_2]
if: ${{ !failure() && !cancelled() && contains(fromJson(needs.RunConfig.outputs.data).stages_data.stages_to_do, 'Tests_2_ww') }}
uses: ./.github/workflows/reusable_test_stage.yml
with:
stage: Tests_2_ww
data: ${{ needs.RunConfig.outputs.data }}
Tests_2:
# Test_3 should not wait for Test_1/Test_2 and should not be blocked by them on master branch since all jobs need to run there.
needs: [RunConfig, Builds_1]
if: ${{ !failure() && !cancelled() && contains(fromJson(needs.RunConfig.outputs.data).stages_data.stages_to_do, 'Tests_2') }}
uses: ./.github/workflows/reusable_test_stage.yml
with:
stage: Tests_2
data: ${{ needs.RunConfig.outputs.data }}
Tests_3:
# Test_3 should not wait for Test_1/Test_2 and should not be blocked by them on master branch since all jobs need to run there.
needs: [RunConfig, Builds_1]
if: ${{ !failure() && !cancelled() && contains(fromJson(needs.RunConfig.outputs.data).stages_data.stages_to_do, 'Tests_3') }}
uses: ./.github/workflows/reusable_test_stage.yml
with:
stage: Tests_3
data: ${{ needs.RunConfig.outputs.data }}
################################# Reports #################################
# Reports should run even if Builds_1/2 fail - run them separately, not in Tests_1/2/3
@ -123,7 +123,7 @@ jobs:
FinishCheck:
if: ${{ !cancelled() }}
needs: [RunConfig, Builds_1, Builds_2, Builds_Report, Tests_1, Tests_2, Tests_3]
needs: [RunConfig, Builds_1, Builds_2, Builds_Report, Tests_1, Tests_2_ww, Tests_2]
runs-on: [self-hosted, style-checker-aarch64]
steps:
- name: Check out repository code
@ -133,6 +133,7 @@ jobs:
cd "$GITHUB_WORKSPACE/tests/ci"
python3 finish_check.py --wf-status ${{ contains(needs.*.result, 'failure') && 'failure' || 'success' }}
- name: Check Workflow results
if: ${{ !cancelled() }}
run: |
export WORKFLOW_RESULT_FILE="/tmp/workflow_results.json"
cat > "$WORKFLOW_RESULT_FILE" << 'EOF'

View File

@ -123,20 +123,20 @@ jobs:
stage: Builds_2
data: ${{ needs.RunConfig.outputs.data }}
# stage for running non-required checks without being blocked by required checks (Test_1) if corresponding settings is selected
Tests_2:
Tests_2_ww:
needs: [RunConfig, Builds_1]
if: ${{ !failure() && !cancelled() && contains(fromJson(needs.RunConfig.outputs.data).stages_data.stages_to_do, 'Tests_2_ww') }}
uses: ./.github/workflows/reusable_test_stage.yml
with:
stage: Tests_2_ww
data: ${{ needs.RunConfig.outputs.data }}
Tests_2:
needs: [RunConfig, Builds_1, Tests_1]
if: ${{ !failure() && !cancelled() && contains(fromJson(needs.RunConfig.outputs.data).stages_data.stages_to_do, 'Tests_2') }}
uses: ./.github/workflows/reusable_test_stage.yml
with:
stage: Tests_2
data: ${{ needs.RunConfig.outputs.data }}
Tests_3:
needs: [RunConfig, Builds_1, Tests_1]
if: ${{ !failure() && !cancelled() && contains(fromJson(needs.RunConfig.outputs.data).stages_data.stages_to_do, 'Tests_3') }}
uses: ./.github/workflows/reusable_test_stage.yml
with:
stage: Tests_3
data: ${{ needs.RunConfig.outputs.data }}
################################# Reports #################################
# Reports should run even if Builds_1/2 fail - run them separately (not in Tests_1/2/3)
@ -154,7 +154,7 @@ jobs:
if: ${{ !cancelled() }}
# Test_2 or Test_3 do not have the jobs required for Mergeable check,
# however, set them as "needs" to get all checks results before the automatic merge occurs.
needs: [RunConfig, BuildDockers, StyleCheck, FastTest, Builds_1, Builds_2, Builds_Report, Tests_1, Tests_2, Tests_3]
needs: [RunConfig, BuildDockers, StyleCheck, FastTest, Builds_1, Builds_2, Builds_Report, Tests_1, Tests_2_ww, Tests_2]
runs-on: [self-hosted, style-checker-aarch64]
steps:
- name: Check out repository code
@ -178,7 +178,7 @@ jobs:
#
FinishCheck:
if: ${{ !failure() && !cancelled() }}
needs: [RunConfig, BuildDockers, StyleCheck, FastTest, Builds_1, Builds_2, Builds_Report, Tests_1, Tests_2, Tests_3]
needs: [RunConfig, BuildDockers, StyleCheck, FastTest, Builds_1, Builds_2, Builds_Report, Tests_1, Tests_2_ww, Tests_2]
runs-on: [self-hosted, style-checker-aarch64]
steps:
- name: Check out repository code

2
contrib/icu vendored

@ -1 +1 @@
Subproject commit a56dde820dc35665a66f2e9ee8ba58e75049b668
Subproject commit 7750081bda4b3bc1768ae03849ec70f67ea10625

View File

@ -4,7 +4,9 @@ else ()
option(ENABLE_ICU "Enable ICU" 0)
endif ()
if (NOT ENABLE_ICU)
# Temporarily disabled s390x because the ICU build links a blob (icudt71b_dat.S) and our friends from IBM did not explain how they generated
# the blob on s390x: https://github.com/ClickHouse/icudata/pull/2#issuecomment-2226957255
if (NOT ENABLE_ICU OR ARCH_S390X)
message(STATUS "Not using ICU")
return()
endif()
@ -12,8 +14,6 @@ endif()
set(ICU_SOURCE_DIR "${ClickHouse_SOURCE_DIR}/contrib/icu/icu4c/source")
set(ICUDATA_SOURCE_DIR "${ClickHouse_SOURCE_DIR}/contrib/icudata/")
set (CMAKE_CXX_STANDARD 17)
# These lists of sources were generated from build log of the original ICU build system (configure + make).
set(ICUUC_SOURCES
@ -462,9 +462,9 @@ file(GENERATE OUTPUT "${CMAKE_CURRENT_BINARY_DIR}/empty.cpp" CONTENT " ")
enable_language(ASM)
if (ARCH_S390X)
set(ICUDATA_SOURCE_FILE "${ICUDATA_SOURCE_DIR}/icudt70b_dat.S" )
set(ICUDATA_SOURCE_FILE "${ICUDATA_SOURCE_DIR}/icudt75b_dat.S" )
else()
set(ICUDATA_SOURCE_FILE "${ICUDATA_SOURCE_DIR}/icudt70l_dat.S" )
set(ICUDATA_SOURCE_FILE "${ICUDATA_SOURCE_DIR}/icudt75l_dat.S" )
endif()
set(ICUDATA_SOURCES

2
contrib/icudata vendored

@ -1 +1 @@
Subproject commit c8e717892a557b4d2852317c7d628aacc0a0e5ab
Subproject commit d345d6ac22f381c882420de9053d30ae1ff38d75

View File

@ -54,7 +54,7 @@ CREATE TABLE keeper_map_table
`v2` String,
`v3` Float32
)
ENGINE = KeeperMap(/keeper_map_table, 4)
ENGINE = KeeperMap('/keeper_map_table', 4)
PRIMARY KEY key
```

File diff suppressed because it is too large Load Diff

View File

@ -1135,8 +1135,6 @@ void Client::processOptions(const OptionsDescription & options_description,
if ((query_fuzzer_runs = options["query-fuzzer-runs"].as<int>()))
{
// Fuzzer implies multiquery.
config().setBool("multiquery", true);
// Ignore errors in parsing queries.
config().setBool("ignore-error", true);
ignore_error = true;
@ -1144,8 +1142,6 @@ void Client::processOptions(const OptionsDescription & options_description,
if ((create_query_fuzzer_runs = options["create-query-fuzzer-runs"].as<int>()))
{
// Fuzzer implies multiquery.
config().setBool("multiquery", true);
// Ignore errors in parsing queries.
config().setBool("ignore-error", true);
@ -1201,9 +1197,6 @@ void Client::processConfig()
}
print_stack_trace = config().getBool("stacktrace", false);
if (config().has("multiquery"))
is_multiquery = true;
pager = config().getString("pager", "");
setDefaultFormatsAndCompressionFromConfiguration();
@ -1359,13 +1352,6 @@ void Client::readArguments(
allow_repeated_settings = true;
else if (arg == "--allow_merge_tree_settings")
allow_merge_tree_settings = true;
else if (arg == "--multiquery" && (arg_num + 1) < argc && !std::string_view(argv[arg_num + 1]).starts_with('-'))
{
/// Transform the abbreviated syntax '--multiquery <SQL>' into the full syntax '--multiquery -q <SQL>'
++arg_num;
arg = argv[arg_num];
addMultiquery(arg, common_arguments);
}
else if (arg == "--password" && ((arg_num + 1) >= argc || std::string_view(argv[arg_num + 1]).starts_with('-')))
{
common_arguments.emplace_back(arg);

View File

@ -569,9 +569,6 @@ void LocalServer::processConfig()
if (!queries.empty() && getClientConfiguration().has("queries-file"))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Options '--query' and '--queries-file' cannot be specified at the same time");
if (getClientConfiguration().has("multiquery"))
is_multiquery = true;
pager = getClientConfiguration().getString("pager", "");
delayed_interactive = getClientConfiguration().has("interactive") && (!queries.empty() || getClientConfiguration().has("queries-file"));
@ -936,13 +933,6 @@ void LocalServer::readArguments(int argc, char ** argv, Arguments & common_argum
query_parameters.emplace(param_continuation.substr(0, equal_pos), param_continuation.substr(equal_pos + 1));
}
}
else if (arg == "--multiquery" && (arg_num + 1) < argc && !std::string_view(argv[arg_num + 1]).starts_with('-'))
{
/// Transform the abbreviated syntax '--multiquery <SQL>' into the full syntax '--multiquery -q <SQL>'
++arg_num;
arg = argv[arg_num];
addMultiquery(arg, common_arguments);
}
else
{
common_arguments.emplace_back(arg);

View File

@ -70,7 +70,6 @@
#include <boost/algorithm/string/case_conv.hpp>
#include <boost/algorithm/string/replace.hpp>
#include <boost/algorithm/string/split.hpp>
#include <iostream>
#include <filesystem>
#include <limits>
@ -80,6 +79,8 @@
#include <Common/config_version.h>
#include "config.h"
#include <IO/ReadHelpers.h>
#include <Processors/Formats/Impl/ValuesBlockInputFormat.h>
#if USE_GWP_ASAN
# include <Common/GWPAsan.h>
@ -745,14 +746,6 @@ void ClientBase::adjustSettings()
/// NOTE: Do not forget to set changed=false to avoid sending it to the server (to avoid breakage read only profiles)
/// In case of multi-query we allow data after semicolon since it will be
/// parsed by the client and interpreted as new query
if (is_multiquery && !global_context->getSettingsRef().input_format_values_allow_data_after_semicolon.changed)
{
settings.input_format_values_allow_data_after_semicolon = true;
settings.input_format_values_allow_data_after_semicolon.changed = false;
}
/// Do not limit pretty format output in case of --pager specified or in case of stdout is not a tty.
if (!pager.empty() || !stdout_is_a_tty)
{
@ -1525,13 +1518,6 @@ void ClientBase::setInsertionTable(const ASTInsertQuery & insert_query)
}
void ClientBase::addMultiquery(std::string_view query, Arguments & common_arguments) const
{
common_arguments.emplace_back("--multiquery");
common_arguments.emplace_back("-q");
common_arguments.emplace_back(query);
}
namespace
{
bool isStdinNotEmptyAndValid(ReadBufferFromFileDescriptor & std_in)
@ -2186,23 +2172,48 @@ MultiQueryProcessingStage ClientBase::analyzeMultiQueryText(
return MultiQueryProcessingStage::PARSING_FAILED;
}
// INSERT queries may have the inserted data in the query text
// that follow the query itself, e.g. "insert into t format CSV 1;2".
// They need special handling. First of all, here we find where the
// inserted data ends. In multi-query mode, it is delimited by a
// newline.
// The VALUES format needs even more handling - we also allow the
// data to be delimited by semicolon. This case is handled later by
// the format parser itself.
// We can't do multiline INSERTs with inline data, because most
// row input formats (e.g. TSV) can't tell when the input stops,
// unlike VALUES.
// INSERT queries may have the inserted data in the query text that follow the query itself, e.g. "insert into t format CSV 1,2". They
// need special handling.
// - If the INSERT statement FORMAT is VALUES, we use the VALUES format parser to skip the inserted data until we reach the trailing single semicolon.
// - Other formats (e.g. FORMAT CSV) are arbitrarily more complex and tricky to parse. For example, we may be unable to distinguish if the semicolon
// is part of the data or ends the statement. In this case, we simply assume that the end of the INSERT statement is determined by \n\n (two newlines).
auto * insert_ast = parsed_query->as<ASTInsertQuery>();
const char * query_to_execute_end = this_query_end;
if (insert_ast && insert_ast->data)
{
this_query_end = find_first_symbols<'\n'>(insert_ast->data, all_queries_end);
if (insert_ast->format == "Values")
{
// Invoke the VALUES format parser to skip the inserted data
ReadBufferFromMemory data_in(insert_ast->data, all_queries_end - insert_ast->data);
skipBOMIfExists(data_in);
do
{
skipWhitespaceIfAny(data_in);
if (data_in.eof() || *data_in.position() == ';')
break;
}
while (ValuesBlockInputFormat::skipToNextRow(&data_in, 1, 0));
// Handle the case of a comment followed by a semicolon
// Example: INSERT INTO tab VALUES xx; -- {serverError xx}
// If we use this error hint, the next query should not be placed on the same line
this_query_end = insert_ast->data + data_in.count();
const auto * pos_newline = find_first_symbols<'\n'>(this_query_end, all_queries_end);
if (pos_newline != this_query_end)
{
TestHint hint(String(this_query_end, pos_newline - this_query_end));
if (hint.hasClientErrors() || hint.hasServerErrors())
this_query_end = pos_newline;
}
}
else
{
// Handling of generic formats
auto pos_newline = String(insert_ast->data, all_queries_end).find("\n\n");
if (pos_newline != std::string::npos)
this_query_end = insert_ast->data + pos_newline;
else
this_query_end = all_queries_end;
}
insert_ast->end = this_query_end;
query_to_execute_end = isSyncInsertWithData(*insert_ast, client_context) ? insert_ast->data : this_query_end;
}
@ -2237,7 +2248,10 @@ bool ClientBase::executeMultiQuery(const String & all_queries_text)
size_t test_tags_length = getTestTagsLength(all_queries_text);
/// Several queries separated by ';'.
/// INSERT data is ended by the end of line, not ';'.
/// INSERT data is ended by the empty line (\n\n), not ';'.
/// Unnecessary semicolons may cause data to be parsed containing ';'
/// e.g. 'insert into xx format csv val;' will insert "val;" instead of "val"
/// 'insert into xx format csv val\n;' will insert "val" and ";"
/// An exception is VALUES format where we also support semicolon in
/// addition to end of line.
const char * this_query_begin = all_queries_text.data() + test_tags_length;
@ -2248,8 +2262,8 @@ bool ClientBase::executeMultiQuery(const String & all_queries_text)
String query_to_execute;
ASTPtr parsed_query;
std::unique_ptr<Exception> current_exception;
size_t retries_count = 0;
bool is_first = true;
while (true)
{
@ -2258,16 +2272,24 @@ bool ClientBase::executeMultiQuery(const String & all_queries_text)
switch (stage)
{
case MultiQueryProcessingStage::QUERIES_END:
{
/// Compatible with old version when run interactive, e.g. "", "\ld"
if (is_first && is_interactive)
processTextAsSingleQuery(all_queries_text);
return true;
}
case MultiQueryProcessingStage::PARSING_FAILED:
{
return true;
}
case MultiQueryProcessingStage::CONTINUE_PARSING:
{
is_first = false;
continue;
}
case MultiQueryProcessingStage::PARSING_EXCEPTION:
{
is_first = false;
this_query_end = find_first_symbols<'\n'>(this_query_end, all_queries_end);
// Try to find test hint for syntax error. We don't know where
@ -2297,6 +2319,7 @@ bool ClientBase::executeMultiQuery(const String & all_queries_text)
}
case MultiQueryProcessingStage::EXECUTE_QUERY:
{
is_first = false;
full_query = all_queries_text.substr(this_query_begin - all_queries_text.data(), this_query_end - this_query_begin);
if (query_fuzzer_runs)
{
@ -2306,6 +2329,8 @@ bool ClientBase::executeMultiQuery(const String & all_queries_text)
this_query_begin = this_query_end;
continue;
}
if (suggest)
updateSuggest(parsed_query);
// Now we know for sure where the query ends.
// Look for the hint in the text of query + insert data + trailing
@ -2456,14 +2481,6 @@ bool ClientBase::processQueryText(const String & text)
return processMultiQueryFromFile(file_name);
}
if (!is_multiquery)
{
assert(!query_fuzzer_runs);
processTextAsSingleQuery(text);
return true;
}
if (query_fuzzer_runs)
{
processWithFuzzing(text);
@ -2901,9 +2918,9 @@ void ClientBase::init(int argc, char ** argv)
("config-file,C", po::value<std::string>(), "config-file path")
("query,q", po::value<std::vector<std::string>>()->multitoken(), R"(query; can be specified multiple times (--query "SELECT 1" --query "SELECT 2"...))")
("query,q", po::value<std::vector<std::string>>()->multitoken(), R"(Query. Can be specified multiple times (--query "SELECT 1" --query "SELECT 2") or once with multiple comma-separated queries (--query "SELECT 1; SELECT 2;"). In the latter case, INSERT queries with non-VALUE format must be separated by empty lines.)")
("queries-file", po::value<std::vector<std::string>>()->multitoken(), "file path with queries to execute; multiple files can be specified (--queries-file file1 file2...)")
("multiquery,n", "If specified, multiple queries separated by semicolons can be listed after --query. For convenience, it is also possible to omit --query and pass the queries directly after --multiquery.")
("multiquery,n", "Obsolete, does nothing")
("multiline,m", "If specified, allow multiline queries (do not send the query on Enter)")
("database,d", po::value<std::string>(), "database")
("query_kind", po::value<std::string>()->default_value("initial_query"), "One of initial_query/secondary_query/no_query")
@ -2932,7 +2949,7 @@ void ClientBase::init(int argc, char ** argv)
("vertical,E", "vertical output format, same as --format=Vertical or FORMAT Vertical or \\G at end of command")
("highlight", po::value<bool>()->default_value(true), "enable or disable basic syntax highlight in interactive command line")
("ignore-error", "do not stop processing in multiquery mode")
("ignore-error", "do not stop processing when an error occurs")
("stacktrace", "print stack traces of exceptions")
("hardware-utilization", "print hardware utilization information in progress bar")
("print-profile-events", po::value(&profile_events.print)->zero_tokens(), "Printing ProfileEvents packets")
@ -3025,8 +3042,6 @@ void ClientBase::init(int argc, char ** argv)
queries_files = options["queries-file"].as<std::vector<std::string>>();
if (options.count("multiline"))
getClientConfiguration().setBool("multiline", true);
if (options.count("multiquery"))
getClientConfiguration().setBool("multiquery", true);
if (options.count("ignore-error"))
getClientConfiguration().setBool("ignore-error", true);
if (options.count("format"))

View File

@ -156,8 +156,6 @@ protected:
void setInsertionTable(const ASTInsertQuery & insert_query);
void addMultiquery(std::string_view query, Arguments & common_arguments) const;
private:
void receiveResult(ASTPtr parsed_query, Int32 signals_before_stop, bool partial_result_on_first_cancel);
bool receiveAndProcessPacket(ASTPtr parsed_query, bool cancelled_);
@ -229,7 +227,6 @@ protected:
std::unique_ptr<Poco::Runnable> signal_listener;
bool is_interactive = false; /// Use either interactive line editing interface or batch mode.
bool is_multiquery = false;
bool delayed_interactive = false;
bool echo_queries = false; /// Print queries before execution in batch mode.

View File

@ -13,10 +13,10 @@
namespace DB
{
InterpolateDescription::InterpolateDescription(ActionsDAGPtr actions_, const Aliases & aliases)
: actions(actions_)
InterpolateDescription::InterpolateDescription(ActionsDAG actions_, const Aliases & aliases)
: actions(std::move(actions_))
{
for (const auto & name_type : actions->getRequiredColumns())
for (const auto & name_type : actions.getRequiredColumns())
{
if (const auto & p = aliases.find(name_type.name); p != aliases.end())
required_columns_map[p->second->getColumnName()] = name_type;
@ -24,7 +24,7 @@ namespace DB
required_columns_map[name_type.name] = name_type;
}
for (const ColumnWithTypeAndName & column : actions->getResultColumns())
for (const ColumnWithTypeAndName & column : actions.getResultColumns())
{
std::string name = column.name;
if (const auto & p = aliases.find(name); p != aliases.end())

View File

@ -5,21 +5,20 @@
#include <string>
#include <Core/NamesAndTypes.h>
#include <Parsers/IAST_fwd.h>
#include <Interpreters/ActionsDAG.h>
namespace DB
{
class ActionsDAG;
using ActionsDAGPtr = std::shared_ptr<ActionsDAG>;
using Aliases = std::unordered_map<String, ASTPtr>;
/// Interpolate description
struct InterpolateDescription
{
explicit InterpolateDescription(ActionsDAGPtr actions, const Aliases & aliases);
explicit InterpolateDescription(ActionsDAG actions, const Aliases & aliases);
ActionsDAGPtr actions;
ActionsDAG actions;
std::unordered_map<std::string, NameAndTypePair> required_columns_map; /// input column name -> {alias, type}
std::unordered_set<std::string> result_columns_set; /// result block columns

View File

@ -186,7 +186,7 @@ class IColumn;
M(Bool, allow_suspicious_ttl_expressions, false, "Reject TTL expressions that don't depend on any of table's columns. It indicates a user error most of the time.", 0) \
M(Bool, allow_suspicious_variant_types, false, "In CREATE TABLE statement allows specifying Variant type with similar variant types (for example, with different numeric or date types). Enabling this setting may introduce some ambiguity when working with values with similar types.", 0) \
M(Bool, allow_suspicious_primary_key, false, "Forbid suspicious PRIMARY KEY/ORDER BY for MergeTree (i.e. SimpleAggregateFunction)", 0) \
M(Bool, compile_expressions, false, "Compile some scalar functions and operators to native code.", 0) \
M(Bool, compile_expressions, false, "Compile some scalar functions and operators to native code. Due to a bug in the LLVM compiler infrastructure, on AArch64 machines, it is known to lead to a nullptr dereference and, consequently, server crash. Do not enable this setting.", 0) \
M(UInt64, min_count_to_compile_expression, 3, "The number of identical expressions before they are JIT-compiled", 0) \
M(Bool, compile_aggregate_expressions, true, "Compile aggregate functions to native code.", 0) \
M(UInt64, min_count_to_compile_aggregate_expression, 3, "The number of identical aggregate expressions before they are JIT-compiled", 0) \
@ -1156,7 +1156,6 @@ class IColumn;
M(Bool, input_format_values_interpret_expressions, true, "For Values format: if the field could not be parsed by streaming parser, run SQL parser and try to interpret it as SQL expression.", 0) \
M(Bool, input_format_values_deduce_templates_of_expressions, true, "For Values format: if the field could not be parsed by streaming parser, run SQL parser, deduce template of the SQL expression, try to parse all rows using template and then interpret expression for all rows.", 0) \
M(Bool, input_format_values_accurate_types_of_literals, true, "For Values format: when parsing and interpreting expressions using template, check actual type of literal to avoid possible overflow and precision issues.", 0) \
M(Bool, input_format_values_allow_data_after_semicolon, false, "For Values format: allow extra data after semicolon (used by client to interpret comments).", 0) \
M(Bool, input_format_avro_allow_missing_fields, false, "For Avro/AvroConfluent format: when field is not found in schema use default value instead of error", 0) \
/** This setting is obsolete and do nothing, left for compatibility reasons. */ \
M(Bool, input_format_avro_null_as_default, false, "For Avro/AvroConfluent format: insert default in case of null and non Nullable column", 0) \

View File

@ -257,8 +257,8 @@ static DataTypePtr create(const ASTPtr & arguments)
}
else
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Unexpected AST element passed as aggregate function name for data type AggregateFunction. "
"Must be identifier or function.");
"Unexpected AST element {} passed as aggregate function name for data type AggregateFunction. "
"Must be identifier or function", data_type_ast->getID());
for (size_t i = argument_types_start_idx; i < arguments->children.size(); ++i)
argument_types.push_back(DataTypeFactory::instance().get(arguments->children[i]));

View File

@ -2,7 +2,7 @@
#include <DataTypes/DataTypeCustom.h>
#include <Parsers/parseQuery.h>
#include <Parsers/ParserCreateQuery.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTDataType.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTLiteral.h>
#include <Common/typeid_cast.h>
@ -22,7 +22,6 @@ namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int UNKNOWN_TYPE;
extern const int ILLEGAL_SYNTAX_FOR_DATA_TYPE;
extern const int UNEXPECTED_AST_STRUCTURE;
extern const int DATA_TYPE_CANNOT_HAVE_ARGUMENTS;
}
@ -83,15 +82,9 @@ DataTypePtr DataTypeFactory::tryGet(const ASTPtr & ast) const
template <bool nullptr_on_error>
DataTypePtr DataTypeFactory::getImpl(const ASTPtr & ast) const
{
if (const auto * func = ast->as<ASTFunction>())
if (const auto * type = ast->as<ASTDataType>())
{
if (func->parameters)
{
if constexpr (nullptr_on_error)
return nullptr;
throw Exception(ErrorCodes::ILLEGAL_SYNTAX_FOR_DATA_TYPE, "Data type cannot have multiple parenthesized parameters.");
}
return getImpl<nullptr_on_error>(func->name, func->arguments);
return getImpl<nullptr_on_error>(type->name, type->arguments);
}
if (const auto * ident = ast->as<ASTIdentifier>())
@ -107,7 +100,7 @@ DataTypePtr DataTypeFactory::getImpl(const ASTPtr & ast) const
if constexpr (nullptr_on_error)
return nullptr;
throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE, "Unexpected AST element for data type.");
throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE, "Unexpected AST element for data type: {}.", ast->getID());
}
DataTypePtr DataTypeFactory::get(const String & family_name_param, const ASTPtr & parameters) const

View File

@ -4,9 +4,10 @@
#include <Parsers/IAST.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTDataType.h>
#include <IO/Operators.h>
namespace DB
{
@ -53,13 +54,13 @@ static DataTypePtr create(const ASTPtr & arguments)
ASTPtr schema_argument = arguments->children[0];
bool is_nullable = false;
if (const auto * func = schema_argument->as<ASTFunction>())
if (const auto * type = schema_argument->as<ASTDataType>())
{
if (func->name != "Nullable" || func->arguments->children.size() != 1)
if (type->name != "Nullable" || type->arguments->children.size() != 1)
throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE,
"Expected 'Nullable(<schema_name>)' as parameter for type Object (function: {})", func->name);
"Expected 'Nullable(<schema_name>)' as parameter for type Object (function: {})", type->name);
schema_argument = func->arguments->children[0];
schema_argument = type->arguments->children[0];
is_nullable = true;
}

View File

@ -149,7 +149,7 @@ ASTPtr getCreateQueryFromStorage(const StoragePtr & storage, const ASTPtr & ast_
columns = metadata_ptr->columns.getAll();
for (const auto & column_name_and_type: columns)
{
const auto & ast_column_declaration = std::make_shared<ASTColumnDeclaration>();
const auto ast_column_declaration = std::make_shared<ASTColumnDeclaration>();
ast_column_declaration->name = column_name_and_type.name;
/// parser typename
{
@ -164,7 +164,7 @@ ASTPtr getCreateQueryFromStorage(const StoragePtr & storage, const ASTPtr & ast_
if (!parser.parse(pos, ast_type, expected))
{
if (throw_on_error)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot parser metadata of {}.{}",
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot parse metadata of {}.{}",
backQuote(table_id.database_name), backQuote(table_id.table_name));
else
return nullptr;

View File

@ -12,9 +12,9 @@
#include <Interpreters/evaluateConstantExpression.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTDataType.h>
#include <Parsers/ParserCreateQuery.h>
#include <Parsers/parseQuery.h>
#include <Parsers/queryToString.h>
#include <Common/escapeForFileName.h>
#include <Common/parseRemoteDescription.h>
#include <Databases/DatabaseFactory.h>
@ -25,6 +25,7 @@
#include <Core/Settings.h>
#include <filesystem>
namespace fs = std::filesystem;
namespace DB
@ -432,7 +433,7 @@ ASTPtr DatabasePostgreSQL::getCreateTableQueryImpl(const String & table_name, Co
auto metadata_snapshot = storage->getInMemoryMetadataPtr();
for (const auto & column_type_and_name : metadata_snapshot->getColumns().getOrdinary())
{
const auto & column_declaration = std::make_shared<ASTColumnDeclaration>();
const auto column_declaration = std::make_shared<ASTColumnDeclaration>();
column_declaration->name = column_type_and_name.name;
column_declaration->type = getColumnDeclaration(column_type_and_name.type);
columns_expression_list->children.emplace_back(column_declaration);
@ -470,17 +471,15 @@ ASTPtr DatabasePostgreSQL::getColumnDeclaration(const DataTypePtr & data_type) c
WhichDataType which(data_type);
if (which.isNullable())
return makeASTFunction("Nullable", getColumnDeclaration(typeid_cast<const DataTypeNullable *>(data_type.get())->getNestedType()));
return makeASTDataType("Nullable", getColumnDeclaration(typeid_cast<const DataTypeNullable *>(data_type.get())->getNestedType()));
if (which.isArray())
return makeASTFunction("Array", getColumnDeclaration(typeid_cast<const DataTypeArray *>(data_type.get())->getNestedType()));
return makeASTDataType("Array", getColumnDeclaration(typeid_cast<const DataTypeArray *>(data_type.get())->getNestedType()));
if (which.isDateTime64())
{
return makeASTFunction("DateTime64", std::make_shared<ASTLiteral>(static_cast<UInt32>(6)));
}
return makeASTDataType("DateTime64", std::make_shared<ASTLiteral>(static_cast<UInt32>(6)));
return std::make_shared<ASTIdentifier>(data_type->getName());
return makeASTDataType(data_type->getName());
}
void registerDatabasePostgreSQL(DatabaseFactory & factory)

View File

@ -810,6 +810,7 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep()
{
last_caller_id = FileSegment::getCallerId();
chassert(file_offset_of_buffer_end <= read_until_position);
if (file_offset_of_buffer_end == read_until_position)
return false;
@ -1051,7 +1052,11 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep()
if (download_current_segment && download_current_segment_succeeded)
chassert(file_segment.getCurrentWriteOffset() >= file_offset_of_buffer_end);
chassert(file_offset_of_buffer_end <= read_until_position);
chassert(
file_offset_of_buffer_end <= read_until_position,
fmt::format("Expected {} <= {} (size: {}, read range: {})",
file_offset_of_buffer_end, read_until_position, size, current_read_range.toString()));
}
swap(*implementation_buffer);

View File

@ -215,7 +215,6 @@ FormatSettings getFormatSettings(const ContextPtr & context, const Settings & se
format_settings.tsv.allow_variable_number_of_columns = settings.input_format_tsv_allow_variable_number_of_columns;
format_settings.tsv.crlf_end_of_line_input = settings.input_format_tsv_crlf_end_of_line;
format_settings.values.accurate_types_of_literals = settings.input_format_values_accurate_types_of_literals;
format_settings.values.allow_data_after_semicolon = settings.input_format_values_allow_data_after_semicolon;
format_settings.values.deduce_templates_of_expressions = settings.input_format_values_deduce_templates_of_expressions;
format_settings.values.interpret_expressions = settings.input_format_values_interpret_expressions;
format_settings.values.escape_quote_with_quote = settings.output_format_values_escape_quote_with_quote;

View File

@ -2,14 +2,12 @@
#include <Functions/IFunction.h>
#include <Functions/FunctionFactory.h>
#include <DataTypes/DataTypesNumber.h>
#include <Interpreters/ActionsDAG.h>
namespace DB
{
class ActionsDAG;
using ActionsDAGPtr = std::shared_ptr<ActionsDAG>;
/** The `indexHint` function takes any number of any arguments and always returns one.
*
* This function has a special meaning (see ExpressionAnalyzer, KeyCondition)
@ -64,11 +62,11 @@ public:
return DataTypeUInt8().createColumnConst(input_rows_count, 1u);
}
void setActions(ActionsDAGPtr actions_) { actions = std::move(actions_); }
const ActionsDAGPtr & getActions() const { return actions; }
void setActions(ActionsDAG actions_) { actions = std::move(actions_); }
const ActionsDAG & getActions() const { return actions; }
private:
ActionsDAGPtr actions;
ActionsDAG actions;
};
}

View File

@ -93,6 +93,9 @@ struct ChiSquaredDistribution
static void generate(Float64 degree_of_freedom, ColumnFloat64::Container & container)
{
if (degree_of_freedom <= 0)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Argument (degrees of freedom) of function {} should be greater than zero", getName());
auto distribution = std::chi_squared_distribution<>(degree_of_freedom);
for (auto & elem : container)
elem = distribution(thread_local_rng);
@ -107,6 +110,9 @@ struct StudentTDistribution
static void generate(Float64 degree_of_freedom, ColumnFloat64::Container & container)
{
if (degree_of_freedom <= 0)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Argument (degrees of freedom) of function {} should be greater than zero", getName());
auto distribution = std::student_t_distribution<>(degree_of_freedom);
for (auto & elem : container)
elem = distribution(thread_local_rng);
@ -121,6 +127,9 @@ struct FisherFDistribution
static void generate(Float64 d1, Float64 d2, ColumnFloat64::Container & container)
{
if (d1 <= 0 || d2 <= 0)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Argument (degrees of freedom) of function {} should be greater than zero", getName());
auto distribution = std::fisher_f_distribution<>(d1, d2);
for (auto & elem : container)
elem = distribution(thread_local_rng);
@ -300,7 +309,7 @@ public:
}
else
{
throw Exception(ErrorCodes::BAD_ARGUMENTS, "More than two argument specified for function {}", getName());
throw Exception(ErrorCodes::BAD_ARGUMENTS, "More than two arguments specified for function {}", getName());
}
return res_column;

View File

@ -5,6 +5,7 @@
#include <IO/ReadBufferFromFileDecorator.h>
#include <IO/PeekableReadBuffer.h>
namespace DB
{
@ -88,5 +89,4 @@ size_t getDataOffsetMaybeCompressed(const ReadBuffer & in)
return in.count();
}
}

View File

@ -624,9 +624,9 @@ void ActionsDAG::removeAliasesForFilter(const std::string & filter_name)
}
}
ActionsDAGPtr ActionsDAG::cloneSubDAG(const NodeRawConstPtrs & outputs, bool remove_aliases)
ActionsDAG ActionsDAG::cloneSubDAG(const NodeRawConstPtrs & outputs, bool remove_aliases)
{
auto actions = std::make_shared<ActionsDAG>();
ActionsDAG actions;
std::unordered_map<const Node *, Node *> copy_map;
struct Frame
@ -661,21 +661,21 @@ ActionsDAGPtr ActionsDAG::cloneSubDAG(const NodeRawConstPtrs & outputs, bool rem
if (remove_aliases && frame.node->type == ActionType::ALIAS)
copy_node = copy_map[frame.node->children.front()];
else
copy_node = &actions->nodes.emplace_back(*frame.node);
copy_node = &actions.nodes.emplace_back(*frame.node);
if (frame.node->type == ActionType::INPUT)
actions->inputs.push_back(copy_node);
actions.inputs.push_back(copy_node);
stack.pop();
}
}
for (auto & node : actions->nodes)
for (auto & node : actions.nodes)
for (auto & child : node.children)
child = copy_map[child];
for (const auto * output : outputs)
actions->outputs.push_back(copy_map[output]);
actions.outputs.push_back(copy_map[output]);
return actions;
}
@ -961,9 +961,9 @@ NameSet ActionsDAG::foldActionsByProjection(
}
ActionsDAGPtr ActionsDAG::foldActionsByProjection(const std::unordered_map<const Node *, const Node *> & new_inputs, const NodeRawConstPtrs & required_outputs)
ActionsDAG ActionsDAG::foldActionsByProjection(const std::unordered_map<const Node *, const Node *> & new_inputs, const NodeRawConstPtrs & required_outputs)
{
auto dag = std::make_unique<ActionsDAG>();
ActionsDAG dag;
std::unordered_map<const Node *, const Node *> inputs_mapping;
std::unordered_map<const Node *, const Node *> mapping;
struct Frame
@ -1003,9 +1003,9 @@ ActionsDAGPtr ActionsDAG::foldActionsByProjection(const std::unordered_map<const
{
bool should_rename = new_input->result_name != rename->result_name;
const auto & input_name = should_rename ? rename->result_name : new_input->result_name;
mapped_input = &dag->addInput(input_name, new_input->result_type);
mapped_input = &dag.addInput(input_name, new_input->result_type);
if (should_rename)
mapped_input = &dag->addAlias(*mapped_input, new_input->result_name);
mapped_input = &dag.addAlias(*mapped_input, new_input->result_name);
}
node = mapped_input;
@ -1034,7 +1034,7 @@ ActionsDAGPtr ActionsDAG::foldActionsByProjection(const std::unordered_map<const
"Cannot fold actions for projection. Node {} requires input {} which does not belong to projection",
stack.front().node->result_name, frame.node->result_name);
auto & node = dag->nodes.emplace_back(*frame.node);
auto & node = dag.nodes.emplace_back(*frame.node);
for (auto & child : node.children)
child = mapping[child];
@ -1049,8 +1049,8 @@ ActionsDAGPtr ActionsDAG::foldActionsByProjection(const std::unordered_map<const
/// Add an alias if the mapped node has a different result name.
const auto * mapped_output = mapping[output];
if (output->result_name != mapped_output->result_name)
mapped_output = &dag->addAlias(*mapped_output, output->result_name);
dag->outputs.push_back(mapped_output);
mapped_output = &dag.addAlias(*mapped_output, output->result_name);
dag.outputs.push_back(mapped_output);
}
return dag;
@ -1246,27 +1246,31 @@ bool ActionsDAG::removeUnusedResult(const std::string & column_name)
return true;
}
ActionsDAGPtr ActionsDAG::clone() const
ActionsDAG ActionsDAG::clone() const
{
auto actions = std::make_shared<ActionsDAG>();
std::unordered_map<const Node *, Node *> old_to_new_nodes;
return clone(old_to_new_nodes);
}
std::unordered_map<const Node *, Node *> copy_map;
ActionsDAG ActionsDAG::clone(std::unordered_map<const Node *, Node *> & old_to_new_nodes) const
{
ActionsDAG actions;
for (const auto & node : nodes)
{
auto & copy_node = actions->nodes.emplace_back(node);
copy_map[&node] = &copy_node;
auto & copy_node = actions.nodes.emplace_back(node);
old_to_new_nodes[&node] = &copy_node;
}
for (auto & node : actions->nodes)
for (auto & node : actions.nodes)
for (auto & child : node.children)
child = copy_map[child];
child = old_to_new_nodes[child];
for (const auto & output_node : outputs)
actions->outputs.push_back(copy_map[output_node]);
actions.outputs.push_back(old_to_new_nodes[output_node]);
for (const auto & input_node : inputs)
actions->inputs.push_back(copy_map[input_node]);
actions.inputs.push_back(old_to_new_nodes[input_node]);
return actions;
}
@ -1404,7 +1408,7 @@ const ActionsDAG::Node & ActionsDAG::materializeNode(const Node & node)
return addAlias(*func, name);
}
ActionsDAGPtr ActionsDAG::makeConvertingActions(
ActionsDAG ActionsDAG::makeConvertingActions(
const ColumnsWithTypeAndName & source,
const ColumnsWithTypeAndName & result,
MatchColumnsMode mode,
@ -1421,7 +1425,7 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions(
if (add_casted_columns && mode != MatchColumnsMode::Name)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Converting with add_casted_columns supported only for MatchColumnsMode::Name");
auto actions_dag = std::make_shared<ActionsDAG>(source);
ActionsDAG actions_dag(source);
NodeRawConstPtrs projection(num_result_columns);
FunctionOverloadResolverPtr func_builder_materialize = std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionMaterialize>());
@ -1429,9 +1433,9 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions(
std::unordered_map<std::string_view, std::list<size_t>> inputs;
if (mode == MatchColumnsMode::Name)
{
size_t input_nodes_size = actions_dag->inputs.size();
size_t input_nodes_size = actions_dag.inputs.size();
for (size_t pos = 0; pos < input_nodes_size; ++pos)
inputs[actions_dag->inputs[pos]->result_name].push_back(pos);
inputs[actions_dag.inputs[pos]->result_name].push_back(pos);
}
for (size_t result_col_num = 0; result_col_num < num_result_columns; ++result_col_num)
@ -1444,7 +1448,7 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions(
{
case MatchColumnsMode::Position:
{
src_node = dst_node = actions_dag->inputs[result_col_num];
src_node = dst_node = actions_dag.inputs[result_col_num];
break;
}
@ -1455,7 +1459,7 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions(
{
const auto * res_const = typeid_cast<const ColumnConst *>(res_elem.column.get());
if (ignore_constant_values && res_const)
src_node = dst_node = &actions_dag->addColumn(res_elem);
src_node = dst_node = &actions_dag.addColumn(res_elem);
else
throw Exception(ErrorCodes::THERE_IS_NO_COLUMN,
"Cannot find column `{}` in source stream, there are only columns: [{}]",
@ -1463,7 +1467,7 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions(
}
else
{
src_node = dst_node = actions_dag->inputs[input.front()];
src_node = dst_node = actions_dag.inputs[input.front()];
input.pop_front();
}
break;
@ -1476,7 +1480,7 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions(
if (const auto * src_const = typeid_cast<const ColumnConst *>(dst_node->column.get()))
{
if (ignore_constant_values)
dst_node = &actions_dag->addColumn(res_elem);
dst_node = &actions_dag.addColumn(res_elem);
else if (res_const->getField() != src_const->getField())
throw Exception(
ErrorCodes::ILLEGAL_COLUMN,
@ -1498,7 +1502,7 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions(
column.column = DataTypeString().createColumnConst(0, column.name);
column.type = std::make_shared<DataTypeString>();
const auto * right_arg = &actions_dag->addColumn(std::move(column));
const auto * right_arg = &actions_dag.addColumn(std::move(column));
const auto * left_arg = dst_node;
CastDiagnostic diagnostic = {dst_node->result_name, res_elem.name};
@ -1506,13 +1510,13 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions(
= createInternalCastOverloadResolver(CastType::nonAccurate, std::move(diagnostic));
NodeRawConstPtrs children = { left_arg, right_arg };
dst_node = &actions_dag->addFunction(func_builder_cast, std::move(children), {});
dst_node = &actions_dag.addFunction(func_builder_cast, std::move(children), {});
}
if (dst_node->column && isColumnConst(*dst_node->column) && !(res_elem.column && isColumnConst(*res_elem.column)))
{
NodeRawConstPtrs children = {dst_node};
dst_node = &actions_dag->addFunction(func_builder_materialize, std::move(children), {});
dst_node = &actions_dag.addFunction(func_builder_materialize, std::move(children), {});
}
if (dst_node->result_name != res_elem.name)
@ -1531,7 +1535,7 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions(
}
else
{
dst_node = &actions_dag->addAlias(*dst_node, res_elem.name);
dst_node = &actions_dag.addAlias(*dst_node, res_elem.name);
projection[result_col_num] = dst_node;
}
}
@ -1541,36 +1545,36 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions(
}
}
actions_dag->outputs.swap(projection);
actions_dag->removeUnusedActions(false);
actions_dag.outputs.swap(projection);
actions_dag.removeUnusedActions(false);
return actions_dag;
}
ActionsDAGPtr ActionsDAG::makeAddingColumnActions(ColumnWithTypeAndName column)
ActionsDAG ActionsDAG::makeAddingColumnActions(ColumnWithTypeAndName column)
{
auto adding_column_action = std::make_shared<ActionsDAG>();
ActionsDAG adding_column_action;
FunctionOverloadResolverPtr func_builder_materialize
= std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionMaterialize>());
auto column_name = column.name;
const auto * column_node = &adding_column_action->addColumn(std::move(column));
const auto * column_node = &adding_column_action.addColumn(std::move(column));
NodeRawConstPtrs inputs = {column_node};
const auto & function_node = adding_column_action->addFunction(func_builder_materialize, std::move(inputs), {});
const auto & alias_node = adding_column_action->addAlias(function_node, std::move(column_name));
const auto & function_node = adding_column_action.addFunction(func_builder_materialize, std::move(inputs), {});
const auto & alias_node = adding_column_action.addAlias(function_node, std::move(column_name));
adding_column_action->outputs.push_back(&alias_node);
adding_column_action.outputs.push_back(&alias_node);
return adding_column_action;
}
ActionsDAGPtr ActionsDAG::merge(ActionsDAG && first, ActionsDAG && second)
ActionsDAG ActionsDAG::merge(ActionsDAG && first, ActionsDAG && second)
{
first.mergeInplace(std::move(second));
/// Some actions could become unused. Do not drop inputs to preserve the header.
first.removeUnusedActions(false);
return std::make_shared<ActionsDAG>(std::move(first));
return std::move(first);
}
void ActionsDAG::mergeInplace(ActionsDAG && second)
@ -1963,15 +1967,15 @@ ActionsDAG::SplitResult ActionsDAG::split(std::unordered_set<const Node *> split
second_inputs.push_back(cur.to_second);
}
auto first_actions = std::make_shared<ActionsDAG>();
first_actions->nodes.swap(first_nodes);
first_actions->outputs.swap(first_outputs);
first_actions->inputs.swap(first_inputs);
ActionsDAG first_actions;
first_actions.nodes.swap(first_nodes);
first_actions.outputs.swap(first_outputs);
first_actions.inputs.swap(first_inputs);
auto second_actions = std::make_shared<ActionsDAG>();
second_actions->nodes.swap(second_nodes);
second_actions->outputs.swap(second_outputs);
second_actions->inputs.swap(second_inputs);
ActionsDAG second_actions;
second_actions.nodes.swap(second_nodes);
second_actions.outputs.swap(second_outputs);
second_actions.inputs.swap(second_inputs);
std::unordered_map<const Node *, const Node *> split_nodes_mapping;
if (create_split_nodes_mapping)
@ -2091,7 +2095,7 @@ ActionsDAG::SplitResult ActionsDAG::splitActionsBySortingDescription(const NameS
return res;
}
bool ActionsDAG::isFilterAlwaysFalseForDefaultValueInputs(const std::string & filter_name, const Block & input_stream_header)
bool ActionsDAG::isFilterAlwaysFalseForDefaultValueInputs(const std::string & filter_name, const Block & input_stream_header) const
{
const auto * filter_node = tryFindInOutputs(filter_name);
if (!filter_node)
@ -2115,7 +2119,7 @@ bool ActionsDAG::isFilterAlwaysFalseForDefaultValueInputs(const std::string & fi
input_node_name_to_default_input_column.emplace(input->result_name, std::move(constant_column_with_type_and_name));
}
ActionsDAGPtr filter_with_default_value_inputs;
std::optional<ActionsDAG> filter_with_default_value_inputs;
try
{
@ -2297,12 +2301,12 @@ ColumnsWithTypeAndName prepareFunctionArguments(const ActionsDAG::NodeRawConstPt
///
/// Result actions add single column with conjunction result (it is always first in outputs).
/// No other columns are added or removed.
ActionsDAGPtr ActionsDAG::createActionsForConjunction(NodeRawConstPtrs conjunction, const ColumnsWithTypeAndName & all_inputs)
std::optional<ActionsDAG> ActionsDAG::createActionsForConjunction(NodeRawConstPtrs conjunction, const ColumnsWithTypeAndName & all_inputs)
{
if (conjunction.empty())
return nullptr;
return {};
auto actions = std::make_shared<ActionsDAG>();
ActionsDAG actions;
FunctionOverloadResolverPtr func_builder_and = std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionAnd>());
@ -2343,7 +2347,7 @@ ActionsDAGPtr ActionsDAG::createActionsForConjunction(NodeRawConstPtrs conjuncti
if (cur.next_child_to_visit == cur.node->children.size())
{
auto & node = actions->nodes.emplace_back(*cur.node);
auto & node = actions.nodes.emplace_back(*cur.node);
nodes_mapping[cur.node] = &node;
for (auto & child : node.children)
@ -2366,33 +2370,33 @@ ActionsDAGPtr ActionsDAG::createActionsForConjunction(NodeRawConstPtrs conjuncti
for (const auto * predicate : conjunction)
args.emplace_back(nodes_mapping[predicate]);
result_predicate = &actions->addFunction(func_builder_and, std::move(args), {});
result_predicate = &actions.addFunction(func_builder_and, std::move(args), {});
}
actions->outputs.push_back(result_predicate);
actions.outputs.push_back(result_predicate);
for (const auto & col : all_inputs)
{
const Node * input;
auto & list = required_inputs[col.name];
if (list.empty())
input = &actions->addInput(col);
input = &actions.addInput(col);
else
{
input = list.front();
list.pop_front();
actions->inputs.push_back(input);
actions.inputs.push_back(input);
}
/// We should not add result_predicate into the outputs for the second time.
if (input->result_name != result_predicate->result_name)
actions->outputs.push_back(input);
actions.outputs.push_back(input);
}
return actions;
}
ActionsDAGPtr ActionsDAG::splitActionsForFilterPushDown(
std::optional<ActionsDAG> ActionsDAG::splitActionsForFilterPushDown(
const std::string & filter_name,
bool removes_filter,
const Names & available_inputs,
@ -2408,7 +2412,7 @@ ActionsDAGPtr ActionsDAG::splitActionsForFilterPushDown(
/// If condition is constant let's do nothing.
/// It means there is nothing to push down or optimization was already applied.
if (predicate->type == ActionType::COLUMN)
return nullptr;
return {};
std::unordered_set<const Node *> allowed_nodes;
@ -2432,7 +2436,7 @@ ActionsDAGPtr ActionsDAG::splitActionsForFilterPushDown(
auto conjunction = getConjunctionNodes(predicate, allowed_nodes);
if (conjunction.allowed.empty())
return nullptr;
return {};
chassert(predicate->result_type);
@ -2444,13 +2448,13 @@ ActionsDAGPtr ActionsDAG::splitActionsForFilterPushDown(
&& !conjunction.rejected.front()->result_type->equals(*predicate->result_type))
{
/// No further optimization can be done
return nullptr;
return {};
}
}
auto actions = createActionsForConjunction(conjunction.allowed, all_inputs);
if (!actions)
return nullptr;
return {};
/// Now, when actions are created, update the current DAG.
removeUnusedConjunctions(std::move(conjunction.rejected), predicate, removes_filter);
@ -2555,11 +2559,11 @@ ActionsDAG::ActionsForJOINFilterPushDown ActionsDAG::splitActionsForJOINFilterPu
auto left_stream_filter_to_push_down = createActionsForConjunction(left_stream_allowed_conjunctions, left_stream_header.getColumnsWithTypeAndName());
auto right_stream_filter_to_push_down = createActionsForConjunction(right_stream_allowed_conjunctions, right_stream_header.getColumnsWithTypeAndName());
auto replace_equivalent_columns_in_filter = [](const ActionsDAGPtr & filter,
auto replace_equivalent_columns_in_filter = [](const ActionsDAG & filter,
const Block & stream_header,
const std::unordered_map<std::string, ColumnWithTypeAndName> & columns_to_replace)
{
auto updated_filter = ActionsDAG::buildFilterActionsDAG({filter->getOutputs()[0]}, columns_to_replace);
auto updated_filter = ActionsDAG::buildFilterActionsDAG({filter.getOutputs()[0]}, columns_to_replace);
chassert(updated_filter->getOutputs().size() == 1);
/** If result filter to left or right stream has column that is one of the stream inputs, we need distinguish filter column from
@ -2580,7 +2584,7 @@ ActionsDAG::ActionsForJOINFilterPushDown ActionsDAG::splitActionsForJOINFilterPu
for (const auto & input : updated_filter->getInputs())
updated_filter_inputs[input->result_name].push_back(input);
for (const auto & input : filter->getInputs())
for (const auto & input : filter.getInputs())
{
if (updated_filter_inputs.contains(input->result_name))
continue;
@ -2618,12 +2622,12 @@ ActionsDAG::ActionsForJOINFilterPushDown ActionsDAG::splitActionsForJOINFilterPu
};
if (left_stream_filter_to_push_down)
left_stream_filter_to_push_down = replace_equivalent_columns_in_filter(left_stream_filter_to_push_down,
left_stream_filter_to_push_down = replace_equivalent_columns_in_filter(*left_stream_filter_to_push_down,
left_stream_header,
equivalent_right_stream_column_to_left_stream_column);
if (right_stream_filter_to_push_down)
right_stream_filter_to_push_down = replace_equivalent_columns_in_filter(right_stream_filter_to_push_down,
right_stream_filter_to_push_down = replace_equivalent_columns_in_filter(*right_stream_filter_to_push_down,
right_stream_header,
equivalent_left_stream_column_to_right_stream_column);
@ -2852,13 +2856,13 @@ bool ActionsDAG::isSortingPreserved(
return true;
}
ActionsDAGPtr ActionsDAG::buildFilterActionsDAG(
std::optional<ActionsDAG> ActionsDAG::buildFilterActionsDAG(
const NodeRawConstPtrs & filter_nodes,
const std::unordered_map<std::string, ColumnWithTypeAndName> & node_name_to_input_node_column,
bool single_output_condition_node)
{
if (filter_nodes.empty())
return nullptr;
return {};
struct Frame
{
@ -2866,7 +2870,7 @@ ActionsDAGPtr ActionsDAG::buildFilterActionsDAG(
bool visited_children = false;
};
auto result_dag = std::make_shared<ActionsDAG>();
ActionsDAG result_dag;
std::unordered_map<std::string, const ActionsDAG::Node *> result_inputs;
std::unordered_map<const ActionsDAG::Node *, const ActionsDAG::Node *> node_to_result_node;
@ -2897,7 +2901,7 @@ ActionsDAGPtr ActionsDAG::buildFilterActionsDAG(
{
auto & result_input = result_inputs[input_node_it->second.name];
if (!result_input)
result_input = &result_dag->addInput(input_node_it->second);
result_input = &result_dag.addInput(input_node_it->second);
node_to_result_node.emplace(node, result_input);
nodes_to_process.pop_back();
@ -2924,25 +2928,25 @@ ActionsDAGPtr ActionsDAG::buildFilterActionsDAG(
{
auto & result_input = result_inputs[node->result_name];
if (!result_input)
result_input = &result_dag->addInput({node->column, node->result_type, node->result_name});
result_input = &result_dag.addInput({node->column, node->result_type, node->result_name});
result_node = result_input;
break;
}
case ActionsDAG::ActionType::COLUMN:
{
result_node = &result_dag->addColumn({node->column, node->result_type, node->result_name});
result_node = &result_dag.addColumn({node->column, node->result_type, node->result_name});
break;
}
case ActionsDAG::ActionType::ALIAS:
{
const auto * child = node->children.front();
result_node = &result_dag->addAlias(*(node_to_result_node.find(child)->second), node->result_name);
result_node = &result_dag.addAlias(*(node_to_result_node.find(child)->second), node->result_name);
break;
}
case ActionsDAG::ActionType::ARRAY_JOIN:
{
const auto * child = node->children.front();
result_node = &result_dag->addArrayJoin(*(node_to_result_node.find(child)->second), {});
result_node = &result_dag.addArrayJoin(*(node_to_result_node.find(child)->second), {});
break;
}
case ActionsDAG::ActionType::FUNCTION:
@ -2960,13 +2964,11 @@ ActionsDAGPtr ActionsDAG::buildFilterActionsDAG(
{
if (const auto * index_hint = typeid_cast<const FunctionIndexHint *>(adaptor->getFunction().get()))
{
ActionsDAGPtr index_hint_filter_dag;
const auto & index_hint_args = index_hint->getActions()->getOutputs();
ActionsDAG index_hint_filter_dag;
const auto & index_hint_args = index_hint->getActions().getOutputs();
if (index_hint_args.empty())
index_hint_filter_dag = std::make_shared<ActionsDAG>();
else
index_hint_filter_dag = buildFilterActionsDAG(index_hint_args,
if (!index_hint_args.empty())
index_hint_filter_dag = *buildFilterActionsDAG(index_hint_args,
node_name_to_input_node_column,
false /*single_output_condition_node*/);
@ -2988,7 +2990,7 @@ ActionsDAGPtr ActionsDAG::buildFilterActionsDAG(
auto [arguments, all_const] = getFunctionArguments(function_children);
auto function_base = function_overload_resolver ? function_overload_resolver->build(arguments) : node->function_base;
result_node = &result_dag->addFunctionImpl(
result_node = &result_dag.addFunctionImpl(
function_base,
std::move(function_children),
std::move(arguments),
@ -3003,7 +3005,7 @@ ActionsDAGPtr ActionsDAG::buildFilterActionsDAG(
nodes_to_process.pop_back();
}
auto & result_dag_outputs = result_dag->getOutputs();
auto & result_dag_outputs = result_dag.getOutputs();
result_dag_outputs.reserve(filter_nodes_size);
for (const auto & node : filter_nodes)
@ -3012,7 +3014,7 @@ ActionsDAGPtr ActionsDAG::buildFilterActionsDAG(
if (result_dag_outputs.size() > 1 && single_output_condition_node)
{
FunctionOverloadResolverPtr func_builder_and = std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionAnd>());
result_dag_outputs = { &result_dag->addFunction(func_builder_and, result_dag_outputs, {}) };
result_dag_outputs = { &result_dag.addFunction(func_builder_and, result_dag_outputs, {}) };
}
return result_dag;
@ -3108,10 +3110,9 @@ ActionsDAG::NodeRawConstPtrs ActionsDAG::filterNodesByAllowedInputs(
return nodes;
}
FindOriginalNodeForOutputName::FindOriginalNodeForOutputName(const ActionsDAGPtr & actions_)
:actions(actions_)
FindOriginalNodeForOutputName::FindOriginalNodeForOutputName(const ActionsDAG & actions_)
{
const auto & actions_outputs = actions->getOutputs();
const auto & actions_outputs = actions_.getOutputs();
for (const auto * output_node : actions_outputs)
{
/// find input node which refers to the output node
@ -3147,10 +3148,9 @@ const ActionsDAG::Node * FindOriginalNodeForOutputName::find(const String & outp
return it->second;
}
FindAliasForInputName::FindAliasForInputName(const ActionsDAGPtr & actions_)
:actions(actions_)
FindAliasForInputName::FindAliasForInputName(const ActionsDAG & actions_)
{
const auto & actions_outputs = actions->getOutputs();
const auto & actions_outputs = actions_.getOutputs();
for (const auto * output_node : actions_outputs)
{
/// find input node which corresponds to alias

View File

@ -11,9 +11,6 @@
namespace DB
{
class ActionsDAG;
using ActionsDAGPtr = std::shared_ptr<ActionsDAG>;
class IExecutableFunction;
using ExecutableFunctionPtr = std::shared_ptr<IExecutableFunction>;
@ -247,7 +244,7 @@ public:
/// c * d e
/// \ /
/// c * d - e
static ActionsDAGPtr foldActionsByProjection(
static ActionsDAG foldActionsByProjection(
const std::unordered_map<const Node *, const Node *> & new_inputs,
const NodeRawConstPtrs & required_outputs);
@ -261,9 +258,10 @@ public:
void compileExpressions(size_t min_count_to_compile_expression, const std::unordered_set<const Node *> & lazy_executed_nodes = {});
#endif
ActionsDAGPtr clone() const;
ActionsDAG clone(std::unordered_map<const Node *, Node *> & old_to_new_nodes) const;
ActionsDAG clone() const;
static ActionsDAGPtr cloneSubDAG(const NodeRawConstPtrs & outputs, bool remove_aliases);
static ActionsDAG cloneSubDAG(const NodeRawConstPtrs & outputs, bool remove_aliases);
/// Execute actions for header. Input block must have empty columns.
/// Result should be equal to the execution of ExpressionActions built from this DAG.
@ -301,7 +299,7 @@ public:
/// @param ignore_constant_values - Do not check that constants are same. Use value from result_header.
/// @param add_casted_columns - Create new columns with converted values instead of replacing original.
/// @param new_names - Output parameter for new column names when add_casted_columns is used.
static ActionsDAGPtr makeConvertingActions(
static ActionsDAG makeConvertingActions(
const ColumnsWithTypeAndName & source,
const ColumnsWithTypeAndName & result,
MatchColumnsMode mode,
@ -310,13 +308,13 @@ public:
NameToNameMap * new_names = nullptr);
/// Create expression which add const column and then materialize it.
static ActionsDAGPtr makeAddingColumnActions(ColumnWithTypeAndName column);
static ActionsDAG makeAddingColumnActions(ColumnWithTypeAndName column);
/// Create ActionsDAG which represents expression equivalent to applying first and second actions consequently.
/// Is used to replace `(first -> second)` expression chain to single `merge(first, second)` expression.
/// If first.settings.project_input is set, then outputs of `first` must include inputs of `second`.
/// Otherwise, any two actions may be combined.
static ActionsDAGPtr merge(ActionsDAG && first, ActionsDAG && second);
static ActionsDAG merge(ActionsDAG && first, ActionsDAG && second);
/// The result is similar to merge(*this, second);
/// Invariant : no nodes are removed from the first (this) DAG.
@ -327,12 +325,7 @@ public:
/// *out_outputs is filled with pointers to the nodes corresponding to second.getOutputs().
void mergeNodes(ActionsDAG && second, NodeRawConstPtrs * out_outputs = nullptr);
struct SplitResult
{
ActionsDAGPtr first;
ActionsDAGPtr second;
std::unordered_map<const Node *, const Node *> split_nodes_mapping;
};
struct SplitResult;
/// Split ActionsDAG into two DAGs, where first part contains all nodes from split_nodes and their children.
/// Execution of first then second parts on block is equivalent to execution of initial DAG.
@ -360,7 +353,7 @@ public:
* @param filter_name - name of filter node in current DAG.
* @param input_stream_header - input stream header.
*/
bool isFilterAlwaysFalseForDefaultValueInputs(const std::string & filter_name, const Block & input_stream_header);
bool isFilterAlwaysFalseForDefaultValueInputs(const std::string & filter_name, const Block & input_stream_header) const;
/// Create actions which may calculate part of filter using only available_inputs.
/// If nothing may be calculated, returns nullptr.
@ -379,19 +372,13 @@ public:
/// columns will be transformed like `x, y, z` -> `z > 0, z, x, y` -(remove filter)-> `z, x, y`.
/// To avoid it, add inputs from `all_inputs` list,
/// so actions `x, y, z -> z > 0, x, y, z` -(remove filter)-> `x, y, z` will not change columns order.
ActionsDAGPtr splitActionsForFilterPushDown(
std::optional<ActionsDAG> splitActionsForFilterPushDown(
const std::string & filter_name,
bool removes_filter,
const Names & available_inputs,
const ColumnsWithTypeAndName & all_inputs);
struct ActionsForJOINFilterPushDown
{
ActionsDAGPtr left_stream_filter_to_push_down;
bool left_stream_filter_removes_filter;
ActionsDAGPtr right_stream_filter_to_push_down;
bool right_stream_filter_removes_filter;
};
struct ActionsForJOINFilterPushDown;
/** Split actions for JOIN filter push down.
*
@ -438,7 +425,7 @@ public:
*
* If single_output_condition_node = false, result dag has multiple output nodes.
*/
static ActionsDAGPtr buildFilterActionsDAG(
static std::optional<ActionsDAG> buildFilterActionsDAG(
const NodeRawConstPtrs & filter_nodes,
const std::unordered_map<std::string, ColumnWithTypeAndName> & node_name_to_input_node_column = {},
bool single_output_condition_node = true);
@ -470,21 +457,35 @@ private:
void compileFunctions(size_t min_count_to_compile_expression, const std::unordered_set<const Node *> & lazy_executed_nodes = {});
#endif
static ActionsDAGPtr createActionsForConjunction(NodeRawConstPtrs conjunction, const ColumnsWithTypeAndName & all_inputs);
static std::optional<ActionsDAG> createActionsForConjunction(NodeRawConstPtrs conjunction, const ColumnsWithTypeAndName & all_inputs);
void removeUnusedConjunctions(NodeRawConstPtrs rejected_conjunctions, Node * predicate, bool removes_filter);
};
struct ActionsDAG::SplitResult
{
ActionsDAG first;
ActionsDAG second;
std::unordered_map<const Node *, const Node *> split_nodes_mapping;
};
struct ActionsDAG::ActionsForJOINFilterPushDown
{
std::optional<ActionsDAG> left_stream_filter_to_push_down;
bool left_stream_filter_removes_filter;
std::optional<ActionsDAG> right_stream_filter_to_push_down;
bool right_stream_filter_removes_filter;
};
class FindOriginalNodeForOutputName
{
using NameToNodeIndex = std::unordered_map<std::string_view, const ActionsDAG::Node *>;
public:
explicit FindOriginalNodeForOutputName(const ActionsDAGPtr & actions);
explicit FindOriginalNodeForOutputName(const ActionsDAG & actions);
const ActionsDAG::Node * find(const String & output_name);
private:
ActionsDAGPtr actions;
NameToNodeIndex index;
};
@ -493,11 +494,10 @@ class FindAliasForInputName
using NameToNodeIndex = std::unordered_map<std::string_view, const ActionsDAG::Node *>;
public:
explicit FindAliasForInputName(const ActionsDAGPtr & actions);
explicit FindAliasForInputName(const ActionsDAG & actions);
const ActionsDAG::Node * find(const String & name);
private:
ActionsDAGPtr actions;
NameToNodeIndex index;
};

View File

@ -1024,7 +1024,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data &
dag.project(args);
auto index_hint = std::make_shared<FunctionIndexHint>();
index_hint->setActions(std::make_shared<ActionsDAG>(std::move(dag)));
index_hint->setActions(std::move(dag));
// Arguments are removed. We add function instead of constant column to avoid constant folding.
data.addFunction(std::make_unique<FunctionToOverloadResolverAdaptor>(index_hint), {}, column_name);
@ -1287,7 +1287,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data &
lambda_dag.removeUnusedActions(Names(1, result_name));
auto lambda_actions = std::make_shared<ExpressionActions>(
std::make_shared<ActionsDAG>(std::move(lambda_dag)),
std::move(lambda_dag),
ExpressionActionsSettings::fromContext(data.getContext(), CompileExpressions::yes));
DataTypePtr result_type = lambda_actions->getSampleBlock().getByName(result_name).type;

View File

@ -18,12 +18,6 @@ namespace DB
class ASTExpressionList;
class ASTFunction;
class ExpressionActions;
using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
class ActionsDAG;
using ActionsDAGPtr = std::shared_ptr<ActionsDAG>;
class IFunctionOverloadResolver;
using FunctionOverloadResolverPtr = std::shared_ptr<IFunctionOverloadResolver>;
@ -32,7 +26,7 @@ FutureSetPtr makeExplicitSet(
const ASTFunction * node, const ActionsDAG & actions, ContextPtr context, PreparedSets & prepared_sets);
/** For ActionsVisitor
* A stack of ExpressionActions corresponding to nested lambda expressions.
* A stack of ActionsDAG corresponding to nested lambda expressions.
* The new action should be added to the highest possible level.
* For example, in the expression "select arrayMap(x -> x + column1 * column2, array1)"
* calculation of the product must be done outside the lambda expression (it does not depend on x),

View File

@ -49,18 +49,17 @@ namespace ErrorCodes
static std::unordered_set<const ActionsDAG::Node *> processShortCircuitFunctions(const ActionsDAG & actions_dag, ShortCircuitFunctionEvaluation short_circuit_function_evaluation);
ExpressionActions::ExpressionActions(ActionsDAGPtr actions_dag_, const ExpressionActionsSettings & settings_, bool project_inputs_)
: project_inputs(project_inputs_)
ExpressionActions::ExpressionActions(ActionsDAG actions_dag_, const ExpressionActionsSettings & settings_, bool project_inputs_)
: actions_dag(std::move(actions_dag_))
, project_inputs(project_inputs_)
, settings(settings_)
{
actions_dag = actions_dag_->clone();
/// It's important to determine lazy executed nodes before compiling expressions.
std::unordered_set<const ActionsDAG::Node *> lazy_executed_nodes = processShortCircuitFunctions(*actions_dag, settings.short_circuit_function_evaluation);
std::unordered_set<const ActionsDAG::Node *> lazy_executed_nodes = processShortCircuitFunctions(actions_dag, settings.short_circuit_function_evaluation);
#if USE_EMBEDDED_COMPILER
if (settings.can_compile_expressions && settings.compile_expressions == CompileExpressions::yes)
actions_dag->compileExpressions(settings.min_count_to_compile_expression, lazy_executed_nodes);
actions_dag.compileExpressions(settings.min_count_to_compile_expression, lazy_executed_nodes);
#endif
linearizeActions(lazy_executed_nodes);
@ -68,12 +67,32 @@ ExpressionActions::ExpressionActions(ActionsDAGPtr actions_dag_, const Expressio
if (settings.max_temporary_columns && num_columns > settings.max_temporary_columns)
throw Exception(ErrorCodes::TOO_MANY_TEMPORARY_COLUMNS,
"Too many temporary columns: {}. Maximum: {}",
actions_dag->dumpNames(), settings.max_temporary_columns);
actions_dag.dumpNames(), settings.max_temporary_columns);
}
ExpressionActionsPtr ExpressionActions::clone() const
{
return std::make_shared<ExpressionActions>(*this);
auto copy = std::make_shared<ExpressionActions>(ExpressionActions());
std::unordered_map<const Node *, Node *> copy_map;
copy->actions_dag = actions_dag.clone(copy_map);
copy->actions = actions;
for (auto & action : copy->actions)
action.node = copy_map[action.node];
for (const auto * input : copy->actions_dag.getInputs())
copy->input_positions.emplace(input->result_name, input_positions.at(input->result_name));
copy->num_columns = num_columns;
copy->required_columns = required_columns;
copy->result_positions = result_positions;
copy->sample_block = sample_block;
copy->project_inputs = project_inputs;
copy->settings = settings;
return copy;
}
namespace
@ -341,8 +360,8 @@ void ExpressionActions::linearizeActions(const std::unordered_set<const ActionsD
};
const auto & nodes = getNodes();
const auto & outputs = actions_dag->getOutputs();
const auto & inputs = actions_dag->getInputs();
const auto & outputs = actions_dag.getOutputs();
const auto & inputs = actions_dag.getInputs();
auto reverse_info = getActionsDAGReverseInfo(nodes, outputs);
std::vector<Data> data;

View File

@ -70,7 +70,7 @@ public:
using NameToInputMap = std::unordered_map<std::string_view, std::list<size_t>>;
private:
ActionsDAGPtr actions_dag;
ActionsDAG actions_dag;
Actions actions;
size_t num_columns = 0;
@ -84,14 +84,13 @@ private:
ExpressionActionsSettings settings;
public:
ExpressionActions() = delete;
explicit ExpressionActions(ActionsDAGPtr actions_dag_, const ExpressionActionsSettings & settings_ = {}, bool project_inputs_ = false);
ExpressionActions(const ExpressionActions &) = default;
ExpressionActions & operator=(const ExpressionActions &) = default;
explicit ExpressionActions(ActionsDAG actions_dag_, const ExpressionActionsSettings & settings_ = {}, bool project_inputs_ = false);
ExpressionActions(ExpressionActions &&) = default;
ExpressionActions & operator=(ExpressionActions &&) = default;
const Actions & getActions() const { return actions; }
const std::list<Node> & getNodes() const { return actions_dag->getNodes(); }
const ActionsDAG & getActionsDAG() const { return *actions_dag; }
const std::list<Node> & getNodes() const { return actions_dag.getNodes(); }
const ActionsDAG & getActionsDAG() const { return actions_dag; }
const ColumnNumbers & getResultPositions() const { return result_positions; }
const ExpressionActionsSettings & getSettings() const { return settings; }
@ -131,6 +130,7 @@ public:
ExpressionActionsPtr clone() const;
private:
ExpressionActions() = default;
void checkLimits(const ColumnsWithTypeAndName & columns) const;
void linearizeActions(const std::unordered_set<const Node *> & lazy_executed_nodes);

View File

@ -658,7 +658,7 @@ void ExpressionAnalyzer::makeWindowDescriptionFromAST(const Context & context_,
with_alias->getColumnName(), 1 /* direction */,
1 /* nulls_direction */));
auto actions_dag = std::make_shared<ActionsDAG>(aggregated_columns);
auto actions_dag = std::make_unique<ActionsDAG>(aggregated_columns);
getRootActions(column_ast, false, *actions_dag);
desc.partition_by_actions.push_back(std::move(actions_dag));
}
@ -679,7 +679,7 @@ void ExpressionAnalyzer::makeWindowDescriptionFromAST(const Context & context_,
order_by_element.direction,
order_by_element.nulls_direction));
auto actions_dag = std::make_shared<ActionsDAG>(aggregated_columns);
auto actions_dag = std::make_unique<ActionsDAG>(aggregated_columns);
getRootActions(column_ast, false, *actions_dag);
desc.order_by_actions.push_back(std::move(actions_dag));
}
@ -823,13 +823,14 @@ void ExpressionAnalyzer::makeWindowDescriptions(ActionsDAG & actions)
makeWindowDescriptionFromAST(*current_context, window_descriptions,
desc, &definition);
auto full_sort_description = desc.full_sort_description;
auto [it, inserted] = window_descriptions.insert(
{default_window_name, desc});
{default_window_name, std::move(desc)});
if (!inserted)
{
assert(it->second.full_sort_description
== desc.full_sort_description);
assert(it->second.full_sort_description == full_sort_description);
}
it->second.window_functions.push_back(window_function);
@ -927,7 +928,7 @@ JoinPtr SelectQueryExpressionAnalyzer::appendJoin(
{
const ColumnsWithTypeAndName & left_sample_columns = chain.getLastStep().getResultColumns();
ActionsDAGPtr converting_actions;
std::optional<ActionsDAG> converting_actions;
JoinPtr join = makeJoin(*syntax->ast_join, left_sample_columns, converting_actions);
if (converting_actions)
@ -1039,7 +1040,7 @@ static std::unique_ptr<QueryPlan> buildJoinedPlan(
/// Actions which need to be calculated on joined block.
auto joined_block_actions = analyzed_join.createJoinedBlockActions(context);
NamesWithAliases required_columns_with_aliases = analyzed_join.getRequiredColumns(
Block(joined_block_actions->getResultColumns()), joined_block_actions->getRequiredColumns().getNames());
Block(joined_block_actions.getResultColumns()), joined_block_actions.getRequiredColumns().getNames());
Names original_right_column_names;
for (auto & pr : required_columns_with_aliases)
@ -1060,17 +1061,17 @@ static std::unique_ptr<QueryPlan> buildJoinedPlan(
interpreter->buildQueryPlan(*joined_plan);
{
Block original_right_columns = interpreter->getSampleBlock();
auto rename_dag = std::make_unique<ActionsDAG>(original_right_columns.getColumnsWithTypeAndName());
ActionsDAG rename_dag(original_right_columns.getColumnsWithTypeAndName());
for (const auto & name_with_alias : required_columns_with_aliases)
{
if (name_with_alias.first != name_with_alias.second && original_right_columns.has(name_with_alias.first))
{
auto pos = original_right_columns.getPositionByName(name_with_alias.first);
const auto & alias = rename_dag->addAlias(*rename_dag->getInputs()[pos], name_with_alias.second);
rename_dag->getOutputs()[pos] = &alias;
const auto & alias = rename_dag.addAlias(*rename_dag.getInputs()[pos], name_with_alias.second);
rename_dag.getOutputs()[pos] = &alias;
}
}
rename_dag->appendInputsForUnusedColumns(joined_plan->getCurrentDataStream().header);
rename_dag.appendInputsForUnusedColumns(joined_plan->getCurrentDataStream().header);
auto rename_step = std::make_unique<ExpressionStep>(joined_plan->getCurrentDataStream(), std::move(rename_dag));
rename_step->setStepDescription("Rename joined columns");
joined_plan->addStep(std::move(rename_step));
@ -1130,14 +1131,14 @@ std::shared_ptr<DirectKeyValueJoin> tryKeyValueJoin(std::shared_ptr<TableJoin> a
JoinPtr SelectQueryExpressionAnalyzer::makeJoin(
const ASTTablesInSelectQueryElement & join_element,
const ColumnsWithTypeAndName & left_columns,
ActionsDAGPtr & left_convert_actions)
std::optional<ActionsDAG> & left_convert_actions)
{
/// Two JOINs are not supported with the same subquery, but different USINGs.
if (joined_plan)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Table join was already created for query");
ActionsDAGPtr right_convert_actions = nullptr;
std::optional<ActionsDAG> right_convert_actions;
const auto & analyzed_join = syntax->analyzed_join;
@ -1145,7 +1146,7 @@ JoinPtr SelectQueryExpressionAnalyzer::makeJoin(
{
auto joined_block_actions = analyzed_join->createJoinedBlockActions(getContext());
NamesWithAliases required_columns_with_aliases = analyzed_join->getRequiredColumns(
Block(joined_block_actions->getResultColumns()), joined_block_actions->getRequiredColumns().getNames());
Block(joined_block_actions.getResultColumns()), joined_block_actions.getRequiredColumns().getNames());
Names original_right_column_names;
for (auto & pr : required_columns_with_aliases)
@ -1162,7 +1163,7 @@ JoinPtr SelectQueryExpressionAnalyzer::makeJoin(
std::tie(left_convert_actions, right_convert_actions) = analyzed_join->createConvertingActions(left_columns, right_columns);
if (right_convert_actions)
{
auto converting_step = std::make_unique<ExpressionStep>(joined_plan->getCurrentDataStream(), right_convert_actions);
auto converting_step = std::make_unique<ExpressionStep>(joined_plan->getCurrentDataStream(), std::move(*right_convert_actions));
converting_step->setStepDescription("Convert joined columns");
joined_plan->addStep(std::move(converting_step));
}
@ -1354,10 +1355,10 @@ bool SelectQueryExpressionAnalyzer::appendGroupBy(ExpressionActionsChain & chain
{
for (auto & child : asts)
{
auto actions_dag = std::make_shared<ActionsDAG>(columns_after_join);
getRootActions(child, only_types, *actions_dag);
ActionsDAG actions_dag(columns_after_join);
getRootActions(child, only_types, actions_dag);
group_by_elements_actions.emplace_back(
std::make_shared<ExpressionActions>(actions_dag, ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes)));
std::make_shared<ExpressionActions>(std::move(actions_dag), ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes)));
}
}
@ -1471,7 +1472,7 @@ void SelectQueryExpressionAnalyzer::appendGroupByModifiers(ActionsDAG & before_a
ExpressionActionsChain::Step & step = chain.addStep(before_aggregation.getNamesAndTypesList());
step.required_output = std::move(required_output);
step.actions()->dag = std::move(*ActionsDAG::makeConvertingActions(source_columns, result_columns, ActionsDAG::MatchColumnsMode::Position));
step.actions()->dag = ActionsDAG::makeConvertingActions(source_columns, result_columns, ActionsDAG::MatchColumnsMode::Position);
}
void SelectQueryExpressionAnalyzer::appendSelectSkipWindowExpressions(ExpressionActionsChain::Step & step, ASTPtr const & node)
@ -1607,10 +1608,10 @@ ActionsAndProjectInputsFlagPtr SelectQueryExpressionAnalyzer::appendOrderBy(Expr
{
for (const auto & child : select_query->orderBy()->children)
{
auto actions_dag = std::make_shared<ActionsDAG>(columns_after_join);
getRootActions(child, only_types, *actions_dag);
ActionsDAG actions_dag(columns_after_join);
getRootActions(child, only_types, actions_dag);
order_by_elements_actions.emplace_back(
std::make_shared<ExpressionActions>(actions_dag, ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes)));
std::make_shared<ExpressionActions>(std::move(actions_dag), ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes)));
}
}
@ -1737,7 +1738,7 @@ void ExpressionAnalyzer::appendExpression(ExpressionActionsChain & chain, const
step.addRequiredOutput(expr->getColumnName());
}
ActionsDAGPtr ExpressionAnalyzer::getActionsDAG(bool add_aliases, bool remove_unused_result)
ActionsDAG ExpressionAnalyzer::getActionsDAG(bool add_aliases, bool remove_unused_result)
{
ActionsDAG actions_dag(aggregated_columns);
NamesWithAliases result_columns;
@ -1789,7 +1790,7 @@ ActionsDAGPtr ExpressionAnalyzer::getActionsDAG(bool add_aliases, bool remove_un
actions_dag.removeUnusedActions(name_set);
}
return std::make_unique<ActionsDAG>(std::move(actions_dag));
return actions_dag;
}
ExpressionActionsPtr ExpressionAnalyzer::getActions(bool add_aliases, bool remove_unused_result, CompileExpressions compile_expressions)
@ -1798,17 +1799,17 @@ ExpressionActionsPtr ExpressionAnalyzer::getActions(bool add_aliases, bool remov
getActionsDAG(add_aliases, remove_unused_result), ExpressionActionsSettings::fromContext(getContext(), compile_expressions), add_aliases && remove_unused_result);
}
ActionsDAGPtr ExpressionAnalyzer::getConstActionsDAG(const ColumnsWithTypeAndName & constant_inputs)
ActionsDAG ExpressionAnalyzer::getConstActionsDAG(const ColumnsWithTypeAndName & constant_inputs)
{
auto actions = std::make_shared<ActionsDAG>(constant_inputs);
getRootActions(query, true /* no_makeset_for_subqueries */, *actions, true /* only_consts */);
ActionsDAG actions(constant_inputs);
getRootActions(query, true /* no_makeset_for_subqueries */, actions, true /* only_consts */);
return actions;
}
ExpressionActionsPtr ExpressionAnalyzer::getConstActions(const ColumnsWithTypeAndName & constant_inputs)
{
auto actions = getConstActionsDAG(constant_inputs);
return std::make_shared<ExpressionActions>(actions, ExpressionActionsSettings::fromContext(getContext()));
return std::make_shared<ExpressionActions>(std::move(actions), ExpressionActionsSettings::fromContext(getContext()));
}
std::unique_ptr<QueryPlan> SelectQueryExpressionAnalyzer::getJoinedPlan()
@ -1879,8 +1880,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult(
if (prewhere_dag_and_flags)
{
auto dag = std::make_shared<ActionsDAG>(std::move(prewhere_dag_and_flags->dag));
prewhere_info = std::make_shared<PrewhereInfo>(std::move(dag), query.prewhere()->getColumnName());
prewhere_info = std::make_shared<PrewhereInfo>(std::move(prewhere_dag_and_flags->dag), query.prewhere()->getColumnName());
prewhere_dag_and_flags.reset();
}
@ -1923,7 +1923,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult(
if (storage && additional_filter)
{
Names columns_for_additional_filter = additional_filter->actions->getRequiredColumnsNames();
Names columns_for_additional_filter = additional_filter->actions.getRequiredColumnsNames();
additional_required_columns_after_prewhere.insert(additional_required_columns_after_prewhere.end(),
columns_for_additional_filter.begin(), columns_for_additional_filter.end());
}
@ -1944,10 +1944,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult(
Block before_prewhere_sample = source_header;
if (sanitizeBlock(before_prewhere_sample))
{
auto dag = prewhere_dag_and_flags->dag.clone();
ExpressionActions(
dag,
ExpressionActionsSettings::fromSettings(context->getSettingsRef())).execute(before_prewhere_sample);
before_prewhere_sample = prewhere_dag_and_flags->dag.updateHeader(before_prewhere_sample);
auto & column_elem = before_prewhere_sample.getByName(query.prewhere()->getColumnName());
/// If the filter column is a constant, record it.
if (column_elem.column)
@ -1979,9 +1976,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult(
before_where_sample = source_header;
if (sanitizeBlock(before_where_sample))
{
ExpressionActions(
before_where->dag.clone(),
ExpressionActionsSettings::fromSettings(context->getSettingsRef())).execute(before_where_sample);
before_where_sample = before_where->dag.updateHeader(before_where_sample);
auto & column_elem
= before_where_sample.getByName(query.where()->getColumnName());
@ -2054,7 +2049,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult(
auto & step = chain.lastStep(query_analyzer.aggregated_columns);
auto & actions = step.actions()->dag;
actions = std::move(*ActionsDAG::merge(std::move(actions), std::move(*converting)));
actions = ActionsDAG::merge(std::move(actions), std::move(converting));
}
}
@ -2235,12 +2230,11 @@ void ExpressionAnalysisResult::checkActions() const
/// Check that PREWHERE doesn't contain unusual actions. Unusual actions are that can change number of rows.
if (hasPrewhere())
{
auto check_actions = [](const ActionsDAGPtr & actions)
auto check_actions = [](ActionsDAG & actions)
{
if (actions)
for (const auto & node : actions->getNodes())
if (node.type == ActionsDAG::ActionType::ARRAY_JOIN)
throw Exception(ErrorCodes::ILLEGAL_PREWHERE, "PREWHERE cannot contain ARRAY JOIN action");
for (const auto & node : actions.getNodes())
if (node.type == ActionsDAG::ActionType::ARRAY_JOIN)
throw Exception(ErrorCodes::ILLEGAL_PREWHERE, "PREWHERE cannot contain ARRAY JOIN action");
};
check_actions(prewhere_info->prewhere_actions);

View File

@ -38,9 +38,6 @@ using StorageMetadataPtr = std::shared_ptr<const StorageInMemoryMetadata>;
class ArrayJoinAction;
using ArrayJoinActionPtr = std::shared_ptr<ArrayJoinAction>;
class ActionsDAG;
using ActionsDAGPtr = std::shared_ptr<ActionsDAG>;
/// Create columns in block or return false if not possible
bool sanitizeBlock(Block & block, bool throw_if_cannot_create_column = false);
@ -117,12 +114,12 @@ public:
/// If add_aliases, only the calculated values in the desired order and add aliases.
/// If also remove_unused_result, than only aliases remain in the output block.
/// Otherwise, only temporary columns will be deleted from the block.
ActionsDAGPtr getActionsDAG(bool add_aliases, bool remove_unused_result = true);
ActionsDAG getActionsDAG(bool add_aliases, bool remove_unused_result = true);
ExpressionActionsPtr getActions(bool add_aliases, bool remove_unused_result = true, CompileExpressions compile_expressions = CompileExpressions::no);
/// Get actions to evaluate a constant expression. The function adds constants and applies functions that depend only on constants.
/// Does not execute subqueries.
ActionsDAGPtr getConstActionsDAG(const ColumnsWithTypeAndName & constant_inputs = {});
ActionsDAG getConstActionsDAG(const ColumnsWithTypeAndName & constant_inputs = {});
ExpressionActionsPtr getConstActions(const ColumnsWithTypeAndName & constant_inputs = {});
/** Sets that require a subquery to be create.
@ -367,7 +364,7 @@ private:
JoinPtr makeJoin(
const ASTTablesInSelectQueryElement & join_element,
const ColumnsWithTypeAndName & left_columns,
ActionsDAGPtr & left_convert_actions);
std::optional<ActionsDAG> & left_convert_actions);
const ASTSelectQuery * getAggregatingQuery() const;

View File

@ -296,7 +296,7 @@ private:
{
auto joined_block_actions = data.table_join->createJoinedBlockActions(data.getContext());
NamesWithAliases required_columns_with_aliases = data.table_join->getRequiredColumns(
Block(joined_block_actions->getResultColumns()), joined_block_actions->getRequiredColumns().getNames());
Block(joined_block_actions.getResultColumns()), joined_block_actions.getRequiredColumns().getNames());
for (auto & pr : required_columns_with_aliases)
required_columns.push_back(pr.first);

View File

@ -124,16 +124,16 @@ static ASTPtr parseAdditionalPostFilter(const Context & context)
"additional filter", settings.max_query_size, settings.max_parser_depth, settings.max_parser_backtracks);
}
static ActionsDAGPtr makeAdditionalPostFilter(ASTPtr & ast, ContextPtr context, const Block & header)
static ActionsDAG makeAdditionalPostFilter(ASTPtr & ast, ContextPtr context, const Block & header)
{
auto syntax_result = TreeRewriter(context).analyze(ast, header.getNamesAndTypesList());
String result_column_name = ast->getColumnName();
auto dag = ExpressionAnalyzer(ast, syntax_result, context).getActionsDAG(false, false);
const ActionsDAG::Node * result_node = &dag->findInOutputs(result_column_name);
auto & outputs = dag->getOutputs();
const ActionsDAG::Node * result_node = &dag.findInOutputs(result_column_name);
auto & outputs = dag.getOutputs();
outputs.clear();
outputs.reserve(dag->getInputs().size() + 1);
for (const auto * node : dag->getInputs())
outputs.reserve(dag.getInputs().size() + 1);
for (const auto * node : dag.getInputs())
outputs.push_back(node);
outputs.push_back(result_node);
@ -151,7 +151,7 @@ void IInterpreterUnionOrSelectQuery::addAdditionalPostFilter(QueryPlan & plan) c
return;
auto dag = makeAdditionalPostFilter(ast, context, plan.getCurrentDataStream().header);
std::string filter_name = dag->getOutputs().back()->result_name;
std::string filter_name = dag.getOutputs().back()->result_name;
auto filter_step = std::make_unique<FilterStep>(
plan.getCurrentDataStream(), std::move(dag), std::move(filter_name), true);
filter_step->setStepDescription("Additional result filter");

View File

@ -390,7 +390,7 @@ Chain InterpreterInsertQuery::buildPreSinkChain(
context_ptr,
null_as_default);
auto adding_missing_defaults_actions = std::make_shared<ExpressionActions>(adding_missing_defaults_dag);
auto adding_missing_defaults_actions = std::make_shared<ExpressionActions>(std::move(adding_missing_defaults_dag));
/// Actually we don't know structure of input blocks from query/table,
/// because some clients break insertion protocol (columns != header)
@ -536,7 +536,7 @@ QueryPipeline InterpreterInsertQuery::buildInsertSelectPipeline(ASTInsertQuery &
pipeline.getHeader().getColumnsWithTypeAndName(),
query_sample_block.getColumnsWithTypeAndName(),
ActionsDAG::MatchColumnsMode::Position);
auto actions = std::make_shared<ExpressionActions>(actions_dag, ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes));
auto actions = std::make_shared<ExpressionActions>(std::move(actions_dag), ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes));
pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr
{

View File

@ -177,15 +177,15 @@ FilterDAGInfoPtr generateFilterActions(
/// Using separate expression analyzer to prevent any possible alias injection
auto syntax_result = TreeRewriter(context).analyzeSelect(query_ast, TreeRewriterResult({}, storage, storage_snapshot));
SelectQueryExpressionAnalyzer analyzer(query_ast, syntax_result, context, metadata_snapshot, {}, false, {}, prepared_sets);
filter_info->actions = std::make_unique<ActionsDAG>(std::move(analyzer.simpleSelectActions()->dag));
filter_info->actions = std::move(analyzer.simpleSelectActions()->dag);
filter_info->column_name = expr_list->children.at(0)->getColumnName();
filter_info->actions->removeUnusedActions(NameSet{filter_info->column_name});
filter_info->actions.removeUnusedActions(NameSet{filter_info->column_name});
for (const auto * node : filter_info->actions->getInputs())
filter_info->actions->getOutputs().push_back(node);
for (const auto * node : filter_info->actions.getInputs())
filter_info->actions.getOutputs().push_back(node);
auto required_columns_from_filter = filter_info->actions->getRequiredColumns();
auto required_columns_from_filter = filter_info->actions.getRequiredColumns();
for (const auto & column : required_columns_from_filter)
{
@ -937,7 +937,7 @@ bool InterpreterSelectQuery::adjustParallelReplicasAfterAnalysis()
{
{
const auto & node
= query_info_copy.prewhere_info->prewhere_actions->findInOutputs(query_info_copy.prewhere_info->prewhere_column_name);
= query_info_copy.prewhere_info->prewhere_actions.findInOutputs(query_info_copy.prewhere_info->prewhere_column_name);
added_filter_nodes.nodes.push_back(&node);
}
@ -949,7 +949,8 @@ bool InterpreterSelectQuery::adjustParallelReplicasAfterAnalysis()
}
}
query_info_copy.filter_actions_dag = ActionsDAG::buildFilterActionsDAG(added_filter_nodes.nodes);
if (auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(added_filter_nodes.nodes))
query_info_copy.filter_actions_dag = std::make_shared<const ActionsDAG>(std::move(*filter_actions_dag));
UInt64 rows_to_read = storage_merge_tree->estimateNumberOfRowsToRead(context, storage_snapshot, query_info_copy);
/// Note that we treat an estimation of 0 rows as a real estimation
size_t number_of_replicas_to_use = rows_to_read / settings.parallel_replicas_min_number_of_rows_per_replica;
@ -984,7 +985,7 @@ void InterpreterSelectQuery::buildQueryPlan(QueryPlan & query_plan)
ActionsDAG::MatchColumnsMode::Name,
true);
auto converting = std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), convert_actions_dag);
auto converting = std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), std::move(convert_actions_dag));
query_plan.addStep(std::move(converting));
}
@ -1057,7 +1058,7 @@ Block InterpreterSelectQuery::getSampleBlockImpl()
if (analysis_result.prewhere_info)
{
header = analysis_result.prewhere_info->prewhere_actions->updateHeader(header);
header = analysis_result.prewhere_info->prewhere_actions.updateHeader(header);
if (analysis_result.prewhere_info->remove_prewhere_column)
header.erase(analysis_result.prewhere_info->prewhere_column_name);
}
@ -1308,12 +1309,12 @@ static InterpolateDescriptionPtr getInterpolateDescription(
auto syntax_result = TreeRewriter(context).analyze(exprs, source_columns);
ExpressionAnalyzer analyzer(exprs, syntax_result, context);
ActionsDAGPtr actions = analyzer.getActionsDAG(true);
ActionsDAGPtr conv_dag = ActionsDAG::makeConvertingActions(actions->getResultColumns(),
ActionsDAG actions = analyzer.getActionsDAG(true);
ActionsDAG conv_dag = ActionsDAG::makeConvertingActions(actions.getResultColumns(),
result_columns, ActionsDAG::MatchColumnsMode::Position, true);
ActionsDAGPtr merge_dag = ActionsDAG::merge(std::move(*actions->clone()), std::move(*conv_dag));
ActionsDAG merge_dag = ActionsDAG::merge(std::move(actions), std::move(conv_dag));
interpolate_descr = std::make_shared<InterpolateDescription>(merge_dag, aliases);
interpolate_descr = std::make_shared<InterpolateDescription>(std::move(merge_dag), aliases);
}
return interpolate_descr;
@ -1496,7 +1497,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional<P
{
auto row_level_security_step = std::make_unique<FilterStep>(
query_plan.getCurrentDataStream(),
expressions.filter_info->actions,
expressions.filter_info->actions.clone(),
expressions.filter_info->column_name,
expressions.filter_info->do_remove_column);
@ -1510,7 +1511,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional<P
{
auto row_level_filter_step = std::make_unique<FilterStep>(
query_plan.getCurrentDataStream(),
expressions.prewhere_info->row_level_filter,
expressions.prewhere_info->row_level_filter->clone(),
expressions.prewhere_info->row_level_column_name,
true);
@ -1520,7 +1521,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional<P
auto prewhere_step = std::make_unique<FilterStep>(
query_plan.getCurrentDataStream(),
expressions.prewhere_info->prewhere_actions,
expressions.prewhere_info->prewhere_actions.clone(),
expressions.prewhere_info->prewhere_column_name,
expressions.prewhere_info->remove_prewhere_column);
@ -1622,7 +1623,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional<P
{
auto row_level_security_step = std::make_unique<FilterStep>(
query_plan.getCurrentDataStream(),
expressions.filter_info->actions,
expressions.filter_info->actions.clone(),
expressions.filter_info->column_name,
expressions.filter_info->do_remove_column);
@ -1630,11 +1631,11 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional<P
query_plan.addStep(std::move(row_level_security_step));
}
const auto add_filter_step = [&](const auto & new_filter_info, const std::string & description)
const auto add_filter_step = [&](auto & new_filter_info, const std::string & description)
{
auto filter_step = std::make_unique<FilterStep>(
query_plan.getCurrentDataStream(),
new_filter_info->actions,
std::move(new_filter_info->actions),
new_filter_info->column_name,
new_filter_info->do_remove_column);
@ -2056,18 +2057,20 @@ void InterpreterSelectQuery::addEmptySourceToQueryPlan(QueryPlan & query_plan, c
if (prewhere_info.row_level_filter)
{
auto row_level_actions = std::make_shared<ExpressionActions>(prewhere_info.row_level_filter->clone());
pipe.addSimpleTransform([&](const Block & header)
{
return std::make_shared<FilterTransform>(header,
std::make_shared<ExpressionActions>(prewhere_info.row_level_filter),
row_level_actions,
prewhere_info.row_level_column_name, true);
});
}
auto filter_actions = std::make_shared<ExpressionActions>(prewhere_info.prewhere_actions.clone());
pipe.addSimpleTransform([&](const Block & header)
{
return std::make_shared<FilterTransform>(
header, std::make_shared<ExpressionActions>(prewhere_info.prewhere_actions),
header, filter_actions,
prewhere_info.prewhere_column_name, prewhere_info.remove_prewhere_column);
});
}
@ -2111,8 +2114,8 @@ void InterpreterSelectQuery::applyFiltersToPrewhereInAnalysis(ExpressionAnalysis
if (does_storage_support_prewhere && shouldMoveToPrewhere())
{
/// Execute row level filter in prewhere as a part of "move to prewhere" optimization.
analysis.prewhere_info = std::make_shared<PrewhereInfo>(analysis.filter_info->actions, analysis.filter_info->column_name);
analysis.prewhere_info->remove_prewhere_column = analysis.filter_info->do_remove_column;
analysis.prewhere_info = std::make_shared<PrewhereInfo>(std::move(analysis.filter_info->actions), analysis.filter_info->column_name);
analysis.prewhere_info->remove_prewhere_column = std::move(analysis.filter_info->do_remove_column);
analysis.prewhere_info->need_filter = true;
analysis.filter_info = nullptr;
}
@ -2120,8 +2123,8 @@ void InterpreterSelectQuery::applyFiltersToPrewhereInAnalysis(ExpressionAnalysis
else
{
/// Add row level security actions to prewhere.
analysis.prewhere_info->row_level_filter = analysis.filter_info->actions;
analysis.prewhere_info->row_level_column_name = analysis.filter_info->column_name;
analysis.prewhere_info->row_level_filter = std::move(analysis.filter_info->actions);
analysis.prewhere_info->row_level_column_name = std::move(analysis.filter_info->column_name);
analysis.filter_info = nullptr;
}
}
@ -2154,7 +2157,7 @@ void InterpreterSelectQuery::addPrewhereAliasActions()
if (prewhere_info)
{
/// Get some columns directly from PREWHERE expression actions
auto prewhere_required_columns = prewhere_info->prewhere_actions->getRequiredColumns().getNames();
auto prewhere_required_columns = prewhere_info->prewhere_actions.getRequiredColumns().getNames();
columns.insert(prewhere_required_columns.begin(), prewhere_required_columns.end());
if (prewhere_info->row_level_filter)
@ -2226,7 +2229,7 @@ void InterpreterSelectQuery::addPrewhereAliasActions()
if (prewhere_info)
{
NameSet columns_to_remove(columns_to_remove_after_prewhere.begin(), columns_to_remove_after_prewhere.end());
Block prewhere_actions_result = prewhere_info->prewhere_actions->getResultColumns();
Block prewhere_actions_result = prewhere_info->prewhere_actions.getResultColumns();
/// Populate required columns with the columns, added by PREWHERE actions and not removed afterwards.
/// XXX: looks hacky that we already know which columns after PREWHERE we won't need for sure.
@ -2265,7 +2268,7 @@ void InterpreterSelectQuery::addPrewhereAliasActions()
{
/// Don't remove columns which are needed to be aliased.
for (const auto & name : required_columns)
prewhere_info->prewhere_actions->tryRestoreColumn(name);
prewhere_info->prewhere_actions.tryRestoreColumn(name);
/// Add physical columns required by prewhere actions.
for (const auto & column : required_columns_from_prewhere)
@ -2323,7 +2326,7 @@ std::optional<UInt64> InterpreterSelectQuery::getTrivialCount(UInt64 max_paralle
if (analysis_result.hasPrewhere())
{
auto & prewhere_info = analysis_result.prewhere_info;
filter_nodes.push_back(&prewhere_info->prewhere_actions->findInOutputs(prewhere_info->prewhere_column_name));
filter_nodes.push_back(&prewhere_info->prewhere_actions.findInOutputs(prewhere_info->prewhere_column_name));
if (prewhere_info->row_level_filter)
filter_nodes.push_back(&prewhere_info->row_level_filter->findInOutputs(prewhere_info->row_level_column_name));
@ -2337,7 +2340,7 @@ std::optional<UInt64> InterpreterSelectQuery::getTrivialCount(UInt64 max_paralle
if (!filter_actions_dag)
return {};
return storage->totalRowsByPartitionPredicate(filter_actions_dag, context);
return storage->totalRowsByPartitionPredicate(*filter_actions_dag, context);
}
}
@ -2587,7 +2590,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc
/// Aliases in table declaration.
if (processing_stage == QueryProcessingStage::FetchColumns && alias_actions)
{
auto table_aliases = std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), alias_actions);
auto table_aliases = std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), alias_actions->clone());
table_aliases->setStepDescription("Add table aliases");
query_plan.addStep(std::move(table_aliases));
}
@ -2597,7 +2600,7 @@ void InterpreterSelectQuery::executeWhere(QueryPlan & query_plan, const ActionsA
{
auto dag = expression->dag.clone();
if (expression->project_input)
dag->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header);
dag.appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header);
auto where_step = std::make_unique<FilterStep>(
query_plan.getCurrentDataStream(), std::move(dag), getSelectQuery().where()->getColumnName(), remove_filter);
@ -2771,7 +2774,7 @@ void InterpreterSelectQuery::executeHaving(QueryPlan & query_plan, const Actions
{
auto dag = expression->dag.clone();
if (expression->project_input)
dag->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header);
dag.appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header);
auto having_step
= std::make_unique<FilterStep>(query_plan.getCurrentDataStream(), std::move(dag), getSelectQuery().having()->getColumnName(), remove_filter);
@ -2784,7 +2787,7 @@ void InterpreterSelectQuery::executeHaving(QueryPlan & query_plan, const Actions
void InterpreterSelectQuery::executeTotalsAndHaving(
QueryPlan & query_plan, bool has_having, const ActionsAndProjectInputsFlagPtr & expression, bool remove_filter, bool overflow_row, bool final)
{
ActionsDAGPtr dag;
std::optional<ActionsDAG> dag;
if (expression)
{
dag = expression->dag.clone();
@ -2838,7 +2841,7 @@ void InterpreterSelectQuery::executeExpression(QueryPlan & query_plan, const Act
auto dag = expression->dag.clone();
if (expression->project_input)
dag->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header);
dag.appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header);
auto expression_step = std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), std::move(dag));

View File

@ -239,7 +239,7 @@ private:
Block source_header;
/// Actions to calculate ALIAS if required.
ActionsDAGPtr alias_actions;
std::optional<ActionsDAG> alias_actions;
/// The subquery interpreter, if the subquery
std::unique_ptr<InterpreterSelectWithUnionQuery> interpreter_subquery;

View File

@ -1,9 +1,7 @@
#include <Storages/IStorage.h>
#include <Parsers/TablePropertiesQueriesASTs.h>
#include <Parsers/formatAST.h>
#include <Processors/Sources/SourceFromSingleChunk.h>
#include <QueryPipeline/BlockIO.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeString.h>
#include <Columns/ColumnString.h>
#include <Common/typeid_cast.h>

View File

@ -1200,7 +1200,7 @@ void MutationsInterpreter::Source::read(
const auto & names = first_stage.filter_column_names;
size_t num_filters = names.size();
ActionsDAGPtr filter;
std::optional<ActionsDAG> filter;
if (!first_stage.filter_column_names.empty())
{
ActionsDAG::NodeRawConstPtrs nodes(num_filters);
@ -1214,7 +1214,7 @@ void MutationsInterpreter::Source::read(
MergeTreeSequentialSourceType::Mutation,
plan, *data, storage_snapshot,
part, required_columns,
apply_deleted_mask_, filter, context_,
apply_deleted_mask_, std::move(filter), context_,
getLogger("MutationsInterpreter"));
}
else
@ -1283,17 +1283,17 @@ QueryPipelineBuilder MutationsInterpreter::addStreamsForLaterStages(const std::v
{
auto dag = step->actions()->dag.clone();
if (step->actions()->project_input)
dag->appendInputsForUnusedColumns(plan.getCurrentDataStream().header);
dag.appendInputsForUnusedColumns(plan.getCurrentDataStream().header);
/// Execute DELETEs.
plan.addStep(std::make_unique<FilterStep>(plan.getCurrentDataStream(), dag, stage.filter_column_names[i], false));
plan.addStep(std::make_unique<FilterStep>(plan.getCurrentDataStream(), std::move(dag), stage.filter_column_names[i], false));
}
else
{
auto dag = step->actions()->dag.clone();
if (step->actions()->project_input)
dag->appendInputsForUnusedColumns(plan.getCurrentDataStream().header);
dag.appendInputsForUnusedColumns(plan.getCurrentDataStream().header);
/// Execute UPDATE or final projection.
plan.addStep(std::make_unique<ExpressionStep>(plan.getCurrentDataStream(), dag));
plan.addStep(std::make_unique<ExpressionStep>(plan.getCurrentDataStream(), std::move(dag)));
}
}

View File

@ -3,6 +3,7 @@
#include <Parsers/IAST.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTDataType.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTAlterQuery.h>
#include <Parsers/ASTCreateQuery.h>
@ -16,7 +17,6 @@
#include <Parsers/MySQL/ASTCreateDefines.h>
#include <DataTypes/DataTypeFactory.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeNullable.h>
#include <Parsers/MySQL/ASTDeclareIndex.h>
#include <Common/quoteString.h>
@ -29,6 +29,7 @@
#include <Interpreters/applyTableOverride.h>
#include <Storages/IStorage.h>
namespace DB
{
@ -95,22 +96,22 @@ NamesAndTypesList getColumnsList(const ASTExpressionList * columns_definition)
}
ASTPtr data_type = declare_column->data_type;
auto * data_type_function = data_type->as<ASTFunction>();
auto * data_type_node = data_type->as<ASTDataType>();
if (data_type_function)
if (data_type_node)
{
String type_name_upper = Poco::toUpper(data_type_function->name);
String type_name_upper = Poco::toUpper(data_type_node->name);
if (is_unsigned)
{
/// For example(in MySQL): CREATE TABLE test(column_name INT NOT NULL ... UNSIGNED)
if (type_name_upper.find("INT") != String::npos && !endsWith(type_name_upper, "SIGNED")
&& !endsWith(type_name_upper, "UNSIGNED"))
data_type_function->name = type_name_upper + " UNSIGNED";
data_type_node->name = type_name_upper + " UNSIGNED";
}
if (type_name_upper == "SET")
data_type_function->arguments.reset();
data_type_node->arguments.reset();
/// Transforms MySQL ENUM's list of strings to ClickHouse string-integer pairs
/// For example ENUM('a', 'b', 'c') -> ENUM('a'=1, 'b'=2, 'c'=3)
@ -119,7 +120,7 @@ NamesAndTypesList getColumnsList(const ASTExpressionList * columns_definition)
if (type_name_upper.find("ENUM") != String::npos)
{
UInt16 i = 0;
for (ASTPtr & child : data_type_function->arguments->children)
for (ASTPtr & child : data_type_node->arguments->children)
{
auto new_child = std::make_shared<ASTFunction>();
new_child->name = "equals";
@ -133,10 +134,10 @@ NamesAndTypesList getColumnsList(const ASTExpressionList * columns_definition)
}
if (type_name_upper == "DATE")
data_type_function->name = "Date32";
data_type_node->name = "Date32";
}
if (is_nullable)
data_type = makeASTFunction("Nullable", data_type);
data_type = makeASTDataType("Nullable", data_type);
columns_name_and_type.emplace_back(declare_column->name, DataTypeFactory::instance().get(data_type));
}
@ -156,7 +157,7 @@ static ColumnsDescription createColumnsDescription(const NamesAndTypesList & col
/// (see git blame for details).
auto column_name_and_type = columns_name_and_type.begin();
const auto * declare_column_ast = columns_definition->children.begin();
for (; column_name_and_type != columns_name_and_type.end(); column_name_and_type++, declare_column_ast++)
for (; column_name_and_type != columns_name_and_type.end(); ++column_name_and_type, ++declare_column_ast)
{
const auto & declare_column = (*declare_column_ast)->as<MySQLParser::ASTDeclareColumn>();
String comment;
@ -182,7 +183,7 @@ static NamesAndTypesList getNames(const ASTFunction & expr, ContextPtr context,
ASTPtr temp_ast = expr.clone();
auto syntax = TreeRewriter(context).analyze(temp_ast, columns);
auto required_columns = ExpressionAnalyzer(temp_ast, syntax, context).getActionsDAG(false)->getRequiredColumns();
auto required_columns = ExpressionAnalyzer(temp_ast, syntax, context).getActionsDAG(false).getRequiredColumns();
return required_columns;
}
@ -482,7 +483,7 @@ ASTs InterpreterCreateImpl::getRewrittenQueries(
{
auto column_declaration = std::make_shared<ASTColumnDeclaration>();
column_declaration->name = name;
column_declaration->type = makeASTFunction(type);
column_declaration->type = makeASTDataType(type);
column_declaration->default_specifier = "MATERIALIZED";
column_declaration->default_expression = std::make_shared<ASTLiteral>(default_value);
column_declaration->children.emplace_back(column_declaration->type);

View File

@ -2,12 +2,10 @@
#include <gtest/gtest.h>
#include <Parsers/IAST.h>
#include <Parsers/queryToString.h>
#include <Parsers/ASTExternalDDLQuery.h>
#include <Parsers/ParserExternalDDLQuery.h>
#include <Parsers/parseQuery.h>
#include <Interpreters/Context.h>
#include <Interpreters/MySQL/InterpretersMySQLDDLQuery.h>
#include <Common/tests/gtest_global_context.h>
#include <Common/tests/gtest_global_register.h>
@ -26,8 +24,8 @@ static inline ASTPtr tryRewrittenCreateQuery(const String & query, ContextPtr co
context, "test_database", "test_database")[0];
}
static const char MATERIALIZEDMYSQL_TABLE_COLUMNS[] = ", `_sign` Int8() MATERIALIZED 1"
", `_version` UInt64() MATERIALIZED 1"
static const char MATERIALIZEDMYSQL_TABLE_COLUMNS[] = ", `_sign` Int8 MATERIALIZED 1"
", `_version` UInt64 MATERIALIZED 1"
", INDEX _version _version TYPE minmax GRANULARITY 1";
TEST(MySQLCreateRewritten, ColumnsDataType)

View File

@ -462,19 +462,19 @@ static void makeColumnNameUnique(const ColumnsWithTypeAndName & source_columns,
}
}
static ActionsDAGPtr createWrapWithTupleActions(
static std::optional<ActionsDAG> createWrapWithTupleActions(
const ColumnsWithTypeAndName & source_columns,
std::unordered_set<std::string_view> && column_names_to_wrap,
NameToNameMap & new_names)
{
if (column_names_to_wrap.empty())
return nullptr;
return {};
auto actions_dag = std::make_shared<ActionsDAG>(source_columns);
ActionsDAG actions_dag(source_columns);
FunctionOverloadResolverPtr func_builder = std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionTuple>());
for (const auto * input_node : actions_dag->getInputs())
for (const auto * input_node : actions_dag.getInputs())
{
const auto & column_name = input_node->result_name;
auto it = column_names_to_wrap.find(column_name);
@ -485,9 +485,9 @@ static ActionsDAGPtr createWrapWithTupleActions(
String node_name = "__wrapNullsafe(" + column_name + ")";
makeColumnNameUnique(source_columns, node_name);
const auto & dst_node = actions_dag->addFunction(func_builder, {input_node}, node_name);
const auto & dst_node = actions_dag.addFunction(func_builder, {input_node}, node_name);
new_names[column_name] = dst_node.result_name;
actions_dag->addOrReplaceInOutputs(dst_node);
actions_dag.addOrReplaceInOutputs(dst_node);
}
if (!column_names_to_wrap.empty())
@ -537,21 +537,23 @@ std::pair<NameSet, NameSet> TableJoin::getKeysForNullSafeComparion(const Columns
return {left_keys_to_wrap, right_keys_to_wrap};
}
static void mergeDags(ActionsDAGPtr & result_dag, ActionsDAGPtr && new_dag)
static void mergeDags(std::optional<ActionsDAG> & result_dag, std::optional<ActionsDAG> && new_dag)
{
if (!new_dag)
return;
if (result_dag)
result_dag->mergeInplace(std::move(*new_dag));
else
result_dag = std::move(new_dag);
}
std::pair<ActionsDAGPtr, ActionsDAGPtr>
std::pair<std::optional<ActionsDAG>, std::optional<ActionsDAG>>
TableJoin::createConvertingActions(
const ColumnsWithTypeAndName & left_sample_columns,
const ColumnsWithTypeAndName & right_sample_columns)
{
ActionsDAGPtr left_dag = nullptr;
ActionsDAGPtr right_dag = nullptr;
std::optional<ActionsDAG> left_dag;
std::optional<ActionsDAG> right_dag;
/** If the types are not equal, we need to convert them to a common type.
* Example:
* SELECT * FROM t1 JOIN t2 ON t1.a = t2.b
@ -616,7 +618,7 @@ TableJoin::createConvertingActions(
mergeDags(right_dag, std::move(new_right_dag));
}
return {left_dag, right_dag};
return {std::move(left_dag), std::move(right_dag)};
}
template <typename LeftNamesAndTypes, typename RightNamesAndTypes>
@ -693,7 +695,7 @@ void TableJoin::inferJoinKeyCommonType(const LeftNamesAndTypes & left, const Rig
}
}
static ActionsDAGPtr changeKeyTypes(const ColumnsWithTypeAndName & cols_src,
static std::optional<ActionsDAG> changeKeyTypes(const ColumnsWithTypeAndName & cols_src,
const TableJoin::NameToTypeMap & type_mapping,
bool add_new_cols,
NameToNameMap & key_column_rename)
@ -710,7 +712,7 @@ static ActionsDAGPtr changeKeyTypes(const ColumnsWithTypeAndName & cols_src,
}
}
if (!has_some_to_do)
return nullptr;
return {};
return ActionsDAG::makeConvertingActions(
/* source= */ cols_src,
@ -721,7 +723,7 @@ static ActionsDAGPtr changeKeyTypes(const ColumnsWithTypeAndName & cols_src,
/* new_names= */ &key_column_rename);
}
static ActionsDAGPtr changeTypesToNullable(
static std::optional<ActionsDAG> changeTypesToNullable(
const ColumnsWithTypeAndName & cols_src,
const NameSet & exception_cols)
{
@ -737,7 +739,7 @@ static ActionsDAGPtr changeTypesToNullable(
}
if (!has_some_to_do)
return nullptr;
return {};
return ActionsDAG::makeConvertingActions(
/* source= */ cols_src,
@ -748,29 +750,29 @@ static ActionsDAGPtr changeTypesToNullable(
/* new_names= */ nullptr);
}
ActionsDAGPtr TableJoin::applyKeyConvertToTable(
std::optional<ActionsDAG> TableJoin::applyKeyConvertToTable(
const ColumnsWithTypeAndName & cols_src,
const NameToTypeMap & type_mapping,
JoinTableSide table_side,
NameToNameMap & key_column_rename)
{
if (type_mapping.empty())
return nullptr;
return {};
/// Create DAG to convert key columns
ActionsDAGPtr convert_dag = changeKeyTypes(cols_src, type_mapping, !hasUsing(), key_column_rename);
auto convert_dag = changeKeyTypes(cols_src, type_mapping, !hasUsing(), key_column_rename);
applyRename(table_side, key_column_rename);
return convert_dag;
}
ActionsDAGPtr TableJoin::applyNullsafeWrapper(
std::optional<ActionsDAG> TableJoin::applyNullsafeWrapper(
const ColumnsWithTypeAndName & cols_src,
const NameSet & columns_for_nullsafe_comparison,
JoinTableSide table_side,
NameToNameMap & key_column_rename)
{
if (columns_for_nullsafe_comparison.empty())
return nullptr;
return {};
std::unordered_set<std::string_view> column_names_to_wrap;
for (const auto & name : columns_for_nullsafe_comparison)
@ -784,7 +786,7 @@ ActionsDAGPtr TableJoin::applyNullsafeWrapper(
}
/// Create DAG to wrap keys with tuple for null-safe comparison
ActionsDAGPtr null_safe_wrap_dag = createWrapWithTupleActions(cols_src, std::move(column_names_to_wrap), key_column_rename);
auto null_safe_wrap_dag = createWrapWithTupleActions(cols_src, std::move(column_names_to_wrap), key_column_rename);
for (auto & clause : clauses)
{
for (size_t i : clause.nullsafe_compare_key_indexes)
@ -799,7 +801,7 @@ ActionsDAGPtr TableJoin::applyNullsafeWrapper(
return null_safe_wrap_dag;
}
ActionsDAGPtr TableJoin::applyJoinUseNullsConversion(
std::optional<ActionsDAG> TableJoin::applyJoinUseNullsConversion(
const ColumnsWithTypeAndName & cols_src,
const NameToNameMap & key_column_rename)
{
@ -809,8 +811,7 @@ ActionsDAGPtr TableJoin::applyJoinUseNullsConversion(
exclude_columns.insert(it.second);
/// Create DAG to make columns nullable if needed
ActionsDAGPtr add_nullable_dag = changeTypesToNullable(cols_src, exclude_columns);
return add_nullable_dag;
return changeTypesToNullable(cols_src, exclude_columns);
}
void TableJoin::setStorageJoin(std::shared_ptr<const IKeyValueEntity> storage)
@ -957,7 +958,7 @@ bool TableJoin::allowParallelHashJoin() const
return true;
}
ActionsDAGPtr TableJoin::createJoinedBlockActions(ContextPtr context) const
ActionsDAG TableJoin::createJoinedBlockActions(ContextPtr context) const
{
ASTPtr expression_list = rightKeysList();
auto syntax_result = TreeRewriter(context).analyze(expression_list, columnsFromJoinedTable());

View File

@ -201,19 +201,19 @@ private:
Names requiredJoinedNames() const;
/// Create converting actions and change key column names if required
ActionsDAGPtr applyKeyConvertToTable(
std::optional<ActionsDAG> applyKeyConvertToTable(
const ColumnsWithTypeAndName & cols_src,
const NameToTypeMap & type_mapping,
JoinTableSide table_side,
NameToNameMap & key_column_rename);
ActionsDAGPtr applyNullsafeWrapper(
std::optional<ActionsDAG> applyNullsafeWrapper(
const ColumnsWithTypeAndName & cols_src,
const NameSet & columns_for_nullsafe_comparison,
JoinTableSide table_side,
NameToNameMap & key_column_rename);
ActionsDAGPtr applyJoinUseNullsConversion(
std::optional<ActionsDAG> applyJoinUseNullsConversion(
const ColumnsWithTypeAndName & cols_src,
const NameToNameMap & key_column_rename);
@ -263,7 +263,7 @@ public:
TemporaryDataOnDiskScopePtr getTempDataOnDisk() { return tmp_data; }
ActionsDAGPtr createJoinedBlockActions(ContextPtr context) const;
ActionsDAG createJoinedBlockActions(ContextPtr context) const;
const std::vector<JoinAlgorithm> & getEnabledJoinAlgorithms() const { return join_algorithm; }
@ -378,7 +378,7 @@ public:
/// Calculate converting actions, rename key columns in required
/// For `USING` join we will convert key columns inplace and affect into types in the result table
/// For `JOIN ON` we will create new columns with converted keys to join by.
std::pair<ActionsDAGPtr, ActionsDAGPtr>
std::pair<std::optional<ActionsDAG>, std::optional<ActionsDAG>>
createConvertingActions(
const ColumnsWithTypeAndName & left_sample_columns,
const ColumnsWithTypeAndName & right_sample_columns);

View File

@ -14,7 +14,6 @@ namespace DB
class ASTFunction;
class ActionsDAG;
using ActionsDAGPtr = std::shared_ptr<ActionsDAG>;
struct WindowFunctionDescription
{
@ -93,8 +92,8 @@ struct WindowDescription
// then by ORDER BY. This field holds this combined sort order.
SortDescription full_sort_description;
std::vector<ActionsDAGPtr> partition_by_actions;
std::vector<ActionsDAGPtr> order_by_actions;
std::vector<std::shared_ptr<const ActionsDAG>> partition_by_actions;
std::vector<std::shared_ptr<const ActionsDAG>> order_by_actions;
WindowFrame frame;

View File

@ -14,15 +14,15 @@
namespace DB
{
ActionsDAGPtr addMissingDefaults(
ActionsDAG addMissingDefaults(
const Block & header,
const NamesAndTypesList & required_columns,
const ColumnsDescription & columns,
ContextPtr context,
bool null_as_default)
{
auto actions = std::make_shared<ActionsDAG>(header.getColumnsWithTypeAndName());
auto & index = actions->getOutputs();
ActionsDAG actions(header.getColumnsWithTypeAndName());
auto & index = actions.getOutputs();
/// For missing columns of nested structure, you need to create not a column of empty arrays, but a column of arrays of correct lengths.
/// First, remember the offset columns for all arrays in the block.
@ -40,7 +40,7 @@ ActionsDAGPtr addMissingDefaults(
if (group.empty())
group.push_back(nullptr);
group.push_back(actions->getInputs()[i]);
group.push_back(actions.getInputs()[i]);
}
}
@ -62,11 +62,11 @@ ActionsDAGPtr addMissingDefaults(
{
const auto & nested_type = array_type->getNestedType();
ColumnPtr nested_column = nested_type->createColumnConstWithDefaultValue(0);
const auto & constant = actions->addColumn({nested_column, nested_type, column.name});
const auto & constant = actions.addColumn({nested_column, nested_type, column.name});
auto & group = nested_groups[offsets_name];
group[0] = &constant;
index.push_back(&actions->addFunction(func_builder_replicate, group, constant.result_name));
index.push_back(&actions.addFunction(func_builder_replicate, group, constant.result_name));
continue;
}
@ -75,17 +75,17 @@ ActionsDAGPtr addMissingDefaults(
* it can be full (or the interpreter may decide that it is constant everywhere).
*/
auto new_column = column.type->createColumnConstWithDefaultValue(0);
const auto * col = &actions->addColumn({new_column, column.type, column.name});
index.push_back(&actions->materializeNode(*col));
const auto * col = &actions.addColumn({new_column, column.type, column.name});
index.push_back(&actions.materializeNode(*col));
}
/// Computes explicitly specified values by default and materialized columns.
if (auto dag = evaluateMissingDefaults(actions->getResultColumns(), required_columns, columns, context, true, null_as_default))
actions = ActionsDAG::merge(std::move(*actions), std::move(*dag));
if (auto dag = evaluateMissingDefaults(actions.getResultColumns(), required_columns, columns, context, true, null_as_default))
actions = ActionsDAG::merge(std::move(actions), std::move(*dag));
/// Removes unused columns and reorders result.
actions->removeUnusedActions(required_columns.getNames(), false);
actions->addMaterializingOutputActions();
actions.removeUnusedActions(required_columns.getNames(), false);
actions.addMaterializingOutputActions();
return actions;
}

View File

@ -2,11 +2,6 @@
#include <Interpreters/Context_fwd.h>
#include <memory>
#include <string>
#include <unordered_map>
namespace DB
{
@ -15,7 +10,6 @@ class NamesAndTypesList;
class ColumnsDescription;
class ActionsDAG;
using ActionsDAGPtr = std::shared_ptr<ActionsDAG>;
/** Adds three types of columns into block
* 1. Columns, that are missed inside request, but present in table without defaults (missed columns)
@ -24,7 +18,7 @@ using ActionsDAGPtr = std::shared_ptr<ActionsDAG>;
* Also can substitute NULL with DEFAULT value in case of INSERT SELECT query (null_as_default) if according setting is 1.
* All three types of columns are materialized (not constants).
*/
ActionsDAGPtr addMissingDefaults(
ActionsDAG addMissingDefaults(
const Block & header, const NamesAndTypesList & required_columns,
const ColumnsDescription & columns, ContextPtr context, bool null_as_default = false);
}

View File

@ -91,7 +91,7 @@ std::optional<EvaluateConstantExpressionResult> evaluateConstantExpressionImpl(c
ColumnPtr result_column;
DataTypePtr result_type;
String result_name = ast->getColumnName();
for (const auto & action_node : actions->getOutputs())
for (const auto & action_node : actions.getOutputs())
{
if ((action_node->result_name == result_name) && action_node->column)
{
@ -679,9 +679,9 @@ std::optional<ConstantVariants> evaluateExpressionOverConstantCondition(
size_t max_elements)
{
auto inverted_dag = KeyCondition::cloneASTWithInversionPushDown({predicate}, context);
auto matches = matchTrees(expr, *inverted_dag, false);
auto matches = matchTrees(expr, inverted_dag, false);
auto predicates = analyze(inverted_dag->getOutputs().at(0), matches, context, max_elements);
auto predicates = analyze(inverted_dag.getOutputs().at(0), matches, context, max_elements);
if (!predicates)
return {};

View File

@ -1,11 +1,14 @@
#pragma once
#include "Access/ContextAccess.h"
#include "Interpreters/Context.h"
#include <Access/ContextAccess.h>
#include <Interpreters/Context.h>
#include <Core/Settings.h>
namespace DB
{
struct SecretHidingFormatSettings
{
// We can't store const Context& as there's a dangerous usage {.ctx = *getContext()}
@ -24,4 +27,5 @@ inline String format(const SecretHidingFormatSettings & settings)
return settings.query.formatWithPossiblyHidingSensitiveData(settings.max_length, settings.one_line, show_secrets);
}
}

View File

@ -152,22 +152,20 @@ ASTPtr convertRequiredExpressions(Block & block, const NamesAndTypesList & requi
return conversion_expr_list;
}
ActionsDAGPtr createExpressions(
std::optional<ActionsDAG> createExpressions(
const Block & header,
ASTPtr expr_list,
bool save_unneeded_columns,
ContextPtr context)
{
if (!expr_list)
return nullptr;
return {};
auto syntax_result = TreeRewriter(context).analyze(expr_list, header.getNamesAndTypesList());
auto expression_analyzer = ExpressionAnalyzer{expr_list, syntax_result, context};
auto dag = std::make_shared<ActionsDAG>(header.getNamesAndTypesList());
ActionsDAG dag(header.getNamesAndTypesList());
auto actions = expression_analyzer.getActionsDAG(true, !save_unneeded_columns);
dag = ActionsDAG::merge(std::move(*dag), std::move(*actions));
return dag;
return ActionsDAG::merge(std::move(dag), std::move(actions));
}
}
@ -180,7 +178,7 @@ void performRequiredConversions(Block & block, const NamesAndTypesList & require
if (auto dag = createExpressions(block, conversion_expr_list, true, context))
{
auto expression = std::make_shared<ExpressionActions>(std::move(dag), ExpressionActionsSettings::fromContext(context));
auto expression = std::make_shared<ExpressionActions>(std::move(*dag), ExpressionActionsSettings::fromContext(context));
expression->execute(block);
}
}
@ -195,7 +193,7 @@ bool needConvertAnyNullToDefault(const Block & header, const NamesAndTypesList &
return false;
}
ActionsDAGPtr evaluateMissingDefaults(
std::optional<ActionsDAG> evaluateMissingDefaults(
const Block & header,
const NamesAndTypesList & required_columns,
const ColumnsDescription & columns,
@ -204,7 +202,7 @@ ActionsDAGPtr evaluateMissingDefaults(
bool null_as_default)
{
if (!columns.hasDefaults() && (!null_as_default || !needConvertAnyNullToDefault(header, required_columns, columns)))
return nullptr;
return {};
ASTPtr expr_list = defaultRequiredExpressions(header, required_columns, columns, null_as_default);
return createExpressions(header, expr_list, save_unneeded_columns, context);

View File

@ -5,9 +5,6 @@
#include <Common/COW.h>
#include <memory>
#include <string>
#include <unordered_map>
namespace DB
{
@ -24,12 +21,11 @@ struct StorageInMemoryMetadata;
using StorageMetadataPtr = std::shared_ptr<const StorageInMemoryMetadata>;
class ActionsDAG;
using ActionsDAGPtr = std::shared_ptr<ActionsDAG>;
/// Create actions which adds missing defaults to block according to required_columns using columns description
/// or substitute NULL into DEFAULT value in case of INSERT SELECT query (null_as_default) if according setting is 1.
/// Return nullptr if no actions required.
ActionsDAGPtr evaluateMissingDefaults(
std::optional<ActionsDAG> evaluateMissingDefaults(
const Block & header,
const NamesAndTypesList & required_columns,
const ColumnsDescription & columns,

View File

@ -1,8 +1,6 @@
#include <Parsers/ASTColumnDeclaration.h>
#include <Common/quoteString.h>
#include <IO/Operators.h>
#include <Parsers/ASTLiteral.h>
#include <DataTypes/DataTypeFactory.h>
namespace DB
@ -15,8 +13,6 @@ ASTPtr ASTColumnDeclaration::clone() const
if (type)
{
// Type may be an ASTFunction (e.g. `create table t (a Decimal(9,0))`),
// so we have to clone it properly as well.
res->type = type->clone();
res->children.push_back(res->type);
}

View File

@ -0,0 +1,57 @@
#include <Parsers/ASTDataType.h>
#include <Common/SipHash.h>
#include <IO/Operators.h>
namespace DB
{
String ASTDataType::getID(char delim) const
{
return "DataType" + (delim + name);
}
ASTPtr ASTDataType::clone() const
{
auto res = std::make_shared<ASTDataType>(*this);
res->children.clear();
if (arguments)
{
res->arguments = arguments->clone();
res->children.push_back(res->arguments);
}
return res;
}
void ASTDataType::updateTreeHashImpl(SipHash & hash_state, bool) const
{
hash_state.update(name.size());
hash_state.update(name);
/// Children are hashed automatically.
}
void ASTDataType::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
settings.ostr << (settings.hilite ? hilite_function : "") << name;
if (arguments && !arguments->children.empty())
{
settings.ostr << '(' << (settings.hilite ? hilite_none : "");
for (size_t i = 0, size = arguments->children.size(); i < size; ++i)
{
if (i != 0)
settings.ostr << ", ";
arguments->children[i]->formatImpl(settings, state, frame);
}
settings.ostr << (settings.hilite ? hilite_function : "") << ')';
}
settings.ostr << (settings.hilite ? hilite_none : "");
}
}

38
src/Parsers/ASTDataType.h Normal file
View File

@ -0,0 +1,38 @@
#pragma once
#include <Parsers/ASTExpressionList.h>
namespace DB
{
/// AST for data types, e.g. UInt8 or Tuple(x UInt8, y Enum(a = 1))
class ASTDataType : public IAST
{
public:
String name;
ASTPtr arguments;
String getID(char delim) const override;
ASTPtr clone() const override;
void updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const override;
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
};
template <typename... Args>
std::shared_ptr<ASTDataType> makeASTDataType(const String & name, Args &&... args)
{
auto data_type = std::make_shared<ASTDataType>();
data_type->name = name;
if constexpr (sizeof...(args))
{
data_type->arguments = std::make_shared<ASTExpressionList>();
data_type->children.push_back(data_type->arguments);
data_type->arguments->children = { std::forward<Args>(args)... };
}
return data_type;
}
}

View File

@ -46,7 +46,7 @@ public:
NullsAction nulls_action = NullsAction::EMPTY;
/// do not print empty parentheses if there are no args - compatibility with new AST for data types and engine names.
/// do not print empty parentheses if there are no args - compatibility with engine names.
bool no_empty_args = false;
/// Specifies where this function-like expression is used.

View File

@ -9,8 +9,8 @@
#include <Common/PODArray.h>
#include <Common/StringUtils.h>
#include <Common/typeid_cast.h>
#include "Parsers/CommonParsers.h"
#include <Parsers/CommonParsers.h>
#include <Parsers/DumpASTNode.h>
#include <Parsers/ASTAsterisk.h>
#include <Parsers/ASTCollation.h>
@ -725,7 +725,6 @@ bool ParserStatisticsType::parseImpl(Pos & pos, ASTPtr & node, Expected & expect
function_node->name = "STATISTICS";
function_node->arguments = stat_type;
function_node->children.push_back(function_node->arguments);
node = function_node;
return true;
}

View File

@ -2388,6 +2388,24 @@ bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
}
}
bool ParserExpressionWithOptionalArguments::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserIdentifier id_p;
ParserFunction func_p;
if (ParserFunction(false, false).parse(pos, node, expected))
return true;
if (ParserIdentifier().parse(pos, node, expected))
{
node = makeASTFunction(node->as<ASTIdentifier>()->name());
node->as<ASTFunction &>().no_empty_args = true;
return true;
}
return false;
}
const std::vector<std::pair<std::string_view, Operator>> ParserExpressionImpl::operators_table
{
{"->", Operator("lambda", 1, 2, OperatorType::Lambda)},

View File

@ -144,6 +144,16 @@ protected:
};
/** Similar to ParserFunction (and yields ASTFunction), but can also parse identifiers without braces.
*/
class ParserExpressionWithOptionalArguments : public IParserBase
{
protected:
const char * getName() const override { return "expression with optional parameters"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
/** An expression with an infix binary left-associative operator.
* For example, a + b - c + d.
*/

View File

@ -271,16 +271,15 @@ public:
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown element in AST: {}", getID());
}
// Secrets are displayed regarding show_secrets, then SensitiveDataMasker is applied.
// You can use Interpreters/formatWithPossiblyHidingSecrets.h for convenience.
/// Secrets are displayed regarding show_secrets, then SensitiveDataMasker is applied.
/// You can use Interpreters/formatWithPossiblyHidingSecrets.h for convenience.
String formatWithPossiblyHidingSensitiveData(size_t max_length, bool one_line, bool show_secrets) const;
/*
* formatForLogging and formatForErrorMessage always hide secrets. This inconsistent
* behaviour is due to the fact such functions are called from Client which knows nothing about
* access rights and settings. Moreover, the only use case for displaying secrets are backups,
* and backup tools use only direct input and ignore logs and error messages.
*/
/** formatForLogging and formatForErrorMessage always hide secrets. This inconsistent
* behaviour is due to the fact such functions are called from Client which knows nothing about
* access rights and settings. Moreover, the only use case for displaying secrets are backups,
* and backup tools use only direct input and ignore logs and error messages.
*/
String formatForLogging(size_t max_length = 0) const
{
return formatWithPossiblyHidingSensitiveData(max_length, true, false);

View File

@ -1,13 +1,14 @@
#include <gtest/gtest.h>
#include <Parsers/parseQuery.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTDataType.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/MySQL/ASTDeclareColumn.h>
#include <Parsers/MySQL/ASTDeclareOption.h>
#include <Parsers/MySQL/ASTDeclareReference.h>
#include <Parsers/MySQL/ASTDeclareConstraint.h>
using namespace DB;
using namespace DB::MySQLParser;
@ -19,8 +20,8 @@ TEST(ParserColumn, AllNonGeneratedColumnOption)
"COLUMN_FORMAT FIXED STORAGE MEMORY REFERENCES tbl_name (col_01) CHECK 1";
ASTPtr ast = parseQuery(p_column, input.data(), input.data() + input.size(), "", 0, 0, 0);
EXPECT_EQ(ast->as<ASTDeclareColumn>()->name, "col_01");
EXPECT_EQ(ast->as<ASTDeclareColumn>()->data_type->as<ASTFunction>()->name, "VARCHAR");
EXPECT_EQ(ast->as<ASTDeclareColumn>()->data_type->as<ASTFunction>()->arguments->children[0]->as<ASTLiteral>()->value.safeGet<UInt64>(), 100);
EXPECT_EQ(ast->as<ASTDeclareColumn>()->data_type->as<ASTDataType>()->name, "VARCHAR");
EXPECT_EQ(ast->as<ASTDeclareColumn>()->data_type->as<ASTDataType>()->arguments->children[0]->as<ASTLiteral>()->value.safeGet<UInt64>(), 100);
ASTDeclareOptions * declare_options = ast->as<ASTDeclareColumn>()->column_options->as<ASTDeclareOptions>();
EXPECT_EQ(declare_options->changes["is_null"]->as<ASTLiteral>()->value.safeGet<UInt64>(), 0);
@ -44,8 +45,8 @@ TEST(ParserColumn, AllGeneratedColumnOption)
"REFERENCES tbl_name (col_01) CHECK 1 GENERATED ALWAYS AS (1) STORED";
ASTPtr ast = parseQuery(p_column, input.data(), input.data() + input.size(), "", 0, 0, 0);
EXPECT_EQ(ast->as<ASTDeclareColumn>()->name, "col_01");
EXPECT_EQ(ast->as<ASTDeclareColumn>()->data_type->as<ASTFunction>()->name, "VARCHAR");
EXPECT_EQ(ast->as<ASTDeclareColumn>()->data_type->as<ASTFunction>()->arguments->children[0]->as<ASTLiteral>()->value.safeGet<UInt64>(), 100);
EXPECT_EQ(ast->as<ASTDeclareColumn>()->data_type->as<ASTDataType>()->name, "VARCHAR");
EXPECT_EQ(ast->as<ASTDeclareColumn>()->data_type->as<ASTDataType>()->arguments->children[0]->as<ASTLiteral>()->value.safeGet<UInt64>(), 100);
ASTDeclareOptions * declare_options = ast->as<ASTDeclareColumn>()->column_options->as<ASTDeclareOptions>();
EXPECT_EQ(declare_options->changes["is_null"]->as<ASTLiteral>()->value.safeGet<UInt64>(), 1);

View File

@ -9,8 +9,6 @@
#include <Parsers/ParserRefreshStrategy.h>
#include <Parsers/ParserSelectWithUnionQuery.h>
#include <Parsers/ParserSetQuery.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTIndexDeclaration.h>
#include <Parsers/ASTAlterQuery.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/parseDatabaseAndTableName.h>

View File

@ -7,9 +7,9 @@
#include <Parsers/ASTLiteral.h>
#include <Parsers/CommonParsers.h>
#include <Parsers/ExpressionListParsers.h>
#include <Parsers/ParserDataType.h>
#include <Parsers/parseDatabaseAndTableName.h>
namespace DB
{
@ -21,7 +21,7 @@ bool ParserCreateIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected
ParserToken close_p(TokenType::ClosingRoundBracket);
ParserOrderByExpressionList order_list_p;
ParserDataType data_type_p;
ParserExpressionWithOptionalArguments type_p;
ParserExpression expression_p;
ParserUnsignedInteger granularity_p;
@ -68,7 +68,7 @@ bool ParserCreateIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected
if (s_type.ignore(pos, expected))
{
if (!data_type_p.parse(pos, type, expected))
if (!type_p.parse(pos, type, expected))
return false;
}

View File

@ -5,6 +5,7 @@
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTForeignKeyDeclaration.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTDataType.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTIndexDeclaration.h>
#include <Parsers/ASTStatisticsDeclaration.h>
@ -76,9 +77,9 @@ bool ParserNestedTable::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
if (!close.ignore(pos, expected))
return false;
auto func = std::make_shared<ASTFunction>();
auto func = std::make_shared<ASTDataType>();
tryGetIdentifierNameInto(name, func->name);
// FIXME(ilezhankin): func->no_empty_args = true; ?
func->arguments = columns;
func->children.push_back(columns);
node = func;
@ -179,7 +180,7 @@ bool ParserIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
ParserKeyword s_granularity(Keyword::GRANULARITY);
ParserIdentifier name_p;
ParserDataType data_type_p;
ParserExpressionWithOptionalArguments type_p;
ParserExpression expression_p;
ParserUnsignedInteger granularity_p;
@ -197,7 +198,7 @@ bool ParserIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
if (!s_type.ignore(pos, expected))
return false;
if (!data_type_p.parse(pos, type, expected))
if (!type_p.parse(pos, type, expected))
return false;
if (s_granularity.ignore(pos, expected))
@ -231,7 +232,7 @@ bool ParserStatisticsDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected &
ParserKeyword s_type(Keyword::TYPE);
ParserList columns_p(std::make_unique<ParserIdentifier>(), std::make_unique<ParserToken>(TokenType::Comma), false);
ParserList types_p(std::make_unique<ParserDataType>(), std::make_unique<ParserToken>(TokenType::Comma), false);
ParserList types_p(std::make_unique<ParserExpressionWithOptionalArguments>(), std::make_unique<ParserToken>(TokenType::Comma), false);
ASTPtr columns;
ASTPtr types;
@ -751,7 +752,7 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
auto * table_id = table->as<ASTTableIdentifier>();
// Shortcut for ATTACH a previously detached table
/// A shortcut for ATTACH a previously detached table.
bool short_attach = attach && !from_path;
if (short_attach && (!pos.isValid() || pos.get().type == TokenType::Semicolon))
{

View File

@ -1,6 +1,7 @@
#pragma once
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTDataType.h>
#include <Parsers/ASTColumnDeclaration.h>
#include <Parsers/ASTIdentifier_fwd.h>
#include <Parsers/ASTLiteral.h>
@ -13,6 +14,7 @@
#include <Parsers/ParserSetQuery.h>
#include <Poco/String.h>
namespace DB
{
@ -101,17 +103,15 @@ class IParserColumnDeclaration : public IParserBase
{
public:
explicit IParserColumnDeclaration(bool require_type_ = true, bool allow_null_modifiers_ = false, bool check_keywords_after_name_ = false)
: require_type(require_type_)
, allow_null_modifiers(allow_null_modifiers_)
, check_keywords_after_name(check_keywords_after_name_)
: require_type(require_type_)
, allow_null_modifiers(allow_null_modifiers_)
, check_keywords_after_name(check_keywords_after_name_)
{
}
void enableCheckTypeKeyword() { check_type_keyword = true; }
protected:
using ASTDeclarePtr = std::shared_ptr<ASTColumnDeclaration>;
const char * getName() const override{ return "column declaration"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
@ -270,9 +270,8 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, ASTPtr & node, E
auto default_function = std::make_shared<ASTFunction>();
default_function->name = "defaultValueOfTypeName";
default_function->arguments = std::make_shared<ASTExpressionList>();
// Ephemeral columns don't really have secrets but we need to format
// into a String, hence the strange call
default_function->arguments->children.emplace_back(std::make_shared<ASTLiteral>(type->as<ASTFunction>()->formatForLogging()));
/// Ephemeral columns don't really have secrets but we need to format into a String, hence the strange call
default_function->arguments->children.emplace_back(std::make_shared<ASTLiteral>(type->as<ASTDataType>()->formatForLogging()));
default_expression = default_function;
}

View File

@ -1,8 +1,8 @@
#include <Parsers/ParserDataType.h>
#include <boost/algorithm/string/case_conv.hpp>
#include <Parsers/ASTDataType.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTIdentifier_fwd.h>
#include <Parsers/CommonParsers.h>
#include <Parsers/ExpressionElementParsers.h>
@ -46,48 +46,6 @@ private:
}
};
/// Wrapper to allow mixed lists of nested and normal types.
/// Parameters are either:
/// - Nested table elements;
/// - Enum element in form of 'a' = 1;
/// - literal;
/// - Dynamic type arguments;
/// - another data type (or identifier);
class ParserDataTypeArgument : public IParserBase
{
public:
explicit ParserDataTypeArgument(std::string_view type_name_) : type_name(type_name_)
{
}
private:
const char * getName() const override { return "data type argument"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override
{
if (type_name == "Dynamic")
{
DynamicArgumentsParser parser;
return parser.parse(pos, node, expected);
}
ParserNestedTable nested_parser;
ParserDataType data_type_parser;
ParserAllCollectionsOfLiterals literal_parser(false);
const char * operators[] = {"=", "equals", nullptr};
ParserLeftAssociativeBinaryOperatorList enum_parser(operators, std::make_unique<ParserLiteral>());
if (pos->type == TokenType::BareWord && std::string_view(pos->begin, pos->size()) == "Nested")
return nested_parser.parse(pos, node, expected);
return enum_parser.parse(pos, node, expected)
|| literal_parser.parse(pos, node, expected)
|| data_type_parser.parse(pos, node, expected);
}
std::string_view type_name;
};
}
bool ParserDataType::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
@ -198,23 +156,102 @@ bool ParserDataType::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
}
}
auto function_node = std::make_shared<ASTFunction>();
function_node->name = type_name;
function_node->no_empty_args = true;
auto data_type_node = std::make_shared<ASTDataType>();
data_type_node->name = type_name;
if (pos->type != TokenType::OpeningRoundBracket)
{
node = function_node;
node = data_type_node;
return true;
}
++pos;
/// Parse optional parameters
ParserList args_parser(std::make_unique<ParserDataTypeArgument>(type_name), std::make_unique<ParserToken>(TokenType::Comma));
ASTPtr expr_list_args;
ASTPtr expr_list_args = std::make_shared<ASTExpressionList>();
/// Allow mixed lists of nested and normal types.
/// Parameters are either:
/// - Nested table elements;
/// - Enum element in form of 'a' = 1;
/// - literal;
/// - Dynamic type arguments;
/// - another data type (or identifier);
size_t arg_num = 0;
bool have_version_of_aggregate_function = false;
while (true)
{
if (arg_num > 0)
{
if (pos->type == TokenType::Comma)
++pos;
else
break;
}
ASTPtr arg;
if (type_name == "Dynamic")
{
DynamicArgumentsParser parser;
parser.parse(pos, arg, expected);
}
else if (type_name == "Nested")
{
ParserNestedTable nested_parser;
nested_parser.parse(pos, arg, expected);
}
else if (type_name == "AggregateFunction" || type_name == "SimpleAggregateFunction")
{
/// This is less trivial.
/// The first optional argument for AggregateFunction is a numeric literal, defining the version.
/// The next argument is the function name, optionally with parameters.
/// Subsequent arguments are data types.
if (arg_num == 0 && type_name == "AggregateFunction")
{
ParserUnsignedInteger version_parser;
if (version_parser.parse(pos, arg, expected))
{
have_version_of_aggregate_function = true;
expr_list_args->children.emplace_back(std::move(arg));
++arg_num;
continue;
}
}
if (arg_num == (have_version_of_aggregate_function ? 1 : 0))
{
ParserFunction function_parser;
ParserIdentifier identifier_parser;
function_parser.parse(pos, arg, expected)
|| identifier_parser.parse(pos, arg, expected);
}
else
{
ParserDataType data_type_parser;
data_type_parser.parse(pos, arg, expected);
}
}
else
{
ParserDataType data_type_parser;
ParserAllCollectionsOfLiterals literal_parser(false);
const char * operators[] = {"=", "equals", nullptr};
ParserLeftAssociativeBinaryOperatorList enum_parser(operators, std::make_unique<ParserLiteral>());
enum_parser.parse(pos, arg, expected)
|| literal_parser.parse(pos, arg, expected)
|| data_type_parser.parse(pos, arg, expected);
}
if (!arg)
break;
expr_list_args->children.emplace_back(std::move(arg));
++arg_num;
}
if (!args_parser.parse(pos, expr_list_args, expected))
return false;
if (pos->type == TokenType::Comma)
// ignore trailing comma inside Nested structures like Tuple(Int, Tuple(Int, String),)
++pos;
@ -222,10 +259,10 @@ bool ParserDataType::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
return false;
++pos;
function_node->arguments = expr_list_args;
function_node->children.push_back(function_node->arguments);
data_type_node->arguments = expr_list_args;
data_type_node->children.push_back(data_type_node->arguments);
node = function_node;
node = data_type_node;
return true;
}

View File

@ -88,16 +88,16 @@ public:
auto column_identifier = planner_context->getGlobalPlannerContext()->createColumnIdentifier(node);
ActionsDAGPtr alias_column_actions_dag = std::make_shared<ActionsDAG>();
ActionsDAG alias_column_actions_dag;
PlannerActionsVisitor actions_visitor(planner_context, false);
auto outputs = actions_visitor.visit(*alias_column_actions_dag, column_node->getExpression());
auto outputs = actions_visitor.visit(alias_column_actions_dag, column_node->getExpression());
if (outputs.size() != 1)
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Expected single output in actions dag for alias column {}. Actual {}", column_node->dumpTree(), outputs.size());
const auto & column_name = column_node->getColumnName();
const auto & alias_node = alias_column_actions_dag->addAlias(*outputs[0], column_name);
alias_column_actions_dag->addOrReplaceInOutputs(alias_node);
table_expression_data.addAliasColumn(column_node->getColumn(), column_identifier, alias_column_actions_dag, select_added_columns);
const auto & alias_node = alias_column_actions_dag.addAlias(*outputs[0], column_name);
alias_column_actions_dag.addOrReplaceInOutputs(alias_node);
table_expression_data.addAliasColumn(column_node->getColumn(), column_identifier, std::move(alias_column_actions_dag), select_added_columns);
}
return;
@ -335,22 +335,22 @@ void collectTableExpressionData(QueryTreeNodePtr & query_node, PlannerContextPtr
collect_source_columns_visitor.setKeepAliasColumns(false);
collect_source_columns_visitor.visit(query_node_typed.getPrewhere());
auto prewhere_actions_dag = std::make_shared<ActionsDAG>();
ActionsDAG prewhere_actions_dag;
QueryTreeNodePtr query_tree_node = query_node_typed.getPrewhere();
PlannerActionsVisitor visitor(planner_context, false /*use_column_identifier_as_action_node_name*/);
auto expression_nodes = visitor.visit(*prewhere_actions_dag, query_tree_node);
auto expression_nodes = visitor.visit(prewhere_actions_dag, query_tree_node);
if (expression_nodes.size() != 1)
throw Exception(ErrorCodes::ILLEGAL_PREWHERE,
"Invalid PREWHERE. Expected single boolean expression. In query {}",
query_node->formatASTForErrorMessage());
prewhere_actions_dag->getOutputs().push_back(expression_nodes.back());
prewhere_actions_dag.getOutputs().push_back(expression_nodes.back());
for (const auto & prewhere_input_node : prewhere_actions_dag->getInputs())
for (const auto & prewhere_input_node : prewhere_actions_dag.getInputs())
if (required_column_names_without_prewhere.contains(prewhere_input_node->result_name))
prewhere_actions_dag->getOutputs().push_back(prewhere_input_node);
prewhere_actions_dag.getOutputs().push_back(prewhere_input_node);
table_expression_data.setPrewhereFilterActions(std::move(prewhere_actions_dag));
}

View File

@ -12,7 +12,6 @@
#include <Functions/FunctionFactory.h>
#include <Functions/CastOverloadResolver.h>
#include <Functions/indexHint.h>
#include <QueryPipeline/Pipe.h>
#include <Processors/Sources/SourceFromSingleChunk.h>
@ -216,9 +215,11 @@ FiltersForTableExpressionMap collectFiltersForAnalysis(const QueryTreeNodePtr &
if (!read_from_dummy)
continue;
auto filter_actions = read_from_dummy->getFilterActionsDAG();
const auto & table_node = dummy_storage_to_table.at(&read_from_dummy->getStorage());
res[table_node] = FiltersForTableExpression{std::move(filter_actions), read_from_dummy->getPrewhereInfo()};
if (auto filter_actions = read_from_dummy->detachFilterActionsDAG())
{
const auto & table_node = dummy_storage_to_table.at(&read_from_dummy->getStorage());
res[table_node] = FiltersForTableExpression{std::move(filter_actions), read_from_dummy->getPrewhereInfo()};
}
}
return res;
@ -332,34 +333,34 @@ public:
};
void addExpressionStep(QueryPlan & query_plan,
const ActionsAndProjectInputsFlagPtr & expression_actions,
ActionsAndProjectInputsFlagPtr & expression_actions,
const std::string & step_description,
std::vector<ActionsDAGPtr> & result_actions_to_execute)
UsefulSets & useful_sets)
{
auto actions = expression_actions->dag.clone();
auto actions = std::move(expression_actions->dag);
if (expression_actions->project_input)
actions->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header);
actions.appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header);
result_actions_to_execute.push_back(actions);
auto expression_step = std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), actions);
auto expression_step = std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), std::move(actions));
appendSetsFromActionsDAG(expression_step->getExpression(), useful_sets);
expression_step->setStepDescription(step_description);
query_plan.addStep(std::move(expression_step));
}
void addFilterStep(QueryPlan & query_plan,
const FilterAnalysisResult & filter_analysis_result,
FilterAnalysisResult & filter_analysis_result,
const std::string & step_description,
std::vector<ActionsDAGPtr> & result_actions_to_execute)
UsefulSets & useful_sets)
{
auto actions = filter_analysis_result.filter_actions->dag.clone();
auto actions = std::move(filter_analysis_result.filter_actions->dag);
if (filter_analysis_result.filter_actions->project_input)
actions->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header);
actions.appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header);
result_actions_to_execute.push_back(actions);
auto where_step = std::make_unique<FilterStep>(query_plan.getCurrentDataStream(),
actions,
std::move(actions),
filter_analysis_result.filter_column_name,
filter_analysis_result.remove_filter_column);
appendSetsFromActionsDAG(where_step->getExpression(), useful_sets);
where_step->setStepDescription(step_description);
query_plan.addStep(std::move(where_step));
}
@ -543,39 +544,41 @@ void addMergingAggregatedStep(QueryPlan & query_plan,
}
void addTotalsHavingStep(QueryPlan & query_plan,
const PlannerExpressionsAnalysisResult & expression_analysis_result,
PlannerExpressionsAnalysisResult & expression_analysis_result,
const QueryAnalysisResult & query_analysis_result,
const PlannerContextPtr & planner_context,
const QueryNode & query_node,
std::vector<ActionsDAGPtr> & result_actions_to_execute)
UsefulSets & useful_sets)
{
const auto & query_context = planner_context->getQueryContext();
const auto & settings = query_context->getSettingsRef();
const auto & aggregation_analysis_result = expression_analysis_result.getAggregation();
const auto & having_analysis_result = expression_analysis_result.getHaving();
auto & aggregation_analysis_result = expression_analysis_result.getAggregation();
auto & having_analysis_result = expression_analysis_result.getHaving();
bool need_finalize = !query_node.isGroupByWithRollup() && !query_node.isGroupByWithCube();
ActionsDAGPtr actions;
std::optional<ActionsDAG> actions;
if (having_analysis_result.filter_actions)
{
actions = having_analysis_result.filter_actions->dag.clone();
actions = std::move(having_analysis_result.filter_actions->dag);
if (having_analysis_result.filter_actions->project_input)
actions->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header);
result_actions_to_execute.push_back(actions);
}
auto totals_having_step = std::make_unique<TotalsHavingStep>(
query_plan.getCurrentDataStream(),
aggregation_analysis_result.aggregate_descriptions,
query_analysis_result.aggregate_overflow_row,
actions,
std::move(actions),
having_analysis_result.filter_column_name,
having_analysis_result.remove_filter_column,
settings.totals_mode,
settings.totals_auto_threshold,
need_finalize);
if (having_analysis_result.filter_actions)
appendSetsFromActionsDAG(*totals_having_step->getActions(), useful_sets);
query_plan.addStep(std::move(totals_having_step));
}
@ -717,13 +720,13 @@ void addWithFillStepIfNeeded(QueryPlan & query_plan,
if (query_node.hasInterpolate())
{
auto interpolate_actions_dag = std::make_shared<ActionsDAG>();
ActionsDAG interpolate_actions_dag;
auto query_plan_columns = query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName();
for (auto & query_plan_column : query_plan_columns)
{
/// INTERPOLATE actions dag input columns must be non constant
query_plan_column.column = nullptr;
interpolate_actions_dag->addInput(query_plan_column);
interpolate_actions_dag.addInput(query_plan_column);
}
auto & interpolate_list_node = query_node.getInterpolate()->as<ListNode &>();
@ -731,12 +734,12 @@ void addWithFillStepIfNeeded(QueryPlan & query_plan,
if (interpolate_list_nodes.empty())
{
for (const auto * input_node : interpolate_actions_dag->getInputs())
for (const auto * input_node : interpolate_actions_dag.getInputs())
{
if (column_names_with_fill.contains(input_node->result_name))
continue;
interpolate_actions_dag->getOutputs().push_back(input_node);
interpolate_actions_dag.getOutputs().push_back(input_node);
}
}
else
@ -746,12 +749,12 @@ void addWithFillStepIfNeeded(QueryPlan & query_plan,
auto & interpolate_node_typed = interpolate_node->as<InterpolateNode &>();
PlannerActionsVisitor planner_actions_visitor(planner_context);
auto expression_to_interpolate_expression_nodes = planner_actions_visitor.visit(*interpolate_actions_dag,
auto expression_to_interpolate_expression_nodes = planner_actions_visitor.visit(interpolate_actions_dag,
interpolate_node_typed.getExpression());
if (expression_to_interpolate_expression_nodes.size() != 1)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expression to interpolate expected to have single action node");
auto interpolate_expression_nodes = planner_actions_visitor.visit(*interpolate_actions_dag,
auto interpolate_expression_nodes = planner_actions_visitor.visit(interpolate_actions_dag,
interpolate_node_typed.getInterpolateExpression());
if (interpolate_expression_nodes.size() != 1)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Interpolate expression expected to have single action node");
@ -762,16 +765,16 @@ void addWithFillStepIfNeeded(QueryPlan & query_plan,
const auto * interpolate_expression = interpolate_expression_nodes[0];
if (!interpolate_expression->result_type->equals(*expression_to_interpolate->result_type))
{
interpolate_expression = &interpolate_actions_dag->addCast(*interpolate_expression,
interpolate_expression = &interpolate_actions_dag.addCast(*interpolate_expression,
expression_to_interpolate->result_type,
interpolate_expression->result_name);
}
const auto * alias_node = &interpolate_actions_dag->addAlias(*interpolate_expression, expression_to_interpolate_name);
interpolate_actions_dag->getOutputs().push_back(alias_node);
const auto * alias_node = &interpolate_actions_dag.addAlias(*interpolate_expression, expression_to_interpolate_name);
interpolate_actions_dag.getOutputs().push_back(alias_node);
}
interpolate_actions_dag->removeUnusedActions();
interpolate_actions_dag.removeUnusedActions();
}
Aliases empty_aliases;
@ -883,12 +886,12 @@ bool addPreliminaryLimitOptimizationStepIfNeeded(QueryPlan & query_plan,
* WINDOW functions.
*/
void addPreliminarySortOrDistinctOrLimitStepsIfNeeded(QueryPlan & query_plan,
const PlannerExpressionsAnalysisResult & expressions_analysis_result,
PlannerExpressionsAnalysisResult & expressions_analysis_result,
const QueryAnalysisResult & query_analysis_result,
const PlannerContextPtr & planner_context,
const PlannerQueryProcessingInfo & query_processing_info,
const QueryTreeNodePtr & query_tree,
std::vector<ActionsDAGPtr> & result_actions_to_execute)
UsefulSets & useful_sets)
{
const auto & query_node = query_tree->as<QueryNode &>();
@ -919,8 +922,8 @@ void addPreliminarySortOrDistinctOrLimitStepsIfNeeded(QueryPlan & query_plan,
if (expressions_analysis_result.hasLimitBy())
{
const auto & limit_by_analysis_result = expressions_analysis_result.getLimitBy();
addExpressionStep(query_plan, limit_by_analysis_result.before_limit_by_actions, "Before LIMIT BY", result_actions_to_execute);
auto & limit_by_analysis_result = expressions_analysis_result.getLimitBy();
addExpressionStep(query_plan, limit_by_analysis_result.before_limit_by_actions, "Before LIMIT BY", useful_sets);
addLimitByStep(query_plan, limit_by_analysis_result, query_node);
}
@ -930,12 +933,12 @@ void addPreliminarySortOrDistinctOrLimitStepsIfNeeded(QueryPlan & query_plan,
void addWindowSteps(QueryPlan & query_plan,
const PlannerContextPtr & planner_context,
const WindowAnalysisResult & window_analysis_result)
WindowAnalysisResult & window_analysis_result)
{
const auto & query_context = planner_context->getQueryContext();
const auto & settings = query_context->getSettingsRef();
auto window_descriptions = window_analysis_result.window_descriptions;
auto & window_descriptions = window_analysis_result.window_descriptions;
sortWindowDescriptions(window_descriptions);
size_t window_descriptions_size = window_descriptions.size();
@ -1057,47 +1060,15 @@ void addOffsetStep(QueryPlan & query_plan, const QueryAnalysisResult & query_ana
}
}
void collectSetsFromActionsDAG(const ActionsDAGPtr & dag, std::unordered_set<const FutureSet *> & useful_sets)
{
for (const auto & node : dag->getNodes())
{
if (node.column)
{
const IColumn * column = node.column.get();
if (const auto * column_const = typeid_cast<const ColumnConst *>(column))
column = &column_const->getDataColumn();
if (const auto * column_set = typeid_cast<const ColumnSet *>(column))
useful_sets.insert(column_set->getData().get());
}
if (node.type == ActionsDAG::ActionType::FUNCTION && node.function_base->getName() == "indexHint")
{
ActionsDAG::NodeRawConstPtrs children;
if (const auto * adaptor = typeid_cast<const FunctionToFunctionBaseAdaptor *>(node.function_base.get()))
{
if (const auto * index_hint = typeid_cast<const FunctionIndexHint *>(adaptor->getFunction().get()))
{
collectSetsFromActionsDAG(index_hint->getActions(), useful_sets);
}
}
}
}
}
void addBuildSubqueriesForSetsStepIfNeeded(
QueryPlan & query_plan,
const SelectQueryOptions & select_query_options,
const PlannerContextPtr & planner_context,
const std::vector<ActionsDAGPtr> & result_actions_to_execute)
const UsefulSets & useful_sets)
{
auto subqueries = planner_context->getPreparedSets().getSubqueries();
std::unordered_set<const FutureSet *> useful_sets;
for (const auto & actions_to_execute : result_actions_to_execute)
collectSetsFromActionsDAG(actions_to_execute, useful_sets);
auto predicate = [&useful_sets](const auto & set) { return !useful_sets.contains(set.get()); };
auto predicate = [&useful_sets](const auto & set) { return !useful_sets.contains(set); };
auto it = std::remove_if(subqueries.begin(), subqueries.end(), std::move(predicate));
subqueries.erase(it, subqueries.end());
@ -1159,11 +1130,11 @@ void addAdditionalFilterStepIfNeeded(QueryPlan & query_plan,
auto fake_table_expression = std::make_shared<TableNode>(std::move(storage), query_context);
auto filter_info = buildFilterInfo(additional_result_filter_ast, fake_table_expression, planner_context, std::move(fake_name_set));
if (!filter_info.actions || !query_plan.isInitialized())
if (!query_plan.isInitialized())
return;
auto filter_step = std::make_unique<FilterStep>(query_plan.getCurrentDataStream(),
filter_info.actions,
std::move(filter_info.actions),
filter_info.column_name,
filter_info.do_remove_column);
filter_step->setStepDescription("additional result filter");
@ -1443,7 +1414,7 @@ void Planner::buildPlanForQueryNode()
checkStoragesSupportTransactions(planner_context);
const auto & table_filters = planner_context->getGlobalPlannerContext()->filters_for_table_expressions;
if (!select_query_options.only_analyze && !table_filters.empty()) // && top_level)
if (!select_query_options.only_analyze && !table_filters.empty())
{
for (auto & [table_node, table_expression_data] : planner_context->getTableExpressionNodeToData())
{
@ -1451,7 +1422,7 @@ void Planner::buildPlanForQueryNode()
if (it != table_filters.end())
{
const auto & filters = it->second;
table_expression_data.setFilterActions(filters.filter_actions);
table_expression_data.setFilterActions(filters.filter_actions->clone());
table_expression_data.setPrewhereInfo(filters.prewhere_info);
}
}
@ -1542,15 +1513,15 @@ void Planner::buildPlanForQueryNode()
planner_context,
query_processing_info);
std::vector<ActionsDAGPtr> result_actions_to_execute = std::move(join_tree_query_plan.actions_dags);
auto useful_sets = std::move(join_tree_query_plan.useful_sets);
for (auto & [_, table_expression_data] : planner_context->getTableExpressionNodeToData())
{
if (table_expression_data.getPrewhereFilterActions())
result_actions_to_execute.push_back(table_expression_data.getPrewhereFilterActions());
appendSetsFromActionsDAG(*table_expression_data.getPrewhereFilterActions(), useful_sets);
if (table_expression_data.getRowLevelFilterActions())
result_actions_to_execute.push_back(table_expression_data.getRowLevelFilterActions());
appendSetsFromActionsDAG(*table_expression_data.getRowLevelFilterActions(), useful_sets);
}
if (query_processing_info.isIntermediateStage())
@ -1561,7 +1532,7 @@ void Planner::buildPlanForQueryNode()
planner_context,
query_processing_info,
query_tree,
result_actions_to_execute);
useful_sets);
if (expression_analysis_result.hasAggregation())
{
@ -1573,13 +1544,13 @@ void Planner::buildPlanForQueryNode()
if (query_processing_info.isFirstStage())
{
if (expression_analysis_result.hasWhere())
addFilterStep(query_plan, expression_analysis_result.getWhere(), "WHERE", result_actions_to_execute);
addFilterStep(query_plan, expression_analysis_result.getWhere(), "WHERE", useful_sets);
if (expression_analysis_result.hasAggregation())
{
const auto & aggregation_analysis_result = expression_analysis_result.getAggregation();
auto & aggregation_analysis_result = expression_analysis_result.getAggregation();
if (aggregation_analysis_result.before_aggregation_actions)
addExpressionStep(query_plan, aggregation_analysis_result.before_aggregation_actions, "Before GROUP BY", result_actions_to_execute);
addExpressionStep(query_plan, aggregation_analysis_result.before_aggregation_actions, "Before GROUP BY", useful_sets);
addAggregationStep(query_plan, aggregation_analysis_result, query_analysis_result, planner_context, select_query_info);
}
@ -1596,9 +1567,9 @@ void Planner::buildPlanForQueryNode()
* window functions, we can't execute ORDER BY and DISTINCT
* now, on shard (first_stage).
*/
const auto & window_analysis_result = expression_analysis_result.getWindow();
auto & window_analysis_result = expression_analysis_result.getWindow();
if (window_analysis_result.before_window_actions)
addExpressionStep(query_plan, window_analysis_result.before_window_actions, "Before WINDOW", result_actions_to_execute);
addExpressionStep(query_plan, window_analysis_result.before_window_actions, "Before WINDOW", useful_sets);
}
else
{
@ -1606,8 +1577,8 @@ void Planner::buildPlanForQueryNode()
* Projection expressions, preliminary DISTINCT and before ORDER BY expressions
* now, on shards (first_stage).
*/
const auto & projection_analysis_result = expression_analysis_result.getProjection();
addExpressionStep(query_plan, projection_analysis_result.projection_actions, "Projection", result_actions_to_execute);
auto & projection_analysis_result = expression_analysis_result.getProjection();
addExpressionStep(query_plan, projection_analysis_result.projection_actions, "Projection", useful_sets);
if (query_node.isDistinct())
{
@ -1622,8 +1593,8 @@ void Planner::buildPlanForQueryNode()
if (expression_analysis_result.hasSort())
{
const auto & sort_analysis_result = expression_analysis_result.getSort();
addExpressionStep(query_plan, sort_analysis_result.before_order_by_actions, "Before ORDER BY", result_actions_to_execute);
auto & sort_analysis_result = expression_analysis_result.getSort();
addExpressionStep(query_plan, sort_analysis_result.before_order_by_actions, "Before ORDER BY", useful_sets);
}
}
}
@ -1634,7 +1605,7 @@ void Planner::buildPlanForQueryNode()
planner_context,
query_processing_info,
query_tree,
result_actions_to_execute);
useful_sets);
}
if (query_processing_info.isSecondStage() || query_processing_info.isFromAggregationState())
@ -1656,14 +1627,14 @@ void Planner::buildPlanForQueryNode()
if (query_node.isGroupByWithTotals())
{
addTotalsHavingStep(query_plan, expression_analysis_result, query_analysis_result, planner_context, query_node, result_actions_to_execute);
addTotalsHavingStep(query_plan, expression_analysis_result, query_analysis_result, planner_context, query_node, useful_sets);
having_executed = true;
}
addCubeOrRollupStepIfNeeded(query_plan, aggregation_analysis_result, query_analysis_result, planner_context, select_query_info, query_node);
if (!having_executed && expression_analysis_result.hasHaving())
addFilterStep(query_plan, expression_analysis_result.getHaving(), "HAVING", result_actions_to_execute);
addFilterStep(query_plan, expression_analysis_result.getHaving(), "HAVING", useful_sets);
}
if (query_processing_info.isFromAggregationState())
@ -1676,18 +1647,18 @@ void Planner::buildPlanForQueryNode()
{
if (expression_analysis_result.hasWindow())
{
const auto & window_analysis_result = expression_analysis_result.getWindow();
auto & window_analysis_result = expression_analysis_result.getWindow();
if (expression_analysis_result.hasAggregation())
addExpressionStep(query_plan, window_analysis_result.before_window_actions, "Before window functions", result_actions_to_execute);
addExpressionStep(query_plan, window_analysis_result.before_window_actions, "Before window functions", useful_sets);
addWindowSteps(query_plan, planner_context, window_analysis_result);
}
if (expression_analysis_result.hasQualify())
addFilterStep(query_plan, expression_analysis_result.getQualify(), "QUALIFY", result_actions_to_execute);
addFilterStep(query_plan, expression_analysis_result.getQualify(), "QUALIFY", useful_sets);
const auto & projection_analysis_result = expression_analysis_result.getProjection();
addExpressionStep(query_plan, projection_analysis_result.projection_actions, "Projection", result_actions_to_execute);
auto & projection_analysis_result = expression_analysis_result.getProjection();
addExpressionStep(query_plan, projection_analysis_result.projection_actions, "Projection", useful_sets);
if (query_node.isDistinct())
{
@ -1702,8 +1673,8 @@ void Planner::buildPlanForQueryNode()
if (expression_analysis_result.hasSort())
{
const auto & sort_analysis_result = expression_analysis_result.getSort();
addExpressionStep(query_plan, sort_analysis_result.before_order_by_actions, "Before ORDER BY", result_actions_to_execute);
auto & sort_analysis_result = expression_analysis_result.getSort();
addExpressionStep(query_plan, sort_analysis_result.before_order_by_actions, "Before ORDER BY", useful_sets);
}
}
else
@ -1755,8 +1726,8 @@ void Planner::buildPlanForQueryNode()
if (!query_processing_info.isFromAggregationState() && expression_analysis_result.hasLimitBy())
{
const auto & limit_by_analysis_result = expression_analysis_result.getLimitBy();
addExpressionStep(query_plan, limit_by_analysis_result.before_limit_by_actions, "Before LIMIT BY", result_actions_to_execute);
auto & limit_by_analysis_result = expression_analysis_result.getLimitBy();
addExpressionStep(query_plan, limit_by_analysis_result.before_limit_by_actions, "Before LIMIT BY", useful_sets);
addLimitByStep(query_plan, limit_by_analysis_result, query_node);
}
@ -1787,8 +1758,8 @@ void Planner::buildPlanForQueryNode()
/// Project names is not done on shards, because initiator will not find columns in blocks
if (!query_processing_info.isToAggregationState())
{
const auto & projection_analysis_result = expression_analysis_result.getProjection();
addExpressionStep(query_plan, projection_analysis_result.project_names_actions, "Project names", result_actions_to_execute);
auto & projection_analysis_result = expression_analysis_result.getProjection();
addExpressionStep(query_plan, projection_analysis_result.project_names_actions, "Project names", useful_sets);
}
// For additional_result_filter setting
@ -1796,7 +1767,7 @@ void Planner::buildPlanForQueryNode()
}
if (!select_query_options.only_analyze)
addBuildSubqueriesForSetsStepIfNeeded(query_plan, select_query_options, planner_context, result_actions_to_execute);
addBuildSubqueriesForSetsStepIfNeeded(query_plan, select_query_options, planner_context, useful_sets);
query_node_to_plan_step_mapping[&query_node] = query_plan.getRootNode();
}

View File

@ -759,15 +759,15 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi
lambda_arguments_names_and_types.emplace_back(lambda_argument_name, std::move(lambda_argument_type));
}
auto lambda_actions_dag = std::make_shared<ActionsDAG>();
actions_stack.emplace_back(*lambda_actions_dag, node);
ActionsDAG lambda_actions_dag;
actions_stack.emplace_back(lambda_actions_dag, node);
auto [lambda_expression_node_name, levels] = visitImpl(lambda_node.getExpression());
lambda_actions_dag->getOutputs().push_back(actions_stack.back().getNodeOrThrow(lambda_expression_node_name));
lambda_actions_dag->removeUnusedActions(Names(1, lambda_expression_node_name));
lambda_actions_dag.getOutputs().push_back(actions_stack.back().getNodeOrThrow(lambda_expression_node_name));
lambda_actions_dag.removeUnusedActions(Names(1, lambda_expression_node_name));
auto expression_actions_settings = ExpressionActionsSettings::fromContext(planner_context->getQueryContext(), CompileExpressions::yes);
auto lambda_actions = std::make_shared<ExpressionActions>(lambda_actions_dag, expression_actions_settings);
auto lambda_actions = std::make_shared<ExpressionActions>(std::move(lambda_actions_dag), expression_actions_settings);
Names captured_column_names;
ActionsDAG::NodeRawConstPtrs lambda_children;
@ -881,14 +881,14 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi
const auto & function_node = node->as<FunctionNode &>();
auto function_node_name = action_node_name_helper.calculateActionNodeName(node);
auto index_hint_actions_dag = std::make_shared<ActionsDAG>();
auto & index_hint_actions_dag_outputs = index_hint_actions_dag->getOutputs();
ActionsDAG index_hint_actions_dag;
auto & index_hint_actions_dag_outputs = index_hint_actions_dag.getOutputs();
std::unordered_set<std::string_view> index_hint_actions_dag_output_node_names;
PlannerActionsVisitor actions_visitor(planner_context);
for (const auto & argument : function_node.getArguments())
{
auto index_hint_argument_expression_dag_nodes = actions_visitor.visit(*index_hint_actions_dag, argument);
auto index_hint_argument_expression_dag_nodes = actions_visitor.visit(index_hint_actions_dag, argument);
for (auto & expression_dag_node : index_hint_argument_expression_dag_nodes)
{

View File

@ -25,7 +25,7 @@ class TableNode;
struct FiltersForTableExpression
{
ActionsDAGPtr filter_actions;
std::optional<ActionsDAG> filter_actions;
PrewhereInfoPtr prewhere_info;
};

View File

@ -64,7 +64,7 @@ public:
: projection_analysis_result(std::move(projection_analysis_result_))
{}
const ProjectionAnalysisResult & getProjection() const
ProjectionAnalysisResult & getProjection()
{
return projection_analysis_result;
}
@ -74,7 +74,7 @@ public:
return where_analysis_result.filter_actions != nullptr;
}
const FilterAnalysisResult & getWhere() const
FilterAnalysisResult & getWhere()
{
return where_analysis_result;
}
@ -89,7 +89,7 @@ public:
return !aggregation_analysis_result.aggregation_keys.empty() || !aggregation_analysis_result.aggregate_descriptions.empty();
}
const AggregationAnalysisResult & getAggregation() const
AggregationAnalysisResult & getAggregation()
{
return aggregation_analysis_result;
}
@ -104,7 +104,7 @@ public:
return having_analysis_result.filter_actions != nullptr;
}
const FilterAnalysisResult & getHaving() const
FilterAnalysisResult & getHaving()
{
return having_analysis_result;
}
@ -119,7 +119,7 @@ public:
return !window_analysis_result.window_descriptions.empty();
}
const WindowAnalysisResult & getWindow() const
WindowAnalysisResult & getWindow()
{
return window_analysis_result;
}
@ -134,7 +134,7 @@ public:
return qualify_analysis_result.filter_actions != nullptr;
}
const FilterAnalysisResult & getQualify() const
FilterAnalysisResult & getQualify()
{
return qualify_analysis_result;
}
@ -149,7 +149,7 @@ public:
return sort_analysis_result.before_order_by_actions != nullptr;
}
const SortAnalysisResult & getSort() const
SortAnalysisResult & getSort()
{
return sort_analysis_result;
}
@ -164,7 +164,7 @@ public:
return limit_by_analysis_result.before_limit_by_actions != nullptr;
}
const LimitByAnalysisResult & getLimitBy() const
LimitByAnalysisResult & getLimitBy()
{
return limit_by_analysis_result;
}

View File

@ -437,7 +437,7 @@ void updatePrewhereOutputsIfNeeded(SelectQueryInfo & table_expression_query_info
std::unordered_set<const ActionsDAG::Node *> required_output_nodes;
for (const auto * input : prewhere_actions->getInputs())
for (const auto * input : prewhere_actions.getInputs())
{
if (required_columns.contains(input->result_name))
required_output_nodes.insert(input);
@ -446,7 +446,7 @@ void updatePrewhereOutputsIfNeeded(SelectQueryInfo & table_expression_query_info
if (required_output_nodes.empty())
return;
auto & prewhere_outputs = prewhere_actions->getOutputs();
auto & prewhere_outputs = prewhere_actions.getOutputs();
for (const auto & output : prewhere_outputs)
{
auto required_output_node_it = required_output_nodes.find(output);
@ -459,7 +459,7 @@ void updatePrewhereOutputsIfNeeded(SelectQueryInfo & table_expression_query_info
prewhere_outputs.insert(prewhere_outputs.end(), required_output_nodes.begin(), required_output_nodes.end());
}
FilterDAGInfo buildRowPolicyFilterIfNeeded(const StoragePtr & storage,
std::optional<FilterDAGInfo> buildRowPolicyFilterIfNeeded(const StoragePtr & storage,
SelectQueryInfo & table_expression_query_info,
PlannerContextPtr & planner_context,
std::set<std::string> & used_row_policies)
@ -480,7 +480,7 @@ FilterDAGInfo buildRowPolicyFilterIfNeeded(const StoragePtr & storage,
return buildFilterInfo(row_policy_filter->expression, table_expression_query_info.table_expression, planner_context);
}
FilterDAGInfo buildCustomKeyFilterIfNeeded(const StoragePtr & storage,
std::optional<FilterDAGInfo> buildCustomKeyFilterIfNeeded(const StoragePtr & storage,
SelectQueryInfo & table_expression_query_info,
PlannerContextPtr & planner_context)
{
@ -514,7 +514,7 @@ FilterDAGInfo buildCustomKeyFilterIfNeeded(const StoragePtr & storage,
}
/// Apply filters from additional_table_filters setting
FilterDAGInfo buildAdditionalFiltersIfNeeded(const StoragePtr & storage,
std::optional<FilterDAGInfo> buildAdditionalFiltersIfNeeded(const StoragePtr & storage,
const String & table_expression_alias,
SelectQueryInfo & table_expression_query_info,
PlannerContextPtr & planner_context)
@ -590,21 +590,21 @@ UInt64 mainQueryNodeBlockSizeByLimit(const SelectQueryInfo & select_query_info)
}
std::unique_ptr<ExpressionStep> createComputeAliasColumnsStep(
const std::unordered_map<std::string, ActionsDAGPtr> & alias_column_expressions, const DataStream & current_data_stream)
std::unordered_map<std::string, ActionsDAG> & alias_column_expressions, const DataStream & current_data_stream)
{
ActionsDAGPtr merged_alias_columns_actions_dag = std::make_shared<ActionsDAG>(current_data_stream.header.getColumnsWithTypeAndName());
ActionsDAG::NodeRawConstPtrs action_dag_outputs = merged_alias_columns_actions_dag->getInputs();
ActionsDAG merged_alias_columns_actions_dag(current_data_stream.header.getColumnsWithTypeAndName());
ActionsDAG::NodeRawConstPtrs action_dag_outputs = merged_alias_columns_actions_dag.getInputs();
for (const auto & [column_name, alias_column_actions_dag] : alias_column_expressions)
for (auto & [column_name, alias_column_actions_dag] : alias_column_expressions)
{
const auto & current_outputs = alias_column_actions_dag->getOutputs();
const auto & current_outputs = alias_column_actions_dag.getOutputs();
action_dag_outputs.insert(action_dag_outputs.end(), current_outputs.begin(), current_outputs.end());
merged_alias_columns_actions_dag->mergeNodes(std::move(*alias_column_actions_dag));
merged_alias_columns_actions_dag.mergeNodes(std::move(alias_column_actions_dag));
}
for (const auto * output_node : action_dag_outputs)
merged_alias_columns_actions_dag->addOrReplaceInOutputs(*output_node);
merged_alias_columns_actions_dag->removeUnusedActions(false);
merged_alias_columns_actions_dag.addOrReplaceInOutputs(*output_node);
merged_alias_columns_actions_dag.removeUnusedActions(false);
auto alias_column_step = std::make_unique<ExpressionStep>(current_data_stream, std::move(merged_alias_columns_actions_dag));
alias_column_step->setStepDescription("Compute alias columns");
@ -647,7 +647,8 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
auto table_expression_query_info = select_query_info;
table_expression_query_info.table_expression = table_expression;
table_expression_query_info.filter_actions_dag = table_expression_data.getFilterActions();
if (const auto & filter_actions = table_expression_data.getFilterActions())
table_expression_query_info.filter_actions_dag = std::make_shared<const ActionsDAG>(filter_actions->clone());
table_expression_query_info.current_table_chosen_for_reading_with_parallel_replicas
= table_node == planner_context->getGlobalPlannerContext()->parallel_replicas_table;
@ -778,7 +779,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
if (prewhere_actions)
{
prewhere_info = std::make_shared<PrewhereInfo>();
prewhere_info->prewhere_actions = prewhere_actions;
prewhere_info->prewhere_actions = prewhere_actions->clone();
prewhere_info->prewhere_column_name = prewhere_actions->getOutputs().at(0)->result_name;
prewhere_info->remove_prewhere_column = true;
prewhere_info->need_filter = true;
@ -789,11 +790,8 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
const auto & columns_names = table_expression_data.getColumnNames();
std::vector<std::pair<FilterDAGInfo, std::string>> where_filters;
const auto add_filter = [&](const FilterDAGInfo & filter_info, std::string description)
const auto add_filter = [&](FilterDAGInfo & filter_info, std::string description)
{
if (!filter_info.actions)
return;
bool is_final = table_expression_query_info.table_expression_modifiers
&& table_expression_query_info.table_expression_modifiers->hasFinal();
bool optimize_move_to_prewhere
@ -803,46 +801,45 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
if (storage->canMoveConditionsToPrewhere() && optimize_move_to_prewhere && (!supported_prewhere_columns || supported_prewhere_columns->contains(filter_info.column_name)))
{
if (!prewhere_info)
prewhere_info = std::make_shared<PrewhereInfo>();
if (!prewhere_info->prewhere_actions)
{
prewhere_info->prewhere_actions = filter_info.actions;
prewhere_info = std::make_shared<PrewhereInfo>();
prewhere_info->prewhere_actions = std::move(filter_info.actions);
prewhere_info->prewhere_column_name = filter_info.column_name;
prewhere_info->remove_prewhere_column = filter_info.do_remove_column;
prewhere_info->need_filter = true;
}
else if (!prewhere_info->row_level_filter)
{
prewhere_info->row_level_filter = filter_info.actions;
prewhere_info->row_level_filter = std::move(filter_info.actions);
prewhere_info->row_level_column_name = filter_info.column_name;
prewhere_info->need_filter = true;
}
else
{
where_filters.emplace_back(filter_info, std::move(description));
where_filters.emplace_back(std::move(filter_info), std::move(description));
}
}
else
{
where_filters.emplace_back(filter_info, std::move(description));
where_filters.emplace_back(std::move(filter_info), std::move(description));
}
};
auto row_policy_filter_info
= buildRowPolicyFilterIfNeeded(storage, table_expression_query_info, planner_context, used_row_policies);
add_filter(row_policy_filter_info, "Row-level security filter");
if (row_policy_filter_info.actions)
table_expression_data.setRowLevelFilterActions(row_policy_filter_info.actions);
if (row_policy_filter_info)
{
table_expression_data.setRowLevelFilterActions(row_policy_filter_info->actions.clone());
add_filter(*row_policy_filter_info, "Row-level security filter");
}
if (query_context->canUseParallelReplicasCustomKey())
{
if (settings.parallel_replicas_count > 1)
{
auto parallel_replicas_custom_key_filter_info
= buildCustomKeyFilterIfNeeded(storage, table_expression_query_info, planner_context);
add_filter(parallel_replicas_custom_key_filter_info, "Parallel replicas custom key filter");
if (auto parallel_replicas_custom_key_filter_info= buildCustomKeyFilterIfNeeded(storage, table_expression_query_info, planner_context))
add_filter(*parallel_replicas_custom_key_filter_info, "Parallel replicas custom key filter");
}
else if (auto * distributed = typeid_cast<StorageDistributed *>(storage.get());
distributed && query_context->canUseParallelReplicasCustomKeyForCluster(*distributed->getCluster()))
@ -857,9 +854,8 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
}
const auto & table_expression_alias = table_expression->getOriginalAlias();
auto additional_filters_info
= buildAdditionalFiltersIfNeeded(storage, table_expression_alias, table_expression_query_info, planner_context);
add_filter(additional_filters_info, "additional filter");
if (auto additional_filters_info = buildAdditionalFiltersIfNeeded(storage, table_expression_alias, table_expression_query_info, planner_context))
add_filter(*additional_filters_info, "additional filter");
from_stage = storage->getQueryProcessingStage(
query_context, select_query_options.to_stage, storage_snapshot, table_expression_query_info);
@ -997,22 +993,20 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
}
}
const auto & alias_column_expressions = table_expression_data.getAliasColumnExpressions();
auto & alias_column_expressions = table_expression_data.getAliasColumnExpressions();
if (!alias_column_expressions.empty() && query_plan.isInitialized() && from_stage == QueryProcessingStage::FetchColumns)
{
auto alias_column_step = createComputeAliasColumnsStep(alias_column_expressions, query_plan.getCurrentDataStream());
query_plan.addStep(std::move(alias_column_step));
}
for (const auto & filter_info_and_description : where_filters)
for (auto && [filter_info, description] : where_filters)
{
const auto & [filter_info, description] = filter_info_and_description;
if (query_plan.isInitialized() &&
from_stage == QueryProcessingStage::FetchColumns &&
filter_info.actions)
from_stage == QueryProcessingStage::FetchColumns)
{
auto filter_step = std::make_unique<FilterStep>(query_plan.getCurrentDataStream(),
filter_info.actions,
std::move(filter_info.actions),
filter_info.column_name,
filter_info.do_remove_column);
filter_step->setStepDescription(description);
@ -1088,7 +1082,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
query_plan = std::move(subquery_planner).extractQueryPlan();
}
const auto & alias_column_expressions = table_expression_data.getAliasColumnExpressions();
auto & alias_column_expressions = table_expression_data.getAliasColumnExpressions();
if (!alias_column_expressions.empty() && query_plan.isInitialized() && from_stage == QueryProcessingStage::FetchColumns)
{
auto alias_column_step = createComputeAliasColumnsStep(alias_column_expressions, query_plan.getCurrentDataStream());
@ -1103,21 +1097,21 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
if (from_stage == QueryProcessingStage::FetchColumns)
{
auto rename_actions_dag = std::make_shared<ActionsDAG>(query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName());
ActionsDAG rename_actions_dag(query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName());
ActionsDAG::NodeRawConstPtrs updated_actions_dag_outputs;
for (auto & output_node : rename_actions_dag->getOutputs())
for (auto & output_node : rename_actions_dag.getOutputs())
{
const auto * column_identifier = table_expression_data.getColumnIdentifierOrNull(output_node->result_name);
if (!column_identifier)
continue;
updated_actions_dag_outputs.push_back(&rename_actions_dag->addAlias(*output_node, *column_identifier));
updated_actions_dag_outputs.push_back(&rename_actions_dag.addAlias(*output_node, *column_identifier));
}
rename_actions_dag->getOutputs() = std::move(updated_actions_dag_outputs);
rename_actions_dag.getOutputs() = std::move(updated_actions_dag_outputs);
auto rename_step = std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), rename_actions_dag);
auto rename_step = std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), std::move(rename_actions_dag));
rename_step->setStepDescription("Change column names to column identifiers");
query_plan.addStep(std::move(rename_step));
}
@ -1157,9 +1151,9 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
void joinCastPlanColumnsToNullable(QueryPlan & plan_to_add_cast, PlannerContextPtr & planner_context, const FunctionOverloadResolverPtr & to_nullable_function)
{
auto cast_actions_dag = std::make_shared<ActionsDAG>(plan_to_add_cast.getCurrentDataStream().header.getColumnsWithTypeAndName());
ActionsDAG cast_actions_dag(plan_to_add_cast.getCurrentDataStream().header.getColumnsWithTypeAndName());
for (auto & output_node : cast_actions_dag->getOutputs())
for (auto & output_node : cast_actions_dag.getOutputs())
{
if (planner_context->getGlobalPlannerContext()->hasColumnIdentifier(output_node->result_name))
{
@ -1168,11 +1162,11 @@ void joinCastPlanColumnsToNullable(QueryPlan & plan_to_add_cast, PlannerContextP
type_to_check = type_to_check_low_cardinality->getDictionaryType();
if (type_to_check->canBeInsideNullable())
output_node = &cast_actions_dag->addFunction(to_nullable_function, {output_node}, output_node->result_name);
output_node = &cast_actions_dag.addFunction(to_nullable_function, {output_node}, output_node->result_name);
}
}
cast_actions_dag->appendInputsForUnusedColumns(plan_to_add_cast.getCurrentDataStream().header);
cast_actions_dag.appendInputsForUnusedColumns(plan_to_add_cast.getCurrentDataStream().header);
auto cast_join_columns_step = std::make_unique<ExpressionStep>(plan_to_add_cast.getCurrentDataStream(), std::move(cast_actions_dag));
cast_join_columns_step->setStepDescription("Cast JOIN columns to Nullable");
plan_to_add_cast.addStep(std::move(cast_join_columns_step));
@ -1218,14 +1212,16 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
join_table_expression,
planner_context);
join_clauses_and_actions.left_join_expressions_actions->appendInputsForUnusedColumns(left_plan.getCurrentDataStream().header);
auto left_join_expressions_actions_step = std::make_unique<ExpressionStep>(left_plan.getCurrentDataStream(), join_clauses_and_actions.left_join_expressions_actions);
join_clauses_and_actions.left_join_expressions_actions.appendInputsForUnusedColumns(left_plan.getCurrentDataStream().header);
auto left_join_expressions_actions_step = std::make_unique<ExpressionStep>(left_plan.getCurrentDataStream(), std::move(join_clauses_and_actions.left_join_expressions_actions));
left_join_expressions_actions_step->setStepDescription("JOIN actions");
appendSetsFromActionsDAG(left_join_expressions_actions_step->getExpression(), left_join_tree_query_plan.useful_sets);
left_plan.addStep(std::move(left_join_expressions_actions_step));
join_clauses_and_actions.right_join_expressions_actions->appendInputsForUnusedColumns(right_plan.getCurrentDataStream().header);
auto right_join_expressions_actions_step = std::make_unique<ExpressionStep>(right_plan.getCurrentDataStream(), join_clauses_and_actions.right_join_expressions_actions);
join_clauses_and_actions.right_join_expressions_actions.appendInputsForUnusedColumns(right_plan.getCurrentDataStream().header);
auto right_join_expressions_actions_step = std::make_unique<ExpressionStep>(right_plan.getCurrentDataStream(), std::move(join_clauses_and_actions.right_join_expressions_actions));
right_join_expressions_actions_step->setStepDescription("JOIN actions");
appendSetsFromActionsDAG(right_join_expressions_actions_step->getExpression(), right_join_tree_query_plan.useful_sets);
right_plan.addStep(std::move(right_join_expressions_actions_step));
}
@ -1263,19 +1259,19 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
auto join_cast_plan_output_nodes = [&](QueryPlan & plan_to_add_cast, std::unordered_map<std::string, DataTypePtr> & plan_column_name_to_cast_type)
{
auto cast_actions_dag = std::make_shared<ActionsDAG>(plan_to_add_cast.getCurrentDataStream().header.getColumnsWithTypeAndName());
ActionsDAG cast_actions_dag(plan_to_add_cast.getCurrentDataStream().header.getColumnsWithTypeAndName());
for (auto & output_node : cast_actions_dag->getOutputs())
for (auto & output_node : cast_actions_dag.getOutputs())
{
auto it = plan_column_name_to_cast_type.find(output_node->result_name);
if (it == plan_column_name_to_cast_type.end())
continue;
const auto & cast_type = it->second;
output_node = &cast_actions_dag->addCast(*output_node, cast_type, output_node->result_name);
output_node = &cast_actions_dag.addCast(*output_node, cast_type, output_node->result_name);
}
cast_actions_dag->appendInputsForUnusedColumns(plan_to_add_cast.getCurrentDataStream().header);
cast_actions_dag.appendInputsForUnusedColumns(plan_to_add_cast.getCurrentDataStream().header);
auto cast_join_columns_step
= std::make_unique<ExpressionStep>(plan_to_add_cast.getCurrentDataStream(), std::move(cast_actions_dag));
cast_join_columns_step->setStepDescription("Cast JOIN USING columns");
@ -1418,8 +1414,10 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
{
ExpressionActionsPtr & mixed_join_expression = table_join->getMixedJoinExpression();
mixed_join_expression = std::make_shared<ExpressionActions>(
join_clauses_and_actions.mixed_join_expressions_actions,
std::move(*join_clauses_and_actions.mixed_join_expressions_actions),
ExpressionActionsSettings::fromContext(planner_context->getQueryContext()));
appendSetsFromActionsDAG(mixed_join_expression->getActionsDAG(), left_join_tree_query_plan.useful_sets);
}
}
else if (join_node.isUsingJoinExpression())
@ -1465,7 +1463,8 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
auto result_plan = QueryPlan();
if (join_algorithm->isFilled())
bool is_filled_join = join_algorithm->isFilled();
if (is_filled_join)
{
auto filled_join_step = std::make_unique<FilledJoinStep>(
left_plan.getCurrentDataStream(),
@ -1574,12 +1573,12 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
result_plan.unitePlans(std::move(join_step), {std::move(plans)});
}
auto drop_unused_columns_after_join_actions_dag = std::make_shared<ActionsDAG>(result_plan.getCurrentDataStream().header.getColumnsWithTypeAndName());
ActionsDAG drop_unused_columns_after_join_actions_dag(result_plan.getCurrentDataStream().header.getColumnsWithTypeAndName());
ActionsDAG::NodeRawConstPtrs drop_unused_columns_after_join_actions_dag_updated_outputs;
std::unordered_set<std::string_view> drop_unused_columns_after_join_actions_dag_updated_outputs_names;
std::optional<size_t> first_skipped_column_node_index;
auto & drop_unused_columns_after_join_actions_dag_outputs = drop_unused_columns_after_join_actions_dag->getOutputs();
auto & drop_unused_columns_after_join_actions_dag_outputs = drop_unused_columns_after_join_actions_dag.getOutputs();
size_t drop_unused_columns_after_join_actions_dag_outputs_size = drop_unused_columns_after_join_actions_dag_outputs.size();
for (size_t i = 0; i < drop_unused_columns_after_join_actions_dag_outputs_size; ++i)
@ -1618,15 +1617,10 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
for (const auto & right_join_tree_query_plan_row_policy : right_join_tree_query_plan.used_row_policies)
left_join_tree_query_plan.used_row_policies.insert(right_join_tree_query_plan_row_policy);
/// Collect all required actions dags in `left_join_tree_query_plan.actions_dags`
for (auto && action_dag : right_join_tree_query_plan.actions_dags)
left_join_tree_query_plan.actions_dags.emplace_back(action_dag);
if (join_clauses_and_actions.left_join_expressions_actions)
left_join_tree_query_plan.actions_dags.emplace_back(std::move(join_clauses_and_actions.left_join_expressions_actions));
if (join_clauses_and_actions.right_join_expressions_actions)
left_join_tree_query_plan.actions_dags.emplace_back(std::move(join_clauses_and_actions.right_join_expressions_actions));
if (join_clauses_and_actions.mixed_join_expressions_actions)
left_join_tree_query_plan.actions_dags.push_back(join_clauses_and_actions.mixed_join_expressions_actions);
/// Collect all required actions sets in `left_join_tree_query_plan.useful_sets`
if (!is_filled_join)
for (const auto & useful_set : right_join_tree_query_plan.useful_sets)
left_join_tree_query_plan.useful_sets.insert(useful_set);
auto mapping = std::move(left_join_tree_query_plan.query_node_to_plan_step_mapping);
auto & r_mapping = right_join_tree_query_plan.query_node_to_plan_step_mapping;
@ -1636,7 +1630,7 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
.query_plan = std::move(result_plan),
.from_stage = QueryProcessingStage::FetchColumns,
.used_row_policies = std::move(left_join_tree_query_plan.used_row_policies),
.actions_dags = std::move(left_join_tree_query_plan.actions_dags),
.useful_sets = std::move(left_join_tree_query_plan.useful_sets),
.query_node_to_plan_step_mapping = std::move(mapping),
};
}
@ -1656,7 +1650,7 @@ JoinTreeQueryPlan buildQueryPlanForArrayJoinNode(const QueryTreeNodePtr & array_
auto plan = std::move(join_tree_query_plan.query_plan);
auto plan_output_columns = plan.getCurrentDataStream().header.getColumnsWithTypeAndName();
ActionsDAGPtr array_join_action_dag = std::make_shared<ActionsDAG>(plan_output_columns);
ActionsDAG array_join_action_dag(plan_output_columns);
PlannerActionsVisitor actions_visitor(planner_context);
std::unordered_set<std::string> array_join_expressions_output_nodes;
@ -1667,29 +1661,28 @@ JoinTreeQueryPlan buildQueryPlanForArrayJoinNode(const QueryTreeNodePtr & array_
array_join_column_names.insert(array_join_column_identifier);
auto & array_join_expression_column = array_join_expression->as<ColumnNode &>();
auto expression_dag_index_nodes = actions_visitor.visit(*array_join_action_dag, array_join_expression_column.getExpressionOrThrow());
auto expression_dag_index_nodes = actions_visitor.visit(array_join_action_dag, array_join_expression_column.getExpressionOrThrow());
for (auto & expression_dag_index_node : expression_dag_index_nodes)
{
const auto * array_join_column_node = &array_join_action_dag->addAlias(*expression_dag_index_node, array_join_column_identifier);
array_join_action_dag->getOutputs().push_back(array_join_column_node);
const auto * array_join_column_node = &array_join_action_dag.addAlias(*expression_dag_index_node, array_join_column_identifier);
array_join_action_dag.getOutputs().push_back(array_join_column_node);
array_join_expressions_output_nodes.insert(array_join_column_node->result_name);
}
}
array_join_action_dag->appendInputsForUnusedColumns(plan.getCurrentDataStream().header);
array_join_action_dag.appendInputsForUnusedColumns(plan.getCurrentDataStream().header);
join_tree_query_plan.actions_dags.push_back(array_join_action_dag);
auto array_join_actions = std::make_unique<ExpressionStep>(plan.getCurrentDataStream(), array_join_action_dag);
auto array_join_actions = std::make_unique<ExpressionStep>(plan.getCurrentDataStream(), std::move(array_join_action_dag));
array_join_actions->setStepDescription("ARRAY JOIN actions");
appendSetsFromActionsDAG(array_join_actions->getExpression(), join_tree_query_plan.useful_sets);
plan.addStep(std::move(array_join_actions));
auto drop_unused_columns_before_array_join_actions_dag = std::make_shared<ActionsDAG>(plan.getCurrentDataStream().header.getColumnsWithTypeAndName());
ActionsDAG drop_unused_columns_before_array_join_actions_dag(plan.getCurrentDataStream().header.getColumnsWithTypeAndName());
ActionsDAG::NodeRawConstPtrs drop_unused_columns_before_array_join_actions_dag_updated_outputs;
std::unordered_set<std::string_view> drop_unused_columns_before_array_join_actions_dag_updated_outputs_names;
auto & drop_unused_columns_before_array_join_actions_dag_outputs = drop_unused_columns_before_array_join_actions_dag->getOutputs();
auto & drop_unused_columns_before_array_join_actions_dag_outputs = drop_unused_columns_before_array_join_actions_dag.getOutputs();
size_t drop_unused_columns_before_array_join_actions_dag_outputs_size = drop_unused_columns_before_array_join_actions_dag_outputs.size();
for (size_t i = 0; i < drop_unused_columns_before_array_join_actions_dag_outputs_size; ++i)
@ -1723,7 +1716,7 @@ JoinTreeQueryPlan buildQueryPlanForArrayJoinNode(const QueryTreeNodePtr & array_
.query_plan = std::move(plan),
.from_stage = QueryProcessingStage::FetchColumns,
.used_row_policies = std::move(join_tree_query_plan.used_row_policies),
.actions_dags = std::move(join_tree_query_plan.actions_dags),
.useful_sets = std::move(join_tree_query_plan.useful_sets),
.query_node_to_plan_step_mapping = std::move(join_tree_query_plan.query_node_to_plan_step_mapping),
};
}

View File

@ -11,12 +11,14 @@
namespace DB
{
using UsefulSets = std::unordered_set<FutureSetPtr>;
struct JoinTreeQueryPlan
{
QueryPlan query_plan;
QueryProcessingStage::Enum from_stage;
std::set<std::string> used_row_policies{};
std::vector<ActionsDAGPtr> actions_dags{};
UsefulSets useful_sets{};
std::unordered_map<const QueryNode *, const QueryPlan::Node *> query_node_to_plan_step_mapping{};
};

View File

@ -180,13 +180,13 @@ std::set<JoinTableSide> extractJoinTableSidesFromExpression(//const ActionsDAG::
}
const ActionsDAG::Node * appendExpression(
ActionsDAGPtr & dag,
ActionsDAG & dag,
const QueryTreeNodePtr & expression,
const PlannerContextPtr & planner_context,
const JoinNode & join_node)
{
PlannerActionsVisitor join_expression_visitor(planner_context);
auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(*dag, expression);
auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(dag, expression);
if (join_expression_dag_node_raw_pointers.size() != 1)
throw Exception(ErrorCodes::LOGICAL_ERROR,
"JOIN {} ON clause contains multiple expressions",
@ -196,9 +196,9 @@ const ActionsDAG::Node * appendExpression(
}
void buildJoinClause(
ActionsDAGPtr & left_dag,
ActionsDAGPtr & right_dag,
ActionsDAGPtr & mixed_dag,
ActionsDAG & left_dag,
ActionsDAG & right_dag,
ActionsDAG & mixed_dag,
const PlannerContextPtr & planner_context,
const QueryTreeNodePtr & join_expression,
const TableExpressionSet & left_table_expressions,
@ -379,8 +379,8 @@ JoinClausesAndActions buildJoinClausesAndActions(
const JoinNode & join_node,
const PlannerContextPtr & planner_context)
{
ActionsDAGPtr left_join_actions = std::make_shared<ActionsDAG>(left_table_expression_columns);
ActionsDAGPtr right_join_actions = std::make_shared<ActionsDAG>(right_table_expression_columns);
ActionsDAG left_join_actions(left_table_expression_columns);
ActionsDAG right_join_actions(right_table_expression_columns);
ColumnsWithTypeAndName mixed_table_expression_columns;
for (const auto & left_column : left_table_expression_columns)
{
@ -390,7 +390,7 @@ JoinClausesAndActions buildJoinClausesAndActions(
{
mixed_table_expression_columns.push_back(right_column);
}
ActionsDAGPtr mixed_join_actions = std::make_shared<ActionsDAG>(mixed_table_expression_columns);
ActionsDAG mixed_join_actions(mixed_table_expression_columns);
/** It is possible to have constant value in JOIN ON section, that we need to ignore during DAG construction.
* If we do not ignore it, this function will be replaced by underlying constant.
@ -501,12 +501,12 @@ JoinClausesAndActions buildJoinClausesAndActions(
{
const ActionsDAG::Node * dag_filter_condition_node = nullptr;
if (left_filter_condition_nodes.size() > 1)
dag_filter_condition_node = &left_join_actions->addFunction(and_function, left_filter_condition_nodes, {});
dag_filter_condition_node = &left_join_actions.addFunction(and_function, left_filter_condition_nodes, {});
else
dag_filter_condition_node = left_filter_condition_nodes[0];
join_clause.getLeftFilterConditionNodes() = {dag_filter_condition_node};
left_join_actions->addOrReplaceInOutputs(*dag_filter_condition_node);
left_join_actions.addOrReplaceInOutputs(*dag_filter_condition_node);
add_necessary_name_if_needed(JoinTableSide::Left, dag_filter_condition_node->result_name);
}
@ -517,12 +517,12 @@ JoinClausesAndActions buildJoinClausesAndActions(
const ActionsDAG::Node * dag_filter_condition_node = nullptr;
if (right_filter_condition_nodes.size() > 1)
dag_filter_condition_node = &right_join_actions->addFunction(and_function, right_filter_condition_nodes, {});
dag_filter_condition_node = &right_join_actions.addFunction(and_function, right_filter_condition_nodes, {});
else
dag_filter_condition_node = right_filter_condition_nodes[0];
join_clause.getRightFilterConditionNodes() = {dag_filter_condition_node};
right_join_actions->addOrReplaceInOutputs(*dag_filter_condition_node);
right_join_actions.addOrReplaceInOutputs(*dag_filter_condition_node);
add_necessary_name_if_needed(JoinTableSide::Right, dag_filter_condition_node->result_name);
}
@ -559,10 +559,10 @@ JoinClausesAndActions buildJoinClausesAndActions(
}
if (!left_key_node->result_type->equals(*common_type))
left_key_node = &left_join_actions->addCast(*left_key_node, common_type, {});
left_key_node = &left_join_actions.addCast(*left_key_node, common_type, {});
if (!right_key_node->result_type->equals(*common_type))
right_key_node = &right_join_actions->addCast(*right_key_node, common_type, {});
right_key_node = &right_join_actions.addCast(*right_key_node, common_type, {});
}
if (join_clause.isNullsafeCompareKey(i) && left_key_node->result_type->isNullable() && right_key_node->result_type->isNullable())
@ -579,24 +579,24 @@ JoinClausesAndActions buildJoinClausesAndActions(
* SELECT * FROM t1 JOIN t2 ON tuple(t1.a) == tuple(t2.b)
*/
auto wrap_nullsafe_function = FunctionFactory::instance().get("tuple", planner_context->getQueryContext());
left_key_node = &left_join_actions->addFunction(wrap_nullsafe_function, {left_key_node}, {});
right_key_node = &right_join_actions->addFunction(wrap_nullsafe_function, {right_key_node}, {});
left_key_node = &left_join_actions.addFunction(wrap_nullsafe_function, {left_key_node}, {});
right_key_node = &right_join_actions.addFunction(wrap_nullsafe_function, {right_key_node}, {});
}
left_join_actions->addOrReplaceInOutputs(*left_key_node);
right_join_actions->addOrReplaceInOutputs(*right_key_node);
left_join_actions.addOrReplaceInOutputs(*left_key_node);
right_join_actions.addOrReplaceInOutputs(*right_key_node);
add_necessary_name_if_needed(JoinTableSide::Left, left_key_node->result_name);
add_necessary_name_if_needed(JoinTableSide::Right, right_key_node->result_name);
}
}
result.left_join_expressions_actions = left_join_actions->clone();
result.left_join_expressions_actions = left_join_actions.clone();
result.left_join_tmp_expression_actions = std::move(left_join_actions);
result.left_join_expressions_actions->removeUnusedActions(join_left_actions_names);
result.right_join_expressions_actions = right_join_actions->clone();
result.left_join_expressions_actions.removeUnusedActions(join_left_actions_names);
result.right_join_expressions_actions = right_join_actions.clone();
result.right_join_tmp_expression_actions = std::move(right_join_actions);
result.right_join_expressions_actions->removeUnusedActions(join_right_actions_names);
result.right_join_expressions_actions.removeUnusedActions(join_right_actions_names);
if (is_inequal_join)
{
@ -604,24 +604,24 @@ JoinClausesAndActions buildJoinClausesAndActions(
/// So, for each column, we recalculate the value of the whole expression from JOIN ON to check if rows should be joined.
if (result.join_clauses.size() > 1)
{
auto mixed_join_expressions_actions = std::make_shared<ActionsDAG>(mixed_table_expression_columns);
ActionsDAG mixed_join_expressions_actions(mixed_table_expression_columns);
PlannerActionsVisitor join_expression_visitor(planner_context);
auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(*mixed_join_expressions_actions, join_expression);
auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(mixed_join_expressions_actions, join_expression);
if (join_expression_dag_node_raw_pointers.size() != 1)
throw Exception(
ErrorCodes::LOGICAL_ERROR, "JOIN {} ON clause contains multiple expressions", join_node.formatASTForErrorMessage());
mixed_join_expressions_actions->addOrReplaceInOutputs(*join_expression_dag_node_raw_pointers[0]);
mixed_join_expressions_actions.addOrReplaceInOutputs(*join_expression_dag_node_raw_pointers[0]);
Names required_names{join_expression_dag_node_raw_pointers[0]->result_name};
mixed_join_expressions_actions->removeUnusedActions(required_names);
result.mixed_join_expressions_actions = mixed_join_expressions_actions;
mixed_join_expressions_actions.removeUnusedActions(required_names);
result.mixed_join_expressions_actions = std::move(mixed_join_expressions_actions);
}
else
{
const auto & join_clause = result.join_clauses.front();
const auto & mixed_filter_condition_nodes = join_clause.getMixedFilterConditionNodes();
auto mixed_join_expressions_actions = ActionsDAG::buildFilterActionsDAG(mixed_filter_condition_nodes, {}, true);
result.mixed_join_expressions_actions = mixed_join_expressions_actions;
result.mixed_join_expressions_actions = std::move(mixed_join_expressions_actions);
}
auto outputs = result.mixed_join_expressions_actions->getOutputs();
if (outputs.size() != 1)

View File

@ -182,15 +182,15 @@ struct JoinClausesAndActions
/// Join clauses. Actions dag nodes point into join_expression_actions.
JoinClauses join_clauses;
/// Whole JOIN ON section expressions
ActionsDAGPtr left_join_tmp_expression_actions;
ActionsDAGPtr right_join_tmp_expression_actions;
ActionsDAG left_join_tmp_expression_actions;
ActionsDAG right_join_tmp_expression_actions;
/// Left join expressions actions
ActionsDAGPtr left_join_expressions_actions;
ActionsDAG left_join_expressions_actions;
/// Right join expressions actions
ActionsDAGPtr right_join_expressions_actions;
ActionsDAG right_join_expressions_actions;
/// Originally used for inequal join. it's the total join expression.
/// If there is no inequal join conditions, it's null.
ActionsDAGPtr mixed_join_expressions_actions;
std::optional<ActionsDAG> mixed_join_expressions_actions;
};
/** Calculate join clauses and actions for JOIN ON section.

View File

@ -73,7 +73,7 @@ public:
}
/// Add alias column
void addAliasColumn(const NameAndTypePair & column, const ColumnIdentifier & column_identifier, ActionsDAGPtr actions_dag, bool is_selected_column = true)
void addAliasColumn(const NameAndTypePair & column, const ColumnIdentifier & column_identifier, ActionsDAG actions_dag, bool is_selected_column = true)
{
alias_column_expressions.emplace(column.name, std::move(actions_dag));
addColumnImpl(column, column_identifier, is_selected_column);
@ -94,7 +94,7 @@ public:
}
/// Get ALIAS columns names mapped to expressions
const std::unordered_map<std::string, ActionsDAGPtr> & getAliasColumnExpressions() const
std::unordered_map<std::string, ActionsDAG> & getAliasColumnExpressions()
{
return alias_column_expressions;
}
@ -211,32 +211,32 @@ public:
is_merge_tree = is_merge_tree_value;
}
const ActionsDAGPtr & getPrewhereFilterActions() const
const std::optional<ActionsDAG> & getPrewhereFilterActions() const
{
return prewhere_filter_actions;
}
void setRowLevelFilterActions(ActionsDAGPtr row_level_filter_actions_value)
void setRowLevelFilterActions(ActionsDAG row_level_filter_actions_value)
{
row_level_filter_actions = std::move(row_level_filter_actions_value);
}
const ActionsDAGPtr & getRowLevelFilterActions() const
const std::optional<ActionsDAG> & getRowLevelFilterActions() const
{
return row_level_filter_actions;
}
void setPrewhereFilterActions(ActionsDAGPtr prewhere_filter_actions_value)
void setPrewhereFilterActions(ActionsDAG prewhere_filter_actions_value)
{
prewhere_filter_actions = std::move(prewhere_filter_actions_value);
}
const ActionsDAGPtr & getFilterActions() const
const std::optional<ActionsDAG> & getFilterActions() const
{
return filter_actions;
}
void setFilterActions(ActionsDAGPtr filter_actions_value)
void setFilterActions(ActionsDAG filter_actions_value)
{
filter_actions = std::move(filter_actions_value);
}
@ -277,7 +277,7 @@ private:
NameSet selected_column_names_set;
/// Expression to calculate ALIAS columns
std::unordered_map<std::string, ActionsDAGPtr> alias_column_expressions;
std::unordered_map<std::string, ActionsDAG> alias_column_expressions;
/// Valid for table, table function, array join, query, union nodes
ColumnNameToColumn column_name_to_column;
@ -289,16 +289,16 @@ private:
ColumnIdentifierToColumnName column_identifier_to_column_name;
/// Valid for table, table function
ActionsDAGPtr filter_actions;
std::optional<ActionsDAG> filter_actions;
/// Valid for table, table function
PrewhereInfoPtr prewhere_info;
/// Valid for table, table function
ActionsDAGPtr prewhere_filter_actions;
std::optional<ActionsDAG> prewhere_filter_actions;
/// Valid for table, table function
ActionsDAGPtr row_level_filter_actions;
std::optional<ActionsDAG> row_level_filter_actions;
/// Is storage remote
bool is_remote = false;

View File

@ -11,10 +11,12 @@
#include <DataTypes/DataTypeNullable.h>
#include <Columns/getLeastSuperColumn.h>
#include <Columns/ColumnSet.h>
#include <IO/WriteBufferFromString.h>
#include <Functions/FunctionFactory.h>
#include <Functions/indexHint.h>
#include <Storages/StorageDummy.h>
@ -442,22 +444,22 @@ FilterDAGInfo buildFilterInfo(QueryTreeNodePtr filter_query_tree,
collectSourceColumns(filter_query_tree, planner_context, false /*keep_alias_columns*/);
collectSets(filter_query_tree, *planner_context);
auto filter_actions_dag = std::make_shared<ActionsDAG>();
ActionsDAG filter_actions_dag;
PlannerActionsVisitor actions_visitor(planner_context, false /*use_column_identifier_as_action_node_name*/);
auto expression_nodes = actions_visitor.visit(*filter_actions_dag, filter_query_tree);
auto expression_nodes = actions_visitor.visit(filter_actions_dag, filter_query_tree);
if (expression_nodes.size() != 1)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Filter actions must return single output node. Actual {}",
expression_nodes.size());
auto & filter_actions_outputs = filter_actions_dag->getOutputs();
auto & filter_actions_outputs = filter_actions_dag.getOutputs();
filter_actions_outputs = std::move(expression_nodes);
std::string filter_node_name = filter_actions_outputs[0]->result_name;
bool remove_filter_column = true;
for (const auto & filter_input_node : filter_actions_dag->getInputs())
for (const auto & filter_input_node : filter_actions_dag.getInputs())
if (table_expression_required_names_without_filter.contains(filter_input_node->result_name))
filter_actions_outputs.push_back(filter_input_node);
@ -477,4 +479,32 @@ ASTPtr parseAdditionalResultFilter(const Settings & settings)
return additional_result_filter_ast;
}
void appendSetsFromActionsDAG(const ActionsDAG & dag, UsefulSets & useful_sets)
{
for (const auto & node : dag.getNodes())
{
if (node.column)
{
const IColumn * column = node.column.get();
if (const auto * column_const = typeid_cast<const ColumnConst *>(column))
column = &column_const->getDataColumn();
if (const auto * column_set = typeid_cast<const ColumnSet *>(column))
useful_sets.insert(column_set->getData());
}
if (node.type == ActionsDAG::ActionType::FUNCTION && node.function_base->getName() == "indexHint")
{
ActionsDAG::NodeRawConstPtrs children;
if (const auto * adaptor = typeid_cast<const FunctionToFunctionBaseAdaptor *>(node.function_base.get()))
{
if (const auto * index_hint = typeid_cast<const FunctionIndexHint *>(adaptor->getFunction().get()))
{
appendSetsFromActionsDAG(index_hint->getActions(), useful_sets);
}
}
}
}
}
}

View File

@ -88,4 +88,7 @@ FilterDAGInfo buildFilterInfo(QueryTreeNodePtr filter_query_tree,
ASTPtr parseAdditionalResultFilter(const Settings & settings);
using UsefulSets = std::unordered_set<FutureSetPtr>;
void appendSetsFromActionsDAG(const ActionsDAG & dag, UsefulSets & useful_sets);
}

View File

@ -625,8 +625,6 @@ void ValuesBlockInputFormat::readSuffix()
skipWhitespaceIfAny(*buf);
if (buf->hasUnreadData())
throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Cannot read data after semicolon");
if (!format_settings.values.allow_data_after_semicolon && !buf->eof())
throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Cannot read data after semicolon (and input_format_values_allow_data_after_semicolon=0)");
return;
}

View File

@ -303,15 +303,15 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B
const auto & header = ports[set_counter]->getHeader();
/// Here we create a DAG which fills missing keys and adds `__grouping_set` column
auto dag = std::make_shared<ActionsDAG>(header.getColumnsWithTypeAndName());
ActionsDAG dag(header.getColumnsWithTypeAndName());
ActionsDAG::NodeRawConstPtrs outputs;
outputs.reserve(output_header.columns() + 1);
auto grouping_col = ColumnConst::create(ColumnUInt64::create(1, set_counter), 0);
const auto * grouping_node = &dag->addColumn(
const auto * grouping_node = &dag.addColumn(
{ColumnPtr(std::move(grouping_col)), std::make_shared<DataTypeUInt64>(), "__grouping_set"});
grouping_node = &dag->materializeNode(*grouping_node);
grouping_node = &dag.materializeNode(*grouping_node);
outputs.push_back(grouping_node);
const auto & missing_columns = grouping_sets_params[set_counter].missing_keys;
@ -332,22 +332,22 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B
column_with_default->finalize();
auto column = ColumnConst::create(std::move(column_with_default), 0);
const auto * node = &dag->addColumn({ColumnPtr(std::move(column)), col.type, col.name});
node = &dag->materializeNode(*node);
const auto * node = &dag.addColumn({ColumnPtr(std::move(column)), col.type, col.name});
node = &dag.materializeNode(*node);
outputs.push_back(node);
}
else
{
const auto * column_node = dag->getOutputs()[header.getPositionByName(col.name)];
const auto * column_node = dag.getOutputs()[header.getPositionByName(col.name)];
if (used_it != used_keys.end() && group_by_use_nulls && column_node->result_type->canBeInsideNullable())
outputs.push_back(&dag->addFunction(to_nullable_function, { column_node }, col.name));
outputs.push_back(&dag.addFunction(to_nullable_function, { column_node }, col.name));
else
outputs.push_back(column_node);
}
}
dag->getOutputs().swap(outputs);
auto expression = std::make_shared<ExpressionActions>(dag, settings.getActionsSettings());
dag.getOutputs().swap(outputs);
auto expression = std::make_shared<ExpressionActions>(std::move(dag), settings.getActionsSettings());
auto transform = std::make_shared<ExpressionTransform>(header, expression);
connect(*ports[set_counter], transform->getInputPort());

View File

@ -36,30 +36,30 @@ CubeStep::CubeStep(const DataStream & input_stream_, Aggregator::Params params_,
ProcessorPtr addGroupingSetForTotals(const Block & header, const Names & keys, bool use_nulls, const BuildQueryPipelineSettings & settings, UInt64 grouping_set_number)
{
auto dag = std::make_shared<ActionsDAG>(header.getColumnsWithTypeAndName());
auto & outputs = dag->getOutputs();
ActionsDAG dag(header.getColumnsWithTypeAndName());
auto & outputs = dag.getOutputs();
if (use_nulls)
{
auto to_nullable = FunctionFactory::instance().get("toNullable", nullptr);
for (const auto & key : keys)
{
const auto * node = dag->getOutputs()[header.getPositionByName(key)];
const auto * node = dag.getOutputs()[header.getPositionByName(key)];
if (node->result_type->canBeInsideNullable())
{
dag->addOrReplaceInOutputs(dag->addFunction(to_nullable, { node }, node->result_name));
dag.addOrReplaceInOutputs(dag.addFunction(to_nullable, { node }, node->result_name));
}
}
}
auto grouping_col = ColumnUInt64::create(1, grouping_set_number);
const auto * grouping_node = &dag->addColumn(
const auto * grouping_node = &dag.addColumn(
{ColumnPtr(std::move(grouping_col)), std::make_shared<DataTypeUInt64>(), "__grouping_set"});
grouping_node = &dag->materializeNode(*grouping_node);
grouping_node = &dag.materializeNode(*grouping_node);
outputs.insert(outputs.begin(), grouping_node);
auto expression = std::make_shared<ExpressionActions>(dag, settings.getActionsSettings());
auto expression = std::make_shared<ExpressionActions>(std::move(dag), settings.getActionsSettings());
return std::make_shared<ExpressionTransform>(header, expression);
}

View File

@ -33,7 +33,7 @@ void addConvertingActions(QueryPlan & plan, const Block & header, bool has_missi
};
auto convert_actions_dag = get_converting_dag(plan.getCurrentDataStream().header, header);
auto converting = std::make_unique<ExpressionStep>(plan.getCurrentDataStream(), convert_actions_dag);
auto converting = std::make_unique<ExpressionStep>(plan.getCurrentDataStream(), std::move(convert_actions_dag));
plan.addStep(std::move(converting));
}

View File

@ -10,33 +10,33 @@
namespace DB
{
static ITransformingStep::Traits getTraits(const ActionsDAGPtr & actions, const Block & header, const SortDescription & sort_description)
static ITransformingStep::Traits getTraits(const ActionsDAG & actions, const Block & header, const SortDescription & sort_description)
{
return ITransformingStep::Traits
{
{
.returns_single_stream = false,
.preserves_number_of_streams = true,
.preserves_sorting = actions->isSortingPreserved(header, sort_description),
.preserves_sorting = actions.isSortingPreserved(header, sort_description),
},
{
.preserves_number_of_rows = !actions->hasArrayJoin(),
.preserves_number_of_rows = !actions.hasArrayJoin(),
}
};
}
ExpressionStep::ExpressionStep(const DataStream & input_stream_, const ActionsDAGPtr & actions_dag_)
ExpressionStep::ExpressionStep(const DataStream & input_stream_, ActionsDAG actions_dag_)
: ITransformingStep(
input_stream_,
ExpressionTransform::transformHeader(input_stream_.header, *actions_dag_),
ExpressionTransform::transformHeader(input_stream_.header, actions_dag_),
getTraits(actions_dag_, input_stream_.header, input_stream_.sort_description))
, actions_dag(actions_dag_)
, actions_dag(std::move(actions_dag_))
{
}
void ExpressionStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings)
{
auto expression = std::make_shared<ExpressionActions>(actions_dag, settings.getActionsSettings());
auto expression = std::make_shared<ExpressionActions>(std::move(actions_dag), settings.getActionsSettings());
pipeline.addSimpleTransform([&](const Block & header)
{
@ -49,7 +49,7 @@ void ExpressionStep::transformPipeline(QueryPipelineBuilder & pipeline, const Bu
pipeline.getHeader().getColumnsWithTypeAndName(),
output_stream->header.getColumnsWithTypeAndName(),
ActionsDAG::MatchColumnsMode::Name);
auto convert_actions = std::make_shared<ExpressionActions>(convert_actions_dag, settings.getActionsSettings());
auto convert_actions = std::make_shared<ExpressionActions>(std::move(convert_actions_dag), settings.getActionsSettings());
pipeline.addSimpleTransform([&](const Block & header)
{
@ -61,20 +61,20 @@ void ExpressionStep::transformPipeline(QueryPipelineBuilder & pipeline, const Bu
void ExpressionStep::describeActions(FormatSettings & settings) const
{
String prefix(settings.offset, settings.indent_char);
auto expression = std::make_shared<ExpressionActions>(actions_dag);
auto expression = std::make_shared<ExpressionActions>(actions_dag.clone());
expression->describeActions(settings.out, prefix);
}
void ExpressionStep::describeActions(JSONBuilder::JSONMap & map) const
{
auto expression = std::make_shared<ExpressionActions>(actions_dag);
auto expression = std::make_shared<ExpressionActions>(actions_dag.clone());
map.add("Expression", expression->toTree());
}
void ExpressionStep::updateOutputStream()
{
output_stream = createOutputStream(
input_streams.front(), ExpressionTransform::transformHeader(input_streams.front().header, *actions_dag), getDataStreamTraits());
input_streams.front(), ExpressionTransform::transformHeader(input_streams.front().header, actions_dag), getDataStreamTraits());
if (!getDataStreamTraits().preserves_sorting)
return;

View File

@ -1,12 +1,10 @@
#pragma once
#include <Processors/QueryPlan/ITransformingStep.h>
#include <Interpreters/ActionsDAG.h>
namespace DB
{
class ActionsDAG;
using ActionsDAGPtr = std::shared_ptr<ActionsDAG>;
class ExpressionTransform;
class JoiningTransform;
@ -15,21 +13,22 @@ class ExpressionStep : public ITransformingStep
{
public:
explicit ExpressionStep(const DataStream & input_stream_, const ActionsDAGPtr & actions_dag_);
explicit ExpressionStep(const DataStream & input_stream_, ActionsDAG actions_dag_);
String getName() const override { return "Expression"; }
void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) override;
void describeActions(FormatSettings & settings) const override;
const ActionsDAGPtr & getExpression() const { return actions_dag; }
ActionsDAG & getExpression() { return actions_dag; }
const ActionsDAG & getExpression() const { return actions_dag; }
void describeActions(JSONBuilder::JSONMap & map) const override;
private:
void updateOutputStream() override;
ActionsDAGPtr actions_dag;
ActionsDAG actions_dag;
};
}

View File

@ -9,9 +9,9 @@
namespace DB
{
static ITransformingStep::Traits getTraits(const ActionsDAGPtr & expression, const Block & header, const SortDescription & sort_description, bool remove_filter_column, const String & filter_column_name)
static ITransformingStep::Traits getTraits(const ActionsDAG & expression, const Block & header, const SortDescription & sort_description, bool remove_filter_column, const String & filter_column_name)
{
bool preserves_sorting = expression->isSortingPreserved(header, sort_description, remove_filter_column ? filter_column_name : "");
bool preserves_sorting = expression.isSortingPreserved(header, sort_description, remove_filter_column ? filter_column_name : "");
if (remove_filter_column)
{
preserves_sorting &= std::find_if(
@ -35,28 +35,27 @@ static ITransformingStep::Traits getTraits(const ActionsDAGPtr & expression, con
FilterStep::FilterStep(
const DataStream & input_stream_,
const ActionsDAGPtr & actions_dag_,
ActionsDAG actions_dag_,
String filter_column_name_,
bool remove_filter_column_)
: ITransformingStep(
input_stream_,
FilterTransform::transformHeader(
input_stream_.header,
actions_dag_.get(),
&actions_dag_,
filter_column_name_,
remove_filter_column_),
getTraits(actions_dag_, input_stream_.header, input_stream_.sort_description, remove_filter_column_, filter_column_name_))
, actions_dag(actions_dag_)
, actions_dag(std::move(actions_dag_))
, filter_column_name(std::move(filter_column_name_))
, remove_filter_column(remove_filter_column_)
{
actions_dag = actions_dag->clone();
actions_dag->removeAliasesForFilter(filter_column_name);
actions_dag.removeAliasesForFilter(filter_column_name);
}
void FilterStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings)
{
auto expression = std::make_shared<ExpressionActions>(actions_dag, settings.getActionsSettings());
auto expression = std::make_shared<ExpressionActions>(std::move(actions_dag), settings.getActionsSettings());
pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type)
{
@ -70,7 +69,7 @@ void FilterStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQ
pipeline.getHeader().getColumnsWithTypeAndName(),
output_stream->header.getColumnsWithTypeAndName(),
ActionsDAG::MatchColumnsMode::Name);
auto convert_actions = std::make_shared<ExpressionActions>(convert_actions_dag, settings.getActionsSettings());
auto convert_actions = std::make_shared<ExpressionActions>(std::move(convert_actions_dag), settings.getActionsSettings());
pipeline.addSimpleTransform([&](const Block & header)
{
@ -88,7 +87,7 @@ void FilterStep::describeActions(FormatSettings & settings) const
settings.out << " (removed)";
settings.out << '\n';
auto expression = std::make_shared<ExpressionActions>(actions_dag);
auto expression = std::make_shared<ExpressionActions>(actions_dag.clone());
expression->describeActions(settings.out, prefix);
}
@ -97,7 +96,7 @@ void FilterStep::describeActions(JSONBuilder::JSONMap & map) const
map.add("Filter Column", filter_column_name);
map.add("Removes Filter", remove_filter_column);
auto expression = std::make_shared<ExpressionActions>(actions_dag);
auto expression = std::make_shared<ExpressionActions>(actions_dag.clone());
map.add("Expression", expression->toTree());
}
@ -105,7 +104,7 @@ void FilterStep::updateOutputStream()
{
output_stream = createOutputStream(
input_streams.front(),
FilterTransform::transformHeader(input_streams.front().header, actions_dag.get(), filter_column_name, remove_filter_column),
FilterTransform::transformHeader(input_streams.front().header, &actions_dag, filter_column_name, remove_filter_column),
getDataStreamTraits());
if (!getDataStreamTraits().preserves_sorting)

View File

@ -1,19 +1,17 @@
#pragma once
#include <Processors/QueryPlan/ITransformingStep.h>
#include <Interpreters/ActionsDAG.h>
namespace DB
{
class ActionsDAG;
using ActionsDAGPtr = std::shared_ptr<ActionsDAG>;
/// Implements WHERE, HAVING operations. See FilterTransform.
class FilterStep : public ITransformingStep
{
public:
FilterStep(
const DataStream & input_stream_,
const ActionsDAGPtr & actions_dag_,
ActionsDAG actions_dag_,
String filter_column_name_,
bool remove_filter_column_);
@ -23,15 +21,15 @@ public:
void describeActions(JSONBuilder::JSONMap & map) const override;
void describeActions(FormatSettings & settings) const override;
const ActionsDAGPtr & getExpression() const { return actions_dag; }
ActionsDAGPtr & getExpression() { return actions_dag; }
const ActionsDAG & getExpression() const { return actions_dag; }
ActionsDAG & getExpression() { return actions_dag; }
const String & getFilterColumnName() const { return filter_column_name; }
bool removesFilterColumn() const { return remove_filter_column; }
private:
void updateOutputStream() override;
ActionsDAGPtr actions_dag;
ActionsDAG actions_dag;
String filter_column_name;
bool remove_filter_column;
};

View File

@ -48,10 +48,10 @@ size_t tryConvertOuterJoinToInnerJoin(QueryPlan::Node * parent_node, QueryPlan::
bool right_stream_safe = true;
if (check_left_stream)
left_stream_safe = filter_dag->isFilterAlwaysFalseForDefaultValueInputs(filter_column_name, left_stream_input_header);
left_stream_safe = filter_dag.isFilterAlwaysFalseForDefaultValueInputs(filter_column_name, left_stream_input_header);
if (check_right_stream)
right_stream_safe = filter_dag->isFilterAlwaysFalseForDefaultValueInputs(filter_column_name, right_stream_input_header);
right_stream_safe = filter_dag.isFilterAlwaysFalseForDefaultValueInputs(filter_column_name, right_stream_input_header);
if (!left_stream_safe || !right_stream_safe)
return 0;

View File

@ -10,27 +10,27 @@
namespace DB::QueryPlanOptimizations
{
/// build actions DAG from stack of steps
static ActionsDAGPtr buildActionsForPlanPath(std::vector<ActionsDAGPtr> & dag_stack)
static std::optional<ActionsDAG> buildActionsForPlanPath(std::vector<const ActionsDAG *> & dag_stack)
{
if (dag_stack.empty())
return nullptr;
return {};
ActionsDAGPtr path_actions = dag_stack.back()->clone();
ActionsDAG path_actions = dag_stack.back()->clone();
dag_stack.pop_back();
while (!dag_stack.empty())
{
ActionsDAGPtr clone = dag_stack.back()->clone();
ActionsDAG clone = dag_stack.back()->clone();
dag_stack.pop_back();
path_actions->mergeInplace(std::move(*clone));
path_actions.mergeInplace(std::move(clone));
}
return path_actions;
}
static std::set<std::string>
getOriginalDistinctColumns(const ColumnsWithTypeAndName & distinct_columns, std::vector<ActionsDAGPtr> & dag_stack)
getOriginalDistinctColumns(const ColumnsWithTypeAndName & distinct_columns, std::vector<const ActionsDAG *> & dag_stack)
{
auto actions = buildActionsForPlanPath(dag_stack);
FindOriginalNodeForOutputName original_node_finder(actions);
FindOriginalNodeForOutputName original_node_finder(*actions);
std::set<std::string> original_distinct_columns;
for (const auto & column : distinct_columns)
{
@ -65,7 +65,7 @@ size_t tryDistinctReadInOrder(QueryPlan::Node * parent_node)
/// (3) gather actions DAG to find original names for columns in distinct step later
std::vector<ITransformingStep *> steps_to_update;
QueryPlan::Node * node = parent_node;
std::vector<ActionsDAGPtr> dag_stack;
std::vector<const ActionsDAG *> dag_stack;
while (!node->children.empty())
{
auto * step = dynamic_cast<ITransformingStep *>(node->step.get());
@ -79,9 +79,9 @@ size_t tryDistinctReadInOrder(QueryPlan::Node * parent_node)
steps_to_update.push_back(step);
if (const auto * const expr = typeid_cast<const ExpressionStep *>(step); expr)
dag_stack.push_back(expr->getExpression());
dag_stack.push_back(&expr->getExpression());
else if (const auto * const filter = typeid_cast<const FilterStep *>(step); filter)
dag_stack.push_back(filter->getExpression());
dag_stack.push_back(&filter->getExpression());
node = node->children.front();
}

View File

@ -101,7 +101,7 @@ static NameSet findIdentifiersOfNode(const ActionsDAG::Node * node)
return res;
}
static ActionsDAGPtr splitFilter(QueryPlan::Node * parent_node, const Names & available_inputs, size_t child_idx = 0)
static std::optional<ActionsDAG> splitFilter(QueryPlan::Node * parent_node, const Names & available_inputs, size_t child_idx = 0)
{
QueryPlan::Node * child_node = parent_node->children.front();
checkChildrenSize(child_node, child_idx + 1);
@ -110,16 +110,16 @@ static ActionsDAGPtr splitFilter(QueryPlan::Node * parent_node, const Names & av
auto & child = child_node->step;
auto * filter = assert_cast<FilterStep *>(parent.get());
const auto & expression = filter->getExpression();
auto & expression = filter->getExpression();
const auto & filter_column_name = filter->getFilterColumnName();
bool removes_filter = filter->removesFilterColumn();
const auto & all_inputs = child->getInputStreams()[child_idx].header.getColumnsWithTypeAndName();
return expression->splitActionsForFilterPushDown(filter_column_name, removes_filter, available_inputs, all_inputs);
return expression.splitActionsForFilterPushDown(filter_column_name, removes_filter, available_inputs, all_inputs);
}
static size_t
addNewFilterStepOrThrow(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes, const ActionsDAGPtr & split_filter,
addNewFilterStepOrThrow(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes, ActionsDAG split_filter,
bool can_remove_filter = true, size_t child_idx = 0, bool update_parent_filter = true)
{
QueryPlan::Node * child_node = parent_node->children.front();
@ -129,14 +129,14 @@ addNewFilterStepOrThrow(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes,
auto & child = child_node->step;
auto * filter = assert_cast<FilterStep *>(parent.get());
const auto & expression = filter->getExpression();
auto & expression = filter->getExpression();
const auto & filter_column_name = filter->getFilterColumnName();
const auto * filter_node = expression->tryFindInOutputs(filter_column_name);
const auto * filter_node = expression.tryFindInOutputs(filter_column_name);
if (update_parent_filter && !filter_node && !filter->removesFilterColumn())
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Filter column {} was removed from ActionsDAG but it is needed in result. DAG:\n{}",
filter_column_name, expression->dumpDAG());
filter_column_name, expression.dumpDAG());
/// Add new Filter step before Child.
/// Expression/Filter -> Child -> Something
@ -147,10 +147,10 @@ addNewFilterStepOrThrow(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes,
/// Expression/Filter -> Child -> Filter -> Something
/// New filter column is the first one.
String split_filter_column_name = split_filter->getOutputs().front()->result_name;
String split_filter_column_name = split_filter.getOutputs().front()->result_name;
node.step = std::make_unique<FilterStep>(
node.children.at(0)->step->getOutputStream(), split_filter, std::move(split_filter_column_name), can_remove_filter);
node.children.at(0)->step->getOutputStream(), std::move(split_filter), std::move(split_filter_column_name), can_remove_filter);
if (auto * transforming_step = dynamic_cast<ITransformingStep *>(child.get()))
{
@ -176,7 +176,7 @@ addNewFilterStepOrThrow(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes,
{
/// This means that all predicates of filter were pushed down.
/// Replace current actions to expression, as we don't need to filter anything.
parent = std::make_unique<ExpressionStep>(child->getOutputStream(), expression);
parent = std::make_unique<ExpressionStep>(child->getOutputStream(), std::move(expression));
}
else
{
@ -192,7 +192,7 @@ tryAddNewFilterStep(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes, con
bool can_remove_filter = true, size_t child_idx = 0)
{
if (auto split_filter = splitFilter(parent_node, allowed_inputs, child_idx))
return addNewFilterStepOrThrow(parent_node, nodes, split_filter, can_remove_filter, child_idx);
return addNewFilterStepOrThrow(parent_node, nodes, std::move(*split_filter), can_remove_filter, child_idx);
return 0;
}
@ -332,7 +332,7 @@ static size_t tryPushDownOverJoinStep(QueryPlan::Node * parent_node, QueryPlan::
Names left_stream_available_columns_to_push_down = get_available_columns_for_filter(true /*push_to_left_stream*/, left_stream_filter_push_down_input_columns_available);
Names right_stream_available_columns_to_push_down = get_available_columns_for_filter(false /*push_to_left_stream*/, right_stream_filter_push_down_input_columns_available);
auto join_filter_push_down_actions = filter->getExpression()->splitActionsForJOINFilterPushDown(filter->getFilterColumnName(),
auto join_filter_push_down_actions = filter->getExpression().splitActionsForJOINFilterPushDown(filter->getFilterColumnName(),
filter->removesFilterColumn(),
left_stream_available_columns_to_push_down,
left_stream_input_header,
@ -346,42 +346,44 @@ static size_t tryPushDownOverJoinStep(QueryPlan::Node * parent_node, QueryPlan::
if (join_filter_push_down_actions.left_stream_filter_to_push_down)
{
const auto & result_name = join_filter_push_down_actions.left_stream_filter_to_push_down->getOutputs()[0]->result_name;
updated_steps += addNewFilterStepOrThrow(parent_node,
nodes,
join_filter_push_down_actions.left_stream_filter_to_push_down,
std::move(*join_filter_push_down_actions.left_stream_filter_to_push_down),
join_filter_push_down_actions.left_stream_filter_removes_filter,
0 /*child_idx*/,
false /*update_parent_filter*/);
LOG_DEBUG(&Poco::Logger::get("QueryPlanOptimizations"),
"Pushed down filter {} to the {} side of join",
join_filter_push_down_actions.left_stream_filter_to_push_down->getOutputs()[0]->result_name,
result_name,
JoinKind::Left);
}
if (join_filter_push_down_actions.right_stream_filter_to_push_down && allow_push_down_to_right)
{
const auto & result_name = join_filter_push_down_actions.right_stream_filter_to_push_down->getOutputs()[0]->result_name;
updated_steps += addNewFilterStepOrThrow(parent_node,
nodes,
join_filter_push_down_actions.right_stream_filter_to_push_down,
std::move(*join_filter_push_down_actions.right_stream_filter_to_push_down),
join_filter_push_down_actions.right_stream_filter_removes_filter,
1 /*child_idx*/,
false /*update_parent_filter*/);
LOG_DEBUG(&Poco::Logger::get("QueryPlanOptimizations"),
"Pushed down filter {} to the {} side of join",
join_filter_push_down_actions.right_stream_filter_to_push_down->getOutputs()[0]->result_name,
result_name,
JoinKind::Right);
}
if (updated_steps > 0)
{
const auto & filter_column_name = filter->getFilterColumnName();
const auto & filter_expression = filter->getExpression();
auto & filter_expression = filter->getExpression();
const auto * filter_node = filter_expression->tryFindInOutputs(filter_column_name);
const auto * filter_node = filter_expression.tryFindInOutputs(filter_column_name);
if (!filter_node && !filter->removesFilterColumn())
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Filter column {} was removed from ActionsDAG but it is needed in result. DAG:\n{}",
filter_column_name, filter_expression->dumpDAG());
filter_column_name, filter_expression.dumpDAG());
/// Filter column was replaced to constant.
@ -391,7 +393,7 @@ static size_t tryPushDownOverJoinStep(QueryPlan::Node * parent_node, QueryPlan::
{
/// This means that all predicates of filter were pushed down.
/// Replace current actions to expression, as we don't need to filter anything.
parent = std::make_unique<ExpressionStep>(child->getOutputStream(), filter_expression);
parent = std::make_unique<ExpressionStep>(child->getOutputStream(), std::move(filter_expression));
}
else
{
@ -416,7 +418,7 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes
if (!filter)
return 0;
if (filter->getExpression()->hasStatefulFunctions())
if (filter->getExpression().hasStatefulFunctions())
return 0;
if (auto * aggregating = typeid_cast<AggregatingStep *>(child.get()))
@ -430,7 +432,7 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes
return 0;
const auto & actions = filter->getExpression();
const auto & filter_node = actions->findInOutputs(filter->getFilterColumnName());
const auto & filter_node = actions.findInOutputs(filter->getFilterColumnName());
auto identifiers_in_predicate = findIdentifiersOfNode(&filter_node);
@ -597,7 +599,7 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes
filter_node.step = std::make_unique<FilterStep>(
filter_node.children.front()->step->getOutputStream(),
filter->getExpression()->clone(),
filter->getExpression().clone(),
filter->getFilterColumnName(),
filter->removesFilterColumn());
}
@ -611,7 +613,7 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes
if (auto * read_from_merge = typeid_cast<ReadFromMerge *>(child.get()))
{
FilterDAGInfo info{filter->getExpression(), filter->getFilterColumnName(), filter->removesFilterColumn()};
FilterDAGInfo info{filter->getExpression().clone(), filter->getFilterColumnName(), filter->removesFilterColumn()};
read_from_merge->addFilter(std::move(info));
std::swap(*parent_node, *child_node);
return 1;

View File

@ -28,10 +28,10 @@ size_t tryLiftUpArrayJoin(QueryPlan::Node * parent_node, QueryPlan::Nodes & node
const auto & expression = expression_step ? expression_step->getExpression()
: filter_step->getExpression();
auto split_actions = expression->splitActionsBeforeArrayJoin(array_join->columns);
auto split_actions = expression.splitActionsBeforeArrayJoin(array_join->columns);
/// No actions can be moved before ARRAY JOIN.
if (split_actions.first->trivial())
if (split_actions.first.trivial())
return 0;
auto description = parent->getStepDescription();
@ -49,9 +49,9 @@ size_t tryLiftUpArrayJoin(QueryPlan::Node * parent_node, QueryPlan::Nodes & node
array_join_step->updateInputStream(node.step->getOutputStream());
if (expression_step)
parent = std::make_unique<ExpressionStep>(array_join_step->getOutputStream(), split_actions.second);
parent = std::make_unique<ExpressionStep>(array_join_step->getOutputStream(), std::move(split_actions.second));
else
parent = std::make_unique<FilterStep>(array_join_step->getOutputStream(), split_actions.second,
parent = std::make_unique<FilterStep>(array_join_step->getOutputStream(), std::move(split_actions.second),
filter_step->getFilterColumnName(), filter_step->removesFilterColumn());
parent->setStepDescription(description + " [split]");

View File

@ -66,13 +66,13 @@ size_t tryExecuteFunctionsAfterSorting(QueryPlan::Node * parent_node, QueryPlan:
NameSet sort_columns;
for (const auto & col : sorting_step->getSortDescription())
sort_columns.insert(col.column_name);
auto [needed_for_sorting, unneeded_for_sorting, _] = expression_step->getExpression()->splitActionsBySortingDescription(sort_columns);
auto [needed_for_sorting, unneeded_for_sorting, _] = expression_step->getExpression().splitActionsBySortingDescription(sort_columns);
// No calculations can be postponed.
if (unneeded_for_sorting->trivial())
if (unneeded_for_sorting.trivial())
return 0;
if (!areNodesConvertableToBlock(needed_for_sorting->getOutputs()) || !areNodesConvertableToBlock(unneeded_for_sorting->getInputs()))
if (!areNodesConvertableToBlock(needed_for_sorting.getOutputs()) || !areNodesConvertableToBlock(unneeded_for_sorting.getInputs()))
return 0;
// Sorting (parent_node) -> Expression (child_node)

View File

@ -49,7 +49,7 @@ size_t tryLiftUpUnion(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes)
expr_node.step = std::make_unique<ExpressionStep>(
expr_node.children.front()->step->getOutputStream(),
expression->getExpression()->clone());
expression->getExpression().clone());
}
/// - Expression - Something

View File

@ -37,18 +37,18 @@ size_t tryMergeExpressions(QueryPlan::Node * parent_node, QueryPlan::Nodes &)
if (parent_expr && child_expr)
{
const auto & child_actions = child_expr->getExpression();
const auto & parent_actions = parent_expr->getExpression();
auto & child_actions = child_expr->getExpression();
auto & parent_actions = parent_expr->getExpression();
/// We cannot combine actions with arrayJoin and stateful function because we not always can reorder them.
/// Example: select rowNumberInBlock() from (select arrayJoin([1, 2]))
/// Such a query will return two zeroes if we combine actions together.
if (child_actions->hasArrayJoin() && parent_actions->hasStatefulFunctions())
if (child_actions.hasArrayJoin() && parent_actions.hasStatefulFunctions())
return 0;
auto merged = ActionsDAG::merge(std::move(*child_actions), std::move(*parent_actions));
auto merged = ActionsDAG::merge(std::move(child_actions), std::move(parent_actions));
auto expr = std::make_unique<ExpressionStep>(child_expr->getInputStreams().front(), merged);
auto expr = std::make_unique<ExpressionStep>(child_expr->getInputStreams().front(), std::move(merged));
expr->setStepDescription("(" + parent_expr->getStepDescription() + " + " + child_expr->getStepDescription() + ")");
parent_node->step = std::move(expr);
@ -57,16 +57,16 @@ size_t tryMergeExpressions(QueryPlan::Node * parent_node, QueryPlan::Nodes &)
}
else if (parent_filter && child_expr)
{
const auto & child_actions = child_expr->getExpression();
const auto & parent_actions = parent_filter->getExpression();
auto & child_actions = child_expr->getExpression();
auto & parent_actions = parent_filter->getExpression();
if (child_actions->hasArrayJoin() && parent_actions->hasStatefulFunctions())
if (child_actions.hasArrayJoin() && parent_actions.hasStatefulFunctions())
return 0;
auto merged = ActionsDAG::merge(std::move(*child_actions), std::move(*parent_actions));
auto merged = ActionsDAG::merge(std::move(child_actions), std::move(parent_actions));
auto filter = std::make_unique<FilterStep>(child_expr->getInputStreams().front(),
merged,
std::move(merged),
parent_filter->getFilterColumnName(),
parent_filter->removesFilterColumn());
filter->setStepDescription("(" + parent_filter->getStepDescription() + " + " + child_expr->getStepDescription() + ")");
@ -93,32 +93,31 @@ size_t tryMergeFilters(QueryPlan::Node * parent_node, QueryPlan::Nodes &)
if (parent_filter && child_filter)
{
const auto & child_actions = child_filter->getExpression();
const auto & parent_actions = parent_filter->getExpression();
auto & child_actions = child_filter->getExpression();
auto & parent_actions = parent_filter->getExpression();
if (child_actions->hasArrayJoin())
if (child_actions.hasArrayJoin())
return 0;
auto actions = child_actions->clone();
const auto & child_filter_node = actions->findInOutputs(child_filter->getFilterColumnName());
const auto & child_filter_node = child_actions.findInOutputs(child_filter->getFilterColumnName());
if (child_filter->removesFilterColumn())
removeFromOutputs(*actions, child_filter_node);
removeFromOutputs(child_actions, child_filter_node);
actions->mergeInplace(std::move(*parent_actions->clone()));
child_actions.mergeInplace(std::move(parent_actions));
const auto & parent_filter_node = actions->findInOutputs(parent_filter->getFilterColumnName());
const auto & parent_filter_node = child_actions.findInOutputs(parent_filter->getFilterColumnName());
if (parent_filter->removesFilterColumn())
removeFromOutputs(*actions, parent_filter_node);
removeFromOutputs(child_actions, parent_filter_node);
FunctionOverloadResolverPtr func_builder_and = std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionAnd>());
const auto & condition = actions->addFunction(func_builder_and, {&child_filter_node, &parent_filter_node}, {});
auto & outputs = actions->getOutputs();
const auto & condition = child_actions.addFunction(func_builder_and, {&child_filter_node, &parent_filter_node}, {});
auto & outputs = child_actions.getOutputs();
outputs.insert(outputs.begin(), &condition);
actions->removeUnusedActions(false);
child_actions.removeUnusedActions(false);
auto filter = std::make_unique<FilterStep>(child_filter->getInputStreams().front(),
actions,
std::move(child_actions),
condition.result_name,
true);
filter->setStepDescription("(" + parent_filter->getStepDescription() + " + " + child_filter->getStepDescription() + ")");

View File

@ -56,12 +56,12 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &)
return;
const auto & storage_prewhere_info = source_step_with_filter->getPrewhereInfo();
if (storage_prewhere_info && storage_prewhere_info->prewhere_actions)
if (storage_prewhere_info)
return;
/// TODO: We can also check for UnionStep, such as StorageBuffer and local distributed plans.
QueryPlan::Node * filter_node = (stack.rbegin() + 1)->node;
const auto * filter_step = typeid_cast<FilterStep *>(filter_node->step.get());
auto * filter_step = typeid_cast<FilterStep *>(filter_node->step.get());
if (!filter_step)
return;
@ -85,10 +85,11 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &)
Names queried_columns = source_step_with_filter->requiredSourceColumns();
const auto & source_filter_actions_dag = source_step_with_filter->getFilterActionsDAG();
MergeTreeWhereOptimizer where_optimizer{
std::move(column_compressed_sizes),
storage_metadata,
storage.getConditionSelectivityEstimatorByPredicate(storage_snapshot, source_step_with_filter->getFilterActionsDAG(), context),
storage.getConditionSelectivityEstimatorByPredicate(storage_snapshot, source_filter_actions_dag ? &*source_filter_actions_dag : nullptr, context),
queried_columns,
storage.supportedPrewhereColumns(),
getLogger("QueryPlanOptimizePrewhere")};
@ -110,20 +111,20 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &)
prewhere_info->need_filter = true;
prewhere_info->remove_prewhere_column = optimize_result.fully_moved_to_prewhere && filter_step->removesFilterColumn();
auto filter_expression = filter_step->getExpression();
auto filter_expression = std::move(filter_step->getExpression());
const auto & filter_column_name = filter_step->getFilterColumnName();
if (prewhere_info->remove_prewhere_column)
{
removeFromOutput(*filter_expression, filter_column_name);
auto & outputs = filter_expression->getOutputs();
removeFromOutput(filter_expression, filter_column_name);
auto & outputs = filter_expression.getOutputs();
size_t size = outputs.size();
outputs.insert(outputs.end(), optimize_result.prewhere_nodes.begin(), optimize_result.prewhere_nodes.end());
filter_expression->removeUnusedActions(false);
filter_expression.removeUnusedActions(false);
outputs.resize(size);
}
auto split_result = filter_step->getExpression()->split(optimize_result.prewhere_nodes, true, true);
auto split_result = filter_expression.split(optimize_result.prewhere_nodes, true, true);
/// This is the leak of abstraction.
/// Splited actions may have inputs which are needed only for PREWHERE.
@ -139,15 +140,15 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &)
/// So, here we restore removed inputs for PREWHERE actions
{
std::unordered_set<const ActionsDAG::Node *> first_outputs(
split_result.first->getOutputs().begin(), split_result.first->getOutputs().end());
for (const auto * input : split_result.first->getInputs())
split_result.first.getOutputs().begin(), split_result.first.getOutputs().end());
for (const auto * input : split_result.first.getInputs())
{
if (!first_outputs.contains(input))
{
split_result.first->getOutputs().push_back(input);
split_result.first.getOutputs().push_back(input);
/// Add column to second actions as input.
/// Do not add it to result, so it would be removed.
split_result.second->addInput(input->result_name, input->result_type);
split_result.second.addInput(input->result_name, input->result_type);
}
}
}
@ -164,16 +165,16 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &)
{
prewhere_info->prewhere_column_name = conditions.front()->result_name;
if (prewhere_info->remove_prewhere_column)
prewhere_info->prewhere_actions->getOutputs().push_back(conditions.front());
prewhere_info->prewhere_actions.getOutputs().push_back(conditions.front());
}
else
{
prewhere_info->remove_prewhere_column = true;
FunctionOverloadResolverPtr func_builder_and = std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionAnd>());
const auto * node = &prewhere_info->prewhere_actions->addFunction(func_builder_and, std::move(conditions), {});
const auto * node = &prewhere_info->prewhere_actions.addFunction(func_builder_and, std::move(conditions), {});
prewhere_info->prewhere_column_name = node->result_name;
prewhere_info->prewhere_actions->getOutputs().push_back(node);
prewhere_info->prewhere_actions.getOutputs().push_back(node);
}
source_step_with_filter->updatePrewhereInfo(prewhere_info);

View File

@ -18,16 +18,16 @@ void optimizePrimaryKeyConditionAndLimit(const Stack & stack)
const auto & storage_prewhere_info = source_step_with_filter->getPrewhereInfo();
if (storage_prewhere_info)
{
source_step_with_filter->addFilter(storage_prewhere_info->prewhere_actions, storage_prewhere_info->prewhere_column_name);
source_step_with_filter->addFilter(storage_prewhere_info->prewhere_actions.clone(), storage_prewhere_info->prewhere_column_name);
if (storage_prewhere_info->row_level_filter)
source_step_with_filter->addFilter(storage_prewhere_info->row_level_filter, storage_prewhere_info->row_level_column_name);
source_step_with_filter->addFilter(storage_prewhere_info->row_level_filter->clone(), storage_prewhere_info->row_level_column_name);
}
for (auto iter = stack.rbegin() + 1; iter != stack.rend(); ++iter)
{
if (auto * filter_step = typeid_cast<FilterStep *>(iter->node->step.get()))
{
source_step_with_filter->addFilter(filter_step->getExpression(), filter_step->getFilterColumnName());
source_step_with_filter->addFilter(filter_step->getExpression().clone(), filter_step->getFilterColumnName());
}
else if (auto * limit_step = typeid_cast<LimitStep *>(iter->node->step.get()))
{

View File

@ -171,17 +171,17 @@ static void appendFixedColumnsFromFilterExpression(const ActionsDAG::Node & filt
}
}
static void appendExpression(ActionsDAGPtr & dag, const ActionsDAGPtr & expression)
static void appendExpression(std::optional<ActionsDAG> & dag, const ActionsDAG & expression)
{
if (dag)
dag->mergeInplace(std::move(*expression->clone()));
dag->mergeInplace(expression.clone());
else
dag = expression->clone();
dag = expression.clone();
}
/// This function builds a common DAG which is a merge of DAGs from Filter and Expression steps chain.
/// Additionally, build a set of fixed columns.
void buildSortingDAG(QueryPlan::Node & node, ActionsDAGPtr & dag, FixedColumns & fixed_columns, size_t & limit)
void buildSortingDAG(QueryPlan::Node & node, std::optional<ActionsDAG> & dag, FixedColumns & fixed_columns, size_t & limit)
{
IQueryPlanStep * step = node.step.get();
if (auto * reading = typeid_cast<ReadFromMergeTree *>(step))
@ -191,13 +191,11 @@ void buildSortingDAG(QueryPlan::Node & node, ActionsDAGPtr & dag, FixedColumns &
/// Should ignore limit if there is filtering.
limit = 0;
if (prewhere_info->prewhere_actions)
{
//std::cerr << "====== Adding prewhere " << std::endl;
appendExpression(dag, prewhere_info->prewhere_actions);
if (const auto * filter_expression = dag->tryFindInOutputs(prewhere_info->prewhere_column_name))
appendFixedColumnsFromFilterExpression(*filter_expression, fixed_columns);
}
//std::cerr << "====== Adding prewhere " << std::endl;
appendExpression(dag, prewhere_info->prewhere_actions);
if (const auto * filter_expression = dag->tryFindInOutputs(prewhere_info->prewhere_column_name))
appendFixedColumnsFromFilterExpression(*filter_expression, fixed_columns);
}
return;
}
@ -212,7 +210,7 @@ void buildSortingDAG(QueryPlan::Node & node, ActionsDAGPtr & dag, FixedColumns &
const auto & actions = expression->getExpression();
/// Should ignore limit because arrayJoin() can reduce the number of rows in case of empty array.
if (actions->hasArrayJoin())
if (actions.hasArrayJoin())
limit = 0;
appendExpression(dag, actions);
@ -330,7 +328,7 @@ void enreachFixedColumns(const ActionsDAG & dag, FixedColumns & fixed_columns)
InputOrderInfoPtr buildInputOrderInfo(
const FixedColumns & fixed_columns,
const ActionsDAGPtr & dag,
const std::optional<ActionsDAG> & dag,
const SortDescription & description,
const KeyDescription & sorting_key,
size_t limit)
@ -507,7 +505,7 @@ struct AggregationInputOrder
AggregationInputOrder buildInputOrderInfo(
const FixedColumns & fixed_columns,
const ActionsDAGPtr & dag,
const std::optional<ActionsDAG> & dag,
const Names & group_by_keys,
const ActionsDAG & sorting_key_dag,
const Names & sorting_key_columns)
@ -693,7 +691,7 @@ AggregationInputOrder buildInputOrderInfo(
InputOrderInfoPtr buildInputOrderInfo(
const ReadFromMergeTree * reading,
const FixedColumns & fixed_columns,
const ActionsDAGPtr & dag,
const std::optional<ActionsDAG> & dag,
const SortDescription & description,
size_t limit)
{
@ -709,7 +707,7 @@ InputOrderInfoPtr buildInputOrderInfo(
InputOrderInfoPtr buildInputOrderInfo(
ReadFromMerge * merge,
const FixedColumns & fixed_columns,
const ActionsDAGPtr & dag,
const std::optional<ActionsDAG> & dag,
const SortDescription & description,
size_t limit)
{
@ -745,7 +743,7 @@ InputOrderInfoPtr buildInputOrderInfo(
AggregationInputOrder buildInputOrderInfo(
ReadFromMergeTree * reading,
const FixedColumns & fixed_columns,
const ActionsDAGPtr & dag,
const std::optional<ActionsDAG> & dag,
const Names & group_by_keys)
{
const auto & sorting_key = reading->getStorageMetadata()->getSortingKey();
@ -760,7 +758,7 @@ AggregationInputOrder buildInputOrderInfo(
AggregationInputOrder buildInputOrderInfo(
ReadFromMerge * merge,
const FixedColumns & fixed_columns,
const ActionsDAGPtr & dag,
const std::optional<ActionsDAG> & dag,
const Names & group_by_keys)
{
const auto & tables = merge->getSelectedTables();
@ -801,7 +799,7 @@ InputOrderInfoPtr buildInputOrderInfo(SortingStep & sorting, QueryPlan::Node & n
const auto & description = sorting.getSortDescription();
size_t limit = sorting.getLimit();
ActionsDAGPtr dag;
std::optional<ActionsDAG> dag;
FixedColumns fixed_columns;
buildSortingDAG(node, dag, fixed_columns, limit);
@ -855,7 +853,7 @@ AggregationInputOrder buildInputOrderInfo(AggregatingStep & aggregating, QueryPl
const auto & keys = aggregating.getParams().keys;
size_t limit = 0;
ActionsDAGPtr dag;
std::optional<ActionsDAG> dag;
FixedColumns fixed_columns;
buildSortingDAG(node, dag, fixed_columns, limit);
@ -1076,13 +1074,13 @@ size_t tryReuseStorageOrderingForWindowFunctions(QueryPlan::Node * parent_node,
for (const auto & actions_dag : window_desc.partition_by_actions)
{
order_by_elements_actions.emplace_back(
std::make_shared<ExpressionActions>(actions_dag, ExpressionActionsSettings::fromContext(context, CompileExpressions::yes)));
std::make_shared<ExpressionActions>(actions_dag->clone(), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes)));
}
for (const auto & actions_dag : window_desc.order_by_actions)
{
order_by_elements_actions.emplace_back(
std::make_shared<ExpressionActions>(actions_dag, ExpressionActionsSettings::fromContext(context, CompileExpressions::yes)));
std::make_shared<ExpressionActions>(actions_dag->clone(), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes)));
}
auto order_optimizer = std::make_shared<ReadInOrderOptimizer>(

View File

@ -43,7 +43,7 @@ static DAGIndex buildDAGIndex(const ActionsDAG & dag)
/// Required analysis info from aggregate projection.
struct AggregateProjectionInfo
{
ActionsDAGPtr before_aggregation;
std::optional<ActionsDAG> before_aggregation;
Names keys;
AggregateDescriptions aggregates;
@ -274,7 +274,7 @@ static void appendAggregateFunctions(
}
}
ActionsDAGPtr analyzeAggregateProjection(
std::optional<ActionsDAG> analyzeAggregateProjection(
const AggregateProjectionInfo & info,
const QueryDAG & query,
const DAGIndex & query_index,
@ -394,7 +394,7 @@ ActionsDAGPtr analyzeAggregateProjection(
// LOG_TRACE(getLogger("optimizeUseProjections"), "Folding actions by projection");
auto proj_dag = query.dag->foldActionsByProjection(new_inputs, query_key_nodes);
appendAggregateFunctions(*proj_dag, aggregates, *matched_aggregates);
appendAggregateFunctions(proj_dag, aggregates, *matched_aggregates);
return proj_dag;
}
@ -406,7 +406,7 @@ struct AggregateProjectionCandidate : public ProjectionCandidate
/// Actions which need to be applied to columns from projection
/// in order to get all the columns required for aggregation.
ActionsDAGPtr dag;
ActionsDAG dag;
};
struct MinMaxProjectionCandidate
@ -481,13 +481,13 @@ AggregateProjectionCandidates getAggregateProjectionCandidates(
if (auto proj_dag = analyzeAggregateProjection(info, dag, query_index, keys, aggregates))
{
// LOG_TRACE(getLogger("optimizeUseProjections"), "Projection analyzed DAG {}", proj_dag->dumpDAG());
AggregateProjectionCandidate candidate{.info = std::move(info), .dag = std::move(proj_dag)};
AggregateProjectionCandidate candidate{.info = std::move(info), .dag = std::move(*proj_dag)};
// LOG_TRACE(getLogger("optimizeUseProjections"), "Projection sample block {}", sample_block.dumpStructure());
auto block = reading.getMergeTreeData().getMinMaxCountProjectionBlock(
metadata,
candidate.dag->getRequiredColumnsNames(),
(dag.filter_node ? dag.dag : nullptr),
candidate.dag.getRequiredColumnsNames(),
(dag.filter_node ? &*dag.dag : nullptr),
parts,
max_added_blocks.get(),
context);
@ -538,7 +538,7 @@ AggregateProjectionCandidates getAggregateProjectionCandidates(
if (auto proj_dag = analyzeAggregateProjection(info, dag, query_index, keys, aggregates))
{
// LOG_TRACE(getLogger("optimizeUseProjections"), "Projection analyzed DAG {}", proj_dag->dumpDAG());
AggregateProjectionCandidate candidate{.info = std::move(info), .dag = std::move(proj_dag)};
AggregateProjectionCandidate candidate{.info = std::move(info), .dag = std::move(*proj_dag)};
candidate.projection = projection;
candidates.real.emplace_back(std::move(candidate));
}
@ -666,7 +666,7 @@ std::optional<String> optimizeUseAggregateProjections(QueryPlan::Node & node, Qu
/// Selecting best candidate.
for (auto & candidate : candidates.real)
{
auto required_column_names = candidate.dag->getRequiredColumnsNames();
auto required_column_names = candidate.dag.getRequiredColumnsNames();
bool analyzed = analyzeProjectionCandidate(
candidate,
@ -677,7 +677,7 @@ std::optional<String> optimizeUseAggregateProjections(QueryPlan::Node & node, Qu
query_info,
context,
max_added_blocks,
candidate.dag);
&candidate.dag);
if (!analyzed)
continue;
@ -767,7 +767,7 @@ std::optional<String> optimizeUseAggregateProjections(QueryPlan::Node & node, Qu
projection_reading = reader.readFromParts(
/* parts = */ {},
/* alter_conversions = */ {},
best_candidate->dag->getRequiredColumnsNames(),
best_candidate->dag.getRequiredColumnsNames(),
proj_snapshot,
projection_query_info,
context,
@ -779,7 +779,7 @@ std::optional<String> optimizeUseAggregateProjections(QueryPlan::Node & node, Qu
if (!projection_reading)
{
auto header = proj_snapshot->getSampleBlockForColumns(best_candidate->dag->getRequiredColumnsNames());
auto header = proj_snapshot->getSampleBlockForColumns(best_candidate->dag.getRequiredColumnsNames());
Pipe pipe(std::make_shared<NullSource>(std::move(header)));
projection_reading = std::make_unique<ReadFromPreparedSource>(std::move(pipe));
}
@ -810,17 +810,19 @@ std::optional<String> optimizeUseAggregateProjections(QueryPlan::Node & node, Qu
if (best_candidate)
{
aggregate_projection_node = &nodes.emplace_back();
if (candidates.has_filter)
{
const auto & result_name = best_candidate->dag.getOutputs().front()->result_name;
aggregate_projection_node->step = std::make_unique<FilterStep>(
projection_reading_node.step->getOutputStream(),
best_candidate->dag,
best_candidate->dag->getOutputs().front()->result_name,
std::move(best_candidate->dag),
result_name,
true);
}
else
aggregate_projection_node->step
= std::make_unique<ExpressionStep>(projection_reading_node.step->getOutputStream(), best_candidate->dag);
= std::make_unique<ExpressionStep>(projection_reading_node.step->getOutputStream(), std::move(best_candidate->dag));
aggregate_projection_node->children.push_back(&projection_reading_node);
}

View File

@ -25,7 +25,7 @@ struct NormalProjectionCandidate : public ProjectionCandidate
{
};
static ActionsDAGPtr makeMaterializingDAG(const Block & proj_header, const Block main_header)
static std::optional<ActionsDAG> makeMaterializingDAG(const Block & proj_header, const Block main_header)
{
/// Materialize constants in case we don't have it in output header.
/// This may happen e.g. if we have PREWHERE.
@ -33,7 +33,7 @@ static ActionsDAGPtr makeMaterializingDAG(const Block & proj_header, const Block
size_t num_columns = main_header.columns();
/// This is a error; will have block structure mismatch later.
if (proj_header.columns() != num_columns)
return nullptr;
return {};
std::vector<size_t> const_positions;
for (size_t i = 0; i < num_columns; ++i)
@ -47,17 +47,17 @@ static ActionsDAGPtr makeMaterializingDAG(const Block & proj_header, const Block
}
if (const_positions.empty())
return nullptr;
return {};
ActionsDAGPtr dag = std::make_unique<ActionsDAG>();
auto & outputs = dag->getOutputs();
ActionsDAG dag;
auto & outputs = dag.getOutputs();
for (const auto & col : proj_header.getColumnsWithTypeAndName())
outputs.push_back(&dag->addInput(col));
outputs.push_back(&dag.addInput(col));
for (auto pos : const_positions)
{
auto & output = outputs[pos];
output = &dag->materializeNode(*output);
output = &dag.materializeNode(*output);
}
return dag;
@ -174,7 +174,7 @@ std::optional<String> optimizeUseNormalProjections(Stack & stack, QueryPlan::Nod
query_info,
context,
max_added_blocks,
query.filter_node ? query.dag : nullptr);
query.filter_node ? &*query.dag : nullptr);
if (!analyzed)
continue;
@ -244,14 +244,14 @@ std::optional<String> optimizeUseNormalProjections(Stack & stack, QueryPlan::Nod
{
expr_or_filter_node.step = std::make_unique<FilterStep>(
projection_reading_node.step->getOutputStream(),
query.dag,
std::move(*query.dag),
query.filter_node->result_name,
true);
}
else
expr_or_filter_node.step = std::make_unique<ExpressionStep>(
projection_reading_node.step->getOutputStream(),
query.dag);
std::move(*query.dag));
expr_or_filter_node.children.push_back(&projection_reading_node);
next_node = &expr_or_filter_node;
@ -269,7 +269,7 @@ std::optional<String> optimizeUseNormalProjections(Stack & stack, QueryPlan::Nod
if (auto materializing = makeMaterializingDAG(proj_stream->header, main_stream.header))
{
auto converting = std::make_unique<ExpressionStep>(*proj_stream, materializing);
auto converting = std::make_unique<ExpressionStep>(*proj_stream, std::move(*materializing));
proj_stream = &converting->getOutputStream();
auto & expr_node = nodes.emplace_back();
expr_node.step = std::move(converting);

View File

@ -65,12 +65,12 @@ std::shared_ptr<PartitionIdToMaxBlock> getMaxAddedBlocks(ReadFromMergeTree * rea
return {};
}
void QueryDAG::appendExpression(const ActionsDAGPtr & expression)
void QueryDAG::appendExpression(const ActionsDAG & expression)
{
if (dag)
dag->mergeInplace(std::move(*expression->clone()));
dag->mergeInplace(expression.clone());
else
dag = expression->clone();
dag = expression.clone();
}
const ActionsDAG::Node * findInOutputs(ActionsDAG & dag, const std::string & name, bool remove)
@ -121,22 +121,19 @@ bool QueryDAG::buildImpl(QueryPlan::Node & node, ActionsDAG::NodeRawConstPtrs &
{
if (prewhere_info->row_level_filter)
{
appendExpression(prewhere_info->row_level_filter);
appendExpression(*prewhere_info->row_level_filter);
if (const auto * filter_expression = findInOutputs(*dag, prewhere_info->row_level_column_name, false))
filter_nodes.push_back(filter_expression);
else
return false;
}
if (prewhere_info->prewhere_actions)
{
appendExpression(prewhere_info->prewhere_actions);
if (const auto * filter_expression
= findInOutputs(*dag, prewhere_info->prewhere_column_name, prewhere_info->remove_prewhere_column))
filter_nodes.push_back(filter_expression);
else
return false;
}
appendExpression(prewhere_info->prewhere_actions);
if (const auto * filter_expression
= findInOutputs(*dag, prewhere_info->prewhere_column_name, prewhere_info->remove_prewhere_column))
filter_nodes.push_back(filter_expression);
else
return false;
}
return true;
}
@ -150,7 +147,7 @@ bool QueryDAG::buildImpl(QueryPlan::Node & node, ActionsDAG::NodeRawConstPtrs &
if (auto * expression = typeid_cast<ExpressionStep *>(step))
{
const auto & actions = expression->getExpression();
if (actions->hasArrayJoin())
if (actions.hasArrayJoin())
return false;
appendExpression(actions);
@ -160,7 +157,7 @@ bool QueryDAG::buildImpl(QueryPlan::Node & node, ActionsDAG::NodeRawConstPtrs &
if (auto * filter = typeid_cast<FilterStep *>(step))
{
const auto & actions = filter->getExpression();
if (actions->hasArrayJoin())
if (actions.hasArrayJoin())
return false;
appendExpression(actions);
@ -214,7 +211,7 @@ bool analyzeProjectionCandidate(
const SelectQueryInfo & query_info,
const ContextPtr & context,
const std::shared_ptr<PartitionIdToMaxBlock> & max_added_blocks,
const ActionsDAGPtr & dag)
const ActionsDAG * dag)
{
MergeTreeData::DataPartsVector projection_parts;
MergeTreeData::DataPartsVector normal_parts;
@ -239,7 +236,8 @@ bool analyzeProjectionCandidate(
auto projection_query_info = query_info;
projection_query_info.prewhere_info = nullptr;
projection_query_info.filter_actions_dag = dag;
if (dag)
projection_query_info.filter_actions_dag = std::make_unique<ActionsDAG>(dag->clone());
auto projection_result_ptr = reader.estimateNumMarksToRead(
std::move(projection_parts),

View File

@ -25,14 +25,14 @@ std::shared_ptr<PartitionIdToMaxBlock> getMaxAddedBlocks(ReadFromMergeTree * rea
/// Additionally, for all the Filter steps, we collect filter conditions into filter_nodes.
struct QueryDAG
{
ActionsDAGPtr dag;
std::optional<ActionsDAG> dag;
const ActionsDAG::Node * filter_node = nullptr;
bool build(QueryPlan::Node & node);
private:
bool buildImpl(QueryPlan::Node & node, ActionsDAG::NodeRawConstPtrs & filter_nodes);
void appendExpression(const ActionsDAGPtr & expression);
void appendExpression(const ActionsDAG & expression);
};
struct ProjectionCandidate
@ -60,6 +60,6 @@ bool analyzeProjectionCandidate(
const SelectQueryInfo & query_info,
const ContextPtr & context,
const std::shared_ptr<PartitionIdToMaxBlock> & max_added_blocks,
const ActionsDAGPtr & dag);
const ActionsDAG * dag);
}

View File

@ -43,10 +43,10 @@ namespace
}
}
void logActionsDAG(const String & prefix, const ActionsDAGPtr & actions)
void logActionsDAG(const String & prefix, const ActionsDAG & actions)
{
if constexpr (debug_logging_enabled)
LOG_DEBUG(getLogger("redundantDistinct"), "{} :\n{}", prefix, actions->dumpDAG());
LOG_DEBUG(getLogger("redundantDistinct"), "{} :\n{}", prefix, actions.dumpDAG());
}
using DistinctColumns = std::set<std::string_view>;
@ -65,25 +65,25 @@ namespace
}
/// build actions DAG from stack of steps
ActionsDAGPtr buildActionsForPlanPath(std::vector<ActionsDAGPtr> & dag_stack)
std::optional<ActionsDAG> buildActionsForPlanPath(std::vector<const ActionsDAG *> & dag_stack)
{
if (dag_stack.empty())
return nullptr;
return {};
ActionsDAGPtr path_actions = dag_stack.back()->clone();
ActionsDAG path_actions = dag_stack.back()->clone();
dag_stack.pop_back();
while (!dag_stack.empty())
{
ActionsDAGPtr clone = dag_stack.back()->clone();
ActionsDAG clone = dag_stack.back()->clone();
logActionsDAG("DAG to merge", clone);
dag_stack.pop_back();
path_actions->mergeInplace(std::move(*clone));
path_actions.mergeInplace(std::move(clone));
}
return path_actions;
}
bool compareAggregationKeysWithDistinctColumns(
const Names & aggregation_keys, const DistinctColumns & distinct_columns, std::vector<std::vector<ActionsDAGPtr>> actions_chain)
const Names & aggregation_keys, const DistinctColumns & distinct_columns, std::vector<std::vector<const ActionsDAG *>> actions_chain)
{
logDebug("aggregation_keys", aggregation_keys);
logDebug("aggregation_keys size", aggregation_keys.size());
@ -93,7 +93,8 @@ namespace
std::set<String> source_columns;
for (auto & actions : actions_chain)
{
FindOriginalNodeForOutputName original_node_finder(buildActionsForPlanPath(actions));
auto tmp_actions = buildActionsForPlanPath(actions);
FindOriginalNodeForOutputName original_node_finder(*tmp_actions);
for (const auto & column : current_columns)
{
logDebug("distinct column name", column);
@ -131,10 +132,10 @@ namespace
return true;
if (const auto * const expr = typeid_cast<const ExpressionStep *>(step); expr)
return !expr->getExpression()->hasArrayJoin();
return !expr->getExpression().hasArrayJoin();
if (const auto * const filter = typeid_cast<const FilterStep *>(step); filter)
return !filter->getExpression()->hasArrayJoin();
return !filter->getExpression().hasArrayJoin();
if (typeid_cast<const LimitStep *>(step) || typeid_cast<const LimitByStep *>(step) || typeid_cast<const SortingStep *>(step)
|| typeid_cast<const WindowStep *>(step))
@ -152,8 +153,8 @@ namespace
const DistinctStep * distinct_step = typeid_cast<DistinctStep *>(distinct_node->step.get());
chassert(distinct_step);
std::vector<ActionsDAGPtr> dag_stack;
std::vector<std::vector<ActionsDAGPtr>> actions_chain;
std::vector<const ActionsDAG *> dag_stack;
std::vector<std::vector<const ActionsDAG *>> actions_chain;
const DistinctStep * inner_distinct_step = nullptr;
const IQueryPlanStep * aggregation_before_distinct = nullptr;
const QueryPlan::Node * node = distinct_node;
@ -182,9 +183,9 @@ namespace
}
if (const auto * const expr = typeid_cast<const ExpressionStep *>(current_step); expr)
dag_stack.push_back(expr->getExpression());
dag_stack.push_back(&expr->getExpression());
else if (const auto * const filter = typeid_cast<const FilterStep *>(current_step); filter)
dag_stack.push_back(filter->getExpression());
dag_stack.push_back(&filter->getExpression());
node = node->children.front();
if (inner_distinct_step = typeid_cast<DistinctStep *>(node->step.get()); inner_distinct_step)
@ -222,7 +223,7 @@ namespace
chassert(distinct_step);
const auto distinct_columns = getDistinctColumns(distinct_step);
std::vector<ActionsDAGPtr> dag_stack;
std::vector<const ActionsDAG *> dag_stack;
const DistinctStep * inner_distinct_step = nullptr;
const QueryPlan::Node * node = distinct_node;
while (!node->children.empty())
@ -235,9 +236,9 @@ namespace
}
if (const auto * const expr = typeid_cast<const ExpressionStep *>(current_step); expr)
dag_stack.push_back(expr->getExpression());
dag_stack.push_back(&expr->getExpression());
else if (const auto * const filter = typeid_cast<const FilterStep *>(current_step); filter)
dag_stack.push_back(filter->getExpression());
dag_stack.push_back(&filter->getExpression());
node = node->children.front();
inner_distinct_step = typeid_cast<DistinctStep *>(node->step.get());
@ -259,11 +260,11 @@ namespace
if (distinct_columns.size() != inner_distinct_columns.size())
return false;
ActionsDAGPtr path_actions;
ActionsDAG path_actions;
if (!dag_stack.empty())
{
/// build actions DAG to find original column names
path_actions = buildActionsForPlanPath(dag_stack);
path_actions = std::move(*buildActionsForPlanPath(dag_stack));
logActionsDAG("distinct pass: merged DAG", path_actions);
/// compare columns of two DISTINCTs

View File

@ -213,12 +213,12 @@ private:
logStep("checking for stateful function", node);
if (const auto * expr = typeid_cast<const ExpressionStep *>(step); expr)
{
if (expr->getExpression()->hasStatefulFunctions())
if (expr->getExpression().hasStatefulFunctions())
return false;
}
else if (const auto * filter = typeid_cast<const FilterStep *>(step); filter)
{
if (filter->getExpression()->hasStatefulFunctions())
if (filter->getExpression().hasStatefulFunctions())
return false;
}
else

Some files were not shown because too many files have changed in this diff Show More