From 3f9e9a7025bec607bb1e646556c415e3336c796d Mon Sep 17 00:00:00 2001 From: Kirill Ershov Date: Fri, 16 Apr 2021 23:18:39 +0300 Subject: [PATCH 001/161] Add INTERSECT and EXCEPT --- src/Common/ErrorCodes.cpp | 6 +- src/Interpreters/InterpreterFactory.cpp | 18 +- .../InterpreterIntersectOrExcept.cpp | 116 +++++++++++ .../InterpreterIntersectOrExcept.h | 35 ++++ src/Parsers/ASTIntersectOrExcept.cpp | 28 +++ src/Parsers/ASTIntersectOrExcept.h | 18 ++ src/Parsers/ParserIntersectOrExcept.cpp | 50 +++++ src/Parsers/ParserIntersectOrExcept.h | 14 ++ src/Parsers/ParserQueryWithOutput.cpp | 36 ++-- .../QueryPlan/IntersectOrExceptStep.cpp | 38 ++++ .../QueryPlan/IntersectOrExceptStep.h | 26 +++ .../Transforms/IntersectOrExceptTransform.cpp | 192 ++++++++++++++++++ .../Transforms/IntersectOrExceptTransform.h | 53 +++++ 13 files changed, 606 insertions(+), 24 deletions(-) create mode 100644 src/Interpreters/InterpreterIntersectOrExcept.cpp create mode 100644 src/Interpreters/InterpreterIntersectOrExcept.h create mode 100644 src/Parsers/ASTIntersectOrExcept.cpp create mode 100644 src/Parsers/ASTIntersectOrExcept.h create mode 100644 src/Parsers/ParserIntersectOrExcept.cpp create mode 100644 src/Parsers/ParserIntersectOrExcept.h create mode 100644 src/Processors/QueryPlan/IntersectOrExceptStep.cpp create mode 100644 src/Processors/QueryPlan/IntersectOrExceptStep.h create mode 100644 src/Processors/Transforms/IntersectOrExceptTransform.cpp create mode 100644 src/Processors/Transforms/IntersectOrExceptTransform.h diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index a2cd65137c0..0d1fd5bd7d8 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -533,7 +533,11 @@ M(564, INTERSERVER_SCHEME_DOESNT_MATCH) \ M(565, TOO_MANY_PARTITIONS) \ M(566, CANNOT_RMDIR) \ - \ + M(567, DUPLICATED_PART_UUIDS) \ + M(568, RAFT_ERROR) \ + M(569, MULTIPLE_COLUMNS_SERIALIZED_TO_SAME_PROTOBUF_FIELD) \ + M(570, DATA_TYPE_INCOMPATIBLE_WITH_PROTOBUF_FIELD) \ + M(571, INTERSECT_OR_EXCEPT_RESULT_STRUCTURES_MISMATCH) \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ M(1001, STD_EXCEPTION) \ diff --git a/src/Interpreters/InterpreterFactory.cpp b/src/Interpreters/InterpreterFactory.cpp index 15e4c52f040..e0f6479cc0e 100644 --- a/src/Interpreters/InterpreterFactory.cpp +++ b/src/Interpreters/InterpreterFactory.cpp @@ -1,14 +1,17 @@ #include #include #include -#include -#include #include +#include #include #include +#include #include #include +#include +#include #include +#include #include #include #include @@ -24,11 +27,9 @@ #include #include #include -#include -#include #include -#include #include +#include #include #include @@ -44,9 +45,11 @@ #include #include #include +#include #include #include #include +#include #include #include #include @@ -65,7 +68,6 @@ #include #include #include -#include #include #include @@ -109,6 +111,10 @@ std::unique_ptr InterpreterFactory::get(ASTPtr & query, Context & ProfileEvents::increment(ProfileEvents::SelectQuery); return std::make_unique(query, context, options); } + else if (query->as()) + { + return std::make_unique(query, context); + } else if (query->as()) { ProfileEvents::increment(ProfileEvents::InsertQuery); diff --git a/src/Interpreters/InterpreterIntersectOrExcept.cpp b/src/Interpreters/InterpreterIntersectOrExcept.cpp new file mode 100644 index 00000000000..c85bd29e16f --- /dev/null +++ b/src/Interpreters/InterpreterIntersectOrExcept.cpp @@ -0,0 +1,116 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int INTERSECT_OR_EXCEPT_RESULT_STRUCTURES_MISMATCH; +} + +InterpreterIntersectOrExcept::InterpreterIntersectOrExcept(const ASTPtr & query_ptr_, ContextPtr context_) + : query_ptr(query_ptr_), context(Context::createCopy(context_)) +{ + ASTIntersectOrExcept * ast = query_ptr->as(); + size_t num_children = ast->children.size(); + nested_interpreters.resize(num_children); + for (size_t i = 0; i < num_children; ++i) + { + nested_interpreters[i] = buildCurrentChildInterpreter(ast->children[i]); + } + + Blocks headers(num_children); + for (size_t query_num = 0; query_num < num_children; ++query_num) + headers[query_num] = nested_interpreters[query_num]->getSampleBlock(); + + result_header = getCommonHeader(headers); +} + + +Block InterpreterIntersectOrExcept::getCommonHeader(const Blocks & headers) +{ + size_t num_selects = headers.size(); + Block common_header = headers.front(); + size_t num_columns = common_header.columns(); + + for (size_t query_num = 1; query_num < num_selects; ++query_num) + { + if (headers[query_num].columns() != num_columns) + throw Exception( + "Different number of columns in " + + toString(query_ptr->as()->is_except ? "EXCEPT" : "INTERSECT") + + " elements:\n" + common_header.dumpNames() + "\nand\n" + + headers[query_num].dumpNames() + "\n", + ErrorCodes::INTERSECT_OR_EXCEPT_RESULT_STRUCTURES_MISMATCH); + } + + std::vector columns(num_selects); + + for (size_t column_num = 0; column_num < num_columns; ++column_num) + { + for (size_t i = 0; i < num_selects; ++i) + columns[i] = &headers[i].getByPosition(column_num); + + ColumnWithTypeAndName & result_elem = common_header.getByPosition(column_num); + result_elem = getLeastSuperColumn(columns); + } + + return common_header; +} + + +std::unique_ptr +InterpreterIntersectOrExcept::buildCurrentChildInterpreter(const ASTPtr & ast_ptr_) +{ + if (ast_ptr_->as()) + return std::make_unique(ast_ptr_, context, SelectQueryOptions()); + else + return std::make_unique(ast_ptr_, context, SelectQueryOptions()); +} + +void InterpreterIntersectOrExcept::buildQueryPlan(QueryPlan & query_plan) +{ + size_t num_plans = nested_interpreters.size(); + + std::vector> plans(num_plans); + DataStreams data_streams(num_plans); + + for (size_t i = 0; i < num_plans; ++i) + { + plans[i] = std::make_unique(); + nested_interpreters[i]->buildQueryPlan(*plans[i]); + data_streams[i] = plans[i]->getCurrentDataStream(); + } + + auto max_threads = context->getSettingsRef().max_threads; + auto step = std::make_unique( + query_ptr->as()->is_except, std::move(data_streams), result_header, max_threads); + query_plan.unitePlans(std::move(step), std::move(plans)); +} + +BlockIO InterpreterIntersectOrExcept::execute() +{ + BlockIO res; + + QueryPlan query_plan; + buildQueryPlan(query_plan); + + auto pipeline = query_plan.buildQueryPipeline( + QueryPlanOptimizationSettings::fromContext(context), + BuildQueryPipelineSettings::fromContext(context)); + + res.pipeline = std::move(*pipeline); + res.pipeline.addInterpreterContext(context); + + return res; +} +} diff --git a/src/Interpreters/InterpreterIntersectOrExcept.h b/src/Interpreters/InterpreterIntersectOrExcept.h new file mode 100644 index 00000000000..0069dc02f1d --- /dev/null +++ b/src/Interpreters/InterpreterIntersectOrExcept.h @@ -0,0 +1,35 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +class Context; +class InterpreterSelectQuery; +class QueryPlan; + +class InterpreterIntersectOrExcept : public IInterpreter +{ +public: + InterpreterIntersectOrExcept(const ASTPtr & query_ptr_, ContextPtr context_); + + /// Builds QueryPlan for current query. + virtual void buildQueryPlan(QueryPlan & query_plan); + + BlockIO execute() override; + +private: + ASTPtr query_ptr; + ContextPtr context; + Block result_header; + std::vector> nested_interpreters; + Block getCommonHeader(const Blocks & headers); + + std::unique_ptr + buildCurrentChildInterpreter(const ASTPtr & ast_ptr_); +}; + +} diff --git a/src/Parsers/ASTIntersectOrExcept.cpp b/src/Parsers/ASTIntersectOrExcept.cpp new file mode 100644 index 00000000000..073d63963a9 --- /dev/null +++ b/src/Parsers/ASTIntersectOrExcept.cpp @@ -0,0 +1,28 @@ +#include +#include + +namespace DB +{ + +ASTPtr ASTIntersectOrExcept::clone() const +{ + auto res = std::make_shared(*this); + res->children.clear(); + res->children.push_back(children[0]->clone()); + res->children.push_back(children[1]->clone()); + res->is_except = is_except; + cloneOutputOptions(*res); + return res; +} + +void ASTIntersectOrExcept::formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const +{ + children[0]->formatImpl(settings, state, frame); + std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' '); + settings.ostr << settings.nl_or_ws << indent_str << (settings.hilite ? hilite_keyword : "") + << (is_except ? "EXCEPT" : "INTERSECT ") + << (settings.hilite ? hilite_none : "") << settings.nl_or_ws; + children[1]->formatImpl(settings, state, frame); +} + +} diff --git a/src/Parsers/ASTIntersectOrExcept.h b/src/Parsers/ASTIntersectOrExcept.h new file mode 100644 index 00000000000..a02cb9f7d77 --- /dev/null +++ b/src/Parsers/ASTIntersectOrExcept.h @@ -0,0 +1,18 @@ +#pragma once + +#include + + +namespace DB +{ + +class ASTIntersectOrExcept : public ASTQueryWithOutput +{ +public: + String getID(char) const override { return is_except ? "Except" : "Intersect"; } + ASTPtr clone() const override; + void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; + bool is_except; +}; + +} diff --git a/src/Parsers/ParserIntersectOrExcept.cpp b/src/Parsers/ParserIntersectOrExcept.cpp new file mode 100644 index 00000000000..a82b8c2b06b --- /dev/null +++ b/src/Parsers/ParserIntersectOrExcept.cpp @@ -0,0 +1,50 @@ +#include +#include +#include +#include +#include +#include + +namespace DB +{ +bool ParserIntersectOrExcept::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + ParserKeyword intersect_keyword("INTERSECT"); + ParserKeyword except_keyword("EXCEPT"); + ASTPtr left_node; + ASTPtr right_node; + + auto ast = std::make_shared(); + ast->is_except = false; + + if (!ParserSelectQuery().parse(pos, left_node, expected) && !ParserSubquery().parse(pos, left_node, expected)) + return false; + + if (!intersect_keyword.ignore(pos)) + { + if (!except_keyword.ignore(pos)) + { + return false; + } + else + { + ast->is_except = true; + } + } + + if (!ParserSelectQuery().parse(pos, right_node, expected) && !ParserSubquery().parse(pos, right_node, expected)) + return false; + + if (const auto * ast_subquery = left_node->as()) + left_node = ast_subquery->children.at(0); + if (const auto * ast_subquery = right_node->as()) + right_node = ast_subquery->children.at(0); + + ast->children.push_back(left_node); + ast->children.push_back(right_node); + + node = ast; + return true; +} + +} diff --git a/src/Parsers/ParserIntersectOrExcept.h b/src/Parsers/ParserIntersectOrExcept.h new file mode 100644 index 00000000000..61cc74cf0a9 --- /dev/null +++ b/src/Parsers/ParserIntersectOrExcept.h @@ -0,0 +1,14 @@ +#pragma once +#include + + +namespace DB +{ +class ParserIntersectOrExcept : public IParserBase +{ +protected: + const char * getName() const override { return "INTERSECT or EXCEPT"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + +} diff --git a/src/Parsers/ParserQueryWithOutput.cpp b/src/Parsers/ParserQueryWithOutput.cpp index d5aa1e47533..35355b29ebf 100644 --- a/src/Parsers/ParserQueryWithOutput.cpp +++ b/src/Parsers/ParserQueryWithOutput.cpp @@ -1,36 +1,37 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include #include #include #include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include #include #include #include #include #include -#include +#include +#include +#include +#include #include - namespace DB { bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ParserShowTablesQuery show_tables_p; + ParserIntersectOrExcept intersect_p; ParserSelectWithUnionQuery select_p; ParserTablePropertiesQuery table_p; ParserDescribeTableQuery describe_table_p; @@ -54,6 +55,7 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec bool parsed = explain_p.parse(pos, query, expected) + || intersect_p.parse(pos, query, expected) || select_p.parse(pos, query, expected) || show_create_access_entity_p.parse(pos, query, expected) /// should be before `show_tables_p` || show_tables_p.parse(pos, query, expected) diff --git a/src/Processors/QueryPlan/IntersectOrExceptStep.cpp b/src/Processors/QueryPlan/IntersectOrExceptStep.cpp new file mode 100644 index 00000000000..d0a820339d7 --- /dev/null +++ b/src/Processors/QueryPlan/IntersectOrExceptStep.cpp @@ -0,0 +1,38 @@ +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +IntersectOrExceptStep::IntersectOrExceptStep(bool is_except_, DataStreams input_streams_, Block result_header, size_t max_threads_) + : is_except(is_except_), header(std::move(result_header)), max_threads(max_threads_) +{ + input_streams = std::move(input_streams_); + output_stream = DataStream{.header = header}; +} + +QueryPipelinePtr IntersectOrExceptStep::updatePipeline(QueryPipelines pipelines, const BuildQueryPipelineSettings & ) +{ + auto pipeline = std::make_unique(); + QueryPipelineProcessorsCollector collector(*pipeline, this); + + pipelines[0]->addTransform(std::make_shared(header, pipelines[0]->getNumStreams(), 1)); + pipelines[1]->addTransform(std::make_shared(header, pipelines[1]->getNumStreams(), 1)); + + *pipeline = QueryPipeline::unitePipelines(std::move(pipelines), max_threads); + pipeline->addTransform(std::make_shared(is_except, header)); + + processors = collector.detachProcessors(); + return pipeline; +} + +void IntersectOrExceptStep::describePipeline(FormatSettings & settings) const +{ + IQueryPlanStep::describePipeline(processors, settings); +} + +} diff --git a/src/Processors/QueryPlan/IntersectOrExceptStep.h b/src/Processors/QueryPlan/IntersectOrExceptStep.h new file mode 100644 index 00000000000..d2b515bb1c4 --- /dev/null +++ b/src/Processors/QueryPlan/IntersectOrExceptStep.h @@ -0,0 +1,26 @@ +#pragma once +#include + +namespace DB +{ + +class IntersectOrExceptStep : public IQueryPlanStep +{ +public: + /// max_threads is used to limit the number of threads for result pipeline. + IntersectOrExceptStep(bool is_except_, DataStreams input_streams_, Block result_header, size_t max_threads_ = 0); + + String getName() const override { return is_except ? "Except" : "Intersect"; } + + QueryPipelinePtr updatePipeline(QueryPipelines pipelines, const BuildQueryPipelineSettings & settings) override; + + void describePipeline(FormatSettings & settings) const override; +private: + bool is_except; + Block header; + size_t max_threads; + Processors processors; +}; + +} + diff --git a/src/Processors/Transforms/IntersectOrExceptTransform.cpp b/src/Processors/Transforms/IntersectOrExceptTransform.cpp new file mode 100644 index 00000000000..199498bf762 --- /dev/null +++ b/src/Processors/Transforms/IntersectOrExceptTransform.cpp @@ -0,0 +1,192 @@ +#include + +namespace DB +{ +IntersectOrExceptTransform::IntersectOrExceptTransform(bool is_except_, const Block & header_) + : IProcessor(InputPorts(2, header_), {header_}), is_except(is_except_), output(outputs.front()) +{ + const Names & columns = header_.getNames(); + size_t num_columns = columns.empty() ? header_.columns() : columns.size(); + + key_columns_pos.reserve(columns.size()); + for (size_t i = 0; i < num_columns; ++i) + { + auto pos = columns.empty() ? i : header_.getPositionByName(columns[i]); + + const auto & col = header_.getByPosition(pos).column; + + if (!(col && isColumnConst(*col))) + key_columns_pos.emplace_back(pos); + } +} + +IntersectOrExceptTransform::Status IntersectOrExceptTransform::prepare() +{ + /// Check can output. + if (output.isFinished()) + { + for (auto & in : inputs) + in.close(); + return Status::Finished; + } + + if (!output.canPush()) + { + if (inputs.front().isFinished()) + { + inputs.back().setNotNeeded(); + } + else + { + inputs.front().setNotNeeded(); + } + return Status::PortFull; + } + + /// Output if has data. + if (current_output_chunk) + { + output.push(std::move(current_output_chunk)); + } + + if (push_empty_chunk) + { + output.push(std::move(empty_chunk)); + push_empty_chunk = false; + } + + if (finished_second_input) + { + if (inputs.front().isFinished()) + { + output.finish(); + return Status::Finished; + } + } + else if (inputs.back().isFinished()) + { + finished_second_input = true; + } + + InputPort & input = finished_second_input ? inputs.front() : inputs.back(); + + /// Check can input. + if (!has_input) + { + input.setNeeded(); + if (!input.hasData()) + { + return Status::NeedData; + } + + current_input_chunk = input.pull(); + has_input = true; + } + + return Status::Ready; +} + +void IntersectOrExceptTransform::work() +{ + if (!finished_second_input) + { + accumulate(std::move(current_input_chunk)); + } + else + { + filter(current_input_chunk); + current_output_chunk = std::move(current_input_chunk); + } + + has_input = false; +} + +template +void IntersectOrExceptTransform::addToSet(Method & method, const ColumnRawPtrs & columns, size_t rows, SetVariants & variants) const +{ + typename Method::State state(columns, key_sizes, nullptr); + + for (size_t i = 0; i < rows; ++i) + { + state.emplaceKey(method.data, i, variants.string_pool); + } +} + +template +size_t IntersectOrExceptTransform::buildFilter( + Method & method, const ColumnRawPtrs & columns, IColumn::Filter & filter, size_t rows, SetVariants & variants) const +{ + typename Method::State state(columns, key_sizes, nullptr); + size_t new_rows_num = 0; + + for (size_t i = 0; i < rows; ++i) + { + auto find_result = state.findKey(method.data, i, variants.string_pool); + filter[i] = is_except ? !find_result.isFound() : find_result.isFound(); + if (filter[i]) + ++new_rows_num; + } + return new_rows_num; +} + +void IntersectOrExceptTransform::accumulate(Chunk chunk) +{ + auto num_rows = chunk.getNumRows(); + auto columns = chunk.detachColumns(); + + ColumnRawPtrs column_ptrs; + column_ptrs.reserve(key_columns_pos.size()); + for (auto pos : key_columns_pos) + column_ptrs.emplace_back(columns[pos].get()); + + if (data.empty()) + data.init(SetVariants::chooseMethod(column_ptrs, key_sizes)); + + switch (data.type) + { + case SetVariants::Type::EMPTY: + break; +#define M(NAME) \ + case SetVariants::Type::NAME: \ + addToSet(*data.NAME, column_ptrs, num_rows, data); \ + break; + APPLY_FOR_SET_VARIANTS(M) +#undef M + } +} + +void IntersectOrExceptTransform::filter(Chunk & chunk) +{ + auto num_rows = chunk.getNumRows(); + auto columns = chunk.detachColumns(); + + ColumnRawPtrs column_ptrs; + column_ptrs.reserve(key_columns_pos.size()); + for (auto pos : key_columns_pos) + column_ptrs.emplace_back(columns[pos].get()); + + if (data.empty()) + data.init(SetVariants::chooseMethod(column_ptrs, key_sizes)); + + IColumn::Filter filter(num_rows); + + size_t new_rows_num = 0; + switch (data.type) + { + case SetVariants::Type::EMPTY: + break; +#define M(NAME) \ + case SetVariants::Type::NAME: \ + new_rows_num = buildFilter(*data.NAME, column_ptrs, filter, num_rows, data); \ + break; + APPLY_FOR_SET_VARIANTS(M) +#undef M + } + + for (auto & column : columns) + column = column->filter(filter, -1); + + chunk.setColumns(std::move(columns), new_rows_num); +} + +} diff --git a/src/Processors/Transforms/IntersectOrExceptTransform.h b/src/Processors/Transforms/IntersectOrExceptTransform.h new file mode 100644 index 00000000000..ebe73fdeb26 --- /dev/null +++ b/src/Processors/Transforms/IntersectOrExceptTransform.h @@ -0,0 +1,53 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +class IntersectOrExceptTransform : public IProcessor +{ +public: + IntersectOrExceptTransform(bool is_except_, const Block & header_); + + Status prepare() override; + void work() override; + + String getName() const override { return is_except ? "Except" : "Intersect"; } + +private: + + bool push_empty_chunk = false; + Chunk empty_chunk; + + bool is_except; + ColumnNumbers key_columns_pos; + SetVariants data; + Sizes key_sizes; + Chunk current_input_chunk; + Chunk current_output_chunk; + bool finished_second_input = false; + bool has_input = false; + OutputPort & output; + + void accumulate(Chunk chunk); + void filter(Chunk & chunk); + template + void addToSet( + Method & method, + const ColumnRawPtrs & key_columns, + size_t rows, + SetVariants & variants) const; + + template + size_t buildFilter( + Method & method, + const ColumnRawPtrs & columns, + IColumn::Filter & filter, + size_t rows, + SetVariants & variants) const; +}; + +} From 566b5335ddc505d66ad1fa4db95497751f692cd0 Mon Sep 17 00:00:00 2001 From: Kirill Ershov Date: Sun, 30 May 2021 20:58:08 +0300 Subject: [PATCH 002/161] Add ANY, ALL --- src/Parsers/ExpressionListParsers.cpp | 129 +++++++++++++++++++++++++- src/Parsers/ExpressionListParsers.h | 16 +++- 2 files changed, 141 insertions(+), 4 deletions(-) diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index eec79edc05e..e9705843b6e 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -9,8 +9,7 @@ namespace DB -{ - +{ const char * ParserMultiplicativeExpression::operators[] = { @@ -56,6 +55,19 @@ const char * ParserComparisonExpression::operators[] = nullptr }; +const char * ParserComparisonWithSubqueryExpression::operators[] = +{ + "==", "equals", + "!=", "notEquals", + "<>", "notEquals", + "<=", "lessOrEquals", + ">=", "greaterOrEquals", + "<", "less", + ">", "greater", + "=", "equals", + nullptr +}; + const char * ParserComparisonExpression::overlapping_operators_to_skip[] = { "IN PARTITION", @@ -359,6 +371,119 @@ bool ParserBetweenExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & exp return true; } +bool ParserComparisonWithSubqueryExpression::modifySubquery(String operator_name, ASTPtr subquery_node, bool is_any) +{ + ASTPtr select_with_union_node = subquery_node->children[0]; + if (select_with_union_node->children[0]->children.size() != 1) + return false; + ASTPtr select_node = select_with_union_node->children[0]->children[0]; + ASTPtr exp_list = select_node->children[0]; + auto function = std::make_shared(); + function->arguments = exp_list; + function->children.push_back(exp_list); + + ASTPtr new_exp_list = std::make_shared(); + new_exp_list->children.push_back(function); + + if (operator_name == "greaterOrEquals" || operator_name == "greater") + { + function->name = is_any ? "min" : "max"; + select_node->children[0] = new_exp_list; + return true; + } + + if (operator_name == "lessOrEquals" || operator_name == "less") + { + function->name = is_any ? "max" : "min"; + select_node->children[0] = new_exp_list; + return true; + } + return false; +} + +bool ParserComparisonWithSubqueryExpression::addFunctionIn(String operator_name, ASTPtr & node, bool is_any) +{ + + auto function_in = std::make_shared(); + auto exp_list_in = std::make_shared(); + exp_list_in->children.push_back(node->children[0]->children[0]); + exp_list_in->children.push_back(node->children[0]->children[1]); + function_in->name = "in"; + function_in->children.push_back(exp_list_in); + function_in->arguments = exp_list_in; + + if (operator_name == "equals" && is_any) + { + node = function_in; + return true; + } + + if (operator_name == "notEquals" && !is_any) + { + auto function_not = std::make_shared(); + auto exp_list_not = std::make_shared(); + exp_list_not->children.push_back(function_in); + function_not->name = "not"; + function_not->children.push_back(exp_list_not); + function_not->arguments = exp_list_not; + node = function_not; + return true; + } + return false; +} + +bool ParserComparisonWithSubqueryExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + Pos begin = pos; + ASTPtr elem; + if (!elem_parser.parse(pos, elem, expected)) + return next_parser.parse(pos, node, expected); + + /// try to find any of the valid operators + const char ** it; + for (it = operators; *it; it += 2) + if (parseOperator(pos, *it, expected)) + break; + + if (!*it) + { + pos = begin; + return next_parser.parse(pos, node, expected); + } + + bool is_any = true; + if (!ParserKeyword("ANY").ignore(pos, expected)) + { + is_any = false; + if (!ParserKeyword("ALL").ignore(pos, expected)) + { + pos = begin; + return next_parser.parse(pos, node, expected); + } + } + + ASTPtr subquery_node; + if (!ParserSubquery().parse(pos, subquery_node, expected)) + return false; + + /// the first argument of the function is the previous element, the second is the next one + String operator_name = it[1]; + + /// the function corresponding to the operator + auto function = std::make_shared(); + + /// function arguments + auto exp_list = std::make_shared(); + exp_list->children.push_back(elem); + exp_list->children.push_back(subquery_node); + + function->name = operator_name; + function->arguments = exp_list; + function->children.push_back(exp_list); + node = function; + return modifySubquery(operator_name, subquery_node, is_any) || addFunctionIn(operator_name, node, is_any); +} + bool ParserTernaryOperatorExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ParserToken symbol1(TokenType::QuestionMark); diff --git a/src/Parsers/ExpressionListParsers.h b/src/Parsers/ExpressionListParsers.h index 3a65141d751..acdeac29543 100644 --- a/src/Parsers/ExpressionListParsers.h +++ b/src/Parsers/ExpressionListParsers.h @@ -364,14 +364,26 @@ protected: } }; +class ParserComparisonWithSubqueryExpression : public IParserBase +{ +private: + static const char * operators[]; + ParserComparisonExpression next_parser; + ParserConcatExpression elem_parser; + static bool addFunctionIn(String operator_name, ASTPtr & node, bool is_any); + static bool modifySubquery(String operator_name, ASTPtr subquery_node, bool is_any); +protected: + const char * getName() const override { return "comparison with ANY/ALL expression"; } + + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; /** Parser for nullity checking with IS (NOT) NULL. */ class ParserNullityChecking : public IParserBase { private: - ParserComparisonExpression elem_parser; - + ParserComparisonWithSubqueryExpression elem_parser; protected: const char * getName() const override { return "nullity checking"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; From 1c2e9ba0038e5c18de10530e9f591833b5e0bd62 Mon Sep 17 00:00:00 2001 From: Kirill Ershov Date: Wed, 2 Jun 2021 02:18:15 +0300 Subject: [PATCH 003/161] Fix bug in parser --- src/Parsers/ExpressionListParsers.cpp | 6 +++--- src/Parsers/ExpressionListParsers.h | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index e9705843b6e..a228dc6617b 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -9,7 +9,7 @@ namespace DB -{ +{ const char * ParserMultiplicativeExpression::operators[] = { @@ -447,8 +447,8 @@ bool ParserComparisonWithSubqueryExpression::parseImpl(Pos & pos, ASTPtr & node, if (!*it) { - pos = begin; - return next_parser.parse(pos, node, expected); + node = elem; + return true; } bool is_any = true; diff --git a/src/Parsers/ExpressionListParsers.h b/src/Parsers/ExpressionListParsers.h index acdeac29543..29335e72b2a 100644 --- a/src/Parsers/ExpressionListParsers.h +++ b/src/Parsers/ExpressionListParsers.h @@ -369,7 +369,7 @@ class ParserComparisonWithSubqueryExpression : public IParserBase private: static const char * operators[]; ParserComparisonExpression next_parser; - ParserConcatExpression elem_parser; + ParserBetweenExpression elem_parser; static bool addFunctionIn(String operator_name, ASTPtr & node, bool is_any); static bool modifySubquery(String operator_name, ASTPtr subquery_node, bool is_any); protected: From 289c5d3ad680c6e07ef189eed9ea30417acdac65 Mon Sep 17 00:00:00 2001 From: Kirill Ershov Date: Fri, 4 Jun 2021 05:56:02 +0300 Subject: [PATCH 004/161] Add SingleValueOrNull aggregate function --- .../AggregateFunctionMinMaxAny.h | 58 ++++++++++++++++++- .../AggregateFunctionSingleValueOrNull.cpp | 27 +++++++++ .../registerAggregateFunctions.cpp | 2 + 3 files changed, 86 insertions(+), 1 deletion(-) create mode 100644 src/AggregateFunctions/AggregateFunctionSingleValueOrNull.cpp diff --git a/src/AggregateFunctions/AggregateFunctionMinMaxAny.h b/src/AggregateFunctions/AggregateFunctionMinMaxAny.h index de1f4fad296..6791b98e7e3 100644 --- a/src/AggregateFunctions/AggregateFunctionMinMaxAny.h +++ b/src/AggregateFunctions/AggregateFunctionMinMaxAny.h @@ -5,8 +5,10 @@ #include #include +#include #include #include +#include #include #include @@ -628,6 +630,60 @@ struct AggregateFunctionAnyLastData : Data static const char * name() { return "anyLast"; } }; +template +struct AggregateFunctionSingleValueOrNullData : Data +{ + using Self = AggregateFunctionSingleValueOrNullData; + + bool first_value = true; + bool is_null = false; + + bool changeIfBetter(const IColumn & column, size_t row_num, Arena * arena) + { + if (first_value) + { + first_value = false; + this->change(column, row_num, arena); + return true; + } + else if (!this->isEqualTo(column, row_num)) + { + is_null = true; + } + return false; + } + + bool changeIfBetter(const Self & to, Arena * arena) + { + if (first_value) + { + first_value = false; + this->change(to, arena); + return true; + } + else if (!this->isEqualTo(to)) + { + is_null = true; + } + return false; + } + + void insertResultInto(IColumn & to) const + { + if (is_null || first_value) + { + to.insertDefault(); + } + else + { + ColumnNullable & col = typeid_cast(to); + col.getNullMapColumn().insertDefault(); + this->Data::insertResultInto(col.getNestedColumn()); + } + } + + static const char * name() { return "singleValueOrNull"; } +}; /** Implement 'heavy hitters' algorithm. * Selects most frequent value if its frequency is more than 50% in each thread of execution. @@ -722,7 +778,7 @@ public: DataTypePtr getReturnType() const override { - return type; + return Data::name() == "singleValueOrNull" ? std::make_shared(type) : type; } void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override diff --git a/src/AggregateFunctions/AggregateFunctionSingleValueOrNull.cpp b/src/AggregateFunctions/AggregateFunctionSingleValueOrNull.cpp new file mode 100644 index 00000000000..cd897dfcf6e --- /dev/null +++ b/src/AggregateFunctions/AggregateFunctionSingleValueOrNull.cpp @@ -0,0 +1,27 @@ +#include +#include +#include +#include "registerAggregateFunctions.h" + + +namespace DB +{ +struct Settings; + +namespace +{ + +AggregateFunctionPtr createAggregateFunctionSingleValueOrNull(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings * settings) +{ + return AggregateFunctionPtr(createAggregateFunctionSingleValue(name, argument_types, parameters, settings)); +} + +} + +void registerAggregateFunctionSingleValueOrNull(AggregateFunctionFactory & factory) +{ + factory.registerFunction("singleValueOrNull", createAggregateFunctionSingleValueOrNull); +} + + +} diff --git a/src/AggregateFunctions/registerAggregateFunctions.cpp b/src/AggregateFunctions/registerAggregateFunctions.cpp index db6d8eb75bc..6fb373a1ce8 100644 --- a/src/AggregateFunctions/registerAggregateFunctions.cpp +++ b/src/AggregateFunctions/registerAggregateFunctions.cpp @@ -48,6 +48,7 @@ void registerAggregateFunctionRankCorrelation(AggregateFunctionFactory &); void registerAggregateFunctionMannWhitney(AggregateFunctionFactory &); void registerAggregateFunctionWelchTTest(AggregateFunctionFactory &); void registerAggregateFunctionStudentTTest(AggregateFunctionFactory &); +void registerAggregateFunctionSingleValueOrNull(AggregateFunctionFactory &); class AggregateFunctionCombinatorFactory; void registerAggregateFunctionCombinatorIf(AggregateFunctionCombinatorFactory &); @@ -111,6 +112,7 @@ void registerAggregateFunctions() registerAggregateFunctionMannWhitney(factory); registerAggregateFunctionWelchTTest(factory); registerAggregateFunctionStudentTTest(factory); + registerAggregateFunctionSingleValueOrNull(factory); registerWindowFunctions(factory); From a524feb73272b172c03a22c1540fd117ea93c4a9 Mon Sep 17 00:00:00 2001 From: Kirill Ershov Date: Thu, 3 Jun 2021 05:19:07 +0300 Subject: [PATCH 005/161] Rewrite ALL/ANY parser --- src/Parsers/ExpressionListParsers.cpp | 282 ++++++++++++++------------ src/Parsers/ExpressionListParsers.h | 21 +- 2 files changed, 158 insertions(+), 145 deletions(-) diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index a228dc6617b..acb2e35a84e 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -1,8 +1,12 @@ #include +#include #include #include #include +#include +#include +#include #include #include #include @@ -55,19 +59,6 @@ const char * ParserComparisonExpression::operators[] = nullptr }; -const char * ParserComparisonWithSubqueryExpression::operators[] = -{ - "==", "equals", - "!=", "notEquals", - "<>", "notEquals", - "<=", "lessOrEquals", - ">=", "greaterOrEquals", - "<", "less", - ">", "greater", - "=", "equals", - nullptr -}; - const char * ParserComparisonExpression::overlapping_operators_to_skip[] = { "IN PARTITION", @@ -180,6 +171,158 @@ static bool parseOperator(IParser::Pos & pos, const char * op, Expected & expect } } +enum class SubqueryFunctionType +{ + NONE, + ANY, + ALL +}; + +static bool modifyAST(String operator_name, std::shared_ptr & function, SubqueryFunctionType type) +{ + // = ANY --> IN, != ALL --> NOT IN + if ((operator_name == "equals" && type == SubqueryFunctionType::ANY) + || (operator_name == "notEquals" && type == SubqueryFunctionType::ALL)) + { + function->name = "in"; + if (operator_name == "notEquals") + { + auto function_not = std::make_shared(); + auto exp_list_not = std::make_shared(); + exp_list_not->children.push_back(function); + function_not->name = "not"; + function_not->children.push_back(exp_list_not); + function_not->arguments = exp_list_not; + function = function_not; + } + return true; + } + + // subquery --> (SELECT aggregate_function(*) FROM subquery) + auto aggregate_function = std::make_shared(); + auto aggregate_function_exp_list = std::make_shared(); + aggregate_function_exp_list ->children.push_back(std::make_shared()); + aggregate_function->arguments = aggregate_function_exp_list; + aggregate_function->children.push_back(aggregate_function_exp_list); + + ASTPtr subquery_node = function->children[0]->children[1]; + auto select_query = std::make_shared(); + auto tables_in_select = std::make_shared(); + auto tables_in_select_element = std::make_shared(); + auto table_expression = std::make_shared(); + table_expression->subquery = subquery_node; + table_expression->children.push_back(subquery_node); + tables_in_select_element->table_expression = table_expression; + tables_in_select_element->children.push_back(table_expression); + tables_in_select->children.push_back(tables_in_select_element); + auto select_exp_list = std::make_shared(); + select_exp_list->children.push_back(aggregate_function); + select_query->children.push_back(select_exp_list); + select_query->children.push_back(tables_in_select); + select_query->setExpression(ASTSelectQuery::Expression::SELECT, std::move(select_exp_list)); + select_query->setExpression(ASTSelectQuery::Expression::TABLES, std::move(tables_in_select)); + + auto select_with_union_query = std::make_shared(); + auto list_of_selects = std::make_shared(); + list_of_selects->children.push_back(select_query); + select_with_union_query->list_of_selects = list_of_selects; + select_with_union_query->children.push_back(select_with_union_query->list_of_selects); + + auto new_subquery = std::make_shared(); + new_subquery->children.push_back(select_with_union_query); + function->children[0]->children.pop_back(); + function->children[0]->children.push_back(new_subquery); + + if (operator_name == "greaterOrEquals" || operator_name == "greater") + { + aggregate_function->name = type == SubqueryFunctionType::ANY ? "min" : "max"; + return true; + } + if (operator_name == "lessOrEquals" || operator_name == "less") + { + aggregate_function->name = type == SubqueryFunctionType::ANY ? "max" : "min"; + return true; + } + if (operator_name == "equals" || operator_name == "notEquals") + { + aggregate_function->name = "singleValueOrNull"; + return true; + } + return false; +} + +bool ParserComparisonExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + bool first = true; + + auto current_depth = pos.depth; + while (true) + { + if (first) + { + ASTPtr elem; + if (!elem_parser.parse(pos, elem, expected)) + return false; + + node = elem; + first = false; + } + else + { + /// try to find any of the valid operators + const char ** it; + Expected stub; + for (it = overlapping_operators_to_skip; *it; ++it) + if (ParserKeyword{*it}.checkWithoutMoving(pos, stub)) + break; + + if (*it) + break; + + for (it = operators; *it; it += 2) + if (parseOperator(pos, *it, expected)) + break; + + if (!*it) + break; + + /// the function corresponding to the operator + auto function = std::make_shared(); + + /// function arguments + auto exp_list = std::make_shared(); + + ASTPtr elem; + SubqueryFunctionType subquery_function_type = SubqueryFunctionType::NONE; + if (ParserKeyword("ANY").ignore(pos, expected)) + subquery_function_type = SubqueryFunctionType::ANY; + else if (ParserKeyword("ALL").ignore(pos, expected)) + subquery_function_type = SubqueryFunctionType::ALL; + else if (!elem_parser.parse(pos, elem, expected)) + return false; + + if (subquery_function_type != SubqueryFunctionType::NONE && !ParserSubquery().parse(pos, elem, expected)) + return false; + + /// the first argument of the function is the previous element, the second is the next one + function->name = it[1]; + function->arguments = exp_list; + function->children.push_back(exp_list); + + exp_list->children.push_back(node); + exp_list->children.push_back(elem); + + if (subquery_function_type != SubqueryFunctionType::NONE && !modifyAST(function->name, function, subquery_function_type)) + return false; + + pos.increaseDepth(); + node = function; + } + } + + pos.depth = current_depth; + return true; +} bool ParserLeftAssociativeBinaryOperatorList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { @@ -371,119 +514,6 @@ bool ParserBetweenExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & exp return true; } -bool ParserComparisonWithSubqueryExpression::modifySubquery(String operator_name, ASTPtr subquery_node, bool is_any) -{ - ASTPtr select_with_union_node = subquery_node->children[0]; - if (select_with_union_node->children[0]->children.size() != 1) - return false; - ASTPtr select_node = select_with_union_node->children[0]->children[0]; - ASTPtr exp_list = select_node->children[0]; - auto function = std::make_shared(); - function->arguments = exp_list; - function->children.push_back(exp_list); - - ASTPtr new_exp_list = std::make_shared(); - new_exp_list->children.push_back(function); - - if (operator_name == "greaterOrEquals" || operator_name == "greater") - { - function->name = is_any ? "min" : "max"; - select_node->children[0] = new_exp_list; - return true; - } - - if (operator_name == "lessOrEquals" || operator_name == "less") - { - function->name = is_any ? "max" : "min"; - select_node->children[0] = new_exp_list; - return true; - } - return false; -} - -bool ParserComparisonWithSubqueryExpression::addFunctionIn(String operator_name, ASTPtr & node, bool is_any) -{ - - auto function_in = std::make_shared(); - auto exp_list_in = std::make_shared(); - exp_list_in->children.push_back(node->children[0]->children[0]); - exp_list_in->children.push_back(node->children[0]->children[1]); - function_in->name = "in"; - function_in->children.push_back(exp_list_in); - function_in->arguments = exp_list_in; - - if (operator_name == "equals" && is_any) - { - node = function_in; - return true; - } - - if (operator_name == "notEquals" && !is_any) - { - auto function_not = std::make_shared(); - auto exp_list_not = std::make_shared(); - exp_list_not->children.push_back(function_in); - function_not->name = "not"; - function_not->children.push_back(exp_list_not); - function_not->arguments = exp_list_not; - node = function_not; - return true; - } - return false; -} - -bool ParserComparisonWithSubqueryExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) -{ - Pos begin = pos; - ASTPtr elem; - if (!elem_parser.parse(pos, elem, expected)) - return next_parser.parse(pos, node, expected); - - /// try to find any of the valid operators - const char ** it; - for (it = operators; *it; it += 2) - if (parseOperator(pos, *it, expected)) - break; - - if (!*it) - { - node = elem; - return true; - } - - bool is_any = true; - if (!ParserKeyword("ANY").ignore(pos, expected)) - { - is_any = false; - if (!ParserKeyword("ALL").ignore(pos, expected)) - { - pos = begin; - return next_parser.parse(pos, node, expected); - } - } - - ASTPtr subquery_node; - if (!ParserSubquery().parse(pos, subquery_node, expected)) - return false; - - /// the first argument of the function is the previous element, the second is the next one - String operator_name = it[1]; - - /// the function corresponding to the operator - auto function = std::make_shared(); - - /// function arguments - auto exp_list = std::make_shared(); - exp_list->children.push_back(elem); - exp_list->children.push_back(subquery_node); - - function->name = operator_name; - function->arguments = exp_list; - function->children.push_back(exp_list); - node = function; - return modifySubquery(operator_name, subquery_node, is_any) || addFunctionIn(operator_name, node, is_any); -} - bool ParserTernaryOperatorExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ParserToken symbol1(TokenType::QuestionMark); diff --git a/src/Parsers/ExpressionListParsers.h b/src/Parsers/ExpressionListParsers.h index 29335e72b2a..82fd0eefc8e 100644 --- a/src/Parsers/ExpressionListParsers.h +++ b/src/Parsers/ExpressionListParsers.h @@ -353,28 +353,11 @@ class ParserComparisonExpression : public IParserBase private: static const char * operators[]; static const char * overlapping_operators_to_skip[]; - ParserLeftAssociativeBinaryOperatorList operator_parser {operators, overlapping_operators_to_skip, std::make_unique()}; + ParserBetweenExpression elem_parser; protected: const char * getName() const override{ return "comparison expression"; } - bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override - { - return operator_parser.parse(pos, node, expected); - } -}; - -class ParserComparisonWithSubqueryExpression : public IParserBase -{ -private: - static const char * operators[]; - ParserComparisonExpression next_parser; - ParserBetweenExpression elem_parser; - static bool addFunctionIn(String operator_name, ASTPtr & node, bool is_any); - static bool modifySubquery(String operator_name, ASTPtr subquery_node, bool is_any); -protected: - const char * getName() const override { return "comparison with ANY/ALL expression"; } - bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; }; @@ -383,7 +366,7 @@ protected: class ParserNullityChecking : public IParserBase { private: - ParserComparisonWithSubqueryExpression elem_parser; + ParserComparisonExpression elem_parser; protected: const char * getName() const override { return "nullity checking"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; From 15843723ec9b8149e876a7414629b72c68744bb7 Mon Sep 17 00:00:00 2001 From: Kirill Ershov Date: Sat, 5 Jun 2021 02:30:57 +0300 Subject: [PATCH 006/161] Fix ALL/ANY implementation --- src/Parsers/ExpressionListParsers.cpp | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index acb2e35a84e..920a31199d4 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -243,9 +243,23 @@ static bool modifyAST(String operator_name, std::shared_ptr & funct aggregate_function->name = type == SubqueryFunctionType::ANY ? "max" : "min"; return true; } + + // = ALL --> IN (SELECT singleValueOrNull(*) FROM subquery) + // != ANY --> NOT IN (SELECT singleValueOrNull(*) FROM subquery) if (operator_name == "equals" || operator_name == "notEquals") { aggregate_function->name = "singleValueOrNull"; + function->name = "in"; + if (operator_name == "notEquals") + { + auto function_not = std::make_shared(); + auto exp_list_not = std::make_shared(); + exp_list_not->children.push_back(function); + function_not->name = "not"; + function_not->children.push_back(exp_list_not); + function_not->arguments = exp_list_not; + function = function_not; + } return true; } return false; From 1d46e443e29dd77c401b6345f483d750530c1bc2 Mon Sep 17 00:00:00 2001 From: Kirill Ershov Date: Sat, 5 Jun 2021 03:58:22 +0300 Subject: [PATCH 007/161] Fix INTERSECT/EXCEPT parser --- src/Parsers/ASTIntersectOrExcept.cpp | 2 +- src/Parsers/ExpressionElementParsers.cpp | 2 ++ src/Parsers/ParserIntersectOrExcept.cpp | 11 +++-------- src/Processors/QueryPlan/IntersectOrExceptStep.cpp | 2 +- 4 files changed, 7 insertions(+), 10 deletions(-) diff --git a/src/Parsers/ASTIntersectOrExcept.cpp b/src/Parsers/ASTIntersectOrExcept.cpp index 073d63963a9..a05d7ee86c9 100644 --- a/src/Parsers/ASTIntersectOrExcept.cpp +++ b/src/Parsers/ASTIntersectOrExcept.cpp @@ -20,7 +20,7 @@ void ASTIntersectOrExcept::formatQueryImpl(const FormatSettings & settings, Form children[0]->formatImpl(settings, state, frame); std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' '); settings.ostr << settings.nl_or_ws << indent_str << (settings.hilite ? hilite_keyword : "") - << (is_except ? "EXCEPT" : "INTERSECT ") + << (is_except ? "EXCEPT" : "INTERSECT") << (settings.hilite ? hilite_none : "") << settings.nl_or_ws; children[1]->formatImpl(settings, state, frame); } diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 75a341a543d..d08ce2a215b 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -1697,6 +1697,8 @@ const char * ParserAlias::restricted_keywords[] = "WHERE", "WINDOW", "WITH", + "INTERSECT", + "EXCEPT", nullptr }; diff --git a/src/Parsers/ParserIntersectOrExcept.cpp b/src/Parsers/ParserIntersectOrExcept.cpp index a82b8c2b06b..6d5da54fa38 100644 --- a/src/Parsers/ParserIntersectOrExcept.cpp +++ b/src/Parsers/ParserIntersectOrExcept.cpp @@ -3,7 +3,7 @@ #include #include #include -#include +#include namespace DB { @@ -17,7 +17,7 @@ bool ParserIntersectOrExcept::parseImpl(Pos & pos, ASTPtr & node, Expected & exp auto ast = std::make_shared(); ast->is_except = false; - if (!ParserSelectQuery().parse(pos, left_node, expected) && !ParserSubquery().parse(pos, left_node, expected)) + if (!ParserSelectWithUnionQuery().parse(pos, left_node, expected) && !ParserSubquery().parse(pos, left_node, expected)) return false; if (!intersect_keyword.ignore(pos)) @@ -32,14 +32,9 @@ bool ParserIntersectOrExcept::parseImpl(Pos & pos, ASTPtr & node, Expected & exp } } - if (!ParserSelectQuery().parse(pos, right_node, expected) && !ParserSubquery().parse(pos, right_node, expected)) + if (!ParserSelectWithUnionQuery().parse(pos, right_node, expected) && !ParserSubquery().parse(pos, right_node, expected)) return false; - if (const auto * ast_subquery = left_node->as()) - left_node = ast_subquery->children.at(0); - if (const auto * ast_subquery = right_node->as()) - right_node = ast_subquery->children.at(0); - ast->children.push_back(left_node); ast->children.push_back(right_node); diff --git a/src/Processors/QueryPlan/IntersectOrExceptStep.cpp b/src/Processors/QueryPlan/IntersectOrExceptStep.cpp index d0a820339d7..28f34bda5db 100644 --- a/src/Processors/QueryPlan/IntersectOrExceptStep.cpp +++ b/src/Processors/QueryPlan/IntersectOrExceptStep.cpp @@ -15,7 +15,7 @@ IntersectOrExceptStep::IntersectOrExceptStep(bool is_except_, DataStreams input_ output_stream = DataStream{.header = header}; } -QueryPipelinePtr IntersectOrExceptStep::updatePipeline(QueryPipelines pipelines, const BuildQueryPipelineSettings & ) +QueryPipelinePtr IntersectOrExceptStep::updatePipeline(QueryPipelines pipelines, const BuildQueryPipelineSettings &) { auto pipeline = std::make_unique(); QueryPipelineProcessorsCollector collector(*pipeline, this); From 46d688deb3fc99fad7132388e04754ad0dc55f5a Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Tue, 3 Aug 2021 16:07:46 +0300 Subject: [PATCH 008/161] Add stings empty function. --- .../functions/string-functions.md | 40 ++++++++++++++++--- .../functions/string-functions.md | 39 ++++++++++++++++-- 2 files changed, 70 insertions(+), 9 deletions(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 8ec8aa7339d..8723b9d278a 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -10,11 +10,41 @@ toc_title: Strings ## empty {#empty} -Returns 1 for an empty string or 0 for a non-empty string. -The result type is UInt8. -A string is considered non-empty if it contains at least one byte, even if this is a space or a null byte. -The function also works for arrays or UUID. -UUID is empty if it is all zeros (nil UUID). +Checks whether the input string is empty. + +**Syntax** + +``` sql +empty(x) +``` + +A string is considered non-empty if it contains at least one byte, even if this is a space or a null byte. The UUID is empty if it contains all zeros (zero UUID). + +**Arguments** + +- `x` — Input value. [Array](../data-types/array.md), [String](../data-types/string.md), [UUID](../data-types/uuid.md). + +**Returned value** + +- Returns `1` for an empty string or `0` for a non-empty string. + +Type: [UInt8](../data-types/int-uint.md). + +**Example** + +Query: + +```sql +SELECT empty(''); +``` + +Result: + +```text +┌─empty('')─┐ +│ 1 │ +└───────────┘ +``` ## notEmpty {#notempty} diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index b587a991db1..176cd83b0d0 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -7,10 +7,41 @@ toc_title: "Функции для работы со строками" ## empty {#empty} -Возвращает 1 для пустой строки, и 0 для непустой строки. -Тип результата — UInt8. -Строка считается непустой, если содержит хотя бы один байт, пусть даже это пробел или нулевой байт. -Функция также работает для массивов. +Проверяет, является ли входное значение пустым. + +**Синтаксис** + +``` sql +empty(x) +``` + +Строка считается непустой, если содержит хотя бы один байт, пусть даже это пробел или нулевой байт. UUID считается пустой, если он содержит только нули (нулевой UUID). + +**Параметры** + +- `x` — Входное значение. [Array](../data-types/array.md), [String](../data-types/string.md), [UUID](../data-types/uuid.md). + +**Возвращаемое значение** + +- Возвращает `1` для пустой строки, и `0` — для непустой строки. + +Тип: [UInt8](../data-types/int-uint.md). + +**Пример** + +Запрос: + +```sql +SELECT empty(''); +``` + +Результат: + +```text +┌─empty('')─┐ +│ 1 │ +└───────────┘ +``` ## notEmpty {#notempty} From dad7f443a6c458450a9b2b57a90758f284219094 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Tue, 3 Aug 2021 16:47:16 +0300 Subject: [PATCH 009/161] Update materialize-mysql. --- .../database-engines/materialized-mysql.md | 11 +++++++---- .../database-engines/materialized-mysql.md | 17 +++++++++++++++++ 2 files changed, 24 insertions(+), 4 deletions(-) diff --git a/docs/en/engines/database-engines/materialized-mysql.md b/docs/en/engines/database-engines/materialized-mysql.md index 20e16473115..57bc1fe44a2 100644 --- a/docs/en/engines/database-engines/materialized-mysql.md +++ b/docs/en/engines/database-engines/materialized-mysql.md @@ -42,12 +42,15 @@ CREATE DATABASE mysql ENGINE = MaterializedMySQL('localhost:3306', 'db', 'user', max_wait_time_when_mysql_unavailable=10000; ``` -**Settings on MySQL-server side** +**Settings on Mysql-server Side** -For the correct work of `MaterializeMySQL`, there are few mandatory `MySQL`-side configuration settings that should be set: +For the correct work of `MaterializeMySQL`, there are few mandatory `MySQL`-side configuration settings that must be set: - `default_authentication_plugin = mysql_native_password` since `MaterializeMySQL` can only authorize with this method. -- `gtid_mode = on` since GTID based logging is a mandatory for providing correct `MaterializeMySQL` replication. Pay attention that while turning this mode `On` you should also specify `enforce_gtid_consistency = on`. +- `gtid_mode = on` since GTID based logging is a mandatory for providing correct `MaterializeMySQL` replication. + +!!! attention "Attention" + While turning on `gtid_mode` you should also specify `enforce_gtid_consistency = on`. ## Virtual columns {#virtual-columns} @@ -85,7 +88,7 @@ Other types are not supported. If MySQL table contains a column of such type, Cl ## Specifics and Recommendations {#specifics-and-recommendations} -### Compatibility restrictions +### Compatibility restrictions {#compatibility-restrictions} Apart of the data types limitations there are few restrictions comparing to `MySQL` databases, that should be resolved before replication will be possible: diff --git a/docs/ru/engines/database-engines/materialized-mysql.md b/docs/ru/engines/database-engines/materialized-mysql.md index f5f0166c9dc..c02e81231dd 100644 --- a/docs/ru/engines/database-engines/materialized-mysql.md +++ b/docs/ru/engines/database-engines/materialized-mysql.md @@ -26,6 +26,15 @@ ENGINE = MaterializedMySQL('host:port', ['database' | database], 'user', 'passwo - `user` — пользователь MySQL. - `password` — пароль пользователя. +**Настройки на стороне MySQL-сервера** + +Для правильной работы `MaterializeMySQL` следует обязательно указать на MySQL сервере следующие параметры конфигурации: +- `default_authentication_plugin = mysql_native_password` — `MaterializeMySQL` может авторизоваться только с помощью этого метода. +- `gtid_mode = on` — ведение журнала на основе GTID является обязательным для обеспечения правильной репликации. + +!!! attention "Внимание" + При включении `gtid_mode` вы также должны указать `enforce_gtid_consistency = on`. + ## Виртуальные столбцы {#virtual-columns} При работе с движком баз данных `MaterializedMySQL` используются таблицы семейства [ReplacingMergeTree](../../engines/table-engines/mergetree-family/replacingmergetree.md) с виртуальными столбцами `_sign` и `_version`. @@ -54,6 +63,7 @@ ENGINE = MaterializedMySQL('host:port', ['database' | database], 'user', 'passwo | STRING | [String](../../sql-reference/data-types/string.md) | | VARCHAR, VAR_STRING | [String](../../sql-reference/data-types/string.md) | | BLOB | [String](../../sql-reference/data-types/string.md) | +| BINARY | [FixedString](../../sql-reference/data-types/fixedstring.md) | Другие типы не поддерживаются. Если таблица MySQL содержит столбец другого типа, ClickHouse выдаст исключение "Неподдерживаемый тип данных" ("Unhandled data type") и остановит репликацию. @@ -61,6 +71,13 @@ ENGINE = MaterializedMySQL('host:port', ['database' | database], 'user', 'passwo ## Особенности и рекомендации {#specifics-and-recommendations} +### Ограничения совместимости {#compatibility-restrictions} + +Кроме ограничений на типы данных, существует несколько ограничений по сравнению с базами данных MySQL, которые следует решить до того, как станет возможной репликация: + +- Каждая таблица в `MySQL` должна содержать `PRIMARY KEY`. +- Репликация для таблиц, содержащих строки со значениями полей `ENUM` вне диапазона значений (определяется размерностью `ENUM`) вне диапазона (указанного в подписи "ПЕРЕЧИСЛЕНИЕ"), не будет работать. + ### DDL-запросы {#ddl-queries} DDL-запросы в MySQL конвертируются в соответствующие DDL-запросы в ClickHouse ([ALTER](../../sql-reference/statements/alter/index.md), [CREATE](../../sql-reference/statements/create/index.md), [DROP](../../sql-reference/statements/drop.md), [RENAME](../../sql-reference/statements/rename.md)). Если ClickHouse не может конвертировать какой-либо DDL-запрос, он его игнорирует. From 44bb44cc2c3ddb1ee03e1b762d7bb27841c0f9b3 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Tue, 3 Aug 2021 16:55:38 +0300 Subject: [PATCH 010/161] Update materialize-mysql. --- docs/ru/engines/database-engines/materialized-mysql.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/engines/database-engines/materialized-mysql.md b/docs/ru/engines/database-engines/materialized-mysql.md index c02e81231dd..761a5ef4aaa 100644 --- a/docs/ru/engines/database-engines/materialized-mysql.md +++ b/docs/ru/engines/database-engines/materialized-mysql.md @@ -76,7 +76,7 @@ ENGINE = MaterializedMySQL('host:port', ['database' | database], 'user', 'passwo Кроме ограничений на типы данных, существует несколько ограничений по сравнению с базами данных MySQL, которые следует решить до того, как станет возможной репликация: - Каждая таблица в `MySQL` должна содержать `PRIMARY KEY`. -- Репликация для таблиц, содержащих строки со значениями полей `ENUM` вне диапазона значений (определяется размерностью `ENUM`) вне диапазона (указанного в подписи "ПЕРЕЧИСЛЕНИЕ"), не будет работать. +- Репликация для таблиц, содержащих строки со значениями полей `ENUM` вне диапазона значений (определяется размерностью `ENUM`), не будет работать. ### DDL-запросы {#ddl-queries} From c310cef87f91b1e87684987620d5d8caba31c2ba Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Tue, 3 Aug 2021 16:56:35 +0300 Subject: [PATCH 011/161] Update materialize-mysql. --- docs/ru/sql-reference/functions/string-functions.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index 176cd83b0d0..16a2c16c466 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -7,7 +7,7 @@ toc_title: "Функции для работы со строками" ## empty {#empty} -Проверяет, является ли входное значение пустым. +Проверяет, является ли входная стока пустой. **Синтаксис** @@ -19,7 +19,7 @@ empty(x) **Параметры** -- `x` — Входное значение. [Array](../data-types/array.md), [String](../data-types/string.md), [UUID](../data-types/uuid.md). +- `x` — Входная строка. [Array](../data-types/array.md), [String](../data-types/string.md), [UUID](../data-types/uuid.md). **Возвращаемое значение** From a57f274dd185a907041d5e7a60427b724d878d86 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Tue, 3 Aug 2021 17:28:05 +0300 Subject: [PATCH 012/161] Update materialize-mysql. --- docs/ru/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index 16a2c16c466..e6c8e17bdca 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -7,7 +7,7 @@ toc_title: "Функции для работы со строками" ## empty {#empty} -Проверяет, является ли входная стока пустой. +Проверяет, является ли входная строка пустой. **Синтаксис** From e939dc0b9ca369ee667a01acb25a8814045cd170 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 3 Aug 2021 20:30:05 +0300 Subject: [PATCH 013/161] fix --- src/Access/ya.make | 1 - src/Access/ya.make.in | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Access/ya.make b/src/Access/ya.make index 5f2f410cabd..38c1b007ff8 100644 --- a/src/Access/ya.make +++ b/src/Access/ya.make @@ -46,7 +46,6 @@ SRCS( SettingsProfilesInfo.cpp User.cpp UsersConfigAccessStorage.cpp - tests/gtest_access_rights_ops.cpp ) diff --git a/src/Access/ya.make.in b/src/Access/ya.make.in index 1f11c7d7d2a..5fa69cec4bb 100644 --- a/src/Access/ya.make.in +++ b/src/Access/ya.make.in @@ -8,7 +8,7 @@ PEERDIR( SRCS( - + ) END() From 6e74728714ebf71c29d93e742651b62c44a48e4f Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 4 Aug 2021 10:58:39 +0300 Subject: [PATCH 014/161] more sync fixes --- programs/server/Server.cpp | 2 +- src/Compression/CompressionCodecEncrypted.cpp | 4 ++-- src/Compression/CompressionCodecEncrypted.h | 4 ++-- src/Functions/stem.cpp | 2 +- src/Interpreters/Lemmatizers.cpp | 4 ++-- src/Interpreters/SynonymsExtensions.cpp | 2 +- src/Processors/ya.make | 4 ++-- src/Processors/ya.make.in | 6 ++++++ 8 files changed, 17 insertions(+), 11 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 86bb04351b1..a8abb439b62 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -97,7 +97,7 @@ #endif #if USE_SSL -# if USE_INTERNAL_SSL_LIBRARY +# if USE_INTERNAL_SSL_LIBRARY && !defined(ARCADIA_BUILD) # include # endif # include diff --git a/src/Compression/CompressionCodecEncrypted.cpp b/src/Compression/CompressionCodecEncrypted.cpp index d0904b4bf24..5dfdd6f4fcb 100644 --- a/src/Compression/CompressionCodecEncrypted.cpp +++ b/src/Compression/CompressionCodecEncrypted.cpp @@ -5,9 +5,9 @@ #include #include #include -#include +#include // Y_IGNORE #include -#include +#include // Y_IGNORE #include namespace DB diff --git a/src/Compression/CompressionCodecEncrypted.h b/src/Compression/CompressionCodecEncrypted.h index e58fd4ab173..bacd58bcd2f 100644 --- a/src/Compression/CompressionCodecEncrypted.h +++ b/src/Compression/CompressionCodecEncrypted.h @@ -2,11 +2,11 @@ // This depends on BoringSSL-specific API, notably . #include -#if USE_SSL && USE_INTERNAL_SSL_LIBRARY +#if USE_SSL && USE_INTERNAL_SSL_LIBRARY && !defined(ARCADIA_BUILD) #include #include -#include +#include // Y_IGNORE #include namespace DB diff --git a/src/Functions/stem.cpp b/src/Functions/stem.cpp index 98dcbccd005..7092bac06ec 100644 --- a/src/Functions/stem.cpp +++ b/src/Functions/stem.cpp @@ -11,7 +11,7 @@ #include #include -#include +#include // Y_IGNORE namespace DB diff --git a/src/Interpreters/Lemmatizers.cpp b/src/Interpreters/Lemmatizers.cpp index 38cd4c33678..78af43285ef 100644 --- a/src/Interpreters/Lemmatizers.cpp +++ b/src/Interpreters/Lemmatizers.cpp @@ -6,8 +6,8 @@ #if USE_NLP #include -#include -#include +#include // Y_IGNORE +#include // Y_IGNORE #include #include diff --git a/src/Interpreters/SynonymsExtensions.cpp b/src/Interpreters/SynonymsExtensions.cpp index 22fa91a4349..6147fa14674 100644 --- a/src/Interpreters/SynonymsExtensions.cpp +++ b/src/Interpreters/SynonymsExtensions.cpp @@ -11,7 +11,7 @@ #include #include -#include +#include // Y_IGNORE namespace DB { diff --git a/src/Processors/ya.make b/src/Processors/ya.make index 4b95484a828..97b730dd8a4 100644 --- a/src/Processors/ya.make +++ b/src/Processors/ya.make @@ -7,11 +7,11 @@ PEERDIR( clickhouse/src/Common contrib/libs/msgpack contrib/libs/protobuf - contrib/libs/arrow + contrib/libs/apache/arrow ) ADDINCL( - contrib/libs/arrow/src + contrib/libs/apache/arrow/src ) CFLAGS(-DUSE_ARROW=1) diff --git a/src/Processors/ya.make.in b/src/Processors/ya.make.in index 06230b96be8..d4b98acf01c 100644 --- a/src/Processors/ya.make.in +++ b/src/Processors/ya.make.in @@ -6,8 +6,14 @@ PEERDIR( clickhouse/src/Common contrib/libs/msgpack contrib/libs/protobuf + contrib/libs/apache/arrow ) +ADDINCL( + contrib/libs/apache/arrow/src +) + +CFLAGS(-DUSE_ARROW=1) SRCS( From 9539ef18d2768a6843ece53895c25685536fbbfd Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 4 Aug 2021 11:55:51 +0300 Subject: [PATCH 015/161] fix --- src/Compression/CompressionCodecEncrypted.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Compression/CompressionCodecEncrypted.cpp b/src/Compression/CompressionCodecEncrypted.cpp index 5dfdd6f4fcb..6b921fb9c0a 100644 --- a/src/Compression/CompressionCodecEncrypted.cpp +++ b/src/Compression/CompressionCodecEncrypted.cpp @@ -1,4 +1,6 @@ -#include +#if !defined(ARCADIA_BUILD) +# include +#endif #include #if USE_SSL && USE_INTERNAL_SSL_LIBRARY From 55e06ea16b5b4f4b7366b6d5e4ec53aba7714275 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 4 Aug 2021 13:06:01 +0300 Subject: [PATCH 016/161] fix --- src/Storages/MergeTree/IMergeTreeReader.cpp | 2 +- src/Storages/StorageInMemoryMetadata.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeReader.cpp b/src/Storages/MergeTree/IMergeTreeReader.cpp index 4efd3d669eb..5378b84a5d0 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.cpp +++ b/src/Storages/MergeTree/IMergeTreeReader.cpp @@ -50,7 +50,7 @@ IMergeTreeReader::IMergeTreeReader( columns_from_part.set_empty_key(StringRef()); for (const auto & column_from_part : part_columns) - columns_from_part.emplace(column_from_part.name, &column_from_part.type); + columns_from_part[column_from_part.name] = &column_from_part.type; } IMergeTreeReader::~IMergeTreeReader() = default; diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index dad83f64c70..7b1a126b1ab 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -331,7 +331,7 @@ Block StorageInMemoryMetadata::getSampleBlockForColumns( /// Virtual columns must be appended after ordinary, because user can /// override them. for (const auto & column : virtuals) - virtuals_map.emplace(column.name, &column.type); + virtuals_map[column.name] = &column.type; for (const auto & name : column_names) { From 1a607af42920eb18016526b5682f00484ef888a5 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 3 Aug 2021 17:53:18 +0300 Subject: [PATCH 017/161] add test for parsing maps --- src/DataTypes/DataTypeMap.cpp | 2 +- .../0_stateless/02002_parse_map_int_key.reference | 1 + tests/queries/0_stateless/02002_parse_map_int_key.sql | 11 +++++++++++ 3 files changed, 13 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02002_parse_map_int_key.reference create mode 100644 tests/queries/0_stateless/02002_parse_map_int_key.sql diff --git a/src/DataTypes/DataTypeMap.cpp b/src/DataTypes/DataTypeMap.cpp index 8fd375aa86e..b0bf459b4ca 100644 --- a/src/DataTypes/DataTypeMap.cpp +++ b/src/DataTypes/DataTypeMap.cpp @@ -79,7 +79,7 @@ void DataTypeMap::assertKeyType() const std::string DataTypeMap::doGetName() const { WriteBufferFromOwnString s; - s << "Map(" << key_type->getName() << "," << value_type->getName() << ")"; + s << "Map(" << key_type->getName() << ", " << value_type->getName() << ")"; return s.str(); } diff --git a/tests/queries/0_stateless/02002_parse_map_int_key.reference b/tests/queries/0_stateless/02002_parse_map_int_key.reference new file mode 100644 index 00000000000..dc02589d4bc --- /dev/null +++ b/tests/queries/0_stateless/02002_parse_map_int_key.reference @@ -0,0 +1 @@ +{1:2,3:4,5:6,7:8} {'2021-05-20':1,'2021-05-21':2,'2021-05-22':3,'2021-05-23':4} diff --git a/tests/queries/0_stateless/02002_parse_map_int_key.sql b/tests/queries/0_stateless/02002_parse_map_int_key.sql new file mode 100644 index 00000000000..ecd2a090975 --- /dev/null +++ b/tests/queries/0_stateless/02002_parse_map_int_key.sql @@ -0,0 +1,11 @@ +SET allow_experimental_map_type = 1; + +DROP TABLE IF EXISTS t_map_int_key; +CREATE TABLE t_map_int_key (m1 Map(UInt32, UInt32), m2 Map(Date, UInt32)) ENGINE = Memory; + +INSERT INTO t_map_int_key FORMAT CSV "{1:2, 3: 4, 5 :6, 7 : 8}","{'2021-05-20':1, '2021-05-21': 2, '2021-05-22' :3, '2021-05-23' : 4}" +; + +SELECT m1, m2 FROM t_map_int_key; + +DROP TABLE t_map_int_key; From a6ad811652a0757b972b96f572a97502c2f60b24 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Fri, 6 Aug 2021 11:29:03 +0300 Subject: [PATCH 018/161] Update MySql settings. --- .../database-engines/materialized-mysql.md | 22 ++++++++------- .../database-engines/materialized-mysql.md | 27 ++++++++++++++++--- 2 files changed, 35 insertions(+), 14 deletions(-) diff --git a/docs/en/engines/database-engines/materialized-mysql.md b/docs/en/engines/database-engines/materialized-mysql.md index 57bc1fe44a2..1fad4414d57 100644 --- a/docs/en/engines/database-engines/materialized-mysql.md +++ b/docs/en/engines/database-engines/materialized-mysql.md @@ -3,7 +3,7 @@ toc_priority: 29 toc_title: MaterializedMySQL --- -# MaterializedMySQL {#materialized-mysql} +# [experimental] MaterializedMySQL {#materialized-mysql} **This is experimental feature that should not be used in production.** @@ -28,14 +28,16 @@ ENGINE = MaterializedMySQL('host:port', ['database' | database], 'user', 'passwo - `password` — User password. **Engine Settings** -- `max_rows_in_buffer` — Max rows that data is allowed to cache in memory(for single table and the cache data unable to query). when rows is exceeded, the data will be materialized. Default: `65505`. -- `max_bytes_in_buffer` — Max bytes that data is allowed to cache in memory(for single table and the cache data unable to query). when rows is exceeded, the data will be materialized. Default: `1048576`. -- `max_rows_in_buffers` — Max rows that data is allowed to cache in memory(for database and the cache data unable to query). when rows is exceeded, the data will be materialized. Default: `65505`. -- `max_bytes_in_buffers` — Max bytes that data is allowed to cache in memory(for database and the cache data unable to query). when rows is exceeded, the data will be materialized. Default: `1048576`. -- `max_flush_data_time` — Max milliseconds that data is allowed to cache in memory(for database and the cache data unable to query). when this time is exceeded, the data will be materialized. Default: `1000`. + +- `max_rows_in_buffer` — Max rows that data is allowed to cache in memory (for single table and the cache data unable to query). When rows is exceeded, the data will be materialized. Default: `65 505`. +- `max_bytes_in_buffer` — Max bytes that data is allowed to cache in memory (for single table and the cache data unable to query). when rows is exceeded, the data will be materialized. Default: `1 048 576`. +- `max_rows_in_buffers` — Max rows that data is allowed to cache in memory (for database and the cache data unable to query). when rows is exceeded, the data will be materialized. Default: `65505`. +- `max_bytes_in_buffers` — Max bytes that data is allowed to cache in memory (for database and the cache data unable to query). when rows is exceeded, the data will be materialized. Default: `1 048 576`. +- `max_flush_data_time` — Max milliseconds that data is allowed to cache in memory (for database and the cache data unable to query). when this time is exceeded, the data will be materialized. Default: `1000`. - `max_wait_time_when_mysql_unavailable` — Retry interval when MySQL is not available (milliseconds). Negative value disable retry. Default: `1000`. -- `allows_query_when_mysql_lost` — Allow query materialized table when mysql is lost. Default: `0` (`false`). -``` +- `allows_query_when_mysql_lost` — Allow query materialized table when MySql is lost. Default: `0` (`false`). + +```sql CREATE DATABASE mysql ENGINE = MaterializedMySQL('localhost:3306', 'db', 'user', '***') SETTINGS allows_query_when_mysql_lost=true, @@ -82,10 +84,10 @@ When working with the `MaterializedMySQL` database engine, [ReplacingMergeTree]( | BLOB | [String](../../sql-reference/data-types/string.md) | | BINARY | [FixedString](../../sql-reference/data-types/fixedstring.md) | -Other types are not supported. If MySQL table contains a column of such type, ClickHouse throws exception "Unhandled data type" and stops replication. - [Nullable](../../sql-reference/data-types/nullable.md) is supported. +Other types are not supported. If MySQL table contains a column of such type, ClickHouse throws exception "Unhandled data type" and stops replication. + ## Specifics and Recommendations {#specifics-and-recommendations} ### Compatibility restrictions {#compatibility-restrictions} diff --git a/docs/ru/engines/database-engines/materialized-mysql.md b/docs/ru/engines/database-engines/materialized-mysql.md index 761a5ef4aaa..b886dc9b4e7 100644 --- a/docs/ru/engines/database-engines/materialized-mysql.md +++ b/docs/ru/engines/database-engines/materialized-mysql.md @@ -1,10 +1,11 @@ - --- toc_priority: 29 toc_title: MaterializedMySQL --- -# MaterializedMySQL {#materialized-mysql} +# [experimental] MaterializedMySQL {#materialized-mysql} + +**Это экспериментальная функция, которую не следует использовать в продуктивной среде.** Создает базу данных ClickHouse со всеми таблицами, существующими в MySQL, и всеми данными в этих таблицах. @@ -26,6 +27,23 @@ ENGINE = MaterializedMySQL('host:port', ['database' | database], 'user', 'passwo - `user` — пользователь MySQL. - `password` — пароль пользователя. +**Настройки движка** + +- `max_rows_in_buffer` — максимальное количество строк, содержимое которых может кэшироваться в памяти (для одной таблицы и данных кэша, которые невозможно запросить). При превышении количества строк, данные будут материализованы. Значение по умолчанию: `65 505`. +- `max_bytes_in_buffer` — максимальное количество байтов, которое разрешено кэшировать в памяти (для одной таблицы и данных кэша, которые невозможно запросить). При превышении количества строк, данные будут материализованы. Значение по умолчанию: `1 048 576`. +- `max_rows_in_buffers` — максимальное количество строк, содержимое которых может кэшироваться в памяти (для базы данных и данных кэша, которые невозможно запросить). При превышении количества строк, данные будут материализованы. Значение по умолчанию: `65 505`. +- `max_bytes_in_buffers` — максимальное количество байтов, которое разрешено кэшировать данным в памяти (для базы данных и данных кэша, которые невозможно запросить). При превышении количества строк, данные будут материализованы. Значение по умолчанию: `1 048 576`. +- `max_flush_data_time` — Максимальное время в миллисекундах, в течение которого разрешено кэшировать данные в памяти (для базы данных и данных кэша, которые невозможно запросить). При превышении количества указанного периода, данные будут материализованы. Значение по умолчанию: `1000`. +- `max_wait_time_when_mysql_unavailable` — интервал повторных попыток, если MySQL недоступен. Указывается в миллисекундах. Отрицательное значение отключает повторные попытки. Значение по умолчанию: `1000`. +- `allows_query_when_mysql_lost` — разрешить запрос материализованной таблицы при потере MySQL. Значение по умолчанию: `0` (`false`). + +```sql +CREATE DATABASE mysql ENGINE = MaterializedMySQL('localhost:3306', 'db', 'user', '***') + SETTINGS + allows_query_when_mysql_lost=true, + max_wait_time_when_mysql_unavailable=10000; +``` + **Настройки на стороне MySQL-сервера** Для правильной работы `MaterializeMySQL` следует обязательно указать на MySQL сервере следующие параметры конфигурации: @@ -65,10 +83,10 @@ ENGINE = MaterializedMySQL('host:port', ['database' | database], 'user', 'passwo | BLOB | [String](../../sql-reference/data-types/string.md) | | BINARY | [FixedString](../../sql-reference/data-types/fixedstring.md) | -Другие типы не поддерживаются. Если таблица MySQL содержит столбец другого типа, ClickHouse выдаст исключение "Неподдерживаемый тип данных" ("Unhandled data type") и остановит репликацию. - Тип [Nullable](../../sql-reference/data-types/nullable.md) поддерживается. +Другие типы не поддерживаются. Если таблица MySQL содержит столбец другого типа, ClickHouse выдаст исключение "Неподдерживаемый тип данных" ("Unhandled data type") и остановит репликацию. + ## Особенности и рекомендации {#specifics-and-recommendations} ### Ограничения совместимости {#compatibility-restrictions} @@ -178,3 +196,4 @@ SELECT * FROM mysql.test; └───┴─────┴──────┘ ``` +[Оригинальная статья](https://clickhouse.tech/docs/ru/engines/database-engines/materialized-mysql/) From d8db279c75f9671f3faf9380e6a74204507b562d Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Fri, 6 Aug 2021 11:39:47 +0300 Subject: [PATCH 019/161] Add experimental title. --- docs/en/engines/database-engines/materialized-mysql.md | 2 +- docs/ru/engines/database-engines/materialized-mysql.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/engines/database-engines/materialized-mysql.md b/docs/en/engines/database-engines/materialized-mysql.md index 1fad4414d57..1c69fd072d8 100644 --- a/docs/en/engines/database-engines/materialized-mysql.md +++ b/docs/en/engines/database-engines/materialized-mysql.md @@ -1,6 +1,6 @@ --- toc_priority: 29 -toc_title: MaterializedMySQL +toc_title: [experimental] MaterializedMySQL --- # [experimental] MaterializedMySQL {#materialized-mysql} diff --git a/docs/ru/engines/database-engines/materialized-mysql.md b/docs/ru/engines/database-engines/materialized-mysql.md index b886dc9b4e7..7b14867fa62 100644 --- a/docs/ru/engines/database-engines/materialized-mysql.md +++ b/docs/ru/engines/database-engines/materialized-mysql.md @@ -1,6 +1,6 @@ --- toc_priority: 29 -toc_title: MaterializedMySQL +toc_title: [experimental] MaterializedMySQL --- # [experimental] MaterializedMySQL {#materialized-mysql} From f16a26ee6a110738655ac7a198394ade32c81ba3 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Fri, 6 Aug 2021 11:43:46 +0300 Subject: [PATCH 020/161] Minor fix. --- docs/en/engines/database-engines/materialized-mysql.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/engines/database-engines/materialized-mysql.md b/docs/en/engines/database-engines/materialized-mysql.md index 1c69fd072d8..27af13a6976 100644 --- a/docs/en/engines/database-engines/materialized-mysql.md +++ b/docs/en/engines/database-engines/materialized-mysql.md @@ -54,7 +54,7 @@ For the correct work of `MaterializeMySQL`, there are few mandatory `MySQL`-side !!! attention "Attention" While turning on `gtid_mode` you should also specify `enforce_gtid_consistency = on`. -## Virtual columns {#virtual-columns} +## Virtual Columns {#virtual-columns} When working with the `MaterializedMySQL` database engine, [ReplacingMergeTree](../../engines/table-engines/mergetree-family/replacingmergetree.md) tables are used with virtual `_sign` and `_version` columns. @@ -90,7 +90,7 @@ Other types are not supported. If MySQL table contains a column of such type, Cl ## Specifics and Recommendations {#specifics-and-recommendations} -### Compatibility restrictions {#compatibility-restrictions} +### Compatibility Restrictions {#compatibility-restrictions} Apart of the data types limitations there are few restrictions comparing to `MySQL` databases, that should be resolved before replication will be possible: From 0f2c880876949cd8f024136887be7956a0701259 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Fri, 6 Aug 2021 11:45:19 +0300 Subject: [PATCH 021/161] Minor fix. --- docs/ru/engines/database-engines/materialized-mysql.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/engines/database-engines/materialized-mysql.md b/docs/ru/engines/database-engines/materialized-mysql.md index 7b14867fa62..ef0d9be054e 100644 --- a/docs/ru/engines/database-engines/materialized-mysql.md +++ b/docs/ru/engines/database-engines/materialized-mysql.md @@ -33,7 +33,7 @@ ENGINE = MaterializedMySQL('host:port', ['database' | database], 'user', 'passwo - `max_bytes_in_buffer` — максимальное количество байтов, которое разрешено кэшировать в памяти (для одной таблицы и данных кэша, которые невозможно запросить). При превышении количества строк, данные будут материализованы. Значение по умолчанию: `1 048 576`. - `max_rows_in_buffers` — максимальное количество строк, содержимое которых может кэшироваться в памяти (для базы данных и данных кэша, которые невозможно запросить). При превышении количества строк, данные будут материализованы. Значение по умолчанию: `65 505`. - `max_bytes_in_buffers` — максимальное количество байтов, которое разрешено кэшировать данным в памяти (для базы данных и данных кэша, которые невозможно запросить). При превышении количества строк, данные будут материализованы. Значение по умолчанию: `1 048 576`. -- `max_flush_data_time` — Максимальное время в миллисекундах, в течение которого разрешено кэшировать данные в памяти (для базы данных и данных кэша, которые невозможно запросить). При превышении количества указанного периода, данные будут материализованы. Значение по умолчанию: `1000`. +- `max_flush_data_time` — максимальное время в миллисекундах, в течение которого разрешено кэшировать данные в памяти (для базы данных и данных кэша, которые невозможно запросить). При превышении количества указанного периода, данные будут материализованы. Значение по умолчанию: `1000`. - `max_wait_time_when_mysql_unavailable` — интервал повторных попыток, если MySQL недоступен. Указывается в миллисекундах. Отрицательное значение отключает повторные попытки. Значение по умолчанию: `1000`. - `allows_query_when_mysql_lost` — разрешить запрос материализованной таблицы при потере MySQL. Значение по умолчанию: `0` (`false`). From ae756dd2b0d8ba53cbf7e150cdfa98ec5e4f003f Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Fri, 6 Aug 2021 12:04:55 +0300 Subject: [PATCH 022/161] Add stings empty nonempty functions --- .../functions/string-functions.md | 38 +++++++++++++-- .../functions/string-functions.md | 46 ++++++++++++++++--- 2 files changed, 74 insertions(+), 10 deletions(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 8723b9d278a..0156b4331ba 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -48,9 +48,41 @@ Result: ## notEmpty {#notempty} -Returns 0 for an empty string or 1 for a non-empty string. -The result type is UInt8. -The function also works for arrays or UUID. +Checks whether the input string is not empty. + +**Syntax** + +``` sql +notempty(x) +``` + +A string is considered non-empty if it contains at least one byte, even if this is a space or a null byte. The UUID is empty if it contains all zeros (zero UUID). + +**Arguments** + +- `x` — Input value. [Array](../data-types/array.md), [String](../data-types/string.md), [UUID](../data-types/uuid.md). + +**Returned value** + +- Returns `1` for a non-empty string or `0` for an empty string string. + +Type: [UInt8](../data-types/int-uint.md). + +**Example** + +Query: + +```sql +SELECT notempty('text'); +``` + +Result: + +```text +┌─empty('')─┐ +│ 1 │ +└───────────┘ +``` ## length {#length} diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index e6c8e17bdca..097ff593ef7 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -23,7 +23,45 @@ empty(x) **Возвращаемое значение** -- Возвращает `1` для пустой строки, и `0` — для непустой строки. +- Возвращает `1` для пустой строки и `0` — для непустой строки. + +Тип: [UInt8](../data-types/int-uint.md). + +**Пример** + +Запрос: + +```sql +SELECT notempty('text'); +``` + +Результат: + +```text +┌─empty('')─┐ +│ 1 │ +└───────────┘ +``` + +## notEmpty {#notempty} + +Проверяет, является ли входная строка непустой. + +**Синтаксис** + +``` sql +empty(x) +``` + +Строка считается непустой, если содержит хотя бы один байт, пусть даже это пробел или нулевой байт. UUID считается пустой, если он содержит только нули (нулевой UUID). + +**Параметры** + +- `x` — Входная строка. [Array](../data-types/array.md), [String](../data-types/string.md), [UUID](../data-types/uuid.md). + +**Возвращаемое значение** + +- Возвращает `1` для непустой строки и `0` — для пустой строки. Тип: [UInt8](../data-types/int-uint.md). @@ -43,12 +81,6 @@ SELECT empty(''); └───────────┘ ``` -## notEmpty {#notempty} - -Возвращает 0 для пустой строки, и 1 для непустой строки. -Тип результата — UInt8. -Функция также работает для массивов. - ## length {#length} Возвращает длину строки в байтах (не символах, не кодовых точках). From afd3341b29192cf8232883dd44c9fe7aa0b4d1d0 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Fri, 6 Aug 2021 12:11:08 +0300 Subject: [PATCH 023/161] Minor fix. --- docs/en/engines/database-engines/materialized-mysql.md | 2 +- docs/ru/engines/database-engines/materialized-mysql.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/engines/database-engines/materialized-mysql.md b/docs/en/engines/database-engines/materialized-mysql.md index 27af13a6976..163c86df84f 100644 --- a/docs/en/engines/database-engines/materialized-mysql.md +++ b/docs/en/engines/database-engines/materialized-mysql.md @@ -1,6 +1,6 @@ --- toc_priority: 29 -toc_title: [experimental] MaterializedMySQL +toc_title: "[experimental] MaterializedMySQL" --- # [experimental] MaterializedMySQL {#materialized-mysql} diff --git a/docs/ru/engines/database-engines/materialized-mysql.md b/docs/ru/engines/database-engines/materialized-mysql.md index ef0d9be054e..2f57b9f5a84 100644 --- a/docs/ru/engines/database-engines/materialized-mysql.md +++ b/docs/ru/engines/database-engines/materialized-mysql.md @@ -1,6 +1,6 @@ --- toc_priority: 29 -toc_title: [experimental] MaterializedMySQL +toc_title: "[experimental] MaterializedMySQL" --- # [experimental] MaterializedMySQL {#materialized-mysql} From c20bf2fbe48a2b7cba4709c9fc587d7de8791756 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Sat, 7 Aug 2021 10:36:12 +0300 Subject: [PATCH 024/161] Update docs/ru/sql-reference/functions/string-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index 097ff593ef7..39498fcea9f 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -15,7 +15,7 @@ toc_title: "Функции для работы со строками" empty(x) ``` -Строка считается непустой, если содержит хотя бы один байт, пусть даже это пробел или нулевой байт. UUID считается пустой, если он содержит только нули (нулевой UUID). +Строка считается непустой, если содержит хотя бы один байт, пусть даже это пробел или нулевой байт. UUID считается пустым, если он содержит только нули (нулевой UUID). **Параметры** From 3a69ad9d2bfa32b02e7a848682cf3403f9d89919 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Sat, 7 Aug 2021 10:36:19 +0300 Subject: [PATCH 025/161] Update docs/en/sql-reference/functions/string-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 0156b4331ba..d54bb16abbc 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -46,7 +46,7 @@ Result: └───────────┘ ``` -## notEmpty {#notempty} +## notempty {#notempty} Checks whether the input string is not empty. From c9124d979d439a9be000260f150d7bf9382b4ce9 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Sat, 7 Aug 2021 10:36:28 +0300 Subject: [PATCH 026/161] Update docs/ru/engines/database-engines/materialized-mysql.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/engines/database-engines/materialized-mysql.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/engines/database-engines/materialized-mysql.md b/docs/ru/engines/database-engines/materialized-mysql.md index 2f57b9f5a84..9ad89628e0b 100644 --- a/docs/ru/engines/database-engines/materialized-mysql.md +++ b/docs/ru/engines/database-engines/materialized-mysql.md @@ -93,7 +93,7 @@ CREATE DATABASE mysql ENGINE = MaterializedMySQL('localhost:3306', 'db', 'user', Кроме ограничений на типы данных, существует несколько ограничений по сравнению с базами данных MySQL, которые следует решить до того, как станет возможной репликация: -- Каждая таблица в `MySQL` должна содержать `PRIMARY KEY`. +- Каждая таблица в MySQL должна содержать `PRIMARY KEY`. - Репликация для таблиц, содержащих строки со значениями полей `ENUM` вне диапазона значений (определяется размерностью `ENUM`), не будет работать. ### DDL-запросы {#ddl-queries} From 6c71fcdeb7b5532c4010aa6e9de5c15484fe93d3 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Sat, 7 Aug 2021 10:36:38 +0300 Subject: [PATCH 027/161] Update docs/en/engines/database-engines/materialized-mysql.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/engines/database-engines/materialized-mysql.md | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/en/engines/database-engines/materialized-mysql.md b/docs/en/engines/database-engines/materialized-mysql.md index 163c86df84f..8a6975e4f2b 100644 --- a/docs/en/engines/database-engines/materialized-mysql.md +++ b/docs/en/engines/database-engines/materialized-mysql.md @@ -29,11 +29,11 @@ ENGINE = MaterializedMySQL('host:port', ['database' | database], 'user', 'passwo **Engine Settings** -- `max_rows_in_buffer` — Max rows that data is allowed to cache in memory (for single table and the cache data unable to query). When rows is exceeded, the data will be materialized. Default: `65 505`. -- `max_bytes_in_buffer` — Max bytes that data is allowed to cache in memory (for single table and the cache data unable to query). when rows is exceeded, the data will be materialized. Default: `1 048 576`. -- `max_rows_in_buffers` — Max rows that data is allowed to cache in memory (for database and the cache data unable to query). when rows is exceeded, the data will be materialized. Default: `65505`. -- `max_bytes_in_buffers` — Max bytes that data is allowed to cache in memory (for database and the cache data unable to query). when rows is exceeded, the data will be materialized. Default: `1 048 576`. -- `max_flush_data_time` — Max milliseconds that data is allowed to cache in memory (for database and the cache data unable to query). when this time is exceeded, the data will be materialized. Default: `1000`. +- `max_rows_in_buffer` — Maximum number of rows that data is allowed to cache in memory (for single table and the cache data unable to query). When this number is exceeded, the data will be materialized. Default: `65 505`. +- `max_bytes_in_buffer` — Maximum number of bytes that data is allowed to cache in memory (for single table and the cache data unable to query). When this number is exceeded, the data will be materialized. Default: `1 048 576`. +- `max_rows_in_buffers` — Maximum number of rows that data is allowed to cache in memory (for database and the cache data unable to query). When this number is exceeded, the data will be materialized. Default: `65 505`. +- `max_bytes_in_buffers` — Maximum number of bytes that data is allowed to cache in memory (for database and the cache data unable to query). When this number is exceeded, the data will be materialized. Default: `1 048 576`. +- `max_flush_data_time` — Maximum number of milliseconds that data is allowed to cache in memory (for database and the cache data unable to query). When this time is exceeded, the data will be materialized. Default: `1000`. - `max_wait_time_when_mysql_unavailable` — Retry interval when MySQL is not available (milliseconds). Negative value disable retry. Default: `1000`. - `allows_query_when_mysql_lost` — Allow query materialized table when MySql is lost. Default: `0` (`false`). From c6fcf90bf0f444d4d081f501771493bbe62e6fa3 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Sat, 7 Aug 2021 10:36:48 +0300 Subject: [PATCH 028/161] Update docs/en/engines/database-engines/materialized-mysql.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/engines/database-engines/materialized-mysql.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/database-engines/materialized-mysql.md b/docs/en/engines/database-engines/materialized-mysql.md index 8a6975e4f2b..815fa152fee 100644 --- a/docs/en/engines/database-engines/materialized-mysql.md +++ b/docs/en/engines/database-engines/materialized-mysql.md @@ -35,7 +35,7 @@ ENGINE = MaterializedMySQL('host:port', ['database' | database], 'user', 'passwo - `max_bytes_in_buffers` — Maximum number of bytes that data is allowed to cache in memory (for database and the cache data unable to query). When this number is exceeded, the data will be materialized. Default: `1 048 576`. - `max_flush_data_time` — Maximum number of milliseconds that data is allowed to cache in memory (for database and the cache data unable to query). When this time is exceeded, the data will be materialized. Default: `1000`. - `max_wait_time_when_mysql_unavailable` — Retry interval when MySQL is not available (milliseconds). Negative value disable retry. Default: `1000`. -- `allows_query_when_mysql_lost` — Allow query materialized table when MySql is lost. Default: `0` (`false`). +- `allows_query_when_mysql_lost` — Allows to query a materialized table when MySQL is lost. Default: `0` (`false`). ```sql CREATE DATABASE mysql ENGINE = MaterializedMySQL('localhost:3306', 'db', 'user', '***') From 2a98353208e0e88f1ba52e448c0467f635d8aefd Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Sat, 7 Aug 2021 10:36:52 +0300 Subject: [PATCH 029/161] Update docs/en/engines/database-engines/materialized-mysql.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/engines/database-engines/materialized-mysql.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/database-engines/materialized-mysql.md b/docs/en/engines/database-engines/materialized-mysql.md index 815fa152fee..0e6528fe414 100644 --- a/docs/en/engines/database-engines/materialized-mysql.md +++ b/docs/en/engines/database-engines/materialized-mysql.md @@ -44,7 +44,7 @@ CREATE DATABASE mysql ENGINE = MaterializedMySQL('localhost:3306', 'db', 'user', max_wait_time_when_mysql_unavailable=10000; ``` -**Settings on Mysql-server Side** +**Settings on MySQL-server Side** For the correct work of `MaterializeMySQL`, there are few mandatory `MySQL`-side configuration settings that must be set: From 8201b5605f4899d439ca5c4a8dcd4abc41af2bdb Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Sat, 7 Aug 2021 10:37:14 +0300 Subject: [PATCH 030/161] Update docs/ru/engines/database-engines/materialized-mysql.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- .../engines/database-engines/materialized-mysql.md | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/ru/engines/database-engines/materialized-mysql.md b/docs/ru/engines/database-engines/materialized-mysql.md index 9ad89628e0b..13f33a2e952 100644 --- a/docs/ru/engines/database-engines/materialized-mysql.md +++ b/docs/ru/engines/database-engines/materialized-mysql.md @@ -29,12 +29,12 @@ ENGINE = MaterializedMySQL('host:port', ['database' | database], 'user', 'passwo **Настройки движка** -- `max_rows_in_buffer` — максимальное количество строк, содержимое которых может кэшироваться в памяти (для одной таблицы и данных кэша, которые невозможно запросить). При превышении количества строк, данные будут материализованы. Значение по умолчанию: `65 505`. -- `max_bytes_in_buffer` — максимальное количество байтов, которое разрешено кэшировать в памяти (для одной таблицы и данных кэша, которые невозможно запросить). При превышении количества строк, данные будут материализованы. Значение по умолчанию: `1 048 576`. -- `max_rows_in_buffers` — максимальное количество строк, содержимое которых может кэшироваться в памяти (для базы данных и данных кэша, которые невозможно запросить). При превышении количества строк, данные будут материализованы. Значение по умолчанию: `65 505`. -- `max_bytes_in_buffers` — максимальное количество байтов, которое разрешено кэшировать данным в памяти (для базы данных и данных кэша, которые невозможно запросить). При превышении количества строк, данные будут материализованы. Значение по умолчанию: `1 048 576`. -- `max_flush_data_time` — максимальное время в миллисекундах, в течение которого разрешено кэшировать данные в памяти (для базы данных и данных кэша, которые невозможно запросить). При превышении количества указанного периода, данные будут материализованы. Значение по умолчанию: `1000`. -- `max_wait_time_when_mysql_unavailable` — интервал повторных попыток, если MySQL недоступен. Указывается в миллисекундах. Отрицательное значение отключает повторные попытки. Значение по умолчанию: `1000`. +- `max_rows_in_buffer` — максимальное количество строк, содержимое которых может кешироваться в памяти (для одной таблицы и данных кеша, которые невозможно запросить). При превышении количества строк, данные будут материализованы. Значение по умолчанию: `65 505`. +- `max_bytes_in_buffer` — максимальное количество байтов, которое разрешено кешировать в памяти (для одной таблицы и данных кеша, которые невозможно запросить). При превышении количества строк, данные будут материализованы. Значение по умолчанию: `1 048 576`. +- `max_rows_in_buffers` — максимальное количество строк, содержимое которых может кешироваться в памяти (для базы данных и данных кеша, которые невозможно запросить). При превышении количества строк, данные будут материализованы. Значение по умолчанию: `65 505`. +- `max_bytes_in_buffers` — максимальное количество байтов, которое разрешено кешировать данным в памяти (для базы данных и данных кеша, которые невозможно запросить). При превышении количества строк, данные будут материализованы. Значение по умолчанию: `1 048 576`. +- `max_flush_data_time` — максимальное время в миллисекундах, в течение которого разрешено кешировать данные в памяти (для базы данных и данных кеша, которые невозможно запросить). При превышении количества указанного периода, данные будут материализованы. Значение по умолчанию: `1000`. +- `max_wait_time_when_mysql_unavailable` — интервал между повторными попытками, если MySQL недоступен. Указывается в миллисекундах. Отрицательное значение отключает повторные попытки. Значение по умолчанию: `1000`. - `allows_query_when_mysql_lost` — разрешить запрос материализованной таблицы при потере MySQL. Значение по умолчанию: `0` (`false`). ```sql From 094b191d708a6884bc9b36765d8ccc5c68e0a667 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Sat, 7 Aug 2021 10:37:22 +0300 Subject: [PATCH 031/161] Update docs/ru/engines/database-engines/materialized-mysql.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/engines/database-engines/materialized-mysql.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/engines/database-engines/materialized-mysql.md b/docs/ru/engines/database-engines/materialized-mysql.md index 13f33a2e952..2f2f79281ee 100644 --- a/docs/ru/engines/database-engines/materialized-mysql.md +++ b/docs/ru/engines/database-engines/materialized-mysql.md @@ -46,7 +46,7 @@ CREATE DATABASE mysql ENGINE = MaterializedMySQL('localhost:3306', 'db', 'user', **Настройки на стороне MySQL-сервера** -Для правильной работы `MaterializeMySQL` следует обязательно указать на MySQL сервере следующие параметры конфигурации: +Для правильной работы `MaterializeMySQL` следует обязательно указать на сервере MySQL следующие параметры конфигурации: - `default_authentication_plugin = mysql_native_password` — `MaterializeMySQL` может авторизоваться только с помощью этого метода. - `gtid_mode = on` — ведение журнала на основе GTID является обязательным для обеспечения правильной репликации. From a63d7bd36f53e1cb9055cc5c9518159596ce2407 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Sat, 7 Aug 2021 10:38:34 +0300 Subject: [PATCH 032/161] Update docs/en/engines/database-engines/materialized-mysql.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/engines/database-engines/materialized-mysql.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/database-engines/materialized-mysql.md b/docs/en/engines/database-engines/materialized-mysql.md index 0e6528fe414..a80bfe5f6bc 100644 --- a/docs/en/engines/database-engines/materialized-mysql.md +++ b/docs/en/engines/database-engines/materialized-mysql.md @@ -34,7 +34,7 @@ ENGINE = MaterializedMySQL('host:port', ['database' | database], 'user', 'passwo - `max_rows_in_buffers` — Maximum number of rows that data is allowed to cache in memory (for database and the cache data unable to query). When this number is exceeded, the data will be materialized. Default: `65 505`. - `max_bytes_in_buffers` — Maximum number of bytes that data is allowed to cache in memory (for database and the cache data unable to query). When this number is exceeded, the data will be materialized. Default: `1 048 576`. - `max_flush_data_time` — Maximum number of milliseconds that data is allowed to cache in memory (for database and the cache data unable to query). When this time is exceeded, the data will be materialized. Default: `1000`. -- `max_wait_time_when_mysql_unavailable` — Retry interval when MySQL is not available (milliseconds). Negative value disable retry. Default: `1000`. +- `max_wait_time_when_mysql_unavailable` — Retry interval when MySQL is not available (milliseconds). Negative value disables retry. Default: `1000`. - `allows_query_when_mysql_lost` — Allows to query a materialized table when MySQL is lost. Default: `0` (`false`). ```sql From 9ca422f0c5380124b57270ad18505e572d1b3afb Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 7 Aug 2021 08:11:40 +0000 Subject: [PATCH 033/161] Introduce CAST for internal usage --- src/Functions/FunctionsConversion.cpp | 3 + src/Functions/FunctionsConversion.h | 147 +++++++++++------- src/Interpreters/ActionsDAG.cpp | 6 +- .../ConvertStringsToEnumVisitor.cpp | 9 +- src/Interpreters/MutationsInterpreter.cpp | 4 +- ...OptimizeIfWithConstantConditionVisitor.cpp | 3 +- src/Interpreters/addTypeConversionToAST.cpp | 2 +- src/Interpreters/castColumn.cpp | 2 +- src/Interpreters/inplaceBlockConversions.cpp | 4 +- src/Parsers/ASTFunctionHelpers.h | 16 ++ .../Impl/ConstantExpressionTemplate.cpp | 2 +- src/Storages/MergeTree/KeyCondition.cpp | 2 +- .../configs/users.xml | 6 + .../test.py | 36 +++++ 14 files changed, 169 insertions(+), 73 deletions(-) create mode 100644 src/Parsers/ASTFunctionHelpers.h create mode 100644 tests/integration/test_alter_update_cast_keep_nullable/configs/users.xml create mode 100644 tests/integration/test_alter_update_cast_keep_nullable/test.py diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index d7686318efc..cdbd32b189c 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -43,6 +43,9 @@ void registerFunctionsConversion(FunctionFactory & factory) factory.registerFunction(); + factory.registerFunction>(FunctionFactory::CaseInsensitive); + factory.registerFunction>(); + factory.registerFunction>(); factory.registerFunction>(FunctionFactory::CaseInsensitive); factory.registerFunction>(); factory.registerFunction>(); diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 67a02e3fd34..b085e29a786 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -2412,7 +2412,8 @@ private: std::optional diagnostic; }; -struct NameCast { static constexpr auto name = "CAST"; }; +struct CastName { static constexpr auto name = "CAST"; }; +struct CastInternalName { static constexpr auto name = "_CAST"; }; enum class CastType { @@ -2421,17 +2422,26 @@ enum class CastType accurateOrNull }; -class FunctionCast final : public IFunctionBase +class FunctionCastBase : public IFunctionBase +{ +public: + using MonotonicityForRange = std::function; + using Diagnostic = ExecutableFunctionCast::Diagnostic; +}; + +template +class FunctionCast final : public FunctionCastBase { public: using WrapperType = std::function; - using MonotonicityForRange = std::function; - using Diagnostic = ExecutableFunctionCast::Diagnostic; - FunctionCast(const char * name_, MonotonicityForRange && monotonicity_for_range_ - , const DataTypes & argument_types_, const DataTypePtr & return_type_ - , std::optional diagnostic_, CastType cast_type_) - : name(name_), monotonicity_for_range(std::move(monotonicity_for_range_)) + FunctionCast(const char * cast_name_ + , MonotonicityForRange && monotonicity_for_range_ + , const DataTypes & argument_types_ + , const DataTypePtr & return_type_ + , std::optional diagnostic_ + , CastType cast_type_) + : cast_name(cast_name_), monotonicity_for_range(std::move(monotonicity_for_range_)) , argument_types(argument_types_), return_type(return_type_), diagnostic(std::move(diagnostic_)) , cast_type(cast_type_) { @@ -2445,7 +2455,7 @@ public: try { return std::make_unique( - prepareUnpackDictionaries(getArgumentTypes()[0], getResultType()), name, diagnostic); + prepareUnpackDictionaries(getArgumentTypes()[0], getResultType()), cast_name, diagnostic); } catch (Exception & e) { @@ -2456,7 +2466,7 @@ public: } } - String getName() const override { return name; } + String getName() const override { return cast_name; } bool isDeterministic() const override { return true; } bool isDeterministicInScopeOfQuery() const override { return true; } @@ -2473,7 +2483,7 @@ public: private: - const char * name; + const char * cast_name; MonotonicityForRange monotonicity_for_range; DataTypes argument_types; @@ -2515,7 +2525,7 @@ private: { /// In case when converting to Nullable type, we apply different parsing rule, /// that will not throw an exception but return NULL in case of malformed input. - FunctionPtr function = FunctionConvertFromString::create(); + FunctionPtr function = FunctionConvertFromString::create(); return createFunctionAdaptor(function, from_type); } else if (!can_apply_accurate_cast) @@ -2539,12 +2549,12 @@ private: { if (wrapper_cast_type == CastType::accurate) { - result_column = ConvertImpl::execute( + result_column = ConvertImpl::execute( arguments, result_type, input_rows_count, AccurateConvertStrategyAdditions()); } else { - result_column = ConvertImpl::execute( + result_column = ConvertImpl::execute( arguments, result_type, input_rows_count, AccurateOrNullConvertStrategyAdditions()); } @@ -2559,7 +2569,7 @@ private: { if (wrapper_cast_type == CastType::accurateOrNull) { - auto nullable_column_wrapper = FunctionCast::createToNullableColumnWrapper(); + auto nullable_column_wrapper = FunctionCast::createToNullableColumnWrapper(); return nullable_column_wrapper(arguments, result_type, column_nullable, input_rows_count); } else @@ -2631,7 +2641,7 @@ private: { AccurateConvertStrategyAdditions additions; additions.scale = scale; - result_column = ConvertImpl::execute( + result_column = ConvertImpl::execute( arguments, result_type, input_rows_count, additions); return true; @@ -2640,7 +2650,7 @@ private: { AccurateOrNullConvertStrategyAdditions additions; additions.scale = scale; - result_column = ConvertImpl::execute( + result_column = ConvertImpl::execute( arguments, result_type, input_rows_count, additions); return true; @@ -2653,14 +2663,14 @@ private: /// Consistent with CAST(Nullable(String) AS Nullable(Numbers)) /// In case when converting to Nullable type, we apply different parsing rule, /// that will not throw an exception but return NULL in case of malformed input. - result_column = ConvertImpl::execute( + result_column = ConvertImpl::execute( arguments, result_type, input_rows_count, scale); return true; } } - result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, scale); + result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, scale); return true; }); @@ -2670,7 +2680,7 @@ private: { if (wrapper_cast_type == CastType::accurateOrNull) { - auto nullable_column_wrapper = FunctionCast::createToNullableColumnWrapper(); + auto nullable_column_wrapper = FunctionCast::createToNullableColumnWrapper(); return nullable_column_wrapper(arguments, result_type, column_nullable, input_rows_count); } else @@ -2990,7 +3000,7 @@ private: template WrapperType createStringToEnumWrapper() const { - const char * function_name = name; + const char * function_name = cast_name; return [function_name] ( ColumnsWithTypeAndName & arguments, const DataTypePtr & res_type, const ColumnNullable * nullable_col, size_t /*input_rows_count*/) { @@ -3324,7 +3334,7 @@ private: class MonotonicityHelper { public: - using MonotonicityForRange = FunctionCast::MonotonicityForRange; + using MonotonicityForRange = FunctionCastBase::MonotonicityForRange; template static auto monotonicityForType(const DataType * const) @@ -3382,35 +3392,24 @@ public: } }; -template -class CastOverloadResolver : public IFunctionOverloadResolver + +/* + * CastInternal does not preserve nullability of the data type, + * i.e. CastInternal(toNullable(toInt8(1)) as Int32) will be Int32(1). + * + * Cast preserves nullability according to setting `cast_keep_nullable`, + * i.e. Cast(toNullable(toInt8(1)) as Int32) will be Nullable(Int32(1)) if `cast_keep_nullable` == 1. +**/ +template +class CastOverloadResolverImpl : public IFunctionOverloadResolver { public: - using MonotonicityForRange = FunctionCast::MonotonicityForRange; - using Diagnostic = FunctionCast::Diagnostic; - - static constexpr auto accurate_cast_name = "accurateCast"; - static constexpr auto accurate_cast_or_null_name = "accurateCastOrNull"; - static constexpr auto cast_name = "CAST"; + using MonotonicityForRange = FunctionCastBase::MonotonicityForRange; + using Diagnostic = FunctionCastBase::Diagnostic; static constexpr auto name = cast_type == CastType::accurate - ? accurate_cast_name - : (cast_type == CastType::accurateOrNull ? accurate_cast_or_null_name : cast_name); - - static FunctionOverloadResolverPtr create(ContextPtr context) - { - return createImpl(context->getSettingsRef().cast_keep_nullable); - } - - static FunctionOverloadResolverPtr createImpl(bool keep_nullable, std::optional diagnostic = {}) - { - return std::make_unique(keep_nullable, std::move(diagnostic)); - } - - - explicit CastOverloadResolver(bool keep_nullable_, std::optional diagnostic_ = {}) - : keep_nullable(keep_nullable_), diagnostic(std::move(diagnostic_)) - {} + ? CastName::accurate_cast_name + : (cast_type == CastType::accurateOrNull ? CastName::accurate_cast_or_null_name : CastName::cast_name); String getName() const override { return name; } @@ -3418,6 +3417,24 @@ public: ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } + explicit CastOverloadResolverImpl(std::optional diagnostic_, bool keep_nullable_) + : diagnostic(std::move(diagnostic_)), keep_nullable(keep_nullable_) + { + } + + static FunctionOverloadResolverPtr create(ContextPtr context) + { + if constexpr (internal) + return createImpl(); + return createImpl({}, context->getSettingsRef().cast_keep_nullable); + } + + static FunctionOverloadResolverPtr createImpl(std::optional diagnostic = {}, bool keep_nullable = false) + { + assert(!internal || !keep_nullable); + return std::make_unique(std::move(diagnostic), keep_nullable); + } + protected: FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const override @@ -3428,7 +3445,7 @@ protected: data_types[i] = arguments[i].type; auto monotonicity = MonotonicityHelper::getMonotonicityInformation(arguments.front().type, return_type.get()); - return std::make_unique(name, std::move(monotonicity), data_types, return_type, diagnostic, cast_type); + return std::make_unique>(name, std::move(monotonicity), data_types, return_type, diagnostic, cast_type); } DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override @@ -3448,23 +3465,41 @@ protected: DataTypePtr type = DataTypeFactory::instance().get(type_col->getValue()); if constexpr (cast_type == CastType::accurateOrNull) - { return makeNullable(type); - } - else - { - if (keep_nullable && arguments.front().type->isNullable()) - return makeNullable(type); + + if constexpr (internal) return type; - } + + if (keep_nullable && arguments.front().type->isNullable()) + return makeNullable(type); + + return type; } bool useDefaultImplementationForNulls() const override { return false; } bool useDefaultImplementationForLowCardinalityColumns() const override { return false; } private: - bool keep_nullable; std::optional diagnostic; + bool keep_nullable; }; + +struct CastOverloadName +{ + static constexpr auto cast_name = "CAST"; + static constexpr auto accurate_cast_name = "CastAccurate"; + static constexpr auto accurate_cast_or_null_name = "CastAccurateOrNull"; +}; + +struct CastInternalOverloadName +{ + static constexpr auto cast_name = "_CAST"; + static constexpr auto accurate_cast_name = "_CastAccurate"; + static constexpr auto accurate_cast_or_null_name = "_CastAccurateOrNull"; +}; + +template using CastOverloadResolver = CastOverloadResolverImpl; +template using CastInternalOverloadResolver = CastOverloadResolverImpl; + } diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 63b0345b372..a42d6053e9a 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -1110,8 +1110,8 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions( const auto * right_arg = &actions_dag->addColumn(std::move(column)); const auto * left_arg = dst_node; - FunctionCast::Diagnostic diagnostic = {dst_node->result_name, res_elem.name}; - FunctionOverloadResolverPtr func_builder_cast = CastOverloadResolver::createImpl(false, std::move(diagnostic)); + FunctionCastBase::Diagnostic diagnostic = {dst_node->result_name, res_elem.name}; + FunctionOverloadResolverPtr func_builder_cast = CastInternalOverloadResolver::createImpl(std::move(diagnostic)); NodeRawConstPtrs children = { left_arg, right_arg }; dst_node = &actions_dag->addFunction(func_builder_cast, std::move(children), {}); @@ -1876,7 +1876,7 @@ ActionsDAGPtr ActionsDAG::cloneActionsForFilterPushDown( predicate->children = {left_arg, right_arg}; auto arguments = prepareFunctionArguments(predicate->children); - FunctionOverloadResolverPtr func_builder_cast = CastOverloadResolver::createImpl(false); + FunctionOverloadResolverPtr func_builder_cast = CastInternalOverloadResolver::createImpl(); predicate->function_builder = func_builder_cast; predicate->function_base = predicate->function_builder->build(arguments); diff --git a/src/Interpreters/ConvertStringsToEnumVisitor.cpp b/src/Interpreters/ConvertStringsToEnumVisitor.cpp index fa2e0b6613a..e483bc9b5b6 100644 --- a/src/Interpreters/ConvertStringsToEnumVisitor.cpp +++ b/src/Interpreters/ConvertStringsToEnumVisitor.cpp @@ -43,11 +43,11 @@ void changeIfArguments(ASTPtr & first, ASTPtr & second) String enum_string = makeStringsEnum(values); auto enum_literal = std::make_shared(enum_string); - auto first_cast = makeASTFunction("CAST"); + auto first_cast = makeASTFunction("_CAST"); first_cast->arguments->children.push_back(first); first_cast->arguments->children.push_back(enum_literal); - auto second_cast = makeASTFunction("CAST"); + auto second_cast = makeASTFunction("_CAST"); second_cast->arguments->children.push_back(second); second_cast->arguments->children.push_back(enum_literal); @@ -65,12 +65,12 @@ void changeTransformArguments(ASTPtr & array_to, ASTPtr & other) String enum_string = makeStringsEnum(values); - auto array_cast = makeASTFunction("CAST"); + auto array_cast = makeASTFunction("_CAST"); array_cast->arguments->children.push_back(array_to); array_cast->arguments->children.push_back(std::make_shared("Array(" + enum_string + ")")); array_to = array_cast; - auto other_cast = makeASTFunction("CAST"); + auto other_cast = makeASTFunction("_CAST"); other_cast->arguments->children.push_back(other); other_cast->arguments->children.push_back(std::make_shared(enum_string)); other = other_cast; @@ -183,4 +183,3 @@ void ConvertStringsToEnumMatcher::visit(ASTFunction & function_node, Data & data } } - diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index fe0594bb58f..a5c625ef4f8 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -503,10 +503,10 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) } } - auto updated_column = makeASTFunction("CAST", + auto updated_column = makeASTFunction("_CAST", makeASTFunction("if", condition, - makeASTFunction("CAST", + makeASTFunction("_CAST", update_expr->clone(), type_literal), std::make_shared(column)), diff --git a/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp b/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp index a8e2d371e05..a9814ce50f5 100644 --- a/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp +++ b/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -29,7 +30,7 @@ static bool tryExtractConstValueFromCondition(const ASTPtr & condition, bool & v /// cast of numeric constant in condition to UInt8 if (const auto * function = condition->as()) { - if (function->name == "CAST") + if (isFunctionCast(function)) { if (const auto * expr_list = function->arguments->as()) { diff --git a/src/Interpreters/addTypeConversionToAST.cpp b/src/Interpreters/addTypeConversionToAST.cpp index ba67ec762a9..2f766880253 100644 --- a/src/Interpreters/addTypeConversionToAST.cpp +++ b/src/Interpreters/addTypeConversionToAST.cpp @@ -20,7 +20,7 @@ namespace ErrorCodes ASTPtr addTypeConversionToAST(ASTPtr && ast, const String & type_name) { - auto func = makeASTFunction("CAST", ast, std::make_shared(type_name)); + auto func = makeASTFunction("_CAST", ast, std::make_shared(type_name)); if (ASTWithAlias * ast_with_alias = dynamic_cast(ast.get())) { diff --git a/src/Interpreters/castColumn.cpp b/src/Interpreters/castColumn.cpp index 181cca1e017..3356d37ba7f 100644 --- a/src/Interpreters/castColumn.cpp +++ b/src/Interpreters/castColumn.cpp @@ -21,7 +21,7 @@ static ColumnPtr castColumn(const ColumnWithTypeAndName & arg, const DataTypePtr } }; - FunctionOverloadResolverPtr func_builder_cast = CastOverloadResolver::createImpl(false); + FunctionOverloadResolverPtr func_builder_cast = CastInternalOverloadResolver::createImpl(); auto func_cast = func_builder_cast->build(arguments); diff --git a/src/Interpreters/inplaceBlockConversions.cpp b/src/Interpreters/inplaceBlockConversions.cpp index 26cf6912bc7..e40e0635a85 100644 --- a/src/Interpreters/inplaceBlockConversions.cpp +++ b/src/Interpreters/inplaceBlockConversions.cpp @@ -52,7 +52,7 @@ void addDefaultRequiredExpressionsRecursively( RequiredSourceColumnsVisitor(columns_context).visit(column_default_expr); NameSet required_columns_names = columns_context.requiredColumns(); - auto expr = makeASTFunction("CAST", column_default_expr, std::make_shared(columns.get(required_column_name).type->getName())); + auto expr = makeASTFunction("_CAST", column_default_expr, std::make_shared(columns.get(required_column_name).type->getName())); if (is_column_in_query && convert_null_to_default) expr = makeASTFunction("ifNull", std::make_shared(required_column_name), std::move(expr)); @@ -101,7 +101,7 @@ ASTPtr convertRequiredExpressions(Block & block, const NamesAndTypesList & requi continue; auto cast_func = makeASTFunction( - "CAST", std::make_shared(required_column.name), std::make_shared(required_column.type->getName())); + "_CAST", std::make_shared(required_column.name), std::make_shared(required_column.type->getName())); conversion_expr_list->children.emplace_back(setAlias(cast_func, required_column.name)); diff --git a/src/Parsers/ASTFunctionHelpers.h b/src/Parsers/ASTFunctionHelpers.h new file mode 100644 index 00000000000..76da2dd1501 --- /dev/null +++ b/src/Parsers/ASTFunctionHelpers.h @@ -0,0 +1,16 @@ +#pragma once + +#include + + +namespace DB +{ + +static bool isFunctionCast(const ASTFunction * function) +{ + if (function) + return function->name == "CAST" || function->name == "_CAST"; + return false; +} + +} diff --git a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp index cc9ae5e65bb..1f780a206dd 100644 --- a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp +++ b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp @@ -639,7 +639,7 @@ void ConstantExpressionTemplate::TemplateStructure::addNodesToCastResult(const I expr = makeASTFunction("assumeNotNull", std::move(expr)); } - expr = makeASTFunction("CAST", std::move(expr), std::make_shared(result_column_type.getName())); + expr = makeASTFunction("_CAST", std::move(expr), std::make_shared(result_column_type.getName())); if (null_as_default) { diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 235cadfba11..e6dfdb859b8 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -1367,7 +1367,7 @@ bool KeyCondition::tryParseAtomFromAST(const ASTPtr & node, ContextPtr context, { ColumnsWithTypeAndName arguments{ {nullptr, key_expr_type, ""}, {DataTypeString().createColumnConst(1, common_type->getName()), common_type, ""}}; - FunctionOverloadResolverPtr func_builder_cast = CastOverloadResolver::createImpl(false); + FunctionOverloadResolverPtr func_builder_cast = CastInternalOverloadResolver::createImpl(); auto func_cast = func_builder_cast->build(arguments); /// If we know the given range only contains one value, then we treat all functions as positive monotonic. diff --git a/tests/integration/test_alter_update_cast_keep_nullable/configs/users.xml b/tests/integration/test_alter_update_cast_keep_nullable/configs/users.xml new file mode 100644 index 00000000000..16b032daacb --- /dev/null +++ b/tests/integration/test_alter_update_cast_keep_nullable/configs/users.xml @@ -0,0 +1,6 @@ + + + + 1 + + diff --git a/tests/integration/test_alter_update_cast_keep_nullable/test.py b/tests/integration/test_alter_update_cast_keep_nullable/test.py new file mode 100644 index 00000000000..497a9e21d94 --- /dev/null +++ b/tests/integration/test_alter_update_cast_keep_nullable/test.py @@ -0,0 +1,36 @@ +import pytest +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + +node1 = cluster.add_instance('node1', user_configs=['configs/users.xml'], with_zookeeper=True) + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + +def test_cast_keep_nullable(started_cluster): + setting = node1.query("SELECT value FROM system.settings WHERE name='cast_keep_nullable'") + assert(setting.strip() == "1") + + result = node1.query(""" + DROP TABLE IF EXISTS t; + CREATE TABLE t (x UInt64) ENGINE = MergeTree ORDER BY tuple(); + INSERT INTO t SELECT number FROM numbers(10); + SELECT * FROM t; + """) + assert(result.strip() == "0\n1\n2\n3\n4\n5\n6\n7\n8\n9") + + error = node1.query_and_get_error(""" + SET mutations_sync = 1; + ALTER TABLE t UPDATE x = x % 3 = 0 ? NULL : x WHERE x % 2 = 1;  + """) + assert("DB::Exception: Cannot convert NULL value to non-Nullable type" in error) + + result = node1.query("SELECT * FROM t;") + assert(result.strip() == "0\n1\n2\n3\n4\n5\n6\n7\n8\n9") + From 365e743831cefc1ffece62d8a7f3379fec94be94 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 7 Aug 2021 09:03:23 +0000 Subject: [PATCH 034/161] Temporarily set cast_keep_nullable = 1 --- src/Core/Settings.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 20404089210..5b75ca0d3ab 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -432,7 +432,8 @@ class IColumn; M(String, function_implementation, "", "Choose function implementation for specific target or variant (experimental). If empty enable all of them.", 0) \ M(Bool, allow_experimental_geo_types, false, "Allow geo data types such as Point, Ring, Polygon, MultiPolygon", 0) \ M(Bool, data_type_default_nullable, false, "Data types without NULL or NOT NULL will make Nullable", 0) \ - M(Bool, cast_keep_nullable, false, "CAST operator keep Nullable for result data type", 0) \ + /** Temporarily set to true, to check how tests will feel.*/ \ + M(Bool, cast_keep_nullable, true, "CAST operator keep Nullable for result data type", 0) \ M(Bool, alter_partition_verbose_result, false, "Output information about affected parts. Currently works only for FREEZE and ATTACH commands.", 0) \ M(Bool, allow_experimental_database_materialized_mysql, false, "Allow to create database with Engine=MaterializedMySQL(...).", 0) \ M(Bool, allow_experimental_database_materialized_postgresql, false, "Allow to create database with Engine=MaterializedPostgreSQL(...).", 0) \ From ec9e82fb0c648ea98950913337d2bdf83cadc3e9 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 7 Aug 2021 10:12:59 +0000 Subject: [PATCH 035/161] Fix some tests --- src/Functions/FunctionsConversion.h | 8 ++++---- .../0_stateless/01029_early_constant_folding.reference | 2 +- .../01324_if_transform_strings_to_enum.reference | 2 +- .../01455_nullable_type_with_if_agg_combinator.sql | 2 ++ .../01611_constant_folding_subqueries.reference | 2 +- 5 files changed, 9 insertions(+), 7 deletions(-) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index b085e29a786..c277997593c 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -3488,15 +3488,15 @@ private: struct CastOverloadName { static constexpr auto cast_name = "CAST"; - static constexpr auto accurate_cast_name = "CastAccurate"; - static constexpr auto accurate_cast_or_null_name = "CastAccurateOrNull"; + static constexpr auto accurate_cast_name = "accurateCast"; + static constexpr auto accurate_cast_or_null_name = "accurateCastOrNull"; }; struct CastInternalOverloadName { static constexpr auto cast_name = "_CAST"; - static constexpr auto accurate_cast_name = "_CastAccurate"; - static constexpr auto accurate_cast_or_null_name = "_CastAccurateOrNull"; + static constexpr auto accurate_cast_name = "accurate_Cast"; + static constexpr auto accurate_cast_or_null_name = "accurate_CastOrNull"; }; template using CastOverloadResolver = CastOverloadResolverImpl; diff --git a/tests/queries/0_stateless/01029_early_constant_folding.reference b/tests/queries/0_stateless/01029_early_constant_folding.reference index 88139b7e2b8..abcb2ddc6a7 100644 --- a/tests/queries/0_stateless/01029_early_constant_folding.reference +++ b/tests/queries/0_stateless/01029_early_constant_folding.reference @@ -2,7 +2,7 @@ SELECT 1 WHERE 0 SELECT 1 SELECT 1 -WHERE (1 IN (0, 2)) AND (2 = (identity(CAST(2, \'Nullable(UInt8)\')) AS subquery)) +WHERE (1 IN (0, 2)) AND (2 = (identity(_CAST(2, \'Nullable(UInt8)\')) AS subquery)) SELECT 1 WHERE 1 IN (( SELECT arrayJoin([1, 2, 3]) diff --git a/tests/queries/0_stateless/01324_if_transform_strings_to_enum.reference b/tests/queries/0_stateless/01324_if_transform_strings_to_enum.reference index 99298d28bad..994e3f24aaf 100644 --- a/tests/queries/0_stateless/01324_if_transform_strings_to_enum.reference +++ b/tests/queries/0_stateless/01324_if_transform_strings_to_enum.reference @@ -8,7 +8,7 @@ yahoo other other other -SELECT transform(number, [2, 4, 6], CAST([\'google\', \'yandex\', \'yahoo\'], \'Array(Enum8(\\\'google\\\' = 1, \\\'other\\\' = 2, \\\'yahoo\\\' = 3, \\\'yandex\\\' = 4))\'), CAST(\'other\', \'Enum8(\\\'google\\\' = 1, \\\'other\\\' = 2, \\\'yahoo\\\' = 3, \\\'yandex\\\' = 4)\')) +SELECT transform(number, [2, 4, 6], _CAST([\'google\', \'yandex\', \'yahoo\'], \'Array(Enum8(\\\'google\\\' = 1, \\\'other\\\' = 2, \\\'yahoo\\\' = 3, \\\'yandex\\\' = 4))\'), _CAST(\'other\', \'Enum8(\\\'google\\\' = 1, \\\'other\\\' = 2, \\\'yahoo\\\' = 3, \\\'yandex\\\' = 4)\')) FROM system.numbers LIMIT 10 google diff --git a/tests/queries/0_stateless/01455_nullable_type_with_if_agg_combinator.sql b/tests/queries/0_stateless/01455_nullable_type_with_if_agg_combinator.sql index 852660117f5..0e951af73d0 100644 --- a/tests/queries/0_stateless/01455_nullable_type_with_if_agg_combinator.sql +++ b/tests/queries/0_stateless/01455_nullable_type_with_if_agg_combinator.sql @@ -1,3 +1,5 @@ +SET cast_keep_nullable = 0; + -- Value nullable SELECT anyIf(CAST(number, 'Nullable(UInt8)'), number = 3) AS a, toTypeName(a) FROM numbers(2); -- Value and condition nullable diff --git a/tests/queries/0_stateless/01611_constant_folding_subqueries.reference b/tests/queries/0_stateless/01611_constant_folding_subqueries.reference index 6128cd109e2..c3df2314112 100644 --- a/tests/queries/0_stateless/01611_constant_folding_subqueries.reference +++ b/tests/queries/0_stateless/01611_constant_folding_subqueries.reference @@ -5,7 +5,7 @@ SELECT (SELECT * FROM system.numbers LIMIT 1 OFFSET 1) AS n, toUInt64(10 / n) FO 1,10 EXPLAIN SYNTAX SELECT (SELECT * FROM system.numbers LIMIT 1 OFFSET 1) AS n, toUInt64(10 / n); SELECT - identity(CAST(0, \'Nullable(UInt64)\')) AS n, + identity(_CAST(0, \'Nullable(UInt64)\')) AS n, toUInt64(10 / n) SELECT * FROM (WITH (SELECT * FROM system.numbers LIMIT 1 OFFSET 1) AS n, toUInt64(10 / n) as q SELECT * FROM system.one WHERE q > 0); 0 From 01b4bd3f91806d8c806a5dfc9af21c69b99099e5 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 7 Aug 2021 11:31:53 +0000 Subject: [PATCH 036/161] Fix some more tests --- src/Functions/FunctionsConversion.h | 2 +- .../0_stateless/00974_low_cardinality_cast.sql | 2 ++ .../01402_cast_nullable_string_to_enum.sql | 2 ++ tests/queries/0_stateless/01536_fuzz_cast.sql | 1 + .../0_stateless/01655_plan_optimizations.reference | 2 +- .../queries/0_stateless/01655_plan_optimizations.sh | 2 +- .../0_stateless/01720_join_implicit_cast.sql.j2 | 4 ++-- ..._optimize_skip_unused_shards_rewrite_in.reference | 12 ++++++------ 8 files changed, 16 insertions(+), 11 deletions(-) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index c277997593c..362249b3b27 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -3470,7 +3470,7 @@ protected: if constexpr (internal) return type; - if (keep_nullable && arguments.front().type->isNullable()) + if (keep_nullable && arguments.front().type->isNullable() && type->canBeInsideNullable()) return makeNullable(type); return type; diff --git a/tests/queries/0_stateless/00974_low_cardinality_cast.sql b/tests/queries/0_stateless/00974_low_cardinality_cast.sql index e369a8c169e..b52c00513d3 100644 --- a/tests/queries/0_stateless/00974_low_cardinality_cast.sql +++ b/tests/queries/0_stateless/00974_low_cardinality_cast.sql @@ -1,3 +1,5 @@ +SET cast_keep_nullable = 0; + SELECT CAST('Hello' AS LowCardinality(Nullable(String))); SELECT CAST(Null AS LowCardinality(Nullable(String))); SELECT CAST(CAST('Hello' AS LowCardinality(Nullable(String))) AS String); diff --git a/tests/queries/0_stateless/01402_cast_nullable_string_to_enum.sql b/tests/queries/0_stateless/01402_cast_nullable_string_to_enum.sql index aa8e8be6673..3b53e593095 100644 --- a/tests/queries/0_stateless/01402_cast_nullable_string_to_enum.sql +++ b/tests/queries/0_stateless/01402_cast_nullable_string_to_enum.sql @@ -1,3 +1,5 @@ +SET cast_keep_nullable = 0; + -- https://github.com/ClickHouse/ClickHouse/issues/5818#issuecomment-619628445 SELECT CAST(CAST(NULL AS Nullable(String)) AS Nullable(Enum8('Hello' = 1))); SELECT CAST(CAST(NULL AS Nullable(FixedString(1))) AS Nullable(Enum8('Hello' = 1))); diff --git a/tests/queries/0_stateless/01536_fuzz_cast.sql b/tests/queries/0_stateless/01536_fuzz_cast.sql index 436d76b5c4c..fb1303549b6 100644 --- a/tests/queries/0_stateless/01536_fuzz_cast.sql +++ b/tests/queries/0_stateless/01536_fuzz_cast.sql @@ -1 +1,2 @@ +SET cast_keep_nullable = 0; SELECT CAST(arrayJoin([NULL, '', '', NULL, '', NULL, '01.02.2017 03:04\005GMT', '', NULL, '01/02/2017 03:04:05 MSK01/02/\0017 03:04:05 MSK', '', NULL, '03/04/201903/04/201903/04/\001903/04/2019']), 'Enum8(\'a\' = 1, \'b\' = 2)') AS x; -- { serverError 349 } diff --git a/tests/queries/0_stateless/01655_plan_optimizations.reference b/tests/queries/0_stateless/01655_plan_optimizations.reference index 22f5a2e73e3..22876207862 100644 --- a/tests/queries/0_stateless/01655_plan_optimizations.reference +++ b/tests/queries/0_stateless/01655_plan_optimizations.reference @@ -56,7 +56,7 @@ Filter column: notEquals(y, 0) 9 10 > one condition of filter should be pushed down after aggregating, other condition is casted Filter column -FUNCTION CAST(minus(s, 4) :: 1, UInt8 :: 3) -> and(notEquals(y, 0), minus(s, 4)) +FUNCTION _CAST(minus(s, 4) :: 1, UInt8 :: 3) -> and(notEquals(y, 0), minus(s, 4)) Aggregating Filter column: notEquals(y, 0) 0 1 diff --git a/tests/queries/0_stateless/01655_plan_optimizations.sh b/tests/queries/0_stateless/01655_plan_optimizations.sh index 148e6157773..b835bae0e27 100755 --- a/tests/queries/0_stateless/01655_plan_optimizations.sh +++ b/tests/queries/0_stateless/01655_plan_optimizations.sh @@ -56,7 +56,7 @@ $CLICKHOUSE_CLIENT -q " select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 and s - 4 settings enable_optimize_predicate_expression=0" | - grep -o "Aggregating\|Filter column\|Filter column: notEquals(y, 0)\|FUNCTION CAST(minus(s, 4) :: 1, UInt8 :: 3) -> and(notEquals(y, 0), minus(s, 4))" + grep -o "Aggregating\|Filter column\|Filter column: notEquals(y, 0)\|FUNCTION _CAST(minus(s, 4) :: 1, UInt8 :: 3) -> and(notEquals(y, 0), minus(s, 4))" $CLICKHOUSE_CLIENT -q " select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y diff --git a/tests/queries/0_stateless/01720_join_implicit_cast.sql.j2 b/tests/queries/0_stateless/01720_join_implicit_cast.sql.j2 index f7760c38163..f2b13e9824b 100644 --- a/tests/queries/0_stateless/01720_join_implicit_cast.sql.j2 +++ b/tests/queries/0_stateless/01720_join_implicit_cast.sql.j2 @@ -48,8 +48,8 @@ SELECT any(toTypeName(a)) == 'Int32' AND any(toTypeName(b)) == 'Nullable(Int64)' SELECT any(toTypeName(a)) == 'Int32' AND any(toTypeName(b)) == 'Nullable(Int64)' FROM t_ab1 RIGHT JOIN t_ab2 USING (a, b); SELECT any(toTypeName(a)) == 'Int32' AND any(toTypeName(b)) == 'Nullable(Int64)' FROM t_ab1 INNER JOIN t_ab2 USING (a, b); -SELECT * FROM ( SELECT a, b as "CAST(a, Int32)" FROM t_ab1 ) t_ab1 FULL JOIN t_ab2 ON (t_ab1.a == t_ab2.a); -- { serverError 44 } -SELECT * FROM ( SELECT a, b as "CAST(a, Int32)" FROM t_ab1 ) t_ab1 FULL JOIN t_ab2 USING (a) FORMAT Null; +SELECT * FROM ( SELECT a, b as "_CAST(a, Int32)" FROM t_ab1 ) t_ab1 FULL JOIN t_ab2 ON (t_ab1.a == t_ab2.a); -- { serverError 44 } +SELECT * FROM ( SELECT a, b as "_CAST(a, Int32)" FROM t_ab1 ) t_ab1 FULL JOIN t_ab2 USING (a) FORMAT Null; {% endfor %} diff --git a/tests/queries/0_stateless/01756_optimize_skip_unused_shards_rewrite_in.reference b/tests/queries/0_stateless/01756_optimize_skip_unused_shards_rewrite_in.reference index 972f4c89bdf..66fbe8a5d1c 100644 --- a/tests/queries/0_stateless/01756_optimize_skip_unused_shards_rewrite_in.reference +++ b/tests/queries/0_stateless/01756_optimize_skip_unused_shards_rewrite_in.reference @@ -1,17 +1,17 @@ (0, 2) 0 0 0 0 -WITH CAST(\'default\', \'Nullable(String)\') AS id_no SELECT one.dummy, ignore(id_no) FROM system.one WHERE dummy IN (0, 2) -WITH CAST(\'default\', \'Nullable(String)\') AS id_no SELECT one.dummy, ignore(id_no) FROM system.one WHERE dummy IN (0, 2) +WITH _CAST(\'default\', \'Nullable(String)\') AS id_no SELECT one.dummy, ignore(id_no) FROM system.one WHERE dummy IN (0, 2) +WITH _CAST(\'default\', \'Nullable(String)\') AS id_no SELECT one.dummy, ignore(id_no) FROM system.one WHERE dummy IN (0, 2) optimize_skip_unused_shards_rewrite_in(0, 2) 0 0 -WITH CAST(\'default\', \'Nullable(String)\') AS id_02 SELECT one.dummy, ignore(id_02) FROM system.one WHERE dummy IN tuple(0) -WITH CAST(\'default\', \'Nullable(String)\') AS id_02 SELECT one.dummy, ignore(id_02) FROM system.one WHERE dummy IN tuple(2) +WITH _CAST(\'default\', \'Nullable(String)\') AS id_02 SELECT one.dummy, ignore(id_02) FROM system.one WHERE dummy IN tuple(0) +WITH _CAST(\'default\', \'Nullable(String)\') AS id_02 SELECT one.dummy, ignore(id_02) FROM system.one WHERE dummy IN tuple(2) optimize_skip_unused_shards_rewrite_in(2,) -WITH CAST(\'default\', \'Nullable(String)\') AS id_2 SELECT one.dummy, ignore(id_2) FROM system.one WHERE dummy IN tuple(2) +WITH _CAST(\'default\', \'Nullable(String)\') AS id_2 SELECT one.dummy, ignore(id_2) FROM system.one WHERE dummy IN tuple(2) optimize_skip_unused_shards_rewrite_in(0,) 0 0 -WITH CAST(\'default\', \'Nullable(String)\') AS id_0 SELECT one.dummy, ignore(id_0) FROM system.one WHERE dummy IN tuple(0) +WITH _CAST(\'default\', \'Nullable(String)\') AS id_0 SELECT one.dummy, ignore(id_0) FROM system.one WHERE dummy IN tuple(0) 0 0 errors From 5abe33e1a38cd44e585ec106468edc475d7170cb Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 8 Aug 2021 20:12:12 +0300 Subject: [PATCH 037/161] Better --- .../InterpreterIntersectOrExcept.cpp | 45 ++++++++++------- .../InterpreterIntersectOrExcept.h | 18 ++++--- .../QueryPlan/IntersectOrExceptStep.cpp | 50 +++++++++++++++++-- .../QueryPlan/IntersectOrExceptStep.h | 6 ++- .../Transforms/IntersectOrExceptTransform.cpp | 40 +++++++-------- .../Transforms/IntersectOrExceptTransform.h | 30 +++++------ 6 files changed, 119 insertions(+), 70 deletions(-) diff --git a/src/Interpreters/InterpreterIntersectOrExcept.cpp b/src/Interpreters/InterpreterIntersectOrExcept.cpp index c85bd29e16f..c85c39824d8 100644 --- a/src/Interpreters/InterpreterIntersectOrExcept.cpp +++ b/src/Interpreters/InterpreterIntersectOrExcept.cpp @@ -8,6 +8,8 @@ #include #include #include +#include + namespace DB { @@ -17,16 +19,20 @@ namespace ErrorCodes extern const int INTERSECT_OR_EXCEPT_RESULT_STRUCTURES_MISMATCH; } -InterpreterIntersectOrExcept::InterpreterIntersectOrExcept(const ASTPtr & query_ptr_, ContextPtr context_) - : query_ptr(query_ptr_), context(Context::createCopy(context_)) +InterpreterIntersectOrExcept::InterpreterIntersectOrExcept(const ASTPtr & query_ptr, ContextPtr context_) + : context(Context::createCopy(context_)) + , is_except(query_ptr->as()->is_except) { ASTIntersectOrExcept * ast = query_ptr->as(); + size_t num_children = ast->children.size(); + if (!num_children) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: no children in ASTIntersectOrExceptQuery"); + nested_interpreters.resize(num_children); + for (size_t i = 0; i < num_children; ++i) - { nested_interpreters[i] = buildCurrentChildInterpreter(ast->children[i]); - } Blocks headers(num_children); for (size_t query_num = 0; query_num < num_children; ++query_num) @@ -35,8 +41,7 @@ InterpreterIntersectOrExcept::InterpreterIntersectOrExcept(const ASTPtr & query_ result_header = getCommonHeader(headers); } - -Block InterpreterIntersectOrExcept::getCommonHeader(const Blocks & headers) +Block InterpreterIntersectOrExcept::getCommonHeader(const Blocks & headers) const { size_t num_selects = headers.size(); Block common_header = headers.front(); @@ -45,16 +50,12 @@ Block InterpreterIntersectOrExcept::getCommonHeader(const Blocks & headers) for (size_t query_num = 1; query_num < num_selects; ++query_num) { if (headers[query_num].columns() != num_columns) - throw Exception( - "Different number of columns in " - + toString(query_ptr->as()->is_except ? "EXCEPT" : "INTERSECT") - + " elements:\n" + common_header.dumpNames() + "\nand\n" - + headers[query_num].dumpNames() + "\n", - ErrorCodes::INTERSECT_OR_EXCEPT_RESULT_STRUCTURES_MISMATCH); + throw Exception(ErrorCodes::INTERSECT_OR_EXCEPT_RESULT_STRUCTURES_MISMATCH, + "Different number of columns in {} elements:\n {} \nand\n {}", + getName(), common_header.dumpNames(), headers[query_num].dumpNames()); } std::vector columns(num_selects); - for (size_t column_num = 0; column_num < num_columns; ++column_num) { for (size_t i = 0; i < num_selects; ++i) @@ -67,7 +68,6 @@ Block InterpreterIntersectOrExcept::getCommonHeader(const Blocks & headers) return common_header; } - std::unique_ptr InterpreterIntersectOrExcept::buildCurrentChildInterpreter(const ASTPtr & ast_ptr_) { @@ -80,7 +80,6 @@ InterpreterIntersectOrExcept::buildCurrentChildInterpreter(const ASTPtr & ast_pt void InterpreterIntersectOrExcept::buildQueryPlan(QueryPlan & query_plan) { size_t num_plans = nested_interpreters.size(); - std::vector> plans(num_plans); DataStreams data_streams(num_plans); @@ -88,12 +87,23 @@ void InterpreterIntersectOrExcept::buildQueryPlan(QueryPlan & query_plan) { plans[i] = std::make_unique(); nested_interpreters[i]->buildQueryPlan(*plans[i]); + + if (!blocksHaveEqualStructure(plans[i]->getCurrentDataStream().header, result_header)) + { + auto actions_dag = ActionsDAG::makeConvertingActions( + plans[i]->getCurrentDataStream().header.getColumnsWithTypeAndName(), + result_header.getColumnsWithTypeAndName(), + ActionsDAG::MatchColumnsMode::Position); + auto converting_step = std::make_unique(plans[i]->getCurrentDataStream(), std::move(actions_dag)); + converting_step->setStepDescription("Conversion before UNION"); + plans[i]->addStep(std::move(converting_step)); + } + data_streams[i] = plans[i]->getCurrentDataStream(); } auto max_threads = context->getSettingsRef().max_threads; - auto step = std::make_unique( - query_ptr->as()->is_except, std::move(data_streams), result_header, max_threads); + auto step = std::make_unique(is_except, std::move(data_streams), max_threads); query_plan.unitePlans(std::move(step), std::move(plans)); } @@ -113,4 +123,5 @@ BlockIO InterpreterIntersectOrExcept::execute() return res; } + } diff --git a/src/Interpreters/InterpreterIntersectOrExcept.h b/src/Interpreters/InterpreterIntersectOrExcept.h index 0069dc02f1d..34a58c0c05a 100644 --- a/src/Interpreters/InterpreterIntersectOrExcept.h +++ b/src/Interpreters/InterpreterIntersectOrExcept.h @@ -16,20 +16,22 @@ class InterpreterIntersectOrExcept : public IInterpreter public: InterpreterIntersectOrExcept(const ASTPtr & query_ptr_, ContextPtr context_); - /// Builds QueryPlan for current query. - virtual void buildQueryPlan(QueryPlan & query_plan); - BlockIO execute() override; private: - ASTPtr query_ptr; - ContextPtr context; - Block result_header; - std::vector> nested_interpreters; - Block getCommonHeader(const Blocks & headers); + String getName() const { return is_except ? "EXCEPT" : "INTERSECT"; } + + Block getCommonHeader(const Blocks & headers) const; std::unique_ptr buildCurrentChildInterpreter(const ASTPtr & ast_ptr_); + + void buildQueryPlan(QueryPlan & query_plan); + + ContextPtr context; + bool is_except; + Block result_header; + std::vector> nested_interpreters; }; } diff --git a/src/Processors/QueryPlan/IntersectOrExceptStep.cpp b/src/Processors/QueryPlan/IntersectOrExceptStep.cpp index 28f34bda5db..f04885f4640 100644 --- a/src/Processors/QueryPlan/IntersectOrExceptStep.cpp +++ b/src/Processors/QueryPlan/IntersectOrExceptStep.cpp @@ -1,18 +1,36 @@ #include +#include #include #include #include +#include #include #include + namespace DB { -IntersectOrExceptStep::IntersectOrExceptStep(bool is_except_, DataStreams input_streams_, Block result_header, size_t max_threads_) - : is_except(is_except_), header(std::move(result_header)), max_threads(max_threads_) +Block IntersectOrExceptStep::checkHeaders(const DataStreams & input_streams_) const +{ + if (input_streams_.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot perform {} on empty set of query plan steps", getName()); + + Block res = input_streams_.front().header; + for (const auto & stream : input_streams_) + assertBlocksHaveEqualStructure(stream.header, res, "IntersectOrExceptStep"); + + return res; +} + +IntersectOrExceptStep::IntersectOrExceptStep(bool is_except_, DataStreams input_streams_, size_t max_threads_) + : is_except(is_except_), header(checkHeaders(input_streams_)), max_threads(max_threads_) { input_streams = std::move(input_streams_); - output_stream = DataStream{.header = header}; + if (input_streams.size() == 1) + output_stream = input_streams.front(); + else + output_stream = DataStream{.header = header}; } QueryPipelinePtr IntersectOrExceptStep::updatePipeline(QueryPipelines pipelines, const BuildQueryPipelineSettings &) @@ -20,8 +38,30 @@ QueryPipelinePtr IntersectOrExceptStep::updatePipeline(QueryPipelines pipelines, auto pipeline = std::make_unique(); QueryPipelineProcessorsCollector collector(*pipeline, this); - pipelines[0]->addTransform(std::make_shared(header, pipelines[0]->getNumStreams(), 1)); - pipelines[1]->addTransform(std::make_shared(header, pipelines[1]->getNumStreams(), 1)); + if (pipelines.empty()) + { + pipeline->init(Pipe(std::make_shared(output_stream->header))); + processors = collector.detachProcessors(); + return pipeline; + } + + for (auto & cur_pipeline : pipelines) + { + /// Just in case. + if (!isCompatibleHeader(cur_pipeline->getHeader(), getOutputStream().header)) + { + auto converting_dag = ActionsDAG::makeConvertingActions( + cur_pipeline->getHeader().getColumnsWithTypeAndName(), + getOutputStream().header.getColumnsWithTypeAndName(), + ActionsDAG::MatchColumnsMode::Name); + + auto converting_actions = std::make_shared(std::move(converting_dag)); + cur_pipeline->addSimpleTransform([&](const Block & cur_header) + { + return std::make_shared(cur_header, converting_actions); + }); + } + } *pipeline = QueryPipeline::unitePipelines(std::move(pipelines), max_threads); pipeline->addTransform(std::make_shared(is_except, header)); diff --git a/src/Processors/QueryPlan/IntersectOrExceptStep.h b/src/Processors/QueryPlan/IntersectOrExceptStep.h index d2b515bb1c4..7938a9adad5 100644 --- a/src/Processors/QueryPlan/IntersectOrExceptStep.h +++ b/src/Processors/QueryPlan/IntersectOrExceptStep.h @@ -8,14 +8,17 @@ class IntersectOrExceptStep : public IQueryPlanStep { public: /// max_threads is used to limit the number of threads for result pipeline. - IntersectOrExceptStep(bool is_except_, DataStreams input_streams_, Block result_header, size_t max_threads_ = 0); + IntersectOrExceptStep(bool is_except_, DataStreams input_streams_, size_t max_threads_ = 0); String getName() const override { return is_except ? "Except" : "Intersect"; } QueryPipelinePtr updatePipeline(QueryPipelines pipelines, const BuildQueryPipelineSettings & settings) override; void describePipeline(FormatSettings & settings) const override; + private: + Block checkHeaders(const DataStreams & input_streams_) const; + bool is_except; Block header; size_t max_threads; @@ -23,4 +26,3 @@ private: }; } - diff --git a/src/Processors/Transforms/IntersectOrExceptTransform.cpp b/src/Processors/Transforms/IntersectOrExceptTransform.cpp index 199498bf762..e5e8ff705c8 100644 --- a/src/Processors/Transforms/IntersectOrExceptTransform.cpp +++ b/src/Processors/Transforms/IntersectOrExceptTransform.cpp @@ -1,9 +1,12 @@ #include + namespace DB { + IntersectOrExceptTransform::IntersectOrExceptTransform(bool is_except_, const Block & header_) - : IProcessor(InputPorts(2, header_), {header_}), is_except(is_except_), output(outputs.front()) + : IProcessor(InputPorts(2, header_), {header_}) + , is_except(is_except_) { const Names & columns = header_.getNames(); size_t num_columns = columns.empty() ? header_.columns() : columns.size(); @@ -11,18 +14,17 @@ IntersectOrExceptTransform::IntersectOrExceptTransform(bool is_except_, const Bl key_columns_pos.reserve(columns.size()); for (size_t i = 0; i < num_columns; ++i) { - auto pos = columns.empty() ? i : header_.getPositionByName(columns[i]); - - const auto & col = header_.getByPosition(pos).column; - - if (!(col && isColumnConst(*col))) - key_columns_pos.emplace_back(pos); + auto pos = columns.empty() ? i + : header_.getPositionByName(columns[i]); + key_columns_pos.emplace_back(pos); } } + IntersectOrExceptTransform::Status IntersectOrExceptTransform::prepare() { - /// Check can output. + auto & output = outputs.front(); + if (output.isFinished()) { for (auto & in : inputs) @@ -32,14 +34,8 @@ IntersectOrExceptTransform::Status IntersectOrExceptTransform::prepare() if (!output.canPush()) { - if (inputs.front().isFinished()) - { - inputs.back().setNotNeeded(); - } - else - { - inputs.front().setNotNeeded(); - } + for (auto & input : inputs) + input.setNotNeeded(); return Status::PortFull; } @@ -74,10 +70,9 @@ IntersectOrExceptTransform::Status IntersectOrExceptTransform::prepare() if (!has_input) { input.setNeeded(); + if (!input.hasData()) - { return Status::NeedData; - } current_input_chunk = input.pull(); has_input = true; @@ -86,6 +81,7 @@ IntersectOrExceptTransform::Status IntersectOrExceptTransform::prepare() return Status::Ready; } + void IntersectOrExceptTransform::work() { if (!finished_second_input) @@ -101,17 +97,17 @@ void IntersectOrExceptTransform::work() has_input = false; } + template void IntersectOrExceptTransform::addToSet(Method & method, const ColumnRawPtrs & columns, size_t rows, SetVariants & variants) const { typename Method::State state(columns, key_sizes, nullptr); for (size_t i = 0; i < rows; ++i) - { state.emplaceKey(method.data, i, variants.string_pool); - } } + template size_t IntersectOrExceptTransform::buildFilter( Method & method, const ColumnRawPtrs & columns, IColumn::Filter & filter, size_t rows, SetVariants & variants) const @@ -129,6 +125,7 @@ size_t IntersectOrExceptTransform::buildFilter( return new_rows_num; } + void IntersectOrExceptTransform::accumulate(Chunk chunk) { auto num_rows = chunk.getNumRows(); @@ -136,6 +133,7 @@ void IntersectOrExceptTransform::accumulate(Chunk chunk) ColumnRawPtrs column_ptrs; column_ptrs.reserve(key_columns_pos.size()); + for (auto pos : key_columns_pos) column_ptrs.emplace_back(columns[pos].get()); @@ -155,6 +153,7 @@ void IntersectOrExceptTransform::accumulate(Chunk chunk) } } + void IntersectOrExceptTransform::filter(Chunk & chunk) { auto num_rows = chunk.getNumRows(); @@ -162,6 +161,7 @@ void IntersectOrExceptTransform::filter(Chunk & chunk) ColumnRawPtrs column_ptrs; column_ptrs.reserve(key_columns_pos.size()); + for (auto pos : key_columns_pos) column_ptrs.emplace_back(columns[pos].get()); diff --git a/src/Processors/Transforms/IntersectOrExceptTransform.h b/src/Processors/Transforms/IntersectOrExceptTransform.h index ebe73fdeb26..3c2b9581d6d 100644 --- a/src/Processors/Transforms/IntersectOrExceptTransform.h +++ b/src/Processors/Transforms/IntersectOrExceptTransform.h @@ -12,17 +12,18 @@ class IntersectOrExceptTransform : public IProcessor public: IntersectOrExceptTransform(bool is_except_, const Block & header_); - Status prepare() override; - void work() override; - String getName() const override { return is_except ? "Except" : "Intersect"; } +protected: + Status prepare() override; + + void work() override; + private: + bool is_except; bool push_empty_chunk = false; Chunk empty_chunk; - - bool is_except; ColumnNumbers key_columns_pos; SetVariants data; Sizes key_sizes; @@ -30,24 +31,17 @@ private: Chunk current_output_chunk; bool finished_second_input = false; bool has_input = false; - OutputPort & output; void accumulate(Chunk chunk); + void filter(Chunk & chunk); - template - void addToSet( - Method & method, - const ColumnRawPtrs & key_columns, - size_t rows, - SetVariants & variants) const; template - size_t buildFilter( - Method & method, - const ColumnRawPtrs & columns, - IColumn::Filter & filter, - size_t rows, - SetVariants & variants) const; + void addToSet(Method & method, const ColumnRawPtrs & key_columns, size_t rows, SetVariants & variants) const; + + template + size_t buildFilter(Method & method, const ColumnRawPtrs & columns, + IColumn::Filter & filter, size_t rows, SetVariants & variants) const; }; } From 0f67acf6482ae1b5d736a01144255c8daf60e7eb Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 8 Aug 2021 20:16:22 +0300 Subject: [PATCH 038/161] Add test --- ...02004_intersect_except_operators.reference | 25 +++++++++++++++++++ .../02004_intersect_except_operators.sql | 8 ++++++ 2 files changed, 33 insertions(+) create mode 100644 tests/queries/0_stateless/02004_intersect_except_operators.reference create mode 100644 tests/queries/0_stateless/02004_intersect_except_operators.sql diff --git a/tests/queries/0_stateless/02004_intersect_except_operators.reference b/tests/queries/0_stateless/02004_intersect_except_operators.reference new file mode 100644 index 00000000000..763a5872cce --- /dev/null +++ b/tests/queries/0_stateless/02004_intersect_except_operators.reference @@ -0,0 +1,25 @@ +-- { echo } +select 1 intersect select 1; +1 +select 2 intersect select 1; +select 1 except select 1; +select 2 except select 1; +2 +select number from numbers(5, 5) intersect select number from numbers(20); +5 +6 +7 +8 +9 +select number from numbers(10) except select number from numbers(5); +5 +6 +7 +8 +9 +select number, number+10 from numbers(12) except select number+5, number+15 from numbers(10); +0 10 +1 11 +2 12 +3 13 +4 14 diff --git a/tests/queries/0_stateless/02004_intersect_except_operators.sql b/tests/queries/0_stateless/02004_intersect_except_operators.sql new file mode 100644 index 00000000000..d0416fd899e --- /dev/null +++ b/tests/queries/0_stateless/02004_intersect_except_operators.sql @@ -0,0 +1,8 @@ +-- { echo } +select 1 intersect select 1; +select 2 intersect select 1; +select 1 except select 1; +select 2 except select 1; +select number from numbers(5, 5) intersect select number from numbers(20); +select number from numbers(10) except select number from numbers(5); +select number, number+10 from numbers(12) except select number+5, number+15 from numbers(10); From 9ce7669e1fa3eef3e4a633e58012a2bf7c3de28a Mon Sep 17 00:00:00 2001 From: olgarev Date: Mon, 9 Aug 2021 01:25:22 +0000 Subject: [PATCH 039/161] Initial --- .../statements/select/distinct.md | 128 ++++++++++++------ .../sql-reference/statements/select/index.md | 6 +- .../statements/select/distinct.md | 61 +++++++-- .../sql-reference/statements/select/index.md | 6 +- 4 files changed, 149 insertions(+), 52 deletions(-) diff --git a/docs/en/sql-reference/statements/select/distinct.md b/docs/en/sql-reference/statements/select/distinct.md index 87154cba05a..1c739aab190 100644 --- a/docs/en/sql-reference/statements/select/distinct.md +++ b/docs/en/sql-reference/statements/select/distinct.md @@ -6,6 +6,96 @@ toc_title: DISTINCT If `SELECT DISTINCT` is specified, only unique rows will remain in a query result. Thus only a single row will remain out of all the sets of fully matching rows in the result. +You can narrow the list of columns which must have unique values: `SELECT DISTINCT ON (column1, column2,...)`. If the columns are not specified, all of them are taken into consideration. + +Consider the table: + +```text +┌─a─┬─b─┬─c─┐ +│ 1 │ 1 │ 1 │ +│ 1 │ 1 │ 1 │ +│ 2 │ 2 │ 2 │ +│ 2 │ 2 │ 2 │ +│ 1 │ 1 │ 2 │ +│ 1 │ 2 │ 2 │ +└───┴───┴───┘ +``` + +Using `DISTINCT` without specifying columns: + +```sql +SELECT DISTINCT * FROM t1; +``` + +```text +┌─a─┬─b─┬─c─┐ +│ 1 │ 1 │ 1 │ +│ 2 │ 2 │ 2 │ +│ 1 │ 1 │ 2 │ +│ 1 │ 2 │ 2 │ +└───┴───┴───┘ +``` + +Using `DISTINCT` with specified columns: + +```sql +SELECT DISTINCT ON (a,b) * FROM t1; +``` + +```text +┌─a─┬─b─┬─c─┐ +│ 1 │ 1 │ 1 │ +│ 2 │ 2 │ 2 │ +│ 1 │ 2 │ 2 │ +└───┴───┴───┘ +``` + +## DISTINCT and ORDER BY {#distinct-orderby} + +ClickHouse supports using the `DISTINCT` and `ORDER BY` clauses for different columns in one query. The `DISTINCT` clause is executed before the `ORDER BY` clause. + +Consider the table: + +``` text +┌─a─┬─b─┐ +│ 2 │ 1 │ +│ 1 │ 2 │ +│ 3 │ 3 │ +│ 2 │ 4 │ +└───┴───┘ +``` + +Selecting data: + +```sql +SELECT DISTINCT a FROM t1 ORDER BY b ASC; +``` + +``` text +┌─a─┐ +│ 2 │ +│ 1 │ +│ 3 │ +└───┘ +``` +Selecting data with the different sorting direction: + +```sql +SELECT DISTINCT a FROM t1 ORDER BY b DESC; +``` + +``` text +┌─a─┐ +│ 3 │ +│ 1 │ +│ 2 │ +└───┘ +``` + +Row `2, 4` was cut before sorting. + +Take this implementation specificity into account when programming queries. + ## Null Processing {#null-processing} `DISTINCT` works with [NULL](../../../sql-reference/syntax.md#null-literal) as if `NULL` were a specific value, and `NULL==NULL`. In other words, in the `DISTINCT` results, different combinations with `NULL` occur only once. It differs from `NULL` processing in most other contexts. @@ -18,41 +108,3 @@ It is possible to obtain the same result by applying [GROUP BY](../../../sql-ref - When [ORDER BY](../../../sql-reference/statements/select/order-by.md) is omitted and [LIMIT](../../../sql-reference/statements/select/limit.md) is defined, the query stops running immediately after the required number of different rows has been read. - Data blocks are output as they are processed, without waiting for the entire query to finish running. -## Examples {#examples} - -ClickHouse supports using the `DISTINCT` and `ORDER BY` clauses for different columns in one query. The `DISTINCT` clause is executed before the `ORDER BY` clause. - -Example table: - -``` text -┌─a─┬─b─┐ -│ 2 │ 1 │ -│ 1 │ 2 │ -│ 3 │ 3 │ -│ 2 │ 4 │ -└───┴───┘ -``` - -When selecting data with the `SELECT DISTINCT a FROM t1 ORDER BY b ASC` query, we get the following result: - -``` text -┌─a─┐ -│ 2 │ -│ 1 │ -│ 3 │ -└───┘ -``` - -If we change the sorting direction `SELECT DISTINCT a FROM t1 ORDER BY b DESC`, we get the following result: - -``` text -┌─a─┐ -│ 3 │ -│ 1 │ -│ 2 │ -└───┘ -``` - -Row `2, 4` was cut before sorting. - -Take this implementation specificity into account when programming queries. diff --git a/docs/en/sql-reference/statements/select/index.md b/docs/en/sql-reference/statements/select/index.md index 04273ca1d4d..4e96bae8493 100644 --- a/docs/en/sql-reference/statements/select/index.md +++ b/docs/en/sql-reference/statements/select/index.md @@ -13,7 +13,7 @@ toc_title: Overview ``` sql [WITH expr_list|(subquery)] -SELECT [DISTINCT] expr_list +SELECT [DISTINCT [ON (column1, column2, ...)]] expr_list [FROM [db.]table | (subquery) | table_function] [FINAL] [SAMPLE sample_coeff] [ARRAY JOIN ...] @@ -36,6 +36,8 @@ All clauses are optional, except for the required list of expressions immediatel Specifics of each optional clause are covered in separate sections, which are listed in the same order as they are executed: - [WITH clause](../../../sql-reference/statements/select/with.md) +- [SELECT clause](#select-clause) +- [DISTINCT clause](../../../sql-reference/statements/select/distinct.md) - [FROM clause](../../../sql-reference/statements/select/from.md) - [SAMPLE clause](../../../sql-reference/statements/select/sample.md) - [JOIN clause](../../../sql-reference/statements/select/join.md) @@ -44,8 +46,6 @@ Specifics of each optional clause are covered in separate sections, which are li - [GROUP BY clause](../../../sql-reference/statements/select/group-by.md) - [LIMIT BY clause](../../../sql-reference/statements/select/limit-by.md) - [HAVING clause](../../../sql-reference/statements/select/having.md) -- [SELECT clause](#select-clause) -- [DISTINCT clause](../../../sql-reference/statements/select/distinct.md) - [LIMIT clause](../../../sql-reference/statements/select/limit.md) - [OFFSET clause](../../../sql-reference/statements/select/offset.md) - [UNION clause](../../../sql-reference/statements/select/union.md) diff --git a/docs/ru/sql-reference/statements/select/distinct.md b/docs/ru/sql-reference/statements/select/distinct.md index f57c2a42593..42c1df64540 100644 --- a/docs/ru/sql-reference/statements/select/distinct.md +++ b/docs/ru/sql-reference/statements/select/distinct.md @@ -6,19 +6,51 @@ toc_title: DISTINCT Если указан `SELECT DISTINCT`, то в результате запроса останутся только уникальные строки. Таким образом, из всех наборов полностью совпадающих строк в результате останется только одна строка. -## Обработка NULL {#null-processing} +Вы можете указать столбцы, по которым хотите отбирать уникальные значения: `SELECT DISTINCT ON (column1, column2,...)`. Если столбцы не указаны, то отбираются строки, в которых значения уникальны во всех столбцах. -`DISTINCT` работает с [NULL](../../syntax.md#null-literal) как-будто `NULL` — обычное значение и `NULL==NULL`. Другими словами, в результате `DISTINCT`, различные комбинации с `NULL` встретятся только один раз. Это отличается от обработки `NULL` в большинстве других контекстов. +Рассмотрим таблицу: -## Альтернативы {#alternatives} +```text +┌─a─┬─b─┬─c─┐ +│ 1 │ 1 │ 1 │ +│ 1 │ 1 │ 1 │ +│ 2 │ 2 │ 2 │ +│ 2 │ 2 │ 2 │ +│ 1 │ 1 │ 2 │ +│ 1 │ 2 │ 2 │ +└───┴───┴───┘ +``` -Такой же результат можно получить, применив секцию [GROUP BY](group-by.md) для того же набора значений, которые указан в секции `SELECT`, без использования каких-либо агрегатных функций. Но есть от `GROUP BY` несколько отличий: +Использование `DISTINCT` без указания столбцов: -- `DISTINCT` может применяться вместе с `GROUP BY`. -- Когда секция [ORDER BY](order-by.md) опущена, а секция [LIMIT](limit.md) присутствует, запрос прекращает выполнение сразу после считывания необходимого количества различных строк. -- Блоки данных выводятся по мере их обработки, не дожидаясь завершения выполнения всего запроса. +```sql +SELECT DISTINCT * FROM t1; +``` -## Примеры {#examples} +```text +┌─a─┬─b─┬─c─┐ +│ 1 │ 1 │ 1 │ +│ 2 │ 2 │ 2 │ +│ 1 │ 1 │ 2 │ +│ 1 │ 2 │ 2 │ +└───┴───┴───┘ +``` + +Использование `DISTINCT` с указанием столбцов: + +```sql +SELECT DISTINCT ON (a,b) * FROM t1; +``` + +```text +┌─a─┬─b─┬─c─┐ +│ 1 │ 1 │ 1 │ +│ 2 │ 2 │ 2 │ +│ 1 │ 2 │ 2 │ +└───┴───┴───┘ +``` + +## DISTINCT и ORDER BY {#distinct-orderby} ClickHouse поддерживает использование секций `DISTINCT` и `ORDER BY` для разных столбцов в одном запросе. Секция `DISTINCT` выполняется до секции `ORDER BY`. @@ -56,3 +88,16 @@ ClickHouse поддерживает использование секций `DIS Ряд `2, 4` был разрезан перед сортировкой. Учитывайте эту специфику при разработке запросов. + +## Обработка NULL {#null-processing} + +`DISTINCT` работает с [NULL](../../syntax.md#null-literal) как-будто `NULL` — обычное значение и `NULL==NULL`. Другими словами, в результате `DISTINCT`, различные комбинации с `NULL` встретятся только один раз. Это отличается от обработки `NULL` в большинстве других контекстов. + +## Альтернативы {#alternatives} + +Можно получить такой же результат, применив [GROUP BY](group-by.md) для того же набора значений, которые указан в секции `SELECT`, без использования каких-либо агрегатных функций. Но есть несколько отличий от `GROUP BY`: + +- `DISTINCT` может применяться вместе с `GROUP BY`. +- Когда секция [ORDER BY](order-by.md) опущена, а секция [LIMIT](limit.md) присутствует, запрос прекращает выполнение сразу после считывания необходимого количества различных строк. +- Блоки данных выводятся по мере их обработки, не дожидаясь завершения выполнения всего запроса. + diff --git a/docs/ru/sql-reference/statements/select/index.md b/docs/ru/sql-reference/statements/select/index.md index a0a862cbf55..c2820bc7be4 100644 --- a/docs/ru/sql-reference/statements/select/index.md +++ b/docs/ru/sql-reference/statements/select/index.md @@ -11,7 +11,7 @@ toc_title: "Обзор" ``` sql [WITH expr_list|(subquery)] -SELECT [DISTINCT] expr_list +SELECT [DISTINCT [ON (column1, column2, ...)]] expr_list [FROM [db.]table | (subquery) | table_function] [FINAL] [SAMPLE sample_coeff] [ARRAY JOIN ...] @@ -34,6 +34,8 @@ SELECT [DISTINCT] expr_list Особенности каждой необязательной секции рассматриваются в отдельных разделах, которые перечислены в том же порядке, в каком они выполняются: - [Секция WITH](with.md) +- [Секция SELECT](#select-clause) +- [Секция DISTINCT](distinct.md) - [Секция FROM](from.md) - [Секция SAMPLE](sample.md) - [Секция JOIN](join.md) @@ -42,8 +44,6 @@ SELECT [DISTINCT] expr_list - [Секция GROUP BY](group-by.md) - [Секция LIMIT BY](limit-by.md) - [Секция HAVING](having.md) -- [Секция SELECT](#select-clause) -- [Секция DISTINCT](distinct.md) - [Секция LIMIT](limit.md) [Секция OFFSET](offset.md) - [Секция UNION ALL](union.md) From a5a60a045170e68d3374552e56fb7acc95449776 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 9 Aug 2021 05:20:41 +0000 Subject: [PATCH 040/161] Fix tests --- .../test_alter_update_cast_keep_nullable/__init__.py | 0 .../0_stateless/00597_push_down_predicate_long.reference | 2 +- 2 files changed, 1 insertion(+), 1 deletion(-) create mode 100644 tests/integration/test_alter_update_cast_keep_nullable/__init__.py diff --git a/tests/integration/test_alter_update_cast_keep_nullable/__init__.py b/tests/integration/test_alter_update_cast_keep_nullable/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/00597_push_down_predicate_long.reference b/tests/queries/0_stateless/00597_push_down_predicate_long.reference index 3eaa1139c5d..f6f1320c2f8 100644 --- a/tests/queries/0_stateless/00597_push_down_predicate_long.reference +++ b/tests/queries/0_stateless/00597_push_down_predicate_long.reference @@ -114,7 +114,7 @@ FROM ( SELECT 1 AS id, - identity(CAST(1, \'Nullable(UInt8)\')) AS subquery + identity(_CAST(1, \'Nullable(UInt8)\')) AS subquery WHERE subquery = 1 ) WHERE subquery = 1 From 484c3a5d2ee964a9728c53f064e040bb84f64a99 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 9 Aug 2021 11:55:17 +0000 Subject: [PATCH 041/161] Update kafka test --- tests/integration/test_storage_kafka/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 947b71b5f96..2ce1c1676ad 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -2753,7 +2753,7 @@ def test_kafka_formats_with_broken_message(kafka_cluster): # broken message "(0,'BAD','AM',0.5,1)", ], - 'expected':r'''{"raw_message":"(0,'BAD','AM',0.5,1)","error":"Cannot parse string 'BAD' as UInt16: syntax error at begin of string. Note: there are toUInt16OrZero and toUInt16OrNull functions, which returns zero\/NULL instead of throwing exception.: while executing 'FUNCTION CAST(assumeNotNull(_dummy_0) :: 2, 'UInt16' :: 1) -> CAST(assumeNotNull(_dummy_0), 'UInt16') UInt16 : 4'"}''', + 'expected':r'''{"raw_message":"(0,'BAD','AM',0.5,1)","error":"Cannot parse string 'BAD' as UInt16: syntax error at begin of string. Note: there are toUInt16OrZero and toUInt16OrNull functions, which returns zero\/NULL instead of throwing exception.: while executing 'FUNCTION _CAST(assumeNotNull(_dummy_0) :: 2, 'UInt16' :: 1) -> _CAST(assumeNotNull(_dummy_0), 'UInt16') UInt16 : 4'"}''', 'supports_empty_value': True, 'printable':True, }, From b90dc1017bfdc4e826cd3181209415bbb0b7e754 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 9 Aug 2021 16:43:10 +0300 Subject: [PATCH 042/161] fix tests --- ...map_add_map_subtract_on_map_type.reference | 54 +++++++++---------- .../01550_type_map_formats.reference | 6 +-- 2 files changed, 30 insertions(+), 30 deletions(-) diff --git a/tests/queries/0_stateless/01318_map_add_map_subtract_on_map_type.reference b/tests/queries/0_stateless/01318_map_add_map_subtract_on_map_type.reference index 96bafc2c79c..304f7407cf5 100644 --- a/tests/queries/0_stateless/01318_map_add_map_subtract_on_map_type.reference +++ b/tests/queries/0_stateless/01318_map_add_map_subtract_on_map_type.reference @@ -18,38 +18,38 @@ {1:3,2:2,8:2} {1:3,2:2,9:2} {1:3,2:2,10:2} -{1:2,2:2} Map(UInt8,UInt64) -{1:2,2:2} Map(UInt16,UInt64) -{1:2,2:2} Map(UInt32,UInt64) -{1:2,2:2} Map(UInt64,UInt64) -{1:2,2:2} Map(UInt128,UInt128) -{1:2,2:2} Map(UInt256,UInt256) -{1:2,2:2} Map(Int16,UInt64) -{1:2,2:2} Map(Int16,Int64) -{1:2,2:2} Map(Int32,Int64) -{1:2,2:2} Map(Int64,Int64) -{1:2,2:2} Map(Int128,Int128) -{1:2,2:2} Map(Int256,Int256) -{1:3.300000023841858,2:2} Map(UInt8,Float64) -{1:3.3000000000000003,2:2} Map(UInt8,Float64) +{1:2,2:2} Map(UInt8, UInt64) +{1:2,2:2} Map(UInt16, UInt64) +{1:2,2:2} Map(UInt32, UInt64) +{1:2,2:2} Map(UInt64, UInt64) +{1:2,2:2} Map(UInt128, UInt128) +{1:2,2:2} Map(UInt256, UInt256) +{1:2,2:2} Map(Int16, UInt64) +{1:2,2:2} Map(Int16, Int64) +{1:2,2:2} Map(Int32, Int64) +{1:2,2:2} Map(Int64, Int64) +{1:2,2:2} Map(Int128, Int128) +{1:2,2:2} Map(Int256, Int256) +{1:3.300000023841858,2:2} Map(UInt8, Float64) +{1:3.3000000000000003,2:2} Map(UInt8, Float64) {'a':1,'b':2} {'a':1,'b':1,'c':1} {'a':1,'b':1,'d':1} -{'a':1,'b':2} Map(String,UInt64) -{'a':1,'b':1,'c':1} Map(String,UInt64) -{'a':1,'b':1,'d':1} Map(String,UInt64) +{'a':1,'b':2} Map(String, UInt64) +{'a':1,'b':1,'c':1} Map(String, UInt64) +{'a':1,'b':1,'d':1} Map(String, UInt64) {'a':1,'b':2} {'a':1,'b':1,'c':1} {'a':1,'b':1,'d':1} -{'a':2} Map(Enum16(\'a\' = 1, \'b\' = 2),Int64) -{'b':2} Map(Enum16(\'a\' = 1, \'b\' = 2),Int64) -{'a':2} Map(Enum8(\'a\' = 1, \'b\' = 2),Int64) -{'b':2} Map(Enum8(\'a\' = 1, \'b\' = 2),Int64) -{'00000000-89ab-cdef-0123-456789abcdef':2} Map(UUID,Int64) -{'11111111-89ab-cdef-0123-456789abcdef':4} Map(UUID,Int64) +{'a':2} Map(Enum16(\'a\' = 1, \'b\' = 2), Int64) +{'b':2} Map(Enum16(\'a\' = 1, \'b\' = 2), Int64) +{'a':2} Map(Enum8(\'a\' = 1, \'b\' = 2), Int64) +{'b':2} Map(Enum8(\'a\' = 1, \'b\' = 2), Int64) +{'00000000-89ab-cdef-0123-456789abcdef':2} Map(UUID, Int64) +{'11111111-89ab-cdef-0123-456789abcdef':4} Map(UUID, Int64) {1:0,2:0} Map(UInt8,UInt64) -{1:18446744073709551615,2:18446744073709551615} Map(UInt8,UInt64) +{1:18446744073709551615,2:18446744073709551615} Map(UInt8, UInt64) {1:-1,2:-1} Map(UInt8,Int64) -{1:-1.0999999761581423,2:0} Map(UInt8,Float64) -{1:-1,2:-1} Map(UInt8,Int64) -{1:-2,2:-2,3:1} Map(UInt8,Int64) +{1:-1.0999999761581423,2:0} Map(UInt8, Float64) +{1:-1,2:-1} Map(UInt8, Int64) +{1:-2,2:-2,3:1} Map(UInt8, Int64) diff --git a/tests/queries/0_stateless/01550_type_map_formats.reference b/tests/queries/0_stateless/01550_type_map_formats.reference index ca081db75a2..998473ef63a 100644 --- a/tests/queries/0_stateless/01550_type_map_formats.reference +++ b/tests/queries/0_stateless/01550_type_map_formats.reference @@ -4,15 +4,15 @@ JSON [ { "name": "m", - "type": "Map(String,UInt32)" + "type": "Map(String, UInt32)" }, { "name": "m1", - "type": "Map(String,Date)" + "type": "Map(String, Date)" }, { "name": "m2", - "type": "Map(String,Array(UInt32))" + "type": "Map(String, Array(UInt32))" } ], From 8cc493a3cdd4cd621a7d6b585c542b6f16d88e5d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 9 Aug 2021 18:09:29 +0300 Subject: [PATCH 043/161] Try fix build. --- src/Common/DenseHashMap.h | 19 +++++++++++++++++++ src/Common/DenseHashSet.h | 19 +++++++++++++++++++ src/Common/SparseHashMap.h | 19 +++++++++++++++++++ src/Core/NamesAndTypes.cpp | 8 ++------ src/Dictionaries/HashedDictionary.h | 10 +--------- src/Storages/MergeTree/IMergeTreeReader.h | 8 ++------ src/Storages/StorageInMemoryMetadata.cpp | 20 +++++--------------- 7 files changed, 67 insertions(+), 36 deletions(-) create mode 100644 src/Common/DenseHashMap.h create mode 100644 src/Common/DenseHashSet.h create mode 100644 src/Common/SparseHashMap.h diff --git a/src/Common/DenseHashMap.h b/src/Common/DenseHashMap.h new file mode 100644 index 00000000000..0097faa5259 --- /dev/null +++ b/src/Common/DenseHashMap.h @@ -0,0 +1,19 @@ +#pragma once + +#if defined(ARCADIA_BUILD) +#define HASH_FUN_H +#endif + +#include + +#if !defined(ARCADIA_BUILD) + template , + class EqualKey = std::equal_to, + class Alloc = google::libc_allocator_with_realloc>> + using DenseHashMap = google::dense_hash_map; +#else + template , + class EqualKey = std::equal_to, + class Alloc = google::sparsehash::libc_allocator_with_realloc>> + using DenseHashMap = google::sparsehash::dense_hash_map; +#endif diff --git a/src/Common/DenseHashSet.h b/src/Common/DenseHashSet.h new file mode 100644 index 00000000000..38d281e1966 --- /dev/null +++ b/src/Common/DenseHashSet.h @@ -0,0 +1,19 @@ +#pragma once + +#if defined(ARCADIA_BUILD) +#define HASH_FUN_H +#endif + +#include + +#if !defined(ARCADIA_BUILD) + template , + class EqualKey = std::equal_to, + class Alloc = google::libc_allocator_with_realloc>> + using DenseHashSet = google::dense_hash_set; +#else +template , + class EqualKey = std::equal_to, + class Alloc = google::sparsehash::libc_allocator_with_realloc>> + using DenseHashSet = google::sparsehash::dense_hash_set; +#endif diff --git a/src/Common/SparseHashMap.h b/src/Common/SparseHashMap.h new file mode 100644 index 00000000000..c5fa07410af --- /dev/null +++ b/src/Common/SparseHashMap.h @@ -0,0 +1,19 @@ +#pragma once + +#if defined(ARCADIA_BUILD) +#define HASH_FUN_H +#endif + +#include + +#if !defined(ARCADIA_BUILD) + template , + class EqualKey = std::equal_to, + class Alloc = google::libc_allocator_with_realloc>> + using SparseHashMap = google::sparse_hash_map; +#else + template , + class EqualKey = std::equal_to, + class Alloc = google::sparsehash::libc_allocator_with_realloc>> + using SparseHashMap = google::sparsehash::sparse_hash_map; +#endif diff --git a/src/Core/NamesAndTypes.cpp b/src/Core/NamesAndTypes.cpp index 91191c73fd0..54f83fc13fc 100644 --- a/src/Core/NamesAndTypes.cpp +++ b/src/Core/NamesAndTypes.cpp @@ -6,7 +6,7 @@ #include #include #include -#include +#include namespace DB @@ -163,11 +163,7 @@ NamesAndTypesList NamesAndTypesList::filter(const Names & names) const NamesAndTypesList NamesAndTypesList::addTypes(const Names & names) const { /// NOTE: It's better to make a map in `IStorage` than to create it here every time again. -#if !defined(ARCADIA_BUILD) - google::dense_hash_map types; -#else - google::sparsehash::dense_hash_map types; -#endif + DenseHashMap types; types.set_empty_key(StringRef()); for (const auto & column : *this) diff --git a/src/Dictionaries/HashedDictionary.h b/src/Dictionaries/HashedDictionary.h index 842e49aa8f0..168f4e3bafa 100644 --- a/src/Dictionaries/HashedDictionary.h +++ b/src/Dictionaries/HashedDictionary.h @@ -5,7 +5,7 @@ #include #include -#include +#include #include #include @@ -125,14 +125,6 @@ private: HashMap, HashMapWithSavedHash>>; -#if !defined(ARCADIA_BUILD) - template - using SparseHashMap = google::sparse_hash_map>; -#else - template - using SparseHashMap = google::sparsehash::sparse_hash_map>; -#endif - template using CollectionTypeSparse = std::conditional_t< dictionary_key_type == DictionaryKeyType::simple, diff --git a/src/Storages/MergeTree/IMergeTreeReader.h b/src/Storages/MergeTree/IMergeTreeReader.h index ab412e48822..8d80719efaf 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.h +++ b/src/Storages/MergeTree/IMergeTreeReader.h @@ -1,9 +1,9 @@ #pragma once #include +#include #include #include -#include namespace DB { @@ -95,11 +95,7 @@ private: /// Actual data type of columns in part -#if !defined(ARCADIA_BUILD) - google::dense_hash_map columns_from_part; -#else - google::sparsehash::dense_hash_map columns_from_part; -#endif + DenseHashMap columns_from_part; }; } diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index 7b1a126b1ab..91f69cdac7d 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -1,7 +1,7 @@ #include -#include -#include +#include +#include #include #include #include @@ -320,12 +320,7 @@ Block StorageInMemoryMetadata::getSampleBlockForColumns( { Block res; -#if !defined(ARCADIA_BUILD) - google::dense_hash_map virtuals_map; -#else - google::sparsehash::dense_hash_map virtuals_map; -#endif - + DenseHashMap virtuals_map; virtuals_map.set_empty_key(StringRef()); /// Virtual columns must be appended after ordinary, because user can @@ -475,13 +470,8 @@ bool StorageInMemoryMetadata::hasSelectQuery() const namespace { -#if !defined(ARCADIA_BUILD) - using NamesAndTypesMap = google::dense_hash_map; - using UniqueStrings = google::dense_hash_set; -#else - using NamesAndTypesMap = google::sparsehash::dense_hash_map; - using UniqueStrings = google::sparsehash::dense_hash_set; -#endif + using NamesAndTypesMap = DenseHashMap; + using UniqueStrings = DenseHashSet; String listOfColumns(const NamesAndTypesList & available_columns) { From c4cf1eae0da5e6b462226c60d19e74e137ae38f5 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 9 Aug 2021 18:46:39 +0300 Subject: [PATCH 044/161] Try fix build. --- src/Common/DenseHashSet.h | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/src/Common/DenseHashSet.h b/src/Common/DenseHashSet.h index 38d281e1966..9741bf5ec5c 100644 --- a/src/Common/DenseHashSet.h +++ b/src/Common/DenseHashSet.h @@ -4,16 +4,16 @@ #define HASH_FUN_H #endif -#include +#include #if !defined(ARCADIA_BUILD) - template , - class EqualKey = std::equal_to, - class Alloc = google::libc_allocator_with_realloc>> - using DenseHashSet = google::dense_hash_set; + template , + class EqualKey = std::equal_to, + class Alloc = google::libc_allocator_with_realloc> + using DenseHashSet = google::dense_hash_set; #else -template , - class EqualKey = std::equal_to, - class Alloc = google::sparsehash::libc_allocator_with_realloc>> - using DenseHashSet = google::sparsehash::dense_hash_set; + template , + class EqualKey = std::equal_to, + class Alloc = google::sparsehash::libc_allocator_with_realloc> + using DenseHashSet = google::sparsehash::dense_hash_set; #endif From 3794e9e980c282af5f25fdce7813da79a5464c74 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 9 Aug 2021 23:24:33 +0300 Subject: [PATCH 045/161] Try fix build. --- src/Common/DenseHashMap.h | 1 + src/Common/DenseHashSet.h | 1 + src/Common/SparseHashMap.h | 1 + 3 files changed, 3 insertions(+) diff --git a/src/Common/DenseHashMap.h b/src/Common/DenseHashMap.h index 0097faa5259..d27b8d93a27 100644 --- a/src/Common/DenseHashMap.h +++ b/src/Common/DenseHashMap.h @@ -1,6 +1,7 @@ #pragma once #if defined(ARCADIA_BUILD) +#undef HASH_FUN_H #define HASH_FUN_H #endif diff --git a/src/Common/DenseHashSet.h b/src/Common/DenseHashSet.h index 9741bf5ec5c..750dc829817 100644 --- a/src/Common/DenseHashSet.h +++ b/src/Common/DenseHashSet.h @@ -1,6 +1,7 @@ #pragma once #if defined(ARCADIA_BUILD) +#undef HASH_FUN_H #define HASH_FUN_H #endif diff --git a/src/Common/SparseHashMap.h b/src/Common/SparseHashMap.h index c5fa07410af..22847499bce 100644 --- a/src/Common/SparseHashMap.h +++ b/src/Common/SparseHashMap.h @@ -1,6 +1,7 @@ #pragma once #if defined(ARCADIA_BUILD) +#undef HASH_FUN_H #define HASH_FUN_H #endif From 9763dd18b1838b0e7edde89242e10c168c86ba60 Mon Sep 17 00:00:00 2001 From: olgarev <56617294+olgarev@users.noreply.github.com> Date: Tue, 10 Aug 2021 00:46:25 +0300 Subject: [PATCH 046/161] Update docs/en/sql-reference/statements/select/distinct.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- docs/en/sql-reference/statements/select/distinct.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/select/distinct.md b/docs/en/sql-reference/statements/select/distinct.md index 1c739aab190..390afa46248 100644 --- a/docs/en/sql-reference/statements/select/distinct.md +++ b/docs/en/sql-reference/statements/select/distinct.md @@ -6,7 +6,7 @@ toc_title: DISTINCT If `SELECT DISTINCT` is specified, only unique rows will remain in a query result. Thus only a single row will remain out of all the sets of fully matching rows in the result. -You can narrow the list of columns which must have unique values: `SELECT DISTINCT ON (column1, column2,...)`. If the columns are not specified, all of them are taken into consideration. +You can specify the list of columns that must have unique values: `SELECT DISTINCT ON (column1, column2,...)`. If the columns are not specified, all of them are taken into consideration. Consider the table: @@ -107,4 +107,3 @@ It is possible to obtain the same result by applying [GROUP BY](../../../sql-ref - `DISTINCT` can be applied together with `GROUP BY`. - When [ORDER BY](../../../sql-reference/statements/select/order-by.md) is omitted and [LIMIT](../../../sql-reference/statements/select/limit.md) is defined, the query stops running immediately after the required number of different rows has been read. - Data blocks are output as they are processed, without waiting for the entire query to finish running. - From 4eb4dd97d4d1869d713c901b053fcb18bdb0ae23 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 10 Aug 2021 09:24:12 +0300 Subject: [PATCH 047/161] Support not only 2 inputs, support any size sequence of intersect / except together --- .../InterpreterIntersectOrExcept.cpp | 13 ++-- .../InterpreterIntersectOrExcept.h | 6 +- src/Parsers/ASTIntersectOrExcept.cpp | 55 +++++++++++-- src/Parsers/ASTIntersectOrExcept.h | 16 +++- src/Parsers/ParserIntersectOrExcept.cpp | 61 ++++++++++----- .../QueryPlan/IntersectOrExceptStep.cpp | 7 +- .../QueryPlan/IntersectOrExceptStep.h | 10 ++- .../Transforms/IntersectOrExceptTransform.cpp | 78 +++++++++++++------ .../Transforms/IntersectOrExceptTransform.h | 21 +++-- ...02004_intersect_except_operators.reference | 30 +++++++ .../02004_intersect_except_operators.sql | 10 +++ 11 files changed, 234 insertions(+), 73 deletions(-) diff --git a/src/Interpreters/InterpreterIntersectOrExcept.cpp b/src/Interpreters/InterpreterIntersectOrExcept.cpp index c85c39824d8..d706ab02bfd 100644 --- a/src/Interpreters/InterpreterIntersectOrExcept.cpp +++ b/src/Interpreters/InterpreterIntersectOrExcept.cpp @@ -21,18 +21,21 @@ namespace ErrorCodes InterpreterIntersectOrExcept::InterpreterIntersectOrExcept(const ASTPtr & query_ptr, ContextPtr context_) : context(Context::createCopy(context_)) - , is_except(query_ptr->as()->is_except) { ASTIntersectOrExcept * ast = query_ptr->as(); + auto children = ast->list_of_selects->children; + modes = ast->list_of_modes; + if (modes.size() + 1 != children.size()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Number of modes and number of children are not consistent"); - size_t num_children = ast->children.size(); + size_t num_children = children.size(); if (!num_children) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: no children in ASTIntersectOrExceptQuery"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "No children in ASTIntersectOrExceptQuery"); nested_interpreters.resize(num_children); for (size_t i = 0; i < num_children; ++i) - nested_interpreters[i] = buildCurrentChildInterpreter(ast->children[i]); + nested_interpreters[i] = buildCurrentChildInterpreter(children.at(i)); Blocks headers(num_children); for (size_t query_num = 0; query_num < num_children; ++query_num) @@ -103,7 +106,7 @@ void InterpreterIntersectOrExcept::buildQueryPlan(QueryPlan & query_plan) } auto max_threads = context->getSettingsRef().max_threads; - auto step = std::make_unique(is_except, std::move(data_streams), max_threads); + auto step = std::make_unique(std::move(data_streams), modes, max_threads); query_plan.unitePlans(std::move(step), std::move(plans)); } diff --git a/src/Interpreters/InterpreterIntersectOrExcept.h b/src/Interpreters/InterpreterIntersectOrExcept.h index 34a58c0c05a..8021a25df06 100644 --- a/src/Interpreters/InterpreterIntersectOrExcept.h +++ b/src/Interpreters/InterpreterIntersectOrExcept.h @@ -3,6 +3,8 @@ #include #include #include +#include + namespace DB { @@ -19,7 +21,7 @@ public: BlockIO execute() override; private: - String getName() const { return is_except ? "EXCEPT" : "INTERSECT"; } + String getName() const { return "IntersectExcept"; } Block getCommonHeader(const Blocks & headers) const; @@ -29,9 +31,9 @@ private: void buildQueryPlan(QueryPlan & query_plan); ContextPtr context; - bool is_except; Block result_header; std::vector> nested_interpreters; + ASTIntersectOrExcept::Modes modes; }; } diff --git a/src/Parsers/ASTIntersectOrExcept.cpp b/src/Parsers/ASTIntersectOrExcept.cpp index a05d7ee86c9..7d92055646c 100644 --- a/src/Parsers/ASTIntersectOrExcept.cpp +++ b/src/Parsers/ASTIntersectOrExcept.cpp @@ -1,5 +1,7 @@ #include #include +#include + namespace DB { @@ -8,21 +10,58 @@ ASTPtr ASTIntersectOrExcept::clone() const { auto res = std::make_shared(*this); res->children.clear(); - res->children.push_back(children[0]->clone()); - res->children.push_back(children[1]->clone()); - res->is_except = is_except; + + res->list_of_selects = list_of_selects->clone(); + res->children.push_back(res->list_of_selects); + res->list_of_modes = list_of_modes; + cloneOutputOptions(*res); return res; } void ASTIntersectOrExcept::formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { - children[0]->formatImpl(settings, state, frame); std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' '); - settings.ostr << settings.nl_or_ws << indent_str << (settings.hilite ? hilite_keyword : "") - << (is_except ? "EXCEPT" : "INTERSECT") - << (settings.hilite ? hilite_none : "") << settings.nl_or_ws; - children[1]->formatImpl(settings, state, frame); + + auto mode_to_str = [&](auto mode) + { + if (mode == Mode::INTERSECT) + return "INTERSECT"; + else + return "EXCEPT"; + }; + + for (ASTs::const_iterator it = list_of_selects->children.begin(); it != list_of_selects->children.end(); ++it) + { + if (it != list_of_selects->children.begin()) + { + settings.ostr << settings.nl_or_ws << indent_str << (settings.hilite ? hilite_keyword : "") + << mode_to_str(list_of_modes[it - list_of_selects->children.begin() - 1]) + << (settings.hilite ? hilite_none : ""); + } + + if (auto * node = (*it)->as()) + { + settings.ostr << settings.nl_or_ws << indent_str; + + if (node->list_of_selects->children.size() == 1) + { + (node->list_of_selects->children.at(0))->formatImpl(settings, state, frame); + } + else + { + auto sub_query = std::make_shared(); + sub_query->children.push_back(*it); + sub_query->formatImpl(settings, state, frame); + } + } + else + { + if (it != list_of_selects->children.begin()) + settings.ostr << settings.nl_or_ws; + (*it)->formatImpl(settings, state, frame); + } + } } } diff --git a/src/Parsers/ASTIntersectOrExcept.h b/src/Parsers/ASTIntersectOrExcept.h index a02cb9f7d77..0eb8ab3b113 100644 --- a/src/Parsers/ASTIntersectOrExcept.h +++ b/src/Parsers/ASTIntersectOrExcept.h @@ -9,10 +9,22 @@ namespace DB class ASTIntersectOrExcept : public ASTQueryWithOutput { public: - String getID(char) const override { return is_except ? "Except" : "Intersect"; } + String getID(char) const override { return "IntersectExceptQuery"; } + ASTPtr clone() const override; + void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; - bool is_except; + + enum class Mode + { + INTERSECT, + EXCEPT + }; + + using Modes = std::vector; + + ASTPtr list_of_selects; + Modes list_of_modes; }; } diff --git a/src/Parsers/ParserIntersectOrExcept.cpp b/src/Parsers/ParserIntersectOrExcept.cpp index 6d5da54fa38..fd1511b8af6 100644 --- a/src/Parsers/ParserIntersectOrExcept.cpp +++ b/src/Parsers/ParserIntersectOrExcept.cpp @@ -4,41 +4,62 @@ #include #include #include +#include +#include + namespace DB { + bool ParserIntersectOrExcept::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ParserKeyword intersect_keyword("INTERSECT"); ParserKeyword except_keyword("EXCEPT"); - ASTPtr left_node; - ASTPtr right_node; - auto ast = std::make_shared(); - ast->is_except = false; + ASTs elements; + ASTIntersectOrExcept::Modes modes; - if (!ParserSelectWithUnionQuery().parse(pos, left_node, expected) && !ParserSubquery().parse(pos, left_node, expected)) - return false; - - if (!intersect_keyword.ignore(pos)) + auto parse_element = [&]() -> bool { - if (!except_keyword.ignore(pos)) - { + ASTPtr element; + if (!ParserSelectWithUnionQuery().parse(pos, element, expected) && !ParserSubquery().parse(pos, element, expected)) return false; - } - else - { - ast->is_except = true; - } - } - if (!ParserSelectWithUnionQuery().parse(pos, right_node, expected) && !ParserSubquery().parse(pos, right_node, expected)) + elements.push_back(element); + return true; + }; + + auto parse_separator = [&]() -> bool + { + if (!intersect_keyword.ignore(pos)) + { + if (!except_keyword.ignore(pos)) + return false; + + modes.emplace_back(ASTIntersectOrExcept::Mode::EXCEPT); + return true; + } + + modes.emplace_back(ASTIntersectOrExcept::Mode::INTERSECT); + return true; + }; + + if (!ParserUnionList::parseUtil(pos, parse_element, parse_separator)) return false; - ast->children.push_back(left_node); - ast->children.push_back(right_node); + if (modes.empty()) + return false; + + auto list_node = std::make_shared(); + list_node->children = std::move(elements); + + auto intersect_or_except_ast = std::make_shared(); + + node = intersect_or_except_ast; + intersect_or_except_ast->list_of_selects = list_node; + intersect_or_except_ast->children.push_back(intersect_or_except_ast->list_of_selects); + intersect_or_except_ast->list_of_modes = modes; - node = ast; return true; } diff --git a/src/Processors/QueryPlan/IntersectOrExceptStep.cpp b/src/Processors/QueryPlan/IntersectOrExceptStep.cpp index f04885f4640..b1b5c1b8813 100644 --- a/src/Processors/QueryPlan/IntersectOrExceptStep.cpp +++ b/src/Processors/QueryPlan/IntersectOrExceptStep.cpp @@ -23,8 +23,8 @@ Block IntersectOrExceptStep::checkHeaders(const DataStreams & input_streams_) co return res; } -IntersectOrExceptStep::IntersectOrExceptStep(bool is_except_, DataStreams input_streams_, size_t max_threads_) - : is_except(is_except_), header(checkHeaders(input_streams_)), max_threads(max_threads_) +IntersectOrExceptStep::IntersectOrExceptStep(DataStreams input_streams_, const Modes & modes_, size_t max_threads_) + : header(checkHeaders(input_streams_)), modes(modes_), max_threads(max_threads_) { input_streams = std::move(input_streams_); if (input_streams.size() == 1) @@ -63,8 +63,9 @@ QueryPipelinePtr IntersectOrExceptStep::updatePipeline(QueryPipelines pipelines, } } + std::cerr << "size: " << input_streams.size() << std::endl; *pipeline = QueryPipeline::unitePipelines(std::move(pipelines), max_threads); - pipeline->addTransform(std::make_shared(is_except, header)); + pipeline->addTransform(std::make_shared(header, modes)); processors = collector.detachProcessors(); return pipeline; diff --git a/src/Processors/QueryPlan/IntersectOrExceptStep.h b/src/Processors/QueryPlan/IntersectOrExceptStep.h index 7938a9adad5..4eceb820153 100644 --- a/src/Processors/QueryPlan/IntersectOrExceptStep.h +++ b/src/Processors/QueryPlan/IntersectOrExceptStep.h @@ -1,16 +1,20 @@ #pragma once #include +#include + namespace DB { class IntersectOrExceptStep : public IQueryPlanStep { +using Modes = ASTIntersectOrExcept::Modes; + public: /// max_threads is used to limit the number of threads for result pipeline. - IntersectOrExceptStep(bool is_except_, DataStreams input_streams_, size_t max_threads_ = 0); + IntersectOrExceptStep(DataStreams input_streams_, const Modes & modes_, size_t max_threads_ = 0); - String getName() const override { return is_except ? "Except" : "Intersect"; } + String getName() const override { return "IntersectExcept"; } QueryPipelinePtr updatePipeline(QueryPipelines pipelines, const BuildQueryPipelineSettings & settings) override; @@ -19,8 +23,8 @@ public: private: Block checkHeaders(const DataStreams & input_streams_) const; - bool is_except; Block header; + Modes modes; size_t max_threads; Processors processors; }; diff --git a/src/Processors/Transforms/IntersectOrExceptTransform.cpp b/src/Processors/Transforms/IntersectOrExceptTransform.cpp index e5e8ff705c8..70a86855992 100644 --- a/src/Processors/Transforms/IntersectOrExceptTransform.cpp +++ b/src/Processors/Transforms/IntersectOrExceptTransform.cpp @@ -4,9 +4,11 @@ namespace DB { -IntersectOrExceptTransform::IntersectOrExceptTransform(bool is_except_, const Block & header_) - : IProcessor(InputPorts(2, header_), {header_}) - , is_except(is_except_) +IntersectOrExceptTransform::IntersectOrExceptTransform(const Block & header_, const Modes & modes_) + : IProcessor(InputPorts(modes_.size() + 1, header_), {header_}) + , modes(modes_) + , first_input(inputs.begin()) + , second_input(std::next(inputs.begin())) { const Names & columns = header_.getNames(); size_t num_columns = columns.empty() ? header_.columns() : columns.size(); @@ -14,8 +16,7 @@ IntersectOrExceptTransform::IntersectOrExceptTransform(bool is_except_, const Bl key_columns_pos.reserve(columns.size()); for (size_t i = 0; i < num_columns; ++i) { - auto pos = columns.empty() ? i - : header_.getPositionByName(columns[i]); + auto pos = columns.empty() ? i : header_.getPositionByName(columns[i]); key_columns_pos.emplace_back(pos); } } @@ -40,7 +41,7 @@ IntersectOrExceptTransform::Status IntersectOrExceptTransform::prepare() } /// Output if has data. - if (current_output_chunk) + if (current_output_chunk && second_input == inputs.end()) { output.push(std::move(current_output_chunk)); } @@ -53,28 +54,50 @@ IntersectOrExceptTransform::Status IntersectOrExceptTransform::prepare() if (finished_second_input) { - if (inputs.front().isFinished()) + if (first_input->isFinished() || (more && !current_input_chunk)) { - output.finish(); - return Status::Finished; + std::advance(second_input, 1); + + if (second_input == inputs.end()) + { + if (current_output_chunk) + { + output.push(std::move(current_output_chunk)); + } + output.finish(); + return Status::Finished; + } + else + { + more = true; + data.reset(); + finished_second_input = false; + ++current_operator_pos; + } } } - else if (inputs.back().isFinished()) + else if (second_input->isFinished()) { finished_second_input = true; } - InputPort & input = finished_second_input ? inputs.front() : inputs.back(); + InputPort & input = finished_second_input ? *first_input : *second_input; /// Check can input. if (!has_input) { - input.setNeeded(); + if (finished_second_input && more) + { + current_input_chunk = std::move(current_output_chunk); + } + else + { + input.setNeeded(); + if (!input.hasData()) + return Status::NeedData; + current_input_chunk = input.pull(); + } - if (!input.hasData()) - return Status::NeedData; - - current_input_chunk = input.pull(); has_input = true; } @@ -84,6 +107,9 @@ IntersectOrExceptTransform::Status IntersectOrExceptTransform::prepare() void IntersectOrExceptTransform::work() { + if (!data) + data.emplace(); + if (!finished_second_input) { accumulate(std::move(current_input_chunk)); @@ -118,7 +144,7 @@ size_t IntersectOrExceptTransform::buildFilter( for (size_t i = 0; i < rows; ++i) { auto find_result = state.findKey(method.data, i, variants.string_pool); - filter[i] = is_except ? !find_result.isFound() : find_result.isFound(); + filter[i] = modes[current_operator_pos] == ASTIntersectOrExcept::Mode::EXCEPT ? !find_result.isFound() : find_result.isFound(); if (filter[i]) ++new_rows_num; } @@ -137,16 +163,17 @@ void IntersectOrExceptTransform::accumulate(Chunk chunk) for (auto pos : key_columns_pos) column_ptrs.emplace_back(columns[pos].get()); - if (data.empty()) - data.init(SetVariants::chooseMethod(column_ptrs, key_sizes)); + if (data->empty()) + data->init(SetVariants::chooseMethod(column_ptrs, key_sizes)); - switch (data.type) + auto & data_set = *data; + switch (data->type) { case SetVariants::Type::EMPTY: break; #define M(NAME) \ case SetVariants::Type::NAME: \ - addToSet(*data.NAME, column_ptrs, num_rows, data); \ + addToSet(*data_set.NAME, column_ptrs, num_rows, data_set); \ break; APPLY_FOR_SET_VARIANTS(M) #undef M @@ -165,19 +192,20 @@ void IntersectOrExceptTransform::filter(Chunk & chunk) for (auto pos : key_columns_pos) column_ptrs.emplace_back(columns[pos].get()); - if (data.empty()) - data.init(SetVariants::chooseMethod(column_ptrs, key_sizes)); + if (data->empty()) + data->init(SetVariants::chooseMethod(column_ptrs, key_sizes)); IColumn::Filter filter(num_rows); size_t new_rows_num = 0; - switch (data.type) + auto & data_set = *data; + switch (data->type) { case SetVariants::Type::EMPTY: break; #define M(NAME) \ case SetVariants::Type::NAME: \ - new_rows_num = buildFilter(*data.NAME, column_ptrs, filter, num_rows, data); \ + new_rows_num = buildFilter(*data_set.NAME, column_ptrs, filter, num_rows, data_set); \ break; APPLY_FOR_SET_VARIANTS(M) #undef M diff --git a/src/Processors/Transforms/IntersectOrExceptTransform.h b/src/Processors/Transforms/IntersectOrExceptTransform.h index 3c2b9581d6d..5b62ef65d8d 100644 --- a/src/Processors/Transforms/IntersectOrExceptTransform.h +++ b/src/Processors/Transforms/IntersectOrExceptTransform.h @@ -3,16 +3,20 @@ #include #include #include +#include + namespace DB { class IntersectOrExceptTransform : public IProcessor { -public: - IntersectOrExceptTransform(bool is_except_, const Block & header_); +using Modes = ASTIntersectOrExcept::Modes; - String getName() const override { return is_except ? "Except" : "Intersect"; } +public: + IntersectOrExceptTransform(const Block & header_, const Modes & modes); + + String getName() const override { return "IntersectExcept"; } protected: Status prepare() override; @@ -20,15 +24,22 @@ protected: void work() override; private: - bool is_except; + Modes modes; + InputPorts::iterator first_input; + InputPorts::iterator second_input; + size_t current_operator_pos = 0; bool push_empty_chunk = false; Chunk empty_chunk; + ColumnNumbers key_columns_pos; - SetVariants data; + std::optional data; Sizes key_sizes; + Chunk current_input_chunk; Chunk current_output_chunk; + bool more = false; + bool finished_second_input = false; bool has_input = false; diff --git a/tests/queries/0_stateless/02004_intersect_except_operators.reference b/tests/queries/0_stateless/02004_intersect_except_operators.reference index 763a5872cce..d17216a5ec4 100644 --- a/tests/queries/0_stateless/02004_intersect_except_operators.reference +++ b/tests/queries/0_stateless/02004_intersect_except_operators.reference @@ -23,3 +23,33 @@ select number, number+10 from numbers(12) except select number+5, number+15 from 2 12 3 13 4 14 +select 1 except select 2 intersect select 1; +1 +select 1 except select 2 intersect select 2; +select 1 intersect select 1 except select 2; +1 +select 1 intersect select 1 except select 1; +select 1 intersect select 1 except select 2 intersect select 1 except select 3 intersect select 1; +1 +select 1 intersect select 1 except select 2 intersect select 1 except select 3 intersect select 2; +select number from numbers(100) intersect select number from numbers(20, 60) except select number from numbers(30, 20) except select number from numbers(60, 20); +20 +21 +22 +23 +24 +25 +26 +27 +28 +29 +50 +51 +52 +53 +54 +55 +56 +57 +58 +59 diff --git a/tests/queries/0_stateless/02004_intersect_except_operators.sql b/tests/queries/0_stateless/02004_intersect_except_operators.sql index d0416fd899e..971aa262070 100644 --- a/tests/queries/0_stateless/02004_intersect_except_operators.sql +++ b/tests/queries/0_stateless/02004_intersect_except_operators.sql @@ -3,6 +3,16 @@ select 1 intersect select 1; select 2 intersect select 1; select 1 except select 1; select 2 except select 1; + select number from numbers(5, 5) intersect select number from numbers(20); select number from numbers(10) except select number from numbers(5); select number, number+10 from numbers(12) except select number+5, number+15 from numbers(10); + +select 1 except select 2 intersect select 1; +select 1 except select 2 intersect select 2; +select 1 intersect select 1 except select 2; +select 1 intersect select 1 except select 1; +select 1 intersect select 1 except select 2 intersect select 1 except select 3 intersect select 1; +select 1 intersect select 1 except select 2 intersect select 1 except select 3 intersect select 2; + +select number from numbers(100) intersect select number from numbers(20, 60) except select number from numbers(30, 20) except select number from numbers(60, 20); From 07a6903009697d6eca621aa1bb2070a1b637da23 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 10 Aug 2021 09:35:35 +0300 Subject: [PATCH 048/161] Try fix build. --- src/Common/DenseHashMap.h | 7 +++++++ src/Common/DenseHashSet.h | 7 +++++++ src/Common/SparseHashMap.h | 7 +++++++ 3 files changed, 21 insertions(+) diff --git a/src/Common/DenseHashMap.h b/src/Common/DenseHashMap.h index d27b8d93a27..44287c604a7 100644 --- a/src/Common/DenseHashMap.h +++ b/src/Common/DenseHashMap.h @@ -5,7 +5,14 @@ #define HASH_FUN_H #endif +#ifdef __clang__ +#pragma clang diagnostic push +#pragma clang diagnostic ignored "-Wmacro-redefined" +#endif #include +#ifdef __clang__ +#pragma clang diagnostic pop +#endif #if !defined(ARCADIA_BUILD) template , diff --git a/src/Common/DenseHashSet.h b/src/Common/DenseHashSet.h index 750dc829817..ee3838d7be1 100644 --- a/src/Common/DenseHashSet.h +++ b/src/Common/DenseHashSet.h @@ -5,7 +5,14 @@ #define HASH_FUN_H #endif +#ifdef __clang__ +#pragma clang diagnostic push +#pragma clang diagnostic ignored "-Wmacro-redefined" +#endif #include +#ifdef __clang__ +#pragma clang diagnostic pop +#endif #if !defined(ARCADIA_BUILD) template , diff --git a/src/Common/SparseHashMap.h b/src/Common/SparseHashMap.h index 22847499bce..b6a7f1f9b9c 100644 --- a/src/Common/SparseHashMap.h +++ b/src/Common/SparseHashMap.h @@ -5,7 +5,14 @@ #define HASH_FUN_H #endif +#ifdef __clang__ +#pragma clang diagnostic push +#pragma clang diagnostic ignored "-Wmacro-redefined" +#endif #include +#ifdef __clang__ +#pragma clang diagnostic pop +#endif #if !defined(ARCADIA_BUILD) template , From 2306fbe9be189d9066eae32609d80a59585656da Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 10 Aug 2021 10:23:18 +0300 Subject: [PATCH 049/161] Better --- .../InterpreterIntersectOrExcept.cpp | 11 ++--- .../InterpreterIntersectOrExcept.h | 4 +- src/Parsers/ASTIntersectOrExcept.cpp | 8 ++-- src/Parsers/ASTIntersectOrExcept.h | 6 +-- ...t.cpp => ParserIntersectOrExceptQuery.cpp} | 17 ++++--- ...xcept.h => ParserIntersectOrExceptQuery.h} | 2 +- src/Parsers/ParserQueryWithOutput.cpp | 6 +-- .../QueryPlan/IntersectOrExceptStep.cpp | 14 +++--- .../QueryPlan/IntersectOrExceptStep.h | 8 ++-- .../Transforms/IntersectOrExceptTransform.cpp | 48 +++++++++---------- .../Transforms/IntersectOrExceptTransform.h | 13 ++--- 11 files changed, 66 insertions(+), 71 deletions(-) rename src/Parsers/{ParserIntersectOrExcept.cpp => ParserIntersectOrExceptQuery.cpp} (73%) rename src/Parsers/{ParserIntersectOrExcept.h => ParserIntersectOrExceptQuery.h} (80%) diff --git a/src/Interpreters/InterpreterIntersectOrExcept.cpp b/src/Interpreters/InterpreterIntersectOrExcept.cpp index d706ab02bfd..52dcb24ed27 100644 --- a/src/Interpreters/InterpreterIntersectOrExcept.cpp +++ b/src/Interpreters/InterpreterIntersectOrExcept.cpp @@ -23,15 +23,10 @@ InterpreterIntersectOrExcept::InterpreterIntersectOrExcept(const ASTPtr & query_ : context(Context::createCopy(context_)) { ASTIntersectOrExcept * ast = query_ptr->as(); + operators = ast->list_of_operators; + auto children = ast->list_of_selects->children; - modes = ast->list_of_modes; - if (modes.size() + 1 != children.size()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Number of modes and number of children are not consistent"); - size_t num_children = children.size(); - if (!num_children) - throw Exception(ErrorCodes::LOGICAL_ERROR, "No children in ASTIntersectOrExceptQuery"); - nested_interpreters.resize(num_children); for (size_t i = 0; i < num_children; ++i) @@ -106,7 +101,7 @@ void InterpreterIntersectOrExcept::buildQueryPlan(QueryPlan & query_plan) } auto max_threads = context->getSettingsRef().max_threads; - auto step = std::make_unique(std::move(data_streams), modes, max_threads); + auto step = std::make_unique(std::move(data_streams), operators, max_threads); query_plan.unitePlans(std::move(step), std::move(plans)); } diff --git a/src/Interpreters/InterpreterIntersectOrExcept.h b/src/Interpreters/InterpreterIntersectOrExcept.h index 8021a25df06..359be05db8b 100644 --- a/src/Interpreters/InterpreterIntersectOrExcept.h +++ b/src/Interpreters/InterpreterIntersectOrExcept.h @@ -21,7 +21,7 @@ public: BlockIO execute() override; private: - String getName() const { return "IntersectExcept"; } + String getName() const { return "IntersectOrExcept"; } Block getCommonHeader(const Blocks & headers) const; @@ -33,7 +33,7 @@ private: ContextPtr context; Block result_header; std::vector> nested_interpreters; - ASTIntersectOrExcept::Modes modes; + ASTIntersectOrExcept::Operators operators; }; } diff --git a/src/Parsers/ASTIntersectOrExcept.cpp b/src/Parsers/ASTIntersectOrExcept.cpp index 7d92055646c..33ffb76c2f7 100644 --- a/src/Parsers/ASTIntersectOrExcept.cpp +++ b/src/Parsers/ASTIntersectOrExcept.cpp @@ -13,7 +13,7 @@ ASTPtr ASTIntersectOrExcept::clone() const res->list_of_selects = list_of_selects->clone(); res->children.push_back(res->list_of_selects); - res->list_of_modes = list_of_modes; + res->list_of_operators = list_of_operators; cloneOutputOptions(*res); return res; @@ -23,9 +23,9 @@ void ASTIntersectOrExcept::formatQueryImpl(const FormatSettings & settings, Form { std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' '); - auto mode_to_str = [&](auto mode) + auto operator_to_str = [&](auto current_operator) { - if (mode == Mode::INTERSECT) + if (current_operator == Operator::INTERSECT) return "INTERSECT"; else return "EXCEPT"; @@ -36,7 +36,7 @@ void ASTIntersectOrExcept::formatQueryImpl(const FormatSettings & settings, Form if (it != list_of_selects->children.begin()) { settings.ostr << settings.nl_or_ws << indent_str << (settings.hilite ? hilite_keyword : "") - << mode_to_str(list_of_modes[it - list_of_selects->children.begin() - 1]) + << operator_to_str(list_of_operators[it - list_of_selects->children.begin() - 1]) << (settings.hilite ? hilite_none : ""); } diff --git a/src/Parsers/ASTIntersectOrExcept.h b/src/Parsers/ASTIntersectOrExcept.h index 0eb8ab3b113..9adfdedc497 100644 --- a/src/Parsers/ASTIntersectOrExcept.h +++ b/src/Parsers/ASTIntersectOrExcept.h @@ -15,16 +15,16 @@ public: void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; - enum class Mode + enum class Operator { INTERSECT, EXCEPT }; - using Modes = std::vector; + using Operators = std::vector; ASTPtr list_of_selects; - Modes list_of_modes; + Operators list_of_operators; }; } diff --git a/src/Parsers/ParserIntersectOrExcept.cpp b/src/Parsers/ParserIntersectOrExceptQuery.cpp similarity index 73% rename from src/Parsers/ParserIntersectOrExcept.cpp rename to src/Parsers/ParserIntersectOrExceptQuery.cpp index fd1511b8af6..ef6d68f8534 100644 --- a/src/Parsers/ParserIntersectOrExcept.cpp +++ b/src/Parsers/ParserIntersectOrExceptQuery.cpp @@ -2,7 +2,7 @@ #include #include #include -#include +#include #include #include #include @@ -11,13 +11,13 @@ namespace DB { -bool ParserIntersectOrExcept::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +bool ParserIntersectOrExceptQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ParserKeyword intersect_keyword("INTERSECT"); ParserKeyword except_keyword("EXCEPT"); ASTs elements; - ASTIntersectOrExcept::Modes modes; + ASTIntersectOrExcept::Operators operators; auto parse_element = [&]() -> bool { @@ -36,18 +36,21 @@ bool ParserIntersectOrExcept::parseImpl(Pos & pos, ASTPtr & node, Expected & exp if (!except_keyword.ignore(pos)) return false; - modes.emplace_back(ASTIntersectOrExcept::Mode::EXCEPT); + operators.emplace_back(ASTIntersectOrExcept::Operator::EXCEPT); return true; } - modes.emplace_back(ASTIntersectOrExcept::Mode::INTERSECT); + operators.emplace_back(ASTIntersectOrExcept::Operator::INTERSECT); return true; }; if (!ParserUnionList::parseUtil(pos, parse_element, parse_separator)) return false; - if (modes.empty()) + if (operators.empty() || elements.empty()) + return false; + + if (operators.size() + 1 != elements.size()) return false; auto list_node = std::make_shared(); @@ -58,7 +61,7 @@ bool ParserIntersectOrExcept::parseImpl(Pos & pos, ASTPtr & node, Expected & exp node = intersect_or_except_ast; intersect_or_except_ast->list_of_selects = list_node; intersect_or_except_ast->children.push_back(intersect_or_except_ast->list_of_selects); - intersect_or_except_ast->list_of_modes = modes; + intersect_or_except_ast->list_of_operators = operators; return true; } diff --git a/src/Parsers/ParserIntersectOrExcept.h b/src/Parsers/ParserIntersectOrExceptQuery.h similarity index 80% rename from src/Parsers/ParserIntersectOrExcept.h rename to src/Parsers/ParserIntersectOrExceptQuery.h index 61cc74cf0a9..d8ba82ba053 100644 --- a/src/Parsers/ParserIntersectOrExcept.h +++ b/src/Parsers/ParserIntersectOrExceptQuery.h @@ -4,7 +4,7 @@ namespace DB { -class ParserIntersectOrExcept : public IParserBase +class ParserIntersectOrExceptQuery : public IParserBase { protected: const char * getName() const override { return "INTERSECT or EXCEPT"; } diff --git a/src/Parsers/ParserQueryWithOutput.cpp b/src/Parsers/ParserQueryWithOutput.cpp index 35355b29ebf..d7d87cac9b9 100644 --- a/src/Parsers/ParserQueryWithOutput.cpp +++ b/src/Parsers/ParserQueryWithOutput.cpp @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include #include #include @@ -31,7 +31,7 @@ namespace DB bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ParserShowTablesQuery show_tables_p; - ParserIntersectOrExcept intersect_p; + ParserIntersectOrExceptQuery intersect_except_p; ParserSelectWithUnionQuery select_p; ParserTablePropertiesQuery table_p; ParserDescribeTableQuery describe_table_p; @@ -55,7 +55,7 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec bool parsed = explain_p.parse(pos, query, expected) - || intersect_p.parse(pos, query, expected) + || intersect_except_p.parse(pos, query, expected) || select_p.parse(pos, query, expected) || show_create_access_entity_p.parse(pos, query, expected) /// should be before `show_tables_p` || show_tables_p.parse(pos, query, expected) diff --git a/src/Processors/QueryPlan/IntersectOrExceptStep.cpp b/src/Processors/QueryPlan/IntersectOrExceptStep.cpp index b1b5c1b8813..76f496ba47c 100644 --- a/src/Processors/QueryPlan/IntersectOrExceptStep.cpp +++ b/src/Processors/QueryPlan/IntersectOrExceptStep.cpp @@ -14,17 +14,20 @@ namespace DB Block IntersectOrExceptStep::checkHeaders(const DataStreams & input_streams_) const { if (input_streams_.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot perform {} on empty set of query plan steps", getName()); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot perform intersect/except on empty set of query plan steps"); Block res = input_streams_.front().header; for (const auto & stream : input_streams_) - assertBlocksHaveEqualStructure(stream.header, res, "IntersectOrExceptStep"); + assertBlocksHaveEqualStructure(stream.header, res, "IntersectExceptStep"); return res; } -IntersectOrExceptStep::IntersectOrExceptStep(DataStreams input_streams_, const Modes & modes_, size_t max_threads_) - : header(checkHeaders(input_streams_)), modes(modes_), max_threads(max_threads_) +IntersectOrExceptStep::IntersectOrExceptStep( + DataStreams input_streams_ , const Operators & operators_ , size_t max_threads_) + : header(checkHeaders(input_streams_)) + , operators(operators_) + , max_threads(max_threads_) { input_streams = std::move(input_streams_); if (input_streams.size() == 1) @@ -63,9 +66,8 @@ QueryPipelinePtr IntersectOrExceptStep::updatePipeline(QueryPipelines pipelines, } } - std::cerr << "size: " << input_streams.size() << std::endl; *pipeline = QueryPipeline::unitePipelines(std::move(pipelines), max_threads); - pipeline->addTransform(std::make_shared(header, modes)); + pipeline->addTransform(std::make_shared(header, operators)); processors = collector.detachProcessors(); return pipeline; diff --git a/src/Processors/QueryPlan/IntersectOrExceptStep.h b/src/Processors/QueryPlan/IntersectOrExceptStep.h index 4eceb820153..914a7dce197 100644 --- a/src/Processors/QueryPlan/IntersectOrExceptStep.h +++ b/src/Processors/QueryPlan/IntersectOrExceptStep.h @@ -8,13 +8,13 @@ namespace DB class IntersectOrExceptStep : public IQueryPlanStep { -using Modes = ASTIntersectOrExcept::Modes; +using Operators = ASTIntersectOrExcept::Operators; public: /// max_threads is used to limit the number of threads for result pipeline. - IntersectOrExceptStep(DataStreams input_streams_, const Modes & modes_, size_t max_threads_ = 0); + IntersectOrExceptStep(DataStreams input_streams_, const Operators & operators_, size_t max_threads_ = 0); - String getName() const override { return "IntersectExcept"; } + String getName() const override { return "IntersectOrExcept"; } QueryPipelinePtr updatePipeline(QueryPipelines pipelines, const BuildQueryPipelineSettings & settings) override; @@ -24,7 +24,7 @@ private: Block checkHeaders(const DataStreams & input_streams_) const; Block header; - Modes modes; + Operators operators; size_t max_threads; Processors processors; }; diff --git a/src/Processors/Transforms/IntersectOrExceptTransform.cpp b/src/Processors/Transforms/IntersectOrExceptTransform.cpp index 70a86855992..68d5f6a2e5e 100644 --- a/src/Processors/Transforms/IntersectOrExceptTransform.cpp +++ b/src/Processors/Transforms/IntersectOrExceptTransform.cpp @@ -4,9 +4,13 @@ namespace DB { -IntersectOrExceptTransform::IntersectOrExceptTransform(const Block & header_, const Modes & modes_) - : IProcessor(InputPorts(modes_.size() + 1, header_), {header_}) - , modes(modes_) +/* + * There are always at least two inputs. Number of operators is always number of inputs minus 1. + * input1 {operator1} input2 {operator2} input3 ... +**/ +IntersectOrExceptTransform::IntersectOrExceptTransform(const Block & header_, const Operators & operators_) + : IProcessor(InputPorts(operators_.size() + 1, header_), {header_}) + , operators(operators_) , first_input(inputs.begin()) , second_input(std::next(inputs.begin())) { @@ -30,6 +34,7 @@ IntersectOrExceptTransform::Status IntersectOrExceptTransform::prepare() { for (auto & in : inputs) in.close(); + return Status::Finished; } @@ -37,24 +42,13 @@ IntersectOrExceptTransform::Status IntersectOrExceptTransform::prepare() { for (auto & input : inputs) input.setNotNeeded(); + return Status::PortFull; } - /// Output if has data. - if (current_output_chunk && second_input == inputs.end()) - { - output.push(std::move(current_output_chunk)); - } - - if (push_empty_chunk) - { - output.push(std::move(empty_chunk)); - push_empty_chunk = false; - } - if (finished_second_input) { - if (first_input->isFinished() || (more && !current_input_chunk)) + if (first_input->isFinished() || (use_accumulated_input && !current_input_chunk)) { std::advance(second_input, 1); @@ -64,12 +58,13 @@ IntersectOrExceptTransform::Status IntersectOrExceptTransform::prepare() { output.push(std::move(current_output_chunk)); } + output.finish(); return Status::Finished; } else { - more = true; + use_accumulated_input = true; data.reset(); finished_second_input = false; ++current_operator_pos; @@ -81,20 +76,20 @@ IntersectOrExceptTransform::Status IntersectOrExceptTransform::prepare() finished_second_input = true; } - InputPort & input = finished_second_input ? *first_input : *second_input; - - /// Check can input. if (!has_input) { - if (finished_second_input && more) + if (finished_second_input && use_accumulated_input) { current_input_chunk = std::move(current_output_chunk); } else { + InputPort & input = finished_second_input ? *first_input : *second_input; + input.setNeeded(); if (!input.hasData()) return Status::NeedData; + current_input_chunk = input.pull(); } @@ -107,9 +102,6 @@ IntersectOrExceptTransform::Status IntersectOrExceptTransform::prepare() void IntersectOrExceptTransform::work() { - if (!data) - data.emplace(); - if (!finished_second_input) { accumulate(std::move(current_input_chunk)); @@ -144,7 +136,7 @@ size_t IntersectOrExceptTransform::buildFilter( for (size_t i = 0; i < rows; ++i) { auto find_result = state.findKey(method.data, i, variants.string_pool); - filter[i] = modes[current_operator_pos] == ASTIntersectOrExcept::Mode::EXCEPT ? !find_result.isFound() : find_result.isFound(); + filter[i] = operators[current_operator_pos] == ASTIntersectOrExcept::Operator::EXCEPT ? !find_result.isFound() : find_result.isFound(); if (filter[i]) ++new_rows_num; } @@ -163,6 +155,9 @@ void IntersectOrExceptTransform::accumulate(Chunk chunk) for (auto pos : key_columns_pos) column_ptrs.emplace_back(columns[pos].get()); + if (!data) + data.emplace(); + if (data->empty()) data->init(SetVariants::chooseMethod(column_ptrs, key_sizes)); @@ -192,6 +187,9 @@ void IntersectOrExceptTransform::filter(Chunk & chunk) for (auto pos : key_columns_pos) column_ptrs.emplace_back(columns[pos].get()); + if (!data) + data.emplace(); + if (data->empty()) data->init(SetVariants::chooseMethod(column_ptrs, key_sizes)); diff --git a/src/Processors/Transforms/IntersectOrExceptTransform.h b/src/Processors/Transforms/IntersectOrExceptTransform.h index 5b62ef65d8d..6d0c3516d5d 100644 --- a/src/Processors/Transforms/IntersectOrExceptTransform.h +++ b/src/Processors/Transforms/IntersectOrExceptTransform.h @@ -11,12 +11,12 @@ namespace DB class IntersectOrExceptTransform : public IProcessor { -using Modes = ASTIntersectOrExcept::Modes; +using Operators = ASTIntersectOrExcept::Operators; public: - IntersectOrExceptTransform(const Block & header_, const Modes & modes); + IntersectOrExceptTransform(const Block & header_, const Operators & operators); - String getName() const override { return "IntersectExcept"; } + String getName() const override { return "IntersectOrExcept"; } protected: Status prepare() override; @@ -24,22 +24,19 @@ protected: void work() override; private: - Modes modes; + Operators operators; InputPorts::iterator first_input; InputPorts::iterator second_input; size_t current_operator_pos = 0; - bool push_empty_chunk = false; - Chunk empty_chunk; - ColumnNumbers key_columns_pos; std::optional data; Sizes key_sizes; Chunk current_input_chunk; Chunk current_output_chunk; - bool more = false; + bool use_accumulated_input = false; bool finished_second_input = false; bool has_input = false; From d03c867ad9446d49440efb6ea7b3415096b703e5 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 10 Aug 2021 11:44:44 +0300 Subject: [PATCH 050/161] Add Y_IGNORE --- src/IO/Bzip2ReadBuffer.cpp | 2 +- src/IO/Bzip2WriteBuffer.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/IO/Bzip2ReadBuffer.cpp b/src/IO/Bzip2ReadBuffer.cpp index e264ce75444..99798bca325 100644 --- a/src/IO/Bzip2ReadBuffer.cpp +++ b/src/IO/Bzip2ReadBuffer.cpp @@ -4,7 +4,7 @@ #if USE_BZIP2 # include -# include +# include // Y_IGNORE namespace DB { diff --git a/src/IO/Bzip2WriteBuffer.cpp b/src/IO/Bzip2WriteBuffer.cpp index 41cb972966c..39c5356b792 100644 --- a/src/IO/Bzip2WriteBuffer.cpp +++ b/src/IO/Bzip2WriteBuffer.cpp @@ -4,7 +4,7 @@ #if USE_BROTLI # include -# include +# include // Y_IGNORE #include From 54f161ca1f4a216fcad20fb4a1e4e9444e1663f9 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Tue, 10 Aug 2021 13:53:35 +0300 Subject: [PATCH 051/161] Add empty for UUID and arrays. --- .../functions/array-functions.md | 85 +++++++++++++++++-- .../functions/string-functions.md | 22 +++-- .../sql-reference/functions/uuid-functions.md | 84 ++++++++++++++++++ 3 files changed, 174 insertions(+), 17 deletions(-) diff --git a/docs/en/sql-reference/functions/array-functions.md b/docs/en/sql-reference/functions/array-functions.md index b56d403edf6..655ccdeea81 100644 --- a/docs/en/sql-reference/functions/array-functions.md +++ b/docs/en/sql-reference/functions/array-functions.md @@ -7,19 +7,88 @@ toc_title: Arrays ## empty {#function-empty} -Returns 1 for an empty array, or 0 for a non-empty array. -The result type is UInt8. -The function also works for strings. +Checks whether the input array is empty. -Can be optimized by enabling the [optimize_functions_to_subcolumns](../../operations/settings/settings.md#optimize-functions-to-subcolumns) setting. With `optimize_functions_to_subcolumns = 1` the function reads only [size0](../../sql-reference/data-types/array.md#array-size) subcolumn instead of reading and processing the whole array column. The query `SELECT empty(arr) FROM table` transforms to `SELECT arr.size0 = 0 FROM TABLE`. +**Syntax** + +``` sql +empty([x]) +``` + +An array is considered empty if it contains all empty elements. + +!!! note "Note" + Can be optimized by enabling the [optimize_functions_to_subcolumns](../../operations/settings/settings.md#optimize-functions-to-subcolumns) setting. With `optimize_functions_to_subcolumns = 1` the function reads only [size0](../../sql-reference/data-types/array.md#array-size) subcolumn instead of reading and processing the whole array column. The query `SELECT empty(arr) FROM table;` transforms to `SELECT arr.size0 = 0 FROM TABLE;`. + +The function also works for [strings](string-functions.md#empty) or [UUID](uuid-functions.md#empty). + +**Arguments** + +- `[x]` — Input array. [Array](../data-types/array.md). + +**Returned value** + +- Returns `1` for an empty array or `0` for a non-empty array. + +Type: [UInt8](../data-types/int-uint.md). + +**Example** + +Query: + +```sql +SELECT empty([]); +``` + +Result: + +```text +┌─empty(array())─┐ +│ 1 │ +└────────────────┘ +``` ## notEmpty {#function-notempty} -Returns 0 for an empty array, or 1 for a non-empty array. -The result type is UInt8. -The function also works for strings. +Checks whether the input array is non-empty. -Can be optimized by enabling the [optimize_functions_to_subcolumns](../../operations/settings/settings.md#optimize-functions-to-subcolumns) setting. With `optimize_functions_to_subcolumns = 1` the function reads only [size0](../../sql-reference/data-types/array.md#array-size) subcolumn instead of reading and processing the whole array column. The query `SELECT notEmpty(arr) FROM table` transforms to `SELECT arr.size0 != 0 FROM TABLE`. +**Syntax** + +``` sql +notEmpty([x]) +``` +An array is considered non-empty if it contains at least one non-empty element. + +!!! note "Note" + Can be optimized by enabling the [optimize_functions_to_subcolumns](../../operations/settings/settings.md#optimize-functions-to-subcolumns) setting. With `optimize_functions_to_subcolumns = 1` the function reads only [size0](../../sql-reference/data-types/array.md#array-size) subcolumn instead of reading and processing the whole array column. The query `SELECT notEmpty(arr) FROM table` transforms to `SELECT arr.size0 != 0 FROM TABLE`. + +The function also works for [strings](string-functions.md#notempty) or [UUID](uuid-functions.md#notempty). + +**Arguments** + +- `[x]` — Input array. [Array](../data-types/array.md). + +**Returned value** + +- Returns `1` for a non-empty array or `0` for an empty array. + +Type: [UInt8](../data-types/int-uint.md). + +**Example** + +Query: + +```sql +SELECT notEmpty([1,2]); +``` + +Result: + +```text +┌─notEmpty([1, 2])─┐ +│ 1 │ +└──────────────────┘ +``` ## length {#array_functions-length} diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index d54bb16abbc..7f3ecb80ee9 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -20,9 +20,11 @@ empty(x) A string is considered non-empty if it contains at least one byte, even if this is a space or a null byte. The UUID is empty if it contains all zeros (zero UUID). +The function also works for [arrays](array-functions.md#function-empty) or [UUID](uuid-functions.md#empty). + **Arguments** -- `x` — Input value. [Array](../data-types/array.md), [String](../data-types/string.md), [UUID](../data-types/uuid.md). +- `x` — Input value. [String](../data-types/string.md). **Returned value** @@ -46,21 +48,23 @@ Result: └───────────┘ ``` -## notempty {#notempty} +## notEmpty {#notempty} -Checks whether the input string is not empty. +Checks whether the input string is non-empty. **Syntax** ``` sql -notempty(x) +notEmpty(x) ``` A string is considered non-empty if it contains at least one byte, even if this is a space or a null byte. The UUID is empty if it contains all zeros (zero UUID). +The function also works for [arrays](array-functions.md#function-notempty) or [UUID](uuid-functions.md#notempty). + **Arguments** -- `x` — Input value. [Array](../data-types/array.md), [String](../data-types/string.md), [UUID](../data-types/uuid.md). +- `x` — Input value. [String](../data-types/string.md). **Returned value** @@ -73,15 +77,15 @@ Type: [UInt8](../data-types/int-uint.md). Query: ```sql -SELECT notempty('text'); +SELECT notEmpty('text'); ``` Result: ```text -┌─empty('')─┐ -│ 1 │ -└───────────┘ +┌─notEmpty('text')─┐ +│ 1 │ +└──────────────────┘ ``` ## length {#length} diff --git a/docs/en/sql-reference/functions/uuid-functions.md b/docs/en/sql-reference/functions/uuid-functions.md index e7e55c699cd..63bd2446f39 100644 --- a/docs/en/sql-reference/functions/uuid-functions.md +++ b/docs/en/sql-reference/functions/uuid-functions.md @@ -37,6 +37,90 @@ SELECT * FROM t_uuid └──────────────────────────────────────┘ ``` +## empty {#empty} + +Checks whether the input UUID is empty. + +**Syntax** + +```sql +empty(UUID) +``` + +The UUID is considered empty if it contains all zeros (zero UUID). + +The function also works for [arrays](array-functions.md#function-empty) or [strings](string-functions.md#empty). + +**Arguments** + +- `x` — Input UUID. [UUID](../data-types/UUID.md). + +**Returned value** + +- Returns `1` for an empty UUID or `0` for a non-empty UUID. + +Type: [UInt8](../data-types/int-uint.md). + +**Example** + +To generate the UUID value, ClickHouse provides the [generateUUIDv4](uuid-function-generate) function. + +Query: + +```sql +SELECT empty(generateUUIDv4()); +``` + +Result: + +```text +┌─empty(generateUUIDv4())─┐ +│ 0 │ +└─────────────────────────┘ +``` + +## notEmpty {#notempty} + +Checks whether the input UUID is non-empty. + +**Syntax** + +```sql +notEmpty(UUID) +``` + +The UUID is considered empty if it contains all zeros (zero UUID). + +The function also works for [arrays](array-functions.md#function-notempty) or [strings](string-functions.md#notempty). + +**Arguments** + +- `x` — Input UUID. [UUID](../data-types/UUID.md). + +**Returned value** + +- Returns `1` for a non-empty UUID or `0` for an empty UUID. + +Type: [UInt8](../data-types/int-uint.md). + +**Example** + +To generate the UUID value, ClickHouse provides the [generateUUIDv4](uuid-function-generate) function. + +Query: + +```sql +SELECT notEmpty(generateUUIDv4()); +``` + +Result: + +```text +┌─notEmpty(generateUUIDv4())─┐ +│ 1 │ +└────────────────────────────┘ +``` + ## toUUID (x) {#touuid-x} Converts String type value to UUID type. From e39e995dc3bed730e5fcf2fb4a2ebf3575989cac Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Tue, 10 Aug 2021 14:06:37 +0300 Subject: [PATCH 052/161] Fix errors. --- docs/en/sql-reference/functions/uuid-functions.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/uuid-functions.md b/docs/en/sql-reference/functions/uuid-functions.md index 63bd2446f39..bbfa19dc1c3 100644 --- a/docs/en/sql-reference/functions/uuid-functions.md +++ b/docs/en/sql-reference/functions/uuid-functions.md @@ -63,7 +63,7 @@ Type: [UInt8](../data-types/int-uint.md). **Example** -To generate the UUID value, ClickHouse provides the [generateUUIDv4](uuid-function-generate) function. +To generate the UUID value, ClickHouse provides the [generateUUIDv4](#uuid-function-generate) function. Query: @@ -105,7 +105,7 @@ Type: [UInt8](../data-types/int-uint.md). **Example** -To generate the UUID value, ClickHouse provides the [generateUUIDv4](uuid-function-generate) function. +To generate the UUID value, ClickHouse provides the [generateUUIDv4](#uuid-function-generate) function. Query: From 60c6daa579c9d89fd915a1f267b91b51f9780fc4 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Tue, 10 Aug 2021 14:09:20 +0300 Subject: [PATCH 053/161] Fix errors. --- docs/en/sql-reference/functions/uuid-functions.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/uuid-functions.md b/docs/en/sql-reference/functions/uuid-functions.md index bbfa19dc1c3..d02f5af6cd0 100644 --- a/docs/en/sql-reference/functions/uuid-functions.md +++ b/docs/en/sql-reference/functions/uuid-functions.md @@ -53,7 +53,7 @@ The function also works for [arrays](array-functions.md#function-empty) or [stri **Arguments** -- `x` — Input UUID. [UUID](../data-types/UUID.md). +- `x` — Input UUID. [UUID](../data-types/uuid.md). **Returned value** @@ -95,7 +95,7 @@ The function also works for [arrays](array-functions.md#function-notempty) or [s **Arguments** -- `x` — Input UUID. [UUID](../data-types/UUID.md). +- `x` — Input UUID. [UUID](../data-types/uuid.md). **Returned value** From 5e3bcb557405db4043e62cefb2af5237c98d5f18 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Tue, 10 Aug 2021 14:11:41 +0300 Subject: [PATCH 054/161] Fix errors. --- docs/en/sql-reference/functions/uuid-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/uuid-functions.md b/docs/en/sql-reference/functions/uuid-functions.md index d02f5af6cd0..e5ab45bda40 100644 --- a/docs/en/sql-reference/functions/uuid-functions.md +++ b/docs/en/sql-reference/functions/uuid-functions.md @@ -9,7 +9,7 @@ The functions for working with UUID are listed below. ## generateUUIDv4 {#uuid-function-generate} -Generates the [UUID](../../sql-reference/data-types/uuid.md) of [version 4](https://tools.ietf.org/html/rfc4122#section-4.4). +Generates the [UUID](../data-types/uuid.md) of [version 4](https://tools.ietf.org/html/rfc4122#section-4.4). ``` sql generateUUIDv4() From 669a80000d215c15910d0a84d5e0a6a321f6f211 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 10 Aug 2021 13:20:15 +0200 Subject: [PATCH 055/161] Fix 01236_graphite_mt for random timezones --- .../queries/0_stateless/01236_graphite_mt.sql | 34 +++++++++---------- 1 file changed, 17 insertions(+), 17 deletions(-) diff --git a/tests/queries/0_stateless/01236_graphite_mt.sql b/tests/queries/0_stateless/01236_graphite_mt.sql index 88d2d0ccb63..035fdf6af69 100644 --- a/tests/queries/0_stateless/01236_graphite_mt.sql +++ b/tests/queries/0_stateless/01236_graphite_mt.sql @@ -1,26 +1,26 @@ drop table if exists test_graphite; -create table test_graphite (key UInt32, Path String, Time DateTime, Value Float64, Version UInt32, col UInt64) +create table test_graphite (key UInt32, Path String, Time DateTime('UTC'), Value Float64, Version UInt32, col UInt64) engine = GraphiteMergeTree('graphite_rollup') order by key settings index_granularity=10; insert into test_graphite -select 1, 'sum_1', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300) union all -select 2, 'sum_1', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300) union all -select 1, 'sum_2', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300) union all -select 2, 'sum_2', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300) union all -select 1, 'max_1', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300) union all -select 2, 'max_1', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300) union all -select 1, 'max_2', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300) union all -select 2, 'max_2', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300); +select 1, 'sum_1', toDateTime(today(), 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all +select 2, 'sum_1', toDateTime(today(), 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all +select 1, 'sum_2', toDateTime(today(), 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all +select 2, 'sum_2', toDateTime(today(), 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all +select 1, 'max_1', toDateTime(today(), 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all +select 2, 'max_1', toDateTime(today(), 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all +select 1, 'max_2', toDateTime(today(), 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all +select 2, 'max_2', toDateTime(today(), 'UTC') - number * 60 - 30, number, 1, number from numbers(300); insert into test_graphite -select 1, 'sum_1', toDateTime(today() - 3) - number * 60 - 30, number, 1, number from numbers(1200) union all -select 2, 'sum_1', toDateTime(today() - 3) - number * 60 - 30, number, 1, number from numbers(1200) union all -select 1, 'sum_2', toDateTime(today() - 3) - number * 60 - 30, number, 1, number from numbers(1200) union all -select 2, 'sum_2', toDateTime(today() - 3) - number * 60 - 30, number, 1, number from numbers(1200) union all -select 1, 'max_1', toDateTime(today() - 3) - number * 60 - 30, number, 1, number from numbers(1200) union all -select 2, 'max_1', toDateTime(today() - 3) - number * 60 - 30, number, 1, number from numbers(1200) union all -select 1, 'max_2', toDateTime(today() - 3) - number * 60 - 30, number, 1, number from numbers(1200) union all -select 2, 'max_2', toDateTime(today() - 3) - number * 60 - 30, number, 1, number from numbers(1200); +select 1, 'sum_1', toDateTime(today() -3 , 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all +select 2, 'sum_1', toDateTime(today() -3 , 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all +select 1, 'sum_2', toDateTime(today() -3 , 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all +select 2, 'sum_2', toDateTime(today() -3 , 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all +select 1, 'max_1', toDateTime(today() -3 , 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all +select 2, 'max_1', toDateTime(today() -3 , 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all +select 1, 'max_2', toDateTime(today() -3 , 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all +select 2, 'max_2', toDateTime(today() -3 , 'UTC') - number * 60 - 30, number, 1, number from numbers(1200); optimize table test_graphite final; From 07cc2ec85b69539e56c5bcce25a0bba997160a98 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 10 Aug 2021 15:36:40 +0300 Subject: [PATCH 056/161] Try fix build. --- src/Common/DenseHashMap.h | 15 ++++++--------- src/Common/DenseHashSet.h | 14 +++++--------- src/Common/SparseHashMap.h | 14 +++++--------- 3 files changed, 16 insertions(+), 27 deletions(-) diff --git a/src/Common/DenseHashMap.h b/src/Common/DenseHashMap.h index 44287c604a7..7a9d5a68e2d 100644 --- a/src/Common/DenseHashMap.h +++ b/src/Common/DenseHashMap.h @@ -1,18 +1,13 @@ #pragma once +#include #if defined(ARCADIA_BUILD) -#undef HASH_FUN_H -#define HASH_FUN_H +#define HASH_FUN_H ; +template +struct THash : public std::hash {}; #endif -#ifdef __clang__ -#pragma clang diagnostic push -#pragma clang diagnostic ignored "-Wmacro-redefined" -#endif #include -#ifdef __clang__ -#pragma clang diagnostic pop -#endif #if !defined(ARCADIA_BUILD) template , @@ -24,4 +19,6 @@ class EqualKey = std::equal_to, class Alloc = google::sparsehash::libc_allocator_with_realloc>> using DenseHashMap = google::sparsehash::dense_hash_map; + + #undef THash #endif diff --git a/src/Common/DenseHashSet.h b/src/Common/DenseHashSet.h index ee3838d7be1..004d1c10903 100644 --- a/src/Common/DenseHashSet.h +++ b/src/Common/DenseHashSet.h @@ -1,18 +1,12 @@ #pragma once #if defined(ARCADIA_BUILD) -#undef HASH_FUN_H -#define HASH_FUN_H +#define HASH_FUN_H ; +template +struct THash : public std::hash {}; #endif -#ifdef __clang__ -#pragma clang diagnostic push -#pragma clang diagnostic ignored "-Wmacro-redefined" -#endif #include -#ifdef __clang__ -#pragma clang diagnostic pop -#endif #if !defined(ARCADIA_BUILD) template , @@ -24,4 +18,6 @@ class EqualKey = std::equal_to, class Alloc = google::sparsehash::libc_allocator_with_realloc> using DenseHashSet = google::sparsehash::dense_hash_set; + + #undef THash #endif diff --git a/src/Common/SparseHashMap.h b/src/Common/SparseHashMap.h index b6a7f1f9b9c..651dd22f15d 100644 --- a/src/Common/SparseHashMap.h +++ b/src/Common/SparseHashMap.h @@ -1,18 +1,12 @@ #pragma once #if defined(ARCADIA_BUILD) -#undef HASH_FUN_H -#define HASH_FUN_H +#define HASH_FUN_H ; +template +struct THash : public std::hash {}; #endif -#ifdef __clang__ -#pragma clang diagnostic push -#pragma clang diagnostic ignored "-Wmacro-redefined" -#endif #include -#ifdef __clang__ -#pragma clang diagnostic pop -#endif #if !defined(ARCADIA_BUILD) template , @@ -24,4 +18,6 @@ class EqualKey = std::equal_to, class Alloc = google::sparsehash::libc_allocator_with_realloc>> using SparseHashMap = google::sparsehash::sparse_hash_map; + + #undef THash #endif From e63c26edb713b8452b0979740020e9205cd3f839 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Tue, 10 Aug 2021 21:47:27 +0800 Subject: [PATCH 057/161] Fix projection materialization with missing columns --- src/Interpreters/MutationsInterpreter.cpp | 5 +++-- src/Interpreters/MutationsInterpreter.h | 2 +- .../MergeTree/MergeTreeDataMergerMutator.cpp | 12 ++++-------- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 5 +++-- ...ection_materialize_with_missing_columns.reference | 0 ...0_projection_materialize_with_missing_columns.sql | 9 +++++++++ 6 files changed, 20 insertions(+), 13 deletions(-) create mode 100644 tests/queries/0_stateless/01710_projection_materialize_with_missing_columns.reference create mode 100644 tests/queries/0_stateless/01710_projection_materialize_with_missing_columns.sql diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index fe0594bb58f..ff2f3e97c4e 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -920,9 +920,10 @@ BlockInputStreamPtr MutationsInterpreter::execute() return result_stream; } -const Block & MutationsInterpreter::getUpdatedHeader() const +Block MutationsInterpreter::getUpdatedHeader() const { - return *updated_header; + // If it's an index/projection materialization, we don't write any data columns, thus empty header is used + return mutation_kind.mutation_kind == MutationKind::MUTATE_INDEX_PROJECTION ? Block{} : *updated_header; } const ColumnDependencies & MutationsInterpreter::getColumnDependencies() const diff --git a/src/Interpreters/MutationsInterpreter.h b/src/Interpreters/MutationsInterpreter.h index c9a589e6b6d..4f8960ae8f7 100644 --- a/src/Interpreters/MutationsInterpreter.h +++ b/src/Interpreters/MutationsInterpreter.h @@ -53,7 +53,7 @@ public: BlockInputStreamPtr execute(); /// Only changed columns. - const Block & getUpdatedHeader() const; + Block getUpdatedHeader() const; const ColumnDependencies & getColumnDependencies() const; diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 6279d2d7d6f..0a92efdf61c 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -1332,11 +1332,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor { /// We will modify only some of the columns. Other columns and key values can be copied as-is. NameSet updated_columns; - if (mutation_kind != MutationsInterpreter::MutationKind::MUTATE_INDEX_PROJECTION) - { - for (const auto & name_type : updated_header.getNamesAndTypesList()) - updated_columns.emplace(name_type.name); - } + for (const auto & name_type : updated_header.getNamesAndTypesList()) + updated_columns.emplace(name_type.name); auto indices_to_recalc = getIndicesToRecalculate( in, updated_columns, metadata_snapshot, context, materialized_indices, source_part); @@ -1345,7 +1342,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor NameSet files_to_skip = collectFilesToSkip( source_part, - mutation_kind == MutationsInterpreter::MutationKind::MUTATE_INDEX_PROJECTION ? Block{} : updated_header, + updated_header, indices_to_recalc, mrk_extension, projections_to_recalc); @@ -1413,8 +1410,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor metadata_snapshot, indices_to_recalc, projections_to_recalc, - // If it's an index/projection materialization, we don't write any data columns, thus empty header is used - mutation_kind == MutationsInterpreter::MutationKind::MUTATE_INDEX_PROJECTION ? Block{} : updated_header, + updated_header, new_data_part, in, time_of_mutation, diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 0b5351dcf01..342bcb25b0b 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -361,10 +361,11 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( pipes.emplace_back(std::move(projection_pipe)); pipes.emplace_back(std::move(ordinary_pipe)); auto pipe = Pipe::unitePipes(std::move(pipes)); - // TODO what if pipe is empty? pipe.resize(1); - auto step = std::make_unique(std::move(pipe), "MergeTree(with projection)"); + auto step = std::make_unique( + std::move(pipe), + fmt::format("MergeTree(with {} projection {})", query_info.projection->desc->type, query_info.projection->desc->name)); auto plan = std::make_unique(); plan->addStep(std::move(step)); return plan; diff --git a/tests/queries/0_stateless/01710_projection_materialize_with_missing_columns.reference b/tests/queries/0_stateless/01710_projection_materialize_with_missing_columns.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01710_projection_materialize_with_missing_columns.sql b/tests/queries/0_stateless/01710_projection_materialize_with_missing_columns.sql new file mode 100644 index 00000000000..28bf1c050d0 --- /dev/null +++ b/tests/queries/0_stateless/01710_projection_materialize_with_missing_columns.sql @@ -0,0 +1,9 @@ +drop table if exists x; + +create table x (i int) engine MergeTree order by tuple(); +insert into x values (1); +alter table x add column j int; +alter table x add projection p_agg (select sum(j)); +alter table x materialize projection p_agg settings mutations_sync = 1; + +drop table x; From c2cd4d452a9d91440050d775236d62a3bce4257d Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 10 Aug 2021 17:42:18 +0300 Subject: [PATCH 058/161] Update Settings.h --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 623eb0298dd..05374a398eb 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -434,7 +434,7 @@ class IColumn; M(Bool, allow_experimental_geo_types, false, "Allow geo data types such as Point, Ring, Polygon, MultiPolygon", 0) \ M(Bool, data_type_default_nullable, false, "Data types without NULL or NOT NULL will make Nullable", 0) \ /** Temporarily set to true, to check how tests will feel.*/ \ - M(Bool, cast_keep_nullable, true, "CAST operator keep Nullable for result data type", 0) \ + M(Bool, cast_keep_nullable, false, "CAST operator keep Nullable for result data type", 0) \ M(Bool, alter_partition_verbose_result, false, "Output information about affected parts. Currently works only for FREEZE and ATTACH commands.", 0) \ M(Bool, allow_experimental_database_materialized_mysql, false, "Allow to create database with Engine=MaterializedMySQL(...).", 0) \ M(Bool, allow_experimental_database_materialized_postgresql, false, "Allow to create database with Engine=MaterializedPostgreSQL(...).", 0) \ From 3dfa54bfd8afada336fe9449561614fc80aea25b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 10 Aug 2021 17:08:43 +0200 Subject: [PATCH 059/161] Revert "Fix 01236_graphite_mt for random timezones" This reverts commit 669a80000d215c15910d0a84d5e0a6a321f6f211. --- .../queries/0_stateless/01236_graphite_mt.sql | 34 +++++++++---------- 1 file changed, 17 insertions(+), 17 deletions(-) diff --git a/tests/queries/0_stateless/01236_graphite_mt.sql b/tests/queries/0_stateless/01236_graphite_mt.sql index 035fdf6af69..88d2d0ccb63 100644 --- a/tests/queries/0_stateless/01236_graphite_mt.sql +++ b/tests/queries/0_stateless/01236_graphite_mt.sql @@ -1,26 +1,26 @@ drop table if exists test_graphite; -create table test_graphite (key UInt32, Path String, Time DateTime('UTC'), Value Float64, Version UInt32, col UInt64) +create table test_graphite (key UInt32, Path String, Time DateTime, Value Float64, Version UInt32, col UInt64) engine = GraphiteMergeTree('graphite_rollup') order by key settings index_granularity=10; insert into test_graphite -select 1, 'sum_1', toDateTime(today(), 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all -select 2, 'sum_1', toDateTime(today(), 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all -select 1, 'sum_2', toDateTime(today(), 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all -select 2, 'sum_2', toDateTime(today(), 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all -select 1, 'max_1', toDateTime(today(), 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all -select 2, 'max_1', toDateTime(today(), 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all -select 1, 'max_2', toDateTime(today(), 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all -select 2, 'max_2', toDateTime(today(), 'UTC') - number * 60 - 30, number, 1, number from numbers(300); +select 1, 'sum_1', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300) union all +select 2, 'sum_1', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300) union all +select 1, 'sum_2', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300) union all +select 2, 'sum_2', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300) union all +select 1, 'max_1', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300) union all +select 2, 'max_1', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300) union all +select 1, 'max_2', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300) union all +select 2, 'max_2', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300); insert into test_graphite -select 1, 'sum_1', toDateTime(today() -3 , 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all -select 2, 'sum_1', toDateTime(today() -3 , 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all -select 1, 'sum_2', toDateTime(today() -3 , 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all -select 2, 'sum_2', toDateTime(today() -3 , 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all -select 1, 'max_1', toDateTime(today() -3 , 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all -select 2, 'max_1', toDateTime(today() -3 , 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all -select 1, 'max_2', toDateTime(today() -3 , 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all -select 2, 'max_2', toDateTime(today() -3 , 'UTC') - number * 60 - 30, number, 1, number from numbers(1200); +select 1, 'sum_1', toDateTime(today() - 3) - number * 60 - 30, number, 1, number from numbers(1200) union all +select 2, 'sum_1', toDateTime(today() - 3) - number * 60 - 30, number, 1, number from numbers(1200) union all +select 1, 'sum_2', toDateTime(today() - 3) - number * 60 - 30, number, 1, number from numbers(1200) union all +select 2, 'sum_2', toDateTime(today() - 3) - number * 60 - 30, number, 1, number from numbers(1200) union all +select 1, 'max_1', toDateTime(today() - 3) - number * 60 - 30, number, 1, number from numbers(1200) union all +select 2, 'max_1', toDateTime(today() - 3) - number * 60 - 30, number, 1, number from numbers(1200) union all +select 1, 'max_2', toDateTime(today() - 3) - number * 60 - 30, number, 1, number from numbers(1200) union all +select 2, 'max_2', toDateTime(today() - 3) - number * 60 - 30, number, 1, number from numbers(1200); optimize table test_graphite final; From ee416e79bdaf28dc668ab7a5c593aeb9c32f948f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 10 Aug 2021 17:55:02 +0200 Subject: [PATCH 060/161] 01236_graphite_mt: Use static date and print dates in results --- .../0_stateless/01236_graphite_mt.reference | 688 +++++++++--------- .../queries/0_stateless/01236_graphite_mt.sql | 38 +- 2 files changed, 364 insertions(+), 362 deletions(-) diff --git a/tests/queries/0_stateless/01236_graphite_mt.reference b/tests/queries/0_stateless/01236_graphite_mt.reference index a30d2495265..c68becad53d 100644 --- a/tests/queries/0_stateless/01236_graphite_mt.reference +++ b/tests/queries/0_stateless/01236_graphite_mt.reference @@ -1,344 +1,344 @@ -1 max_1 9 1 0 -1 max_1 19 1 10 -1 max_1 29 1 20 -1 max_1 39 1 30 -1 max_1 49 1 40 -1 max_1 59 1 50 -1 max_1 69 1 60 -1 max_1 79 1 70 -1 max_1 89 1 80 -1 max_1 99 1 90 -1 max_1 109 1 100 -1 max_1 119 1 110 -1 max_1 129 1 120 -1 max_1 139 1 130 -1 max_1 149 1 140 -1 max_1 159 1 150 -1 max_1 169 1 160 -1 max_1 179 1 170 -1 max_1 189 1 180 -1 max_1 199 1 190 -1 max_1 209 1 200 -1 max_1 219 1 210 -1 max_1 229 1 220 -1 max_1 239 1 230 -1 max_1 249 1 240 -1 max_1 259 1 250 -1 max_1 269 1 260 -1 max_1 279 1 270 -1 max_1 289 1 280 -1 max_1 299 1 290 -1 max_1 39 1 0 -1 max_1 139 1 40 -1 max_1 239 1 140 -1 max_1 339 1 240 -1 max_1 439 1 340 -1 max_1 539 1 440 -1 max_1 639 1 540 -1 max_1 739 1 640 -1 max_1 839 1 740 -1 max_1 939 1 840 -1 max_1 1039 1 940 -1 max_1 1139 1 1040 -1 max_1 1199 1 1140 -1 max_2 9 1 0 -1 max_2 19 1 10 -1 max_2 29 1 20 -1 max_2 39 1 30 -1 max_2 49 1 40 -1 max_2 59 1 50 -1 max_2 69 1 60 -1 max_2 79 1 70 -1 max_2 89 1 80 -1 max_2 99 1 90 -1 max_2 109 1 100 -1 max_2 119 1 110 -1 max_2 129 1 120 -1 max_2 139 1 130 -1 max_2 149 1 140 -1 max_2 159 1 150 -1 max_2 169 1 160 -1 max_2 179 1 170 -1 max_2 189 1 180 -1 max_2 199 1 190 -1 max_2 209 1 200 -1 max_2 219 1 210 -1 max_2 229 1 220 -1 max_2 239 1 230 -1 max_2 249 1 240 -1 max_2 259 1 250 -1 max_2 269 1 260 -1 max_2 279 1 270 -1 max_2 289 1 280 -1 max_2 299 1 290 -1 max_2 39 1 0 -1 max_2 139 1 40 -1 max_2 239 1 140 -1 max_2 339 1 240 -1 max_2 439 1 340 -1 max_2 539 1 440 -1 max_2 639 1 540 -1 max_2 739 1 640 -1 max_2 839 1 740 -1 max_2 939 1 840 -1 max_2 1039 1 940 -1 max_2 1139 1 1040 -1 max_2 1199 1 1140 -1 sum_1 45 1 0 -1 sum_1 145 1 10 -1 sum_1 245 1 20 -1 sum_1 345 1 30 -1 sum_1 445 1 40 -1 sum_1 545 1 50 -1 sum_1 645 1 60 -1 sum_1 745 1 70 -1 sum_1 845 1 80 -1 sum_1 945 1 90 -1 sum_1 1045 1 100 -1 sum_1 1145 1 110 -1 sum_1 1245 1 120 -1 sum_1 1345 1 130 -1 sum_1 1445 1 140 -1 sum_1 1545 1 150 -1 sum_1 1645 1 160 -1 sum_1 1745 1 170 -1 sum_1 1845 1 180 -1 sum_1 1945 1 190 -1 sum_1 2045 1 200 -1 sum_1 2145 1 210 -1 sum_1 2245 1 220 -1 sum_1 2345 1 230 -1 sum_1 2445 1 240 -1 sum_1 2545 1 250 -1 sum_1 2645 1 260 -1 sum_1 2745 1 270 -1 sum_1 2845 1 280 -1 sum_1 2945 1 290 -1 sum_1 780 1 0 -1 sum_1 8950 1 40 -1 sum_1 18950 1 140 -1 sum_1 28950 1 240 -1 sum_1 38950 1 340 -1 sum_1 48950 1 440 -1 sum_1 58950 1 540 -1 sum_1 68950 1 640 -1 sum_1 78950 1 740 -1 sum_1 88950 1 840 -1 sum_1 98950 1 940 -1 sum_1 108950 1 1040 -1 sum_1 70170 1 1140 -1 sum_2 45 1 0 -1 sum_2 145 1 10 -1 sum_2 245 1 20 -1 sum_2 345 1 30 -1 sum_2 445 1 40 -1 sum_2 545 1 50 -1 sum_2 645 1 60 -1 sum_2 745 1 70 -1 sum_2 845 1 80 -1 sum_2 945 1 90 -1 sum_2 1045 1 100 -1 sum_2 1145 1 110 -1 sum_2 1245 1 120 -1 sum_2 1345 1 130 -1 sum_2 1445 1 140 -1 sum_2 1545 1 150 -1 sum_2 1645 1 160 -1 sum_2 1745 1 170 -1 sum_2 1845 1 180 -1 sum_2 1945 1 190 -1 sum_2 2045 1 200 -1 sum_2 2145 1 210 -1 sum_2 2245 1 220 -1 sum_2 2345 1 230 -1 sum_2 2445 1 240 -1 sum_2 2545 1 250 -1 sum_2 2645 1 260 -1 sum_2 2745 1 270 -1 sum_2 2845 1 280 -1 sum_2 2945 1 290 -1 sum_2 780 1 0 -1 sum_2 8950 1 40 -1 sum_2 18950 1 140 -1 sum_2 28950 1 240 -1 sum_2 38950 1 340 -1 sum_2 48950 1 440 -1 sum_2 58950 1 540 -1 sum_2 68950 1 640 -1 sum_2 78950 1 740 -1 sum_2 88950 1 840 -1 sum_2 98950 1 940 -1 sum_2 108950 1 1040 -1 sum_2 70170 1 1140 -2 max_1 9 1 0 -2 max_1 19 1 10 -2 max_1 29 1 20 -2 max_1 39 1 30 -2 max_1 49 1 40 -2 max_1 59 1 50 -2 max_1 69 1 60 -2 max_1 79 1 70 -2 max_1 89 1 80 -2 max_1 99 1 90 -2 max_1 109 1 100 -2 max_1 119 1 110 -2 max_1 129 1 120 -2 max_1 139 1 130 -2 max_1 149 1 140 -2 max_1 159 1 150 -2 max_1 169 1 160 -2 max_1 179 1 170 -2 max_1 189 1 180 -2 max_1 199 1 190 -2 max_1 209 1 200 -2 max_1 219 1 210 -2 max_1 229 1 220 -2 max_1 239 1 230 -2 max_1 249 1 240 -2 max_1 259 1 250 -2 max_1 269 1 260 -2 max_1 279 1 270 -2 max_1 289 1 280 -2 max_1 299 1 290 -2 max_1 39 1 0 -2 max_1 139 1 40 -2 max_1 239 1 140 -2 max_1 339 1 240 -2 max_1 439 1 340 -2 max_1 539 1 440 -2 max_1 639 1 540 -2 max_1 739 1 640 -2 max_1 839 1 740 -2 max_1 939 1 840 -2 max_1 1039 1 940 -2 max_1 1139 1 1040 -2 max_1 1199 1 1140 -2 max_2 9 1 0 -2 max_2 19 1 10 -2 max_2 29 1 20 -2 max_2 39 1 30 -2 max_2 49 1 40 -2 max_2 59 1 50 -2 max_2 69 1 60 -2 max_2 79 1 70 -2 max_2 89 1 80 -2 max_2 99 1 90 -2 max_2 109 1 100 -2 max_2 119 1 110 -2 max_2 129 1 120 -2 max_2 139 1 130 -2 max_2 149 1 140 -2 max_2 159 1 150 -2 max_2 169 1 160 -2 max_2 179 1 170 -2 max_2 189 1 180 -2 max_2 199 1 190 -2 max_2 209 1 200 -2 max_2 219 1 210 -2 max_2 229 1 220 -2 max_2 239 1 230 -2 max_2 249 1 240 -2 max_2 259 1 250 -2 max_2 269 1 260 -2 max_2 279 1 270 -2 max_2 289 1 280 -2 max_2 299 1 290 -2 max_2 39 1 0 -2 max_2 139 1 40 -2 max_2 239 1 140 -2 max_2 339 1 240 -2 max_2 439 1 340 -2 max_2 539 1 440 -2 max_2 639 1 540 -2 max_2 739 1 640 -2 max_2 839 1 740 -2 max_2 939 1 840 -2 max_2 1039 1 940 -2 max_2 1139 1 1040 -2 max_2 1199 1 1140 -2 sum_1 45 1 0 -2 sum_1 145 1 10 -2 sum_1 245 1 20 -2 sum_1 345 1 30 -2 sum_1 445 1 40 -2 sum_1 545 1 50 -2 sum_1 645 1 60 -2 sum_1 745 1 70 -2 sum_1 845 1 80 -2 sum_1 945 1 90 -2 sum_1 1045 1 100 -2 sum_1 1145 1 110 -2 sum_1 1245 1 120 -2 sum_1 1345 1 130 -2 sum_1 1445 1 140 -2 sum_1 1545 1 150 -2 sum_1 1645 1 160 -2 sum_1 1745 1 170 -2 sum_1 1845 1 180 -2 sum_1 1945 1 190 -2 sum_1 2045 1 200 -2 sum_1 2145 1 210 -2 sum_1 2245 1 220 -2 sum_1 2345 1 230 -2 sum_1 2445 1 240 -2 sum_1 2545 1 250 -2 sum_1 2645 1 260 -2 sum_1 2745 1 270 -2 sum_1 2845 1 280 -2 sum_1 2945 1 290 -2 sum_1 780 1 0 -2 sum_1 8950 1 40 -2 sum_1 18950 1 140 -2 sum_1 28950 1 240 -2 sum_1 38950 1 340 -2 sum_1 48950 1 440 -2 sum_1 58950 1 540 -2 sum_1 68950 1 640 -2 sum_1 78950 1 740 -2 sum_1 88950 1 840 -2 sum_1 98950 1 940 -2 sum_1 108950 1 1040 -2 sum_1 70170 1 1140 -2 sum_2 45 1 0 -2 sum_2 145 1 10 -2 sum_2 245 1 20 -2 sum_2 345 1 30 -2 sum_2 445 1 40 -2 sum_2 545 1 50 -2 sum_2 645 1 60 -2 sum_2 745 1 70 -2 sum_2 845 1 80 -2 sum_2 945 1 90 -2 sum_2 1045 1 100 -2 sum_2 1145 1 110 -2 sum_2 1245 1 120 -2 sum_2 1345 1 130 -2 sum_2 1445 1 140 -2 sum_2 1545 1 150 -2 sum_2 1645 1 160 -2 sum_2 1745 1 170 -2 sum_2 1845 1 180 -2 sum_2 1945 1 190 -2 sum_2 2045 1 200 -2 sum_2 2145 1 210 -2 sum_2 2245 1 220 -2 sum_2 2345 1 230 -2 sum_2 2445 1 240 -2 sum_2 2545 1 250 -2 sum_2 2645 1 260 -2 sum_2 2745 1 270 -2 sum_2 2845 1 280 -2 sum_2 2945 1 290 -2 sum_2 780 1 0 -2 sum_2 8950 1 40 -2 sum_2 18950 1 140 -2 sum_2 28950 1 240 -2 sum_2 38950 1 340 -2 sum_2 48950 1 440 -2 sum_2 58950 1 540 -2 sum_2 68950 1 640 -2 sum_2 78950 1 740 -2 sum_2 88950 1 840 -2 sum_2 98950 1 940 -2 sum_2 108950 1 1040 -2 sum_2 70170 1 1140 +1 max_1 2021-08-09 23:50:00 9 1 0 +1 max_1 2021-08-09 23:40:00 19 1 10 +1 max_1 2021-08-09 23:30:00 29 1 20 +1 max_1 2021-08-09 23:20:00 39 1 30 +1 max_1 2021-08-09 23:10:00 49 1 40 +1 max_1 2021-08-09 23:00:00 59 1 50 +1 max_1 2021-08-09 22:50:00 69 1 60 +1 max_1 2021-08-09 22:40:00 79 1 70 +1 max_1 2021-08-09 22:30:00 89 1 80 +1 max_1 2021-08-09 22:20:00 99 1 90 +1 max_1 2021-08-09 22:10:00 109 1 100 +1 max_1 2021-08-09 22:00:00 119 1 110 +1 max_1 2021-08-09 21:50:00 129 1 120 +1 max_1 2021-08-09 21:40:00 139 1 130 +1 max_1 2021-08-09 21:30:00 149 1 140 +1 max_1 2021-08-09 21:20:00 159 1 150 +1 max_1 2021-08-09 21:10:00 169 1 160 +1 max_1 2021-08-09 21:00:00 179 1 170 +1 max_1 2021-08-09 20:50:00 189 1 180 +1 max_1 2021-08-09 20:40:00 199 1 190 +1 max_1 2021-08-09 20:30:00 209 1 200 +1 max_1 2021-08-09 20:20:00 219 1 210 +1 max_1 2021-08-09 20:10:00 229 1 220 +1 max_1 2021-08-09 20:00:00 239 1 230 +1 max_1 2021-08-09 19:50:00 249 1 240 +1 max_1 2021-08-09 19:40:00 259 1 250 +1 max_1 2021-08-09 19:30:00 269 1 260 +1 max_1 2021-08-09 19:20:00 279 1 270 +1 max_1 2021-08-09 19:10:00 289 1 280 +1 max_1 2021-08-09 19:00:00 299 1 290 +1 max_1 2021-08-06 23:20:00 39 1 0 +1 max_1 2021-08-06 21:40:00 139 1 40 +1 max_1 2021-08-06 20:00:00 239 1 140 +1 max_1 2021-08-06 18:20:00 339 1 240 +1 max_1 2021-08-06 16:40:00 439 1 340 +1 max_1 2021-08-06 15:00:00 539 1 440 +1 max_1 2021-08-06 13:20:00 639 1 540 +1 max_1 2021-08-06 11:40:00 739 1 640 +1 max_1 2021-08-06 10:00:00 839 1 740 +1 max_1 2021-08-06 08:20:00 939 1 840 +1 max_1 2021-08-06 06:40:00 1039 1 940 +1 max_1 2021-08-06 05:00:00 1139 1 1040 +1 max_1 2021-08-06 03:20:00 1199 1 1140 +1 max_2 2021-08-09 23:50:00 9 1 0 +1 max_2 2021-08-09 23:40:00 19 1 10 +1 max_2 2021-08-09 23:30:00 29 1 20 +1 max_2 2021-08-09 23:20:00 39 1 30 +1 max_2 2021-08-09 23:10:00 49 1 40 +1 max_2 2021-08-09 23:00:00 59 1 50 +1 max_2 2021-08-09 22:50:00 69 1 60 +1 max_2 2021-08-09 22:40:00 79 1 70 +1 max_2 2021-08-09 22:30:00 89 1 80 +1 max_2 2021-08-09 22:20:00 99 1 90 +1 max_2 2021-08-09 22:10:00 109 1 100 +1 max_2 2021-08-09 22:00:00 119 1 110 +1 max_2 2021-08-09 21:50:00 129 1 120 +1 max_2 2021-08-09 21:40:00 139 1 130 +1 max_2 2021-08-09 21:30:00 149 1 140 +1 max_2 2021-08-09 21:20:00 159 1 150 +1 max_2 2021-08-09 21:10:00 169 1 160 +1 max_2 2021-08-09 21:00:00 179 1 170 +1 max_2 2021-08-09 20:50:00 189 1 180 +1 max_2 2021-08-09 20:40:00 199 1 190 +1 max_2 2021-08-09 20:30:00 209 1 200 +1 max_2 2021-08-09 20:20:00 219 1 210 +1 max_2 2021-08-09 20:10:00 229 1 220 +1 max_2 2021-08-09 20:00:00 239 1 230 +1 max_2 2021-08-09 19:50:00 249 1 240 +1 max_2 2021-08-09 19:40:00 259 1 250 +1 max_2 2021-08-09 19:30:00 269 1 260 +1 max_2 2021-08-09 19:20:00 279 1 270 +1 max_2 2021-08-09 19:10:00 289 1 280 +1 max_2 2021-08-09 19:00:00 299 1 290 +1 max_2 2021-08-06 23:20:00 39 1 0 +1 max_2 2021-08-06 21:40:00 139 1 40 +1 max_2 2021-08-06 20:00:00 239 1 140 +1 max_2 2021-08-06 18:20:00 339 1 240 +1 max_2 2021-08-06 16:40:00 439 1 340 +1 max_2 2021-08-06 15:00:00 539 1 440 +1 max_2 2021-08-06 13:20:00 639 1 540 +1 max_2 2021-08-06 11:40:00 739 1 640 +1 max_2 2021-08-06 10:00:00 839 1 740 +1 max_2 2021-08-06 08:20:00 939 1 840 +1 max_2 2021-08-06 06:40:00 1039 1 940 +1 max_2 2021-08-06 05:00:00 1139 1 1040 +1 max_2 2021-08-06 03:20:00 1199 1 1140 +1 sum_1 2021-08-09 23:50:00 45 1 0 +1 sum_1 2021-08-09 23:40:00 145 1 10 +1 sum_1 2021-08-09 23:30:00 245 1 20 +1 sum_1 2021-08-09 23:20:00 345 1 30 +1 sum_1 2021-08-09 23:10:00 445 1 40 +1 sum_1 2021-08-09 23:00:00 545 1 50 +1 sum_1 2021-08-09 22:50:00 645 1 60 +1 sum_1 2021-08-09 22:40:00 745 1 70 +1 sum_1 2021-08-09 22:30:00 845 1 80 +1 sum_1 2021-08-09 22:20:00 945 1 90 +1 sum_1 2021-08-09 22:10:00 1045 1 100 +1 sum_1 2021-08-09 22:00:00 1145 1 110 +1 sum_1 2021-08-09 21:50:00 1245 1 120 +1 sum_1 2021-08-09 21:40:00 1345 1 130 +1 sum_1 2021-08-09 21:30:00 1445 1 140 +1 sum_1 2021-08-09 21:20:00 1545 1 150 +1 sum_1 2021-08-09 21:10:00 1645 1 160 +1 sum_1 2021-08-09 21:00:00 1745 1 170 +1 sum_1 2021-08-09 20:50:00 1845 1 180 +1 sum_1 2021-08-09 20:40:00 1945 1 190 +1 sum_1 2021-08-09 20:30:00 2045 1 200 +1 sum_1 2021-08-09 20:20:00 2145 1 210 +1 sum_1 2021-08-09 20:10:00 2245 1 220 +1 sum_1 2021-08-09 20:00:00 2345 1 230 +1 sum_1 2021-08-09 19:50:00 2445 1 240 +1 sum_1 2021-08-09 19:40:00 2545 1 250 +1 sum_1 2021-08-09 19:30:00 2645 1 260 +1 sum_1 2021-08-09 19:20:00 2745 1 270 +1 sum_1 2021-08-09 19:10:00 2845 1 280 +1 sum_1 2021-08-09 19:00:00 2945 1 290 +1 sum_1 2021-08-06 23:20:00 780 1 0 +1 sum_1 2021-08-06 21:40:00 8950 1 40 +1 sum_1 2021-08-06 20:00:00 18950 1 140 +1 sum_1 2021-08-06 18:20:00 28950 1 240 +1 sum_1 2021-08-06 16:40:00 38950 1 340 +1 sum_1 2021-08-06 15:00:00 48950 1 440 +1 sum_1 2021-08-06 13:20:00 58950 1 540 +1 sum_1 2021-08-06 11:40:00 68950 1 640 +1 sum_1 2021-08-06 10:00:00 78950 1 740 +1 sum_1 2021-08-06 08:20:00 88950 1 840 +1 sum_1 2021-08-06 06:40:00 98950 1 940 +1 sum_1 2021-08-06 05:00:00 108950 1 1040 +1 sum_1 2021-08-06 03:20:00 70170 1 1140 +1 sum_2 2021-08-09 23:50:00 45 1 0 +1 sum_2 2021-08-09 23:40:00 145 1 10 +1 sum_2 2021-08-09 23:30:00 245 1 20 +1 sum_2 2021-08-09 23:20:00 345 1 30 +1 sum_2 2021-08-09 23:10:00 445 1 40 +1 sum_2 2021-08-09 23:00:00 545 1 50 +1 sum_2 2021-08-09 22:50:00 645 1 60 +1 sum_2 2021-08-09 22:40:00 745 1 70 +1 sum_2 2021-08-09 22:30:00 845 1 80 +1 sum_2 2021-08-09 22:20:00 945 1 90 +1 sum_2 2021-08-09 22:10:00 1045 1 100 +1 sum_2 2021-08-09 22:00:00 1145 1 110 +1 sum_2 2021-08-09 21:50:00 1245 1 120 +1 sum_2 2021-08-09 21:40:00 1345 1 130 +1 sum_2 2021-08-09 21:30:00 1445 1 140 +1 sum_2 2021-08-09 21:20:00 1545 1 150 +1 sum_2 2021-08-09 21:10:00 1645 1 160 +1 sum_2 2021-08-09 21:00:00 1745 1 170 +1 sum_2 2021-08-09 20:50:00 1845 1 180 +1 sum_2 2021-08-09 20:40:00 1945 1 190 +1 sum_2 2021-08-09 20:30:00 2045 1 200 +1 sum_2 2021-08-09 20:20:00 2145 1 210 +1 sum_2 2021-08-09 20:10:00 2245 1 220 +1 sum_2 2021-08-09 20:00:00 2345 1 230 +1 sum_2 2021-08-09 19:50:00 2445 1 240 +1 sum_2 2021-08-09 19:40:00 2545 1 250 +1 sum_2 2021-08-09 19:30:00 2645 1 260 +1 sum_2 2021-08-09 19:20:00 2745 1 270 +1 sum_2 2021-08-09 19:10:00 2845 1 280 +1 sum_2 2021-08-09 19:00:00 2945 1 290 +1 sum_2 2021-08-06 23:20:00 780 1 0 +1 sum_2 2021-08-06 21:40:00 8950 1 40 +1 sum_2 2021-08-06 20:00:00 18950 1 140 +1 sum_2 2021-08-06 18:20:00 28950 1 240 +1 sum_2 2021-08-06 16:40:00 38950 1 340 +1 sum_2 2021-08-06 15:00:00 48950 1 440 +1 sum_2 2021-08-06 13:20:00 58950 1 540 +1 sum_2 2021-08-06 11:40:00 68950 1 640 +1 sum_2 2021-08-06 10:00:00 78950 1 740 +1 sum_2 2021-08-06 08:20:00 88950 1 840 +1 sum_2 2021-08-06 06:40:00 98950 1 940 +1 sum_2 2021-08-06 05:00:00 108950 1 1040 +1 sum_2 2021-08-06 03:20:00 70170 1 1140 +2 max_1 2021-08-09 23:50:00 9 1 0 +2 max_1 2021-08-09 23:40:00 19 1 10 +2 max_1 2021-08-09 23:30:00 29 1 20 +2 max_1 2021-08-09 23:20:00 39 1 30 +2 max_1 2021-08-09 23:10:00 49 1 40 +2 max_1 2021-08-09 23:00:00 59 1 50 +2 max_1 2021-08-09 22:50:00 69 1 60 +2 max_1 2021-08-09 22:40:00 79 1 70 +2 max_1 2021-08-09 22:30:00 89 1 80 +2 max_1 2021-08-09 22:20:00 99 1 90 +2 max_1 2021-08-09 22:10:00 109 1 100 +2 max_1 2021-08-09 22:00:00 119 1 110 +2 max_1 2021-08-09 21:50:00 129 1 120 +2 max_1 2021-08-09 21:40:00 139 1 130 +2 max_1 2021-08-09 21:30:00 149 1 140 +2 max_1 2021-08-09 21:20:00 159 1 150 +2 max_1 2021-08-09 21:10:00 169 1 160 +2 max_1 2021-08-09 21:00:00 179 1 170 +2 max_1 2021-08-09 20:50:00 189 1 180 +2 max_1 2021-08-09 20:40:00 199 1 190 +2 max_1 2021-08-09 20:30:00 209 1 200 +2 max_1 2021-08-09 20:20:00 219 1 210 +2 max_1 2021-08-09 20:10:00 229 1 220 +2 max_1 2021-08-09 20:00:00 239 1 230 +2 max_1 2021-08-09 19:50:00 249 1 240 +2 max_1 2021-08-09 19:40:00 259 1 250 +2 max_1 2021-08-09 19:30:00 269 1 260 +2 max_1 2021-08-09 19:20:00 279 1 270 +2 max_1 2021-08-09 19:10:00 289 1 280 +2 max_1 2021-08-09 19:00:00 299 1 290 +2 max_1 2021-08-06 23:20:00 39 1 0 +2 max_1 2021-08-06 21:40:00 139 1 40 +2 max_1 2021-08-06 20:00:00 239 1 140 +2 max_1 2021-08-06 18:20:00 339 1 240 +2 max_1 2021-08-06 16:40:00 439 1 340 +2 max_1 2021-08-06 15:00:00 539 1 440 +2 max_1 2021-08-06 13:20:00 639 1 540 +2 max_1 2021-08-06 11:40:00 739 1 640 +2 max_1 2021-08-06 10:00:00 839 1 740 +2 max_1 2021-08-06 08:20:00 939 1 840 +2 max_1 2021-08-06 06:40:00 1039 1 940 +2 max_1 2021-08-06 05:00:00 1139 1 1040 +2 max_1 2021-08-06 03:20:00 1199 1 1140 +2 max_2 2021-08-09 23:50:00 9 1 0 +2 max_2 2021-08-09 23:40:00 19 1 10 +2 max_2 2021-08-09 23:30:00 29 1 20 +2 max_2 2021-08-09 23:20:00 39 1 30 +2 max_2 2021-08-09 23:10:00 49 1 40 +2 max_2 2021-08-09 23:00:00 59 1 50 +2 max_2 2021-08-09 22:50:00 69 1 60 +2 max_2 2021-08-09 22:40:00 79 1 70 +2 max_2 2021-08-09 22:30:00 89 1 80 +2 max_2 2021-08-09 22:20:00 99 1 90 +2 max_2 2021-08-09 22:10:00 109 1 100 +2 max_2 2021-08-09 22:00:00 119 1 110 +2 max_2 2021-08-09 21:50:00 129 1 120 +2 max_2 2021-08-09 21:40:00 139 1 130 +2 max_2 2021-08-09 21:30:00 149 1 140 +2 max_2 2021-08-09 21:20:00 159 1 150 +2 max_2 2021-08-09 21:10:00 169 1 160 +2 max_2 2021-08-09 21:00:00 179 1 170 +2 max_2 2021-08-09 20:50:00 189 1 180 +2 max_2 2021-08-09 20:40:00 199 1 190 +2 max_2 2021-08-09 20:30:00 209 1 200 +2 max_2 2021-08-09 20:20:00 219 1 210 +2 max_2 2021-08-09 20:10:00 229 1 220 +2 max_2 2021-08-09 20:00:00 239 1 230 +2 max_2 2021-08-09 19:50:00 249 1 240 +2 max_2 2021-08-09 19:40:00 259 1 250 +2 max_2 2021-08-09 19:30:00 269 1 260 +2 max_2 2021-08-09 19:20:00 279 1 270 +2 max_2 2021-08-09 19:10:00 289 1 280 +2 max_2 2021-08-09 19:00:00 299 1 290 +2 max_2 2021-08-06 23:20:00 39 1 0 +2 max_2 2021-08-06 21:40:00 139 1 40 +2 max_2 2021-08-06 20:00:00 239 1 140 +2 max_2 2021-08-06 18:20:00 339 1 240 +2 max_2 2021-08-06 16:40:00 439 1 340 +2 max_2 2021-08-06 15:00:00 539 1 440 +2 max_2 2021-08-06 13:20:00 639 1 540 +2 max_2 2021-08-06 11:40:00 739 1 640 +2 max_2 2021-08-06 10:00:00 839 1 740 +2 max_2 2021-08-06 08:20:00 939 1 840 +2 max_2 2021-08-06 06:40:00 1039 1 940 +2 max_2 2021-08-06 05:00:00 1139 1 1040 +2 max_2 2021-08-06 03:20:00 1199 1 1140 +2 sum_1 2021-08-09 23:50:00 45 1 0 +2 sum_1 2021-08-09 23:40:00 145 1 10 +2 sum_1 2021-08-09 23:30:00 245 1 20 +2 sum_1 2021-08-09 23:20:00 345 1 30 +2 sum_1 2021-08-09 23:10:00 445 1 40 +2 sum_1 2021-08-09 23:00:00 545 1 50 +2 sum_1 2021-08-09 22:50:00 645 1 60 +2 sum_1 2021-08-09 22:40:00 745 1 70 +2 sum_1 2021-08-09 22:30:00 845 1 80 +2 sum_1 2021-08-09 22:20:00 945 1 90 +2 sum_1 2021-08-09 22:10:00 1045 1 100 +2 sum_1 2021-08-09 22:00:00 1145 1 110 +2 sum_1 2021-08-09 21:50:00 1245 1 120 +2 sum_1 2021-08-09 21:40:00 1345 1 130 +2 sum_1 2021-08-09 21:30:00 1445 1 140 +2 sum_1 2021-08-09 21:20:00 1545 1 150 +2 sum_1 2021-08-09 21:10:00 1645 1 160 +2 sum_1 2021-08-09 21:00:00 1745 1 170 +2 sum_1 2021-08-09 20:50:00 1845 1 180 +2 sum_1 2021-08-09 20:40:00 1945 1 190 +2 sum_1 2021-08-09 20:30:00 2045 1 200 +2 sum_1 2021-08-09 20:20:00 2145 1 210 +2 sum_1 2021-08-09 20:10:00 2245 1 220 +2 sum_1 2021-08-09 20:00:00 2345 1 230 +2 sum_1 2021-08-09 19:50:00 2445 1 240 +2 sum_1 2021-08-09 19:40:00 2545 1 250 +2 sum_1 2021-08-09 19:30:00 2645 1 260 +2 sum_1 2021-08-09 19:20:00 2745 1 270 +2 sum_1 2021-08-09 19:10:00 2845 1 280 +2 sum_1 2021-08-09 19:00:00 2945 1 290 +2 sum_1 2021-08-06 23:20:00 780 1 0 +2 sum_1 2021-08-06 21:40:00 8950 1 40 +2 sum_1 2021-08-06 20:00:00 18950 1 140 +2 sum_1 2021-08-06 18:20:00 28950 1 240 +2 sum_1 2021-08-06 16:40:00 38950 1 340 +2 sum_1 2021-08-06 15:00:00 48950 1 440 +2 sum_1 2021-08-06 13:20:00 58950 1 540 +2 sum_1 2021-08-06 11:40:00 68950 1 640 +2 sum_1 2021-08-06 10:00:00 78950 1 740 +2 sum_1 2021-08-06 08:20:00 88950 1 840 +2 sum_1 2021-08-06 06:40:00 98950 1 940 +2 sum_1 2021-08-06 05:00:00 108950 1 1040 +2 sum_1 2021-08-06 03:20:00 70170 1 1140 +2 sum_2 2021-08-09 23:50:00 45 1 0 +2 sum_2 2021-08-09 23:40:00 145 1 10 +2 sum_2 2021-08-09 23:30:00 245 1 20 +2 sum_2 2021-08-09 23:20:00 345 1 30 +2 sum_2 2021-08-09 23:10:00 445 1 40 +2 sum_2 2021-08-09 23:00:00 545 1 50 +2 sum_2 2021-08-09 22:50:00 645 1 60 +2 sum_2 2021-08-09 22:40:00 745 1 70 +2 sum_2 2021-08-09 22:30:00 845 1 80 +2 sum_2 2021-08-09 22:20:00 945 1 90 +2 sum_2 2021-08-09 22:10:00 1045 1 100 +2 sum_2 2021-08-09 22:00:00 1145 1 110 +2 sum_2 2021-08-09 21:50:00 1245 1 120 +2 sum_2 2021-08-09 21:40:00 1345 1 130 +2 sum_2 2021-08-09 21:30:00 1445 1 140 +2 sum_2 2021-08-09 21:20:00 1545 1 150 +2 sum_2 2021-08-09 21:10:00 1645 1 160 +2 sum_2 2021-08-09 21:00:00 1745 1 170 +2 sum_2 2021-08-09 20:50:00 1845 1 180 +2 sum_2 2021-08-09 20:40:00 1945 1 190 +2 sum_2 2021-08-09 20:30:00 2045 1 200 +2 sum_2 2021-08-09 20:20:00 2145 1 210 +2 sum_2 2021-08-09 20:10:00 2245 1 220 +2 sum_2 2021-08-09 20:00:00 2345 1 230 +2 sum_2 2021-08-09 19:50:00 2445 1 240 +2 sum_2 2021-08-09 19:40:00 2545 1 250 +2 sum_2 2021-08-09 19:30:00 2645 1 260 +2 sum_2 2021-08-09 19:20:00 2745 1 270 +2 sum_2 2021-08-09 19:10:00 2845 1 280 +2 sum_2 2021-08-09 19:00:00 2945 1 290 +2 sum_2 2021-08-06 23:20:00 780 1 0 +2 sum_2 2021-08-06 21:40:00 8950 1 40 +2 sum_2 2021-08-06 20:00:00 18950 1 140 +2 sum_2 2021-08-06 18:20:00 28950 1 240 +2 sum_2 2021-08-06 16:40:00 38950 1 340 +2 sum_2 2021-08-06 15:00:00 48950 1 440 +2 sum_2 2021-08-06 13:20:00 58950 1 540 +2 sum_2 2021-08-06 11:40:00 68950 1 640 +2 sum_2 2021-08-06 10:00:00 78950 1 740 +2 sum_2 2021-08-06 08:20:00 88950 1 840 +2 sum_2 2021-08-06 06:40:00 98950 1 940 +2 sum_2 2021-08-06 05:00:00 108950 1 1040 +2 sum_2 2021-08-06 03:20:00 70170 1 1140 diff --git a/tests/queries/0_stateless/01236_graphite_mt.sql b/tests/queries/0_stateless/01236_graphite_mt.sql index 88d2d0ccb63..6f4af186a1c 100644 --- a/tests/queries/0_stateless/01236_graphite_mt.sql +++ b/tests/queries/0_stateless/01236_graphite_mt.sql @@ -1,29 +1,31 @@ + +-- Use DateTime('UTC') to have a common rollup window drop table if exists test_graphite; -create table test_graphite (key UInt32, Path String, Time DateTime, Value Float64, Version UInt32, col UInt64) +create table test_graphite (key UInt32, Path String, Time DateTime('UTC'), Value Float64, Version UInt32, col UInt64) engine = GraphiteMergeTree('graphite_rollup') order by key settings index_granularity=10; insert into test_graphite -select 1, 'sum_1', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300) union all -select 2, 'sum_1', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300) union all -select 1, 'sum_2', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300) union all -select 2, 'sum_2', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300) union all -select 1, 'max_1', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300) union all -select 2, 'max_1', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300) union all -select 1, 'max_2', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300) union all -select 2, 'max_2', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300); +select 1, 'sum_1', toDateTime('2021-08-10', 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all +select 2, 'sum_1', toDateTime('2021-08-10', 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all +select 1, 'sum_2', toDateTime('2021-08-10', 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all +select 2, 'sum_2', toDateTime('2021-08-10', 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all +select 1, 'max_1', toDateTime('2021-08-10', 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all +select 2, 'max_1', toDateTime('2021-08-10', 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all +select 1, 'max_2', toDateTime('2021-08-10', 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all +select 2, 'max_2', toDateTime('2021-08-10', 'UTC') - number * 60 - 30, number, 1, number from numbers(300); insert into test_graphite -select 1, 'sum_1', toDateTime(today() - 3) - number * 60 - 30, number, 1, number from numbers(1200) union all -select 2, 'sum_1', toDateTime(today() - 3) - number * 60 - 30, number, 1, number from numbers(1200) union all -select 1, 'sum_2', toDateTime(today() - 3) - number * 60 - 30, number, 1, number from numbers(1200) union all -select 2, 'sum_2', toDateTime(today() - 3) - number * 60 - 30, number, 1, number from numbers(1200) union all -select 1, 'max_1', toDateTime(today() - 3) - number * 60 - 30, number, 1, number from numbers(1200) union all -select 2, 'max_1', toDateTime(today() - 3) - number * 60 - 30, number, 1, number from numbers(1200) union all -select 1, 'max_2', toDateTime(today() - 3) - number * 60 - 30, number, 1, number from numbers(1200) union all -select 2, 'max_2', toDateTime(today() - 3) - number * 60 - 30, number, 1, number from numbers(1200); +select 1, 'sum_1', toDateTime('2021-08-07', 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all +select 2, 'sum_1', toDateTime('2021-08-07', 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all +select 1, 'sum_2', toDateTime('2021-08-07', 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all +select 2, 'sum_2', toDateTime('2021-08-07', 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all +select 1, 'max_1', toDateTime('2021-08-07', 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all +select 2, 'max_1', toDateTime('2021-08-07', 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all +select 1, 'max_2', toDateTime('2021-08-07', 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all +select 2, 'max_2', toDateTime('2021-08-07', 'UTC') - number * 60 - 30, number, 1, number from numbers(1200); optimize table test_graphite final; -select key, Path, Value, Version, col from test_graphite order by key, Path, Time desc; +select key, Path, Time, Value, Version, col from test_graphite order by key, Path, Time desc; drop table test_graphite; From 32e4a9a5a5b5d34d215326d6f2bb78bf119b00a9 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Tue, 10 Aug 2021 20:58:36 +0300 Subject: [PATCH 061/161] Update ci-runner.py --- tests/integration/ci-runner.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index ecd4cb8d4e7..7aecf2953f8 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -264,7 +264,7 @@ class ClickhouseIntegrationTestsRunner: out_file = "all_tests.txt" out_file_full = "all_tests_full.txt" cmd = "cd {repo_path}/tests/integration && " \ - "./runner --tmpfs {image_cmd} ' --setup-plan' " \ + "timeout 1h ./runner --tmpfs {image_cmd} ' --setup-plan' " \ "| tee {out_file_full} | grep '::' | sed 's/ (fixtures used:.*//g' | sed 's/^ *//g' | sed 's/ *$//g' " \ "| grep -v 'SKIPPED' | sort -u > {out_file}".format( repo_path=repo_path, image_cmd=image_cmd, out_file=out_file, out_file_full=out_file_full) From 7f247becca1e483489ff46d4af28b3437693f1c9 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 11 Aug 2021 07:32:46 +0000 Subject: [PATCH 062/161] Move some templates --- src/Functions/FunctionsConversion.cpp | 1 + src/Functions/FunctionsConversion.h | 17 ----------------- src/Functions/cast_overload_fwd.h | 24 ++++++++++++++++++++++++ src/Interpreters/ActionsDAG.cpp | 1 + src/Interpreters/castColumn.cpp | 1 + src/Storages/MergeTree/KeyCondition.cpp | 1 + 6 files changed, 28 insertions(+), 17 deletions(-) create mode 100644 src/Functions/cast_overload_fwd.h diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index cdbd32b189c..ae115a6e90d 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -1,5 +1,6 @@ #include #include +#include namespace DB diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 362249b3b27..941a2fae0ab 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -3485,21 +3485,4 @@ private: }; -struct CastOverloadName -{ - static constexpr auto cast_name = "CAST"; - static constexpr auto accurate_cast_name = "accurateCast"; - static constexpr auto accurate_cast_or_null_name = "accurateCastOrNull"; -}; - -struct CastInternalOverloadName -{ - static constexpr auto cast_name = "_CAST"; - static constexpr auto accurate_cast_name = "accurate_Cast"; - static constexpr auto accurate_cast_or_null_name = "accurate_CastOrNull"; -}; - -template using CastOverloadResolver = CastOverloadResolverImpl; -template using CastInternalOverloadResolver = CastOverloadResolverImpl; - } diff --git a/src/Functions/cast_overload_fwd.h b/src/Functions/cast_overload_fwd.h new file mode 100644 index 00000000000..1264d01b5d9 --- /dev/null +++ b/src/Functions/cast_overload_fwd.h @@ -0,0 +1,24 @@ +#pragma once +#include + +namespace DB +{ + +struct CastOverloadName +{ + static constexpr auto cast_name = "CAST"; + static constexpr auto accurate_cast_name = "accurateCast"; + static constexpr auto accurate_cast_or_null_name = "accurateCastOrNull"; +}; + +struct CastInternalOverloadName +{ + static constexpr auto cast_name = "_CAST"; + static constexpr auto accurate_cast_name = "accurate_Cast"; + static constexpr auto accurate_cast_or_null_name = "accurate_CastOrNull"; +}; + +template using CastOverloadResolver = CastOverloadResolverImpl; +template using CastInternalOverloadResolver = CastOverloadResolverImpl; + +} diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index a42d6053e9a..664b0d61a4f 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Interpreters/castColumn.cpp b/src/Interpreters/castColumn.cpp index 3356d37ba7f..4c3c564ce58 100644 --- a/src/Interpreters/castColumn.cpp +++ b/src/Interpreters/castColumn.cpp @@ -1,6 +1,7 @@ #include #include +#include namespace DB { diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index e6dfdb859b8..751ac9fd9d9 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include #include #include From c93868babf2439b6d030af30870e944800b82a8a Mon Sep 17 00:00:00 2001 From: tavplubix Date: Wed, 11 Aug 2021 13:12:24 +0300 Subject: [PATCH 063/161] Update ci-runner.py --- tests/integration/ci-runner.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index 7aecf2953f8..a23d7b3485b 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -419,7 +419,7 @@ class ClickhouseIntegrationTestsRunner: test_cmd = ' '.join([test for test in sorted(test_names)]) parallel_cmd = " --parallel {} ".format(num_workers) if num_workers > 0 else "" - cmd = "cd {}/tests/integration && ./runner --tmpfs {} -t {} {} '-rfEp --run-id={} --color=no --durations=0 {}' | tee {}".format( + cmd = "cd {}/tests/integration && timeout 1h ./runner --tmpfs {} -t {} {} '-rfEp --run-id={} --color=no --durations=0 {}' | tee {}".format( repo_path, image_cmd, test_cmd, parallel_cmd, i, _get_deselect_option(self.should_skip_tests()), info_path) log_basename = test_group_str + "_" + str(i) + ".log" From aa8ca01bb96317d9f0cdb112985ace57f8169070 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Wed, 11 Aug 2021 15:17:46 +0300 Subject: [PATCH 064/161] Update ci-runner.py --- tests/integration/ci-runner.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index a23d7b3485b..12ee436e400 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -264,7 +264,7 @@ class ClickhouseIntegrationTestsRunner: out_file = "all_tests.txt" out_file_full = "all_tests_full.txt" cmd = "cd {repo_path}/tests/integration && " \ - "timeout 1h ./runner --tmpfs {image_cmd} ' --setup-plan' " \ + "timeout -s 9 1h ./runner --tmpfs {image_cmd} ' --setup-plan' " \ "| tee {out_file_full} | grep '::' | sed 's/ (fixtures used:.*//g' | sed 's/^ *//g' | sed 's/ *$//g' " \ "| grep -v 'SKIPPED' | sort -u > {out_file}".format( repo_path=repo_path, image_cmd=image_cmd, out_file=out_file, out_file_full=out_file_full) @@ -419,7 +419,7 @@ class ClickhouseIntegrationTestsRunner: test_cmd = ' '.join([test for test in sorted(test_names)]) parallel_cmd = " --parallel {} ".format(num_workers) if num_workers > 0 else "" - cmd = "cd {}/tests/integration && timeout 1h ./runner --tmpfs {} -t {} {} '-rfEp --run-id={} --color=no --durations=0 {}' | tee {}".format( + cmd = "cd {}/tests/integration && timeout -s 9 1h ./runner --tmpfs {} -t {} {} '-rfEp --run-id={} --color=no --durations=0 {}' | tee {}".format( repo_path, image_cmd, test_cmd, parallel_cmd, i, _get_deselect_option(self.should_skip_tests()), info_path) log_basename = test_group_str + "_" + str(i) + ".log" From 2117278f88a5f78543fe1c8c005114fbfff8c9c6 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 11 Aug 2021 15:21:54 +0300 Subject: [PATCH 065/161] Another try --- src/Common/DenseHashMap.h | 2 +- src/Common/DenseHashSet.h | 2 +- src/Common/SparseHashMap.h | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Common/DenseHashMap.h b/src/Common/DenseHashMap.h index 7a9d5a68e2d..9054d049e07 100644 --- a/src/Common/DenseHashMap.h +++ b/src/Common/DenseHashMap.h @@ -2,7 +2,7 @@ #include #if defined(ARCADIA_BUILD) -#define HASH_FUN_H ; +#define HASH_FUN_H template struct THash : public std::hash {}; #endif diff --git a/src/Common/DenseHashSet.h b/src/Common/DenseHashSet.h index 004d1c10903..7f28585d9db 100644 --- a/src/Common/DenseHashSet.h +++ b/src/Common/DenseHashSet.h @@ -1,7 +1,7 @@ #pragma once #if defined(ARCADIA_BUILD) -#define HASH_FUN_H ; +#define HASH_FUN_H template struct THash : public std::hash {}; #endif diff --git a/src/Common/SparseHashMap.h b/src/Common/SparseHashMap.h index 651dd22f15d..7464ebf831a 100644 --- a/src/Common/SparseHashMap.h +++ b/src/Common/SparseHashMap.h @@ -1,7 +1,7 @@ #pragma once #if defined(ARCADIA_BUILD) -#define HASH_FUN_H ; +#define HASH_FUN_H template struct THash : public std::hash {}; #endif From 79290f8e2e2918a619edd28c1b9d5c3f07c03cb1 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 11 Aug 2021 20:49:19 +0300 Subject: [PATCH 066/161] Another try --- src/Common/DenseHashMap.h | 2 +- src/Common/DenseHashSet.h | 2 +- src/Common/SparseHashMap.h | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Common/DenseHashMap.h b/src/Common/DenseHashMap.h index 9054d049e07..26b9224528b 100644 --- a/src/Common/DenseHashMap.h +++ b/src/Common/DenseHashMap.h @@ -4,7 +4,7 @@ #if defined(ARCADIA_BUILD) #define HASH_FUN_H template -struct THash : public std::hash {}; +struct THash; #endif #include diff --git a/src/Common/DenseHashSet.h b/src/Common/DenseHashSet.h index 7f28585d9db..5b30d88a70c 100644 --- a/src/Common/DenseHashSet.h +++ b/src/Common/DenseHashSet.h @@ -3,7 +3,7 @@ #if defined(ARCADIA_BUILD) #define HASH_FUN_H template -struct THash : public std::hash {}; +struct THash; #endif #include diff --git a/src/Common/SparseHashMap.h b/src/Common/SparseHashMap.h index 7464ebf831a..403042c11a9 100644 --- a/src/Common/SparseHashMap.h +++ b/src/Common/SparseHashMap.h @@ -3,7 +3,7 @@ #if defined(ARCADIA_BUILD) #define HASH_FUN_H template -struct THash : public std::hash {}; +struct THash; #endif #include From 739ffe97b658bbcdc5d08b50c48e9cc9fb5540ad Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 11 Aug 2021 21:07:18 +0300 Subject: [PATCH 067/161] Fix ya.make. --- src/Common/ya.make | 2 ++ src/Functions/ya.make | 1 + 2 files changed, 3 insertions(+) diff --git a/src/Common/ya.make b/src/Common/ya.make index 60dfd5f6bee..82962123e56 100644 --- a/src/Common/ya.make +++ b/src/Common/ya.make @@ -102,6 +102,7 @@ SRCS( ZooKeeper/ZooKeeperNodeCache.cpp checkStackSize.cpp clearPasswordFromCommandLine.cpp + clickhouse_malloc.cpp createHardLink.cpp escapeForFileName.cpp filesystemHelpers.cpp @@ -116,6 +117,7 @@ SRCS( hex.cpp isLocalAddress.cpp malloc.cpp + memory.cpp new_delete.cpp parseAddress.cpp parseGlobs.cpp diff --git a/src/Functions/ya.make b/src/Functions/ya.make index 2b9b3d94313..71822924ee3 100644 --- a/src/Functions/ya.make +++ b/src/Functions/ya.make @@ -312,6 +312,7 @@ SRCS( hasToken.cpp hasTokenCaseInsensitive.cpp hostName.cpp + hyperscanRegexpChecker.cpp hypot.cpp identity.cpp if.cpp From 7555fc19fd47d2067e55b0b3aeb1498869226284 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 11 Aug 2021 18:49:25 +0000 Subject: [PATCH 068/161] Revert "Move some templates" This reverts commit 7f247becca1e483489ff46d4af28b3437693f1c9. --- src/Functions/FunctionsConversion.cpp | 1 - src/Functions/FunctionsConversion.h | 17 +++++++++++++++++ src/Functions/cast_overload_fwd.h | 24 ------------------------ src/Interpreters/ActionsDAG.cpp | 1 - src/Interpreters/castColumn.cpp | 1 - src/Storages/MergeTree/KeyCondition.cpp | 1 - 6 files changed, 17 insertions(+), 28 deletions(-) delete mode 100644 src/Functions/cast_overload_fwd.h diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index ae115a6e90d..cdbd32b189c 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -1,6 +1,5 @@ #include #include -#include namespace DB diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 941a2fae0ab..362249b3b27 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -3485,4 +3485,21 @@ private: }; +struct CastOverloadName +{ + static constexpr auto cast_name = "CAST"; + static constexpr auto accurate_cast_name = "accurateCast"; + static constexpr auto accurate_cast_or_null_name = "accurateCastOrNull"; +}; + +struct CastInternalOverloadName +{ + static constexpr auto cast_name = "_CAST"; + static constexpr auto accurate_cast_name = "accurate_Cast"; + static constexpr auto accurate_cast_or_null_name = "accurate_CastOrNull"; +}; + +template using CastOverloadResolver = CastOverloadResolverImpl; +template using CastInternalOverloadResolver = CastOverloadResolverImpl; + } diff --git a/src/Functions/cast_overload_fwd.h b/src/Functions/cast_overload_fwd.h deleted file mode 100644 index 1264d01b5d9..00000000000 --- a/src/Functions/cast_overload_fwd.h +++ /dev/null @@ -1,24 +0,0 @@ -#pragma once -#include - -namespace DB -{ - -struct CastOverloadName -{ - static constexpr auto cast_name = "CAST"; - static constexpr auto accurate_cast_name = "accurateCast"; - static constexpr auto accurate_cast_or_null_name = "accurateCastOrNull"; -}; - -struct CastInternalOverloadName -{ - static constexpr auto cast_name = "_CAST"; - static constexpr auto accurate_cast_name = "accurate_Cast"; - static constexpr auto accurate_cast_or_null_name = "accurate_CastOrNull"; -}; - -template using CastOverloadResolver = CastOverloadResolverImpl; -template using CastInternalOverloadResolver = CastOverloadResolverImpl; - -} diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 664b0d61a4f..a42d6053e9a 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -7,7 +7,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Interpreters/castColumn.cpp b/src/Interpreters/castColumn.cpp index 4c3c564ce58..3356d37ba7f 100644 --- a/src/Interpreters/castColumn.cpp +++ b/src/Interpreters/castColumn.cpp @@ -1,7 +1,6 @@ #include #include -#include namespace DB { diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 751ac9fd9d9..e6dfdb859b8 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -11,7 +11,6 @@ #include #include #include -#include #include #include #include From dcc6f597dcae1511d9644c6a346972ddad5c7993 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 11 Aug 2021 19:09:51 +0000 Subject: [PATCH 069/161] Move templates from FunctionsConvertion --- src/Functions/CastOverloadResolver.cpp | 19 +++ src/Functions/CastOverloadResolver.h | 117 ++++++++++++++++++ .../DateOrDateTimeFunctionsConvertion.cpp | 0 src/Functions/FunctionsConversion.cpp | 9 +- src/Functions/FunctionsConversion.h | 110 ---------------- src/Interpreters/ActionsDAG.cpp | 1 + src/Interpreters/castColumn.cpp | 1 + src/Storages/MergeTree/KeyCondition.cpp | 1 + 8 files changed, 142 insertions(+), 116 deletions(-) create mode 100644 src/Functions/CastOverloadResolver.cpp create mode 100644 src/Functions/CastOverloadResolver.h create mode 100644 src/Functions/DateOrDateTimeFunctionsConvertion.cpp diff --git a/src/Functions/CastOverloadResolver.cpp b/src/Functions/CastOverloadResolver.cpp new file mode 100644 index 00000000000..fd6fecc37d6 --- /dev/null +++ b/src/Functions/CastOverloadResolver.cpp @@ -0,0 +1,19 @@ +#include +#include + + +namespace DB +{ + +void registerCastOverloadResolvers(FunctionFactory & factory) +{ + factory.registerFunction>(FunctionFactory::CaseInsensitive); + factory.registerFunction>(); + factory.registerFunction>(); + + factory.registerFunction>(FunctionFactory::CaseInsensitive); + factory.registerFunction>(); + factory.registerFunction>(); +} + +} diff --git a/src/Functions/CastOverloadResolver.h b/src/Functions/CastOverloadResolver.h new file mode 100644 index 00000000000..c5f6489fc3c --- /dev/null +++ b/src/Functions/CastOverloadResolver.h @@ -0,0 +1,117 @@ +#pragma once +#include + + +namespace DB +{ + +/* + * CastInternal does not preserve nullability of the data type, + * i.e. CastInternal(toNullable(toInt8(1)) as Int32) will be Int32(1). + * + * Cast preserves nullability according to setting `cast_keep_nullable`, + * i.e. Cast(toNullable(toInt8(1)) as Int32) will be Nullable(Int32(1)) if `cast_keep_nullable` == 1. +**/ +template +class CastOverloadResolverImpl : public IFunctionOverloadResolver +{ +public: + using MonotonicityForRange = FunctionCastBase::MonotonicityForRange; + using Diagnostic = FunctionCastBase::Diagnostic; + + static constexpr auto name = cast_type == CastType::accurate + ? CastName::accurate_cast_name + : (cast_type == CastType::accurateOrNull ? CastName::accurate_cast_or_null_name : CastName::cast_name); + + String getName() const override { return name; } + + size_t getNumberOfArguments() const override { return 2; } + + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } + + explicit CastOverloadResolverImpl(std::optional diagnostic_, bool keep_nullable_) + : diagnostic(std::move(diagnostic_)), keep_nullable(keep_nullable_) + { + } + + static FunctionOverloadResolverPtr create(ContextPtr context) + { + if constexpr (internal) + return createImpl(); + return createImpl({}, context->getSettingsRef().cast_keep_nullable); + } + + static FunctionOverloadResolverPtr createImpl(std::optional diagnostic = {}, bool keep_nullable = false) + { + assert(!internal || !keep_nullable); + return std::make_unique(std::move(diagnostic), keep_nullable); + } + +protected: + + FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const override + { + DataTypes data_types(arguments.size()); + + for (size_t i = 0; i < arguments.size(); ++i) + data_types[i] = arguments[i].type; + + auto monotonicity = MonotonicityHelper::getMonotonicityInformation(arguments.front().type, return_type.get()); + return std::make_unique>(name, std::move(monotonicity), data_types, return_type, diagnostic, cast_type); + } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + const auto & column = arguments.back().column; + if (!column) + throw Exception("Second argument to " + getName() + " must be a constant string describing type." + " Instead there is non-constant column of type " + arguments.back().type->getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + const auto * type_col = checkAndGetColumnConst(column.get()); + if (!type_col) + throw Exception("Second argument to " + getName() + " must be a constant string describing type." + " Instead there is a column with the following structure: " + column->dumpStructure(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + DataTypePtr type = DataTypeFactory::instance().get(type_col->getValue()); + + if constexpr (cast_type == CastType::accurateOrNull) + return makeNullable(type); + + if constexpr (internal) + return type; + + if (keep_nullable && arguments.front().type->isNullable() && type->canBeInsideNullable()) + return makeNullable(type); + + return type; + } + + bool useDefaultImplementationForNulls() const override { return false; } + bool useDefaultImplementationForLowCardinalityColumns() const override { return false; } + +private: + std::optional diagnostic; + bool keep_nullable; +}; + + +struct CastOverloadName +{ + static constexpr auto cast_name = "CAST"; + static constexpr auto accurate_cast_name = "accurateCast"; + static constexpr auto accurate_cast_or_null_name = "accurateCastOrNull"; +}; + +struct CastInternalOverloadName +{ + static constexpr auto cast_name = "_CAST"; + static constexpr auto accurate_cast_name = "accurate_Cast"; + static constexpr auto accurate_cast_or_null_name = "accurate_CastOrNull"; +}; + +template using CastOverloadResolver = CastOverloadResolverImpl; +template using CastInternalOverloadResolver = CastOverloadResolverImpl; + +} diff --git a/src/Functions/DateOrDateTimeFunctionsConvertion.cpp b/src/Functions/DateOrDateTimeFunctionsConvertion.cpp new file mode 100644 index 00000000000..e69de29bb2d diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index cdbd32b189c..f32d5df8a21 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -7,6 +7,8 @@ namespace DB void registerFunctionFixedString(FunctionFactory & factory); +void registerCastOverloadResolvers(FunctionFactory & factory); + void registerFunctionsConversion(FunctionFactory & factory) { factory.registerFunction(); @@ -43,12 +45,7 @@ void registerFunctionsConversion(FunctionFactory & factory) factory.registerFunction(); - factory.registerFunction>(FunctionFactory::CaseInsensitive); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(FunctionFactory::CaseInsensitive); - factory.registerFunction>(); - factory.registerFunction>(); + registerCastOverloadResolvers(factory); factory.registerFunction(); factory.registerFunction(); diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 362249b3b27..e57998e4a72 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -3392,114 +3392,4 @@ public: } }; - -/* - * CastInternal does not preserve nullability of the data type, - * i.e. CastInternal(toNullable(toInt8(1)) as Int32) will be Int32(1). - * - * Cast preserves nullability according to setting `cast_keep_nullable`, - * i.e. Cast(toNullable(toInt8(1)) as Int32) will be Nullable(Int32(1)) if `cast_keep_nullable` == 1. -**/ -template -class CastOverloadResolverImpl : public IFunctionOverloadResolver -{ -public: - using MonotonicityForRange = FunctionCastBase::MonotonicityForRange; - using Diagnostic = FunctionCastBase::Diagnostic; - - static constexpr auto name = cast_type == CastType::accurate - ? CastName::accurate_cast_name - : (cast_type == CastType::accurateOrNull ? CastName::accurate_cast_or_null_name : CastName::cast_name); - - String getName() const override { return name; } - - size_t getNumberOfArguments() const override { return 2; } - - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } - - explicit CastOverloadResolverImpl(std::optional diagnostic_, bool keep_nullable_) - : diagnostic(std::move(diagnostic_)), keep_nullable(keep_nullable_) - { - } - - static FunctionOverloadResolverPtr create(ContextPtr context) - { - if constexpr (internal) - return createImpl(); - return createImpl({}, context->getSettingsRef().cast_keep_nullable); - } - - static FunctionOverloadResolverPtr createImpl(std::optional diagnostic = {}, bool keep_nullable = false) - { - assert(!internal || !keep_nullable); - return std::make_unique(std::move(diagnostic), keep_nullable); - } - -protected: - - FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const override - { - DataTypes data_types(arguments.size()); - - for (size_t i = 0; i < arguments.size(); ++i) - data_types[i] = arguments[i].type; - - auto monotonicity = MonotonicityHelper::getMonotonicityInformation(arguments.front().type, return_type.get()); - return std::make_unique>(name, std::move(monotonicity), data_types, return_type, diagnostic, cast_type); - } - - DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override - { - const auto & column = arguments.back().column; - if (!column) - throw Exception("Second argument to " + getName() + " must be a constant string describing type." - " Instead there is non-constant column of type " + arguments.back().type->getName(), - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - const auto * type_col = checkAndGetColumnConst(column.get()); - if (!type_col) - throw Exception("Second argument to " + getName() + " must be a constant string describing type." - " Instead there is a column with the following structure: " + column->dumpStructure(), - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - DataTypePtr type = DataTypeFactory::instance().get(type_col->getValue()); - - if constexpr (cast_type == CastType::accurateOrNull) - return makeNullable(type); - - if constexpr (internal) - return type; - - if (keep_nullable && arguments.front().type->isNullable() && type->canBeInsideNullable()) - return makeNullable(type); - - return type; - } - - bool useDefaultImplementationForNulls() const override { return false; } - bool useDefaultImplementationForLowCardinalityColumns() const override { return false; } - -private: - std::optional diagnostic; - bool keep_nullable; -}; - - -struct CastOverloadName -{ - static constexpr auto cast_name = "CAST"; - static constexpr auto accurate_cast_name = "accurateCast"; - static constexpr auto accurate_cast_or_null_name = "accurateCastOrNull"; -}; - -struct CastInternalOverloadName -{ - static constexpr auto cast_name = "_CAST"; - static constexpr auto accurate_cast_name = "accurate_Cast"; - static constexpr auto accurate_cast_or_null_name = "accurate_CastOrNull"; -}; - -template using CastOverloadResolver = CastOverloadResolverImpl; -template using CastInternalOverloadResolver = CastOverloadResolverImpl; - } diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index a42d6053e9a..e1f1d498367 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Interpreters/castColumn.cpp b/src/Interpreters/castColumn.cpp index 3356d37ba7f..fd71e02ee7e 100644 --- a/src/Interpreters/castColumn.cpp +++ b/src/Interpreters/castColumn.cpp @@ -1,6 +1,7 @@ #include #include +#include namespace DB { diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index e6dfdb859b8..18ca00ebf0d 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include From 124a87684f635443058cc4631f19bb5700e8341a Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 11 Aug 2021 19:44:34 +0300 Subject: [PATCH 070/161] Removed some data streams --- src/Common/ya.make | 2 + src/Core/ExternalResultDescription.h | 2 +- ...BlockInputStream.cpp => MongoDBSource.cpp} | 3 +- ...goDBBlockInputStream.h => MongoDBSource.h} | 0 ...ckInputStream.cpp => PostgreSQLSource.cpp} | 2 +- ...LBlockInputStream.h => PostgreSQLSource.h} | 0 ...eBlockInputStream.cpp => SQLiteSource.cpp} | 48 +++++++------------ ...QLiteBlockInputStream.h => SQLiteSource.h} | 18 +++---- src/DataStreams/ya.make | 4 +- src/Databases/MySQL/DatabaseMySQL.cpp | 2 +- .../MySQL/FetchTablesColumnsList.cpp | 2 +- src/Databases/MySQL/MaterializeMetadata.cpp | 2 +- .../MySQL/MaterializedMySQLSyncThread.cpp | 2 +- .../CassandraDictionarySource.cpp | 6 +-- ...ockInputStream.cpp => CassandraSource.cpp} | 2 +- ...raBlockInputStream.h => CassandraSource.h} | 0 src/Dictionaries/MongoDBDictionarySource.cpp | 2 +- src/Dictionaries/MySQLDictionarySource.h | 2 +- .../PostgreSQLDictionarySource.cpp | 2 +- src/Dictionaries/ya.make | 2 +- ...QLBlockInputStream.cpp => MySQLSource.cpp} | 2 +- ...{MySQLBlockInputStream.h => MySQLSource.h} | 2 +- src/Formats/ya.make | 2 +- src/Functions/ya.make | 1 + .../PostgreSQLReplicationHandler.cpp | 2 +- src/Storages/StorageMongoDB.cpp | 2 +- src/Storages/StorageMySQL.cpp | 2 +- src/Storages/StoragePostgreSQL.cpp | 2 +- src/Storages/StorageSQLite.cpp | 5 +- src/TableFunctions/TableFunctionMySQL.cpp | 2 +- 30 files changed, 55 insertions(+), 70 deletions(-) rename src/DataStreams/{MongoDBBlockInputStream.cpp => MongoDBSource.cpp} (99%) rename src/DataStreams/{MongoDBBlockInputStream.h => MongoDBSource.h} (100%) rename src/DataStreams/{PostgreSQLBlockInputStream.cpp => PostgreSQLSource.cpp} (99%) rename src/DataStreams/{PostgreSQLBlockInputStream.h => PostgreSQLSource.h} (100%) rename src/DataStreams/{SQLiteBlockInputStream.cpp => SQLiteSource.cpp} (81%) rename src/DataStreams/{SQLiteBlockInputStream.h => SQLiteSource.h} (63%) rename src/Dictionaries/{CassandraBlockInputStream.cpp => CassandraSource.cpp} (99%) rename src/Dictionaries/{CassandraBlockInputStream.h => CassandraSource.h} (100%) rename src/Formats/{MySQLBlockInputStream.cpp => MySQLSource.cpp} (99%) rename src/Formats/{MySQLBlockInputStream.h => MySQLSource.h} (96%) diff --git a/src/Common/ya.make b/src/Common/ya.make index 60dfd5f6bee..82962123e56 100644 --- a/src/Common/ya.make +++ b/src/Common/ya.make @@ -102,6 +102,7 @@ SRCS( ZooKeeper/ZooKeeperNodeCache.cpp checkStackSize.cpp clearPasswordFromCommandLine.cpp + clickhouse_malloc.cpp createHardLink.cpp escapeForFileName.cpp filesystemHelpers.cpp @@ -116,6 +117,7 @@ SRCS( hex.cpp isLocalAddress.cpp malloc.cpp + memory.cpp new_delete.cpp parseAddress.cpp parseGlobs.cpp diff --git a/src/Core/ExternalResultDescription.h b/src/Core/ExternalResultDescription.h index 78c054e805f..a9ffe8b2ed2 100644 --- a/src/Core/ExternalResultDescription.h +++ b/src/Core/ExternalResultDescription.h @@ -6,7 +6,7 @@ namespace DB { -/** Common part for implementation of MySQLBlockInputStream, MongoDBBlockInputStream and others. +/** Common part for implementation of MySQLSource, MongoDBSource and others. */ struct ExternalResultDescription { diff --git a/src/DataStreams/MongoDBBlockInputStream.cpp b/src/DataStreams/MongoDBSource.cpp similarity index 99% rename from src/DataStreams/MongoDBBlockInputStream.cpp rename to src/DataStreams/MongoDBSource.cpp index a0a8e3e40a5..c00d214249a 100644 --- a/src/DataStreams/MongoDBBlockInputStream.cpp +++ b/src/DataStreams/MongoDBSource.cpp @@ -1,3 +1,5 @@ +#include "MongoDBSource.h" + #include #include @@ -15,7 +17,6 @@ #include #include #include -#include #include #include #include diff --git a/src/DataStreams/MongoDBBlockInputStream.h b/src/DataStreams/MongoDBSource.h similarity index 100% rename from src/DataStreams/MongoDBBlockInputStream.h rename to src/DataStreams/MongoDBSource.h diff --git a/src/DataStreams/PostgreSQLBlockInputStream.cpp b/src/DataStreams/PostgreSQLSource.cpp similarity index 99% rename from src/DataStreams/PostgreSQLBlockInputStream.cpp rename to src/DataStreams/PostgreSQLSource.cpp index 0b50c453629..c3bde8c84ad 100644 --- a/src/DataStreams/PostgreSQLBlockInputStream.cpp +++ b/src/DataStreams/PostgreSQLSource.cpp @@ -1,4 +1,4 @@ -#include "PostgreSQLBlockInputStream.h" +#include "PostgreSQLSource.h" #if USE_LIBPQXX #include diff --git a/src/DataStreams/PostgreSQLBlockInputStream.h b/src/DataStreams/PostgreSQLSource.h similarity index 100% rename from src/DataStreams/PostgreSQLBlockInputStream.h rename to src/DataStreams/PostgreSQLSource.h diff --git a/src/DataStreams/SQLiteBlockInputStream.cpp b/src/DataStreams/SQLiteSource.cpp similarity index 81% rename from src/DataStreams/SQLiteBlockInputStream.cpp rename to src/DataStreams/SQLiteSource.cpp index da7645d968d..f4995703a1e 100644 --- a/src/DataStreams/SQLiteBlockInputStream.cpp +++ b/src/DataStreams/SQLiteSource.cpp @@ -1,4 +1,4 @@ -#include "SQLiteBlockInputStream.h" +#include "SQLiteSource.h" #if USE_SQLITE #include @@ -22,37 +22,33 @@ namespace ErrorCodes extern const int SQLITE_ENGINE_ERROR; } -SQLiteBlockInputStream::SQLiteBlockInputStream( +SQLiteSource::SQLiteSource( SQLitePtr sqlite_db_, const String & query_str_, const Block & sample_block, const UInt64 max_block_size_) - : query_str(query_str_) + : SourceWithProgress(sample_block.cloneEmpty()) + , query_str(query_str_) , max_block_size(max_block_size_) , sqlite_db(std::move(sqlite_db_)) { description.init(sample_block); } - -void SQLiteBlockInputStream::readPrefix() -{ - sqlite3_stmt * compiled_stmt = nullptr; - int status = sqlite3_prepare_v2(sqlite_db.get(), query_str.c_str(), query_str.size() + 1, &compiled_stmt, nullptr); - - if (status != SQLITE_OK) - throw Exception(ErrorCodes::SQLITE_ENGINE_ERROR, - "Cannot prepate sqlite statement. Status: {}. Message: {}", - status, sqlite3_errstr(status)); - - compiled_statement = std::unique_ptr(compiled_stmt, StatementDeleter()); -} - - -Block SQLiteBlockInputStream::readImpl() +Chunk SQLiteSource::generate() { if (!compiled_statement) - return Block(); + { + sqlite3_stmt * compiled_stmt = nullptr; + int status = sqlite3_prepare_v2(sqlite_db.get(), query_str.c_str(), query_str.size() + 1, &compiled_stmt, nullptr); + + if (status != SQLITE_OK) + throw Exception(ErrorCodes::SQLITE_ENGINE_ERROR, + "Cannot prepate sqlite statement. Status: {}. Message: {}", + status, sqlite3_errstr(status)); + + compiled_statement = std::unique_ptr(compiled_stmt, StatementDeleter()); + } MutableColumns columns = description.sample_block.cloneEmptyColumns(); size_t num_rows = 0; @@ -104,18 +100,10 @@ Block SQLiteBlockInputStream::readImpl() break; } - return description.sample_block.cloneWithColumns(std::move(columns)); + return Chunk(std::move(columns), num_rows); } - -void SQLiteBlockInputStream::readSuffix() -{ - if (compiled_statement) - compiled_statement.reset(); -} - - -void SQLiteBlockInputStream::insertValue(IColumn & column, const ExternalResultDescription::ValueType type, size_t idx) +void SQLiteSource::insertValue(IColumn & column, ExternalResultDescription::ValueType type, size_t idx) { switch (type) { diff --git a/src/DataStreams/SQLiteBlockInputStream.h b/src/DataStreams/SQLiteSource.h similarity index 63% rename from src/DataStreams/SQLiteBlockInputStream.h rename to src/DataStreams/SQLiteSource.h index 35fc4801b4b..653fdb402e3 100644 --- a/src/DataStreams/SQLiteBlockInputStream.h +++ b/src/DataStreams/SQLiteSource.h @@ -6,29 +6,27 @@ #if USE_SQLITE #include -#include +#include #include // Y_IGNORE namespace DB { -class SQLiteBlockInputStream : public IBlockInputStream +class SQLiteSource : public SourceWithProgress { using SQLitePtr = std::shared_ptr; public: - SQLiteBlockInputStream(SQLitePtr sqlite_db_, + SQLiteSource(SQLitePtr sqlite_db_, const String & query_str_, const Block & sample_block, UInt64 max_block_size_); String getName() const override { return "SQLite"; } - Block getHeader() const override { return description.sample_block.cloneEmpty(); } - private: - void insertDefaultSQLiteValue(IColumn & column, const IColumn & sample_column) + static void insertDefaultSQLiteValue(IColumn & column, const IColumn & sample_column) { column.insertFrom(sample_column, 0); } @@ -40,13 +38,9 @@ private: void operator()(sqlite3_stmt * stmt) { sqlite3_finalize(stmt); } }; - void readPrefix() override; + Chunk generate() override; - Block readImpl() override; - - void readSuffix() override; - - void insertValue(IColumn & column, const ExternalResultDescription::ValueType type, size_t idx); + void insertValue(IColumn & column, ExternalResultDescription::ValueType type, size_t idx); String query_str; UInt64 max_block_size; diff --git a/src/DataStreams/ya.make b/src/DataStreams/ya.make index b1205828a7e..c16db808a5b 100644 --- a/src/DataStreams/ya.make +++ b/src/DataStreams/ya.make @@ -29,7 +29,7 @@ SRCS( ITTLAlgorithm.cpp InternalTextLogsRowOutputStream.cpp MaterializingBlockInputStream.cpp - MongoDBBlockInputStream.cpp + MongoDBSource.cpp NativeBlockInputStream.cpp NativeBlockOutputStream.cpp PushingToViewsBlockOutputStream.cpp @@ -37,7 +37,7 @@ SRCS( RemoteBlockOutputStream.cpp RemoteQueryExecutor.cpp RemoteQueryExecutorReadContext.cpp - SQLiteBlockInputStream.cpp + SQLiteSource.cpp SizeLimits.cpp SquashingBlockInputStream.cpp SquashingBlockOutputStream.cpp diff --git a/src/Databases/MySQL/DatabaseMySQL.cpp b/src/Databases/MySQL/DatabaseMySQL.cpp index d4acd2af85e..858255e730a 100644 --- a/src/Databases/MySQL/DatabaseMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMySQL.cpp @@ -11,7 +11,7 @@ # include # include # include -# include +# include # include # include # include diff --git a/src/Databases/MySQL/FetchTablesColumnsList.cpp b/src/Databases/MySQL/FetchTablesColumnsList.cpp index 353bcd877ee..c67dcefb433 100644 --- a/src/Databases/MySQL/FetchTablesColumnsList.cpp +++ b/src/Databases/MySQL/FetchTablesColumnsList.cpp @@ -10,7 +10,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Databases/MySQL/MaterializeMetadata.cpp b/src/Databases/MySQL/MaterializeMetadata.cpp index 9f5100991aa..f684797c675 100644 --- a/src/Databases/MySQL/MaterializeMetadata.cpp +++ b/src/Databases/MySQL/MaterializeMetadata.cpp @@ -5,7 +5,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp index 5175e9d0467..53495aa3cb1 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp @@ -16,7 +16,7 @@ # include # include # include -# include +# include # include # include # include diff --git a/src/Dictionaries/CassandraDictionarySource.cpp b/src/Dictionaries/CassandraDictionarySource.cpp index 8b31b4d6fa2..0dea38ecf73 100644 --- a/src/Dictionaries/CassandraDictionarySource.cpp +++ b/src/Dictionaries/CassandraDictionarySource.cpp @@ -36,10 +36,10 @@ void registerDictionarySourceCassandra(DictionarySourceFactory & factory) #if USE_CASSANDRA -#include -#include -#include "CassandraBlockInputStream.h" #include +#include +#include +#include namespace DB { diff --git a/src/Dictionaries/CassandraBlockInputStream.cpp b/src/Dictionaries/CassandraSource.cpp similarity index 99% rename from src/Dictionaries/CassandraBlockInputStream.cpp rename to src/Dictionaries/CassandraSource.cpp index 384717e2ba2..1ebacdb2c2f 100644 --- a/src/Dictionaries/CassandraBlockInputStream.cpp +++ b/src/Dictionaries/CassandraSource.cpp @@ -10,7 +10,7 @@ #include #include #include -#include "CassandraBlockInputStream.h" +#include "CassandraSource.h" namespace DB diff --git a/src/Dictionaries/CassandraBlockInputStream.h b/src/Dictionaries/CassandraSource.h similarity index 100% rename from src/Dictionaries/CassandraBlockInputStream.h rename to src/Dictionaries/CassandraSource.h diff --git a/src/Dictionaries/MongoDBDictionarySource.cpp b/src/Dictionaries/MongoDBDictionarySource.cpp index a3c5119ade1..23ea9bc00e2 100644 --- a/src/Dictionaries/MongoDBDictionarySource.cpp +++ b/src/Dictionaries/MongoDBDictionarySource.cpp @@ -50,7 +50,7 @@ void registerDictionarySourceMongoDB(DictionarySourceFactory & factory) // Poco/MongoDB/BSONWriter.h:54: void writeCString(const std::string & value); // src/IO/WriteHelpers.h:146 #define writeCString(s, buf) #include -#include +#include namespace DB diff --git a/src/Dictionaries/MySQLDictionarySource.h b/src/Dictionaries/MySQLDictionarySource.h index 49ddc924a86..1c8c1ed4e33 100644 --- a/src/Dictionaries/MySQLDictionarySource.h +++ b/src/Dictionaries/MySQLDictionarySource.h @@ -12,7 +12,7 @@ # include "DictionaryStructure.h" # include "ExternalQueryBuilder.h" # include "IDictionarySource.h" -# include +# include namespace Poco { diff --git a/src/Dictionaries/PostgreSQLDictionarySource.cpp b/src/Dictionaries/PostgreSQLDictionarySource.cpp index f226b7a9165..2a0173c1615 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.cpp +++ b/src/Dictionaries/PostgreSQLDictionarySource.cpp @@ -7,7 +7,7 @@ #if USE_LIBPQXX #include #include -#include +#include #include "readInvalidateQuery.h" #include #endif diff --git a/src/Dictionaries/ya.make b/src/Dictionaries/ya.make index 36152fe439a..2cff8447574 100644 --- a/src/Dictionaries/ya.make +++ b/src/Dictionaries/ya.make @@ -22,9 +22,9 @@ NO_COMPILER_WARNINGS() SRCS( CacheDictionary.cpp CacheDictionaryUpdateQueue.cpp - CassandraBlockInputStream.cpp CassandraDictionarySource.cpp CassandraHelpers.cpp + CassandraSource.cpp ClickHouseDictionarySource.cpp DictionaryBlockInputStream.cpp DictionaryBlockInputStreamBase.cpp diff --git a/src/Formats/MySQLBlockInputStream.cpp b/src/Formats/MySQLSource.cpp similarity index 99% rename from src/Formats/MySQLBlockInputStream.cpp rename to src/Formats/MySQLSource.cpp index 401d85f3d6b..2d305a29df6 100644 --- a/src/Formats/MySQLBlockInputStream.cpp +++ b/src/Formats/MySQLSource.cpp @@ -19,7 +19,7 @@ #include #include #include -#include "MySQLBlockInputStream.h" +#include "MySQLSource.h" namespace DB diff --git a/src/Formats/MySQLBlockInputStream.h b/src/Formats/MySQLSource.h similarity index 96% rename from src/Formats/MySQLBlockInputStream.h rename to src/Formats/MySQLSource.h index 9c33b4404ae..5938cb4b57f 100644 --- a/src/Formats/MySQLBlockInputStream.h +++ b/src/Formats/MySQLSource.h @@ -58,7 +58,7 @@ protected: ExternalResultDescription description; }; -/// Like MySQLBlockInputStream, but allocates connection only when reading is starting. +/// Like MySQLSource, but allocates connection only when reading is starting. /// It allows to create a lot of stream objects without occupation of all connection pool. /// Also makes attempts to reconnect in case of connection failures. class MySQLWithFailoverSource final : public MySQLSource diff --git a/src/Formats/ya.make b/src/Formats/ya.make index 476e13f9a4f..90184350359 100644 --- a/src/Formats/ya.make +++ b/src/Formats/ya.make @@ -14,7 +14,7 @@ SRCS( FormatFactory.cpp FormatSchemaInfo.cpp JSONEachRowUtils.cpp - MySQLBlockInputStream.cpp + MySQLSource.cpp NativeFormat.cpp NullFormat.cpp ParsedTemplateFormatString.cpp diff --git a/src/Functions/ya.make b/src/Functions/ya.make index 2b9b3d94313..71822924ee3 100644 --- a/src/Functions/ya.make +++ b/src/Functions/ya.make @@ -312,6 +312,7 @@ SRCS( hasToken.cpp hasTokenCaseInsensitive.cpp hostName.cpp + hyperscanRegexpChecker.cpp hypot.cpp identity.cpp if.cpp diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index b812c6d2923..3477397adb7 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -1,6 +1,6 @@ #include "PostgreSQLReplicationHandler.h" -#include +#include #include #include #include diff --git a/src/Storages/StorageMongoDB.cpp b/src/Storages/StorageMongoDB.cpp index a973efd7277..3bdef7fd295 100644 --- a/src/Storages/StorageMongoDB.cpp +++ b/src/Storages/StorageMongoDB.cpp @@ -15,7 +15,7 @@ #include #include #include -#include +#include namespace DB { diff --git a/src/Storages/StorageMySQL.cpp b/src/Storages/StorageMySQL.cpp index 431fda530f4..79bb1f59cc7 100644 --- a/src/Storages/StorageMySQL.cpp +++ b/src/Storages/StorageMySQL.cpp @@ -4,7 +4,7 @@ #include #include -#include +#include #include #include #include diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index b71f2415fd8..5f8b81a47cf 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -1,7 +1,7 @@ #include "StoragePostgreSQL.h" #if USE_LIBPQXX -#include +#include #include #include diff --git a/src/Storages/StorageSQLite.cpp b/src/Storages/StorageSQLite.cpp index ba66083fea5..758284e8d50 100644 --- a/src/Storages/StorageSQLite.cpp +++ b/src/Storages/StorageSQLite.cpp @@ -2,7 +2,7 @@ #if USE_SQLITE #include -#include +#include #include #include #include @@ -78,8 +78,7 @@ Pipe StorageSQLite::read( sample_block.insert({column_data.type, column_data.name}); } - return Pipe(std::make_shared( - std::make_shared(sqlite_db, query, sample_block, max_block_size))); + return Pipe(std::make_shared(sqlite_db, query, sample_block, max_block_size)); } diff --git a/src/TableFunctions/TableFunctionMySQL.cpp b/src/TableFunctions/TableFunctionMySQL.cpp index f8e0c41634b..09f9cf8b1f5 100644 --- a/src/TableFunctions/TableFunctionMySQL.cpp +++ b/src/TableFunctions/TableFunctionMySQL.cpp @@ -8,7 +8,7 @@ #include #include #include -#include +#include #include #include #include From 625aa67dad981735f5123a3fe05cc831d7b37926 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 12 Aug 2021 06:34:57 +0000 Subject: [PATCH 071/161] Fix checks --- src/Functions/CastOverloadResolver.h | 6 +++++- .../test_alter_update_cast_keep_nullable/configs/users.xml | 4 +++- 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/src/Functions/CastOverloadResolver.h b/src/Functions/CastOverloadResolver.h index c5f6489fc3c..ffd5dda4af3 100644 --- a/src/Functions/CastOverloadResolver.h +++ b/src/Functions/CastOverloadResolver.h @@ -1,10 +1,14 @@ #pragma once #include - namespace DB { +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + /* * CastInternal does not preserve nullability of the data type, * i.e. CastInternal(toNullable(toInt8(1)) as Int32) will be Int32(1). diff --git a/tests/integration/test_alter_update_cast_keep_nullable/configs/users.xml b/tests/integration/test_alter_update_cast_keep_nullable/configs/users.xml index 16b032daacb..aa2f240b831 100644 --- a/tests/integration/test_alter_update_cast_keep_nullable/configs/users.xml +++ b/tests/integration/test_alter_update_cast_keep_nullable/configs/users.xml @@ -1,6 +1,8 @@ - 1 + + 1 + From f034cbedf1ceba49c6115bd1ed84ddd211ed6c68 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Thu, 12 Aug 2021 09:44:15 +0300 Subject: [PATCH 072/161] Update docs/ru/sql-reference/functions/string-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/string-functions.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index 39498fcea9f..16decddb839 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -76,9 +76,9 @@ SELECT empty(''); Результат: ```text -┌─empty('')─┐ -│ 1 │ -└───────────┘ +┌─notEmpty('text')─┐ +│ 1 │ +└──────────────────┘ ``` ## length {#length} From f53ef136b1ca8f70b0dc1ad454b9ed68058be640 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Thu, 12 Aug 2021 09:44:31 +0300 Subject: [PATCH 073/161] Update docs/ru/sql-reference/functions/string-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index 16decddb839..5b166c5bca6 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -70,7 +70,7 @@ empty(x) Запрос: ```sql -SELECT empty(''); +SELECT notEmpty('text'); ``` Результат: From 54664ed013f4eb76295c18963940dd9dfda94449 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Thu, 12 Aug 2021 09:44:54 +0300 Subject: [PATCH 074/161] Update docs/en/sql-reference/functions/array-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/functions/array-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/array-functions.md b/docs/en/sql-reference/functions/array-functions.md index 38ee2090866..4968d093a19 100644 --- a/docs/en/sql-reference/functions/array-functions.md +++ b/docs/en/sql-reference/functions/array-functions.md @@ -18,7 +18,7 @@ empty([x]) An array is considered empty if it contains all empty elements. !!! note "Note" - Can be optimized by enabling the [optimize_functions_to_subcolumns](../../operations/settings/settings.md#optimize-functions-to-subcolumns) setting. With `optimize_functions_to_subcolumns = 1` the function reads only [size0](../../sql-reference/data-types/array.md#array-size) subcolumn instead of reading and processing the whole array column. The query `SELECT empty(arr) FROM table;` transforms to `SELECT arr.size0 = 0 FROM TABLE;`. + Can be optimized by enabling the [optimize_functions_to_subcolumns](../../operations/settings/settings.md#optimize-functions-to-subcolumns) setting. With `optimize_functions_to_subcolumns = 1` the function reads only [size0](../../sql-reference/data-types/array.md#array-size) subcolumn instead of reading and processing the whole array. The query `SELECT empty(arr) FROM table;` transforms to `SELECT arr.size0 = 0 FROM table;`. The function also works for [strings](string-functions.md#empty) or [UUID](uuid-functions.md#empty). From 3bc91471e43fa6aca7edb36e16061af57a104496 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Thu, 12 Aug 2021 09:45:08 +0300 Subject: [PATCH 075/161] Update docs/en/sql-reference/functions/string-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 7f3ecb80ee9..57e6a3647ca 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -18,7 +18,7 @@ Checks whether the input string is empty. empty(x) ``` -A string is considered non-empty if it contains at least one byte, even if this is a space or a null byte. The UUID is empty if it contains all zeros (zero UUID). +A string is considered non-empty if it contains at least one byte, even if this is a space or a null byte. The function also works for [arrays](array-functions.md#function-empty) or [UUID](uuid-functions.md#empty). From bc85234efaef9f628adaec7f39a8f402841dc154 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Thu, 12 Aug 2021 09:45:23 +0300 Subject: [PATCH 076/161] Update docs/en/sql-reference/functions/string-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 57e6a3647ca..778533a2d51 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -58,7 +58,7 @@ Checks whether the input string is non-empty. notEmpty(x) ``` -A string is considered non-empty if it contains at least one byte, even if this is a space or a null byte. The UUID is empty if it contains all zeros (zero UUID). +A string is considered non-empty if it contains at least one byte, even if this is a space or a null byte. The function also works for [arrays](array-functions.md#function-notempty) or [UUID](uuid-functions.md#notempty). From ef6da4183b8e4860d956db0da156359dadb5c526 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Thu, 12 Aug 2021 09:45:31 +0300 Subject: [PATCH 077/161] Update docs/ru/engines/database-engines/materialized-mysql.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/engines/database-engines/materialized-mysql.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/engines/database-engines/materialized-mysql.md b/docs/ru/engines/database-engines/materialized-mysql.md index 40576ec450b..ff15af05397 100644 --- a/docs/ru/engines/database-engines/materialized-mysql.md +++ b/docs/ru/engines/database-engines/materialized-mysql.md @@ -5,7 +5,7 @@ toc_title: "[experimental] MaterializedMySQL" # [экспериментальный] MaterializedMySQL {#materialized-mysql} -**Это экспериментальная функция, которую не следует использовать в продуктивной среде.** +**Это экспериментальный движок, который не следует использовать в продуктивной среде.** Создает базу данных ClickHouse со всеми таблицами, существующими в MySQL, и всеми данными в этих таблицах. From ecc1a882fcc6b74c96fd570839c1d9106982979a Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Thu, 12 Aug 2021 09:46:03 +0300 Subject: [PATCH 078/161] Update docs/ru/engines/database-engines/materialized-mysql.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/engines/database-engines/materialized-mysql.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/engines/database-engines/materialized-mysql.md b/docs/ru/engines/database-engines/materialized-mysql.md index ff15af05397..63cf5039937 100644 --- a/docs/ru/engines/database-engines/materialized-mysql.md +++ b/docs/ru/engines/database-engines/materialized-mysql.md @@ -33,7 +33,7 @@ ENGINE = MaterializedMySQL('host:port', ['database' | database], 'user', 'passwo - `max_bytes_in_buffers` — максимальное количество байтов, которое разрешено кешировать данным в памяти (для базы данных и данных кеша, которые невозможно запросить). При превышении количества строк, данные будут материализованы. Значение по умолчанию: `1 048 576`. - `max_flush_data_time` — максимальное время в миллисекундах, в течение которого разрешено кешировать данные в памяти (для базы данных и данных кеша, которые невозможно запросить). При превышении количества указанного периода, данные будут материализованы. Значение по умолчанию: `1000`. - `max_wait_time_when_mysql_unavailable` — интервал между повторными попытками, если MySQL недоступен. Указывается в миллисекундах. Отрицательное значение отключает повторные попытки. Значение по умолчанию: `1000`. -- `allows_query_when_mysql_lost` — разрешить запрос материализованной таблицы при потере MySQL. Значение по умолчанию: `0` (`false`). +- `allows_query_when_mysql_lost` — признак, разрешен ли запрос к материализованной таблице при потере соединения с MySQL. Значение по умолчанию: `0` (`false`). ```sql CREATE DATABASE mysql ENGINE = MaterializedMySQL('localhost:3306', 'db', 'user', '***') From 77f343267aad45bd874df37c6b9cfdc9997ff4f4 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Thu, 12 Aug 2021 09:46:16 +0300 Subject: [PATCH 079/161] Update docs/en/engines/database-engines/materialized-mysql.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/engines/database-engines/materialized-mysql.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/database-engines/materialized-mysql.md b/docs/en/engines/database-engines/materialized-mysql.md index 5fab87d4d8c..0937fe20dd4 100644 --- a/docs/en/engines/database-engines/materialized-mysql.md +++ b/docs/en/engines/database-engines/materialized-mysql.md @@ -45,7 +45,7 @@ CREATE DATABASE mysql ENGINE = MaterializedMySQL('localhost:3306', 'db', 'user', **Settings on MySQL-server Side** -For the correct work of `MaterializeMySQL`, there are few mandatory `MySQL`-side configuration settings that must be set: +For the correct work of `MaterializedMySQL`, there are few mandatory `MySQL`-side configuration settings that must be set: - `default_authentication_plugin = mysql_native_password` since `MaterializeMySQL` can only authorize with this method. - `gtid_mode = on` since GTID based logging is a mandatory for providing correct `MaterializeMySQL` replication. From 3b85c3c8ac9b209a5d25025d86fd6980e96e1f7f Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Thu, 12 Aug 2021 09:50:56 +0300 Subject: [PATCH 080/161] Update docs/ru/engines/database-engines/materialized-mysql.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/engines/database-engines/materialized-mysql.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/engines/database-engines/materialized-mysql.md b/docs/ru/engines/database-engines/materialized-mysql.md index 63cf5039937..d3f764b3067 100644 --- a/docs/ru/engines/database-engines/materialized-mysql.md +++ b/docs/ru/engines/database-engines/materialized-mysql.md @@ -45,7 +45,7 @@ CREATE DATABASE mysql ENGINE = MaterializedMySQL('localhost:3306', 'db', 'user', **Настройки на стороне MySQL-сервера** Для правильной работы `MaterializeMySQL` следует обязательно указать на сервере MySQL следующие параметры конфигурации: -- `default_authentication_plugin = mysql_native_password` — `MaterializeMySQL` может авторизоваться только с помощью этого метода. +- `default_authentication_plugin = mysql_native_password` — `MaterializedMySQL` может авторизоваться только с помощью этого метода. - `gtid_mode = on` — ведение журнала на основе GTID является обязательным для обеспечения правильной репликации. !!! attention "Внимание" From 9230ee317547077aed546c6e9702d8be8c4a7b4f Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Thu, 12 Aug 2021 09:51:05 +0300 Subject: [PATCH 081/161] Update docs/en/engines/database-engines/materialized-mysql.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/engines/database-engines/materialized-mysql.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/database-engines/materialized-mysql.md b/docs/en/engines/database-engines/materialized-mysql.md index 0937fe20dd4..72688f056ac 100644 --- a/docs/en/engines/database-engines/materialized-mysql.md +++ b/docs/en/engines/database-engines/materialized-mysql.md @@ -47,7 +47,7 @@ CREATE DATABASE mysql ENGINE = MaterializedMySQL('localhost:3306', 'db', 'user', For the correct work of `MaterializedMySQL`, there are few mandatory `MySQL`-side configuration settings that must be set: -- `default_authentication_plugin = mysql_native_password` since `MaterializeMySQL` can only authorize with this method. +- `default_authentication_plugin = mysql_native_password` since `MaterializedMySQL` can only authorize with this method. - `gtid_mode = on` since GTID based logging is a mandatory for providing correct `MaterializeMySQL` replication. !!! attention "Attention" From 42da5f488dfe46a951b583fb1ad124f8eb83941f Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Thu, 12 Aug 2021 09:51:22 +0300 Subject: [PATCH 082/161] Update docs/en/engines/database-engines/materialized-mysql.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/engines/database-engines/materialized-mysql.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/database-engines/materialized-mysql.md b/docs/en/engines/database-engines/materialized-mysql.md index 72688f056ac..d329dff32c5 100644 --- a/docs/en/engines/database-engines/materialized-mysql.md +++ b/docs/en/engines/database-engines/materialized-mysql.md @@ -48,7 +48,7 @@ CREATE DATABASE mysql ENGINE = MaterializedMySQL('localhost:3306', 'db', 'user', For the correct work of `MaterializedMySQL`, there are few mandatory `MySQL`-side configuration settings that must be set: - `default_authentication_plugin = mysql_native_password` since `MaterializedMySQL` can only authorize with this method. -- `gtid_mode = on` since GTID based logging is a mandatory for providing correct `MaterializeMySQL` replication. +- `gtid_mode = on` since GTID based logging is a mandatory for providing correct `MaterializedMySQL` replication. !!! attention "Attention" While turning on `gtid_mode` you should also specify `enforce_gtid_consistency = on`. From 9bd6db00f99bddd96f09cc7a19a1b9efeed4904d Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Thu, 12 Aug 2021 09:51:31 +0300 Subject: [PATCH 083/161] Update docs/ru/sql-reference/functions/string-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index 5b166c5bca6..48391fe697e 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -53,7 +53,7 @@ SELECT notempty('text'); empty(x) ``` -Строка считается непустой, если содержит хотя бы один байт, пусть даже это пробел или нулевой байт. UUID считается пустой, если он содержит только нули (нулевой UUID). +Строка считается непустой, если содержит хотя бы один байт, пусть даже это пробел или нулевой байт. **Параметры** From 902fb83492069c1ad1edf471868306645d388bf4 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Thu, 12 Aug 2021 09:51:39 +0300 Subject: [PATCH 084/161] Update docs/ru/sql-reference/functions/string-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index 48391fe697e..06edab23325 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -57,7 +57,7 @@ empty(x) **Параметры** -- `x` — Входная строка. [Array](../data-types/array.md), [String](../data-types/string.md), [UUID](../data-types/uuid.md). +- `x` — Входная строка. [String](../data-types/string.md). **Возвращаемое значение** From 0798a3343e196c3140c683569dcb6601963c4fb4 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Thu, 12 Aug 2021 09:51:47 +0300 Subject: [PATCH 085/161] Update docs/ru/sql-reference/functions/string-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index 06edab23325..6f6b6289e28 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -50,7 +50,7 @@ SELECT notempty('text'); **Синтаксис** ``` sql -empty(x) +notEmpty(x) ``` Строка считается непустой, если содержит хотя бы один байт, пусть даже это пробел или нулевой байт. From 79f69034270c6b7b72577a524f5ce3f521d6fd80 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Thu, 12 Aug 2021 09:51:54 +0300 Subject: [PATCH 086/161] Update docs/ru/sql-reference/functions/string-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index 6f6b6289e28..fcf36308703 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -19,7 +19,7 @@ empty(x) **Параметры** -- `x` — Входная строка. [Array](../data-types/array.md), [String](../data-types/string.md), [UUID](../data-types/uuid.md). +- `x` — Входная строка. [String](../data-types/string.md). **Возвращаемое значение** From 7a8cc49e93806c9bb6ed052afb988b9711c66482 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Thu, 12 Aug 2021 09:52:06 +0300 Subject: [PATCH 087/161] Update docs/ru/sql-reference/functions/string-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index fcf36308703..75a4af84297 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -15,7 +15,7 @@ toc_title: "Функции для работы со строками" empty(x) ``` -Строка считается непустой, если содержит хотя бы один байт, пусть даже это пробел или нулевой байт. UUID считается пустым, если он содержит только нули (нулевой UUID). +Строка считается непустой, если содержит хотя бы один байт, пусть даже это пробел или нулевой байт. **Параметры** From 31e7e5cb3af63747c06e9dcf736edb6d673a1e6e Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Thu, 12 Aug 2021 09:52:16 +0300 Subject: [PATCH 088/161] Update docs/ru/engines/database-engines/materialized-mysql.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/engines/database-engines/materialized-mysql.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/engines/database-engines/materialized-mysql.md b/docs/ru/engines/database-engines/materialized-mysql.md index d3f764b3067..b82a3dae553 100644 --- a/docs/ru/engines/database-engines/materialized-mysql.md +++ b/docs/ru/engines/database-engines/materialized-mysql.md @@ -44,7 +44,7 @@ CREATE DATABASE mysql ENGINE = MaterializedMySQL('localhost:3306', 'db', 'user', **Настройки на стороне MySQL-сервера** -Для правильной работы `MaterializeMySQL` следует обязательно указать на сервере MySQL следующие параметры конфигурации: +Для правильной работы `MaterializedMySQL` следует обязательно указать на сервере MySQL следующие параметры конфигурации: - `default_authentication_plugin = mysql_native_password` — `MaterializedMySQL` может авторизоваться только с помощью этого метода. - `gtid_mode = on` — ведение журнала на основе GTID является обязательным для обеспечения правильной репликации. From 014d3c889f4985c340e396dd8b6c383b73c270b8 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 12 Aug 2021 12:02:52 +0300 Subject: [PATCH 089/161] Fix ya.make --- src/Dictionaries/ya.make | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Dictionaries/ya.make b/src/Dictionaries/ya.make index 36152fe439a..ad4ed6e1af7 100644 --- a/src/Dictionaries/ya.make +++ b/src/Dictionaries/ya.make @@ -26,9 +26,9 @@ SRCS( CassandraDictionarySource.cpp CassandraHelpers.cpp ClickHouseDictionarySource.cpp - DictionaryBlockInputStream.cpp - DictionaryBlockInputStreamBase.cpp DictionaryFactory.cpp + DictionarySource.cpp + DictionarySourceBase.cpp DictionarySourceFactory.cpp DictionarySourceHelpers.cpp DictionaryStructure.cpp From 031c15cb0e35b6a5b2bff81442725bb7cb6280bb Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 12 Aug 2021 12:18:20 +0300 Subject: [PATCH 090/161] Add comment. --- src/Common/DenseHashMap.h | 5 +++++ src/Common/DenseHashSet.h | 2 ++ src/Common/SparseHashMap.h | 2 ++ 3 files changed, 9 insertions(+) diff --git a/src/Common/DenseHashMap.h b/src/Common/DenseHashMap.h index 26b9224528b..9ac21c82676 100644 --- a/src/Common/DenseHashMap.h +++ b/src/Common/DenseHashMap.h @@ -1,6 +1,11 @@ #pragma once #include +/// DenseHashMap is a wrapper for google::dense_hash_map. +/// Some hacks are needed to make it work in "Arcadia". +/// "Arcadia" is a proprietary monorepository in Yandex. +/// It uses slightly changed version of sparsehash with a different set of hash functions (which we don't need). +/// Those defines are needed to make it compile. #if defined(ARCADIA_BUILD) #define HASH_FUN_H template diff --git a/src/Common/DenseHashSet.h b/src/Common/DenseHashSet.h index 5b30d88a70c..e8c06f36aa3 100644 --- a/src/Common/DenseHashSet.h +++ b/src/Common/DenseHashSet.h @@ -1,5 +1,7 @@ #pragma once +/// DenseHashSet is a wrapper for google::dense_hash_set. +/// See comment in DenseHashMap.h #if defined(ARCADIA_BUILD) #define HASH_FUN_H template diff --git a/src/Common/SparseHashMap.h b/src/Common/SparseHashMap.h index 403042c11a9..f01fc633d84 100644 --- a/src/Common/SparseHashMap.h +++ b/src/Common/SparseHashMap.h @@ -1,5 +1,7 @@ #pragma once +/// SparseHashMap is a wrapper for google::sparse_hash_map. +/// See comment in DenseHashMap.h #if defined(ARCADIA_BUILD) #define HASH_FUN_H template From 88b4200008ded99ded7dd1ef3315423cc0d819d9 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 12 Aug 2021 13:40:54 +0300 Subject: [PATCH 091/161] fix --- tests/integration/ci-runner.py | 26 ++++++++++++++++++++++---- 1 file changed, 22 insertions(+), 4 deletions(-) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index 12ee436e400..bf7549a83c4 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -125,13 +125,13 @@ def clear_ip_tables_and_restart_daemons(): logging.info("Dump iptables after run %s", subprocess.check_output("iptables -L", shell=True)) try: logging.info("Killing all alive docker containers") - subprocess.check_output("docker kill $(docker ps -q)", shell=True) + subprocess.check_output("timeout -s 9 10m docker kill $(docker ps -q)", shell=True) except subprocess.CalledProcessError as err: logging.info("docker kill excepted: " + str(err)) try: logging.info("Removing all docker containers") - subprocess.check_output("docker rm $(docker ps -a -q) --force", shell=True) + subprocess.check_output("timeout -s 9 10m docker rm $(docker ps -a -q) --force", shell=True) except subprocess.CalledProcessError as err: logging.info("docker rm excepted: " + str(err)) @@ -376,6 +376,24 @@ class ClickhouseIntegrationTestsRunner: res.add(path) return res + def try_run_test_group(self, repo_path, test_group, tests_in_group, num_tries, num_workers): + try: + return self.run_test_group(repo_path, test_group, tests_in_group, num_tries, num_workers) + except Exception as e: + logging.info("Failed to run {}:\n{}".format(str(test_group), str(e))) + counters = { + "ERROR": [], + "PASSED": [], + "FAILED": [], + "SKIPPED": [], + "FLAKY": [], + } + tests_times = defaultdict(float) + for test in tests_in_group: + counters["ERROR"].append(test) + tests_times[test] = 0 + return counters, tests_times, [] + def run_test_group(self, repo_path, test_group, tests_in_group, num_tries, num_workers): counters = { "ERROR": [], @@ -507,7 +525,7 @@ class ClickhouseIntegrationTestsRunner: for i in range(TRIES_COUNT): final_retry += 1 logging.info("Running tests for the %s time", i) - counters, tests_times, log_paths = self.run_test_group(repo_path, "flaky", tests_to_run, 1, 1) + counters, tests_times, log_paths = self.try_run_test_group(repo_path, "flaky", tests_to_run, 1, 1) logs += log_paths if counters["FAILED"]: logging.info("Found failed tests: %s", ' '.join(counters["FAILED"])) @@ -583,7 +601,7 @@ class ClickhouseIntegrationTestsRunner: for group, tests in items_to_run: logging.info("Running test group %s countaining %s tests", group, len(tests)) - group_counters, group_test_times, log_paths = self.run_test_group(repo_path, group, tests, MAX_RETRY, NUM_WORKERS) + group_counters, group_test_times, log_paths = self.try_run_test_group(repo_path, group, tests, MAX_RETRY, NUM_WORKERS) total_tests = 0 for counter, value in group_counters.items(): logging.info("Tests from group %s stats, %s count %s", group, counter, len(value)) From 9b81c075c636d170ba3aba1e26ef10d5161aacb6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 12 Aug 2021 12:53:32 +0200 Subject: [PATCH 092/161] Use today() to ensure the rollup is tested properly --- .../0_stateless/01236_graphite_mt.reference | 688 +++++++++--------- .../queries/0_stateless/01236_graphite_mt.sql | 46 +- 2 files changed, 371 insertions(+), 363 deletions(-) diff --git a/tests/queries/0_stateless/01236_graphite_mt.reference b/tests/queries/0_stateless/01236_graphite_mt.reference index c68becad53d..a30d2495265 100644 --- a/tests/queries/0_stateless/01236_graphite_mt.reference +++ b/tests/queries/0_stateless/01236_graphite_mt.reference @@ -1,344 +1,344 @@ -1 max_1 2021-08-09 23:50:00 9 1 0 -1 max_1 2021-08-09 23:40:00 19 1 10 -1 max_1 2021-08-09 23:30:00 29 1 20 -1 max_1 2021-08-09 23:20:00 39 1 30 -1 max_1 2021-08-09 23:10:00 49 1 40 -1 max_1 2021-08-09 23:00:00 59 1 50 -1 max_1 2021-08-09 22:50:00 69 1 60 -1 max_1 2021-08-09 22:40:00 79 1 70 -1 max_1 2021-08-09 22:30:00 89 1 80 -1 max_1 2021-08-09 22:20:00 99 1 90 -1 max_1 2021-08-09 22:10:00 109 1 100 -1 max_1 2021-08-09 22:00:00 119 1 110 -1 max_1 2021-08-09 21:50:00 129 1 120 -1 max_1 2021-08-09 21:40:00 139 1 130 -1 max_1 2021-08-09 21:30:00 149 1 140 -1 max_1 2021-08-09 21:20:00 159 1 150 -1 max_1 2021-08-09 21:10:00 169 1 160 -1 max_1 2021-08-09 21:00:00 179 1 170 -1 max_1 2021-08-09 20:50:00 189 1 180 -1 max_1 2021-08-09 20:40:00 199 1 190 -1 max_1 2021-08-09 20:30:00 209 1 200 -1 max_1 2021-08-09 20:20:00 219 1 210 -1 max_1 2021-08-09 20:10:00 229 1 220 -1 max_1 2021-08-09 20:00:00 239 1 230 -1 max_1 2021-08-09 19:50:00 249 1 240 -1 max_1 2021-08-09 19:40:00 259 1 250 -1 max_1 2021-08-09 19:30:00 269 1 260 -1 max_1 2021-08-09 19:20:00 279 1 270 -1 max_1 2021-08-09 19:10:00 289 1 280 -1 max_1 2021-08-09 19:00:00 299 1 290 -1 max_1 2021-08-06 23:20:00 39 1 0 -1 max_1 2021-08-06 21:40:00 139 1 40 -1 max_1 2021-08-06 20:00:00 239 1 140 -1 max_1 2021-08-06 18:20:00 339 1 240 -1 max_1 2021-08-06 16:40:00 439 1 340 -1 max_1 2021-08-06 15:00:00 539 1 440 -1 max_1 2021-08-06 13:20:00 639 1 540 -1 max_1 2021-08-06 11:40:00 739 1 640 -1 max_1 2021-08-06 10:00:00 839 1 740 -1 max_1 2021-08-06 08:20:00 939 1 840 -1 max_1 2021-08-06 06:40:00 1039 1 940 -1 max_1 2021-08-06 05:00:00 1139 1 1040 -1 max_1 2021-08-06 03:20:00 1199 1 1140 -1 max_2 2021-08-09 23:50:00 9 1 0 -1 max_2 2021-08-09 23:40:00 19 1 10 -1 max_2 2021-08-09 23:30:00 29 1 20 -1 max_2 2021-08-09 23:20:00 39 1 30 -1 max_2 2021-08-09 23:10:00 49 1 40 -1 max_2 2021-08-09 23:00:00 59 1 50 -1 max_2 2021-08-09 22:50:00 69 1 60 -1 max_2 2021-08-09 22:40:00 79 1 70 -1 max_2 2021-08-09 22:30:00 89 1 80 -1 max_2 2021-08-09 22:20:00 99 1 90 -1 max_2 2021-08-09 22:10:00 109 1 100 -1 max_2 2021-08-09 22:00:00 119 1 110 -1 max_2 2021-08-09 21:50:00 129 1 120 -1 max_2 2021-08-09 21:40:00 139 1 130 -1 max_2 2021-08-09 21:30:00 149 1 140 -1 max_2 2021-08-09 21:20:00 159 1 150 -1 max_2 2021-08-09 21:10:00 169 1 160 -1 max_2 2021-08-09 21:00:00 179 1 170 -1 max_2 2021-08-09 20:50:00 189 1 180 -1 max_2 2021-08-09 20:40:00 199 1 190 -1 max_2 2021-08-09 20:30:00 209 1 200 -1 max_2 2021-08-09 20:20:00 219 1 210 -1 max_2 2021-08-09 20:10:00 229 1 220 -1 max_2 2021-08-09 20:00:00 239 1 230 -1 max_2 2021-08-09 19:50:00 249 1 240 -1 max_2 2021-08-09 19:40:00 259 1 250 -1 max_2 2021-08-09 19:30:00 269 1 260 -1 max_2 2021-08-09 19:20:00 279 1 270 -1 max_2 2021-08-09 19:10:00 289 1 280 -1 max_2 2021-08-09 19:00:00 299 1 290 -1 max_2 2021-08-06 23:20:00 39 1 0 -1 max_2 2021-08-06 21:40:00 139 1 40 -1 max_2 2021-08-06 20:00:00 239 1 140 -1 max_2 2021-08-06 18:20:00 339 1 240 -1 max_2 2021-08-06 16:40:00 439 1 340 -1 max_2 2021-08-06 15:00:00 539 1 440 -1 max_2 2021-08-06 13:20:00 639 1 540 -1 max_2 2021-08-06 11:40:00 739 1 640 -1 max_2 2021-08-06 10:00:00 839 1 740 -1 max_2 2021-08-06 08:20:00 939 1 840 -1 max_2 2021-08-06 06:40:00 1039 1 940 -1 max_2 2021-08-06 05:00:00 1139 1 1040 -1 max_2 2021-08-06 03:20:00 1199 1 1140 -1 sum_1 2021-08-09 23:50:00 45 1 0 -1 sum_1 2021-08-09 23:40:00 145 1 10 -1 sum_1 2021-08-09 23:30:00 245 1 20 -1 sum_1 2021-08-09 23:20:00 345 1 30 -1 sum_1 2021-08-09 23:10:00 445 1 40 -1 sum_1 2021-08-09 23:00:00 545 1 50 -1 sum_1 2021-08-09 22:50:00 645 1 60 -1 sum_1 2021-08-09 22:40:00 745 1 70 -1 sum_1 2021-08-09 22:30:00 845 1 80 -1 sum_1 2021-08-09 22:20:00 945 1 90 -1 sum_1 2021-08-09 22:10:00 1045 1 100 -1 sum_1 2021-08-09 22:00:00 1145 1 110 -1 sum_1 2021-08-09 21:50:00 1245 1 120 -1 sum_1 2021-08-09 21:40:00 1345 1 130 -1 sum_1 2021-08-09 21:30:00 1445 1 140 -1 sum_1 2021-08-09 21:20:00 1545 1 150 -1 sum_1 2021-08-09 21:10:00 1645 1 160 -1 sum_1 2021-08-09 21:00:00 1745 1 170 -1 sum_1 2021-08-09 20:50:00 1845 1 180 -1 sum_1 2021-08-09 20:40:00 1945 1 190 -1 sum_1 2021-08-09 20:30:00 2045 1 200 -1 sum_1 2021-08-09 20:20:00 2145 1 210 -1 sum_1 2021-08-09 20:10:00 2245 1 220 -1 sum_1 2021-08-09 20:00:00 2345 1 230 -1 sum_1 2021-08-09 19:50:00 2445 1 240 -1 sum_1 2021-08-09 19:40:00 2545 1 250 -1 sum_1 2021-08-09 19:30:00 2645 1 260 -1 sum_1 2021-08-09 19:20:00 2745 1 270 -1 sum_1 2021-08-09 19:10:00 2845 1 280 -1 sum_1 2021-08-09 19:00:00 2945 1 290 -1 sum_1 2021-08-06 23:20:00 780 1 0 -1 sum_1 2021-08-06 21:40:00 8950 1 40 -1 sum_1 2021-08-06 20:00:00 18950 1 140 -1 sum_1 2021-08-06 18:20:00 28950 1 240 -1 sum_1 2021-08-06 16:40:00 38950 1 340 -1 sum_1 2021-08-06 15:00:00 48950 1 440 -1 sum_1 2021-08-06 13:20:00 58950 1 540 -1 sum_1 2021-08-06 11:40:00 68950 1 640 -1 sum_1 2021-08-06 10:00:00 78950 1 740 -1 sum_1 2021-08-06 08:20:00 88950 1 840 -1 sum_1 2021-08-06 06:40:00 98950 1 940 -1 sum_1 2021-08-06 05:00:00 108950 1 1040 -1 sum_1 2021-08-06 03:20:00 70170 1 1140 -1 sum_2 2021-08-09 23:50:00 45 1 0 -1 sum_2 2021-08-09 23:40:00 145 1 10 -1 sum_2 2021-08-09 23:30:00 245 1 20 -1 sum_2 2021-08-09 23:20:00 345 1 30 -1 sum_2 2021-08-09 23:10:00 445 1 40 -1 sum_2 2021-08-09 23:00:00 545 1 50 -1 sum_2 2021-08-09 22:50:00 645 1 60 -1 sum_2 2021-08-09 22:40:00 745 1 70 -1 sum_2 2021-08-09 22:30:00 845 1 80 -1 sum_2 2021-08-09 22:20:00 945 1 90 -1 sum_2 2021-08-09 22:10:00 1045 1 100 -1 sum_2 2021-08-09 22:00:00 1145 1 110 -1 sum_2 2021-08-09 21:50:00 1245 1 120 -1 sum_2 2021-08-09 21:40:00 1345 1 130 -1 sum_2 2021-08-09 21:30:00 1445 1 140 -1 sum_2 2021-08-09 21:20:00 1545 1 150 -1 sum_2 2021-08-09 21:10:00 1645 1 160 -1 sum_2 2021-08-09 21:00:00 1745 1 170 -1 sum_2 2021-08-09 20:50:00 1845 1 180 -1 sum_2 2021-08-09 20:40:00 1945 1 190 -1 sum_2 2021-08-09 20:30:00 2045 1 200 -1 sum_2 2021-08-09 20:20:00 2145 1 210 -1 sum_2 2021-08-09 20:10:00 2245 1 220 -1 sum_2 2021-08-09 20:00:00 2345 1 230 -1 sum_2 2021-08-09 19:50:00 2445 1 240 -1 sum_2 2021-08-09 19:40:00 2545 1 250 -1 sum_2 2021-08-09 19:30:00 2645 1 260 -1 sum_2 2021-08-09 19:20:00 2745 1 270 -1 sum_2 2021-08-09 19:10:00 2845 1 280 -1 sum_2 2021-08-09 19:00:00 2945 1 290 -1 sum_2 2021-08-06 23:20:00 780 1 0 -1 sum_2 2021-08-06 21:40:00 8950 1 40 -1 sum_2 2021-08-06 20:00:00 18950 1 140 -1 sum_2 2021-08-06 18:20:00 28950 1 240 -1 sum_2 2021-08-06 16:40:00 38950 1 340 -1 sum_2 2021-08-06 15:00:00 48950 1 440 -1 sum_2 2021-08-06 13:20:00 58950 1 540 -1 sum_2 2021-08-06 11:40:00 68950 1 640 -1 sum_2 2021-08-06 10:00:00 78950 1 740 -1 sum_2 2021-08-06 08:20:00 88950 1 840 -1 sum_2 2021-08-06 06:40:00 98950 1 940 -1 sum_2 2021-08-06 05:00:00 108950 1 1040 -1 sum_2 2021-08-06 03:20:00 70170 1 1140 -2 max_1 2021-08-09 23:50:00 9 1 0 -2 max_1 2021-08-09 23:40:00 19 1 10 -2 max_1 2021-08-09 23:30:00 29 1 20 -2 max_1 2021-08-09 23:20:00 39 1 30 -2 max_1 2021-08-09 23:10:00 49 1 40 -2 max_1 2021-08-09 23:00:00 59 1 50 -2 max_1 2021-08-09 22:50:00 69 1 60 -2 max_1 2021-08-09 22:40:00 79 1 70 -2 max_1 2021-08-09 22:30:00 89 1 80 -2 max_1 2021-08-09 22:20:00 99 1 90 -2 max_1 2021-08-09 22:10:00 109 1 100 -2 max_1 2021-08-09 22:00:00 119 1 110 -2 max_1 2021-08-09 21:50:00 129 1 120 -2 max_1 2021-08-09 21:40:00 139 1 130 -2 max_1 2021-08-09 21:30:00 149 1 140 -2 max_1 2021-08-09 21:20:00 159 1 150 -2 max_1 2021-08-09 21:10:00 169 1 160 -2 max_1 2021-08-09 21:00:00 179 1 170 -2 max_1 2021-08-09 20:50:00 189 1 180 -2 max_1 2021-08-09 20:40:00 199 1 190 -2 max_1 2021-08-09 20:30:00 209 1 200 -2 max_1 2021-08-09 20:20:00 219 1 210 -2 max_1 2021-08-09 20:10:00 229 1 220 -2 max_1 2021-08-09 20:00:00 239 1 230 -2 max_1 2021-08-09 19:50:00 249 1 240 -2 max_1 2021-08-09 19:40:00 259 1 250 -2 max_1 2021-08-09 19:30:00 269 1 260 -2 max_1 2021-08-09 19:20:00 279 1 270 -2 max_1 2021-08-09 19:10:00 289 1 280 -2 max_1 2021-08-09 19:00:00 299 1 290 -2 max_1 2021-08-06 23:20:00 39 1 0 -2 max_1 2021-08-06 21:40:00 139 1 40 -2 max_1 2021-08-06 20:00:00 239 1 140 -2 max_1 2021-08-06 18:20:00 339 1 240 -2 max_1 2021-08-06 16:40:00 439 1 340 -2 max_1 2021-08-06 15:00:00 539 1 440 -2 max_1 2021-08-06 13:20:00 639 1 540 -2 max_1 2021-08-06 11:40:00 739 1 640 -2 max_1 2021-08-06 10:00:00 839 1 740 -2 max_1 2021-08-06 08:20:00 939 1 840 -2 max_1 2021-08-06 06:40:00 1039 1 940 -2 max_1 2021-08-06 05:00:00 1139 1 1040 -2 max_1 2021-08-06 03:20:00 1199 1 1140 -2 max_2 2021-08-09 23:50:00 9 1 0 -2 max_2 2021-08-09 23:40:00 19 1 10 -2 max_2 2021-08-09 23:30:00 29 1 20 -2 max_2 2021-08-09 23:20:00 39 1 30 -2 max_2 2021-08-09 23:10:00 49 1 40 -2 max_2 2021-08-09 23:00:00 59 1 50 -2 max_2 2021-08-09 22:50:00 69 1 60 -2 max_2 2021-08-09 22:40:00 79 1 70 -2 max_2 2021-08-09 22:30:00 89 1 80 -2 max_2 2021-08-09 22:20:00 99 1 90 -2 max_2 2021-08-09 22:10:00 109 1 100 -2 max_2 2021-08-09 22:00:00 119 1 110 -2 max_2 2021-08-09 21:50:00 129 1 120 -2 max_2 2021-08-09 21:40:00 139 1 130 -2 max_2 2021-08-09 21:30:00 149 1 140 -2 max_2 2021-08-09 21:20:00 159 1 150 -2 max_2 2021-08-09 21:10:00 169 1 160 -2 max_2 2021-08-09 21:00:00 179 1 170 -2 max_2 2021-08-09 20:50:00 189 1 180 -2 max_2 2021-08-09 20:40:00 199 1 190 -2 max_2 2021-08-09 20:30:00 209 1 200 -2 max_2 2021-08-09 20:20:00 219 1 210 -2 max_2 2021-08-09 20:10:00 229 1 220 -2 max_2 2021-08-09 20:00:00 239 1 230 -2 max_2 2021-08-09 19:50:00 249 1 240 -2 max_2 2021-08-09 19:40:00 259 1 250 -2 max_2 2021-08-09 19:30:00 269 1 260 -2 max_2 2021-08-09 19:20:00 279 1 270 -2 max_2 2021-08-09 19:10:00 289 1 280 -2 max_2 2021-08-09 19:00:00 299 1 290 -2 max_2 2021-08-06 23:20:00 39 1 0 -2 max_2 2021-08-06 21:40:00 139 1 40 -2 max_2 2021-08-06 20:00:00 239 1 140 -2 max_2 2021-08-06 18:20:00 339 1 240 -2 max_2 2021-08-06 16:40:00 439 1 340 -2 max_2 2021-08-06 15:00:00 539 1 440 -2 max_2 2021-08-06 13:20:00 639 1 540 -2 max_2 2021-08-06 11:40:00 739 1 640 -2 max_2 2021-08-06 10:00:00 839 1 740 -2 max_2 2021-08-06 08:20:00 939 1 840 -2 max_2 2021-08-06 06:40:00 1039 1 940 -2 max_2 2021-08-06 05:00:00 1139 1 1040 -2 max_2 2021-08-06 03:20:00 1199 1 1140 -2 sum_1 2021-08-09 23:50:00 45 1 0 -2 sum_1 2021-08-09 23:40:00 145 1 10 -2 sum_1 2021-08-09 23:30:00 245 1 20 -2 sum_1 2021-08-09 23:20:00 345 1 30 -2 sum_1 2021-08-09 23:10:00 445 1 40 -2 sum_1 2021-08-09 23:00:00 545 1 50 -2 sum_1 2021-08-09 22:50:00 645 1 60 -2 sum_1 2021-08-09 22:40:00 745 1 70 -2 sum_1 2021-08-09 22:30:00 845 1 80 -2 sum_1 2021-08-09 22:20:00 945 1 90 -2 sum_1 2021-08-09 22:10:00 1045 1 100 -2 sum_1 2021-08-09 22:00:00 1145 1 110 -2 sum_1 2021-08-09 21:50:00 1245 1 120 -2 sum_1 2021-08-09 21:40:00 1345 1 130 -2 sum_1 2021-08-09 21:30:00 1445 1 140 -2 sum_1 2021-08-09 21:20:00 1545 1 150 -2 sum_1 2021-08-09 21:10:00 1645 1 160 -2 sum_1 2021-08-09 21:00:00 1745 1 170 -2 sum_1 2021-08-09 20:50:00 1845 1 180 -2 sum_1 2021-08-09 20:40:00 1945 1 190 -2 sum_1 2021-08-09 20:30:00 2045 1 200 -2 sum_1 2021-08-09 20:20:00 2145 1 210 -2 sum_1 2021-08-09 20:10:00 2245 1 220 -2 sum_1 2021-08-09 20:00:00 2345 1 230 -2 sum_1 2021-08-09 19:50:00 2445 1 240 -2 sum_1 2021-08-09 19:40:00 2545 1 250 -2 sum_1 2021-08-09 19:30:00 2645 1 260 -2 sum_1 2021-08-09 19:20:00 2745 1 270 -2 sum_1 2021-08-09 19:10:00 2845 1 280 -2 sum_1 2021-08-09 19:00:00 2945 1 290 -2 sum_1 2021-08-06 23:20:00 780 1 0 -2 sum_1 2021-08-06 21:40:00 8950 1 40 -2 sum_1 2021-08-06 20:00:00 18950 1 140 -2 sum_1 2021-08-06 18:20:00 28950 1 240 -2 sum_1 2021-08-06 16:40:00 38950 1 340 -2 sum_1 2021-08-06 15:00:00 48950 1 440 -2 sum_1 2021-08-06 13:20:00 58950 1 540 -2 sum_1 2021-08-06 11:40:00 68950 1 640 -2 sum_1 2021-08-06 10:00:00 78950 1 740 -2 sum_1 2021-08-06 08:20:00 88950 1 840 -2 sum_1 2021-08-06 06:40:00 98950 1 940 -2 sum_1 2021-08-06 05:00:00 108950 1 1040 -2 sum_1 2021-08-06 03:20:00 70170 1 1140 -2 sum_2 2021-08-09 23:50:00 45 1 0 -2 sum_2 2021-08-09 23:40:00 145 1 10 -2 sum_2 2021-08-09 23:30:00 245 1 20 -2 sum_2 2021-08-09 23:20:00 345 1 30 -2 sum_2 2021-08-09 23:10:00 445 1 40 -2 sum_2 2021-08-09 23:00:00 545 1 50 -2 sum_2 2021-08-09 22:50:00 645 1 60 -2 sum_2 2021-08-09 22:40:00 745 1 70 -2 sum_2 2021-08-09 22:30:00 845 1 80 -2 sum_2 2021-08-09 22:20:00 945 1 90 -2 sum_2 2021-08-09 22:10:00 1045 1 100 -2 sum_2 2021-08-09 22:00:00 1145 1 110 -2 sum_2 2021-08-09 21:50:00 1245 1 120 -2 sum_2 2021-08-09 21:40:00 1345 1 130 -2 sum_2 2021-08-09 21:30:00 1445 1 140 -2 sum_2 2021-08-09 21:20:00 1545 1 150 -2 sum_2 2021-08-09 21:10:00 1645 1 160 -2 sum_2 2021-08-09 21:00:00 1745 1 170 -2 sum_2 2021-08-09 20:50:00 1845 1 180 -2 sum_2 2021-08-09 20:40:00 1945 1 190 -2 sum_2 2021-08-09 20:30:00 2045 1 200 -2 sum_2 2021-08-09 20:20:00 2145 1 210 -2 sum_2 2021-08-09 20:10:00 2245 1 220 -2 sum_2 2021-08-09 20:00:00 2345 1 230 -2 sum_2 2021-08-09 19:50:00 2445 1 240 -2 sum_2 2021-08-09 19:40:00 2545 1 250 -2 sum_2 2021-08-09 19:30:00 2645 1 260 -2 sum_2 2021-08-09 19:20:00 2745 1 270 -2 sum_2 2021-08-09 19:10:00 2845 1 280 -2 sum_2 2021-08-09 19:00:00 2945 1 290 -2 sum_2 2021-08-06 23:20:00 780 1 0 -2 sum_2 2021-08-06 21:40:00 8950 1 40 -2 sum_2 2021-08-06 20:00:00 18950 1 140 -2 sum_2 2021-08-06 18:20:00 28950 1 240 -2 sum_2 2021-08-06 16:40:00 38950 1 340 -2 sum_2 2021-08-06 15:00:00 48950 1 440 -2 sum_2 2021-08-06 13:20:00 58950 1 540 -2 sum_2 2021-08-06 11:40:00 68950 1 640 -2 sum_2 2021-08-06 10:00:00 78950 1 740 -2 sum_2 2021-08-06 08:20:00 88950 1 840 -2 sum_2 2021-08-06 06:40:00 98950 1 940 -2 sum_2 2021-08-06 05:00:00 108950 1 1040 -2 sum_2 2021-08-06 03:20:00 70170 1 1140 +1 max_1 9 1 0 +1 max_1 19 1 10 +1 max_1 29 1 20 +1 max_1 39 1 30 +1 max_1 49 1 40 +1 max_1 59 1 50 +1 max_1 69 1 60 +1 max_1 79 1 70 +1 max_1 89 1 80 +1 max_1 99 1 90 +1 max_1 109 1 100 +1 max_1 119 1 110 +1 max_1 129 1 120 +1 max_1 139 1 130 +1 max_1 149 1 140 +1 max_1 159 1 150 +1 max_1 169 1 160 +1 max_1 179 1 170 +1 max_1 189 1 180 +1 max_1 199 1 190 +1 max_1 209 1 200 +1 max_1 219 1 210 +1 max_1 229 1 220 +1 max_1 239 1 230 +1 max_1 249 1 240 +1 max_1 259 1 250 +1 max_1 269 1 260 +1 max_1 279 1 270 +1 max_1 289 1 280 +1 max_1 299 1 290 +1 max_1 39 1 0 +1 max_1 139 1 40 +1 max_1 239 1 140 +1 max_1 339 1 240 +1 max_1 439 1 340 +1 max_1 539 1 440 +1 max_1 639 1 540 +1 max_1 739 1 640 +1 max_1 839 1 740 +1 max_1 939 1 840 +1 max_1 1039 1 940 +1 max_1 1139 1 1040 +1 max_1 1199 1 1140 +1 max_2 9 1 0 +1 max_2 19 1 10 +1 max_2 29 1 20 +1 max_2 39 1 30 +1 max_2 49 1 40 +1 max_2 59 1 50 +1 max_2 69 1 60 +1 max_2 79 1 70 +1 max_2 89 1 80 +1 max_2 99 1 90 +1 max_2 109 1 100 +1 max_2 119 1 110 +1 max_2 129 1 120 +1 max_2 139 1 130 +1 max_2 149 1 140 +1 max_2 159 1 150 +1 max_2 169 1 160 +1 max_2 179 1 170 +1 max_2 189 1 180 +1 max_2 199 1 190 +1 max_2 209 1 200 +1 max_2 219 1 210 +1 max_2 229 1 220 +1 max_2 239 1 230 +1 max_2 249 1 240 +1 max_2 259 1 250 +1 max_2 269 1 260 +1 max_2 279 1 270 +1 max_2 289 1 280 +1 max_2 299 1 290 +1 max_2 39 1 0 +1 max_2 139 1 40 +1 max_2 239 1 140 +1 max_2 339 1 240 +1 max_2 439 1 340 +1 max_2 539 1 440 +1 max_2 639 1 540 +1 max_2 739 1 640 +1 max_2 839 1 740 +1 max_2 939 1 840 +1 max_2 1039 1 940 +1 max_2 1139 1 1040 +1 max_2 1199 1 1140 +1 sum_1 45 1 0 +1 sum_1 145 1 10 +1 sum_1 245 1 20 +1 sum_1 345 1 30 +1 sum_1 445 1 40 +1 sum_1 545 1 50 +1 sum_1 645 1 60 +1 sum_1 745 1 70 +1 sum_1 845 1 80 +1 sum_1 945 1 90 +1 sum_1 1045 1 100 +1 sum_1 1145 1 110 +1 sum_1 1245 1 120 +1 sum_1 1345 1 130 +1 sum_1 1445 1 140 +1 sum_1 1545 1 150 +1 sum_1 1645 1 160 +1 sum_1 1745 1 170 +1 sum_1 1845 1 180 +1 sum_1 1945 1 190 +1 sum_1 2045 1 200 +1 sum_1 2145 1 210 +1 sum_1 2245 1 220 +1 sum_1 2345 1 230 +1 sum_1 2445 1 240 +1 sum_1 2545 1 250 +1 sum_1 2645 1 260 +1 sum_1 2745 1 270 +1 sum_1 2845 1 280 +1 sum_1 2945 1 290 +1 sum_1 780 1 0 +1 sum_1 8950 1 40 +1 sum_1 18950 1 140 +1 sum_1 28950 1 240 +1 sum_1 38950 1 340 +1 sum_1 48950 1 440 +1 sum_1 58950 1 540 +1 sum_1 68950 1 640 +1 sum_1 78950 1 740 +1 sum_1 88950 1 840 +1 sum_1 98950 1 940 +1 sum_1 108950 1 1040 +1 sum_1 70170 1 1140 +1 sum_2 45 1 0 +1 sum_2 145 1 10 +1 sum_2 245 1 20 +1 sum_2 345 1 30 +1 sum_2 445 1 40 +1 sum_2 545 1 50 +1 sum_2 645 1 60 +1 sum_2 745 1 70 +1 sum_2 845 1 80 +1 sum_2 945 1 90 +1 sum_2 1045 1 100 +1 sum_2 1145 1 110 +1 sum_2 1245 1 120 +1 sum_2 1345 1 130 +1 sum_2 1445 1 140 +1 sum_2 1545 1 150 +1 sum_2 1645 1 160 +1 sum_2 1745 1 170 +1 sum_2 1845 1 180 +1 sum_2 1945 1 190 +1 sum_2 2045 1 200 +1 sum_2 2145 1 210 +1 sum_2 2245 1 220 +1 sum_2 2345 1 230 +1 sum_2 2445 1 240 +1 sum_2 2545 1 250 +1 sum_2 2645 1 260 +1 sum_2 2745 1 270 +1 sum_2 2845 1 280 +1 sum_2 2945 1 290 +1 sum_2 780 1 0 +1 sum_2 8950 1 40 +1 sum_2 18950 1 140 +1 sum_2 28950 1 240 +1 sum_2 38950 1 340 +1 sum_2 48950 1 440 +1 sum_2 58950 1 540 +1 sum_2 68950 1 640 +1 sum_2 78950 1 740 +1 sum_2 88950 1 840 +1 sum_2 98950 1 940 +1 sum_2 108950 1 1040 +1 sum_2 70170 1 1140 +2 max_1 9 1 0 +2 max_1 19 1 10 +2 max_1 29 1 20 +2 max_1 39 1 30 +2 max_1 49 1 40 +2 max_1 59 1 50 +2 max_1 69 1 60 +2 max_1 79 1 70 +2 max_1 89 1 80 +2 max_1 99 1 90 +2 max_1 109 1 100 +2 max_1 119 1 110 +2 max_1 129 1 120 +2 max_1 139 1 130 +2 max_1 149 1 140 +2 max_1 159 1 150 +2 max_1 169 1 160 +2 max_1 179 1 170 +2 max_1 189 1 180 +2 max_1 199 1 190 +2 max_1 209 1 200 +2 max_1 219 1 210 +2 max_1 229 1 220 +2 max_1 239 1 230 +2 max_1 249 1 240 +2 max_1 259 1 250 +2 max_1 269 1 260 +2 max_1 279 1 270 +2 max_1 289 1 280 +2 max_1 299 1 290 +2 max_1 39 1 0 +2 max_1 139 1 40 +2 max_1 239 1 140 +2 max_1 339 1 240 +2 max_1 439 1 340 +2 max_1 539 1 440 +2 max_1 639 1 540 +2 max_1 739 1 640 +2 max_1 839 1 740 +2 max_1 939 1 840 +2 max_1 1039 1 940 +2 max_1 1139 1 1040 +2 max_1 1199 1 1140 +2 max_2 9 1 0 +2 max_2 19 1 10 +2 max_2 29 1 20 +2 max_2 39 1 30 +2 max_2 49 1 40 +2 max_2 59 1 50 +2 max_2 69 1 60 +2 max_2 79 1 70 +2 max_2 89 1 80 +2 max_2 99 1 90 +2 max_2 109 1 100 +2 max_2 119 1 110 +2 max_2 129 1 120 +2 max_2 139 1 130 +2 max_2 149 1 140 +2 max_2 159 1 150 +2 max_2 169 1 160 +2 max_2 179 1 170 +2 max_2 189 1 180 +2 max_2 199 1 190 +2 max_2 209 1 200 +2 max_2 219 1 210 +2 max_2 229 1 220 +2 max_2 239 1 230 +2 max_2 249 1 240 +2 max_2 259 1 250 +2 max_2 269 1 260 +2 max_2 279 1 270 +2 max_2 289 1 280 +2 max_2 299 1 290 +2 max_2 39 1 0 +2 max_2 139 1 40 +2 max_2 239 1 140 +2 max_2 339 1 240 +2 max_2 439 1 340 +2 max_2 539 1 440 +2 max_2 639 1 540 +2 max_2 739 1 640 +2 max_2 839 1 740 +2 max_2 939 1 840 +2 max_2 1039 1 940 +2 max_2 1139 1 1040 +2 max_2 1199 1 1140 +2 sum_1 45 1 0 +2 sum_1 145 1 10 +2 sum_1 245 1 20 +2 sum_1 345 1 30 +2 sum_1 445 1 40 +2 sum_1 545 1 50 +2 sum_1 645 1 60 +2 sum_1 745 1 70 +2 sum_1 845 1 80 +2 sum_1 945 1 90 +2 sum_1 1045 1 100 +2 sum_1 1145 1 110 +2 sum_1 1245 1 120 +2 sum_1 1345 1 130 +2 sum_1 1445 1 140 +2 sum_1 1545 1 150 +2 sum_1 1645 1 160 +2 sum_1 1745 1 170 +2 sum_1 1845 1 180 +2 sum_1 1945 1 190 +2 sum_1 2045 1 200 +2 sum_1 2145 1 210 +2 sum_1 2245 1 220 +2 sum_1 2345 1 230 +2 sum_1 2445 1 240 +2 sum_1 2545 1 250 +2 sum_1 2645 1 260 +2 sum_1 2745 1 270 +2 sum_1 2845 1 280 +2 sum_1 2945 1 290 +2 sum_1 780 1 0 +2 sum_1 8950 1 40 +2 sum_1 18950 1 140 +2 sum_1 28950 1 240 +2 sum_1 38950 1 340 +2 sum_1 48950 1 440 +2 sum_1 58950 1 540 +2 sum_1 68950 1 640 +2 sum_1 78950 1 740 +2 sum_1 88950 1 840 +2 sum_1 98950 1 940 +2 sum_1 108950 1 1040 +2 sum_1 70170 1 1140 +2 sum_2 45 1 0 +2 sum_2 145 1 10 +2 sum_2 245 1 20 +2 sum_2 345 1 30 +2 sum_2 445 1 40 +2 sum_2 545 1 50 +2 sum_2 645 1 60 +2 sum_2 745 1 70 +2 sum_2 845 1 80 +2 sum_2 945 1 90 +2 sum_2 1045 1 100 +2 sum_2 1145 1 110 +2 sum_2 1245 1 120 +2 sum_2 1345 1 130 +2 sum_2 1445 1 140 +2 sum_2 1545 1 150 +2 sum_2 1645 1 160 +2 sum_2 1745 1 170 +2 sum_2 1845 1 180 +2 sum_2 1945 1 190 +2 sum_2 2045 1 200 +2 sum_2 2145 1 210 +2 sum_2 2245 1 220 +2 sum_2 2345 1 230 +2 sum_2 2445 1 240 +2 sum_2 2545 1 250 +2 sum_2 2645 1 260 +2 sum_2 2745 1 270 +2 sum_2 2845 1 280 +2 sum_2 2945 1 290 +2 sum_2 780 1 0 +2 sum_2 8950 1 40 +2 sum_2 18950 1 140 +2 sum_2 28950 1 240 +2 sum_2 38950 1 340 +2 sum_2 48950 1 440 +2 sum_2 58950 1 540 +2 sum_2 68950 1 640 +2 sum_2 78950 1 740 +2 sum_2 88950 1 840 +2 sum_2 98950 1 940 +2 sum_2 108950 1 1040 +2 sum_2 70170 1 1140 diff --git a/tests/queries/0_stateless/01236_graphite_mt.sql b/tests/queries/0_stateless/01236_graphite_mt.sql index 6f4af186a1c..d5289364d32 100644 --- a/tests/queries/0_stateless/01236_graphite_mt.sql +++ b/tests/queries/0_stateless/01236_graphite_mt.sql @@ -4,28 +4,36 @@ drop table if exists test_graphite; create table test_graphite (key UInt32, Path String, Time DateTime('UTC'), Value Float64, Version UInt32, col UInt64) engine = GraphiteMergeTree('graphite_rollup') order by key settings index_granularity=10; -insert into test_graphite -select 1, 'sum_1', toDateTime('2021-08-10', 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all -select 2, 'sum_1', toDateTime('2021-08-10', 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all -select 1, 'sum_2', toDateTime('2021-08-10', 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all -select 2, 'sum_2', toDateTime('2021-08-10', 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all -select 1, 'max_1', toDateTime('2021-08-10', 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all -select 2, 'max_1', toDateTime('2021-08-10', 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all -select 1, 'max_2', toDateTime('2021-08-10', 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all -select 2, 'max_2', toDateTime('2021-08-10', 'UTC') - number * 60 - 30, number, 1, number from numbers(300); +SET joined_subquery_requires_alias = 0; -insert into test_graphite -select 1, 'sum_1', toDateTime('2021-08-07', 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all -select 2, 'sum_1', toDateTime('2021-08-07', 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all -select 1, 'sum_2', toDateTime('2021-08-07', 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all -select 2, 'sum_2', toDateTime('2021-08-07', 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all -select 1, 'max_1', toDateTime('2021-08-07', 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all -select 2, 'max_1', toDateTime('2021-08-07', 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all -select 1, 'max_2', toDateTime('2021-08-07', 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all -select 2, 'max_2', toDateTime('2021-08-07', 'UTC') - number * 60 - 30, number, 1, number from numbers(1200); +INSERT into test_graphite +WITH dates AS + ( + SELECT today() as today, + today - 3 as older_date + ) + -- Newer than 2 days are kept in windows of 600 seconds + select 1, 'sum_1', toDateTime(today, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(300) union all + select 2, 'sum_1', toDateTime(today, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(300) union all + select 1, 'sum_2', toDateTime(today, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(300) union all + select 2, 'sum_2', toDateTime(today, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(300) union all + select 1, 'max_1', toDateTime(today, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(300) union all + select 2, 'max_1', toDateTime(today, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(300) union all + select 1, 'max_2', toDateTime(today, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(300) union all + select 2, 'max_2', toDateTime(today, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(300) union all + + -- Older than 2 days use 6000 second windows + select 1, 'sum_1', toDateTime(older_date, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(1200) union all + select 2, 'sum_1', toDateTime(older_date, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(1200) union all + select 1, 'sum_2', toDateTime(older_date, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(1200) union all + select 2, 'sum_2', toDateTime(older_date, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(1200) union all + select 1, 'max_1', toDateTime(older_date, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(1200) union all + select 2, 'max_1', toDateTime(older_date, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(1200) union all + select 1, 'max_2', toDateTime(older_date, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(1200) union all + select 2, 'max_2', toDateTime(older_date, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(1200); optimize table test_graphite final; -select key, Path, Time, Value, Version, col from test_graphite order by key, Path, Time desc; +select key, Path, Value, Version, col from test_graphite order by key, Path, Time desc; drop table test_graphite; From c2f5a9d556197c7f2a655e0ab469f444ab30bc90 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 12 Aug 2021 16:13:53 +0300 Subject: [PATCH 093/161] Fix ya.make --- src/Processors/ya.make | 5 +++++ src/Processors/ya.make.in | 2 +- 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Processors/ya.make b/src/Processors/ya.make index 891a309b2d8..ad4cc5e4a96 100644 --- a/src/Processors/ya.make +++ b/src/Processors/ya.make @@ -31,8 +31,13 @@ SRCS( Formats/IOutputFormat.cpp Formats/IRowInputFormat.cpp Formats/IRowOutputFormat.cpp + Formats/Impl/ArrowBlockInputFormat.cpp + Formats/Impl/ArrowBlockOutputFormat.cpp + Formats/Impl/ArrowBufferedStreams.cpp + Formats/Impl/ArrowColumnToCHColumn.cpp Formats/Impl/BinaryRowInputFormat.cpp Formats/Impl/BinaryRowOutputFormat.cpp + Formats/Impl/CHColumnToArrowColumn.cpp Formats/Impl/CSVRowInputFormat.cpp Formats/Impl/CSVRowOutputFormat.cpp Formats/Impl/ConstantExpressionTemplate.cpp diff --git a/src/Processors/ya.make.in b/src/Processors/ya.make.in index d4b98acf01c..7160e80bcce 100644 --- a/src/Processors/ya.make.in +++ b/src/Processors/ya.make.in @@ -16,7 +16,7 @@ ADDINCL( CFLAGS(-DUSE_ARROW=1) SRCS( - + ) END() From 36878cd377df1414337a6549daf34d4bee726bdf Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Thu, 12 Aug 2021 16:18:33 +0300 Subject: [PATCH 094/161] Add ru docs for uuid and array functions. --- .../functions/array-functions.md | 3 +- .../functions/array-functions.md | 86 +++++++++++++++++-- .../sql-reference/functions/uuid-functions.md | 84 ++++++++++++++++++ 3 files changed, 164 insertions(+), 9 deletions(-) diff --git a/docs/en/sql-reference/functions/array-functions.md b/docs/en/sql-reference/functions/array-functions.md index 38ee2090866..ddbbab3ecfc 100644 --- a/docs/en/sql-reference/functions/array-functions.md +++ b/docs/en/sql-reference/functions/array-functions.md @@ -18,7 +18,7 @@ empty([x]) An array is considered empty if it contains all empty elements. !!! note "Note" - Can be optimized by enabling the [optimize_functions_to_subcolumns](../../operations/settings/settings.md#optimize-functions-to-subcolumns) setting. With `optimize_functions_to_subcolumns = 1` the function reads only [size0](../../sql-reference/data-types/array.md#array-size) subcolumn instead of reading and processing the whole array column. The query `SELECT empty(arr) FROM table;` transforms to `SELECT arr.size0 = 0 FROM TABLE;`. + Can be optimized by enabling the [optimize_functions_to_subcolumns](../../operations/settings/settings.md#optimize-functions-to-subcolumns) setting. With `optimize_functions_to_subcolumns = 1` the function reads only [size0](../../sql-reference/data-types/array.md#array-size) subcolumn instead of reading and processing the whole array column. The query `SELECT empty(arr) FROM TABLE;` transforms to `SELECT arr.size0 = 0 FROM TABLE;`. The function also works for [strings](string-functions.md#empty) or [UUID](uuid-functions.md#empty). @@ -57,6 +57,7 @@ Checks whether the input array is non-empty. ``` sql notEmpty([x]) ``` + An array is considered non-empty if it contains at least one non-empty element. !!! note "Note" diff --git a/docs/ru/sql-reference/functions/array-functions.md b/docs/ru/sql-reference/functions/array-functions.md index 52fd63864ce..066d37a71f5 100644 --- a/docs/ru/sql-reference/functions/array-functions.md +++ b/docs/ru/sql-reference/functions/array-functions.md @@ -7,19 +7,89 @@ toc_title: "Массивы" ## empty {#function-empty} -Возвращает 1 для пустого массива, и 0 для непустого массива. -Тип результата - UInt8. -Функция также работает для строк. +Проверяет, является ли входной массив пустым. -Функцию можно оптимизировать, если включить настройку [optimize_functions_to_subcolumns](../../operations/settings/settings.md#optimize-functions-to-subcolumns). При `optimize_functions_to_subcolumns = 1` функция читает только подстолбец [size0](../../sql-reference/data-types/array.md#array-size) вместо чтения и обработки всего столбца массива. Запрос `SELECT empty(arr) FROM table` преобразуется к запросу `SELECT arr.size0 = 0 FROM TABLE`. +**Синтаксис** + +``` sql +empty([x]) +``` + +Массив считается пустым, если он содержит все пустые элементы. + +!!! note "Примечание" + Функцию можно оптимизировать, если включить настройку [optimize_functions_to_subcolumns](../../operations/settings/settings.md#optimize-functions-to-subcolumns). При `optimize_functions_to_subcolumns = 1` функция читает только подстолбец [size0](../../sql-reference/data-types/array.md#array-size) вместо чтения и обработки всего столбца массива. Запрос `SELECT empty(arr) FROM TABLE` преобразуется к запросу `SELECT arr.size0 = 0 FROM TABLE`. + +Функция также поддерживает работу с типами [String](string-functions.md#empty) и [UUID](uuid-functions.md#empty). + +**Параметры** + +- `[x]` — массив на входе функции. [Array](../data-types/array.md). + +**Возвращаемое значение** + +- Возвращает `1` для пустого массива или `0` — для непустого массива. + +Тип: [UInt8](../data-types/int-uint.md). + +**Пример** + +Запрос: + +```sql +SELECT empty([]); +``` + +Ответ: + +```text +┌─empty(array())─┐ +│ 1 │ +└────────────────┘ +``` ## notEmpty {#function-notempty} -Возвращает 0 для пустого массива, и 1 для непустого массива. -Тип результата - UInt8. -Функция также работает для строк. +Проверяет, является ли входной массив непустым. -Функцию можно оптимизировать, если включить настройку [optimize_functions_to_subcolumns](../../operations/settings/settings.md#optimize-functions-to-subcolumns). При `optimize_functions_to_subcolumns = 1` функция читает только подстолбец [size0](../../sql-reference/data-types/array.md#array-size) вместо чтения и обработки всего столбца массива. Запрос `SELECT notEmpty(arr) FROM table` преобразуется к запросу `SELECT arr.size0 != 0 FROM TABLE`. +**Синтаксис** + +``` sql +notEmpty([x]) +``` + +Массив считается непустым, если он содержит хотя бы один непустой элемент. + +!!! note "Примечание" + Функцию можно оптимизировать, если включить настройку [optimize_functions_to_subcolumns](../../operations/settings/settings.md#optimize-functions-to-subcolumns). При `optimize_functions_to_subcolumns = 1` функция читает только подстолбец [size0](../../sql-reference/data-types/array.md#array-size) вместо чтения и обработки всего столбца массива. Запрос `SELECT notEmpty(arr) FROM table` преобразуется к запросу `SELECT arr.size0 != 0 FROM TABLE`. + +Функция также поддерживает работу с типами [String](string-functions.md#notempty) и [UUID](uuid-functions.md#notempty). + +**Параметры** + +- `[x]` — массив на входе функции. [Array](../data-types/array.md). + +**Возвращаемое значение** + +- Возвращает `1` для непустого массива или `0` — для пустого массива. + +Тип: [UInt8](../data-types/int-uint.md). + +**Пример** + +Запрос: + +```sql +SELECT notEmpty([1,2]); +``` + +Результат: + +```text +┌─notEmpty([1, 2])─┐ +│ 1 │ +└──────────────────┘ +``` ## length {#array_functions-length} diff --git a/docs/ru/sql-reference/functions/uuid-functions.md b/docs/ru/sql-reference/functions/uuid-functions.md index f0017adbc8b..977446fdf06 100644 --- a/docs/ru/sql-reference/functions/uuid-functions.md +++ b/docs/ru/sql-reference/functions/uuid-functions.md @@ -35,6 +35,90 @@ SELECT * FROM t_uuid └──────────────────────────────────────┘ ``` +## empty {#empty} + +Проверяет, является ли входной UUID пустым. + +**Синтаксис** + +```sql +empty(UUID) +``` + +UUID считается пустым, если он содержит все нули (нулевой UUID). + +Функция также поддерживает работу с типами [Array](array-functions.md#empty) и [String](string-functions.md#empty). + +**Параметры** + +- `x` — UUID на входе функции. [UUID](../data-types/uuid.md). + +**Возвращаемое значение** + +- Возвращает `1` для пустого UUID или `0` — для непустого UUID. + +Тип: [UInt8](../data-types/int-uint.md). + +**Пример** + +Для генерации UUID-значений предназначена функция [generateUUIDv4](#uuid-function-generate). + +Запрос: + +```sql +SELECT empty(generateUUIDv4()); +``` + +Ответ: + +```text +┌─empty(generateUUIDv4())─┐ +│ 0 │ +└─────────────────────────┘ +``` + +## notEmpty {#notempty} + +Проверяет, является ли входной UUID непустым. + +**Синтаксис** + +```sql +notEmpty(UUID) +``` + +UUID считается пустым, если он содержит все нули (нулевой UUID). + +Функция также поддерживает работу с типами [Array](array-functions.md#notempty) и [String](string-functions.md#notempty). + +**Параметры** + +- `x` — UUID на входе функции. [UUID](../data-types/uuid.md). + +**Возвращаемое значение** + +- Возвращает `1` для непустого UUID или `0` — для пустого UUID. + +Тип: [UInt8](../data-types/int-uint.md). + +**Пример** + +Для генерации UUID-значений предназначена функция [generateUUIDv4](#uuid-function-generate). + +Запрос: + +```sql +SELECT notEmpty(generateUUIDv4()); +``` + +Результат: + +```text +┌─notEmpty(generateUUIDv4())─┐ +│ 1 │ +└────────────────────────────┘ +``` + ## toUUID (x) {#touuid-x} Преобразует значение типа String в тип UUID. From a549e29bd4152ac45a2023c739e2e685eb9c7be4 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 12 Aug 2021 14:42:51 +0300 Subject: [PATCH 095/161] Better --- src/Interpreters/InterpreterFactory.cpp | 8 +- .../InterpreterIntersectOrExcept.h | 39 --------- ...InterpreterSelectIntersectExceptQuery.cpp} | 77 ++++++++++------- .../InterpreterSelectIntersectExceptQuery.h | 45 ++++++++++ .../InterpreterSelectWithUnionQuery.cpp | 10 ++- .../SelectIntersectExceptQueryVisitor.cpp | 85 +++++++++++++++++++ .../SelectIntersectExceptQueryVisitor.h | 47 ++++++++++ src/Interpreters/executeQuery.cpp | 15 +++- ....cpp => ASTSelectIntersectExceptQuery.cpp} | 19 +++-- ...cept.h => ASTSelectIntersectExceptQuery.h} | 10 ++- src/Parsers/ParserQueryWithOutput.cpp | 4 +- ...p => ParserSelectIntersectExceptQuery.cpp} | 18 ++-- ...y.h => ParserSelectIntersectExceptQuery.h} | 2 +- src/Parsers/ParserUnionQueryElement.cpp | 5 +- .../QueryPlan/IntersectOrExceptStep.cpp | 13 +-- .../QueryPlan/IntersectOrExceptStep.h | 10 +-- .../Transforms/IntersectOrExceptTransform.cpp | 75 +++++++--------- .../Transforms/IntersectOrExceptTransform.h | 12 +-- ...02004_intersect_except_operators.reference | 25 ++++++ .../02004_intersect_except_operators.sql | 10 +++ 20 files changed, 363 insertions(+), 166 deletions(-) delete mode 100644 src/Interpreters/InterpreterIntersectOrExcept.h rename src/Interpreters/{InterpreterIntersectOrExcept.cpp => InterpreterSelectIntersectExceptQuery.cpp} (68%) create mode 100644 src/Interpreters/InterpreterSelectIntersectExceptQuery.h create mode 100644 src/Interpreters/SelectIntersectExceptQueryVisitor.cpp create mode 100644 src/Interpreters/SelectIntersectExceptQueryVisitor.h rename src/Parsers/{ASTIntersectOrExcept.cpp => ASTSelectIntersectExceptQuery.cpp} (76%) rename src/Parsers/{ASTIntersectOrExcept.h => ASTSelectIntersectExceptQuery.h} (55%) rename src/Parsers/{ParserIntersectOrExceptQuery.cpp => ParserSelectIntersectExceptQuery.cpp} (66%) rename src/Parsers/{ParserIntersectOrExceptQuery.h => ParserSelectIntersectExceptQuery.h} (79%) diff --git a/src/Interpreters/InterpreterFactory.cpp b/src/Interpreters/InterpreterFactory.cpp index e634c072841..e5b381b4d08 100644 --- a/src/Interpreters/InterpreterFactory.cpp +++ b/src/Interpreters/InterpreterFactory.cpp @@ -11,7 +11,7 @@ #include #include #include -#include +#include #include #include #include @@ -49,7 +49,7 @@ #include #include #include -#include +#include #include #include #include @@ -111,9 +111,9 @@ std::unique_ptr InterpreterFactory::get(ASTPtr & query, ContextMut ProfileEvents::increment(ProfileEvents::SelectQuery); return std::make_unique(query, context, options); } - else if (query->as()) + else if (query->as()) { - return std::make_unique(query, context); + return std::make_unique(query, context, options); } else if (query->as()) { diff --git a/src/Interpreters/InterpreterIntersectOrExcept.h b/src/Interpreters/InterpreterIntersectOrExcept.h deleted file mode 100644 index 359be05db8b..00000000000 --- a/src/Interpreters/InterpreterIntersectOrExcept.h +++ /dev/null @@ -1,39 +0,0 @@ -#pragma once - -#include -#include -#include -#include - - -namespace DB -{ - -class Context; -class InterpreterSelectQuery; -class QueryPlan; - -class InterpreterIntersectOrExcept : public IInterpreter -{ -public: - InterpreterIntersectOrExcept(const ASTPtr & query_ptr_, ContextPtr context_); - - BlockIO execute() override; - -private: - String getName() const { return "IntersectOrExcept"; } - - Block getCommonHeader(const Blocks & headers) const; - - std::unique_ptr - buildCurrentChildInterpreter(const ASTPtr & ast_ptr_); - - void buildQueryPlan(QueryPlan & query_plan); - - ContextPtr context; - Block result_header; - std::vector> nested_interpreters; - ASTIntersectOrExcept::Operators operators; -}; - -} diff --git a/src/Interpreters/InterpreterIntersectOrExcept.cpp b/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp similarity index 68% rename from src/Interpreters/InterpreterIntersectOrExcept.cpp rename to src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp index 52dcb24ed27..9e24dd6e6a0 100644 --- a/src/Interpreters/InterpreterIntersectOrExcept.cpp +++ b/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp @@ -1,8 +1,8 @@ #include #include -#include +#include #include -#include +#include #include #include #include @@ -19,27 +19,7 @@ namespace ErrorCodes extern const int INTERSECT_OR_EXCEPT_RESULT_STRUCTURES_MISMATCH; } -InterpreterIntersectOrExcept::InterpreterIntersectOrExcept(const ASTPtr & query_ptr, ContextPtr context_) - : context(Context::createCopy(context_)) -{ - ASTIntersectOrExcept * ast = query_ptr->as(); - operators = ast->list_of_operators; - - auto children = ast->list_of_selects->children; - size_t num_children = children.size(); - nested_interpreters.resize(num_children); - - for (size_t i = 0; i < num_children; ++i) - nested_interpreters[i] = buildCurrentChildInterpreter(children.at(i)); - - Blocks headers(num_children); - for (size_t query_num = 0; query_num < num_children; ++query_num) - headers[query_num] = nested_interpreters[query_num]->getSampleBlock(); - - result_header = getCommonHeader(headers); -} - -Block InterpreterIntersectOrExcept::getCommonHeader(const Blocks & headers) const +static Block getCommonHeader(const Blocks & headers) { size_t num_selects = headers.size(); Block common_header = headers.front(); @@ -49,8 +29,8 @@ Block InterpreterIntersectOrExcept::getCommonHeader(const Blocks & headers) cons { if (headers[query_num].columns() != num_columns) throw Exception(ErrorCodes::INTERSECT_OR_EXCEPT_RESULT_STRUCTURES_MISMATCH, - "Different number of columns in {} elements:\n {} \nand\n {}", - getName(), common_header.dumpNames(), headers[query_num].dumpNames()); + "Different number of columns in IntersectExceptQuery elements:\n {} \nand\n {}", + common_header.dumpNames(), headers[query_num].dumpNames()); } std::vector columns(num_selects); @@ -66,16 +46,53 @@ Block InterpreterIntersectOrExcept::getCommonHeader(const Blocks & headers) cons return common_header; } +InterpreterSelectIntersectExceptQuery::InterpreterSelectIntersectExceptQuery( + const ASTPtr & query_ptr_, + ContextPtr context_, + const SelectQueryOptions & options_) + : IInterpreterUnionOrSelectQuery(query_ptr_->clone(), context_, options_) +{ + ASTSelectIntersectExceptQuery * ast = query_ptr->as(); + final_operator = ast->final_operator; + + const auto & children = ast->children[0]->children; + size_t num_children = children.size(); + + /// AST must have been changed by the visitor. + if (final_operator == Operator::UNKNOWN || num_children != 2) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "SelectIntersectExceptyQuery has not been normalized (number of children: {})", + num_children); + + nested_interpreters.resize(num_children); + + for (size_t i = 0; i < num_children; ++i) + nested_interpreters[i] = buildCurrentChildInterpreter(children.at(i)); + + Blocks headers(num_children); + for (size_t query_num = 0; query_num < num_children; ++query_num) + headers[query_num] = nested_interpreters[query_num]->getSampleBlock(); + + result_header = getCommonHeader(headers); +} + std::unique_ptr -InterpreterIntersectOrExcept::buildCurrentChildInterpreter(const ASTPtr & ast_ptr_) +InterpreterSelectIntersectExceptQuery::buildCurrentChildInterpreter(const ASTPtr & ast_ptr_) { if (ast_ptr_->as()) return std::make_unique(ast_ptr_, context, SelectQueryOptions()); - else + + if (ast_ptr_->as()) return std::make_unique(ast_ptr_, context, SelectQueryOptions()); + + if (ast_ptr_->as()) + return std::make_unique(ast_ptr_, context, SelectQueryOptions()); + + // if (ast_ptr_->as()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected query: {}", ast_ptr_->getID()); } -void InterpreterIntersectOrExcept::buildQueryPlan(QueryPlan & query_plan) +void InterpreterSelectIntersectExceptQuery::buildQueryPlan(QueryPlan & query_plan) { size_t num_plans = nested_interpreters.size(); std::vector> plans(num_plans); @@ -101,11 +118,11 @@ void InterpreterIntersectOrExcept::buildQueryPlan(QueryPlan & query_plan) } auto max_threads = context->getSettingsRef().max_threads; - auto step = std::make_unique(std::move(data_streams), operators, max_threads); + auto step = std::make_unique(std::move(data_streams), final_operator, max_threads); query_plan.unitePlans(std::move(step), std::move(plans)); } -BlockIO InterpreterIntersectOrExcept::execute() +BlockIO InterpreterSelectIntersectExceptQuery::execute() { BlockIO res; diff --git a/src/Interpreters/InterpreterSelectIntersectExceptQuery.h b/src/Interpreters/InterpreterSelectIntersectExceptQuery.h new file mode 100644 index 00000000000..9cbde055b0b --- /dev/null +++ b/src/Interpreters/InterpreterSelectIntersectExceptQuery.h @@ -0,0 +1,45 @@ +#pragma once + +#include +#include +#include +#include +#include + + +namespace DB +{ + +class Context; +class InterpreterSelectQuery; +class QueryPlan; + +class InterpreterSelectIntersectExceptQuery : public IInterpreterUnionOrSelectQuery +{ +using Operator = ASTSelectIntersectExceptQuery::Operator; + +public: + InterpreterSelectIntersectExceptQuery( + const ASTPtr & query_ptr_, + ContextPtr context_, + const SelectQueryOptions & options_); + + BlockIO execute() override; + + Block getSampleBlock() { return result_header; } + +private: + static String getName() { return "SelectIntersectExceptQuery"; } + + std::unique_ptr + buildCurrentChildInterpreter(const ASTPtr & ast_ptr_); + + void buildQueryPlan(QueryPlan & query_plan) override; + + void ignoreWithTotals() override {} + + std::vector> nested_interpreters; + Operator final_operator; +}; + +} diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index 3cf4a905d38..cd06f51cb12 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -2,8 +2,10 @@ #include #include #include +#include #include #include +#include #include #include #include @@ -208,8 +210,10 @@ InterpreterSelectWithUnionQuery::buildCurrentChildInterpreter(const ASTPtr & ast { if (ast_ptr_->as()) return std::make_unique(ast_ptr_, context, options, current_required_result_column_names); - else + else if (ast_ptr_->as()) return std::make_unique(ast_ptr_, context, options, current_required_result_column_names); + else + return std::make_unique(ast_ptr_, context, options); } InterpreterSelectWithUnionQuery::~InterpreterSelectWithUnionQuery() = default; @@ -225,10 +229,14 @@ Block InterpreterSelectWithUnionQuery::getSampleBlock(const ASTPtr & query_ptr_, } if (is_subquery) + { return cache[key] = InterpreterSelectWithUnionQuery(query_ptr_, context_, SelectQueryOptions().subquery().analyze()).getSampleBlock(); + } else + { return cache[key] = InterpreterSelectWithUnionQuery(query_ptr_, context_, SelectQueryOptions().analyze()).getSampleBlock(); + } } diff --git a/src/Interpreters/SelectIntersectExceptQueryVisitor.cpp b/src/Interpreters/SelectIntersectExceptQueryVisitor.cpp new file mode 100644 index 00000000000..a404bf3da40 --- /dev/null +++ b/src/Interpreters/SelectIntersectExceptQueryVisitor.cpp @@ -0,0 +1,85 @@ +#include +#include +#include + + +namespace DB +{ + +void SelectIntersectExceptQueryMatcher::visit(ASTPtr & ast, Data & data) +{ + if (auto * select_intersect_except = ast->as()) + { + std::cerr << "\n\nSelectIntersectExceptVisitor BEFORE:\n" << ast->dumpTree() << std::endl; + data.initialize(select_intersect_except); + visit(*select_intersect_except, data); + std::cerr << "\n\nSelectIntersectExceptVisitor AFTER:\n" << ast->dumpTree() << std::endl; + } +} + +void SelectIntersectExceptQueryMatcher::visit(ASTSelectIntersectExceptQuery & ast, Data & data) +{ + /* Example: select 1 intersect select 1 intsect select 1 intersect select 1 intersect select 1; + * + * --SelectIntersectExceptQuery --SelectIntersectExceptQuery + * ---expressionlist ---ExpressionList + * ----SelectQuery ----SelectIntersectExceptQuery + * ----SelectQuery ------ExpressionList + * ----SelectQuery ---> -------SelectIntersectExceptQuery + * ----SelectQuery --------ExpressionList + * ---------SelectQuery + * ---------SelectQuery + * -------SelectQuery + * ----SelectQuery + **/ + + auto & selects = data.reversed_list_of_selects; + + if (selects.empty()) + return; + + const auto left = selects.back(); + selects.pop_back(); + const auto right = selects.back(); + selects.pop_back(); + + auto & operators = data.reversed_list_of_operators; + const auto current_operator = operators.back(); + operators.pop_back(); + + auto list_node = std::make_shared(); + list_node->children = {left, right}; + + if (selects.empty()) + { + ast.final_operator = current_operator; + ast.children = {std::move(list_node)}; + } + else + { + auto select_intersect_except = std::make_shared(); + select_intersect_except->final_operator = {current_operator}; + select_intersect_except->children.emplace_back(std::move(list_node)); + + selects.emplace_back(std::move(select_intersect_except)); + } + + visit(ast, data); +} + +// void SelectIntersectExceptQueryVisitor::visit(ASTSelectWithUnionQuery & ast, Data & data) +// { +// auto & union_modes = ast.list_of_modes; +// ASTs selects; +// auto & select_list = ast.list_of_selects->children; +// +// +// // reverse children list +// std::reverse(selects.begin(), selects.end()); +// +// ast.is_normalized = true; +// ast.union_mode = ASTSelectWithUnionQuery::Mode::ALL; +// +// ast.list_of_selects->children = std::move(selects); +// } +} diff --git a/src/Interpreters/SelectIntersectExceptQueryVisitor.h b/src/Interpreters/SelectIntersectExceptQueryVisitor.h new file mode 100644 index 00000000000..58f3071972f --- /dev/null +++ b/src/Interpreters/SelectIntersectExceptQueryVisitor.h @@ -0,0 +1,47 @@ +#pragma once + +#include + +#include +#include + +#include +#include + + +namespace DB +{ + +class ASTFunction; + +class SelectIntersectExceptQueryMatcher +{ +public: + struct Data + { + Data() = default; + + void initialize(const ASTSelectIntersectExceptQuery * select_intersect_except) + { + reversed_list_of_selects = select_intersect_except->list_of_selects->clone()->children; + reversed_list_of_operators = select_intersect_except->list_of_operators; + + std::reverse(reversed_list_of_selects.begin(), reversed_list_of_selects.end()); + std::reverse(reversed_list_of_operators.begin(), reversed_list_of_operators.end()); + } + + ASTs reversed_list_of_selects; + ASTSelectIntersectExceptQuery::Operators reversed_list_of_operators; + }; + + static bool needChildVisit(const ASTPtr &, const ASTPtr &) { return true; } + + static void visit(ASTPtr & ast, Data &); + static void visit(ASTSelectIntersectExceptQuery &, Data &); + // static void visit(ASTSelectWithUnionQuery &, Data &); +}; + +/// Visit children first. +using SelectIntersectExceptQueryVisitor + = InDepthNodeVisitor; +} diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 1b59f3bc7df..839447a90f7 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -46,6 +46,7 @@ #include #include #include +#include #include #include @@ -490,9 +491,16 @@ static std::tuple executeQueryImpl( ApplyWithGlobalVisitor().visit(ast); } - /// Normalize SelectWithUnionQuery - NormalizeSelectWithUnionQueryVisitor::Data data{context->getSettingsRef().union_default_mode}; - NormalizeSelectWithUnionQueryVisitor{data}.visit(ast); + { + /// Normalize SelectWithUnionQuery + NormalizeSelectWithUnionQueryVisitor::Data data{context->getSettingsRef().union_default_mode}; + NormalizeSelectWithUnionQueryVisitor{data}.visit(ast); + } + + { + SelectIntersectExceptQueryVisitor::Data data; + SelectIntersectExceptQueryVisitor{data}.visit(ast); + } /// Check the limits. checkASTSizeLimits(*ast, settings); @@ -532,6 +540,7 @@ static std::tuple executeQueryImpl( /// reset Input callbacks if query is not INSERT SELECT context->resetInputCallbacks(); + std::cerr << "\n\nAST: " << ast->dumpTree() << std::endl; auto interpreter = InterpreterFactory::get(ast, context, SelectQueryOptions(stage).setInternal(internal)); std::shared_ptr quota; diff --git a/src/Parsers/ASTIntersectOrExcept.cpp b/src/Parsers/ASTSelectIntersectExceptQuery.cpp similarity index 76% rename from src/Parsers/ASTIntersectOrExcept.cpp rename to src/Parsers/ASTSelectIntersectExceptQuery.cpp index 33ffb76c2f7..26fd9353d5b 100644 --- a/src/Parsers/ASTIntersectOrExcept.cpp +++ b/src/Parsers/ASTSelectIntersectExceptQuery.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include @@ -6,20 +6,25 @@ namespace DB { -ASTPtr ASTIntersectOrExcept::clone() const +ASTPtr ASTSelectIntersectExceptQuery::clone() const { - auto res = std::make_shared(*this); - res->children.clear(); + auto res = std::make_shared(*this); + + res->children.clear(); + for (const auto & child : children) + res->children.push_back(child->clone()); + + if (res->list_of_selects) + res->list_of_selects = list_of_selects->clone(); - res->list_of_selects = list_of_selects->clone(); - res->children.push_back(res->list_of_selects); res->list_of_operators = list_of_operators; + res->final_operator = final_operator; cloneOutputOptions(*res); return res; } -void ASTIntersectOrExcept::formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const +void ASTSelectIntersectExceptQuery::formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' '); diff --git a/src/Parsers/ASTIntersectOrExcept.h b/src/Parsers/ASTSelectIntersectExceptQuery.h similarity index 55% rename from src/Parsers/ASTIntersectOrExcept.h rename to src/Parsers/ASTSelectIntersectExceptQuery.h index 9adfdedc497..8fc5756e370 100644 --- a/src/Parsers/ASTIntersectOrExcept.h +++ b/src/Parsers/ASTSelectIntersectExceptQuery.h @@ -6,17 +6,20 @@ namespace DB { -class ASTIntersectOrExcept : public ASTQueryWithOutput +class ASTSelectIntersectExceptQuery : public ASTQueryWithOutput { public: - String getID(char) const override { return "IntersectExceptQuery"; } + String getID(char) const override { return "SelectIntersectExceptQuery"; } ASTPtr clone() const override; void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; + const char * getQueryKindString() const override { return "SelectIntersectExcept"; } + enum class Operator { + UNKNOWN, INTERSECT, EXCEPT }; @@ -25,6 +28,9 @@ public: ASTPtr list_of_selects; Operators list_of_operators; + + /// Final operator after applying visitor. + Operator final_operator = Operator::UNKNOWN; }; } diff --git a/src/Parsers/ParserQueryWithOutput.cpp b/src/Parsers/ParserQueryWithOutput.cpp index d7d87cac9b9..e2ab8a84cc1 100644 --- a/src/Parsers/ParserQueryWithOutput.cpp +++ b/src/Parsers/ParserQueryWithOutput.cpp @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include #include #include @@ -31,7 +31,6 @@ namespace DB bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ParserShowTablesQuery show_tables_p; - ParserIntersectOrExceptQuery intersect_except_p; ParserSelectWithUnionQuery select_p; ParserTablePropertiesQuery table_p; ParserDescribeTableQuery describe_table_p; @@ -55,7 +54,6 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec bool parsed = explain_p.parse(pos, query, expected) - || intersect_except_p.parse(pos, query, expected) || select_p.parse(pos, query, expected) || show_create_access_entity_p.parse(pos, query, expected) /// should be before `show_tables_p` || show_tables_p.parse(pos, query, expected) diff --git a/src/Parsers/ParserIntersectOrExceptQuery.cpp b/src/Parsers/ParserSelectIntersectExceptQuery.cpp similarity index 66% rename from src/Parsers/ParserIntersectOrExceptQuery.cpp rename to src/Parsers/ParserSelectIntersectExceptQuery.cpp index ef6d68f8534..b56598166c6 100644 --- a/src/Parsers/ParserIntersectOrExceptQuery.cpp +++ b/src/Parsers/ParserSelectIntersectExceptQuery.cpp @@ -1,9 +1,9 @@ -#include +#include #include #include #include -#include -#include +#include +#include #include #include @@ -11,18 +11,18 @@ namespace DB { -bool ParserIntersectOrExceptQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +bool ParserSelectIntersectExceptQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ParserKeyword intersect_keyword("INTERSECT"); ParserKeyword except_keyword("EXCEPT"); ASTs elements; - ASTIntersectOrExcept::Operators operators; + ASTSelectIntersectExceptQuery::Operators operators; auto parse_element = [&]() -> bool { ASTPtr element; - if (!ParserSelectWithUnionQuery().parse(pos, element, expected) && !ParserSubquery().parse(pos, element, expected)) + if (!ParserSelectQuery().parse(pos, element, expected) && !ParserSubquery().parse(pos, element, expected)) return false; elements.push_back(element); @@ -36,11 +36,11 @@ bool ParserIntersectOrExceptQuery::parseImpl(Pos & pos, ASTPtr & node, Expected if (!except_keyword.ignore(pos)) return false; - operators.emplace_back(ASTIntersectOrExcept::Operator::EXCEPT); + operators.emplace_back(ASTSelectIntersectExceptQuery::Operator::EXCEPT); return true; } - operators.emplace_back(ASTIntersectOrExcept::Operator::INTERSECT); + operators.emplace_back(ASTSelectIntersectExceptQuery::Operator::INTERSECT); return true; }; @@ -56,7 +56,7 @@ bool ParserIntersectOrExceptQuery::parseImpl(Pos & pos, ASTPtr & node, Expected auto list_node = std::make_shared(); list_node->children = std::move(elements); - auto intersect_or_except_ast = std::make_shared(); + auto intersect_or_except_ast = std::make_shared(); node = intersect_or_except_ast; intersect_or_except_ast->list_of_selects = list_node; diff --git a/src/Parsers/ParserIntersectOrExceptQuery.h b/src/Parsers/ParserSelectIntersectExceptQuery.h similarity index 79% rename from src/Parsers/ParserIntersectOrExceptQuery.h rename to src/Parsers/ParserSelectIntersectExceptQuery.h index d8ba82ba053..e01785113a8 100644 --- a/src/Parsers/ParserIntersectOrExceptQuery.h +++ b/src/Parsers/ParserSelectIntersectExceptQuery.h @@ -4,7 +4,7 @@ namespace DB { -class ParserIntersectOrExceptQuery : public IParserBase +class ParserSelectIntersectExceptQuery : public IParserBase { protected: const char * getName() const override { return "INTERSECT or EXCEPT"; } diff --git a/src/Parsers/ParserUnionQueryElement.cpp b/src/Parsers/ParserUnionQueryElement.cpp index efd022e6362..5abbce25930 100644 --- a/src/Parsers/ParserUnionQueryElement.cpp +++ b/src/Parsers/ParserUnionQueryElement.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include @@ -10,7 +11,9 @@ namespace DB bool ParserUnionQueryElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { - if (!ParserSubquery().parse(pos, node, expected) && !ParserSelectQuery().parse(pos, node, expected)) + if (!ParserSubquery().parse(pos, node, expected) + && !ParserSelectIntersectExceptQuery().parse(pos, node, expected) + && !ParserSelectQuery().parse(pos, node, expected)) return false; if (const auto * ast_subquery = node->as()) diff --git a/src/Processors/QueryPlan/IntersectOrExceptStep.cpp b/src/Processors/QueryPlan/IntersectOrExceptStep.cpp index 76f496ba47c..e61afb5ba2a 100644 --- a/src/Processors/QueryPlan/IntersectOrExceptStep.cpp +++ b/src/Processors/QueryPlan/IntersectOrExceptStep.cpp @@ -1,7 +1,8 @@ +#include + #include #include #include -#include #include #include #include @@ -11,22 +12,22 @@ namespace DB { -Block IntersectOrExceptStep::checkHeaders(const DataStreams & input_streams_) const +static Block checkHeaders(const DataStreams & input_streams_) { if (input_streams_.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot perform intersect/except on empty set of query plan steps"); Block res = input_streams_.front().header; for (const auto & stream : input_streams_) - assertBlocksHaveEqualStructure(stream.header, res, "IntersectExceptStep"); + assertBlocksHaveEqualStructure(stream.header, res, "IntersectOrExceptStep"); return res; } IntersectOrExceptStep::IntersectOrExceptStep( - DataStreams input_streams_ , const Operators & operators_ , size_t max_threads_) + DataStreams input_streams_ , Operator operator_ , size_t max_threads_) : header(checkHeaders(input_streams_)) - , operators(operators_) + , current_operator(operator_) , max_threads(max_threads_) { input_streams = std::move(input_streams_); @@ -67,7 +68,7 @@ QueryPipelinePtr IntersectOrExceptStep::updatePipeline(QueryPipelines pipelines, } *pipeline = QueryPipeline::unitePipelines(std::move(pipelines), max_threads); - pipeline->addTransform(std::make_shared(header, operators)); + pipeline->addTransform(std::make_shared(header, current_operator)); processors = collector.detachProcessors(); return pipeline; diff --git a/src/Processors/QueryPlan/IntersectOrExceptStep.h b/src/Processors/QueryPlan/IntersectOrExceptStep.h index 914a7dce197..002f1b1570c 100644 --- a/src/Processors/QueryPlan/IntersectOrExceptStep.h +++ b/src/Processors/QueryPlan/IntersectOrExceptStep.h @@ -1,6 +1,6 @@ #pragma once #include -#include +#include namespace DB @@ -8,11 +8,11 @@ namespace DB class IntersectOrExceptStep : public IQueryPlanStep { -using Operators = ASTIntersectOrExcept::Operators; +using Operator = ASTSelectIntersectExceptQuery::Operator; public: /// max_threads is used to limit the number of threads for result pipeline. - IntersectOrExceptStep(DataStreams input_streams_, const Operators & operators_, size_t max_threads_ = 0); + IntersectOrExceptStep(DataStreams input_streams_, Operator operators_, size_t max_threads_ = 0); String getName() const override { return "IntersectOrExcept"; } @@ -21,10 +21,8 @@ public: void describePipeline(FormatSettings & settings) const override; private: - Block checkHeaders(const DataStreams & input_streams_) const; - Block header; - Operators operators; + Operator current_operator; size_t max_threads; Processors processors; }; diff --git a/src/Processors/Transforms/IntersectOrExceptTransform.cpp b/src/Processors/Transforms/IntersectOrExceptTransform.cpp index 68d5f6a2e5e..b16032bde8e 100644 --- a/src/Processors/Transforms/IntersectOrExceptTransform.cpp +++ b/src/Processors/Transforms/IntersectOrExceptTransform.cpp @@ -4,15 +4,14 @@ namespace DB { -/* - * There are always at least two inputs. Number of operators is always number of inputs minus 1. - * input1 {operator1} input2 {operator2} input3 ... -**/ -IntersectOrExceptTransform::IntersectOrExceptTransform(const Block & header_, const Operators & operators_) - : IProcessor(InputPorts(operators_.size() + 1, header_), {header_}) - , operators(operators_) - , first_input(inputs.begin()) - , second_input(std::next(inputs.begin())) +namespace ErrorCodes +{ + extern const int SET_SIZE_LIMIT_EXCEEDED; +} + +IntersectOrExceptTransform::IntersectOrExceptTransform(const Block & header_, Operator operator_) + : IProcessor(InputPorts(2, header_), {header_}) + , current_operator(operator_) { const Names & columns = header_.getNames(); size_t num_columns = columns.empty() ? header_.columns() : columns.size(); @@ -46,53 +45,33 @@ IntersectOrExceptTransform::Status IntersectOrExceptTransform::prepare() return Status::PortFull; } + if (current_output_chunk) + { + output.push(std::move(current_output_chunk)); + } + if (finished_second_input) { - if (first_input->isFinished() || (use_accumulated_input && !current_input_chunk)) + if (inputs.front().isFinished()) { - std::advance(second_input, 1); - - if (second_input == inputs.end()) - { - if (current_output_chunk) - { - output.push(std::move(current_output_chunk)); - } - - output.finish(); - return Status::Finished; - } - else - { - use_accumulated_input = true; - data.reset(); - finished_second_input = false; - ++current_operator_pos; - } + output.finish(); + return Status::Finished; } } - else if (second_input->isFinished()) + else if (inputs.back().isFinished()) { finished_second_input = true; } if (!has_input) { - if (finished_second_input && use_accumulated_input) - { - current_input_chunk = std::move(current_output_chunk); - } - else - { - InputPort & input = finished_second_input ? *first_input : *second_input; + InputPort & input = finished_second_input ? inputs.front() : inputs.back(); - input.setNeeded(); - if (!input.hasData()) - return Status::NeedData; - - current_input_chunk = input.pull(); - } + input.setNeeded(); + if (!input.hasData()) + return Status::NeedData; + current_input_chunk = input.pull(); has_input = true; } @@ -136,7 +115,7 @@ size_t IntersectOrExceptTransform::buildFilter( for (size_t i = 0; i < rows; ++i) { auto find_result = state.findKey(method.data, i, variants.string_pool); - filter[i] = operators[current_operator_pos] == ASTIntersectOrExcept::Operator::EXCEPT ? !find_result.isFound() : find_result.isFound(); + filter[i] = current_operator == ASTSelectIntersectExceptQuery::Operator::EXCEPT ? !find_result.isFound() : find_result.isFound(); if (filter[i]) ++new_rows_num; } @@ -193,10 +172,11 @@ void IntersectOrExceptTransform::filter(Chunk & chunk) if (data->empty()) data->init(SetVariants::chooseMethod(column_ptrs, key_sizes)); - IColumn::Filter filter(num_rows); - size_t new_rows_num = 0; + + IColumn::Filter filter(num_rows); auto & data_set = *data; + switch (data->type) { case SetVariants::Type::EMPTY: @@ -209,6 +189,9 @@ void IntersectOrExceptTransform::filter(Chunk & chunk) #undef M } + if (!new_rows_num) + return; + for (auto & column : columns) column = column->filter(filter, -1); diff --git a/src/Processors/Transforms/IntersectOrExceptTransform.h b/src/Processors/Transforms/IntersectOrExceptTransform.h index 6d0c3516d5d..da1fa6a119e 100644 --- a/src/Processors/Transforms/IntersectOrExceptTransform.h +++ b/src/Processors/Transforms/IntersectOrExceptTransform.h @@ -3,7 +3,7 @@ #include #include #include -#include +#include namespace DB @@ -11,10 +11,10 @@ namespace DB class IntersectOrExceptTransform : public IProcessor { -using Operators = ASTIntersectOrExcept::Operators; +using Operator = ASTSelectIntersectExceptQuery::Operator; public: - IntersectOrExceptTransform(const Block & header_, const Operators & operators); + IntersectOrExceptTransform(const Block & header_, Operator operators); String getName() const override { return "IntersectOrExcept"; } @@ -24,10 +24,7 @@ protected: void work() override; private: - Operators operators; - InputPorts::iterator first_input; - InputPorts::iterator second_input; - size_t current_operator_pos = 0; + Operator current_operator; ColumnNumbers key_columns_pos; std::optional data; @@ -36,7 +33,6 @@ private: Chunk current_input_chunk; Chunk current_output_chunk; - bool use_accumulated_input = false; bool finished_second_input = false; bool has_input = false; diff --git a/tests/queries/0_stateless/02004_intersect_except_operators.reference b/tests/queries/0_stateless/02004_intersect_except_operators.reference index d17216a5ec4..c3272a5d574 100644 --- a/tests/queries/0_stateless/02004_intersect_except_operators.reference +++ b/tests/queries/0_stateless/02004_intersect_except_operators.reference @@ -32,6 +32,16 @@ select 1 intersect select 1 except select 1; select 1 intersect select 1 except select 2 intersect select 1 except select 3 intersect select 1; 1 select 1 intersect select 1 except select 2 intersect select 1 except select 3 intersect select 2; +select number from numbers(10) except select 5; +0 +1 +2 +3 +4 +6 +7 +8 +9 select number from numbers(100) intersect select number from numbers(20, 60) except select number from numbers(30, 20) except select number from numbers(60, 20); 20 21 @@ -53,3 +63,18 @@ select number from numbers(100) intersect select number from numbers(20, 60) exc 57 58 59 +with (select number from numbers(10) intersect select 5) as a select a * 10; +50 +select count() from (select number from numbers(10) except select 5); +9 +select count() from (select number from numbers(1000000) intersect select number from numbers(200000, 600000)); +600000 +select count() from (select number from numbers(100) intersect select number from numbers(20, 60) except select number from numbers(30, 20) except select number from numbers(60, 20)); +20 +select count() from (select number from numbers(1000000) intersect select number from numbers(200000, 600000) except select number from numbers(300000, 200000) except select number from numbers(600000, 200000)); +200000 +select 1 union all select 1 intersect select 1; +1 +1 +select 1 union all select 1 intersect select 2; +1 diff --git a/tests/queries/0_stateless/02004_intersect_except_operators.sql b/tests/queries/0_stateless/02004_intersect_except_operators.sql index 971aa262070..722670732ac 100644 --- a/tests/queries/0_stateless/02004_intersect_except_operators.sql +++ b/tests/queries/0_stateless/02004_intersect_except_operators.sql @@ -15,4 +15,14 @@ select 1 intersect select 1 except select 1; select 1 intersect select 1 except select 2 intersect select 1 except select 3 intersect select 1; select 1 intersect select 1 except select 2 intersect select 1 except select 3 intersect select 2; +select number from numbers(10) except select 5; select number from numbers(100) intersect select number from numbers(20, 60) except select number from numbers(30, 20) except select number from numbers(60, 20); + +with (select number from numbers(10) intersect select 5) as a select a * 10; +select count() from (select number from numbers(10) except select 5); +select count() from (select number from numbers(1000000) intersect select number from numbers(200000, 600000)); +select count() from (select number from numbers(100) intersect select number from numbers(20, 60) except select number from numbers(30, 20) except select number from numbers(60, 20)); +select count() from (select number from numbers(1000000) intersect select number from numbers(200000, 600000) except select number from numbers(300000, 200000) except select number from numbers(600000, 200000)); + +select 1 union all select 1 intersect select 1; +select 1 union all select 1 intersect select 2; From 74db1eafd5c68c0eb2cbfaa028ebf66d26675678 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Thu, 12 Aug 2021 16:43:56 +0300 Subject: [PATCH 096/161] Add ru docs for uuid and array functions. --- docs/ru/sql-reference/functions/uuid-functions.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/sql-reference/functions/uuid-functions.md b/docs/ru/sql-reference/functions/uuid-functions.md index 977446fdf06..0d534a2d7ce 100644 --- a/docs/ru/sql-reference/functions/uuid-functions.md +++ b/docs/ru/sql-reference/functions/uuid-functions.md @@ -47,7 +47,7 @@ empty(UUID) UUID считается пустым, если он содержит все нули (нулевой UUID). -Функция также поддерживает работу с типами [Array](array-functions.md#empty) и [String](string-functions.md#empty). +Функция также поддерживает работу с типами [Array](array-functions.md#function-empty) и [String](string-functions.md#empty). **Параметры** @@ -89,7 +89,7 @@ notEmpty(UUID) UUID считается пустым, если он содержит все нули (нулевой UUID). -Функция также поддерживает работу с типами [Array](array-functions.md#notempty) и [String](string-functions.md#notempty). +Функция также поддерживает работу с типами [Array](array-functions.md#function-notempty) и [String](string-functions.md#function-notempty). **Параметры** From 276d757c67676f25534d7113f0283c8505950932 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Thu, 12 Aug 2021 16:56:00 +0300 Subject: [PATCH 097/161] Add ru docs for uuid and array functions. --- docs/ru/sql-reference/functions/string-functions.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index 75a4af84297..5c2b3226219 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -17,6 +17,8 @@ empty(x) Строка считается непустой, если содержит хотя бы один байт, пусть даже это пробел или нулевой байт. +Функция также поддерживает работу с типами [Array](array-functions.md#function-empty) и [UUID](uuid-functions.md#empty). + **Параметры** - `x` — Входная строка. [String](../data-types/string.md). @@ -55,6 +57,8 @@ notEmpty(x) Строка считается непустой, если содержит хотя бы один байт, пусть даже это пробел или нулевой байт. +Функция также поддерживает работу с типами [Array](array-functions.md#function-notempty) и [UUID](uuid-functions.md#notempty). + **Параметры** - `x` — Входная строка. [String](../data-types/string.md). From 931bed02b0e2b0110c5188c8b740bd0ccc9188d3 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Thu, 12 Aug 2021 17:47:46 +0300 Subject: [PATCH 098/161] Update Settings.h --- src/Core/Settings.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 3c30c6b0b1d..e1bd1d29153 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -437,7 +437,6 @@ class IColumn; M(String, function_implementation, "", "Choose function implementation for specific target or variant (experimental). If empty enable all of them.", 0) \ M(Bool, allow_experimental_geo_types, false, "Allow geo data types such as Point, Ring, Polygon, MultiPolygon", 0) \ M(Bool, data_type_default_nullable, false, "Data types without NULL or NOT NULL will make Nullable", 0) \ - /** Temporarily set to true, to check how tests will feel.*/ \ M(Bool, cast_keep_nullable, false, "CAST operator keep Nullable for result data type", 0) \ M(Bool, alter_partition_verbose_result, false, "Output information about affected parts. Currently works only for FREEZE and ATTACH commands.", 0) \ M(Bool, allow_experimental_database_materialized_mysql, false, "Allow to create database with Engine=MaterializedMySQL(...).", 0) \ From 27cb83f5a74aabbd6f2e62b892068e0e91bf123b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 12 Aug 2021 17:08:39 +0200 Subject: [PATCH 099/161] Get today datetime in UTC --- .../queries/0_stateless/01236_graphite_mt.sql | 36 +++++++++---------- 1 file changed, 18 insertions(+), 18 deletions(-) diff --git a/tests/queries/0_stateless/01236_graphite_mt.sql b/tests/queries/0_stateless/01236_graphite_mt.sql index d5289364d32..ccf7c066e75 100644 --- a/tests/queries/0_stateless/01236_graphite_mt.sql +++ b/tests/queries/0_stateless/01236_graphite_mt.sql @@ -9,28 +9,28 @@ SET joined_subquery_requires_alias = 0; INSERT into test_graphite WITH dates AS ( - SELECT today() as today, - today - 3 as older_date + SELECT toStartOfDay(toDateTime(now('UTC'), 'UTC')) as today, + today - INTERVAL 3 day as older_date ) -- Newer than 2 days are kept in windows of 600 seconds - select 1, 'sum_1', toDateTime(today, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(300) union all - select 2, 'sum_1', toDateTime(today, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(300) union all - select 1, 'sum_2', toDateTime(today, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(300) union all - select 2, 'sum_2', toDateTime(today, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(300) union all - select 1, 'max_1', toDateTime(today, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(300) union all - select 2, 'max_1', toDateTime(today, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(300) union all - select 1, 'max_2', toDateTime(today, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(300) union all - select 2, 'max_2', toDateTime(today, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(300) union all + select 1, 'sum_1', today - number * 60 - 30, number, 1, number from dates, numbers(300) union all + select 2, 'sum_1', today - number * 60 - 30, number, 1, number from dates, numbers(300) union all + select 1, 'sum_2', today - number * 60 - 30, number, 1, number from dates, numbers(300) union all + select 2, 'sum_2', today - number * 60 - 30, number, 1, number from dates, numbers(300) union all + select 1, 'max_1', today - number * 60 - 30, number, 1, number from dates, numbers(300) union all + select 2, 'max_1', today - number * 60 - 30, number, 1, number from dates, numbers(300) union all + select 1, 'max_2', today - number * 60 - 30, number, 1, number from dates, numbers(300) union all + select 2, 'max_2', today - number * 60 - 30, number, 1, number from dates, numbers(300) union all -- Older than 2 days use 6000 second windows - select 1, 'sum_1', toDateTime(older_date, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(1200) union all - select 2, 'sum_1', toDateTime(older_date, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(1200) union all - select 1, 'sum_2', toDateTime(older_date, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(1200) union all - select 2, 'sum_2', toDateTime(older_date, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(1200) union all - select 1, 'max_1', toDateTime(older_date, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(1200) union all - select 2, 'max_1', toDateTime(older_date, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(1200) union all - select 1, 'max_2', toDateTime(older_date, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(1200) union all - select 2, 'max_2', toDateTime(older_date, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(1200); + select 1, 'sum_1', older_date - number * 60 - 30, number, 1, number from dates, numbers(1200) union all + select 2, 'sum_1', older_date - number * 60 - 30, number, 1, number from dates, numbers(1200) union all + select 1, 'sum_2', older_date - number * 60 - 30, number, 1, number from dates, numbers(1200) union all + select 2, 'sum_2', older_date - number * 60 - 30, number, 1, number from dates, numbers(1200) union all + select 1, 'max_1', older_date - number * 60 - 30, number, 1, number from dates, numbers(1200) union all + select 2, 'max_1', older_date - number * 60 - 30, number, 1, number from dates, numbers(1200) union all + select 1, 'max_2', older_date - number * 60 - 30, number, 1, number from dates, numbers(1200) union all + select 2, 'max_2', older_date - number * 60 - 30, number, 1, number from dates, numbers(1200); optimize table test_graphite final; From d9a59370d3774a7738cea3fd32da58a5e4a2240e Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 13 Aug 2021 00:58:24 +0300 Subject: [PATCH 100/161] Fixed SQLiteSource --- src/DataStreams/SQLiteSource.cpp | 58 ++++++++++--------- src/DataStreams/SQLiteSource.h | 7 +-- src/Dictionaries/RedisDictionarySource.cpp | 2 +- ...isBlockInputStream.cpp => RedisSource.cpp} | 2 +- ...{RedisBlockInputStream.h => RedisSource.h} | 0 src/Dictionaries/ya.make | 6 +- 6 files changed, 39 insertions(+), 36 deletions(-) rename src/Dictionaries/{RedisBlockInputStream.cpp => RedisSource.cpp} (99%) rename src/Dictionaries/{RedisBlockInputStream.h => RedisSource.h} (100%) diff --git a/src/DataStreams/SQLiteSource.cpp b/src/DataStreams/SQLiteSource.cpp index f4995703a1e..d0d8724c2dd 100644 --- a/src/DataStreams/SQLiteSource.cpp +++ b/src/DataStreams/SQLiteSource.cpp @@ -23,32 +23,32 @@ namespace ErrorCodes } SQLiteSource::SQLiteSource( - SQLitePtr sqlite_db_, - const String & query_str_, - const Block & sample_block, - const UInt64 max_block_size_) + SQLitePtr sqlite_db_, + const String & query_str_, + const Block & sample_block, + const UInt64 max_block_size_) : SourceWithProgress(sample_block.cloneEmpty()) , query_str(query_str_) , max_block_size(max_block_size_) , sqlite_db(std::move(sqlite_db_)) { description.init(sample_block); + + sqlite3_stmt * compiled_stmt = nullptr; + int status = sqlite3_prepare_v2(sqlite_db.get(), query_str.c_str(), query_str.size() + 1, &compiled_stmt, nullptr); + + if (status != SQLITE_OK) + throw Exception(ErrorCodes::SQLITE_ENGINE_ERROR, + "Cannot prepate sqlite statement. Status: {}. Message: {}", + status, sqlite3_errstr(status)); + + compiled_statement = std::unique_ptr(compiled_stmt, StatementDeleter()); } Chunk SQLiteSource::generate() { if (!compiled_statement) - { - sqlite3_stmt * compiled_stmt = nullptr; - int status = sqlite3_prepare_v2(sqlite_db.get(), query_str.c_str(), query_str.size() + 1, &compiled_stmt, nullptr); - - if (status != SQLITE_OK) - throw Exception(ErrorCodes::SQLITE_ENGINE_ERROR, - "Cannot prepate sqlite statement. Status: {}. Message: {}", - status, sqlite3_errstr(status)); - - compiled_statement = std::unique_ptr(compiled_stmt, StatementDeleter()); - } + return {}; MutableColumns columns = description.sample_block.cloneEmptyColumns(); size_t num_rows = 0; @@ -69,30 +69,30 @@ Chunk SQLiteSource::generate() else if (status != SQLITE_ROW) { throw Exception(ErrorCodes::SQLITE_ENGINE_ERROR, - "Expected SQLITE_ROW status, but got status {}. Error: {}, Message: {}", - status, sqlite3_errstr(status), sqlite3_errmsg(sqlite_db.get())); + "Expected SQLITE_ROW status, but got status {}. Error: {}, Message: {}", + status, sqlite3_errstr(status), sqlite3_errmsg(sqlite_db.get())); } int column_count = sqlite3_column_count(compiled_statement.get()); - for (const auto idx : collections::range(0, column_count)) - { - const auto & sample = description.sample_block.getByPosition(idx); - if (sqlite3_column_type(compiled_statement.get(), idx) == SQLITE_NULL) + for (int column_index = 0; column_index < column_count; ++column_index) + { + if (sqlite3_column_type(compiled_statement.get(), column_index) == SQLITE_NULL) { - insertDefaultSQLiteValue(*columns[idx], *sample.column); + columns[column_index]->insertDefault(); continue; } - if (description.types[idx].second) + auto & [type, is_nullable] = description.types[column_index]; + if (is_nullable) { - ColumnNullable & column_nullable = assert_cast(*columns[idx]); - insertValue(column_nullable.getNestedColumn(), description.types[idx].first, idx); + ColumnNullable & column_nullable = assert_cast(*columns[column_index]); + insertValue(column_nullable.getNestedColumn(), type, column_index); column_nullable.getNullMapData().emplace_back(0); } else { - insertValue(*columns[idx], description.types[idx].first, idx); + insertValue(*columns[column_index], type, column_index); } } @@ -100,6 +100,12 @@ Chunk SQLiteSource::generate() break; } + if (num_rows == 0) + { + compiled_statement.reset(); + return {}; + } + return Chunk(std::move(columns), num_rows); } diff --git a/src/DataStreams/SQLiteSource.h b/src/DataStreams/SQLiteSource.h index 653fdb402e3..0f8b42c536b 100644 --- a/src/DataStreams/SQLiteSource.h +++ b/src/DataStreams/SQLiteSource.h @@ -13,8 +13,10 @@ namespace DB { + class SQLiteSource : public SourceWithProgress { + using SQLitePtr = std::shared_ptr; public: @@ -26,10 +28,6 @@ public: String getName() const override { return "SQLite"; } private: - static void insertDefaultSQLiteValue(IColumn & column, const IColumn & sample_column) - { - column.insertFrom(sample_column, 0); - } using ValueType = ExternalResultDescription::ValueType; @@ -46,7 +44,6 @@ private: UInt64 max_block_size; ExternalResultDescription description; - SQLitePtr sqlite_db; std::unique_ptr compiled_statement; }; diff --git a/src/Dictionaries/RedisDictionarySource.cpp b/src/Dictionaries/RedisDictionarySource.cpp index bf309dd0e8a..6561a122e9d 100644 --- a/src/Dictionaries/RedisDictionarySource.cpp +++ b/src/Dictionaries/RedisDictionarySource.cpp @@ -31,7 +31,7 @@ void registerDictionarySourceRedis(DictionarySourceFactory & factory) #include -#include "RedisBlockInputStream.h" +#include "RedisSource.h" namespace DB diff --git a/src/Dictionaries/RedisBlockInputStream.cpp b/src/Dictionaries/RedisSource.cpp similarity index 99% rename from src/Dictionaries/RedisBlockInputStream.cpp rename to src/Dictionaries/RedisSource.cpp index c6e2546cf1e..ad5cf8a0977 100644 --- a/src/Dictionaries/RedisBlockInputStream.cpp +++ b/src/Dictionaries/RedisSource.cpp @@ -1,4 +1,4 @@ -#include "RedisBlockInputStream.h" +#include "RedisSource.h" #include #include diff --git a/src/Dictionaries/RedisBlockInputStream.h b/src/Dictionaries/RedisSource.h similarity index 100% rename from src/Dictionaries/RedisBlockInputStream.h rename to src/Dictionaries/RedisSource.h diff --git a/src/Dictionaries/ya.make b/src/Dictionaries/ya.make index 2cff8447574..3f287f8bddc 100644 --- a/src/Dictionaries/ya.make +++ b/src/Dictionaries/ya.make @@ -26,9 +26,9 @@ SRCS( CassandraHelpers.cpp CassandraSource.cpp ClickHouseDictionarySource.cpp - DictionaryBlockInputStream.cpp - DictionaryBlockInputStreamBase.cpp DictionaryFactory.cpp + DictionarySource.cpp + DictionarySourceBase.cpp DictionarySourceFactory.cpp DictionarySourceHelpers.cpp DictionaryStructure.cpp @@ -57,8 +57,8 @@ SRCS( PolygonDictionaryImplementations.cpp PolygonDictionaryUtils.cpp RangeHashedDictionary.cpp - RedisBlockInputStream.cpp RedisDictionarySource.cpp + RedisSource.cpp XDBCDictionarySource.cpp getDictionaryConfigurationFromAST.cpp readInvalidateQuery.cpp From 36db9cf55a4a8675fabe3a944146f81f10224e5e Mon Sep 17 00:00:00 2001 From: Vladimir Goncharov Date: Fri, 13 Aug 2021 00:59:16 +0300 Subject: [PATCH 101/161] Update mergetree.md --- .../mergetree-family/mergetree.md | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/docs/ru/engines/table-engines/mergetree-family/mergetree.md b/docs/ru/engines/table-engines/mergetree-family/mergetree.md index 4bced6254d1..728c632370c 100644 --- a/docs/ru/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/ru/engines/table-engines/mergetree-family/mergetree.md @@ -375,6 +375,24 @@ INDEX b (u64 * length(str), i32 + f64 * 100, date, str) TYPE set(100) GRANULARIT - `s != 1` - `NOT startsWith(s, 'test')` +### Проекции {#projections} +Проекции похожи на материализованные представления, но определяются на уровне партов. Это обеспечивает гарантии согласованности наряду с автоматическим использованием в запросах. + +#### Запрос {#projection-query} +Запрос проекции - это то, что определяет проекцию. Он имеет следующую грамматику: + +`SELECT [GROUP BY] [ORDER BY]` + +Это неявно выбирает данные из родительской таблицы. + +#### Хранение {#projection-storage} +Проекции хранятся в каталоге парта. Это похоже на хранение индексов, но используется подкаталог, в котором хранится анонимный парт таблицы MergeTree. Таблица создается запросом определения проекции. Если есть конструкция GROUP BY, то базовый механизм хранения становится AggregatedMergeTree, а все агрегатные функции преобразуются в AggregateFunction. Если есть конструкция ORDER BY, таблица MergeTree будет использовать его в качестве выражения первичного ключа. Во время процесса слияния парт проекции будет слит с помощью процедуры слияния ее хранилища. Контрольная сумма парта родительской таблицы будет включать парт проекции. Другие процедуры аналогичны индексам пропуска данных. + +#### Анализ запросов {#projection-query-analysis} +1. Проверить, можно ли использовать проекцию для результата на данный запрос, то есть она даёт тот же результат, что и запрос к базовой таблице. +2. Выбрать наиболее подходящее совпадение, содержащее наименьшее количество гранул для чтения. +3. План запроса, который использует проекции, будет отличаться от того, который использует исходные парты. Если в некоторых партах проекция отсутствует, мы можем расширить план, чтобы «проецировать» на лету. + ## Конкурентный доступ к данным {#concurrent-data-access} Для конкурентного доступа к таблице используется мультиверсионность. То есть, при одновременном чтении и обновлении таблицы, данные будут читаться из набора кусочков, актуального на момент запроса. Длинных блокировок нет. Вставки никак не мешают чтениям. From c83551ef8ecda55ee77f9125caaf686b3714d35d Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev Date: Thu, 12 Aug 2021 21:10:20 -0300 Subject: [PATCH 102/161] enable part_log by default --- programs/server/config.xml | 4 ++-- programs/server/config.yaml.example | 9 +++++---- .../test_config_corresponding_root/configs/config.xml | 5 ++--- .../integration/test_config_xml_full/configs/config.xml | 6 +++--- .../aes_encryption/configs/clickhouse/config.xml | 4 ++-- .../configs/clickhouse/config.xml | 4 ++-- tests/testflows/example/configs/clickhouse/config.xml | 4 ++-- .../configs/clickhouse/config.xml | 4 ++-- tests/testflows/kerberos/configs/clickhouse/config.xml | 4 ++-- .../ldap/authentication/configs/clickhouse/config.xml | 4 ++-- .../configs/clickhouse/config.xml | 4 ++-- .../ldap/role_mapping/configs/clickhouse/config.xml | 4 ++-- tests/testflows/map_type/configs/clickhouse/config.xml | 4 ++-- tests/testflows/rbac/configs/clickhouse/config.xml | 4 ++-- .../window_functions/configs/clickhouse/config.xml | 4 ++-- 15 files changed, 34 insertions(+), 34 deletions(-) diff --git a/programs/server/config.xml b/programs/server/config.xml index 136b982a181..510a5e230f8 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -888,13 +888,13 @@ system part_log
+ toYYYYMM(event_date) 7500
- --> system part_log
+ toYYYYMM(event_date) 7500
- --> - - + @@ -838,13 +838,13 @@ system part_log
+ toYYYYMM(event_date) 7500
- --> system part_log
+ toYYYYMM(event_date) 7500
- --> system part_log
+ toYYYYMM(event_date) 7500
- --> system part_log
+ toYYYYMM(event_date) 7500
- --> system part_log
+ toYYYYMM(event_date) 7500
- --> system part_log
+ toYYYYMM(event_date) 7500
- --> system part_log
+ toYYYYMM(event_date) 7500
- --> system part_log
+ toYYYYMM(event_date) 7500
- --> system part_log
+ toYYYYMM(event_date) 7500
- --> system part_log
+ toYYYYMM(event_date) 7500
- --> system part_log
+ toYYYYMM(event_date) 7500
- --> system part_log
+ toYYYYMM(event_date) 7500
- --> -------SelectIntersectExceptQuery @@ -59,7 +74,8 @@ void SelectIntersectExceptQueryMatcher::visit(ASTSelectIntersectExceptQuery & as { auto select_intersect_except = std::make_shared(); select_intersect_except->final_operator = {current_operator}; - select_intersect_except->children.emplace_back(std::move(list_node)); + select_intersect_except->list_of_selects = std::move(list_node); + select_intersect_except->children.push_back(select_intersect_except->list_of_selects); selects.emplace_back(std::move(select_intersect_except)); } @@ -67,19 +83,88 @@ void SelectIntersectExceptQueryMatcher::visit(ASTSelectIntersectExceptQuery & as visit(ast, data); } -// void SelectIntersectExceptQueryVisitor::visit(ASTSelectWithUnionQuery & ast, Data & data) -// { -// auto & union_modes = ast.list_of_modes; -// ASTs selects; -// auto & select_list = ast.list_of_selects->children; -// -// -// // reverse children list -// std::reverse(selects.begin(), selects.end()); -// -// ast.is_normalized = true; -// ast.union_mode = ASTSelectWithUnionQuery::Mode::ALL; -// -// ast.list_of_selects->children = std::move(selects); -// } +void SelectIntersectExceptQueryMatcher::visit(ASTSelectWithUnionQuery & ast, Data &) +{ + /* Example: select 1 union all select 2 except select 1 except select 2 union distinct select 5; + * + * --SelectWithUnionQuery --SelectIntersectExceptQuery + * ---ExpressionList ---ExpressionList + * ----SelectQuery ----SelectIntersectExceptQuery + * ----SelectQuery -----ExpressionList + * ----SelectQuery (except) ---> ------SelectIntersectExceptQuery + * ----SelectQuery (except) -------ExpressionList + * ----SelectQuery --------SelectWithUnionQuery (select 1 union all select 2) + * --------SelectQuery (select 1) + * ------SelectQuery (select 2) + * -----SelectQuery (select 5) + **/ + + auto & union_modes = ast.list_of_modes; + + if (union_modes.empty()) + return; + + auto selects = std::move(ast.list_of_selects->children); + + if (union_modes.size() + 1 != selects.size()) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Incorrect ASTSelectWithUnionQuery (modes: {}, selects: {})", + union_modes.size(), selects.size()); + + std::reverse(selects.begin(), selects.end()); + + ASTs children = {selects.back()}; + selects.pop_back(); + ASTSelectWithUnionQuery::UnionModes modes; + + for (const auto & mode : union_modes) + { + /// Flatten all previous selects into ASTSelectIntersectQuery + if (mode == ASTSelectWithUnionQuery::Mode::EXCEPT) + { + auto left = std::make_shared(); + left->union_mode = ASTSelectWithUnionQuery::Mode::ALL; + + left->list_of_selects = std::make_shared(); + left->children.push_back(left->list_of_selects); + left->list_of_selects->children = std::move(children); + + left->list_of_modes = std::move(modes); + modes = {}; + + auto right = selects.back(); + selects.pop_back(); + + auto list_node = std::make_shared(); + list_node->children = {left, right}; + + auto select_intersect_except = std::make_shared(); + select_intersect_except->final_operator = {ASTSelectIntersectExceptQuery::Operator::EXCEPT}; + select_intersect_except->children.emplace_back(std::move(list_node)); + select_intersect_except->list_of_selects = std::make_shared(); + select_intersect_except->list_of_selects->children.push_back(select_intersect_except->children[0]); + + children = {select_intersect_except}; + } + else if (!selects.empty()) + { + auto right = selects.back(); + selects.pop_back(); + children.emplace_back(std::move(right)); + modes.push_back(mode); + } + } + + if (!selects.empty()) + { + auto right = selects.back(); + selects.pop_back(); + children.emplace_back(std::move(right)); + } + + ast.union_mode = ASTSelectWithUnionQuery::Mode::ALL; + ast.list_of_selects->children = std::move(children); + ast.list_of_modes = std::move(modes); +} + } diff --git a/src/Interpreters/SelectIntersectExceptQueryVisitor.h b/src/Interpreters/SelectIntersectExceptQueryVisitor.h index 58f3071972f..1dd0694666d 100644 --- a/src/Interpreters/SelectIntersectExceptQueryVisitor.h +++ b/src/Interpreters/SelectIntersectExceptQueryVisitor.h @@ -38,7 +38,7 @@ public: static void visit(ASTPtr & ast, Data &); static void visit(ASTSelectIntersectExceptQuery &, Data &); - // static void visit(ASTSelectWithUnionQuery &, Data &); + static void visit(ASTSelectWithUnionQuery &, Data &); }; /// Visit children first. diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 839447a90f7..4131bac28d1 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -492,14 +492,14 @@ static std::tuple executeQueryImpl( } { - /// Normalize SelectWithUnionQuery - NormalizeSelectWithUnionQueryVisitor::Data data{context->getSettingsRef().union_default_mode}; - NormalizeSelectWithUnionQueryVisitor{data}.visit(ast); + SelectIntersectExceptQueryVisitor::Data data; + SelectIntersectExceptQueryVisitor{data}.visit(ast); } { - SelectIntersectExceptQueryVisitor::Data data; - SelectIntersectExceptQueryVisitor{data}.visit(ast); + /// Normalize SelectWithUnionQuery + NormalizeSelectWithUnionQueryVisitor::Data data{context->getSettingsRef().union_default_mode}; + NormalizeSelectWithUnionQueryVisitor{data}.visit(ast); } /// Check the limits. @@ -540,7 +540,6 @@ static std::tuple executeQueryImpl( /// reset Input callbacks if query is not INSERT SELECT context->resetInputCallbacks(); - std::cerr << "\n\nAST: " << ast->dumpTree() << std::endl; auto interpreter = InterpreterFactory::get(ast, context, SelectQueryOptions(stage).setInternal(internal)); std::shared_ptr quota; diff --git a/src/Parsers/ASTSelectWithUnionQuery.cpp b/src/Parsers/ASTSelectWithUnionQuery.cpp index fa7359574f8..d19e860c16a 100644 --- a/src/Parsers/ASTSelectWithUnionQuery.cpp +++ b/src/Parsers/ASTSelectWithUnionQuery.cpp @@ -8,6 +8,10 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} ASTPtr ASTSelectWithUnionQuery::clone() const { @@ -28,6 +32,9 @@ ASTPtr ASTSelectWithUnionQuery::clone() const void ASTSelectWithUnionQuery::formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { + if (!list_of_selects || list_of_selects->children.size() != list_of_modes.size() + 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Incorrect ASTSelectWithUnionQuery"); + std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' '); auto mode_to_str = [&](auto mode) diff --git a/src/Parsers/ASTSelectWithUnionQuery.h b/src/Parsers/ASTSelectWithUnionQuery.h index 0465bdac3a6..2c36bcecf6b 100644 --- a/src/Parsers/ASTSelectWithUnionQuery.h +++ b/src/Parsers/ASTSelectWithUnionQuery.h @@ -22,7 +22,8 @@ public: { Unspecified, ALL, - DISTINCT + DISTINCT, + EXCEPT }; using UnionModes = std::vector; diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index e22f2c7cded..e75aad8d02f 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -140,7 +140,14 @@ bool ParserUnionList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) } // SELECT ... UNION SELECT ... else + { union_modes.push_back(ASTSelectWithUnionQuery::Mode::Unspecified); + } + return true; + } + else if (s_except_parser->check(pos, expected)) + { + union_modes.push_back(ASTSelectWithUnionQuery::Mode::EXCEPT); return true; } return false; @@ -1024,4 +1031,3 @@ bool ParserKeyValuePairsList::parseImpl(Pos & pos, ASTPtr & node, Expected & exp } } - diff --git a/src/Parsers/ExpressionListParsers.h b/src/Parsers/ExpressionListParsers.h index ef6a5744603..36f39a50ab3 100644 --- a/src/Parsers/ExpressionListParsers.h +++ b/src/Parsers/ExpressionListParsers.h @@ -79,11 +79,17 @@ private: class ParserUnionList : public IParserBase { public: - ParserUnionList(ParserPtr && elem_parser_, ParserPtr && s_union_parser_, ParserPtr && s_all_parser_, ParserPtr && s_distinct_parser_) + ParserUnionList( + ParserPtr && elem_parser_, + ParserPtr && s_union_parser_, + ParserPtr && s_all_parser_, + ParserPtr && s_distinct_parser_, + ParserPtr && s_except_parser_) : elem_parser(std::move(elem_parser_)) , s_union_parser(std::move(s_union_parser_)) , s_all_parser(std::move(s_all_parser_)) , s_distinct_parser(std::move(s_distinct_parser_)) + , s_except_parser(std::move(s_except_parser_)) { } @@ -120,6 +126,7 @@ private: ParserPtr s_union_parser; ParserPtr s_all_parser; ParserPtr s_distinct_parser; + ParserPtr s_except_parser; ASTSelectWithUnionQuery::UnionModes union_modes; }; diff --git a/src/Parsers/ParserSelectIntersectExceptQuery.cpp b/src/Parsers/ParserSelectIntersectExceptQuery.cpp index b56598166c6..2b4ba9d60e2 100644 --- a/src/Parsers/ParserSelectIntersectExceptQuery.cpp +++ b/src/Parsers/ParserSelectIntersectExceptQuery.cpp @@ -14,7 +14,6 @@ namespace DB bool ParserSelectIntersectExceptQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ParserKeyword intersect_keyword("INTERSECT"); - ParserKeyword except_keyword("EXCEPT"); ASTs elements; ASTSelectIntersectExceptQuery::Operators operators; @@ -32,13 +31,7 @@ bool ParserSelectIntersectExceptQuery::parseImpl(Pos & pos, ASTPtr & node, Expec auto parse_separator = [&]() -> bool { if (!intersect_keyword.ignore(pos)) - { - if (!except_keyword.ignore(pos)) - return false; - - operators.emplace_back(ASTSelectIntersectExceptQuery::Operator::EXCEPT); - return true; - } + return false; operators.emplace_back(ASTSelectIntersectExceptQuery::Operator::INTERSECT); return true; diff --git a/src/Parsers/ParserSelectWithUnionQuery.cpp b/src/Parsers/ParserSelectWithUnionQuery.cpp index 87e2dab1a47..8c4c183a099 100644 --- a/src/Parsers/ParserSelectWithUnionQuery.cpp +++ b/src/Parsers/ParserSelectWithUnionQuery.cpp @@ -15,7 +15,8 @@ bool ParserSelectWithUnionQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & std::make_unique(), std::make_unique("UNION"), std::make_unique("ALL"), - std::make_unique("DISTINCT")); + std::make_unique("DISTINCT"), + std::make_unique("EXCEPT")); if (!parser.parse(pos, list_node, expected)) return false; diff --git a/src/Processors/QueryPlan/IntersectOrExceptStep.cpp b/src/Processors/QueryPlan/IntersectOrExceptStep.cpp index e61afb5ba2a..e4d04115cff 100644 --- a/src/Processors/QueryPlan/IntersectOrExceptStep.cpp +++ b/src/Processors/QueryPlan/IntersectOrExceptStep.cpp @@ -65,6 +65,8 @@ QueryPipelinePtr IntersectOrExceptStep::updatePipeline(QueryPipelines pipelines, return std::make_shared(cur_header, converting_actions); }); } + + cur_pipeline->addTransform(std::make_shared(header, cur_pipeline->getNumStreams(), 1)); } *pipeline = QueryPipeline::unitePipelines(std::move(pipelines), max_threads); diff --git a/tests/queries/0_stateless/02004_intersect_except_operators.reference b/tests/queries/0_stateless/02004_intersect_except_operators.reference index c3272a5d574..9a9e4e1bf58 100644 --- a/tests/queries/0_stateless/02004_intersect_except_operators.reference +++ b/tests/queries/0_stateless/02004_intersect_except_operators.reference @@ -5,7 +5,7 @@ select 2 intersect select 1; select 1 except select 1; select 2 except select 1; 2 -select number from numbers(5, 5) intersect select number from numbers(20); +select number from numbers(20) intersect select number from numbers(5, 5); 5 6 7 @@ -26,12 +26,15 @@ select number, number+10 from numbers(12) except select number+5, number+15 from select 1 except select 2 intersect select 1; 1 select 1 except select 2 intersect select 2; +1 select 1 intersect select 1 except select 2; 1 select 1 intersect select 1 except select 1; select 1 intersect select 1 except select 2 intersect select 1 except select 3 intersect select 1; 1 select 1 intersect select 1 except select 2 intersect select 1 except select 3 intersect select 2; +1 +select 1 intersect select 1 except select 2 intersect select 1 except select 3 intersect select 2 except select 1; select number from numbers(10) except select 5; 0 1 @@ -71,6 +74,8 @@ select count() from (select number from numbers(1000000) intersect select number 600000 select count() from (select number from numbers(100) intersect select number from numbers(20, 60) except select number from numbers(30, 20) except select number from numbers(60, 20)); 20 +select count() from (select number from numbers(100) intersect select number from numbers(20, 60) except select number from numbers(30, 20) except select number from numbers(60, 20) union all select number from numbers(100, 10)); +30 select count() from (select number from numbers(1000000) intersect select number from numbers(200000, 600000) except select number from numbers(300000, 200000) except select number from numbers(600000, 200000)); 200000 select 1 union all select 1 intersect select 1; @@ -78,3 +83,13 @@ select 1 union all select 1 intersect select 1; 1 select 1 union all select 1 intersect select 2; 1 +select * from (select 1 union all select 2 union all select 3 union all select 4 except select 3 union all select 5) order by 1; +1 +2 +4 +5 +select * from (select 1 union all select 2 union all select 3 union all select 4 intersect select 3 union all select 5) order by 1; +1 +2 +3 +5 diff --git a/tests/queries/0_stateless/02004_intersect_except_operators.sql b/tests/queries/0_stateless/02004_intersect_except_operators.sql index 722670732ac..c88951ef353 100644 --- a/tests/queries/0_stateless/02004_intersect_except_operators.sql +++ b/tests/queries/0_stateless/02004_intersect_except_operators.sql @@ -4,7 +4,7 @@ select 2 intersect select 1; select 1 except select 1; select 2 except select 1; -select number from numbers(5, 5) intersect select number from numbers(20); +select number from numbers(20) intersect select number from numbers(5, 5); select number from numbers(10) except select number from numbers(5); select number, number+10 from numbers(12) except select number+5, number+15 from numbers(10); @@ -14,6 +14,7 @@ select 1 intersect select 1 except select 2; select 1 intersect select 1 except select 1; select 1 intersect select 1 except select 2 intersect select 1 except select 3 intersect select 1; select 1 intersect select 1 except select 2 intersect select 1 except select 3 intersect select 2; +select 1 intersect select 1 except select 2 intersect select 1 except select 3 intersect select 2 except select 1; select number from numbers(10) except select 5; select number from numbers(100) intersect select number from numbers(20, 60) except select number from numbers(30, 20) except select number from numbers(60, 20); @@ -22,7 +23,10 @@ with (select number from numbers(10) intersect select 5) as a select a * 10; select count() from (select number from numbers(10) except select 5); select count() from (select number from numbers(1000000) intersect select number from numbers(200000, 600000)); select count() from (select number from numbers(100) intersect select number from numbers(20, 60) except select number from numbers(30, 20) except select number from numbers(60, 20)); +select count() from (select number from numbers(100) intersect select number from numbers(20, 60) except select number from numbers(30, 20) except select number from numbers(60, 20) union all select number from numbers(100, 10)); select count() from (select number from numbers(1000000) intersect select number from numbers(200000, 600000) except select number from numbers(300000, 200000) except select number from numbers(600000, 200000)); select 1 union all select 1 intersect select 1; select 1 union all select 1 intersect select 2; +select * from (select 1 union all select 2 union all select 3 union all select 4 except select 3 union all select 5) order by 1; +select * from (select 1 union all select 2 union all select 3 union all select 4 intersect select 3 union all select 5) order by 1; From c0f3c0a176f2deead29afcc2ce159f4cf59b6a99 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Fri, 13 Aug 2021 13:09:31 +0300 Subject: [PATCH 106/161] Update docs/ru/sql-reference/functions/array-functions.md Co-authored-by: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> --- docs/ru/sql-reference/functions/array-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/array-functions.md b/docs/ru/sql-reference/functions/array-functions.md index 066d37a71f5..a675f728bce 100644 --- a/docs/ru/sql-reference/functions/array-functions.md +++ b/docs/ru/sql-reference/functions/array-functions.md @@ -58,7 +58,7 @@ SELECT empty([]); notEmpty([x]) ``` -Массив считается непустым, если он содержит хотя бы один непустой элемент. +Массив считается непустым, если он содержит хотя бы один элемент. !!! note "Примечание" Функцию можно оптимизировать, если включить настройку [optimize_functions_to_subcolumns](../../operations/settings/settings.md#optimize-functions-to-subcolumns). При `optimize_functions_to_subcolumns = 1` функция читает только подстолбец [size0](../../sql-reference/data-types/array.md#array-size) вместо чтения и обработки всего столбца массива. Запрос `SELECT notEmpty(arr) FROM table` преобразуется к запросу `SELECT arr.size0 != 0 FROM TABLE`. From 279b5ef954018b432cda394685f199985b638fb7 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Fri, 13 Aug 2021 13:09:48 +0300 Subject: [PATCH 107/161] Update docs/ru/sql-reference/functions/array-functions.md Co-authored-by: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> --- docs/ru/sql-reference/functions/array-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/array-functions.md b/docs/ru/sql-reference/functions/array-functions.md index a675f728bce..b7a301d30a9 100644 --- a/docs/ru/sql-reference/functions/array-functions.md +++ b/docs/ru/sql-reference/functions/array-functions.md @@ -15,7 +15,7 @@ toc_title: "Массивы" empty([x]) ``` -Массив считается пустым, если он содержит все пустые элементы. +Массив считается пустым, если он не содержит ни одного элемента. !!! note "Примечание" Функцию можно оптимизировать, если включить настройку [optimize_functions_to_subcolumns](../../operations/settings/settings.md#optimize-functions-to-subcolumns). При `optimize_functions_to_subcolumns = 1` функция читает только подстолбец [size0](../../sql-reference/data-types/array.md#array-size) вместо чтения и обработки всего столбца массива. Запрос `SELECT empty(arr) FROM TABLE` преобразуется к запросу `SELECT arr.size0 = 0 FROM TABLE`. From 0875d72e5c267604140375e75f0ab92f09541dee Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Fri, 13 Aug 2021 13:13:09 +0300 Subject: [PATCH 108/161] Update docs/en/sql-reference/functions/array-functions.md Co-authored-by: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> --- docs/en/sql-reference/functions/array-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/array-functions.md b/docs/en/sql-reference/functions/array-functions.md index ddbbab3ecfc..89de48a81ad 100644 --- a/docs/en/sql-reference/functions/array-functions.md +++ b/docs/en/sql-reference/functions/array-functions.md @@ -15,7 +15,7 @@ Checks whether the input array is empty. empty([x]) ``` -An array is considered empty if it contains all empty elements. +An array is considered empty if it does not contain any elements. !!! note "Note" Can be optimized by enabling the [optimize_functions_to_subcolumns](../../operations/settings/settings.md#optimize-functions-to-subcolumns) setting. With `optimize_functions_to_subcolumns = 1` the function reads only [size0](../../sql-reference/data-types/array.md#array-size) subcolumn instead of reading and processing the whole array column. The query `SELECT empty(arr) FROM TABLE;` transforms to `SELECT arr.size0 = 0 FROM TABLE;`. From bbcdf03a8c937768f893ba064e075170d00b7c9b Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Fri, 13 Aug 2021 13:13:35 +0300 Subject: [PATCH 109/161] Update docs/en/sql-reference/functions/array-functions.md Co-authored-by: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> --- docs/en/sql-reference/functions/array-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/array-functions.md b/docs/en/sql-reference/functions/array-functions.md index 89de48a81ad..e7918c018db 100644 --- a/docs/en/sql-reference/functions/array-functions.md +++ b/docs/en/sql-reference/functions/array-functions.md @@ -58,7 +58,7 @@ Checks whether the input array is non-empty. notEmpty([x]) ``` -An array is considered non-empty if it contains at least one non-empty element. +An array is considered non-empty if it contains at least one element. !!! note "Note" Can be optimized by enabling the [optimize_functions_to_subcolumns](../../operations/settings/settings.md#optimize-functions-to-subcolumns) setting. With `optimize_functions_to_subcolumns = 1` the function reads only [size0](../../sql-reference/data-types/array.md#array-size) subcolumn instead of reading and processing the whole array column. The query `SELECT notEmpty(arr) FROM table` transforms to `SELECT arr.size0 != 0 FROM TABLE`. From bd3fe4fb419e95a84cf9e8ad70f8fe59c0c9a2c7 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Fri, 13 Aug 2021 13:23:54 +0300 Subject: [PATCH 110/161] Update MemoryTracker.cpp --- src/Common/MemoryTracker.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index 0be7ffda958..50ddcb5a9eb 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -360,7 +360,7 @@ void MemoryTracker::setOrRaiseHardLimit(Int64 value) { /// This is just atomic set to maximum. Int64 old_value = hard_limit.load(std::memory_order_relaxed); - while (old_value < value && !hard_limit.compare_exchange_weak(old_value, value)) + while ((value == 0 || old_value < value) && !hard_limit.compare_exchange_weak(old_value, value)) ; } @@ -368,6 +368,6 @@ void MemoryTracker::setOrRaiseHardLimit(Int64 value) void MemoryTracker::setOrRaiseProfilerLimit(Int64 value) { Int64 old_value = profiler_limit.load(std::memory_order_relaxed); - while (old_value < value && !profiler_limit.compare_exchange_weak(old_value, value)) + while ((value == 0 || old_value < value) && !profiler_limit.compare_exchange_weak(old_value, value)) ; } From 7fba508b002a01559dcbfcbaa5c637a98b291033 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 13 Aug 2021 13:43:37 +0200 Subject: [PATCH 111/161] MV: Improve text logs when doing parallel processing --- .../PushingToViewsBlockOutputStream.cpp | 36 +++++++------------ 1 file changed, 13 insertions(+), 23 deletions(-) diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/src/DataStreams/PushingToViewsBlockOutputStream.cpp index aec1209a454..dec5b710f75 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -342,19 +342,7 @@ void PushingToViewsBlockOutputStream::writeSuffix() runViewStage(view, stage_step, [&] { process_suffix(view); }); if (view.exception) - { exception_count.fetch_add(1, std::memory_order_relaxed); - } - else - { - LOG_TRACE( - log, - "Pushing (parallel {}) from {} to {} took {} ms.", - max_threads, - storage->getStorageID().getNameForLogs(), - view.table_id.getNameForLogs(), - view.runtime_stats.elapsed_ms); - } }); } pool.wait(); @@ -371,20 +359,22 @@ void PushingToViewsBlockOutputStream::writeSuffix() } runViewStage(view, stage_step, [&] { process_suffix(view); }); if (view.exception) - { exception_happened = true; - } - else - { - LOG_TRACE( - log, - "Pushing (sequentially) from {} to {} took {} ms.", - storage->getStorageID().getNameForLogs(), - view.table_id.getNameForLogs(), - view.runtime_stats.elapsed_ms); - } } } + + for (auto & view : views) + { + if (!view.exception) + LOG_TRACE( + log, + "Pushing ({}) from {} to {} took {} ms.", + max_threads <= 1 ? "sequentially" : ("parallel " + std::to_string(max_threads)), + storage->getStorageID().getNameForLogs(), + view.table_id.getNameForLogs(), + view.runtime_stats.elapsed_ms); + } + if (exception_happened) checkExceptionsInViews(); From 47fb923975b4b827c1296e71a1781edfc5a3e5d3 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 13 Aug 2021 15:07:44 +0300 Subject: [PATCH 112/161] Some fixes, more tests --- .../AggregateFunctionMinMaxAny.h | 12 +----------- .../InterpreterSelectIntersectExceptQuery.cpp | 1 + .../NormalizeSelectWithUnionQueryVisitor.cpp | 3 --- .../SelectIntersectExceptQueryVisitor.cpp | 2 +- src/Parsers/ASTSelectWithUnionQuery.cpp | 3 --- src/Parsers/ExpressionListParsers.cpp | 10 +++++----- .../QueryPlan/IntersectOrExceptStep.cpp | 5 +++++ .../Transforms/IntersectOrExceptTransform.cpp | 5 ----- ...02004_intersect_except_operators.reference | 6 ++++++ .../02004_intersect_except_operators.sql | 2 ++ .../02007_test_any_all_operators.reference | 19 +++++++++++++++++++ .../02007_test_any_all_operators.sql | 12 ++++++++++++ 12 files changed, 52 insertions(+), 28 deletions(-) create mode 100644 tests/queries/0_stateless/02007_test_any_all_operators.reference create mode 100644 tests/queries/0_stateless/02007_test_any_all_operators.sql diff --git a/src/AggregateFunctions/AggregateFunctionMinMaxAny.h b/src/AggregateFunctions/AggregateFunctionMinMaxAny.h index 410f94c7afd..577b8127fd7 100644 --- a/src/AggregateFunctions/AggregateFunctionMinMaxAny.h +++ b/src/AggregateFunctions/AggregateFunctionMinMaxAny.h @@ -1032,17 +1032,7 @@ struct AggregateFunctionSingleValueOrNullData : Data #if USE_EMBEDDED_COMPILER - static constexpr bool is_compilable = Data::is_compilable; - - static void compileChangeIfBetter(llvm::IRBuilderBase & builder, llvm::Value * aggregate_data_ptr, llvm::Value * value_to_check) - { - Data::compileChangeFirstTime(builder, aggregate_data_ptr, value_to_check); - } - - static void compileChangeIfBetterMerge(llvm::IRBuilderBase & builder, llvm::Value * aggregate_data_dst_ptr, llvm::Value * aggregate_data_src_ptr) - { - Data::compileChangeFirstTimeMerge(builder, aggregate_data_dst_ptr, aggregate_data_src_ptr); - } + static constexpr bool is_compilable = false; #endif }; diff --git a/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp b/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp index 9e24dd6e6a0..34d7ae5b37f 100644 --- a/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp +++ b/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp @@ -17,6 +17,7 @@ namespace DB namespace ErrorCodes { extern const int INTERSECT_OR_EXCEPT_RESULT_STRUCTURES_MISMATCH; + extern const int LOGICAL_ERROR; } static Block getCommonHeader(const Blocks & headers) diff --git a/src/Interpreters/NormalizeSelectWithUnionQueryVisitor.cpp b/src/Interpreters/NormalizeSelectWithUnionQueryVisitor.cpp index bbe1a4e048c..0990667b2a8 100644 --- a/src/Interpreters/NormalizeSelectWithUnionQueryVisitor.cpp +++ b/src/Interpreters/NormalizeSelectWithUnionQueryVisitor.cpp @@ -36,9 +36,6 @@ void NormalizeSelectWithUnionQueryMatcher::visit(ASTSelectWithUnionQuery & ast, ASTs selects; auto & select_list = ast.list_of_selects->children; - if (select_list.size() < 2) - return; - int i; for (i = union_modes.size() - 1; i >= 0; --i) { diff --git a/src/Interpreters/SelectIntersectExceptQueryVisitor.cpp b/src/Interpreters/SelectIntersectExceptQueryVisitor.cpp index 5926d5f6f10..273bc327dc3 100644 --- a/src/Interpreters/SelectIntersectExceptQueryVisitor.cpp +++ b/src/Interpreters/SelectIntersectExceptQueryVisitor.cpp @@ -162,7 +162,7 @@ void SelectIntersectExceptQueryMatcher::visit(ASTSelectWithUnionQuery & ast, Dat children.emplace_back(std::move(right)); } - ast.union_mode = ASTSelectWithUnionQuery::Mode::ALL; + ast.union_mode = ASTSelectWithUnionQuery::Mode::Unspecified; ast.list_of_selects->children = std::move(children); ast.list_of_modes = std::move(modes); } diff --git a/src/Parsers/ASTSelectWithUnionQuery.cpp b/src/Parsers/ASTSelectWithUnionQuery.cpp index d19e860c16a..b882c738c9a 100644 --- a/src/Parsers/ASTSelectWithUnionQuery.cpp +++ b/src/Parsers/ASTSelectWithUnionQuery.cpp @@ -32,9 +32,6 @@ ASTPtr ASTSelectWithUnionQuery::clone() const void ASTSelectWithUnionQuery::formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { - if (!list_of_selects || list_of_selects->children.size() != list_of_modes.size() + 1) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Incorrect ASTSelectWithUnionQuery"); - std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' '); auto mode_to_str = [&](auto mode) diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index e75aad8d02f..33085379abb 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -186,13 +186,13 @@ enum class SubqueryFunctionType ALL }; -static bool modifyAST(String operator_name, std::shared_ptr & function, SubqueryFunctionType type) +static bool modifyAST(const String & operator_name, ASTPtr function, SubqueryFunctionType type) { // = ANY --> IN, != ALL --> NOT IN - if ((operator_name == "equals" && type == SubqueryFunctionType::ANY) - || (operator_name == "notEquals" && type == SubqueryFunctionType::ALL)) + if ((type == SubqueryFunctionType::ANY && operator_name == "equals") + || (type == SubqueryFunctionType::ALL && operator_name == "notEquals")) { - function->name = "in"; + assert_cast(function.get())->name = "in"; if (operator_name == "notEquals") { auto function_not = std::make_shared(); @@ -257,7 +257,7 @@ static bool modifyAST(String operator_name, std::shared_ptr & funct if (operator_name == "equals" || operator_name == "notEquals") { aggregate_function->name = "singleValueOrNull"; - function->name = "in"; + assert_cast(function.get())->name = "in"; if (operator_name == "notEquals") { auto function_not = std::make_shared(); diff --git a/src/Processors/QueryPlan/IntersectOrExceptStep.cpp b/src/Processors/QueryPlan/IntersectOrExceptStep.cpp index e4d04115cff..b75898b815b 100644 --- a/src/Processors/QueryPlan/IntersectOrExceptStep.cpp +++ b/src/Processors/QueryPlan/IntersectOrExceptStep.cpp @@ -12,6 +12,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + static Block checkHeaders(const DataStreams & input_streams_) { if (input_streams_.empty()) diff --git a/src/Processors/Transforms/IntersectOrExceptTransform.cpp b/src/Processors/Transforms/IntersectOrExceptTransform.cpp index b16032bde8e..abfd1a7f0ad 100644 --- a/src/Processors/Transforms/IntersectOrExceptTransform.cpp +++ b/src/Processors/Transforms/IntersectOrExceptTransform.cpp @@ -4,11 +4,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int SET_SIZE_LIMIT_EXCEEDED; -} - IntersectOrExceptTransform::IntersectOrExceptTransform(const Block & header_, Operator operator_) : IProcessor(InputPorts(2, header_), {header_}) , current_operator(operator_) diff --git a/tests/queries/0_stateless/02004_intersect_except_operators.reference b/tests/queries/0_stateless/02004_intersect_except_operators.reference index 9a9e4e1bf58..a097bd0076f 100644 --- a/tests/queries/0_stateless/02004_intersect_except_operators.reference +++ b/tests/queries/0_stateless/02004_intersect_except_operators.reference @@ -66,6 +66,8 @@ select number from numbers(100) intersect select number from numbers(20, 60) exc 57 58 59 +select * from (select 1 intersect select 1); +1 with (select number from numbers(10) intersect select 5) as a select a * 10; 50 select count() from (select number from numbers(10) except select 5); @@ -93,3 +95,7 @@ select * from (select 1 union all select 2 union all select 3 union all select 4 2 3 5 +select * from (select 1 union all select 2 union all select 3 union all select 4 intersect select 3 union all select 5 except select 1) order by 1; +2 +3 +5 diff --git a/tests/queries/0_stateless/02004_intersect_except_operators.sql b/tests/queries/0_stateless/02004_intersect_except_operators.sql index c88951ef353..4602dec7238 100644 --- a/tests/queries/0_stateless/02004_intersect_except_operators.sql +++ b/tests/queries/0_stateless/02004_intersect_except_operators.sql @@ -19,6 +19,7 @@ select 1 intersect select 1 except select 2 intersect select 1 except select 3 i select number from numbers(10) except select 5; select number from numbers(100) intersect select number from numbers(20, 60) except select number from numbers(30, 20) except select number from numbers(60, 20); +select * from (select 1 intersect select 1); with (select number from numbers(10) intersect select 5) as a select a * 10; select count() from (select number from numbers(10) except select 5); select count() from (select number from numbers(1000000) intersect select number from numbers(200000, 600000)); @@ -30,3 +31,4 @@ select 1 union all select 1 intersect select 1; select 1 union all select 1 intersect select 2; select * from (select 1 union all select 2 union all select 3 union all select 4 except select 3 union all select 5) order by 1; select * from (select 1 union all select 2 union all select 3 union all select 4 intersect select 3 union all select 5) order by 1; +select * from (select 1 union all select 2 union all select 3 union all select 4 intersect select 3 union all select 5 except select 1) order by 1; diff --git a/tests/queries/0_stateless/02007_test_any_all_operators.reference b/tests/queries/0_stateless/02007_test_any_all_operators.reference new file mode 100644 index 00000000000..cd36102cb80 --- /dev/null +++ b/tests/queries/0_stateless/02007_test_any_all_operators.reference @@ -0,0 +1,19 @@ +-- { echo } +select 1 == any (select number from numbers(10)); +1 +select 1 == any (select number from numbers(2, 10)); +0 +select 1 == all (select 1 from numbers(10)); +1 +select 1 == all (select number from numbers(10)); +0 +select number as a from numbers(10) where a == any (select number from numbers(3, 3)); +3 +4 +5 +-- TODO: Incorrect: +select 1 != any (select 1 from numbers(10)); +1 +select 1 != all (select 1 from numbers(10)); +1 +select number as a from numbers(10) where a != any (select number from numbers(3, 3)); diff --git a/tests/queries/0_stateless/02007_test_any_all_operators.sql b/tests/queries/0_stateless/02007_test_any_all_operators.sql new file mode 100644 index 00000000000..08fc929bab9 --- /dev/null +++ b/tests/queries/0_stateless/02007_test_any_all_operators.sql @@ -0,0 +1,12 @@ +-- { echo } +select 1 == any (select number from numbers(10)); +select 1 == any (select number from numbers(2, 10)); +select 1 == all (select 1 from numbers(10)); +select 1 == all (select number from numbers(10)); +select number as a from numbers(10) where a == any (select number from numbers(3, 3)); + +-- TODO: Incorrect: +select 1 != any (select 1 from numbers(10)); +select 1 != all (select 1 from numbers(10)); +select number as a from numbers(10) where a != any (select number from numbers(3, 3)); + From d1b93f8b0cbf263bde534845c244b8d6f8b4b1db Mon Sep 17 00:00:00 2001 From: Vladimir Goncharov Date: Fri, 13 Aug 2021 15:32:13 +0300 Subject: [PATCH 113/161] Update docs/ru/engines/table-engines/mergetree-family/mergetree.md Co-authored-by: Alexey Boykov <33257111+mathalex@users.noreply.github.com> --- docs/ru/engines/table-engines/mergetree-family/mergetree.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/engines/table-engines/mergetree-family/mergetree.md b/docs/ru/engines/table-engines/mergetree-family/mergetree.md index 728c632370c..b2290369b4c 100644 --- a/docs/ru/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/ru/engines/table-engines/mergetree-family/mergetree.md @@ -379,7 +379,7 @@ INDEX b (u64 * length(str), i32 + f64 * 100, date, str) TYPE set(100) GRANULARIT Проекции похожи на материализованные представления, но определяются на уровне партов. Это обеспечивает гарантии согласованности наряду с автоматическим использованием в запросах. #### Запрос {#projection-query} -Запрос проекции - это то, что определяет проекцию. Он имеет следующую грамматику: +Запрос проекции — это то, что определяет проекцию. Он имеет следующую грамматику: `SELECT [GROUP BY] [ORDER BY]` From 942bd534d3279076b26d8e3a5b3969633f56a640 Mon Sep 17 00:00:00 2001 From: Vladimir Goncharov Date: Fri, 13 Aug 2021 15:32:23 +0300 Subject: [PATCH 114/161] Update docs/ru/engines/table-engines/mergetree-family/mergetree.md Co-authored-by: Alexey Boykov <33257111+mathalex@users.noreply.github.com> --- docs/ru/engines/table-engines/mergetree-family/mergetree.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/engines/table-engines/mergetree-family/mergetree.md b/docs/ru/engines/table-engines/mergetree-family/mergetree.md index b2290369b4c..7493c1e5fc0 100644 --- a/docs/ru/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/ru/engines/table-engines/mergetree-family/mergetree.md @@ -383,7 +383,7 @@ INDEX b (u64 * length(str), i32 + f64 * 100, date, str) TYPE set(100) GRANULARIT `SELECT [GROUP BY] [ORDER BY]` -Это неявно выбирает данные из родительской таблицы. +Он неявно выбирает данные из родительской таблицы. #### Хранение {#projection-storage} Проекции хранятся в каталоге парта. Это похоже на хранение индексов, но используется подкаталог, в котором хранится анонимный парт таблицы MergeTree. Таблица создается запросом определения проекции. Если есть конструкция GROUP BY, то базовый механизм хранения становится AggregatedMergeTree, а все агрегатные функции преобразуются в AggregateFunction. Если есть конструкция ORDER BY, таблица MergeTree будет использовать его в качестве выражения первичного ключа. Во время процесса слияния парт проекции будет слит с помощью процедуры слияния ее хранилища. Контрольная сумма парта родительской таблицы будет включать парт проекции. Другие процедуры аналогичны индексам пропуска данных. From ff3afb410ddf93f01b8249170e106ff2bd485ece Mon Sep 17 00:00:00 2001 From: Vladimir Goncharov Date: Fri, 13 Aug 2021 15:33:03 +0300 Subject: [PATCH 115/161] Update docs/ru/engines/table-engines/mergetree-family/mergetree.md Co-authored-by: Alexey Boykov <33257111+mathalex@users.noreply.github.com> --- docs/ru/engines/table-engines/mergetree-family/mergetree.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/engines/table-engines/mergetree-family/mergetree.md b/docs/ru/engines/table-engines/mergetree-family/mergetree.md index 7493c1e5fc0..8b7d9bb736c 100644 --- a/docs/ru/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/ru/engines/table-engines/mergetree-family/mergetree.md @@ -389,7 +389,7 @@ INDEX b (u64 * length(str), i32 + f64 * 100, date, str) TYPE set(100) GRANULARIT Проекции хранятся в каталоге парта. Это похоже на хранение индексов, но используется подкаталог, в котором хранится анонимный парт таблицы MergeTree. Таблица создается запросом определения проекции. Если есть конструкция GROUP BY, то базовый механизм хранения становится AggregatedMergeTree, а все агрегатные функции преобразуются в AggregateFunction. Если есть конструкция ORDER BY, таблица MergeTree будет использовать его в качестве выражения первичного ключа. Во время процесса слияния парт проекции будет слит с помощью процедуры слияния ее хранилища. Контрольная сумма парта родительской таблицы будет включать парт проекции. Другие процедуры аналогичны индексам пропуска данных. #### Анализ запросов {#projection-query-analysis} -1. Проверить, можно ли использовать проекцию для результата на данный запрос, то есть она даёт тот же результат, что и запрос к базовой таблице. +1. Проверить, можно ли использовать проекцию в данном запросе, то есть, что с ней выходит тот же результат, что и с запросом к базовой таблице. 2. Выбрать наиболее подходящее совпадение, содержащее наименьшее количество гранул для чтения. 3. План запроса, который использует проекции, будет отличаться от того, который использует исходные парты. Если в некоторых партах проекция отсутствует, мы можем расширить план, чтобы «проецировать» на лету. From af8fcaf26637cc1f826b3241caaecd0dca3d62b0 Mon Sep 17 00:00:00 2001 From: Vladimir Goncharov Date: Fri, 13 Aug 2021 15:33:52 +0300 Subject: [PATCH 116/161] Update docs/ru/engines/table-engines/mergetree-family/mergetree.md Co-authored-by: Alexey Boykov <33257111+mathalex@users.noreply.github.com> --- docs/ru/engines/table-engines/mergetree-family/mergetree.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/engines/table-engines/mergetree-family/mergetree.md b/docs/ru/engines/table-engines/mergetree-family/mergetree.md index 8b7d9bb736c..db6eb8154ba 100644 --- a/docs/ru/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/ru/engines/table-engines/mergetree-family/mergetree.md @@ -391,7 +391,7 @@ INDEX b (u64 * length(str), i32 + f64 * 100, date, str) TYPE set(100) GRANULARIT #### Анализ запросов {#projection-query-analysis} 1. Проверить, можно ли использовать проекцию в данном запросе, то есть, что с ней выходит тот же результат, что и с запросом к базовой таблице. 2. Выбрать наиболее подходящее совпадение, содержащее наименьшее количество гранул для чтения. -3. План запроса, который использует проекции, будет отличаться от того, который использует исходные парты. Если в некоторых партах проекция отсутствует, мы можем расширить план, чтобы «проецировать» на лету. +3. План запроса, который использует проекции, будет отличаться от того, который использует исходные парты. При отсутствии проекции в некоторых партах можно расширить план, чтобы «проецировать» на лету. ## Конкурентный доступ к данным {#concurrent-data-access} From 1d3c11f6a136cff7751186c6f5831198a3e638d8 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Fri, 13 Aug 2021 15:43:22 +0300 Subject: [PATCH 117/161] Update docs/ru/engines/database-engines/materialized-mysql.md Co-authored-by: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> --- docs/ru/engines/database-engines/materialized-mysql.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/engines/database-engines/materialized-mysql.md b/docs/ru/engines/database-engines/materialized-mysql.md index b82a3dae553..1cd864c01e9 100644 --- a/docs/ru/engines/database-engines/materialized-mysql.md +++ b/docs/ru/engines/database-engines/materialized-mysql.md @@ -5,7 +5,7 @@ toc_title: "[experimental] MaterializedMySQL" # [экспериментальный] MaterializedMySQL {#materialized-mysql} -**Это экспериментальный движок, который не следует использовать в продуктивной среде.** +**Это экспериментальный движок, который не следует использовать в продакшене.** Создает базу данных ClickHouse со всеми таблицами, существующими в MySQL, и всеми данными в этих таблицах. From f20660455cd94be141c5d94831efec624b62ec5d Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev Date: Fri, 13 Aug 2021 10:24:30 -0300 Subject: [PATCH 118/161] fix failded tests --- tests/integration/test_system_flush_logs/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_system_flush_logs/test.py b/tests/integration/test_system_flush_logs/test.py index b69105710fb..a4d70339c09 100644 --- a/tests/integration/test_system_flush_logs/test.py +++ b/tests/integration/test_system_flush_logs/test.py @@ -10,12 +10,12 @@ node = cluster.add_instance('node_default') system_logs = [ # disabled by default - ('system.part_log', 0), ('system.text_log', 0), # enabled by default ('system.query_log', 1), ('system.query_thread_log', 1), + ('system.part_log', 1), ('system.trace_log', 1), ('system.metric_log', 1), ] From 67f5889b91b560dbb74cb9ff564ff996a04fc0ae Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev Date: Fri, 13 Aug 2021 10:25:27 -0300 Subject: [PATCH 119/161] fix failed tests --- .../configs/config_without_standard_part_log.xml | 3 +++ tests/integration/test_part_log_table/test.py | 2 +- 2 files changed, 4 insertions(+), 1 deletion(-) create mode 100644 tests/integration/test_part_log_table/configs/config_without_standard_part_log.xml diff --git a/tests/integration/test_part_log_table/configs/config_without_standard_part_log.xml b/tests/integration/test_part_log_table/configs/config_without_standard_part_log.xml new file mode 100644 index 00000000000..6c718dad1ee --- /dev/null +++ b/tests/integration/test_part_log_table/configs/config_without_standard_part_log.xml @@ -0,0 +1,3 @@ + + + diff --git a/tests/integration/test_part_log_table/test.py b/tests/integration/test_part_log_table/test.py index 63adde432b5..050e8c831c7 100644 --- a/tests/integration/test_part_log_table/test.py +++ b/tests/integration/test_part_log_table/test.py @@ -3,7 +3,7 @@ import pytest from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance("node1") +node1 = cluster.add_instance("node1", main_configs=["configs/config_without_standard_part_log.xml"]) node2 = cluster.add_instance("node2", main_configs=["configs/config_with_standard_part_log.xml"]) node3 = cluster.add_instance("node3", main_configs=["configs/config_with_non_standard_part_log.xml"]) From 30191a7c32525890fdcc6a6089051fcb412e9585 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 13 Aug 2021 13:57:06 +0000 Subject: [PATCH 120/161] Fix --- src/Dictionaries/XDBCDictionarySource.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Dictionaries/XDBCDictionarySource.cpp b/src/Dictionaries/XDBCDictionarySource.cpp index 80d3df1caab..e79e55910b7 100644 --- a/src/Dictionaries/XDBCDictionarySource.cpp +++ b/src/Dictionaries/XDBCDictionarySource.cpp @@ -222,7 +222,7 @@ Pipe XDBCDictionarySource::loadFromQuery(const Poco::URI & url, const Block & re }; auto read_buf = std::make_unique(url, Poco::Net::HTTPRequest::HTTP_POST, write_body_callback, timeouts); - auto format = FormatFactory::instance().getInput(IXDBCBridgeHelper::DEFAULT_FORMAT, *read_buf, sample_block, getContext(), max_block_size); + auto format = FormatFactory::instance().getInput(IXDBCBridgeHelper::DEFAULT_FORMAT, *read_buf, required_sample_block, getContext(), max_block_size); format->addBuffer(std::move(read_buf)); return Pipe(std::move(format)); From 73cb7d55ec8a9bc5ac4ee9a8cfaf2230915e9276 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 13 Aug 2021 16:18:46 +0200 Subject: [PATCH 121/161] Disable arrayJoin on partition expressions --- src/Storages/MergeTree/MergeTreeData.cpp | 13 +++++++++++++ .../02009_array_join_partition.reference | 0 .../0_stateless/02009_array_join_partition.sql | 4 ++++ 3 files changed, 17 insertions(+) create mode 100644 tests/queries/0_stateless/02009_array_join_partition.reference create mode 100644 tests/queries/0_stateless/02009_array_join_partition.sql diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 4730bf9f47c..ebd961ab428 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3207,6 +3207,16 @@ Pipe MergeTreeData::alterPartition( return {}; } +void checkPartitionExpressionFunction(const ASTPtr & ast) +{ + if (const auto * func = ast->as()) + if (func->name == "arrayJoin") + throw Exception("The partition expression cannot contain array joins", ErrorCodes::INVALID_PARTITION_VALUE); + for (const auto & child : ast->children) + checkPartitionExpressionFunction(child); +} + + String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr local_context) const { const auto & partition_ast = ast->as(); @@ -3217,6 +3227,9 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr loc return partition_ast.id; } + if (const auto * partition_function = partition_ast.value->as()) + checkPartitionExpressionFunction(ast); + if (format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) { /// Month-partitioning specific - partition ID can be passed in the partition value. diff --git a/tests/queries/0_stateless/02009_array_join_partition.reference b/tests/queries/0_stateless/02009_array_join_partition.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02009_array_join_partition.sql b/tests/queries/0_stateless/02009_array_join_partition.sql new file mode 100644 index 00000000000..a78efe96f66 --- /dev/null +++ b/tests/queries/0_stateless/02009_array_join_partition.sql @@ -0,0 +1,4 @@ +CREATE TABLE table_2009_part (`i` Int64, `d` Date, `s` String) ENGINE = MergeTree PARTITION BY toYYYYMM(d) ORDER BY i; + +ALTER TABLE table_2009_part ATTACH PARTITION tuple(arrayJoin([0, 1])); -- {serverError 248} +ALTER TABLE table_2009_part ATTACH PARTITION tuple(toYYYYMM(toDate([arrayJoin([arrayJoin([arrayJoin([arrayJoin([3, materialize(NULL), arrayJoin([1025, materialize(NULL), materialize(NULL)]), NULL])])]), materialize(NULL)])], NULL))); -- {serverError 248} From c534363abe05b348e0e1a3100d9af80f6f5a0088 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 13 Aug 2021 20:33:10 +0300 Subject: [PATCH 122/161] fix intersecting parts --- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index ea5f7cfc36a..ef276a53df2 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1012,8 +1012,24 @@ bool ReplicatedMergeTreeQueue::isNotCoveredByFuturePartsImpl(const String & log_ bool ReplicatedMergeTreeQueue::addFuturePartIfNotCoveredByThem(const String & part_name, LogEntry & entry, String & reject_reason) { + /// We have found `part_name` on some replica and are going to fetch it instead of covered `entry->new_part_name`. std::lock_guard lock(state_mutex); + if (virtual_parts.getContainingPart(part_name).empty()) + { + /// We should not fetch any parts that absent in our `virtual_parts` set, + /// because we do not know about such parts according to our replication queue (we know about them from some side-channel). + /// Otherwise, it may break invariants in replication queue reordering, for example: + /// 1. Our queue contains GET_PART all_2_2_0, log contains DROP_RANGE all_2_2_0 and MERGE_PARTS all_1_3_1 + /// 2. We execute GET_PART all_2_2_0, but fetch all_1_3_1 instead + /// (drop_ranges.isAffectedByDropRange(...) is false-negative, because DROP_RANGE all_2_2_0 is not pulled yet). + /// It actually means, that MERGE_PARTS all_1_3_1 is executed too, but it's not even pulled yet. + /// 3. Then we pull log, trying to execute DROP_RANGE all_2_2_0 + /// and reveal that it was incorrectly reordered with MERGE_PARTS all_1_3_1 (drop range intersects merged part). + reject_reason = fmt::format("Log entry for part {} or covering part is not pulled from log to queue yet.", part_name); + return false; + } + /// FIXME get rid of actual_part_name. /// If new covering part jumps over DROP_RANGE we should execute drop range first if (drop_ranges.isAffectedByDropRange(part_name, reject_reason)) From 382fd7d4aca4a5220c9a227beffc6a806bcea031 Mon Sep 17 00:00:00 2001 From: Alexey Date: Thu, 5 Aug 2021 19:55:19 +0000 Subject: [PATCH 123/161] translated parts in MergeTree --- docs/en/engines/table-engines/mergetree-family/mergetree.md | 2 +- docs/ru/engines/table-engines/mergetree-family/mergetree.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 0c900454cd0..1b1313e625c 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -79,7 +79,7 @@ For a description of parameters, see the [CREATE query description](../../../sql - `SAMPLE BY` — An expression for sampling. Optional. - If a sampling expression is used, the primary key must contain it. The result of sampling expression must be unsigned integer. Example: `SAMPLE BY intHash32(UserID) ORDER BY (CounterID, EventDate, intHash32(UserID))`. + If a sampling expression is used, the primary key must contain it. The result of a sampling expression must be an unsigned integer. Example: `SAMPLE BY intHash32(UserID) ORDER BY (CounterID, EventDate, intHash32(UserID))`. - `TTL` — A list of rules specifying storage duration of rows and defining logic of automatic parts movement [between disks and volumes](#table_engine-mergetree-multiple-volumes). Optional. diff --git a/docs/ru/engines/table-engines/mergetree-family/mergetree.md b/docs/ru/engines/table-engines/mergetree-family/mergetree.md index db6eb8154ba..61ed34b686c 100644 --- a/docs/ru/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/ru/engines/table-engines/mergetree-family/mergetree.md @@ -68,7 +68,7 @@ ORDER BY expr - `SAMPLE BY` — выражение для сэмплирования. Необязательный параметр. - Если используется выражение для сэмплирования, то первичный ключ должен содержать его. Пример: `SAMPLE BY intHash32(UserID) ORDER BY (CounterID, EventDate, intHash32(UserID))`. + Если используется выражение для сэмплирования, то первичный ключ должен содержать его. Результат выражения для сэмплирования должен быть беззнаковым целым числом. Пример: `SAMPLE BY intHash32(UserID) ORDER BY (CounterID, EventDate, intHash32(UserID))`. - `TTL` — список правил, определяющих длительности хранения строк, а также задающих правила перемещения частей на определённые тома или диски. Необязательный параметр. From 200d75646a2af8b9aa5c62a18927121eefda2307 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 13 Aug 2021 20:57:40 +0000 Subject: [PATCH 124/161] Fix --- src/Databases/PostgreSQL/DatabasePostgreSQL.cpp | 4 ++-- src/Storages/StorageExternalDistributed.cpp | 2 +- src/Storages/StoragePostgreSQL.cpp | 3 --- src/Storages/StoragePostgreSQL.h | 2 -- src/TableFunctions/TableFunctionPostgreSQL.cpp | 1 - 5 files changed, 3 insertions(+), 9 deletions(-) diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index c848c784712..259648f4399 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -164,7 +164,7 @@ StoragePtr DatabasePostgreSQL::tryGetTable(const String & table_name, ContextPtr } -StoragePtr DatabasePostgreSQL::fetchTable(const String & table_name, ContextPtr local_context, const bool table_checked) const +StoragePtr DatabasePostgreSQL::fetchTable(const String & table_name, ContextPtr, const bool table_checked) const { if (!cache_tables || !cached_tables.count(table_name)) { @@ -179,7 +179,7 @@ StoragePtr DatabasePostgreSQL::fetchTable(const String & table_name, ContextPtr auto storage = StoragePostgreSQL::create( StorageID(database_name, table_name), pool, table_name, - ColumnsDescription{*columns}, ConstraintsDescription{}, String{}, local_context, postgres_schema); + ColumnsDescription{*columns}, ConstraintsDescription{}, String{}, postgres_schema); if (cache_tables) cached_tables[table_name] = storage; diff --git a/src/Storages/StorageExternalDistributed.cpp b/src/Storages/StorageExternalDistributed.cpp index 32b9c7e9245..f20e49fe23a 100644 --- a/src/Storages/StorageExternalDistributed.cpp +++ b/src/Storages/StorageExternalDistributed.cpp @@ -98,7 +98,7 @@ StorageExternalDistributed::StorageExternalDistributed( context->getSettingsRef().postgresql_connection_pool_size, context->getSettingsRef().postgresql_connection_pool_wait_timeout); - shard = StoragePostgreSQL::create(table_id_, std::move(pool), remote_table, columns_, constraints_, String{}, context); + shard = StoragePostgreSQL::create(table_id_, std::move(pool), remote_table, columns_, constraints_, String{}); break; } #endif diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index 5f8b81a47cf..603a52b2801 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -47,12 +47,10 @@ StoragePostgreSQL::StoragePostgreSQL( const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, const String & comment, - ContextPtr context_, const String & remote_table_schema_) : IStorage(table_id_) , remote_table_name(remote_table_name_) , remote_table_schema(remote_table_schema_) - , global_context(context_) , pool(std::move(pool_)) { StorageInMemoryMetadata storage_metadata; @@ -347,7 +345,6 @@ void registerStoragePostgreSQL(StorageFactory & factory) args.columns, args.constraints, args.comment, - args.getContext(), remote_table_schema); }, { diff --git a/src/Storages/StoragePostgreSQL.h b/src/Storages/StoragePostgreSQL.h index 064fa481f9d..bd5cd317c3d 100644 --- a/src/Storages/StoragePostgreSQL.h +++ b/src/Storages/StoragePostgreSQL.h @@ -27,7 +27,6 @@ public: const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, const String & comment, - ContextPtr context_, const std::string & remote_table_schema_ = ""); String getName() const override { return "PostgreSQL"; } @@ -48,7 +47,6 @@ private: String remote_table_name; String remote_table_schema; - ContextPtr global_context; postgres::PoolWithFailoverPtr pool; }; diff --git a/src/TableFunctions/TableFunctionPostgreSQL.cpp b/src/TableFunctions/TableFunctionPostgreSQL.cpp index ceea29b335b..d701728479b 100644 --- a/src/TableFunctions/TableFunctionPostgreSQL.cpp +++ b/src/TableFunctions/TableFunctionPostgreSQL.cpp @@ -37,7 +37,6 @@ StoragePtr TableFunctionPostgreSQL::executeImpl(const ASTPtr & /*ast_function*/, columns, ConstraintsDescription{}, String{}, - context, remote_table_schema); result->startup(); From e39bef97b50467ea8eef8164f450239eb9709f99 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 14 Aug 2021 02:49:45 +0300 Subject: [PATCH 125/161] Update CHANGELOG.md --- CHANGELOG.md | 32 ++++++++++---------------------- 1 file changed, 10 insertions(+), 22 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 10ed4dae1c9..12fda73fe13 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -2,15 +2,15 @@ #### New Features -* Collect common system metrics (in `system.asynchronous_metrics` and `system.asynchronous_metric_log`) on CPU usage, disk usage, memory usage, IO, network, files, load average, CPU frequencies, thermal sensors, EDAC counters, system uptime; also added metrics about the scheduling jitter and the time spent collecting the metrics. It works similar to `atop` in ClickHouse and allows access to monitoring data even if you have no additional tools installed. Close [#9430](https://github.com/ClickHouse/ClickHouse/issues/9430). [#24416](https://github.com/ClickHouse/ClickHouse/pull/24416) ([Yegor Levankov](https://github.com/elevankoff)). +* Add support for a part of SQL/JSON standard. [#24148](https://github.com/ClickHouse/ClickHouse/pull/24148) ([l1tsolaiki](https://github.com/l1tsolaiki)). +* Collect common system metrics (in `system.asynchronous_metrics` and `system.asynchronous_metric_log`) on CPU usage, disk usage, memory usage, IO, network, files, load average, CPU frequencies, thermal sensors, EDAC counters, system uptime; also added metrics about the scheduling jitter and the time spent collecting the metrics. It works similar to `atop` in ClickHouse and allows access to monitoring data even if you have no additional tools installed. Close [#9430](https://github.com/ClickHouse/ClickHouse/issues/9430). [#24416](https://github.com/ClickHouse/ClickHouse/pull/24416) (([alexey-milovidov](https://github.com/alexey-milovidov)), [Yegor Levankov](https://github.com/elevankoff), [Kseniia Sumarokova](https://github.com/kssenii)). +* Add MaterializedPostgreSQL table engine and database engine. This database engine allows replicating a whole database or any subset of database tables. [#20470](https://github.com/ClickHouse/ClickHouse/pull/20470) ([Kseniia Sumarokova](https://github.com/kssenii)). * Add new functions `leftPad()`, `rightPad()`, `leftPadUTF8()`, `rightPadUTF8()`. [#26075](https://github.com/ClickHouse/ClickHouse/pull/26075) ([Vitaly Baranov](https://github.com/vitlibar)). * Add the `FIRST` keyword to the `ADD INDEX` command to be able to add the index at the beginning of the indices list. [#25904](https://github.com/ClickHouse/ClickHouse/pull/25904) ([xjewer](https://github.com/xjewer)). * Introduce `system.data_skipping_indices` table containing information about existing data skipping indices. Close [#7659](https://github.com/ClickHouse/ClickHouse/issues/7659). [#25693](https://github.com/ClickHouse/ClickHouse/pull/25693) ([Dmitry Novik](https://github.com/novikd)). * Add `bin`/`unbin` functions. [#25609](https://github.com/ClickHouse/ClickHouse/pull/25609) ([zhaoyu](https://github.com/zxc111)). * Support `Map` and `(U)Int128`, `U(Int256) types in `mapAdd` and `mapSubtract` functions. [#25596](https://github.com/ClickHouse/ClickHouse/pull/25596) ([Ildus Kurbangaliev](https://github.com/ildus)). * Support `DISTINCT ON (columns)` expression, close [#25404](https://github.com/ClickHouse/ClickHouse/issues/25404). [#25589](https://github.com/ClickHouse/ClickHouse/pull/25589) ([Zijie Lu](https://github.com/TszKitLo40)). -* Add support for a part of SQLJSON standard. [#24148](https://github.com/ClickHouse/ClickHouse/pull/24148) ([l1tsolaiki](https://github.com/l1tsolaiki)). -* Add MaterializedPostgreSQL table engine and database engine. This database engine allows replicating a whole database or any subset of database tables. [#20470](https://github.com/ClickHouse/ClickHouse/pull/20470) ([Kseniia Sumarokova](https://github.com/kssenii)). * Add an ability to reset a custom setting to default and remove it from the table's metadata. It allows rolling back the change without knowing the system/config's default. Closes [#14449](https://github.com/ClickHouse/ClickHouse/issues/14449). [#17769](https://github.com/ClickHouse/ClickHouse/pull/17769) ([xjewer](https://github.com/xjewer)). * Render pipelines as graphs in Web UI if `EXPLAIN PIPELINE graph = 1` query is submitted. [#26067](https://github.com/ClickHouse/ClickHouse/pull/26067) ([alexey-milovidov](https://github.com/alexey-milovidov)). @@ -21,11 +21,11 @@ #### Improvements -* Use `Map` data type for system logs tables (`system.query_log`, `system.query_thread_log`, `system.processes`, `system.opentelemetry_span_log`). These tables will be auto-created with new data types. Virtual columns are created to support old queries. Closes [#18698](https://github.com/ClickHouse/ClickHouse/issues/18698). [#23934](https://github.com/ClickHouse/ClickHouse/pull/23934), [#25773](https://github.com/ClickHouse/ClickHouse/pull/25773) ([hexiaoting](https://github.com/hexiaoting), [sundy-li](https://github.com/sundy-li)). +* Use `Map` data type for system logs tables (`system.query_log`, `system.query_thread_log`, `system.processes`, `system.opentelemetry_span_log`). These tables will be auto-created with new data types. Virtual columns are created to support old queries. Closes [#18698](https://github.com/ClickHouse/ClickHouse/issues/18698). [#23934](https://github.com/ClickHouse/ClickHouse/pull/23934), [#25773](https://github.com/ClickHouse/ClickHouse/pull/25773) ([hexiaoting](https://github.com/hexiaoting), [sundy-li](https://github.com/sundy-li), [Maksim Kita](https://github.com/kitaisreal)). * For a dictionary with a complex key containing only one attribute, allow not wrapping the key expression in tuple for functions `dictGet`, `dictHas`. [#26130](https://github.com/ClickHouse/ClickHouse/pull/26130) ([Maksim Kita](https://github.com/kitaisreal)). * Implement function `bin`/`hex` from `AggregateFunction` states. [#26094](https://github.com/ClickHouse/ClickHouse/pull/26094) ([zhaoyu](https://github.com/zxc111)). * Support arguments of `UUID` type for `empty` and `notEmpty` functions. `UUID` is empty if it is all zeros (nil UUID). Closes [#3446](https://github.com/ClickHouse/ClickHouse/issues/3446). [#25974](https://github.com/ClickHouse/ClickHouse/pull/25974) ([zhaoyu](https://github.com/zxc111)). -* Fix error with query `SET SQL_SELECT_LIMIT` in MySQL protocol. Closes [#17115](https://github.com/ClickHouse/ClickHouse/issues/17115). [#25972](https://github.com/ClickHouse/ClickHouse/pull/25972) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Add support for `SET SQL_SELECT_LIMIT` in MySQL protocol. Closes [#17115](https://github.com/ClickHouse/ClickHouse/issues/17115). [#25972](https://github.com/ClickHouse/ClickHouse/pull/25972) ([Kseniia Sumarokova](https://github.com/kssenii)). * More instrumentation for network interaction: add counters for recv/send bytes; add gauges for recvs/sends. Added missing documentation. Close [#5897](https://github.com/ClickHouse/ClickHouse/issues/5897). [#25962](https://github.com/ClickHouse/ClickHouse/pull/25962) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Add setting `optimize_move_to_prewhere_if_final`. If query has `FINAL`, the optimization `move_to_prewhere` will be enabled only if both `optimize_move_to_prewhere` and `optimize_move_to_prewhere_if_final` are enabled. Closes [#8684](https://github.com/ClickHouse/ClickHouse/issues/8684). [#25940](https://github.com/ClickHouse/ClickHouse/pull/25940) ([Kseniia Sumarokova](https://github.com/kssenii)). * Allow complex quoted identifiers of JOINed tables. Close [#17861](https://github.com/ClickHouse/ClickHouse/issues/17861). [#25924](https://github.com/ClickHouse/ClickHouse/pull/25924) ([alexey-milovidov](https://github.com/alexey-milovidov)). @@ -37,7 +37,7 @@ * Support for queries with a column named `"null"` (it must be specified in back-ticks or double quotes) and `ON CLUSTER`. Closes [#24035](https://github.com/ClickHouse/ClickHouse/issues/24035). [#25907](https://github.com/ClickHouse/ClickHouse/pull/25907) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Support `LowCardinality`, `Decimal`, and `UUID` for `JSONExtract`. Closes [#24606](https://github.com/ClickHouse/ClickHouse/issues/24606). [#25900](https://github.com/ClickHouse/ClickHouse/pull/25900) ([Kseniia Sumarokova](https://github.com/kssenii)). * Convert history file from `readline` format to `replxx` format. [#25888](https://github.com/ClickHouse/ClickHouse/pull/25888) ([Azat Khuzhin](https://github.com/azat)). -* Fix bug which can lead to intersecting parts after `DROP PART` or background deletion of an empty part. [#25884](https://github.com/ClickHouse/ClickHouse/pull/25884) ([alesapin](https://github.com/alesapin)). +* Fix an issue which can lead to intersecting parts after `DROP PART` or background deletion of an empty part. [#25884](https://github.com/ClickHouse/ClickHouse/pull/25884) ([alesapin](https://github.com/alesapin)). * Better handling of lost parts for `ReplicatedMergeTree` tables. Fixes rare inconsistencies in `ReplicationQueue`. Fixes [#10368](https://github.com/ClickHouse/ClickHouse/issues/10368). [#25820](https://github.com/ClickHouse/ClickHouse/pull/25820) ([alesapin](https://github.com/alesapin)). * Allow starting clickhouse-client with unreadable working directory. [#25817](https://github.com/ClickHouse/ClickHouse/pull/25817) ([ianton-ru](https://github.com/ianton-ru)). * Fix "No available columns" error for `Merge` storage. [#25801](https://github.com/ClickHouse/ClickHouse/pull/25801) ([Azat Khuzhin](https://github.com/azat)). @@ -48,7 +48,7 @@ * Support materialized and aliased columns in JOIN, close [#13274](https://github.com/ClickHouse/ClickHouse/issues/13274). [#25634](https://github.com/ClickHouse/ClickHouse/pull/25634) ([Vladimir C](https://github.com/vdimir)). * Fix possible logical race condition between `ALTER TABLE ... DETACH` and background merges. [#25605](https://github.com/ClickHouse/ClickHouse/pull/25605) ([Azat Khuzhin](https://github.com/azat)). * Make `NetworkReceiveElapsedMicroseconds` metric to correctly include the time spent waiting for data from the client to `INSERT`. Close [#9958](https://github.com/ClickHouse/ClickHouse/issues/9958). [#25602](https://github.com/ClickHouse/ClickHouse/pull/25602) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Support `TRUNCATE TABLE` for StorageS3 and StorageHDFS. Close [#25530](https://github.com/ClickHouse/ClickHouse/issues/25530). [#25550](https://github.com/ClickHouse/ClickHouse/pull/25550) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Support `TRUNCATE TABLE` for S3 and HDFS. Close [#25530](https://github.com/ClickHouse/ClickHouse/issues/25530). [#25550](https://github.com/ClickHouse/ClickHouse/pull/25550) ([Kseniia Sumarokova](https://github.com/kssenii)). * Support for dynamic reloading of config to change number of threads in pool for background jobs execution (merges, mutations, fetches). [#25548](https://github.com/ClickHouse/ClickHouse/pull/25548) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). * Allow extracting of non-string element as string using `JSONExtract`. This is for [#25414](https://github.com/ClickHouse/ClickHouse/issues/25414). [#25452](https://github.com/ClickHouse/ClickHouse/pull/25452) ([Amos Bird](https://github.com/amosbird)). * Support regular expression in `Database` argument for `StorageMerge`. Close [#776](https://github.com/ClickHouse/ClickHouse/issues/776). [#25064](https://github.com/ClickHouse/ClickHouse/pull/25064) ([flynn](https://github.com/ucasfl)). @@ -60,13 +60,13 @@ * Fix incorrect `SET ROLE` in some cases. [#26707](https://github.com/ClickHouse/ClickHouse/pull/26707) ([Vitaly Baranov](https://github.com/vitlibar)). * Fix potential `nullptr` dereference in window functions. Fix [#25276](https://github.com/ClickHouse/ClickHouse/issues/25276). [#26668](https://github.com/ClickHouse/ClickHouse/pull/26668) ([Alexander Kuzmenkov](https://github.com/akuzm)). * Fix incorrect function names of `groupBitmapAnd/Or/Xor`. Fix [#26557](https://github.com/ClickHouse/ClickHouse/pull/26557) ([Amos Bird](https://github.com/amosbird)). -* Fix crash in rabbitmq shutdown in case rabbitmq setup was not started. Closes [#26504](https://github.com/ClickHouse/ClickHouse/issues/26504). [#26529](https://github.com/ClickHouse/ClickHouse/pull/26529) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix crash in RabbitMQ shutdown in case RabbitMQ setup was not started. Closes [#26504](https://github.com/ClickHouse/ClickHouse/issues/26504). [#26529](https://github.com/ClickHouse/ClickHouse/pull/26529) ([Kseniia Sumarokova](https://github.com/kssenii)). * Fix issues with `CREATE DICTIONARY` query if dictionary name or database name was quoted. Closes [#26491](https://github.com/ClickHouse/ClickHouse/issues/26491). [#26508](https://github.com/ClickHouse/ClickHouse/pull/26508) ([Maksim Kita](https://github.com/kitaisreal)). * Fix broken name resolution after rewriting column aliases. Fix [#26432](https://github.com/ClickHouse/ClickHouse/issues/26432). [#26475](https://github.com/ClickHouse/ClickHouse/pull/26475) ([Amos Bird](https://github.com/amosbird)). * Fix infinite non-joined block stream in `partial_merge_join` close [#26325](https://github.com/ClickHouse/ClickHouse/issues/26325). [#26374](https://github.com/ClickHouse/ClickHouse/pull/26374) ([Vladimir C](https://github.com/vdimir)). * Fix possible crash when login as dropped user. Fix [#26073](https://github.com/ClickHouse/ClickHouse/issues/26073). [#26363](https://github.com/ClickHouse/ClickHouse/pull/26363) ([Vitaly Baranov](https://github.com/vitlibar)). * Fix `optimize_distributed_group_by_sharding_key` for multiple columns (leads to incorrect result w/ `optimize_skip_unused_shards=1`/`allow_nondeterministic_optimize_skip_unused_shards=1` and multiple columns in sharding key expression). [#26353](https://github.com/ClickHouse/ClickHouse/pull/26353) ([Azat Khuzhin](https://github.com/azat)). - * `CAST` from `Date` to `DateTime` (or `DateTime64`) was not using the timezone of the `DateTime` type. It can also affect the comparison between `Date` and `DateTime`. Inference of the common type for `Date` and `DateTime` also was not using the corresponding timezone. It affected the results of function `if` and array construction. Closes [#24128](https://github.com/ClickHouse/ClickHouse/issues/24128). [#24129](https://github.com/ClickHouse/ClickHouse/pull/24129) ([Maksim Kita](https://github.com/kitaisreal)). +* `CAST` from `Date` to `DateTime` (or `DateTime64`) was not using the timezone of the `DateTime` type. It can also affect the comparison between `Date` and `DateTime`. Inference of the common type for `Date` and `DateTime` also was not using the corresponding timezone. It affected the results of function `if` and array construction. Closes [#24128](https://github.com/ClickHouse/ClickHouse/issues/24128). [#24129](https://github.com/ClickHouse/ClickHouse/pull/24129) ([Maksim Kita](https://github.com/kitaisreal)). * Fixed rare bug in lost replica recovery that may cause replicas to diverge. [#26321](https://github.com/ClickHouse/ClickHouse/pull/26321) ([tavplubix](https://github.com/tavplubix)). * Fix zstd decompression in case there are escape sequences at the end of internal buffer. Closes [#26013](https://github.com/ClickHouse/ClickHouse/issues/26013). [#26314](https://github.com/ClickHouse/ClickHouse/pull/26314) ([Kseniia Sumarokova](https://github.com/kssenii)). * Fix logical error on join with totals, close [#26017](https://github.com/ClickHouse/ClickHouse/issues/26017). [#26250](https://github.com/ClickHouse/ClickHouse/pull/26250) ([Vladimir C](https://github.com/vdimir)). @@ -75,7 +75,7 @@ * Fix possible crash in `pointInPolygon` if the setting `validate_polygons` is turned off. [#26113](https://github.com/ClickHouse/ClickHouse/pull/26113) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Fix throwing exception when iterate over non-existing remote directory. [#26087](https://github.com/ClickHouse/ClickHouse/pull/26087) ([ianton-ru](https://github.com/ianton-ru)). * Fix rare server crash because of `abort` in ZooKeeper client. Fixes [#25813](https://github.com/ClickHouse/ClickHouse/issues/25813). [#26079](https://github.com/ClickHouse/ClickHouse/pull/26079) ([alesapin](https://github.com/alesapin)). -* Fix wrong thread estimation for right subquery join in some cases. Close [#24075](https://github.com/ClickHouse/ClickHouse/issues/24075). [#26052](https://github.com/ClickHouse/ClickHouse/pull/26052) ([Vladimir C](https://github.com/vdimir)). +* Fix wrong thread count estimation for right subquery join in some cases. Close [#24075](https://github.com/ClickHouse/ClickHouse/issues/24075). [#26052](https://github.com/ClickHouse/ClickHouse/pull/26052) ([Vladimir C](https://github.com/vdimir)). * Fixed incorrect `sequence_id` in MySQL protocol packets that ClickHouse sends on exception during query execution. It might cause MySQL client to reset connection to ClickHouse server. Fixes [#21184](https://github.com/ClickHouse/ClickHouse/issues/21184). [#26051](https://github.com/ClickHouse/ClickHouse/pull/26051) ([tavplubix](https://github.com/tavplubix)). * Fix possible mismatched header when using normal projection with `PREWHERE`. Fix [#26020](https://github.com/ClickHouse/ClickHouse/issues/26020). [#26038](https://github.com/ClickHouse/ClickHouse/pull/26038) ([Amos Bird](https://github.com/amosbird)). * Fix formatting of type `Map` with integer keys to `JSON`. [#25982](https://github.com/ClickHouse/ClickHouse/pull/25982) ([Anton Popov](https://github.com/CurtizJ)). @@ -94,20 +94,8 @@ * Fix `ALTER MODIFY COLUMN` of columns, which participates in TTL expressions. [#25554](https://github.com/ClickHouse/ClickHouse/pull/25554) ([Anton Popov](https://github.com/CurtizJ)). * Fix assertion in `PREWHERE` with non-UInt8 type, close [#19589](https://github.com/ClickHouse/ClickHouse/issues/19589). [#25484](https://github.com/ClickHouse/ClickHouse/pull/25484) ([Vladimir C](https://github.com/vdimir)). * Fix some fuzzed msan crash. Fixes [#22517](https://github.com/ClickHouse/ClickHouse/issues/22517). [#26428](https://github.com/ClickHouse/ClickHouse/pull/26428) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix empty history file conversion. [#26589](https://github.com/ClickHouse/ClickHouse/pull/26589) ([Azat Khuzhin](https://github.com/azat)). * Update `chown` cmd check in `clickhouse-server` docker entrypoint. It fixes error 'cluster pod restart failed (or timeout)' on kubernetes. [#26545](https://github.com/ClickHouse/ClickHouse/pull/26545) ([Ky Li](https://github.com/Kylinrix)). -#### Build/Testing/Packaging Improvements - -* Disabling TestFlows LDAP module due to test fails. [#26065](https://github.com/ClickHouse/ClickHouse/pull/26065) ([vzakaznikov](https://github.com/vzakaznikov)). -* Enabling all TestFlows modules and fixing some tests. [#26011](https://github.com/ClickHouse/ClickHouse/pull/26011) ([vzakaznikov](https://github.com/vzakaznikov)). -* Add new tests for checking access rights for columns used in filters (`WHERE` / `PREWHERE` / row policy) of the `SELECT` statement after changes in [#24405](https://github.com/ClickHouse/ClickHouse/pull/24405). [#25619](https://github.com/ClickHouse/ClickHouse/pull/25619) ([Vitaly Baranov](https://github.com/vitlibar)). - -#### Other - -* Add `clickhouse-keeper-converter` tool which allows converting zookeeper logs and snapshots into `clickhouse-keeper` snapshot format. [#25428](https://github.com/ClickHouse/ClickHouse/pull/25428) ([alesapin](https://github.com/alesapin)). - - ### ClickHouse release v21.7, 2021-07-09 From c92e1143e3cc332ffdf113b1f067866cf2676ae7 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 14 Aug 2021 02:50:25 +0300 Subject: [PATCH 126/161] Update CHANGELOG.md --- CHANGELOG.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 12fda73fe13..9562703d517 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -2,8 +2,8 @@ #### New Features -* Add support for a part of SQL/JSON standard. [#24148](https://github.com/ClickHouse/ClickHouse/pull/24148) ([l1tsolaiki](https://github.com/l1tsolaiki)). -* Collect common system metrics (in `system.asynchronous_metrics` and `system.asynchronous_metric_log`) on CPU usage, disk usage, memory usage, IO, network, files, load average, CPU frequencies, thermal sensors, EDAC counters, system uptime; also added metrics about the scheduling jitter and the time spent collecting the metrics. It works similar to `atop` in ClickHouse and allows access to monitoring data even if you have no additional tools installed. Close [#9430](https://github.com/ClickHouse/ClickHouse/issues/9430). [#24416](https://github.com/ClickHouse/ClickHouse/pull/24416) (([alexey-milovidov](https://github.com/alexey-milovidov)), [Yegor Levankov](https://github.com/elevankoff), [Kseniia Sumarokova](https://github.com/kssenii)). +* Add support for a part of SQL/JSON standard. [#24148](https://github.com/ClickHouse/ClickHouse/pull/24148) ([l1tsolaiki](https://github.com/l1tsolaiki), [Kseniia Sumarokova](https://github.com/kssenii)). +* Collect common system metrics (in `system.asynchronous_metrics` and `system.asynchronous_metric_log`) on CPU usage, disk usage, memory usage, IO, network, files, load average, CPU frequencies, thermal sensors, EDAC counters, system uptime; also added metrics about the scheduling jitter and the time spent collecting the metrics. It works similar to `atop` in ClickHouse and allows access to monitoring data even if you have no additional tools installed. Close [#9430](https://github.com/ClickHouse/ClickHouse/issues/9430). [#24416](https://github.com/ClickHouse/ClickHouse/pull/24416) (([alexey-milovidov](https://github.com/alexey-milovidov)), [Yegor Levankov](https://github.com/elevankoff)). * Add MaterializedPostgreSQL table engine and database engine. This database engine allows replicating a whole database or any subset of database tables. [#20470](https://github.com/ClickHouse/ClickHouse/pull/20470) ([Kseniia Sumarokova](https://github.com/kssenii)). * Add new functions `leftPad()`, `rightPad()`, `leftPadUTF8()`, `rightPadUTF8()`. [#26075](https://github.com/ClickHouse/ClickHouse/pull/26075) ([Vitaly Baranov](https://github.com/vitlibar)). * Add the `FIRST` keyword to the `ADD INDEX` command to be able to add the index at the beginning of the indices list. [#25904](https://github.com/ClickHouse/ClickHouse/pull/25904) ([xjewer](https://github.com/xjewer)). From 596fa80c15226eb9218b746c21147f34f77fd2d1 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 14 Aug 2021 02:51:01 +0300 Subject: [PATCH 127/161] Update CHANGELOG.md --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 9562703d517..6b0570f0d4a 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -3,7 +3,7 @@ #### New Features * Add support for a part of SQL/JSON standard. [#24148](https://github.com/ClickHouse/ClickHouse/pull/24148) ([l1tsolaiki](https://github.com/l1tsolaiki), [Kseniia Sumarokova](https://github.com/kssenii)). -* Collect common system metrics (in `system.asynchronous_metrics` and `system.asynchronous_metric_log`) on CPU usage, disk usage, memory usage, IO, network, files, load average, CPU frequencies, thermal sensors, EDAC counters, system uptime; also added metrics about the scheduling jitter and the time spent collecting the metrics. It works similar to `atop` in ClickHouse and allows access to monitoring data even if you have no additional tools installed. Close [#9430](https://github.com/ClickHouse/ClickHouse/issues/9430). [#24416](https://github.com/ClickHouse/ClickHouse/pull/24416) (([alexey-milovidov](https://github.com/alexey-milovidov)), [Yegor Levankov](https://github.com/elevankoff)). +* Collect common system metrics (in `system.asynchronous_metrics` and `system.asynchronous_metric_log`) on CPU usage, disk usage, memory usage, IO, network, files, load average, CPU frequencies, thermal sensors, EDAC counters, system uptime; also added metrics about the scheduling jitter and the time spent collecting the metrics. It works similar to `atop` in ClickHouse and allows access to monitoring data even if you have no additional tools installed. Close [#9430](https://github.com/ClickHouse/ClickHouse/issues/9430). [#24416](https://github.com/ClickHouse/ClickHouse/pull/24416) ([alexey-milovidov](https://github.com/alexey-milovidov), [Yegor Levankov](https://github.com/elevankoff)). * Add MaterializedPostgreSQL table engine and database engine. This database engine allows replicating a whole database or any subset of database tables. [#20470](https://github.com/ClickHouse/ClickHouse/pull/20470) ([Kseniia Sumarokova](https://github.com/kssenii)). * Add new functions `leftPad()`, `rightPad()`, `leftPadUTF8()`, `rightPadUTF8()`. [#26075](https://github.com/ClickHouse/ClickHouse/pull/26075) ([Vitaly Baranov](https://github.com/vitlibar)). * Add the `FIRST` keyword to the `ADD INDEX` command to be able to add the index at the beginning of the indices list. [#25904](https://github.com/ClickHouse/ClickHouse/pull/25904) ([xjewer](https://github.com/xjewer)). From 032d75b638a80953ce26810145aa9513af31ad27 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 14 Aug 2021 02:53:36 +0300 Subject: [PATCH 128/161] Update CHANGELOG.md --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 6b0570f0d4a..ca3d21af8e5 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -9,7 +9,7 @@ * Add the `FIRST` keyword to the `ADD INDEX` command to be able to add the index at the beginning of the indices list. [#25904](https://github.com/ClickHouse/ClickHouse/pull/25904) ([xjewer](https://github.com/xjewer)). * Introduce `system.data_skipping_indices` table containing information about existing data skipping indices. Close [#7659](https://github.com/ClickHouse/ClickHouse/issues/7659). [#25693](https://github.com/ClickHouse/ClickHouse/pull/25693) ([Dmitry Novik](https://github.com/novikd)). * Add `bin`/`unbin` functions. [#25609](https://github.com/ClickHouse/ClickHouse/pull/25609) ([zhaoyu](https://github.com/zxc111)). -* Support `Map` and `(U)Int128`, `U(Int256) types in `mapAdd` and `mapSubtract` functions. [#25596](https://github.com/ClickHouse/ClickHouse/pull/25596) ([Ildus Kurbangaliev](https://github.com/ildus)). +* Support `Map` and `UInt128`, `Int128`, `UInt256`, `Int256` types in `mapAdd` and `mapSubtract` functions. [#25596](https://github.com/ClickHouse/ClickHouse/pull/25596) ([Ildus Kurbangaliev](https://github.com/ildus)). * Support `DISTINCT ON (columns)` expression, close [#25404](https://github.com/ClickHouse/ClickHouse/issues/25404). [#25589](https://github.com/ClickHouse/ClickHouse/pull/25589) ([Zijie Lu](https://github.com/TszKitLo40)). * Add an ability to reset a custom setting to default and remove it from the table's metadata. It allows rolling back the change without knowing the system/config's default. Closes [#14449](https://github.com/ClickHouse/ClickHouse/issues/14449). [#17769](https://github.com/ClickHouse/ClickHouse/pull/17769) ([xjewer](https://github.com/xjewer)). * Render pipelines as graphs in Web UI if `EXPLAIN PIPELINE graph = 1` query is submitted. [#26067](https://github.com/ClickHouse/ClickHouse/pull/26067) ([alexey-milovidov](https://github.com/alexey-milovidov)). From 9ea4bea8f9ac83ed9ae4420527d52f9a8ce15f17 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 14 Aug 2021 02:54:05 +0300 Subject: [PATCH 129/161] Update CHANGELOG.md --- CHANGELOG.md | 7 ------- 1 file changed, 7 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index ca3d21af8e5..103d8e40fd9 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1282,13 +1282,6 @@ * PODArray: Avoid call to memcpy with (nullptr, 0) arguments (Fix UBSan report). This fixes [#18525](https://github.com/ClickHouse/ClickHouse/issues/18525). [#18526](https://github.com/ClickHouse/ClickHouse/pull/18526) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Minor improvement for path concatenation of zookeeper paths inside DDLWorker. [#17767](https://github.com/ClickHouse/ClickHouse/pull/17767) ([Bharat Nallan](https://github.com/bharatnc)). * Allow to reload symbols from debug file. This PR also fixes a build-id issue. [#17637](https://github.com/ClickHouse/ClickHouse/pull/17637) ([Amos Bird](https://github.com/amosbird)). -* TestFlows: fixes to LDAP tests that fail due to slow test execution. [#18790](https://github.com/ClickHouse/ClickHouse/pull/18790) ([vzakaznikov](https://github.com/vzakaznikov)). -* TestFlows: Merging requirements for AES encryption functions. Updating aes_encryption tests to use new requirements. Updating TestFlows version to 1.6.72. [#18221](https://github.com/ClickHouse/ClickHouse/pull/18221) ([vzakaznikov](https://github.com/vzakaznikov)). -* TestFlows: Updating TestFlows version to the latest 1.6.72. Re-generating requirements.py. [#18208](https://github.com/ClickHouse/ClickHouse/pull/18208) ([vzakaznikov](https://github.com/vzakaznikov)). -* TestFlows: Updating TestFlows README.md to include "How To Debug Why Test Failed" section. [#17808](https://github.com/ClickHouse/ClickHouse/pull/17808) ([vzakaznikov](https://github.com/vzakaznikov)). -* TestFlows: tests for RBAC [ACCESS MANAGEMENT](https://clickhouse.tech/docs/en/sql-reference/statements/grant/#grant-access-management) privileges. [#17804](https://github.com/ClickHouse/ClickHouse/pull/17804) ([MyroTk](https://github.com/MyroTk)). -* TestFlows: RBAC tests for SHOW, TRUNCATE, KILL, and OPTIMIZE. - Updates to old tests. - Resolved comments from #https://github.com/ClickHouse/ClickHouse/pull/16977. [#17657](https://github.com/ClickHouse/ClickHouse/pull/17657) ([MyroTk](https://github.com/MyroTk)). -* TestFlows: Added RBAC tests for `ATTACH`, `CREATE`, `DROP`, and `DETACH`. [#16977](https://github.com/ClickHouse/ClickHouse/pull/16977) ([MyroTk](https://github.com/MyroTk)). ## [Changelog for 2020](https://github.com/ClickHouse/ClickHouse/blob/master/docs/en/whats-new/changelog/2020.md) From 18ab53488fab117b8c08349b22c914732b9d69eb Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 14 Aug 2021 15:31:55 +0300 Subject: [PATCH 130/161] Better --- .../InterpreterSelectIntersectExceptQuery.cpp | 2 +- .../SelectIntersectExceptQueryVisitor.cpp | 163 +++++++----------- .../SelectIntersectExceptQueryVisitor.h | 18 +- src/Parsers/ASTSelectIntersectExceptQuery.cpp | 42 +---- src/Parsers/ASTSelectIntersectExceptQuery.h | 5 - src/Parsers/ASTSelectWithUnionQuery.h | 3 +- src/Parsers/ExpressionListParsers.cpp | 22 ++- src/Parsers/ExpressionListParsers.h | 19 -- src/Parsers/ParserQueryWithOutput.cpp | 1 - .../ParserSelectIntersectExceptQuery.cpp | 62 ------- .../ParserSelectIntersectExceptQuery.h | 14 -- src/Parsers/ParserSelectWithUnionQuery.cpp | 8 +- src/Parsers/ParserUnionQueryElement.cpp | 2 - ...02004_intersect_except_operators.reference | 23 +++ .../02004_intersect_except_operators.sql | 7 + 15 files changed, 117 insertions(+), 274 deletions(-) delete mode 100644 src/Parsers/ParserSelectIntersectExceptQuery.cpp delete mode 100644 src/Parsers/ParserSelectIntersectExceptQuery.h diff --git a/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp b/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp index 34d7ae5b37f..4edd13d08e5 100644 --- a/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp +++ b/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp @@ -56,7 +56,7 @@ InterpreterSelectIntersectExceptQuery::InterpreterSelectIntersectExceptQuery( ASTSelectIntersectExceptQuery * ast = query_ptr->as(); final_operator = ast->final_operator; - const auto & children = ast->children[0]->children; + const auto & children = ast->children; size_t num_children = children.size(); /// AST must have been changed by the visitor. diff --git a/src/Interpreters/SelectIntersectExceptQueryVisitor.cpp b/src/Interpreters/SelectIntersectExceptQueryVisitor.cpp index 273bc327dc3..3815fb4ad0f 100644 --- a/src/Interpreters/SelectIntersectExceptQueryVisitor.cpp +++ b/src/Interpreters/SelectIntersectExceptQueryVisitor.cpp @@ -18,87 +18,12 @@ namespace DB void SelectIntersectExceptQueryMatcher::visit(ASTPtr & ast, Data & data) { - if (auto * select_intersect_except = ast->as()) - { - if (select_intersect_except->final_operator != ASTSelectIntersectExceptQuery::Operator::UNKNOWN) - return; - - data.initialize(select_intersect_except); - visit(*select_intersect_except, data); - } - else if (auto * select_union = ast->as()) - { + if (auto * select_union = ast->as()) visit(*select_union, data); - } -} - -void SelectIntersectExceptQueryMatcher::visit(ASTSelectIntersectExceptQuery & ast, Data & data) -{ - /* Example: select 1 intersect select 1 intsect select 1 intersect select 1 intersect select 1; - * - * --SelectIntersectExceptQuery --SelectIntersectExceptQuery - * ---ExpressionList ---ExpressionList - * ----SelectQuery ----SelectIntersectExceptQuery - * ----SelectQuery ------ExpressionList - * ----SelectQuery ---> -------SelectIntersectExceptQuery - * ----SelectQuery --------ExpressionList - * ---------SelectQuery - * ---------SelectQuery - * -------SelectQuery - * ----SelectQuery - **/ - - auto & selects = data.reversed_list_of_selects; - - if (selects.empty()) - return; - - const auto left = selects.back(); - selects.pop_back(); - const auto right = selects.back(); - selects.pop_back(); - - auto & operators = data.reversed_list_of_operators; - const auto current_operator = operators.back(); - operators.pop_back(); - - auto list_node = std::make_shared(); - list_node->children = {left, right}; - - if (selects.empty()) - { - ast.final_operator = current_operator; - ast.children = {std::move(list_node)}; - } - else - { - auto select_intersect_except = std::make_shared(); - select_intersect_except->final_operator = {current_operator}; - select_intersect_except->list_of_selects = std::move(list_node); - select_intersect_except->children.push_back(select_intersect_except->list_of_selects); - - selects.emplace_back(std::move(select_intersect_except)); - } - - visit(ast, data); } void SelectIntersectExceptQueryMatcher::visit(ASTSelectWithUnionQuery & ast, Data &) { - /* Example: select 1 union all select 2 except select 1 except select 2 union distinct select 5; - * - * --SelectWithUnionQuery --SelectIntersectExceptQuery - * ---ExpressionList ---ExpressionList - * ----SelectQuery ----SelectIntersectExceptQuery - * ----SelectQuery -----ExpressionList - * ----SelectQuery (except) ---> ------SelectIntersectExceptQuery - * ----SelectQuery (except) -------ExpressionList - * ----SelectQuery --------SelectWithUnionQuery (select 1 union all select 2) - * --------SelectQuery (select 1) - * ------SelectQuery (select 2) - * -----SelectQuery (select 5) - **/ - auto & union_modes = ast.list_of_modes; if (union_modes.empty()) @@ -107,8 +32,7 @@ void SelectIntersectExceptQueryMatcher::visit(ASTSelectWithUnionQuery & ast, Dat auto selects = std::move(ast.list_of_selects->children); if (union_modes.size() + 1 != selects.size()) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Incorrect ASTSelectWithUnionQuery (modes: {}, selects: {})", + throw Exception(ErrorCodes::LOGICAL_ERROR, "Incorrect ASTSelectWithUnionQuery (modes: {}, selects: {})", union_modes.size(), selects.size()); std::reverse(selects.begin(), selects.end()); @@ -119,39 +43,70 @@ void SelectIntersectExceptQueryMatcher::visit(ASTSelectWithUnionQuery & ast, Dat for (const auto & mode : union_modes) { - /// Flatten all previous selects into ASTSelectIntersectQuery - if (mode == ASTSelectWithUnionQuery::Mode::EXCEPT) + switch (mode) { - auto left = std::make_shared(); - left->union_mode = ASTSelectWithUnionQuery::Mode::ALL; + case ASTSelectWithUnionQuery::Mode::EXCEPT: + { + auto left = std::make_shared(); + left->union_mode = ASTSelectWithUnionQuery::Mode::ALL; - left->list_of_selects = std::make_shared(); - left->children.push_back(left->list_of_selects); - left->list_of_selects->children = std::move(children); + left->list_of_selects = std::make_shared(); + left->children.push_back(left->list_of_selects); + left->list_of_selects->children = std::move(children); - left->list_of_modes = std::move(modes); - modes = {}; + left->list_of_modes = std::move(modes); + modes = {}; - auto right = selects.back(); - selects.pop_back(); + auto right = selects.back(); + selects.pop_back(); - auto list_node = std::make_shared(); - list_node->children = {left, right}; + auto except_node = std::make_shared(); + except_node->final_operator = ASTSelectIntersectExceptQuery::Operator::EXCEPT; + except_node->children = {left, right}; - auto select_intersect_except = std::make_shared(); - select_intersect_except->final_operator = {ASTSelectIntersectExceptQuery::Operator::EXCEPT}; - select_intersect_except->children.emplace_back(std::move(list_node)); - select_intersect_except->list_of_selects = std::make_shared(); - select_intersect_except->list_of_selects->children.push_back(select_intersect_except->children[0]); + children = {except_node}; + break; + } + case ASTSelectWithUnionQuery::Mode::INTERSECT: + { + bool from_except = false; + const auto * except_ast = typeid_cast(children.back().get()); + if (except_ast && (except_ast->final_operator == ASTSelectIntersectExceptQuery::Operator::EXCEPT)) + from_except = true; - children = {select_intersect_except}; - } - else if (!selects.empty()) - { - auto right = selects.back(); - selects.pop_back(); - children.emplace_back(std::move(right)); - modes.push_back(mode); + ASTPtr left; + if (from_except) + { + left = std::move(children.back()->children[1]); + } + else + { + left = children.back(); + children.pop_back(); + } + + auto right = selects.back(); + selects.pop_back(); + + auto intersect_node = std::make_shared(); + intersect_node->final_operator = ASTSelectIntersectExceptQuery::Operator::INTERSECT; + intersect_node->children = {left, right}; + + if (from_except) + children.back()->children[1] = std::move(intersect_node); + else + children.push_back(std::move(intersect_node)); + + break; + } + default: + { + auto right = selects.back(); + selects.pop_back(); + children.emplace_back(std::move(right)); + modes.push_back(mode); + break; + } } } diff --git a/src/Interpreters/SelectIntersectExceptQueryVisitor.h b/src/Interpreters/SelectIntersectExceptQueryVisitor.h index 1dd0694666d..07a6ad606a1 100644 --- a/src/Interpreters/SelectIntersectExceptQueryVisitor.h +++ b/src/Interpreters/SelectIntersectExceptQueryVisitor.h @@ -17,27 +17,11 @@ class ASTFunction; class SelectIntersectExceptQueryMatcher { public: - struct Data - { - Data() = default; - - void initialize(const ASTSelectIntersectExceptQuery * select_intersect_except) - { - reversed_list_of_selects = select_intersect_except->list_of_selects->clone()->children; - reversed_list_of_operators = select_intersect_except->list_of_operators; - - std::reverse(reversed_list_of_selects.begin(), reversed_list_of_selects.end()); - std::reverse(reversed_list_of_operators.begin(), reversed_list_of_operators.end()); - } - - ASTs reversed_list_of_selects; - ASTSelectIntersectExceptQuery::Operators reversed_list_of_operators; - }; + struct Data {}; static bool needChildVisit(const ASTPtr &, const ASTPtr &) { return true; } static void visit(ASTPtr & ast, Data &); - static void visit(ASTSelectIntersectExceptQuery &, Data &); static void visit(ASTSelectWithUnionQuery &, Data &); }; diff --git a/src/Parsers/ASTSelectIntersectExceptQuery.cpp b/src/Parsers/ASTSelectIntersectExceptQuery.cpp index 26fd9353d5b..9d7a717fa6c 100644 --- a/src/Parsers/ASTSelectIntersectExceptQuery.cpp +++ b/src/Parsers/ASTSelectIntersectExceptQuery.cpp @@ -14,10 +14,6 @@ ASTPtr ASTSelectIntersectExceptQuery::clone() const for (const auto & child : children) res->children.push_back(child->clone()); - if (res->list_of_selects) - res->list_of_selects = list_of_selects->clone(); - - res->list_of_operators = list_of_operators; res->final_operator = final_operator; cloneOutputOptions(*res); @@ -28,44 +24,18 @@ void ASTSelectIntersectExceptQuery::formatQueryImpl(const FormatSettings & setti { std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' '); - auto operator_to_str = [&](auto current_operator) + for (ASTs::const_iterator it = children.begin(); it != children.end(); ++it) { - if (current_operator == Operator::INTERSECT) - return "INTERSECT"; - else - return "EXCEPT"; - }; - - for (ASTs::const_iterator it = list_of_selects->children.begin(); it != list_of_selects->children.end(); ++it) - { - if (it != list_of_selects->children.begin()) + if (it != children.begin()) { settings.ostr << settings.nl_or_ws << indent_str << (settings.hilite ? hilite_keyword : "") - << operator_to_str(list_of_operators[it - list_of_selects->children.begin() - 1]) + << (final_operator == Operator::INTERSECT ? "INTERSECT" : "EXCEPT") << (settings.hilite ? hilite_none : ""); } - if (auto * node = (*it)->as()) - { - settings.ostr << settings.nl_or_ws << indent_str; - - if (node->list_of_selects->children.size() == 1) - { - (node->list_of_selects->children.at(0))->formatImpl(settings, state, frame); - } - else - { - auto sub_query = std::make_shared(); - sub_query->children.push_back(*it); - sub_query->formatImpl(settings, state, frame); - } - } - else - { - if (it != list_of_selects->children.begin()) - settings.ostr << settings.nl_or_ws; - (*it)->formatImpl(settings, state, frame); - } + if (it != children.begin()) + settings.ostr << settings.nl_or_ws; + (*it)->formatImpl(settings, state, frame); } } diff --git a/src/Parsers/ASTSelectIntersectExceptQuery.h b/src/Parsers/ASTSelectIntersectExceptQuery.h index 8fc5756e370..97a8296ce2c 100644 --- a/src/Parsers/ASTSelectIntersectExceptQuery.h +++ b/src/Parsers/ASTSelectIntersectExceptQuery.h @@ -24,11 +24,6 @@ public: EXCEPT }; - using Operators = std::vector; - - ASTPtr list_of_selects; - Operators list_of_operators; - /// Final operator after applying visitor. Operator final_operator = Operator::UNKNOWN; }; diff --git a/src/Parsers/ASTSelectWithUnionQuery.h b/src/Parsers/ASTSelectWithUnionQuery.h index 2c36bcecf6b..629e9b5d96d 100644 --- a/src/Parsers/ASTSelectWithUnionQuery.h +++ b/src/Parsers/ASTSelectWithUnionQuery.h @@ -23,7 +23,8 @@ public: Unspecified, ALL, DISTINCT, - EXCEPT + EXCEPT, + INTERSECT }; using UnionModes = std::vector; diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index 33085379abb..ef54c627aab 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include @@ -111,12 +112,18 @@ bool ParserList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) bool ParserUnionList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { + ParserUnionQueryElement elem_parser; + ParserKeyword s_union_parser("UNION"); + ParserKeyword s_all_parser("ALL"); + ParserKeyword s_distinct_parser("DISTINCT"); + ParserKeyword s_except_parser("EXCEPT"); + ParserKeyword s_intersect_parser("INTERSECT"); ASTs elements; auto parse_element = [&] { ASTPtr element; - if (!elem_parser->parse(pos, element, expected)) + if (!elem_parser.parse(pos, element, expected)) return false; elements.push_back(element); @@ -126,15 +133,15 @@ bool ParserUnionList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) /// Parse UNION type auto parse_separator = [&] { - if (s_union_parser->ignore(pos, expected)) + if (s_union_parser.ignore(pos, expected)) { // SELECT ... UNION ALL SELECT ... - if (s_all_parser->check(pos, expected)) + if (s_all_parser.check(pos, expected)) { union_modes.push_back(ASTSelectWithUnionQuery::Mode::ALL); } // SELECT ... UNION DISTINCT SELECT ... - else if (s_distinct_parser->check(pos, expected)) + else if (s_distinct_parser.check(pos, expected)) { union_modes.push_back(ASTSelectWithUnionQuery::Mode::DISTINCT); } @@ -145,11 +152,16 @@ bool ParserUnionList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) } return true; } - else if (s_except_parser->check(pos, expected)) + else if (s_except_parser.check(pos, expected)) { union_modes.push_back(ASTSelectWithUnionQuery::Mode::EXCEPT); return true; } + else if (s_intersect_parser.check(pos, expected)) + { + union_modes.push_back(ASTSelectWithUnionQuery::Mode::INTERSECT); + return true; + } return false; }; diff --git a/src/Parsers/ExpressionListParsers.h b/src/Parsers/ExpressionListParsers.h index 36f39a50ab3..e44cacb313f 100644 --- a/src/Parsers/ExpressionListParsers.h +++ b/src/Parsers/ExpressionListParsers.h @@ -79,20 +79,6 @@ private: class ParserUnionList : public IParserBase { public: - ParserUnionList( - ParserPtr && elem_parser_, - ParserPtr && s_union_parser_, - ParserPtr && s_all_parser_, - ParserPtr && s_distinct_parser_, - ParserPtr && s_except_parser_) - : elem_parser(std::move(elem_parser_)) - , s_union_parser(std::move(s_union_parser_)) - , s_all_parser(std::move(s_all_parser_)) - , s_distinct_parser(std::move(s_distinct_parser_)) - , s_except_parser(std::move(s_except_parser_)) - { - } - template static bool parseUtil(Pos & pos, const ElemFunc & parse_element, const SepFunc & parse_separator) { @@ -122,11 +108,6 @@ protected: const char * getName() const override { return "list of union elements"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; private: - ParserPtr elem_parser; - ParserPtr s_union_parser; - ParserPtr s_all_parser; - ParserPtr s_distinct_parser; - ParserPtr s_except_parser; ASTSelectWithUnionQuery::UnionModes union_modes; }; diff --git a/src/Parsers/ParserQueryWithOutput.cpp b/src/Parsers/ParserQueryWithOutput.cpp index e2ab8a84cc1..4a73952674c 100644 --- a/src/Parsers/ParserQueryWithOutput.cpp +++ b/src/Parsers/ParserQueryWithOutput.cpp @@ -7,7 +7,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Parsers/ParserSelectIntersectExceptQuery.cpp b/src/Parsers/ParserSelectIntersectExceptQuery.cpp deleted file mode 100644 index 2b4ba9d60e2..00000000000 --- a/src/Parsers/ParserSelectIntersectExceptQuery.cpp +++ /dev/null @@ -1,62 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -bool ParserSelectIntersectExceptQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) -{ - ParserKeyword intersect_keyword("INTERSECT"); - - ASTs elements; - ASTSelectIntersectExceptQuery::Operators operators; - - auto parse_element = [&]() -> bool - { - ASTPtr element; - if (!ParserSelectQuery().parse(pos, element, expected) && !ParserSubquery().parse(pos, element, expected)) - return false; - - elements.push_back(element); - return true; - }; - - auto parse_separator = [&]() -> bool - { - if (!intersect_keyword.ignore(pos)) - return false; - - operators.emplace_back(ASTSelectIntersectExceptQuery::Operator::INTERSECT); - return true; - }; - - if (!ParserUnionList::parseUtil(pos, parse_element, parse_separator)) - return false; - - if (operators.empty() || elements.empty()) - return false; - - if (operators.size() + 1 != elements.size()) - return false; - - auto list_node = std::make_shared(); - list_node->children = std::move(elements); - - auto intersect_or_except_ast = std::make_shared(); - - node = intersect_or_except_ast; - intersect_or_except_ast->list_of_selects = list_node; - intersect_or_except_ast->children.push_back(intersect_or_except_ast->list_of_selects); - intersect_or_except_ast->list_of_operators = operators; - - return true; -} - -} diff --git a/src/Parsers/ParserSelectIntersectExceptQuery.h b/src/Parsers/ParserSelectIntersectExceptQuery.h deleted file mode 100644 index e01785113a8..00000000000 --- a/src/Parsers/ParserSelectIntersectExceptQuery.h +++ /dev/null @@ -1,14 +0,0 @@ -#pragma once -#include - - -namespace DB -{ -class ParserSelectIntersectExceptQuery : public IParserBase -{ -protected: - const char * getName() const override { return "INTERSECT or EXCEPT"; } - bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; -}; - -} diff --git a/src/Parsers/ParserSelectWithUnionQuery.cpp b/src/Parsers/ParserSelectWithUnionQuery.cpp index 8c4c183a099..532a9e20735 100644 --- a/src/Parsers/ParserSelectWithUnionQuery.cpp +++ b/src/Parsers/ParserSelectWithUnionQuery.cpp @@ -10,13 +10,7 @@ namespace DB bool ParserSelectWithUnionQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ASTPtr list_node; - - ParserUnionList parser( - std::make_unique(), - std::make_unique("UNION"), - std::make_unique("ALL"), - std::make_unique("DISTINCT"), - std::make_unique("EXCEPT")); + ParserUnionList parser; if (!parser.parse(pos, list_node, expected)) return false; diff --git a/src/Parsers/ParserUnionQueryElement.cpp b/src/Parsers/ParserUnionQueryElement.cpp index 5abbce25930..d59a7be2278 100644 --- a/src/Parsers/ParserUnionQueryElement.cpp +++ b/src/Parsers/ParserUnionQueryElement.cpp @@ -2,7 +2,6 @@ #include #include #include -#include #include @@ -12,7 +11,6 @@ namespace DB bool ParserUnionQueryElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { if (!ParserSubquery().parse(pos, node, expected) - && !ParserSelectIntersectExceptQuery().parse(pos, node, expected) && !ParserSelectQuery().parse(pos, node, expected)) return false; diff --git a/tests/queries/0_stateless/02004_intersect_except_operators.reference b/tests/queries/0_stateless/02004_intersect_except_operators.reference index a097bd0076f..7f41faaf83a 100644 --- a/tests/queries/0_stateless/02004_intersect_except_operators.reference +++ b/tests/queries/0_stateless/02004_intersect_except_operators.reference @@ -99,3 +99,26 @@ select * from (select 1 union all select 2 union all select 3 union all select 4 2 3 5 +select 1 intersect (select 1 except select 2); +1 +select 1 union all select 2 except (select 2 except select 1 union all select 1) except select 4; +explain syntax select 1 intersect select 1; +SELECT 1 +INTERSECT +SELECT 1 +explain syntax select 1 except select 1; +SELECT 1 +EXCEPT +SELECT 1 +explain syntax select 1 union all select 2 except (select 2 except select 1 union all select 1) except select 4; +SELECT 1 +UNION ALL +SELECT 2 +EXCEPT +SELECT 2 +EXCEPT +SELECT 1 +UNION ALL +SELECT 1 +EXCEPT +SELECT 4 diff --git a/tests/queries/0_stateless/02004_intersect_except_operators.sql b/tests/queries/0_stateless/02004_intersect_except_operators.sql index 4602dec7238..ef0e52da116 100644 --- a/tests/queries/0_stateless/02004_intersect_except_operators.sql +++ b/tests/queries/0_stateless/02004_intersect_except_operators.sql @@ -32,3 +32,10 @@ select 1 union all select 1 intersect select 2; select * from (select 1 union all select 2 union all select 3 union all select 4 except select 3 union all select 5) order by 1; select * from (select 1 union all select 2 union all select 3 union all select 4 intersect select 3 union all select 5) order by 1; select * from (select 1 union all select 2 union all select 3 union all select 4 intersect select 3 union all select 5 except select 1) order by 1; + +select 1 intersect (select 1 except select 2); +select 1 union all select 2 except (select 2 except select 1 union all select 1) except select 4; + +explain syntax select 1 intersect select 1; +explain syntax select 1 except select 1; +explain syntax select 1 union all select 2 except (select 2 except select 1 union all select 1) except select 4; From f125fb3fef9c417c82ebf52a14d4aa4aa1b8e88f Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 14 Aug 2021 20:04:21 +0300 Subject: [PATCH 131/161] Some fixes around any/all --- .../AggregateFunctionMinMaxAny.h | 12 +- .../SelectIntersectExceptQueryVisitor.cpp | 4 + src/Parsers/ASTSelectWithUnionQuery.cpp | 4 - src/Parsers/ExpressionListParsers.cpp | 124 ++++++++---------- .../QueryPlan/IntersectOrExceptStep.h | 2 +- .../Transforms/IntersectOrExceptTransform.h | 2 +- .../02007_test_any_all_operators.reference | 22 +++- .../02007_test_any_all_operators.sql | 13 +- 8 files changed, 101 insertions(+), 82 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionMinMaxAny.h b/src/AggregateFunctions/AggregateFunctionMinMaxAny.h index 577b8127fd7..e5471b8a727 100644 --- a/src/AggregateFunctions/AggregateFunctionMinMaxAny.h +++ b/src/AggregateFunctions/AggregateFunctionMinMaxAny.h @@ -50,6 +50,8 @@ private: T value; public: + static constexpr bool is_nullable = false; + bool has() const { return has_value; @@ -470,6 +472,8 @@ private: char small_data[MAX_SMALL_STRING_SIZE]; /// Including the terminating zero. public: + static constexpr bool is_nullable = false; + bool has() const { return size >= 0; @@ -693,6 +697,8 @@ private: Field value; public: + static constexpr bool is_nullable = false; + bool has() const { return !value.isNull(); @@ -979,6 +985,8 @@ struct AggregateFunctionAnyLastData : Data template struct AggregateFunctionSingleValueOrNullData : Data { + static constexpr bool is_nullable = true; + using Self = AggregateFunctionSingleValueOrNullData; bool first_value = true; @@ -1136,7 +1144,9 @@ public: DataTypePtr getReturnType() const override { auto result_type = this->argument_types.at(0); - return Data::name() == "singleValueOrNull" ? makeNullable(result_type) : result_type; + if constexpr (Data::is_nullable) + return makeNullable(result_type); + return result_type; } void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override diff --git a/src/Interpreters/SelectIntersectExceptQueryVisitor.cpp b/src/Interpreters/SelectIntersectExceptQueryVisitor.cpp index 3815fb4ad0f..190ec279038 100644 --- a/src/Interpreters/SelectIntersectExceptQueryVisitor.cpp +++ b/src/Interpreters/SelectIntersectExceptQueryVisitor.cpp @@ -5,6 +5,10 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} /* * Note: there is a difference between intersect and except behaviour. diff --git a/src/Parsers/ASTSelectWithUnionQuery.cpp b/src/Parsers/ASTSelectWithUnionQuery.cpp index b882c738c9a..fa7359574f8 100644 --- a/src/Parsers/ASTSelectWithUnionQuery.cpp +++ b/src/Parsers/ASTSelectWithUnionQuery.cpp @@ -8,10 +8,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} ASTPtr ASTSelectWithUnionQuery::clone() const { diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index ef54c627aab..69d95422799 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -198,91 +198,83 @@ enum class SubqueryFunctionType ALL }; -static bool modifyAST(const String & operator_name, ASTPtr function, SubqueryFunctionType type) +static bool modifyAST(ASTPtr ast, SubqueryFunctionType type) { - // = ANY --> IN, != ALL --> NOT IN - if ((type == SubqueryFunctionType::ANY && operator_name == "equals") - || (type == SubqueryFunctionType::ALL && operator_name == "notEquals")) + /* Rewrite in AST: + * = ANY --> IN + * != ALL --> NOT IN + * = ALL --> IN (SELECT singleValueOrNull(*) FROM subquery) + * != ANY --> NOT IN (SELECT singleValueOrNull(*) FROM subquery) + **/ + + auto * function = assert_cast(ast.get()); + String operator_name = function->name; + + auto function_equals = operator_name == "equals"; + auto function_not_equals = operator_name == "notEquals"; + + String aggregate_function_name; + if (function_equals || function_not_equals) { - assert_cast(function.get())->name = "in"; if (operator_name == "notEquals") + function->name = "notIn"; + else + function->name = "in"; + + if ((type == SubqueryFunctionType::ANY && function_equals) + || (type == SubqueryFunctionType::ALL && function_not_equals)) { - auto function_not = std::make_shared(); - auto exp_list_not = std::make_shared(); - exp_list_not->children.push_back(function); - function_not->name = "not"; - function_not->children.push_back(exp_list_not); - function_not->arguments = exp_list_not; - function = function_not; + return true; } - return true; + + aggregate_function_name = "singleValueOrNull"; } + else if (operator_name == "greaterOrEquals" || operator_name == "greater") + { + aggregate_function_name = (type == SubqueryFunctionType::ANY ? "min" : "max"); + } + else if (operator_name == "lessOrEquals" || operator_name == "less") + { + aggregate_function_name = (type == SubqueryFunctionType::ANY ? "max" : "min"); + } + else + return false; - // subquery --> (SELECT aggregate_function(*) FROM subquery) - auto aggregate_function = std::make_shared(); - auto aggregate_function_exp_list = std::make_shared(); - aggregate_function_exp_list ->children.push_back(std::make_shared()); - aggregate_function->arguments = aggregate_function_exp_list; - aggregate_function->children.push_back(aggregate_function_exp_list); + /// subquery --> (SELECT aggregate_function(*) FROM subquery) + auto aggregate_function = makeASTFunction(aggregate_function_name, std::make_shared()); + auto subquery_node = function->children[0]->children[1]; - ASTPtr subquery_node = function->children[0]->children[1]; - auto select_query = std::make_shared(); - auto tables_in_select = std::make_shared(); - auto tables_in_select_element = std::make_shared(); auto table_expression = std::make_shared(); - table_expression->subquery = subquery_node; - table_expression->children.push_back(subquery_node); - tables_in_select_element->table_expression = table_expression; - tables_in_select_element->children.push_back(table_expression); - tables_in_select->children.push_back(tables_in_select_element); + table_expression->subquery = std::move(subquery_node); + table_expression->children.push_back(table_expression->subquery); + + auto tables_in_select_element = std::make_shared(); + tables_in_select_element->table_expression = std::move(table_expression); + tables_in_select_element->children.push_back(tables_in_select_element->table_expression); + + auto tables_in_select = std::make_shared(); + tables_in_select->children.push_back(std::move(tables_in_select_element)); + auto select_exp_list = std::make_shared(); select_exp_list->children.push_back(aggregate_function); + + auto select_query = std::make_shared(); select_query->children.push_back(select_exp_list); select_query->children.push_back(tables_in_select); - select_query->setExpression(ASTSelectQuery::Expression::SELECT, std::move(select_exp_list)); - select_query->setExpression(ASTSelectQuery::Expression::TABLES, std::move(tables_in_select)); + + select_query->setExpression(ASTSelectQuery::Expression::SELECT, select_exp_list); + select_query->setExpression(ASTSelectQuery::Expression::TABLES, tables_in_select); auto select_with_union_query = std::make_shared(); - auto list_of_selects = std::make_shared(); - list_of_selects->children.push_back(select_query); - select_with_union_query->list_of_selects = list_of_selects; + select_with_union_query->list_of_selects = std::make_shared(); + select_with_union_query->list_of_selects->children.push_back(std::move(select_query)); select_with_union_query->children.push_back(select_with_union_query->list_of_selects); auto new_subquery = std::make_shared(); new_subquery->children.push_back(select_with_union_query); - function->children[0]->children.pop_back(); - function->children[0]->children.push_back(new_subquery); + ast->children[0]->children.back() = std::move(new_subquery); - if (operator_name == "greaterOrEquals" || operator_name == "greater") - { - aggregate_function->name = type == SubqueryFunctionType::ANY ? "min" : "max"; - return true; - } - if (operator_name == "lessOrEquals" || operator_name == "less") - { - aggregate_function->name = type == SubqueryFunctionType::ANY ? "max" : "min"; - return true; - } - - // = ALL --> IN (SELECT singleValueOrNull(*) FROM subquery) - // != ANY --> NOT IN (SELECT singleValueOrNull(*) FROM subquery) - if (operator_name == "equals" || operator_name == "notEquals") - { - aggregate_function->name = "singleValueOrNull"; - assert_cast(function.get())->name = "in"; - if (operator_name == "notEquals") - { - auto function_not = std::make_shared(); - auto exp_list_not = std::make_shared(); - exp_list_not->children.push_back(function); - function_not->name = "not"; - function_not->children.push_back(exp_list_not); - function_not->arguments = exp_list_not; - function = function_not; - } - return true; - } - return false; + return true; } bool ParserComparisonExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) @@ -346,7 +338,7 @@ bool ParserComparisonExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & exp_list->children.push_back(node); exp_list->children.push_back(elem); - if (subquery_function_type != SubqueryFunctionType::NONE && !modifyAST(function->name, function, subquery_function_type)) + if (subquery_function_type != SubqueryFunctionType::NONE && !modifyAST(function, subquery_function_type)) return false; pos.increaseDepth(); diff --git a/src/Processors/QueryPlan/IntersectOrExceptStep.h b/src/Processors/QueryPlan/IntersectOrExceptStep.h index 002f1b1570c..9e87c921ab2 100644 --- a/src/Processors/QueryPlan/IntersectOrExceptStep.h +++ b/src/Processors/QueryPlan/IntersectOrExceptStep.h @@ -12,7 +12,7 @@ using Operator = ASTSelectIntersectExceptQuery::Operator; public: /// max_threads is used to limit the number of threads for result pipeline. - IntersectOrExceptStep(DataStreams input_streams_, Operator operators_, size_t max_threads_ = 0); + IntersectOrExceptStep(DataStreams input_streams_, Operator operator_, size_t max_threads_ = 0); String getName() const override { return "IntersectOrExcept"; } diff --git a/src/Processors/Transforms/IntersectOrExceptTransform.h b/src/Processors/Transforms/IntersectOrExceptTransform.h index da1fa6a119e..e200bfd6cc5 100644 --- a/src/Processors/Transforms/IntersectOrExceptTransform.h +++ b/src/Processors/Transforms/IntersectOrExceptTransform.h @@ -14,7 +14,7 @@ class IntersectOrExceptTransform : public IProcessor using Operator = ASTSelectIntersectExceptQuery::Operator; public: - IntersectOrExceptTransform(const Block & header_, Operator operators); + IntersectOrExceptTransform(const Block & header_, Operator operator_); String getName() const override { return "IntersectOrExcept"; } diff --git a/tests/queries/0_stateless/02007_test_any_all_operators.reference b/tests/queries/0_stateless/02007_test_any_all_operators.reference index cd36102cb80..ebd7cd8f6ca 100644 --- a/tests/queries/0_stateless/02007_test_any_all_operators.reference +++ b/tests/queries/0_stateless/02007_test_any_all_operators.reference @@ -3,17 +3,29 @@ select 1 == any (select number from numbers(10)); 1 select 1 == any (select number from numbers(2, 10)); 0 +select 1 != all (select 1 from numbers(10)); +0 +select 1 != all (select number from numbers(10)); +0 select 1 == all (select 1 from numbers(10)); 1 select 1 == all (select number from numbers(10)); 0 +select 1 != any (select 1 from numbers(10)); +0 +select 1 != any (select number from numbers(10)); +1 select number as a from numbers(10) where a == any (select number from numbers(3, 3)); 3 4 5 --- TODO: Incorrect: -select 1 != any (select 1 from numbers(10)); +select number as a from numbers(10) where a != any (select 5 from numbers(3, 3)); +0 1 -select 1 != all (select 1 from numbers(10)); -1 -select number as a from numbers(10) where a != any (select number from numbers(3, 3)); +2 +3 +4 +6 +7 +8 +9 diff --git a/tests/queries/0_stateless/02007_test_any_all_operators.sql b/tests/queries/0_stateless/02007_test_any_all_operators.sql index 08fc929bab9..525f7e1fabd 100644 --- a/tests/queries/0_stateless/02007_test_any_all_operators.sql +++ b/tests/queries/0_stateless/02007_test_any_all_operators.sql @@ -1,12 +1,17 @@ -- { echo } select 1 == any (select number from numbers(10)); select 1 == any (select number from numbers(2, 10)); + +select 1 != all (select 1 from numbers(10)); +select 1 != all (select number from numbers(10)); + select 1 == all (select 1 from numbers(10)); select 1 == all (select number from numbers(10)); -select number as a from numbers(10) where a == any (select number from numbers(3, 3)); --- TODO: Incorrect: + select 1 != any (select 1 from numbers(10)); -select 1 != all (select 1 from numbers(10)); -select number as a from numbers(10) where a != any (select number from numbers(3, 3)); +select 1 != any (select number from numbers(10)); + +select number as a from numbers(10) where a == any (select number from numbers(3, 3)); +select number as a from numbers(10) where a != any (select 5 from numbers(3, 3)); From e64c9733085f4a9bdbe1eb350709ac1e88b34aae Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 14 Aug 2021 23:34:47 +0300 Subject: [PATCH 132/161] Add a test for #10735 --- .../02009_mysql_client_empty_result.reference | 4 ++++ .../02009_mysql_client_empty_result.sh | 21 +++++++++++++++++++ 2 files changed, 25 insertions(+) create mode 100644 tests/queries/0_stateless/02009_mysql_client_empty_result.reference create mode 100755 tests/queries/0_stateless/02009_mysql_client_empty_result.sh diff --git a/tests/queries/0_stateless/02009_mysql_client_empty_result.reference b/tests/queries/0_stateless/02009_mysql_client_empty_result.reference new file mode 100644 index 00000000000..f11935a87d3 --- /dev/null +++ b/tests/queries/0_stateless/02009_mysql_client_empty_result.reference @@ -0,0 +1,4 @@ +x +1 +i +3 diff --git a/tests/queries/0_stateless/02009_mysql_client_empty_result.sh b/tests/queries/0_stateless/02009_mysql_client_empty_result.sh new file mode 100755 index 00000000000..fde2f57b4ee --- /dev/null +++ b/tests/queries/0_stateless/02009_mysql_client_empty_result.sh @@ -0,0 +1,21 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +${MYSQL_CLIENT} --batch --execute 'SELECT 1 AS x' +${MYSQL_CLIENT} --batch --execute 'SELECT 1 AS x WHERE 0' + +${MYSQL_CLIENT} <<<" + DROP TABLE IF EXISTS b; + CREATE TABLE b (i UInt8) ENGINE=MergeTree() PRIMARY KEY(i) ORDER BY (i); + INSERT INTO b VALUES (1), (2), (3); +" + +${MYSQL_CLIENT} --batch --execute 'SELECT * FROM default.b WHERE i>=3;' +${MYSQL_CLIENT} --batch --execute 'SELECT * FROM default.b WHERE i>=300;' + +${MYSQL_CLIENT} <<<" + DROP TABLE b; +" From dafb75c6041dadc1f1e0e8a1882fa270e351426a Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 15 Aug 2021 03:31:39 +0300 Subject: [PATCH 133/161] Update 02009_mysql_client_empty_result.sh --- tests/queries/0_stateless/02009_mysql_client_empty_result.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02009_mysql_client_empty_result.sh b/tests/queries/0_stateless/02009_mysql_client_empty_result.sh index fde2f57b4ee..cc1d0aece77 100755 --- a/tests/queries/0_stateless/02009_mysql_client_empty_result.sh +++ b/tests/queries/0_stateless/02009_mysql_client_empty_result.sh @@ -13,8 +13,8 @@ ${MYSQL_CLIENT} <<<" INSERT INTO b VALUES (1), (2), (3); " -${MYSQL_CLIENT} --batch --execute 'SELECT * FROM default.b WHERE i>=3;' -${MYSQL_CLIENT} --batch --execute 'SELECT * FROM default.b WHERE i>=300;' +${MYSQL_CLIENT} --batch --execute 'SELECT * FROM b WHERE i>=3;' +${MYSQL_CLIENT} --batch --execute 'SELECT * FROM b WHERE i>=300;' ${MYSQL_CLIENT} <<<" DROP TABLE b; From ef803f14a1ed067b15c5671f3761d86cf5c5e438 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 15 Aug 2021 05:11:30 +0300 Subject: [PATCH 134/161] Update 85_bug-report.md --- .github/ISSUE_TEMPLATE/85_bug-report.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/ISSUE_TEMPLATE/85_bug-report.md b/.github/ISSUE_TEMPLATE/85_bug-report.md index bd59d17db3f..d78474670ff 100644 --- a/.github/ISSUE_TEMPLATE/85_bug-report.md +++ b/.github/ISSUE_TEMPLATE/85_bug-report.md @@ -2,7 +2,7 @@ name: Bug report about: Wrong behaviour (visible to users) in official ClickHouse release. title: '' -labels: bug +labels: 'potential bug' assignees: '' --- From ab4415a22b17f21517351412652800590a1ff594 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 15 Aug 2021 05:28:38 +0300 Subject: [PATCH 135/161] Update MergeTreeData.cpp --- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index ebd961ab428..2892efab12d 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3227,7 +3227,7 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr loc return partition_ast.id; } - if (const auto * partition_function = partition_ast.value->as()) + if (partition_ast.value->as()) checkPartitionExpressionFunction(ast); if (format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) From f063e44131a048ba2d9af8075f03700fd5ec3e69 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 15 Aug 2021 09:00:28 +0300 Subject: [PATCH 136/161] Whitespaces --- base/mysqlxx/Pool.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/base/mysqlxx/Pool.cpp b/base/mysqlxx/Pool.cpp index 386b4544b78..2f47aa67356 100644 --- a/base/mysqlxx/Pool.cpp +++ b/base/mysqlxx/Pool.cpp @@ -296,7 +296,7 @@ void Pool::initialize() Pool::Connection * Pool::allocConnection(bool dont_throw_if_failed_first_time) { - std::unique_ptr conn_ptr{new Connection}; + std::unique_ptr conn_ptr = std::make_unique(); try { From cf87dacd4e6630a6f6a33afc8c66d409751836f7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 15 Aug 2021 09:01:04 +0300 Subject: [PATCH 137/161] Disable memory tracking for roaring bitmaps on Mac OS --- contrib/croaring-cmake/CMakeLists.txt | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/contrib/croaring-cmake/CMakeLists.txt b/contrib/croaring-cmake/CMakeLists.txt index 84cdccedbd3..5223027843d 100644 --- a/contrib/croaring-cmake/CMakeLists.txt +++ b/contrib/croaring-cmake/CMakeLists.txt @@ -27,8 +27,10 @@ target_include_directories(roaring SYSTEM BEFORE PUBLIC "${LIBRARY_DIR}/cpp") # We redirect malloc/free family of functions to different functions that will track memory in ClickHouse. # Also note that we exploit implicit function declarations. +# Also it is disabled on Mac OS because it fails). -target_compile_definitions(roaring PRIVATE +if (NOT OS_DARWIN) + target_compile_definitions(roaring PRIVATE -Dmalloc=clickhouse_malloc -Dcalloc=clickhouse_calloc -Drealloc=clickhouse_realloc @@ -36,4 +38,5 @@ target_compile_definitions(roaring PRIVATE -Dfree=clickhouse_free -Dposix_memalign=clickhouse_posix_memalign) -target_link_libraries(roaring PUBLIC clickhouse_common_io) + target_link_libraries(roaring PUBLIC clickhouse_common_io) +endif () From 15eb68d117d89128ee91fe7494f456e72aaa5479 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 15 Aug 2021 09:33:08 +0300 Subject: [PATCH 138/161] Fix bad cast in arrayIndex #26330 --- src/Functions/array/arrayIndex.h | 7 +++++++ .../0_stateless/02010_array_index_bad_cast.reference | 0 tests/queries/0_stateless/02010_array_index_bad_cast.sql | 2 ++ 3 files changed, 9 insertions(+) create mode 100644 tests/queries/0_stateless/02010_array_index_bad_cast.reference create mode 100644 tests/queries/0_stateless/02010_array_index_bad_cast.sql diff --git a/src/Functions/array/arrayIndex.h b/src/Functions/array/arrayIndex.h index a390abc4eaf..d7bbcaf8d46 100644 --- a/src/Functions/array/arrayIndex.h +++ b/src/Functions/array/arrayIndex.h @@ -115,6 +115,13 @@ private: [[maybe_unused]] const NullMap * const null_map_data, [[maybe_unused]] const NullMap * const null_map_item) { + if constexpr (std::is_same_v && std::is_same_v) + { + /// Generic variant is using IColumn::compare function that only allows to compare columns of identical types. + if (typeid(data) != typeid(target)) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Columns {} and {} cannot be compared", data.getName(), target.getName()); + } + const size_t size = offsets.size(); result.resize(size); diff --git a/tests/queries/0_stateless/02010_array_index_bad_cast.reference b/tests/queries/0_stateless/02010_array_index_bad_cast.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02010_array_index_bad_cast.sql b/tests/queries/0_stateless/02010_array_index_bad_cast.sql new file mode 100644 index 00000000000..19c58bb28a7 --- /dev/null +++ b/tests/queries/0_stateless/02010_array_index_bad_cast.sql @@ -0,0 +1,2 @@ +-- This query throws exception about uncomparable data types (but at least it does not introduce bad cast in code). +SELECT has(materialize(CAST(['2021-07-14'] AS Array(LowCardinality(Nullable(DateTime))))), materialize('2021-07-14'::DateTime64(7))); -- { serverError 44 } From 6ff43a614c6ab313f5d87a395a436bba52899bf2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 15 Aug 2021 09:52:55 +0300 Subject: [PATCH 139/161] Use only SSE2 in "unbundled" build --- CMakeLists.txt | 5 +++-- docker/packager/packager | 8 +++++++- release | 3 --- 3 files changed, 10 insertions(+), 6 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 1727caea766..35c22526816 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -395,9 +395,10 @@ endif () # Turns on all external libs like s3, kafka, ODBC, ... option(ENABLE_LIBRARIES "Enable all external libraries by default" ON) -# We recommend avoiding this mode for production builds because we can't guarantee all needed libraries exist in your -# system. +# We recommend avoiding this mode for production builds because we can't guarantee +# all needed libraries exist in your system. # This mode exists for enthusiastic developers who are searching for trouble. +# The whole idea of using unknown version of libraries from the OS distribution is deeply flawed. # Useful for maintainers of OS packages. option (UNBUNDLED "Use system libraries instead of ones in contrib/" OFF) diff --git a/docker/packager/packager b/docker/packager/packager index c05c85d3e28..857df079281 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -151,8 +151,14 @@ def parse_env_variables(build_type, compiler, sanitizer, package_type, image_typ cmake_flags.append('-DENABLE_TESTS=1') cmake_flags.append('-DUSE_GTEST=1') + # "Unbundled" build is not suitable for any production usage. + # But it is occasionally used by some developers. + # The whole idea of using unknown version of libraries from the OS distribution is deeply flawed. + # We wish these developers good luck. if unbundled: - cmake_flags.append('-DUNBUNDLED=1 -DUSE_INTERNAL_RDKAFKA_LIBRARY=1 -DENABLE_ARROW=0 -DENABLE_AVRO=0 -DENABLE_ORC=0 -DENABLE_PARQUET=0') + # We also disable all CPU features except basic x86_64. + # It is only slightly related to "unbundled" build, but it is a good place to test if code compiled without these instruction sets. + cmake_flags.append('-DUNBUNDLED=1 -DUSE_INTERNAL_RDKAFKA_LIBRARY=1 -DENABLE_ARROW=0 -DENABLE_AVRO=0 -DENABLE_ORC=0 -DENABLE_PARQUET=0 -DENABLE_SSSE3=0 -DENABLE_SSE41=0 -DENABLE_SSE42=0 -DENABLE_PCLMULQDQ=0 -DENABLE_POPCNT=0 -DENABLE_AVX=0 -DENABLE_AVX2=0') if split_binary: cmake_flags.append('-DUSE_STATIC_LIBRARIES=0 -DSPLIT_SHARED_LIBRARIES=1 -DCLICKHOUSE_SPLIT_BINARY=1') diff --git a/release b/release index 9484d79630a..de549595d43 100755 --- a/release +++ b/release @@ -71,9 +71,6 @@ then export DEB_CC=${DEB_CC=clang-10} export DEB_CXX=${DEB_CXX=clang++-10} EXTRAPACKAGES="$EXTRAPACKAGES clang-10 lld-10" -elif [[ $BUILD_TYPE == 'valgrind' ]]; then - MALLOC_OPTS="-DENABLE_TCMALLOC=0 -DENABLE_JEMALLOC=0" - VERSION_POSTFIX+="+valgrind" elif [[ $BUILD_TYPE == 'debug' ]]; then CMAKE_BUILD_TYPE=Debug VERSION_POSTFIX+="+debug" From c06f212bb5d4e8058334dc3b34fefea4174591e5 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 15 Aug 2021 09:55:54 +0300 Subject: [PATCH 140/161] Update packager --- docker/packager/packager | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/packager/packager b/docker/packager/packager index 857df079281..95b7fcd8568 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -157,7 +157,7 @@ def parse_env_variables(build_type, compiler, sanitizer, package_type, image_typ # We wish these developers good luck. if unbundled: # We also disable all CPU features except basic x86_64. - # It is only slightly related to "unbundled" build, but it is a good place to test if code compiled without these instruction sets. + # It is only slightly related to "unbundled" build, but it is a good place to test if code compiles without these instruction sets. cmake_flags.append('-DUNBUNDLED=1 -DUSE_INTERNAL_RDKAFKA_LIBRARY=1 -DENABLE_ARROW=0 -DENABLE_AVRO=0 -DENABLE_ORC=0 -DENABLE_PARQUET=0 -DENABLE_SSSE3=0 -DENABLE_SSE41=0 -DENABLE_SSE42=0 -DENABLE_PCLMULQDQ=0 -DENABLE_POPCNT=0 -DENABLE_AVX=0 -DENABLE_AVX2=0') if split_binary: From ba0ba988bafa027b08541dd9c34286fecc53518b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 15 Aug 2021 11:21:30 +0300 Subject: [PATCH 141/161] Fix build --- contrib/libmetrohash/CMakeLists.txt | 4 ---- 1 file changed, 4 deletions(-) diff --git a/contrib/libmetrohash/CMakeLists.txt b/contrib/libmetrohash/CMakeLists.txt index 9304cb3644c..4ec5a58717d 100644 --- a/contrib/libmetrohash/CMakeLists.txt +++ b/contrib/libmetrohash/CMakeLists.txt @@ -2,9 +2,5 @@ set (SRCS src/metrohash64.cpp src/metrohash128.cpp ) -if (HAVE_SSE42) # Not used. Pretty easy to port. - list (APPEND SRCS src/metrohash128crc.cpp) -endif () - add_library(metrohash ${SRCS}) target_include_directories(metrohash PUBLIC src) From 97d921d4da75105094e869ed7b25730946569eeb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 15 Aug 2021 11:38:16 +0300 Subject: [PATCH 142/161] Remove trash --- src/Compression/CompressedWriteBuffer.cpp | 4 ---- 1 file changed, 4 deletions(-) diff --git a/src/Compression/CompressedWriteBuffer.cpp b/src/Compression/CompressedWriteBuffer.cpp index 8d146e8de23..1c15dd77f14 100644 --- a/src/Compression/CompressedWriteBuffer.cpp +++ b/src/Compression/CompressedWriteBuffer.cpp @@ -30,10 +30,6 @@ void CompressedWriteBuffer::nextImpl() compressed_buffer.resize(compressed_reserve_size); UInt32 compressed_size = codec->compress(working_buffer.begin(), decompressed_size, compressed_buffer.data()); - // FIXME remove this after fixing msan report in lz4. - // Almost always reproduces on stateless tests, the exact test unknown. - __msan_unpoison(compressed_buffer.data(), compressed_size); - CityHash_v1_0_2::uint128 checksum = CityHash_v1_0_2::CityHash128(compressed_buffer.data(), compressed_size); out.write(reinterpret_cast(&checksum), CHECKSUM_SIZE); out.write(compressed_buffer.data(), compressed_size); From f66e8464f94b41624099d2a71151a52778ed74d8 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 15 Aug 2021 09:55:43 +0300 Subject: [PATCH 143/161] Some final fixes --- .../InterpreterSelectIntersectExceptQuery.cpp | 7 +- .../InterpreterSelectIntersectExceptQuery.h | 5 +- .../SelectIntersectExceptQueryVisitor.cpp | 6 +- src/Parsers/ASTSelectIntersectExceptQuery.cpp | 5 +- src/Parsers/ExpressionListParsers.cpp | 86 +++---------------- src/Parsers/ExpressionListParsers.h | 17 +++- src/Parsers/ParserQueryWithOutput.cpp | 1 + src/Parsers/ParserUnionQueryElement.cpp | 3 +- .../QueryPlan/IntersectOrExceptStep.cpp | 6 +- .../Transforms/IntersectOrExceptTransform.cpp | 1 + ...02004_intersect_except_operators.reference | 5 ++ .../02004_intersect_except_operators.sql | 4 + .../02007_test_any_all_operators.reference | 20 +++++ .../02007_test_any_all_operators.sql | 11 ++- 14 files changed, 83 insertions(+), 94 deletions(-) diff --git a/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp b/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp index 4edd13d08e5..9c8dda56b44 100644 --- a/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp +++ b/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp @@ -89,7 +89,6 @@ InterpreterSelectIntersectExceptQuery::buildCurrentChildInterpreter(const ASTPtr if (ast_ptr_->as()) return std::make_unique(ast_ptr_, context, SelectQueryOptions()); - // if (ast_ptr_->as()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected query: {}", ast_ptr_->getID()); } @@ -140,4 +139,10 @@ BlockIO InterpreterSelectIntersectExceptQuery::execute() return res; } +void InterpreterSelectIntersectExceptQuery::ignoreWithTotals() +{ + for (auto & interpreter : nested_interpreters) + interpreter->ignoreWithTotals(); +} + } diff --git a/src/Interpreters/InterpreterSelectIntersectExceptQuery.h b/src/Interpreters/InterpreterSelectIntersectExceptQuery.h index 9cbde055b0b..805565e4c51 100644 --- a/src/Interpreters/InterpreterSelectIntersectExceptQuery.h +++ b/src/Interpreters/InterpreterSelectIntersectExceptQuery.h @@ -28,6 +28,8 @@ public: Block getSampleBlock() { return result_header; } + void ignoreWithTotals() override; + private: static String getName() { return "SelectIntersectExceptQuery"; } @@ -36,9 +38,8 @@ private: void buildQueryPlan(QueryPlan & query_plan) override; - void ignoreWithTotals() override {} - std::vector> nested_interpreters; + Operator final_operator; }; diff --git a/src/Interpreters/SelectIntersectExceptQueryVisitor.cpp b/src/Interpreters/SelectIntersectExceptQueryVisitor.cpp index 190ec279038..e26c4371591 100644 --- a/src/Interpreters/SelectIntersectExceptQueryVisitor.cpp +++ b/src/Interpreters/SelectIntersectExceptQueryVisitor.cpp @@ -12,11 +12,11 @@ namespace ErrorCodes /* * Note: there is a difference between intersect and except behaviour. - * `intersect` is supposed to be a part of last SelectQuery, i.e. the sequence with no parenthesis: + * `intersect` is supposed to be a part of the last SelectQuery, i.e. the sequence with no parenthesis: * select 1 union all select 2 except select 1 intersect 2 except select 2 union distinct select 5; * is interpreted as: * select 1 union all select 2 except (select 1 intersect 2) except select 2 union distinct select 5; - * Whereas `except` is applied to all union part like: + * Whereas `except` is applied to all left union part like: * (((select 1 union all select 2) except (select 1 intersect 2)) except select 2) union distinct select 5; **/ @@ -28,7 +28,7 @@ void SelectIntersectExceptQueryMatcher::visit(ASTPtr & ast, Data & data) void SelectIntersectExceptQueryMatcher::visit(ASTSelectWithUnionQuery & ast, Data &) { - auto & union_modes = ast.list_of_modes; + const auto & union_modes = ast.list_of_modes; if (union_modes.empty()) return; diff --git a/src/Parsers/ASTSelectIntersectExceptQuery.cpp b/src/Parsers/ASTSelectIntersectExceptQuery.cpp index 9d7a717fa6c..3b9cb0a2c16 100644 --- a/src/Parsers/ASTSelectIntersectExceptQuery.cpp +++ b/src/Parsers/ASTSelectIntersectExceptQuery.cpp @@ -30,11 +30,10 @@ void ASTSelectIntersectExceptQuery::formatQueryImpl(const FormatSettings & setti { settings.ostr << settings.nl_or_ws << indent_str << (settings.hilite ? hilite_keyword : "") << (final_operator == Operator::INTERSECT ? "INTERSECT" : "EXCEPT") - << (settings.hilite ? hilite_none : ""); + << (settings.hilite ? hilite_none : "") + << settings.nl_or_ws; } - if (it != children.begin()) - settings.ostr << settings.nl_or_ws; (*it)->formatImpl(settings, state, frame); } } diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index 69d95422799..58f5e766905 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -277,79 +277,6 @@ static bool modifyAST(ASTPtr ast, SubqueryFunctionType type) return true; } -bool ParserComparisonExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) -{ - bool first = true; - - auto current_depth = pos.depth; - while (true) - { - if (first) - { - ASTPtr elem; - if (!elem_parser.parse(pos, elem, expected)) - return false; - - node = elem; - first = false; - } - else - { - /// try to find any of the valid operators - const char ** it; - Expected stub; - for (it = overlapping_operators_to_skip; *it; ++it) - if (ParserKeyword{*it}.checkWithoutMoving(pos, stub)) - break; - - if (*it) - break; - - for (it = operators; *it; it += 2) - if (parseOperator(pos, *it, expected)) - break; - - if (!*it) - break; - - /// the function corresponding to the operator - auto function = std::make_shared(); - - /// function arguments - auto exp_list = std::make_shared(); - - ASTPtr elem; - SubqueryFunctionType subquery_function_type = SubqueryFunctionType::NONE; - if (ParserKeyword("ANY").ignore(pos, expected)) - subquery_function_type = SubqueryFunctionType::ANY; - else if (ParserKeyword("ALL").ignore(pos, expected)) - subquery_function_type = SubqueryFunctionType::ALL; - else if (!elem_parser.parse(pos, elem, expected)) - return false; - - if (subquery_function_type != SubqueryFunctionType::NONE && !ParserSubquery().parse(pos, elem, expected)) - return false; - - /// the first argument of the function is the previous element, the second is the next one - function->name = it[1]; - function->arguments = exp_list; - function->children.push_back(exp_list); - - exp_list->children.push_back(node); - exp_list->children.push_back(elem); - - if (subquery_function_type != SubqueryFunctionType::NONE && !modifyAST(function, subquery_function_type)) - return false; - - pos.increaseDepth(); - node = function; - } - } - - pos.depth = current_depth; - return true; -} - bool ParserLeftAssociativeBinaryOperatorList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { bool first = true; @@ -393,7 +320,15 @@ bool ParserLeftAssociativeBinaryOperatorList::parseImpl(Pos & pos, ASTPtr & node auto exp_list = std::make_shared(); ASTPtr elem; - if (!(remaining_elem_parser ? remaining_elem_parser : first_elem_parser)->parse(pos, elem, expected)) + SubqueryFunctionType subquery_function_type = SubqueryFunctionType::NONE; + if (allow_any_all_operators && ParserKeyword("ANY").ignore(pos, expected)) + subquery_function_type = SubqueryFunctionType::ANY; + else if (allow_any_all_operators && ParserKeyword("ALL").ignore(pos, expected)) + subquery_function_type = SubqueryFunctionType::ALL; + else if (!(remaining_elem_parser ? remaining_elem_parser : first_elem_parser)->parse(pos, elem, expected)) + return false; + + if (subquery_function_type != SubqueryFunctionType::NONE && !ParserSubquery().parse(pos, elem, expected)) return false; /// the first argument of the function is the previous element, the second is the next one @@ -404,6 +339,9 @@ bool ParserLeftAssociativeBinaryOperatorList::parseImpl(Pos & pos, ASTPtr & node exp_list->children.push_back(node); exp_list->children.push_back(elem); + if (allow_any_all_operators && subquery_function_type != SubqueryFunctionType::NONE && !modifyAST(function, subquery_function_type)) + return false; + /** special exception for the access operator to the element of the array `x[y]`, which * contains the infix part '[' and the suffix ''] '(specified as' [') */ diff --git a/src/Parsers/ExpressionListParsers.h b/src/Parsers/ExpressionListParsers.h index e44cacb313f..17deec4e9e4 100644 --- a/src/Parsers/ExpressionListParsers.h +++ b/src/Parsers/ExpressionListParsers.h @@ -121,6 +121,8 @@ private: Operators_t overlapping_operators_to_skip = { (const char *[]){ nullptr } }; ParserPtr first_elem_parser; ParserPtr remaining_elem_parser; + /// =, !=, <, > ALL (subquery) / ANY (subquery) + bool allow_any_all_operators = false; public: /** `operators_` - allowed operators and their corresponding functions @@ -130,8 +132,10 @@ public: { } - ParserLeftAssociativeBinaryOperatorList(Operators_t operators_, Operators_t overlapping_operators_to_skip_, ParserPtr && first_elem_parser_) - : operators(operators_), overlapping_operators_to_skip(overlapping_operators_to_skip_), first_elem_parser(std::move(first_elem_parser_)) + ParserLeftAssociativeBinaryOperatorList(Operators_t operators_, + Operators_t overlapping_operators_to_skip_, ParserPtr && first_elem_parser_, bool allow_any_all_operators_ = false) + : operators(operators_), overlapping_operators_to_skip(overlapping_operators_to_skip_), + first_elem_parser(std::move(first_elem_parser_)), allow_any_all_operators(allow_any_all_operators_) { } @@ -341,12 +345,16 @@ class ParserComparisonExpression : public IParserBase private: static const char * operators[]; static const char * overlapping_operators_to_skip[]; - ParserBetweenExpression elem_parser; + ParserLeftAssociativeBinaryOperatorList operator_parser {operators, + overlapping_operators_to_skip, std::make_unique(), true}; protected: const char * getName() const override{ return "comparison expression"; } - bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override + { + return operator_parser.parse(pos, node, expected); + } }; /** Parser for nullity checking with IS (NOT) NULL. @@ -355,6 +363,7 @@ class ParserNullityChecking : public IParserBase { private: ParserComparisonExpression elem_parser; + protected: const char * getName() const override { return "nullity checking"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; diff --git a/src/Parsers/ParserQueryWithOutput.cpp b/src/Parsers/ParserQueryWithOutput.cpp index 4a73952674c..82f9f561187 100644 --- a/src/Parsers/ParserQueryWithOutput.cpp +++ b/src/Parsers/ParserQueryWithOutput.cpp @@ -24,6 +24,7 @@ #include #include + namespace DB { diff --git a/src/Parsers/ParserUnionQueryElement.cpp b/src/Parsers/ParserUnionQueryElement.cpp index d59a7be2278..efd022e6362 100644 --- a/src/Parsers/ParserUnionQueryElement.cpp +++ b/src/Parsers/ParserUnionQueryElement.cpp @@ -10,8 +10,7 @@ namespace DB bool ParserUnionQueryElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { - if (!ParserSubquery().parse(pos, node, expected) - && !ParserSelectQuery().parse(pos, node, expected)) + if (!ParserSubquery().parse(pos, node, expected) && !ParserSelectQuery().parse(pos, node, expected)) return false; if (const auto * ast_subquery = node->as()) diff --git a/src/Processors/QueryPlan/IntersectOrExceptStep.cpp b/src/Processors/QueryPlan/IntersectOrExceptStep.cpp index b75898b815b..d1bb1eb41e9 100644 --- a/src/Processors/QueryPlan/IntersectOrExceptStep.cpp +++ b/src/Processors/QueryPlan/IntersectOrExceptStep.cpp @@ -36,10 +36,7 @@ IntersectOrExceptStep::IntersectOrExceptStep( , max_threads(max_threads_) { input_streams = std::move(input_streams_); - if (input_streams.size() == 1) - output_stream = input_streams.front(); - else - output_stream = DataStream{.header = header}; + output_stream = DataStream{.header = header}; } QueryPipelinePtr IntersectOrExceptStep::updatePipeline(QueryPipelines pipelines, const BuildQueryPipelineSettings &) @@ -71,6 +68,7 @@ QueryPipelinePtr IntersectOrExceptStep::updatePipeline(QueryPipelines pipelines, }); } + /// For the case of union. cur_pipeline->addTransform(std::make_shared(header, cur_pipeline->getNumStreams(), 1)); } diff --git a/src/Processors/Transforms/IntersectOrExceptTransform.cpp b/src/Processors/Transforms/IntersectOrExceptTransform.cpp index abfd1a7f0ad..3e39123ae4b 100644 --- a/src/Processors/Transforms/IntersectOrExceptTransform.cpp +++ b/src/Processors/Transforms/IntersectOrExceptTransform.cpp @@ -4,6 +4,7 @@ namespace DB { +/// After visitor is applied, ASTSelectIntersectExcept always has two child nodes. IntersectOrExceptTransform::IntersectOrExceptTransform(const Block & header_, Operator operator_) : IProcessor(InputPorts(2, header_), {header_}) , current_operator(operator_) diff --git a/tests/queries/0_stateless/02004_intersect_except_operators.reference b/tests/queries/0_stateless/02004_intersect_except_operators.reference index 7f41faaf83a..03b881f690b 100644 --- a/tests/queries/0_stateless/02004_intersect_except_operators.reference +++ b/tests/queries/0_stateless/02004_intersect_except_operators.reference @@ -70,6 +70,10 @@ select * from (select 1 intersect select 1); 1 with (select number from numbers(10) intersect select 5) as a select a * 10; 50 +with (select 5 except select 1) as a select a except select 5; +with (select number from numbers(10) intersect select 5) as a select a intersect select 1; +with (select number from numbers(10) intersect select 5) as a select a except select 1; +5 select count() from (select number from numbers(10) except select 5); 9 select count() from (select number from numbers(1000000) intersect select number from numbers(200000, 600000)); @@ -102,6 +106,7 @@ select * from (select 1 union all select 2 union all select 3 union all select 4 select 1 intersect (select 1 except select 2); 1 select 1 union all select 2 except (select 2 except select 1 union all select 1) except select 4; +select 1 intersect select count() from (select 1 except select 2 intersect select 2 union all select 1); explain syntax select 1 intersect select 1; SELECT 1 INTERSECT diff --git a/tests/queries/0_stateless/02004_intersect_except_operators.sql b/tests/queries/0_stateless/02004_intersect_except_operators.sql index ef0e52da116..7f08cc0adf2 100644 --- a/tests/queries/0_stateless/02004_intersect_except_operators.sql +++ b/tests/queries/0_stateless/02004_intersect_except_operators.sql @@ -21,6 +21,9 @@ select number from numbers(100) intersect select number from numbers(20, 60) exc select * from (select 1 intersect select 1); with (select number from numbers(10) intersect select 5) as a select a * 10; +with (select 5 except select 1) as a select a except select 5; +with (select number from numbers(10) intersect select 5) as a select a intersect select 1; +with (select number from numbers(10) intersect select 5) as a select a except select 1; select count() from (select number from numbers(10) except select 5); select count() from (select number from numbers(1000000) intersect select number from numbers(200000, 600000)); select count() from (select number from numbers(100) intersect select number from numbers(20, 60) except select number from numbers(30, 20) except select number from numbers(60, 20)); @@ -35,6 +38,7 @@ select * from (select 1 union all select 2 union all select 3 union all select 4 select 1 intersect (select 1 except select 2); select 1 union all select 2 except (select 2 except select 1 union all select 1) except select 4; +select 1 intersect select count() from (select 1 except select 2 intersect select 2 union all select 1); explain syntax select 1 intersect select 1; explain syntax select 1 except select 1; diff --git a/tests/queries/0_stateless/02007_test_any_all_operators.reference b/tests/queries/0_stateless/02007_test_any_all_operators.reference index ebd7cd8f6ca..a232320d15c 100644 --- a/tests/queries/0_stateless/02007_test_any_all_operators.reference +++ b/tests/queries/0_stateless/02007_test_any_all_operators.reference @@ -29,3 +29,23 @@ select number as a from numbers(10) where a != any (select 5 from numbers(3, 3)) 7 8 9 +select 1 < any (select 1 from numbers(10)); +0 +select 1 <= any (select 1 from numbers(10)); +1 +select 1 < any (select number from numbers(10)); +1 +select 1 > any (select number from numbers(10)); +1 +select 1 >= any (select number from numbers(10)); +1 +select 11 > all (select number from numbers(10)); +1 +select 11 <= all (select number from numbers(11)); +0 +select 11 < all (select 11 from numbers(10)); +0 +select 11 > all (select 11 from numbers(10)); +0 +select 11 >= all (select 11 from numbers(10)); +1 diff --git a/tests/queries/0_stateless/02007_test_any_all_operators.sql b/tests/queries/0_stateless/02007_test_any_all_operators.sql index 525f7e1fabd..10d7325afca 100644 --- a/tests/queries/0_stateless/02007_test_any_all_operators.sql +++ b/tests/queries/0_stateless/02007_test_any_all_operators.sql @@ -8,10 +8,19 @@ select 1 != all (select number from numbers(10)); select 1 == all (select 1 from numbers(10)); select 1 == all (select number from numbers(10)); - select 1 != any (select 1 from numbers(10)); select 1 != any (select number from numbers(10)); select number as a from numbers(10) where a == any (select number from numbers(3, 3)); select number as a from numbers(10) where a != any (select 5 from numbers(3, 3)); +select 1 < any (select 1 from numbers(10)); +select 1 <= any (select 1 from numbers(10)); +select 1 < any (select number from numbers(10)); +select 1 > any (select number from numbers(10)); +select 1 >= any (select number from numbers(10)); +select 11 > all (select number from numbers(10)); +select 11 <= all (select number from numbers(11)); +select 11 < all (select 11 from numbers(10)); +select 11 > all (select 11 from numbers(10)); +select 11 >= all (select 11 from numbers(10)); From 9ae92798d109e58e95c3a50d3e50227624d98b9a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 15 Aug 2021 11:44:12 +0300 Subject: [PATCH 144/161] Fix stress test in ~CompressedWriteBuffer --- src/Compression/CompressedWriteBuffer.cpp | 7 +++++++ src/Compression/CompressedWriteBuffer.h | 1 + 2 files changed, 8 insertions(+) diff --git a/src/Compression/CompressedWriteBuffer.cpp b/src/Compression/CompressedWriteBuffer.cpp index 1c15dd77f14..7454e09fda0 100644 --- a/src/Compression/CompressedWriteBuffer.cpp +++ b/src/Compression/CompressedWriteBuffer.cpp @@ -36,6 +36,12 @@ void CompressedWriteBuffer::nextImpl() } +void CompressedWriteBuffer::finalize() +{ + next(); +} + + CompressedWriteBuffer::CompressedWriteBuffer( WriteBuffer & out_, CompressionCodecPtr codec_, @@ -44,6 +50,7 @@ CompressedWriteBuffer::CompressedWriteBuffer( { } + CompressedWriteBuffer::~CompressedWriteBuffer() { /// FIXME move final flush into the caller diff --git a/src/Compression/CompressedWriteBuffer.h b/src/Compression/CompressedWriteBuffer.h index a9612b463a5..2268b7bec50 100644 --- a/src/Compression/CompressedWriteBuffer.h +++ b/src/Compression/CompressedWriteBuffer.h @@ -22,6 +22,7 @@ private: PODArray compressed_buffer; void nextImpl() override; + void finalize() override; public: CompressedWriteBuffer( From e28c9c0ba7cea51ffe5b2d1c41d13bc0832270d1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 15 Aug 2021 11:54:28 +0300 Subject: [PATCH 145/161] Mark tests for DatabaseReplicated as green --- docker/test/stateless/process_functional_tests_result.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docker/test/stateless/process_functional_tests_result.py b/docker/test/stateless/process_functional_tests_result.py index e60424ad4d1..a42b0e68d88 100755 --- a/docker/test/stateless/process_functional_tests_result.py +++ b/docker/test/stateless/process_functional_tests_result.py @@ -105,6 +105,10 @@ def process_result(result_path): description += ", skipped: {}".format(skipped) if unknown != 0: description += ", unknown: {}".format(unknown) + + # Temporary green for tests with DatabaseReplicated: + if 1 == int(os.environ.get('USE_DATABASE_REPLICATED', 0)): + state = "success" else: state = "failure" description = "Output log doesn't exist" From 86694a2bbbb049f74e80608527a39fc1fde9aa22 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 15 Aug 2021 12:04:38 +0300 Subject: [PATCH 146/161] Update CompressedWriteBuffer.h --- src/Compression/CompressedWriteBuffer.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Compression/CompressedWriteBuffer.h b/src/Compression/CompressedWriteBuffer.h index 2268b7bec50..57ba679855e 100644 --- a/src/Compression/CompressedWriteBuffer.h +++ b/src/Compression/CompressedWriteBuffer.h @@ -22,7 +22,6 @@ private: PODArray compressed_buffer; void nextImpl() override; - void finalize() override; public: CompressedWriteBuffer( @@ -30,6 +29,8 @@ public: CompressionCodecPtr codec_ = CompressionCodecFactory::instance().getDefaultCodec(), size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE); + void finalize() override; + /// The amount of compressed data size_t getCompressedBytes() { From 5f5470c2cdefdf57e6de9bd49b7312f7bc2a3e50 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sun, 15 Aug 2021 13:32:56 +0300 Subject: [PATCH 147/161] Removed DenseHashMap, DenseHashSet --- src/Common/DenseHashMap.h | 29 ------ src/Common/DenseHashSet.h | 25 ----- src/Common/SparseHashMap.h | 1 - src/Core/NamesAndTypes.cpp | 12 +-- src/Storages/MergeTree/IMergeTreeReader.cpp | 9 +- src/Storages/MergeTree/IMergeTreeReader.h | 5 +- src/Storages/StorageInMemoryMetadata.cpp | 102 +++++++++++--------- 7 files changed, 72 insertions(+), 111 deletions(-) delete mode 100644 src/Common/DenseHashMap.h delete mode 100644 src/Common/DenseHashSet.h diff --git a/src/Common/DenseHashMap.h b/src/Common/DenseHashMap.h deleted file mode 100644 index 9ac21c82676..00000000000 --- a/src/Common/DenseHashMap.h +++ /dev/null @@ -1,29 +0,0 @@ -#pragma once -#include - -/// DenseHashMap is a wrapper for google::dense_hash_map. -/// Some hacks are needed to make it work in "Arcadia". -/// "Arcadia" is a proprietary monorepository in Yandex. -/// It uses slightly changed version of sparsehash with a different set of hash functions (which we don't need). -/// Those defines are needed to make it compile. -#if defined(ARCADIA_BUILD) -#define HASH_FUN_H -template -struct THash; -#endif - -#include - -#if !defined(ARCADIA_BUILD) - template , - class EqualKey = std::equal_to, - class Alloc = google::libc_allocator_with_realloc>> - using DenseHashMap = google::dense_hash_map; -#else - template , - class EqualKey = std::equal_to, - class Alloc = google::sparsehash::libc_allocator_with_realloc>> - using DenseHashMap = google::sparsehash::dense_hash_map; - - #undef THash -#endif diff --git a/src/Common/DenseHashSet.h b/src/Common/DenseHashSet.h deleted file mode 100644 index e8c06f36aa3..00000000000 --- a/src/Common/DenseHashSet.h +++ /dev/null @@ -1,25 +0,0 @@ -#pragma once - -/// DenseHashSet is a wrapper for google::dense_hash_set. -/// See comment in DenseHashMap.h -#if defined(ARCADIA_BUILD) -#define HASH_FUN_H -template -struct THash; -#endif - -#include - -#if !defined(ARCADIA_BUILD) - template , - class EqualKey = std::equal_to, - class Alloc = google::libc_allocator_with_realloc> - using DenseHashSet = google::dense_hash_set; -#else - template , - class EqualKey = std::equal_to, - class Alloc = google::sparsehash::libc_allocator_with_realloc> - using DenseHashSet = google::sparsehash::dense_hash_set; - - #undef THash -#endif diff --git a/src/Common/SparseHashMap.h b/src/Common/SparseHashMap.h index f01fc633d84..0f86cc13612 100644 --- a/src/Common/SparseHashMap.h +++ b/src/Common/SparseHashMap.h @@ -1,7 +1,6 @@ #pragma once /// SparseHashMap is a wrapper for google::sparse_hash_map. -/// See comment in DenseHashMap.h #if defined(ARCADIA_BUILD) #define HASH_FUN_H template diff --git a/src/Core/NamesAndTypes.cpp b/src/Core/NamesAndTypes.cpp index 54f83fc13fc..b47f5a6823b 100644 --- a/src/Core/NamesAndTypes.cpp +++ b/src/Core/NamesAndTypes.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -6,7 +7,6 @@ #include #include #include -#include namespace DB @@ -163,8 +163,7 @@ NamesAndTypesList NamesAndTypesList::filter(const Names & names) const NamesAndTypesList NamesAndTypesList::addTypes(const Names & names) const { /// NOTE: It's better to make a map in `IStorage` than to create it here every time again. - DenseHashMap types; - types.set_empty_key(StringRef()); + HashMapWithSavedHash types; for (const auto & column : *this) types[column.name] = &column.type; @@ -172,10 +171,11 @@ NamesAndTypesList NamesAndTypesList::addTypes(const Names & names) const NamesAndTypesList res; for (const String & name : names) { - auto it = types.find(name); + const auto * it = types.find(name); if (it == types.end()) - throw Exception("No column " + name, ErrorCodes::THERE_IS_NO_COLUMN); - res.emplace_back(name, *it->second); + throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, "No column {}", name); + + res.emplace_back(name, *it->getMapped()); } return res; diff --git a/src/Storages/MergeTree/IMergeTreeReader.cpp b/src/Storages/MergeTree/IMergeTreeReader.cpp index 5378b84a5d0..d659259e1a9 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.cpp +++ b/src/Storages/MergeTree/IMergeTreeReader.cpp @@ -48,7 +48,6 @@ IMergeTreeReader::IMergeTreeReader( part_columns = Nested::collect(part_columns); } - columns_from_part.set_empty_key(StringRef()); for (const auto & column_from_part : part_columns) columns_from_part[column_from_part.name] = &column_from_part.type; } @@ -213,7 +212,7 @@ NameAndTypePair IMergeTreeReader::getColumnFromPart(const NameAndTypePair & requ { auto name_in_storage = required_column.getNameInStorage(); - decltype(columns_from_part.begin()) it; + ColumnsFromPart::ConstLookupResult it; if (alter_conversions.isColumnRenamed(name_in_storage)) { String old_name = alter_conversions.getColumnOldName(name_in_storage); @@ -227,7 +226,7 @@ NameAndTypePair IMergeTreeReader::getColumnFromPart(const NameAndTypePair & requ if (it == columns_from_part.end()) return required_column; - const auto & type = *it->second; + const DataTypePtr & type = *it->getMapped(); if (required_column.isSubcolumn()) { auto subcolumn_name = required_column.getSubcolumnName(); @@ -236,10 +235,10 @@ NameAndTypePair IMergeTreeReader::getColumnFromPart(const NameAndTypePair & requ if (!subcolumn_type) return required_column; - return {String(it->first), subcolumn_name, type, subcolumn_type}; + return {String(it->getKey()), subcolumn_name, type, subcolumn_type}; } - return {String(it->first), type}; + return {String(it->getKey()), type}; } void IMergeTreeReader::performRequiredConversions(Columns & res_columns) diff --git a/src/Storages/MergeTree/IMergeTreeReader.h b/src/Storages/MergeTree/IMergeTreeReader.h index 8d80719efaf..696cc2f105b 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.h +++ b/src/Storages/MergeTree/IMergeTreeReader.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include #include @@ -95,7 +95,8 @@ private: /// Actual data type of columns in part - DenseHashMap columns_from_part; + using ColumnsFromPart = HashMapWithSavedHash; + ColumnsFromPart columns_from_part; }; } diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index 91f69cdac7d..5183b925141 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -1,7 +1,7 @@ #include -#include -#include +#include +#include #include #include #include @@ -320,8 +320,7 @@ Block StorageInMemoryMetadata::getSampleBlockForColumns( { Block res; - DenseHashMap virtuals_map; - virtuals_map.set_empty_key(StringRef()); + HashMapWithSavedHash virtuals_map; /// Virtual columns must be appended after ordinary, because user can /// override them. @@ -335,9 +334,9 @@ Block StorageInMemoryMetadata::getSampleBlockForColumns( { res.insert({column->type->createColumn(), column->type, column->name}); } - else if (auto it = virtuals_map.find(name); it != virtuals_map.end()) + else if (auto * it = virtuals_map.find(name); it != virtuals_map.end()) { - const auto & type = *it->second; + const auto & type = *it->getMapped(); res.insert({type->createColumn(), type, name}); } else @@ -470,8 +469,8 @@ bool StorageInMemoryMetadata::hasSelectQuery() const namespace { - using NamesAndTypesMap = DenseHashMap; - using UniqueStrings = DenseHashSet; + using NamesAndTypesMap = HashMapWithSavedHash; + using UniqueStrings = HashSetWithSavedHash; String listOfColumns(const NamesAndTypesList & available_columns) { @@ -488,20 +487,12 @@ namespace NamesAndTypesMap getColumnsMap(const NamesAndTypesList & columns) { NamesAndTypesMap res; - res.set_empty_key(StringRef()); for (const auto & column : columns) res.insert({column.name, column.type.get()}); return res; } - - UniqueStrings initUniqueStrings() - { - UniqueStrings strings; - strings.set_empty_key(StringRef()); - return strings; - } } void StorageInMemoryMetadata::check(const Names & column_names, const NamesAndTypesList & virtuals, const StorageID & storage_id) const @@ -514,11 +505,12 @@ void StorageInMemoryMetadata::check(const Names & column_names, const NamesAndTy } const auto virtuals_map = getColumnsMap(virtuals); - auto unique_names = initUniqueStrings(); + UniqueStrings unique_names; for (const auto & name : column_names) { - bool has_column = getColumns().hasColumnOrSubcolumn(ColumnsDescription::AllPhysical, name) || virtuals_map.count(name); + bool has_column = getColumns().hasColumnOrSubcolumn(ColumnsDescription::AllPhysical, name) + || virtuals_map.find(name) != nullptr; if (!has_column) { @@ -540,23 +532,31 @@ void StorageInMemoryMetadata::check(const NamesAndTypesList & provided_columns) const NamesAndTypesList & available_columns = getColumns().getAllPhysical(); const auto columns_map = getColumnsMap(available_columns); - auto unique_names = initUniqueStrings(); + UniqueStrings unique_names; + for (const NameAndTypePair & column : provided_columns) { - auto it = columns_map.find(column.name); + const auto * it = columns_map.find(column.name); if (columns_map.end() == it) throw Exception( - "There is no column with name " + column.name + ". There are columns: " + listOfColumns(available_columns), - ErrorCodes::NO_SUCH_COLUMN_IN_TABLE); + ErrorCodes::NO_SUCH_COLUMN_IN_TABLE, + "There is no column with name {}. There are columns: {}", + column.name, + listOfColumns(available_columns)); - if (!column.type->equals(*it->second)) + if (!column.type->equals(*it->getMapped())) throw Exception( - "Type mismatch for column " + column.name + ". Column has type " + it->second->getName() + ", got type " - + column.type->getName(), - ErrorCodes::TYPE_MISMATCH); + ErrorCodes::TYPE_MISMATCH, + "Type mismatch for column {}. Column has type {}, got type {}", + column.name, + it->getMapped()->getName(), + column.type->getName()); if (unique_names.end() != unique_names.find(column.name)) - throw Exception("Column " + column.name + " queried more than once", ErrorCodes::COLUMN_QUERIED_MORE_THAN_ONCE); + throw Exception(ErrorCodes::COLUMN_QUERIED_MORE_THAN_ONCE, + "Column {} queried more than once", + column.name); + unique_names.insert(column.name); } } @@ -572,26 +572,38 @@ void StorageInMemoryMetadata::check(const NamesAndTypesList & provided_columns, "Empty list of columns queried. There are columns: " + listOfColumns(available_columns), ErrorCodes::EMPTY_LIST_OF_COLUMNS_QUERIED); - auto unique_names = initUniqueStrings(); + UniqueStrings unique_names; + for (const String & name : column_names) { - auto it = provided_columns_map.find(name); + const auto * it = provided_columns_map.find(name); if (provided_columns_map.end() == it) continue; - auto jt = available_columns_map.find(name); + const auto * jt = available_columns_map.find(name); if (available_columns_map.end() == jt) throw Exception( - "There is no column with name " + name + ". There are columns: " + listOfColumns(available_columns), - ErrorCodes::NO_SUCH_COLUMN_IN_TABLE); + ErrorCodes::NO_SUCH_COLUMN_IN_TABLE, + "There is no column with name {}. There are columns: {}", + name, + listOfColumns(available_columns)); - if (!it->second->equals(*jt->second)) + const auto & provided_column_type = *it->getMapped(); + const auto & available_column_type = *jt->getMapped(); + + if (!provided_column_type.equals(available_column_type)) throw Exception( - "Type mismatch for column " + name + ". Column has type " + jt->second->getName() + ", got type " + it->second->getName(), - ErrorCodes::TYPE_MISMATCH); + ErrorCodes::TYPE_MISMATCH, + "Type mismatch for column {}. Column has type {}, got type {}", + name, + provided_column_type.getName(), + available_column_type.getName()); if (unique_names.end() != unique_names.find(name)) - throw Exception("Column " + name + " queried more than once", ErrorCodes::COLUMN_QUERIED_MORE_THAN_ONCE); + throw Exception(ErrorCodes::COLUMN_QUERIED_MORE_THAN_ONCE, + "Column {} queried more than once", + name); + unique_names.insert(name); } } @@ -612,17 +624,21 @@ void StorageInMemoryMetadata::check(const Block & block, bool need_all) const names_in_block.insert(column.name); - auto it = columns_map.find(column.name); + const auto * it = columns_map.find(column.name); if (columns_map.end() == it) throw Exception( - "There is no column with name " + column.name + ". There are columns: " + listOfColumns(available_columns), - ErrorCodes::NO_SUCH_COLUMN_IN_TABLE); + ErrorCodes::NO_SUCH_COLUMN_IN_TABLE, + "There is no column with name {}. There are columns: {}", + column.name, + listOfColumns(available_columns)); - if (!column.type->equals(*it->second)) + if (!column.type->equals(*it->getMapped())) throw Exception( - "Type mismatch for column " + column.name + ". Column has type " + it->second->getName() + ", got type " - + column.type->getName(), - ErrorCodes::TYPE_MISMATCH); + ErrorCodes::TYPE_MISMATCH, + "Type mismatch for column {}. Column has type {}, got type {}", + column.name, + it->getMapped()->getName(), + column.type->getName()); } if (need_all && names_in_block.size() < columns_map.size()) From 404eac198684bf08273dfc91743c7c9cadb17534 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sun, 15 Aug 2021 13:52:44 +0300 Subject: [PATCH 148/161] Fixed tests --- ...map_add_map_subtract_on_map_type.reference | 4 +-- ...01925_map_populate_series_on_map.reference | 34 +++++++++---------- 2 files changed, 19 insertions(+), 19 deletions(-) diff --git a/tests/queries/0_stateless/01318_map_add_map_subtract_on_map_type.reference b/tests/queries/0_stateless/01318_map_add_map_subtract_on_map_type.reference index 304f7407cf5..de34b856130 100644 --- a/tests/queries/0_stateless/01318_map_add_map_subtract_on_map_type.reference +++ b/tests/queries/0_stateless/01318_map_add_map_subtract_on_map_type.reference @@ -47,9 +47,9 @@ {'b':2} Map(Enum8(\'a\' = 1, \'b\' = 2), Int64) {'00000000-89ab-cdef-0123-456789abcdef':2} Map(UUID, Int64) {'11111111-89ab-cdef-0123-456789abcdef':4} Map(UUID, Int64) -{1:0,2:0} Map(UInt8,UInt64) +{1:0,2:0} Map(UInt8, UInt64) {1:18446744073709551615,2:18446744073709551615} Map(UInt8, UInt64) -{1:-1,2:-1} Map(UInt8,Int64) +{1:-1,2:-1} Map(UInt8, Int64) {1:-1.0999999761581423,2:0} Map(UInt8, Float64) {1:-1,2:-1} Map(UInt8, Int64) {1:-2,2:-2,3:1} Map(UInt8, Int64) diff --git a/tests/queries/0_stateless/01925_map_populate_series_on_map.reference b/tests/queries/0_stateless/01925_map_populate_series_on_map.reference index 235a227f548..fd3d3b2450d 100644 --- a/tests/queries/0_stateless/01925_map_populate_series_on_map.reference +++ b/tests/queries/0_stateless/01925_map_populate_series_on_map.reference @@ -29,39 +29,39 @@ select mapPopulateSeries(m, n) from map_test; {1:1,2:0,3:0,4:0,5:2,6:0} drop table map_test; select mapPopulateSeries(map(toUInt8(1), toUInt8(1), 2, 1)) as res, toTypeName(res); -{1:1,2:1} Map(UInt8,UInt8) +{1:1,2:1} Map(UInt8, UInt8) select mapPopulateSeries(map(toUInt16(1), toUInt16(1), 2, 1)) as res, toTypeName(res); -{1:1,2:1} Map(UInt16,UInt16) +{1:1,2:1} Map(UInt16, UInt16) select mapPopulateSeries(map(toUInt32(1), toUInt32(1), 2, 1)) as res, toTypeName(res); -{1:1,2:1} Map(UInt32,UInt32) +{1:1,2:1} Map(UInt32, UInt32) select mapPopulateSeries(map(toUInt64(1), toUInt64(1), 2, 1)) as res, toTypeName(res); -{1:1,2:1} Map(UInt64,UInt64) +{1:1,2:1} Map(UInt64, UInt64) select mapPopulateSeries(map(toUInt128(1), toUInt128(1), 2, 1)) as res, toTypeName(res); -{1:1,2:1} Map(UInt128,UInt128) +{1:1,2:1} Map(UInt128, UInt128) select mapPopulateSeries(map(toUInt256(1), toUInt256(1), 2, 1)) as res, toTypeName(res); -{1:1,2:1} Map(UInt256,UInt256) +{1:1,2:1} Map(UInt256, UInt256) select mapPopulateSeries(map(toInt8(1), toInt8(1), 2, 1)) as res, toTypeName(res); -{1:1,2:1} Map(Int16,Int16) +{1:1,2:1} Map(Int16, Int16) select mapPopulateSeries(map(toInt16(1), toInt16(1), 2, 1)) as res, toTypeName(res); -{1:1,2:1} Map(Int16,Int16) +{1:1,2:1} Map(Int16, Int16) select mapPopulateSeries(map(toInt32(1), toInt32(1), 2, 1)) as res, toTypeName(res); -{1:1,2:1} Map(Int32,Int32) +{1:1,2:1} Map(Int32, Int32) select mapPopulateSeries(map(toInt64(1), toInt64(1), 2, 1)) as res, toTypeName(res); -{1:1,2:1} Map(Int64,Int64) +{1:1,2:1} Map(Int64, Int64) select mapPopulateSeries(map(toInt128(1), toInt128(1), 2, 1)) as res, toTypeName(res); -{1:1,2:1} Map(Int128,Int128) +{1:1,2:1} Map(Int128, Int128) select mapPopulateSeries(map(toInt256(1), toInt256(1), 2, 1)) as res, toTypeName(res); -{1:1,2:1} Map(Int256,Int256) +{1:1,2:1} Map(Int256, Int256) select mapPopulateSeries(map(toInt8(-10), toInt8(1), 2, 1)) as res, toTypeName(res); -{-10:1,-9:0,-8:0,-7:0,-6:0,-5:0,-4:0,-3:0,-2:0,-1:0,0:0,1:0,2:1} Map(Int16,Int16) +{-10:1,-9:0,-8:0,-7:0,-6:0,-5:0,-4:0,-3:0,-2:0,-1:0,0:0,1:0,2:1} Map(Int16, Int16) select mapPopulateSeries(map(toInt16(-10), toInt16(1), 2, 1)) as res, toTypeName(res); -{-10:1,-9:0,-8:0,-7:0,-6:0,-5:0,-4:0,-3:0,-2:0,-1:0,0:0,1:0,2:1} Map(Int16,Int16) +{-10:1,-9:0,-8:0,-7:0,-6:0,-5:0,-4:0,-3:0,-2:0,-1:0,0:0,1:0,2:1} Map(Int16, Int16) select mapPopulateSeries(map(toInt32(-10), toInt32(1), 2, 1)) as res, toTypeName(res); -{-10:1,-9:0,-8:0,-7:0,-6:0,-5:0,-4:0,-3:0,-2:0,-1:0,0:0,1:0,2:1} Map(Int32,Int32) +{-10:1,-9:0,-8:0,-7:0,-6:0,-5:0,-4:0,-3:0,-2:0,-1:0,0:0,1:0,2:1} Map(Int32, Int32) select mapPopulateSeries(map(toInt64(-10), toInt64(1), 2, 1)) as res, toTypeName(res); -{-10:1,-9:0,-8:0,-7:0,-6:0,-5:0,-4:0,-3:0,-2:0,-1:0,0:0,1:0,2:1} Map(Int64,Int64) +{-10:1,-9:0,-8:0,-7:0,-6:0,-5:0,-4:0,-3:0,-2:0,-1:0,0:0,1:0,2:1} Map(Int64, Int64) select mapPopulateSeries(map(toInt64(-10), toInt64(1), 2, 1), toInt64(-5)) as res, toTypeName(res); -{-10:1,-9:0,-8:0,-7:0,-6:0,-5:0} Map(Int64,Int64) +{-10:1,-9:0,-8:0,-7:0,-6:0,-5:0} Map(Int64, Int64) select mapPopulateSeries(); -- { serverError 42 } select mapPopulateSeries('asdf'); -- { serverError 43 } select mapPopulateSeries(map('1', 1, '2', 1)) as res, toTypeName(res); -- { serverError 43 } From 178d0f9ba95af192b4ae391f216224233269a6eb Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 10 Aug 2021 17:06:22 +0300 Subject: [PATCH 149/161] Fix checking GRANT OPTION while executing GRANT with REPLACE OPTION. --- src/Access/AccessRightsElement.h | 3 + src/Interpreters/InterpreterGrantQuery.cpp | 391 +++++++++++------- .../integration/test_grant_and_revoke/test.py | 33 ++ 3 files changed, 277 insertions(+), 150 deletions(-) diff --git a/src/Access/AccessRightsElement.h b/src/Access/AccessRightsElement.h index c76f019bc61..c46a4b54e6e 100644 --- a/src/Access/AccessRightsElement.h +++ b/src/Access/AccessRightsElement.h @@ -122,6 +122,9 @@ struct AccessRightsElement class AccessRightsElements : public std::vector { public: + using Base = std::vector; + using Base::Base; + bool empty() const { return std::all_of(begin(), end(), [](const AccessRightsElement & e) { return e.empty(); }); } bool sameDatabaseAndTable() const diff --git a/src/Interpreters/InterpreterGrantQuery.cpp b/src/Interpreters/InterpreterGrantQuery.cpp index 42c440b4c52..2e7b3a58012 100644 --- a/src/Interpreters/InterpreterGrantQuery.cpp +++ b/src/Interpreters/InterpreterGrantQuery.cpp @@ -22,69 +22,108 @@ namespace ErrorCodes namespace { - template - void updateFromQueryTemplate( - T & grantee, + /// Extracts access rights elements which are going to be granted or revoked from a query. + void collectAccessRightsElementsToGrantOrRevoke( const ASTGrantQuery & query, - const std::vector & roles_to_grant_or_revoke) + AccessRightsElements & elements_to_grant, + AccessRightsElements & elements_to_revoke) { - if (!query.is_revoke) + elements_to_grant.clear(); + elements_to_revoke.clear(); + + if (query.is_revoke) { - if (query.replace_access) - grantee.access = {}; - if (query.replace_granted_roles) - grantee.granted_roles = {}; + /// REVOKE + elements_to_revoke = query.access_rights_elements; } - - - if (!query.access_rights_elements.empty()) + else if (query.replace_access) { - if (query.is_revoke) - grantee.access.revoke(query.access_rights_elements); - else - grantee.access.grant(query.access_rights_elements); + /// GRANT WITH REPLACE OPTION + elements_to_grant = query.access_rights_elements; + elements_to_revoke.emplace_back(AccessType::ALL); } - - if (!roles_to_grant_or_revoke.empty()) + else { - if (query.is_revoke) - { - if (query.admin_option) - grantee.granted_roles.revokeAdminOption(roles_to_grant_or_revoke); - else - grantee.granted_roles.revoke(roles_to_grant_or_revoke); - } - else - { - if (query.admin_option) - grantee.granted_roles.grantWithAdminOption(roles_to_grant_or_revoke); - else - grantee.granted_roles.grant(roles_to_grant_or_revoke); - } + /// GRANT + elements_to_grant = query.access_rights_elements; } } - void updateFromQueryImpl( - IAccessEntity & grantee, + /// Extracts roles which are going to be granted or revoked from a query. + void collectRolesToGrantOrRevoke( + const AccessControlManager & access_control, const ASTGrantQuery & query, - const std::vector & roles_to_grant_or_revoke) + std::vector & roles_to_grant, + RolesOrUsersSet & roles_to_revoke) { - if (auto * user = typeid_cast(&grantee)) - updateFromQueryTemplate(*user, query, roles_to_grant_or_revoke); - else if (auto * role = typeid_cast(&grantee)) - updateFromQueryTemplate(*role, query, roles_to_grant_or_revoke); + roles_to_grant.clear(); + roles_to_revoke.clear(); + + RolesOrUsersSet roles_to_grant_or_revoke; + if (query.roles) + roles_to_grant_or_revoke = RolesOrUsersSet{*query.roles, access_control}; + + if (query.is_revoke) + { + /// REVOKE + roles_to_revoke = std::move(roles_to_grant_or_revoke); + } + else if (query.replace_granted_roles) + { + /// GRANT WITH REPLACE OPTION + roles_to_grant = roles_to_grant_or_revoke.getMatchingIDs(access_control); + roles_to_revoke = RolesOrUsersSet::AllTag{}; + } + else + { + /// GRANT + roles_to_grant = roles_to_grant_or_revoke.getMatchingIDs(access_control); + } } - void checkGranteeIsAllowed(const ContextAccess & access, const UUID & grantee_id, const IAccessEntity & grantee) + /// Extracts roles which are going to be granted or revoked from a query. + void collectRolesToGrantOrRevoke( + const ASTGrantQuery & query, + std::vector & roles_to_grant, + RolesOrUsersSet & roles_to_revoke) { - auto current_user = access.getUser(); + roles_to_grant.clear(); + roles_to_revoke.clear(); + + RolesOrUsersSet roles_to_grant_or_revoke; + if (query.roles) + roles_to_grant_or_revoke = RolesOrUsersSet{*query.roles}; + + if (query.is_revoke) + { + /// REVOKE + roles_to_revoke = std::move(roles_to_grant_or_revoke); + } + else if (query.replace_granted_roles) + { + /// GRANT WITH REPLACE OPTION + roles_to_grant = roles_to_grant_or_revoke.getMatchingIDs(); + roles_to_revoke = RolesOrUsersSet::AllTag{}; + } + else + { + /// GRANT + roles_to_grant = roles_to_grant_or_revoke.getMatchingIDs(); + } + } + + /// Checks if a grantee is allowed for the current user, throws an exception if not. + void checkGranteeIsAllowed(const ContextAccess & current_user_access, const UUID & grantee_id, const IAccessEntity & grantee) + { + auto current_user = current_user_access.getUser(); if (current_user && !current_user->grantees.match(grantee_id)) throw Exception(grantee.outputTypeAndName() + " is not allowed as grantee", ErrorCodes::ACCESS_DENIED); } - void checkGranteesAreAllowed(const AccessControlManager & access_control, const ContextAccess & access, const std::vector & grantee_ids) + /// Checks if grantees are allowed for the current user, throws an exception if not. + void checkGranteesAreAllowed(const AccessControlManager & access_control, const ContextAccess & current_user_access, const std::vector & grantee_ids) { - auto current_user = access.getUser(); + auto current_user = current_user_access.getUser(); if (!current_user || (current_user->grantees == RolesOrUsersSet::AllTag{})) return; @@ -92,36 +131,26 @@ namespace { auto entity = access_control.tryRead(id); if (auto role = typeid_cast(entity)) - checkGranteeIsAllowed(access, id, *role); + checkGranteeIsAllowed(current_user_access, id, *role); else if (auto user = typeid_cast(entity)) - checkGranteeIsAllowed(access, id, *user); + checkGranteeIsAllowed(current_user_access, id, *user); } } + /// Checks if the current user has enough access rights granted with grant option to grant or revoke specified access rights. void checkGrantOption( const AccessControlManager & access_control, - const ContextAccess & access, - const ASTGrantQuery & query, + const ContextAccess & current_user_access, const std::vector & grantees_from_query, - bool & need_check_grantees_are_allowed) + bool & need_check_grantees_are_allowed, + const AccessRightsElements & elements_to_grant, + AccessRightsElements & elements_to_revoke) { - const auto & elements = query.access_rights_elements; - need_check_grantees_are_allowed = true; - if (elements.empty()) - { - /// No access rights to grant or revoke. - need_check_grantees_are_allowed = false; - return; - } + /// Check access rights which are going to be granted. + /// To execute the command GRANT the current user needs to have the access granted with GRANT OPTION. + current_user_access.checkGrantOption(elements_to_grant); - if (!query.is_revoke) - { - /// To execute the command GRANT the current user needs to have the access granted with GRANT OPTION. - access.checkGrantOption(elements); - return; - } - - if (access.hasGrantOption(elements)) + if (current_user_access.hasGrantOption(elements_to_revoke)) { /// Simple case: the current user has the grant option for all the access rights specified for REVOKE. return; @@ -141,69 +170,81 @@ namespace auto entity = access_control.tryRead(id); if (auto role = typeid_cast(entity)) { - checkGranteeIsAllowed(access, id, *role); + checkGranteeIsAllowed(current_user_access, id, *role); all_granted_access.makeUnion(role->access); } else if (auto user = typeid_cast(entity)) { - checkGranteeIsAllowed(access, id, *user); + checkGranteeIsAllowed(current_user_access, id, *user); all_granted_access.makeUnion(user->access); } } need_check_grantees_are_allowed = false; /// already checked - AccessRights required_access; - if (elements[0].is_partial_revoke) - { - AccessRightsElements non_revoke_elements = elements; - std::for_each(non_revoke_elements.begin(), non_revoke_elements.end(), [&](AccessRightsElement & element) { element.is_partial_revoke = false; }); - required_access.grant(non_revoke_elements); - } - else - { - required_access.grant(elements); - } - required_access.makeIntersection(all_granted_access); + if (!elements_to_revoke.empty() && elements_to_revoke[0].is_partial_revoke) + std::for_each(elements_to_revoke.begin(), elements_to_revoke.end(), [&](AccessRightsElement & element) { element.is_partial_revoke = false; }); + AccessRights access_to_revoke; + access_to_revoke.grant(elements_to_revoke); + access_to_revoke.makeIntersection(all_granted_access); - for (auto & required_access_element : required_access.getElements()) + /// Build more accurate list of elements to revoke, now we use an intesection of the initial list of elements to revoke + /// and all the granted access rights to these grantees. + bool grant_option = !elements_to_revoke.empty() && elements_to_revoke[0].grant_option; + elements_to_revoke.clear(); + for (auto & element_to_revoke : access_to_revoke.getElements()) { - if (!required_access_element.is_partial_revoke && (required_access_element.grant_option || !elements[0].grant_option)) - access.checkGrantOption(required_access_element); + if (!element_to_revoke.is_partial_revoke && (element_to_revoke.grant_option || !grant_option)) + elements_to_revoke.emplace_back(std::move(element_to_revoke)); } + + current_user_access.checkGrantOption(elements_to_revoke); } - std::vector getRoleIDsAndCheckAdminOption( + /// Checks if the current user has enough access rights granted with grant option to grant or revoke specified access rights. + /// Also checks if grantees are allowed for the current user. + void checkGrantOptionAndGrantees( const AccessControlManager & access_control, - const ContextAccess & access, - const ASTGrantQuery & query, - const RolesOrUsersSet & roles_from_query, + const ContextAccess & current_user_access, const std::vector & grantees_from_query, - bool & need_check_grantees_are_allowed) + const AccessRightsElements & elements_to_grant, + AccessRightsElements & elements_to_revoke) { - need_check_grantees_are_allowed = true; - if (roles_from_query.empty()) - { - /// No roles to grant or revoke. - need_check_grantees_are_allowed = false; - return {}; - } + bool need_check_grantees_are_allowed = true; + checkGrantOption( + access_control, + current_user_access, + grantees_from_query, + need_check_grantees_are_allowed, + elements_to_grant, + elements_to_revoke); - std::vector matching_ids; - if (!query.is_revoke) - { - /// To execute the command GRANT the current user needs to have the roles granted with ADMIN OPTION. - matching_ids = roles_from_query.getMatchingIDs(access_control); - access.checkAdminOption(matching_ids); - return matching_ids; - } + if (need_check_grantees_are_allowed) + checkGranteesAreAllowed(access_control, current_user_access, grantees_from_query); + } - if (!roles_from_query.all) + /// Checks if the current user has enough roles granted with admin option to grant or revoke specified roles. + void checkAdminOption( + const AccessControlManager & access_control, + const ContextAccess & current_user_access, + const std::vector & grantees_from_query, + bool & need_check_grantees_are_allowed, + const std::vector & roles_to_grant, + RolesOrUsersSet & roles_to_revoke, + bool admin_option) + { + /// Check roles which are going to be granted. + /// To execute the command GRANT the current user needs to have the roles granted with ADMIN OPTION. + current_user_access.checkAdminOption(roles_to_grant); + + /// Check roles which are going to be revoked. + std::vector roles_to_revoke_ids; + if (!roles_to_revoke.all) { - matching_ids = roles_from_query.getMatchingIDs(); - if (access.hasAdminOption(matching_ids)) + roles_to_revoke_ids = roles_to_revoke.getMatchingIDs(); + if (current_user_access.hasAdminOption(roles_to_revoke_ids)) { /// Simple case: the current user has the admin option for all the roles specified for REVOKE. - return matching_ids; + return; } } @@ -221,51 +262,109 @@ namespace auto entity = access_control.tryRead(id); if (auto role = typeid_cast(entity)) { - checkGranteeIsAllowed(access, id, *role); + checkGranteeIsAllowed(current_user_access, id, *role); all_granted_roles.makeUnion(role->granted_roles); } else if (auto user = typeid_cast(entity)) { - checkGranteeIsAllowed(access, id, *user); + checkGranteeIsAllowed(current_user_access, id, *user); all_granted_roles.makeUnion(user->granted_roles); } } + const auto & all_granted_roles_set = admin_option ? all_granted_roles.getGrantedWithAdminOption() : all_granted_roles.getGranted(); need_check_grantees_are_allowed = false; /// already checked - const auto & all_granted_roles_set = query.admin_option ? all_granted_roles.getGrantedWithAdminOption() : all_granted_roles.getGranted(); - if (roles_from_query.all) - boost::range::set_difference(all_granted_roles_set, roles_from_query.except_ids, std::back_inserter(matching_ids)); + if (roles_to_revoke.all) + boost::range::set_difference(all_granted_roles_set, roles_to_revoke.except_ids, std::back_inserter(roles_to_revoke_ids)); else - boost::range::remove_erase_if(matching_ids, [&](const UUID & id) { return !all_granted_roles_set.count(id); }); - access.checkAdminOption(matching_ids); - return matching_ids; + boost::range::remove_erase_if(roles_to_revoke_ids, [&](const UUID & id) { return !all_granted_roles_set.count(id); }); + + roles_to_revoke = roles_to_revoke_ids; + current_user_access.checkAdminOption(roles_to_revoke_ids); } - void checkGrantOptionAndGrantees( + /// Checks if the current user has enough roles granted with admin option to grant or revoke specified roles. + /// Also checks if grantees are allowed for the current user. + void checkAdminOptionAndGrantees( const AccessControlManager & access_control, - const ContextAccess & access, - const ASTGrantQuery & query, - const std::vector & grantees_from_query) + const ContextAccess & current_user_access, + const std::vector & grantees_from_query, + const std::vector & roles_to_grant, + RolesOrUsersSet & roles_to_revoke, + bool admin_option) { bool need_check_grantees_are_allowed = true; - checkGrantOption(access_control, access, query, grantees_from_query, need_check_grantees_are_allowed); + checkAdminOption( + access_control, + current_user_access, + grantees_from_query, + need_check_grantees_are_allowed, + roles_to_grant, + roles_to_revoke, + admin_option); + if (need_check_grantees_are_allowed) - checkGranteesAreAllowed(access_control, access, grantees_from_query); + checkGranteesAreAllowed(access_control, current_user_access, grantees_from_query); } - std::vector getRoleIDsAndCheckAdminOptionAndGrantees( - const AccessControlManager & access_control, - const ContextAccess & access, - const ASTGrantQuery & query, - const RolesOrUsersSet & roles_from_query, - const std::vector & grantees_from_query) + template + void updateGrantedAccessRightsAndRolesTemplate( + T & grantee, + const AccessRightsElements & elements_to_grant, + const AccessRightsElements & elements_to_revoke, + const std::vector & roles_to_grant, + const RolesOrUsersSet & roles_to_revoke, + bool admin_option) { - bool need_check_grantees_are_allowed = true; - auto role_ids = getRoleIDsAndCheckAdminOption( - access_control, access, query, roles_from_query, grantees_from_query, need_check_grantees_are_allowed); - if (need_check_grantees_are_allowed) - checkGranteesAreAllowed(access_control, access, grantees_from_query); - return role_ids; + if (!elements_to_revoke.empty()) + grantee.access.revoke(elements_to_revoke); + + if (!elements_to_grant.empty()) + grantee.access.grant(elements_to_grant); + + if (!roles_to_revoke.empty()) + { + if (admin_option) + grantee.granted_roles.revokeAdminOption(grantee.granted_roles.findGrantedWithAdminOption(roles_to_revoke)); + else + grantee.granted_roles.revoke(grantee.granted_roles.findGranted(roles_to_revoke)); + } + + if (!roles_to_grant.empty()) + { + if (admin_option) + grantee.granted_roles.grantWithAdminOption(roles_to_grant); + else + grantee.granted_roles.grant(roles_to_grant); + } + } + + /// Updates grants of a specified user or role. + void updateGrantedAccessRightsAndRoles( + IAccessEntity & grantee, + const AccessRightsElements & elements_to_grant, + const AccessRightsElements & elements_to_revoke, + const std::vector & roles_to_grant, + const RolesOrUsersSet & roles_to_revoke, + bool admin_option) + { + if (auto * user = typeid_cast(&grantee)) + updateGrantedAccessRightsAndRolesTemplate(*user, elements_to_grant, elements_to_revoke, roles_to_grant, roles_to_revoke, admin_option); + else if (auto * role = typeid_cast(&grantee)) + updateGrantedAccessRightsAndRolesTemplate(*role, elements_to_grant, elements_to_revoke, roles_to_grant, roles_to_revoke, admin_option); + } + + /// Updates grants of a specified user or role. + void updateFromQuery(IAccessEntity & grantee, const ASTGrantQuery & query) + { + AccessRightsElements elements_to_grant, elements_to_revoke; + collectAccessRightsElementsToGrantOrRevoke(query, elements_to_grant, elements_to_revoke); + + std::vector roles_to_grant; + RolesOrUsersSet roles_to_revoke; + collectRolesToGrantOrRevoke(query, roles_to_grant, roles_to_revoke); + + updateGrantedAccessRightsAndRoles(grantee, elements_to_grant, elements_to_revoke, roles_to_grant, roles_to_revoke, query.admin_option); } } @@ -283,16 +382,13 @@ BlockIO InterpreterGrantQuery::execute() throw Exception("A partial revoke should be revoked, not granted", ErrorCodes::LOGICAL_ERROR); auto & access_control = getContext()->getAccessControlManager(); - std::optional roles_set; - if (query.roles) - roles_set = RolesOrUsersSet{*query.roles, access_control}; - std::vector grantees = RolesOrUsersSet{*query.grantees, access_control, getContext()->getUserID()}.getMatchingIDs(access_control); /// Check if the current user has corresponding roles granted with admin option. - std::vector roles; - if (roles_set) - roles = getRoleIDsAndCheckAdminOptionAndGrantees(access_control, *getContext()->getAccess(), query, *roles_set, grantees); + std::vector roles_to_grant; + RolesOrUsersSet roles_to_revoke; + collectRolesToGrantOrRevoke(access_control, query, roles_to_grant, roles_to_revoke); + checkAdminOptionAndGrantees(access_control, *getContext()->getAccess(), grantees, roles_to_grant, roles_to_revoke, query.admin_option); if (!query.cluster.empty()) { @@ -306,14 +402,15 @@ BlockIO InterpreterGrantQuery::execute() query.replaceEmptyDatabase(getContext()->getCurrentDatabase()); /// Check if the current user has corresponding access rights with grant option. - if (!query.access_rights_elements.empty()) - checkGrantOptionAndGrantees(access_control, *getContext()->getAccess(), query, grantees); + AccessRightsElements elements_to_grant, elements_to_revoke; + collectAccessRightsElementsToGrantOrRevoke(query, elements_to_grant, elements_to_revoke); + checkGrantOptionAndGrantees(access_control, *getContext()->getAccess(), grantees, elements_to_grant, elements_to_revoke); /// Update roles and users listed in `grantees`. auto update_func = [&](const AccessEntityPtr & entity) -> AccessEntityPtr { auto clone = entity->clone(); - updateFromQueryImpl(*clone, query, roles); + updateGrantedAccessRightsAndRoles(*clone, elements_to_grant, elements_to_revoke, roles_to_grant, roles_to_revoke, query.admin_option); return clone; }; @@ -325,21 +422,15 @@ BlockIO InterpreterGrantQuery::execute() void InterpreterGrantQuery::updateUserFromQuery(User & user, const ASTGrantQuery & query) { - std::vector roles_to_grant_or_revoke; - if (query.roles) - roles_to_grant_or_revoke = RolesOrUsersSet{*query.roles}.getMatchingIDs(); - updateFromQueryImpl(user, query, roles_to_grant_or_revoke); + updateFromQuery(user, query); } - void InterpreterGrantQuery::updateRoleFromQuery(Role & role, const ASTGrantQuery & query) { - std::vector roles_to_grant_or_revoke; - if (query.roles) - roles_to_grant_or_revoke = RolesOrUsersSet{*query.roles}.getMatchingIDs(); - updateFromQueryImpl(role, query, roles_to_grant_or_revoke); + updateFromQuery(role, query); } + void InterpreterGrantQuery::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & /*ast*/, ContextPtr) const { auto & query = query_ptr->as(); diff --git a/tests/integration/test_grant_and_revoke/test.py b/tests/integration/test_grant_and_revoke/test.py index a63d6f136af..79fe4bf9f41 100644 --- a/tests/integration/test_grant_and_revoke/test.py +++ b/tests/integration/test_grant_and_revoke/test.py @@ -282,3 +282,36 @@ def test_current_database(): instance.query("CREATE TABLE default.table(x UInt32, y UInt32) ENGINE = MergeTree ORDER BY tuple()") assert "Not enough privileges" in instance.query_and_get_error("SELECT * FROM table", user='A') + + +def test_grant_with_replace_option(): + instance.query("CREATE USER A") + instance.query('GRANT SELECT ON test.table TO A') + assert instance.query("SHOW GRANTS FOR A") == TSV(["GRANT SELECT ON test.table TO A"]) + + instance.query('GRANT INSERT ON test.table TO A WITH REPLACE OPTION') + assert instance.query("SHOW GRANTS FOR A") == TSV(["GRANT INSERT ON test.table TO A"]) + + instance.query('GRANT NONE ON *.* TO A WITH REPLACE OPTION') + assert instance.query("SHOW GRANTS FOR A") == TSV([]) + + instance.query('CREATE USER B') + instance.query('GRANT SELECT ON test.table TO B') + assert instance.query("SHOW GRANTS FOR A") == TSV([]) + assert instance.query("SHOW GRANTS FOR B") == TSV(["GRANT SELECT ON test.table TO B"]) + + expected_error = "it's necessary to have grant INSERT ON test.table WITH GRANT OPTION" + assert expected_error in instance.query_and_get_error("GRANT INSERT ON test.table TO B WITH REPLACE OPTION", user='A') + assert instance.query("SHOW GRANTS FOR A") == TSV([]) + assert instance.query("SHOW GRANTS FOR B") == TSV(["GRANT SELECT ON test.table TO B"]) + + instance.query("GRANT INSERT ON test.table TO A WITH GRANT OPTION") + expected_error = "it's necessary to have grant SELECT ON test.table WITH GRANT OPTION" + assert expected_error in instance.query_and_get_error("GRANT INSERT ON test.table TO B WITH REPLACE OPTION", user='A') + assert instance.query("SHOW GRANTS FOR A") == TSV(["GRANT INSERT ON test.table TO A WITH GRANT OPTION"]) + assert instance.query("SHOW GRANTS FOR B") == TSV(["GRANT SELECT ON test.table TO B"]) + + instance.query("GRANT SELECT ON test.table TO A WITH GRANT OPTION") + instance.query("GRANT INSERT ON test.table TO B WITH REPLACE OPTION", user='A') + assert instance.query("SHOW GRANTS FOR A") == TSV(["GRANT SELECT, INSERT ON test.table TO A WITH GRANT OPTION"]) + assert instance.query("SHOW GRANTS FOR B") == TSV(["GRANT INSERT ON test.table TO B"]) From 6c4c3df96e41425185beb0c471a8dde0ce6f25a7 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Sun, 15 Aug 2021 16:22:15 +0300 Subject: [PATCH 150/161] Auto version update to [21.9.1.7770] [54454] --- cmake/autogenerated_versions.txt | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/cmake/autogenerated_versions.txt b/cmake/autogenerated_versions.txt index 18072566d04..2435335f669 100644 --- a/cmake/autogenerated_versions.txt +++ b/cmake/autogenerated_versions.txt @@ -6,7 +6,7 @@ SET(VERSION_REVISION 54454) SET(VERSION_MAJOR 21) SET(VERSION_MINOR 9) SET(VERSION_PATCH 1) -SET(VERSION_GITHASH f48c5af90c2ad51955d1ee3b6b05d006b03e4238) -SET(VERSION_DESCRIBE v21.9.1.1-prestable) -SET(VERSION_STRING 21.9.1.1) +SET(VERSION_GITHASH f063e44131a048ba2d9af8075f03700fd5ec3e69) +SET(VERSION_DESCRIBE v21.9.1.7770-prestable) +SET(VERSION_STRING 21.9.1.7770) # end of autochange From fd500d2176a4623145fc53c3e85c3e989da7f2ce Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 15 Aug 2021 17:05:28 +0300 Subject: [PATCH 151/161] Backquote default database in CREATE USER --- src/Parsers/ASTDatabaseOrNone.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Parsers/ASTDatabaseOrNone.cpp b/src/Parsers/ASTDatabaseOrNone.cpp index f93322ef00c..0a0c800ff36 100644 --- a/src/Parsers/ASTDatabaseOrNone.cpp +++ b/src/Parsers/ASTDatabaseOrNone.cpp @@ -1,4 +1,5 @@ #include +#include #include namespace DB @@ -10,7 +11,7 @@ void ASTDatabaseOrNone::formatImpl(const FormatSettings & settings, FormatState settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << "NONE" << (settings.hilite ? IAST::hilite_none : ""); return; } - settings.ostr << database_name; + settings.ostr << backQuoteIfNeed(database_name); } } From d39128df9fc148c38f02cea926ede9ad03458528 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 8 Aug 2021 20:08:36 +0300 Subject: [PATCH 152/161] Add module part for client/TestHint --- programs/client/CMakeLists.txt | 1 + programs/client/TestHint.cpp | 80 ++++++++++++++++++++++++++++++++++ programs/client/TestHint.h | 75 ++----------------------------- 3 files changed, 84 insertions(+), 72 deletions(-) create mode 100644 programs/client/TestHint.cpp diff --git a/programs/client/CMakeLists.txt b/programs/client/CMakeLists.txt index 084e1b45911..1de5ea88aee 100644 --- a/programs/client/CMakeLists.txt +++ b/programs/client/CMakeLists.txt @@ -3,6 +3,7 @@ set (CLICKHOUSE_CLIENT_SOURCES ConnectionParameters.cpp QueryFuzzer.cpp Suggest.cpp + TestHint.cpp ) set (CLICKHOUSE_CLIENT_LINK diff --git a/programs/client/TestHint.cpp b/programs/client/TestHint.cpp new file mode 100644 index 00000000000..03e6ce56232 --- /dev/null +++ b/programs/client/TestHint.cpp @@ -0,0 +1,80 @@ +#include "TestHint.h" + +#include +#include +#include +#include + +namespace DB +{ + +TestHint::TestHint(bool enabled_, const String & query_) + : query(query_) +{ + if (!enabled_) + return; + + // Don't parse error hints in leading comments, because it feels weird. + // Leading 'echo' hint is OK. + bool is_leading_hint = true; + + Lexer lexer(query.data(), query.data() + query.size()); + + for (Token token = lexer.nextToken(); !token.isEnd(); token = lexer.nextToken()) + { + if (token.type != TokenType::Comment + && token.type != TokenType::Whitespace) + { + is_leading_hint = false; + } + else if (token.type == TokenType::Comment) + { + String comment(token.begin, token.begin + token.size()); + + if (!comment.empty()) + { + size_t pos_start = comment.find('{', 0); + if (pos_start != String::npos) + { + size_t pos_end = comment.find('}', pos_start); + if (pos_end != String::npos) + { + String hint(comment.begin() + pos_start + 1, comment.begin() + pos_end); + parse(hint, is_leading_hint); + } + } + } + } + } +} + +void TestHint::parse(const String & hint, bool is_leading_hint) +{ + std::stringstream ss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + ss << hint; + String item; + + while (!ss.eof()) + { + ss >> item; + if (ss.eof()) + break; + + if (!is_leading_hint) + { + if (item == "serverError") + ss >> server_error; + else if (item == "clientError") + ss >> client_error; + } + + if (item == "echo") + echo.emplace(true); + if (item == "echoOn") + echo.emplace(true); + if (item == "echoOff") + echo.emplace(false); + } +} + +} diff --git a/programs/client/TestHint.h b/programs/client/TestHint.h index 100d47f4dd2..fcf50002c16 100644 --- a/programs/client/TestHint.h +++ b/programs/client/TestHint.h @@ -1,11 +1,7 @@ #pragma once -#include -#include -#include +#include #include -#include -#include namespace DB @@ -43,45 +39,7 @@ namespace DB class TestHint { public: - TestHint(bool enabled_, const String & query_) : - query(query_) - { - if (!enabled_) - return; - - // Don't parse error hints in leading comments, because it feels weird. - // Leading 'echo' hint is OK. - bool is_leading_hint = true; - - Lexer lexer(query.data(), query.data() + query.size()); - - for (Token token = lexer.nextToken(); !token.isEnd(); token = lexer.nextToken()) - { - if (token.type != TokenType::Comment - && token.type != TokenType::Whitespace) - { - is_leading_hint = false; - } - else if (token.type == TokenType::Comment) - { - String comment(token.begin, token.begin + token.size()); - - if (!comment.empty()) - { - size_t pos_start = comment.find('{', 0); - if (pos_start != String::npos) - { - size_t pos_end = comment.find('}', pos_start); - if (pos_end != String::npos) - { - String hint(comment.begin() + pos_start + 1, comment.begin() + pos_end); - parse(hint, is_leading_hint); - } - } - } - } - } - } + TestHint(bool enabled_, const String & query_); int serverError() const { return server_error; } int clientError() const { return client_error; } @@ -93,34 +51,7 @@ private: int client_error = 0; std::optional echo; - void parse(const String & hint, bool is_leading_hint) - { - std::stringstream ss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM - ss << hint; - String item; - - while (!ss.eof()) - { - ss >> item; - if (ss.eof()) - break; - - if (!is_leading_hint) - { - if (item == "serverError") - ss >> server_error; - else if (item == "clientError") - ss >> client_error; - } - - if (item == "echo") - echo.emplace(true); - if (item == "echoOn") - echo.emplace(true); - if (item == "echoOff") - echo.emplace(false); - } - } + void parse(const String & hint, bool is_leading_hint); bool allErrorsExpected(int actual_server_error, int actual_client_error) const { From ad4833b4ab116261cfaadd4a6ffef4dca8ff232c Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 8 Aug 2021 20:08:36 +0300 Subject: [PATCH 153/161] Fix endless loop for parsing invalid test hints (lack of ss.fail() check) --- programs/client/TestHint.cpp | 31 +++++++++++++++++++++++++++++-- 1 file changed, 29 insertions(+), 2 deletions(-) diff --git a/programs/client/TestHint.cpp b/programs/client/TestHint.cpp index 03e6ce56232..6149135b597 100644 --- a/programs/client/TestHint.cpp +++ b/programs/client/TestHint.cpp @@ -8,6 +8,33 @@ namespace DB { +namespace ErrorCodes +{ + extern const int CANNOT_PARSE_TEXT; +} + +} + +namespace +{ + +int parseErrorCode(std::stringstream & ss) // STYLE_CHECK_ALLOW_STD_STRING_STREAM +{ + using namespace DB; + + int code; + ss >> code; + if (ss.fail()) + throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, + "Expected integer value for test hint, got: '{}'", ss.str()); + return code; +} + +} + +namespace DB +{ + TestHint::TestHint(bool enabled_, const String & query_) : query(query_) { @@ -63,9 +90,9 @@ void TestHint::parse(const String & hint, bool is_leading_hint) if (!is_leading_hint) { if (item == "serverError") - ss >> server_error; + server_error = parseErrorCode(ss); else if (item == "clientError") - ss >> client_error; + client_error = parseErrorCode(ss); } if (item == "echo") From 098a0b2012cc9467d68185b152b5769223606e84 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 8 Aug 2021 20:08:36 +0300 Subject: [PATCH 154/161] Add ErrorCodes::getErrorCodeByName() --- src/Common/ErrorCodes.cpp | 17 +++++++++++++++++ src/Common/ErrorCodes.h | 6 ++++++ 2 files changed, 23 insertions(+) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index c15605660ef..fd14bddb64c 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -1,4 +1,5 @@ #include +#include #include /** Previously, these constants were located in one enum. @@ -564,6 +565,7 @@ M(594, BZIP2_STREAM_DECODER_FAILED) \ M(595, BZIP2_STREAM_ENCODER_FAILED) \ M(596, INTERSECT_OR_EXCEPT_RESULT_STRUCTURES_MISMATCH) \ + M(597, NO_SUCH_ERROR_CODE) \ \ M(998, POSTGRESQL_CONNECTION_FAILURE) \ M(999, KEEPER_EXCEPTION) \ @@ -602,6 +604,21 @@ namespace ErrorCodes return error_codes_names.names[error_code]; } + ErrorCode getErrorCodeByName(std::string_view error_name) + { + for (size_t i = 0, end = ErrorCodes::end(); i < end; ++i) + { + std::string_view name = ErrorCodes::getName(i); + + if (name.empty()) + continue; + + if (name == error_name) + return i; + } + throw Exception(NO_SUCH_ERROR_CODE, "No error code with name: '{}'", error_name); + } + ErrorCode end() { return END + 1; } void increment(ErrorCode error_code, bool remote, const std::string & message, const FramePointers & trace) diff --git a/src/Common/ErrorCodes.h b/src/Common/ErrorCodes.h index cefd77df868..be36aa6fd50 100644 --- a/src/Common/ErrorCodes.h +++ b/src/Common/ErrorCodes.h @@ -25,6 +25,12 @@ namespace ErrorCodes /// Get name of error_code by identifier. /// Returns statically allocated string. std::string_view getName(ErrorCode error_code); + /// Get error code value by name. + /// + /// It has O(N) complexity, but this is not major, since it is used only + /// for test hints, and it does not worth to keep another structure for + /// this. + ErrorCode getErrorCodeByName(std::string_view error_name); struct Error { From c5d9676779319b13a55b2843d29015c56fd1abff Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 8 Aug 2021 20:08:36 +0300 Subject: [PATCH 155/161] Accept error code by error name in client test hints --- programs/client/TestHint.cpp | 13 +++++++++++-- programs/client/TestHint.h | 4 ++++ .../02006_client_test_hint_error_name.reference | 0 .../02006_client_test_hint_error_name.sql | 1 + ...06_client_test_hint_no_such_error_name.reference | 1 + .../02006_client_test_hint_no_such_error_name.sh | 7 +++++++ 6 files changed, 24 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02006_client_test_hint_error_name.reference create mode 100644 tests/queries/0_stateless/02006_client_test_hint_error_name.sql create mode 100644 tests/queries/0_stateless/02006_client_test_hint_no_such_error_name.reference create mode 100755 tests/queries/0_stateless/02006_client_test_hint_no_such_error_name.sh diff --git a/programs/client/TestHint.cpp b/programs/client/TestHint.cpp index 6149135b597..7eb6575792a 100644 --- a/programs/client/TestHint.cpp +++ b/programs/client/TestHint.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include namespace DB @@ -23,10 +24,18 @@ int parseErrorCode(std::stringstream & ss) // STYLE_CHECK_ALLOW_STD_STRING_STREA using namespace DB; int code; + String code_name; + ss >> code; if (ss.fail()) - throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, - "Expected integer value for test hint, got: '{}'", ss.str()); + { + ss.clear(); + ss >> code_name; + if (ss.fail()) + throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, + "Cannot parse test hint '{}'", ss.str()); + return ErrorCodes::getErrorCodeByName(code_name); + } return code; } diff --git a/programs/client/TestHint.h b/programs/client/TestHint.h index fcf50002c16..377637d0db8 100644 --- a/programs/client/TestHint.h +++ b/programs/client/TestHint.h @@ -15,6 +15,10 @@ namespace DB /// /// - "-- { clientError 20 }" -- in case of you are expecting client error. /// +/// - "-- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO }" -- by error name. +/// +/// - "-- { clientError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO }" -- by error name. +/// /// Remember that the client parse the query first (not the server), so for /// example if you are expecting syntax error, then you should use /// clientError not serverError. diff --git a/tests/queries/0_stateless/02006_client_test_hint_error_name.reference b/tests/queries/0_stateless/02006_client_test_hint_error_name.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02006_client_test_hint_error_name.sql b/tests/queries/0_stateless/02006_client_test_hint_error_name.sql new file mode 100644 index 00000000000..268406a8054 --- /dev/null +++ b/tests/queries/0_stateless/02006_client_test_hint_error_name.sql @@ -0,0 +1 @@ +select throwIf(1); -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } diff --git a/tests/queries/0_stateless/02006_client_test_hint_no_such_error_name.reference b/tests/queries/0_stateless/02006_client_test_hint_no_such_error_name.reference new file mode 100644 index 00000000000..40bea919c93 --- /dev/null +++ b/tests/queries/0_stateless/02006_client_test_hint_no_such_error_name.reference @@ -0,0 +1 @@ +No error code with name: 'FOOBAR'. (NO_SUCH_ERROR_CODE) diff --git a/tests/queries/0_stateless/02006_client_test_hint_no_such_error_name.sh b/tests/queries/0_stateless/02006_client_test_hint_no_such_error_name.sh new file mode 100755 index 00000000000..15ede23a510 --- /dev/null +++ b/tests/queries/0_stateless/02006_client_test_hint_no_such_error_name.sh @@ -0,0 +1,7 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT --testmode -n -q 'select 1 -- { clientError FOOBAR }' |& grep -o 'No error code with name:.*' From 86e14f192c7addcc694d945712b787c3e7645c5b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 8 Aug 2021 20:08:36 +0300 Subject: [PATCH 156/161] Convert std::stringstream to ReadBufferFromString for TestHint parsing --- programs/client/TestHint.cpp | 53 ++++++++++++++---------------------- 1 file changed, 21 insertions(+), 32 deletions(-) diff --git a/programs/client/TestHint.cpp b/programs/client/TestHint.cpp index 7eb6575792a..2f3be2a5350 100644 --- a/programs/client/TestHint.cpp +++ b/programs/client/TestHint.cpp @@ -1,42 +1,30 @@ #include "TestHint.h" -#include -#include #include #include +#include +#include #include -namespace DB -{ - -namespace ErrorCodes -{ - extern const int CANNOT_PARSE_TEXT; -} - -} - namespace { -int parseErrorCode(std::stringstream & ss) // STYLE_CHECK_ALLOW_STD_STRING_STREAM +/// Parse error as number or as a string (name of the error code const) +int parseErrorCode(DB::ReadBufferFromString & in) { - using namespace DB; - - int code; + int code = -1; String code_name; - ss >> code; - if (ss.fail()) + auto * pos = in.position(); + tryReadText(code, in); + if (pos != in.position()) { - ss.clear(); - ss >> code_name; - if (ss.fail()) - throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, - "Cannot parse test hint '{}'", ss.str()); - return ErrorCodes::getErrorCodeByName(code_name); + return code; } - return code; + + /// Try parse as string + readStringUntilWhitespace(code_name, in); + return DB::ErrorCodes::getErrorCodeByName(code_name); } } @@ -86,22 +74,23 @@ TestHint::TestHint(bool enabled_, const String & query_) void TestHint::parse(const String & hint, bool is_leading_hint) { - std::stringstream ss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM - ss << hint; + ReadBufferFromString in(hint); String item; - while (!ss.eof()) + while (!in.eof()) { - ss >> item; - if (ss.eof()) + readStringUntilWhitespace(item, in); + if (in.eof()) break; + skipWhitespaceIfAny(in); + if (!is_leading_hint) { if (item == "serverError") - server_error = parseErrorCode(ss); + server_error = parseErrorCode(in); else if (item == "clientError") - client_error = parseErrorCode(ss); + client_error = parseErrorCode(in); } if (item == "echo") From 5c56d3a7344615921b952f388caa779b0f245e22 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 16 Aug 2021 03:01:30 +0300 Subject: [PATCH 157/161] Testflows are broken --- tests/testflows/regression.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/testflows/regression.py b/tests/testflows/regression.py index ba2ea3b111c..bae1c5db90a 100755 --- a/tests/testflows/regression.py +++ b/tests/testflows/regression.py @@ -22,7 +22,7 @@ def regression(self, local, clickhouse_binary_path, stress=None): # run_scenario(pool, tasks, Feature(test=load("ldap.regression", "regression")), args) # run_scenario(pool, tasks, Feature(test=load("rbac.regression", "regression")), args) Feature(test=load("aes_encryption.regression", "regression"), parallel=True, executor=pool)(**args) - Feature(test=load("map_type.regression", "regression"), parallel=True, executor=pool)(**args) + # Feature(test=load("map_type.regression", "regression"), parallel=True, executor=pool)(**args) Feature(test=load("window_functions.regression", "regression"), parallel=True, executor=pool)(**args) Feature(test=load("datetime64_extended_range.regression", "regression"), parallel=True, executor=pool)(**args) Feature(test=load("kerberos.regression", "regression"), parallel=True, executor=pool)(**args) From 9a4a8d39439314d563c67e334333bf9a16fecd08 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 16 Aug 2021 03:16:45 +0300 Subject: [PATCH 158/161] Fix build with clang-13 --- base/glibc-compatibility/CMakeLists.txt | 12 +++--------- cmake/add_warning.cmake | 19 +++++++++++++++++++ 2 files changed, 22 insertions(+), 9 deletions(-) diff --git a/base/glibc-compatibility/CMakeLists.txt b/base/glibc-compatibility/CMakeLists.txt index 8cba91de33f..4fc2a002cd8 100644 --- a/base/glibc-compatibility/CMakeLists.txt +++ b/base/glibc-compatibility/CMakeLists.txt @@ -9,10 +9,6 @@ if (GLIBC_COMPATIBILITY) check_include_file("sys/random.h" HAVE_SYS_RANDOM_H) - if(COMPILER_CLANG) - set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -Wno-builtin-requires-header") - endif() - add_headers_and_sources(glibc_compatibility .) add_headers_and_sources(glibc_compatibility musl) if (ARCH_AARCH64) @@ -35,11 +31,9 @@ if (GLIBC_COMPATIBILITY) add_library(glibc-compatibility STATIC ${glibc_compatibility_sources}) - if (COMPILER_CLANG) - target_compile_options(glibc-compatibility PRIVATE -Wno-unused-command-line-argument) - elseif (COMPILER_GCC) - target_compile_options(glibc-compatibility PRIVATE -Wno-unused-but-set-variable) - endif () + target_no_warning(glibc-compatibility unused-command-line-argument) + target_no_warning(glibc-compatibility unused-but-set-variable) + target_no_warning(glibc-compatibility builtin-requires-header) target_include_directories(glibc-compatibility PRIVATE libcxxabi ${musl_arch_include_dir}) diff --git a/cmake/add_warning.cmake b/cmake/add_warning.cmake index 3a776c98ab6..bc9642c9cc6 100644 --- a/cmake/add_warning.cmake +++ b/cmake/add_warning.cmake @@ -27,3 +27,22 @@ endmacro () macro (no_warning flag) add_warning(no-${flag}) endmacro () + + +# The same but only for specified target. +macro (target_add_warning target flag) + string (REPLACE "-" "_" underscored_flag ${flag}) + string (REPLACE "+" "x" underscored_flag ${underscored_flag}) + + check_cxx_compiler_flag("-W${flag}" SUPPORTS_CXXFLAG_${underscored_flag}) + + if (SUPPORTS_CXXFLAG_${underscored_flag}) + target_compile_options (${target} PRIVATE "-W${flag}") + else () + message (WARNING "Flag -W${flag} is unsupported") + endif () +endmacro () + +macro (target_no_warning target flag) + target_add_warning(${target} no-${flag}) +endmacro () From d417bd32474a4e659159b7025c7333c80f9dc467 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 16 Aug 2021 18:56:14 +0800 Subject: [PATCH 159/161] remove duplicated source files in arrow-cmake --- contrib/arrow-cmake/CMakeLists.txt | 3 --- 1 file changed, 3 deletions(-) diff --git a/contrib/arrow-cmake/CMakeLists.txt b/contrib/arrow-cmake/CMakeLists.txt index 2c72055a3e7..427379dc9b2 100644 --- a/contrib/arrow-cmake/CMakeLists.txt +++ b/contrib/arrow-cmake/CMakeLists.txt @@ -119,12 +119,9 @@ set(ORC_SRCS "${ORC_SOURCE_SRC_DIR}/ColumnWriter.cc" "${ORC_SOURCE_SRC_DIR}/Common.cc" "${ORC_SOURCE_SRC_DIR}/Compression.cc" - "${ORC_SOURCE_SRC_DIR}/Exceptions.cc" "${ORC_SOURCE_SRC_DIR}/Int128.cc" "${ORC_SOURCE_SRC_DIR}/LzoDecompressor.cc" "${ORC_SOURCE_SRC_DIR}/MemoryPool.cc" - "${ORC_SOURCE_SRC_DIR}/OrcFile.cc" - "${ORC_SOURCE_SRC_DIR}/Reader.cc" "${ORC_SOURCE_SRC_DIR}/RLE.cc" "${ORC_SOURCE_SRC_DIR}/RLEv1.cc" "${ORC_SOURCE_SRC_DIR}/RLEv2.cc" From 7ee2dc04a1f67963e7e32db9873031d325a0c2bb Mon Sep 17 00:00:00 2001 From: Alexey Boykov <33257111+mathalex@users.noreply.github.com> Date: Mon, 16 Aug 2021 15:06:44 +0300 Subject: [PATCH 160/161] Update parametric-functions.md typo --- .../sql-reference/aggregate-functions/parametric-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/aggregate-functions/parametric-functions.md b/docs/ru/sql-reference/aggregate-functions/parametric-functions.md index b1eefc3fc16..b3bb611e28c 100644 --- a/docs/ru/sql-reference/aggregate-functions/parametric-functions.md +++ b/docs/ru/sql-reference/aggregate-functions/parametric-functions.md @@ -172,7 +172,7 @@ SELECT sequenceMatch('(?1)(?2)')(time, number = 1, number = 2, number = 4) FROM ## sequenceCount(pattern)(time, cond1, cond2, …) {#function-sequencecount} -Вычисляет количество цепочек событий, соответствующих шаблону. Функция обнаруживает только непересекающиеся цепочки событий. Она начитает искать следующую цепочку только после того, как полностью совпала текущая цепочка событий. +Вычисляет количество цепочек событий, соответствующих шаблону. Функция обнаруживает только непересекающиеся цепочки событий. Она начинает искать следующую цепочку только после того, как полностью совпала текущая цепочка событий. !!! warning "Предупреждение" События, произошедшие в одну и ту же секунду, располагаются в последовательности в неопределенном порядке, что может повлиять на результат работы функции. From 1f48166bfb94f5ef215cbb7f6bf774d45daa1966 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Mon, 16 Aug 2021 16:55:20 +0300 Subject: [PATCH 161/161] Update run-fuzzer.sh --- docker/test/fuzzer/run-fuzzer.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/fuzzer/run-fuzzer.sh b/docker/test/fuzzer/run-fuzzer.sh index 3b074ae46cd..44183a50ae5 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -226,7 +226,7 @@ continue task_exit_code=$fuzzer_exit_code echo "failure" > status.txt { grep --text -o "Found error:.*" fuzzer.log \ - || grep --text -o "Exception.*" fuzzer.log \ + || grep --text -ao "Exception:.*" fuzzer.log \ || echo "Fuzzer failed ($fuzzer_exit_code). See the logs." ; } \ | tail -1 > description.txt fi