From 2ce657196880d37cf0d891d6a930aef17aee1685 Mon Sep 17 00:00:00 2001 From: Oleg Taizov Date: Sat, 12 Mar 2022 11:09:12 +0300 Subject: [PATCH 001/101] `kafka_num_consumers` prop upper bound doc update Add clarification about the upper bound of `kafka_num_consumers` property that was added in [#26640](https://github.com/ClickHouse/ClickHouse/pull/26642) --- docs/en/engines/table-engines/integrations/kafka.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/integrations/kafka.md b/docs/en/engines/table-engines/integrations/kafka.md index 1d80f143098..3eb00bad33b 100644 --- a/docs/en/engines/table-engines/integrations/kafka.md +++ b/docs/en/engines/table-engines/integrations/kafka.md @@ -47,7 +47,7 @@ Optional parameters: - `kafka_row_delimiter` — Delimiter character, which ends the message. - `kafka_schema` — Parameter that must be used if the format requires a schema definition. For example, [Cap’n Proto](https://capnproto.org/) requires the path to the schema file and the name of the root `schema.capnp:Message` object. -- `kafka_num_consumers` — The number of consumers per table. Default: `1`. Specify more consumers if the throughput of one consumer is insufficient. The total number of consumers should not exceed the number of partitions in the topic, since only one consumer can be assigned per partition. +- `kafka_num_consumers` — The number of consumers per table. Default: `1`. Specify more consumers if the throughput of one consumer is insufficient. The total number of consumers should not exceed the number of partitions in the topic, since only one consumer can be assigned per partition, and must not be greater than the number of physical cores on the server where ClickHouse is deployed. - `kafka_max_block_size` — The maximum batch size (in messages) for poll (default: `max_block_size`). - `kafka_skip_broken_messages` — Kafka message parser tolerance to schema-incompatible messages per block. Default: `0`. If `kafka_skip_broken_messages = N` then the engine skips *N* Kafka messages that cannot be parsed (a message equals a row of data). - `kafka_commit_every_batch` — Commit every consumed and handled batch instead of a single commit after writing a whole block (default: `0`). From 7bb66e6702ae1d12b0a4cd58b033ff830e9631a6 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 17 Mar 2022 01:51:35 -0400 Subject: [PATCH 002/101] added INTERPOLATE extension for ORDER BY WITH FILL --- src/Core/InterpolateDescription.cpp | 61 ++++++++++++++ src/Core/InterpolateDescription.h | 79 +++++++++++++++++++ src/Interpreters/FillingRow.cpp | 57 +++++++------ src/Interpreters/FillingRow.h | 27 ++++++- src/Interpreters/InterpreterSelectQuery.cpp | 21 ++++- src/Parsers/ASTInterpolateElement.cpp | 15 ++++ src/Parsers/ASTInterpolateElement.h | 29 +++++++ src/Parsers/ASTOrderByElement.h | 1 + src/Parsers/ASTSelectQuery.h | 8 +- src/Parsers/ExpressionElementParsers.cpp | 27 +++++++ src/Parsers/ExpressionElementParsers.h | 9 +++ src/Parsers/ExpressionListParsers.cpp | 7 ++ src/Parsers/ExpressionListParsers.h | 6 ++ src/Parsers/ParserSelectQuery.cpp | 20 +++++ src/Processors/QueryPlan/FillingStep.cpp | 11 ++- src/Processors/QueryPlan/FillingStep.h | 4 +- .../Transforms/FillingTransform.cpp | 56 ++++++++++--- src/Processors/Transforms/FillingTransform.h | 8 +- 18 files changed, 398 insertions(+), 48 deletions(-) create mode 100644 src/Core/InterpolateDescription.cpp create mode 100644 src/Core/InterpolateDescription.h create mode 100644 src/Parsers/ASTInterpolateElement.cpp create mode 100644 src/Parsers/ASTInterpolateElement.h diff --git a/src/Core/InterpolateDescription.cpp b/src/Core/InterpolateDescription.cpp new file mode 100644 index 00000000000..1f5c343fddd --- /dev/null +++ b/src/Core/InterpolateDescription.cpp @@ -0,0 +1,61 @@ +#include +#include +#include +#include + +namespace DB +{ + +void dumpInterpolateDescription(const InterpolateDescription & description, const Block & /*header*/, WriteBuffer & out) +{ + bool first = true; + + for (const auto & desc : description) + { + if (!first) + out << ", "; + first = false; + + if (desc.column.name.empty()) + out << "?"; + else + out << desc.column.name; + } +} + +void InterpolateColumnDescription::interpolate(Field & field) const +{ + if(field.isNull()) + return; + Block expr_columns; + expr_columns.insert({column.type->createColumnConst(1, field), column.type, column.name}); + actions->execute(expr_columns); + expr_columns.getByPosition(0).column->get(0, field); +} + +void InterpolateColumnDescription::explain(JSONBuilder::JSONMap & map, const Block & /*header*/) const +{ + map.add("Column", column.name); +} + +std::string dumpInterpolateDescription(const InterpolateDescription & description) +{ + WriteBufferFromOwnString wb; + dumpInterpolateDescription(description, Block{}, wb); + return wb.str(); +} + +JSONBuilder::ItemPtr explainInterpolateDescription(const InterpolateDescription & description, const Block & header) +{ + auto json_array = std::make_unique(); + for (const auto & descr : description) + { + auto json_map = std::make_unique(); + descr.explain(*json_map, header); + json_array->add(std::move(json_map)); + } + + return json_array; +} + +} diff --git a/src/Core/InterpolateDescription.h b/src/Core/InterpolateDescription.h new file mode 100644 index 00000000000..7cd9f86890e --- /dev/null +++ b/src/Core/InterpolateDescription.h @@ -0,0 +1,79 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +class Collator; + +namespace DB +{ + +namespace JSONBuilder +{ + class JSONMap; + class IItem; + using ItemPtr = std::unique_ptr; +} + +class Block; + + +/// Interpolate description +struct InterpolateColumnDescription +{ + using Signature = ExecutableFunctionExpression::Signature; + + ColumnWithTypeAndName column; + ExpressionActionsPtr actions; + // ExecutableFunctionExpression interpolate_func; + // std::string column_name; /// The name of the column. + // size_t column_number; /// Column number (used if no name is given). + + + + explicit InterpolateColumnDescription(const ColumnWithTypeAndName & column_, ExpressionActionsPtr actions_) : + column(column_), actions(actions_) {} + + // explicit InterpolateColumnDescription(size_t column_number_, const ASTInterpolateElement & /*ast*/) : column_number(column_number_) {} + + // explicit InterpolateColumnDescription(const std::string & column_name_, const ASTInterpolateElement & /*ast*/) : column_name(column_name_), column_number(0) {} + + bool operator == (const InterpolateColumnDescription & other) const + { + return column == other.column;// && column_number == other.column_number; + } + + bool operator != (const InterpolateColumnDescription & other) const + { + return !(*this == other); + } + + void interpolate(Field & field) const; + + std::string dump() const + { + return fmt::format("{}", column.name); + } + + void explain(JSONBuilder::JSONMap & map, const Block & header) const; +}; + +/// Description of interpolation for several columns. +using InterpolateDescription = std::vector; + +/// Outputs user-readable description into `out`. +void dumpInterpolateDescription(const InterpolateDescription & description, const Block & header, WriteBuffer & out); + +std::string dumpInterpolateDescription(const InterpolateDescription & description); + +JSONBuilder::ItemPtr explainInterpolateDescription(const InterpolateDescription & description, const Block & header); + +} diff --git a/src/Interpreters/FillingRow.cpp b/src/Interpreters/FillingRow.cpp index 94f185a44cc..8cc918d2c72 100644 --- a/src/Interpreters/FillingRow.cpp +++ b/src/Interpreters/FillingRow.cpp @@ -19,26 +19,30 @@ bool equals(const Field & lhs, const Field & rhs) } -FillingRow::FillingRow(const SortDescription & sort_description) : description(sort_description) +FillingRow::FillingRow(const SortDescription & sort_description_, const InterpolateDescription & interpolate_description_) + : sort{*this} + , interpolate{*this} + , sort_description(sort_description_) + , interpolate_description(interpolate_description_) { - row.resize(description.size()); + row.resize(sort_description.size() + interpolate_description.size()); } bool FillingRow::operator<(const FillingRow & other) const { - for (size_t i = 0; i < size(); ++i) + for (size_t i = 0; i < sort.size(); ++i) { - if (row[i].isNull() || other[i].isNull() || equals(row[i], other[i])) + if (sort[i].isNull() || other.sort[i].isNull() || equals(sort[i], other.sort[i])) continue; - return less(row[i], other[i], getDirection(i)); + return less(sort[i], other.sort[i], getDirection(i)); } return false; } bool FillingRow::operator==(const FillingRow & other) const { - for (size_t i = 0; i < size(); ++i) - if (!equals(row[i], other[i])) + for (size_t i = 0; i < sort.size(); ++i) + if (!equals(sort[i], other.sort[i])) return false; return true; } @@ -47,49 +51,54 @@ bool FillingRow::next(const FillingRow & to_row) { size_t pos = 0; + for(size_t i = 0; i < to_row.interpolate.size(); ++i) { + std::cout << to_row.interpolate[i] <<" : "; + interpolate[i] = to_row.interpolate[i]; + } + /// Find position we need to increment for generating next row. - for (; pos < row.size(); ++pos) - if (!row[pos].isNull() && !to_row[pos].isNull() && !equals(row[pos], to_row[pos])) + for (; pos < sort.size(); ++pos) + if (!sort[pos].isNull() && !to_row.sort[pos].isNull() && !equals(sort[pos], to_row.sort[pos])) break; - if (pos == row.size() || less(to_row[pos], row[pos], getDirection(pos))) + if (pos == sort.size() || less(to_row.sort[pos], sort[pos], getDirection(pos))) return false; /// If we have any 'fill_to' value at position greater than 'pos', /// we need to generate rows up to 'fill_to' value. - for (size_t i = row.size() - 1; i > pos; --i) + for (size_t i = sort.size() - 1; i > pos; --i) { - if (getFillDescription(i).fill_to.isNull() || row[i].isNull()) + if (getFillDescription(i).fill_to.isNull() || sort[i].isNull()) continue; - auto next_value = row[i]; + auto next_value = sort[i]; getFillDescription(i).step_func(next_value); if (less(next_value, getFillDescription(i).fill_to, getDirection(i))) { - row[i] = next_value; + sort[i] = next_value; initFromDefaults(i + 1); return true; } } - auto next_value = row[pos]; + auto next_value = sort[pos]; getFillDescription(pos).step_func(next_value); - if (less(to_row[pos], next_value, getDirection(pos))) + if (less(to_row.sort[pos], next_value, getDirection(pos))) return false; - row[pos] = next_value; - if (equals(row[pos], to_row[pos])) + sort[pos] = next_value; + if (equals(sort[pos], to_row.sort[pos])) { bool is_less = false; - for (size_t i = pos + 1; i < size(); ++i) + for (size_t i = pos + 1; i < sort.size(); ++i) { const auto & fill_from = getFillDescription(i).fill_from; if (!fill_from.isNull()) - row[i] = fill_from; + sort[i] = fill_from; else - row[i] = to_row[i]; - is_less |= less(row[i], to_row[i], getDirection(i)); + sort[i] = to_row.sort[i]; + is_less |= less(sort[i], to_row.sort[i], getDirection(i)); } return is_less; @@ -101,8 +110,8 @@ bool FillingRow::next(const FillingRow & to_row) void FillingRow::initFromDefaults(size_t from_pos) { - for (size_t i = from_pos; i < row.size(); ++i) - row[i] = getFillDescription(i).fill_from; + for (size_t i = from_pos; i < sort.size(); ++i) + sort[i] = getFillDescription(i).fill_from; } diff --git a/src/Interpreters/FillingRow.h b/src/Interpreters/FillingRow.h index 604f4b1ee74..6081effe703 100644 --- a/src/Interpreters/FillingRow.h +++ b/src/Interpreters/FillingRow.h @@ -1,5 +1,6 @@ #pragma once #include +#include #include @@ -17,7 +18,23 @@ bool equals(const Field & lhs, const Field & rhs); class FillingRow { public: - FillingRow(const SortDescription & sort_description); + struct { + FillingRow & filling_row; + + Field & operator[](size_t index) { return filling_row.row[index]; } + const Field & operator[](size_t index) const { return filling_row.row[index]; } + size_t size() const { return filling_row.sort_description.size(); } + } sort; + + struct { + FillingRow & filling_row; + + Field & operator[](size_t index) { return filling_row.row[filling_row.sort_description.size() + index]; } + const Field & operator[](size_t index) const { return filling_row.row[filling_row.sort_description.size() + index]; } + size_t size() const { return filling_row.interpolate_description.size(); } + } interpolate; +public: + FillingRow(const SortDescription & sort_description, const InterpolateDescription & interpolate_description); /// Generates next row according to fill 'from', 'to' and 'step' values. bool next(const FillingRow & to_row); @@ -30,12 +47,14 @@ public: bool operator<(const FillingRow & other) const; bool operator==(const FillingRow & other) const; - int getDirection(size_t index) const { return description[index].direction; } - FillColumnDescription & getFillDescription(size_t index) { return description[index].fill_description; } + int getDirection(size_t index) const { return sort_description[index].direction; } + FillColumnDescription & getFillDescription(size_t index) { return sort_description[index].fill_description; } + InterpolateColumnDescription & getInterpolateDescription(size_t index) { return interpolate_description[index]; } private: Row row; - SortDescription description; + SortDescription sort_description; + InterpolateDescription interpolate_description; }; void insertFromFillingRow(MutableColumns & filling_columns, MutableColumns & other_columns, const FillingRow & filling_row); diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index f2fc17fbf9a..6a4456b1caf 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -827,6 +828,23 @@ static SortDescription getSortDescription(const ASTSelectQuery & query, ContextP return order_descr; } +static InterpolateDescription getInterpolateDescription(const ASTSelectQuery & query, Block block, ContextPtr context) +{ + InterpolateDescription interpolate_descr; + interpolate_descr.reserve(query.interpolate()->children.size()); + + for (const auto & elem : query.interpolate()->children) + { + auto interpolate = elem->as(); + auto syntax_result = TreeRewriter(context).analyze(interpolate.expr, block.getNamesAndTypesList()); + ExpressionAnalyzer analyzer(interpolate.expr, syntax_result, context); + ExpressionActionsPtr actions = analyzer.getActions(true, true, CompileExpressions::yes); + interpolate_descr.emplace_back(block.findByName(interpolate.column->getColumnName())->cloneEmpty(), actions); + } + + return interpolate_descr; +} + static SortDescription getSortDescriptionFromGroupBy(const ASTSelectQuery & query) { SortDescription order_descr; @@ -2498,7 +2516,8 @@ void InterpreterSelectQuery::executeWithFill(QueryPlan & query_plan) if (fill_descr.empty()) return; - auto filling_step = std::make_unique(query_plan.getCurrentDataStream(), std::move(fill_descr)); + InterpolateDescription interpolate_descr = getInterpolateDescription(query, source_header, context); + auto filling_step = std::make_unique(query_plan.getCurrentDataStream(), std::move(fill_descr), std::move(interpolate_descr)); query_plan.addStep(std::move(filling_step)); } } diff --git a/src/Parsers/ASTInterpolateElement.cpp b/src/Parsers/ASTInterpolateElement.cpp new file mode 100644 index 00000000000..14aa5f7e672 --- /dev/null +++ b/src/Parsers/ASTInterpolateElement.cpp @@ -0,0 +1,15 @@ +#include +#include +#include +#include + + +namespace DB +{ + +void ASTInterpolateElement::formatImpl(const FormatSettings & /*settings*/, FormatState & /*state*/, FormatStateStacked /*frame*/) const +{ + +} + +} diff --git a/src/Parsers/ASTInterpolateElement.h b/src/Parsers/ASTInterpolateElement.h new file mode 100644 index 00000000000..e23b10cd434 --- /dev/null +++ b/src/Parsers/ASTInterpolateElement.h @@ -0,0 +1,29 @@ +#pragma once + +#include + + +namespace DB +{ + +class ASTInterpolateElement : public IAST +{ +public: + ASTPtr column; + ASTPtr expr; + + String getID(char) const override { return "InterpolateElement"; } + + ASTPtr clone() const override + { + auto clone = std::make_shared(*this); + clone->cloneChildren(); + return clone; + } + + +protected: + void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; +}; + +} diff --git a/src/Parsers/ASTOrderByElement.h b/src/Parsers/ASTOrderByElement.h index 8c868312834..4d07405c17a 100644 --- a/src/Parsers/ASTOrderByElement.h +++ b/src/Parsers/ASTOrderByElement.h @@ -37,4 +37,5 @@ public: protected: void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; }; + } diff --git a/src/Parsers/ASTSelectQuery.h b/src/Parsers/ASTSelectQuery.h index 1c631783fdb..4a30d6afee3 100644 --- a/src/Parsers/ASTSelectQuery.h +++ b/src/Parsers/ASTSelectQuery.h @@ -32,7 +32,8 @@ public: LIMIT_BY, LIMIT_OFFSET, LIMIT_LENGTH, - SETTINGS + SETTINGS, + INTERPOLATE }; static String expressionToString(Expression expr) @@ -69,6 +70,8 @@ public: return "LIMIT LENGTH"; case Expression::SETTINGS: return "SETTINGS"; + case Expression::INTERPOLATE: + return "INTERPOLATE"; } return ""; } @@ -98,7 +101,7 @@ public: const ASTPtr where() const { return getExpression(Expression::WHERE); } const ASTPtr groupBy() const { return getExpression(Expression::GROUP_BY); } const ASTPtr having() const { return getExpression(Expression::HAVING); } - const ASTPtr window() const { return getExpression(Expression::WINDOW); } + const ASTPtr window() const { return getExpression(Expression::WINDOW); } const ASTPtr orderBy() const { return getExpression(Expression::ORDER_BY); } const ASTPtr limitByOffset() const { return getExpression(Expression::LIMIT_BY_OFFSET); } const ASTPtr limitByLength() const { return getExpression(Expression::LIMIT_BY_LENGTH); } @@ -106,6 +109,7 @@ public: const ASTPtr limitOffset() const { return getExpression(Expression::LIMIT_OFFSET); } const ASTPtr limitLength() const { return getExpression(Expression::LIMIT_LENGTH); } const ASTPtr settings() const { return getExpression(Expression::SETTINGS); } + const ASTPtr interpolate() const { return getExpression(Expression::INTERPOLATE); } bool hasFiltration() const { return where() || prewhere() || having(); } diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index c51201750c5..b0cd2f5e4db 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include #include #include @@ -2314,6 +2315,32 @@ bool ParserOrderByElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expect return true; } +bool ParserInterpolateElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + ParserKeyword as("AS"); + ParserExpression element_p; + ParserIdentifier ident_p; + + ASTPtr ident; + if (!ident_p.parse(pos, ident, expected)) + return false; + + if (!as.ignore(pos, expected)) + return false; + + ASTPtr expr; + if (!element_p.parse(pos, expr, expected)) + return false; + + auto elem = std::make_shared(); + elem->column = ident; + elem->expr = expr; + + node = elem; + + return true; +} + bool ParserFunctionWithKeyValueArguments::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ParserIdentifier id_parser; diff --git a/src/Parsers/ExpressionElementParsers.h b/src/Parsers/ExpressionElementParsers.h index c86721dca18..10b4f5fd7d1 100644 --- a/src/Parsers/ExpressionElementParsers.h +++ b/src/Parsers/ExpressionElementParsers.h @@ -420,6 +420,15 @@ protected: bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; }; +/** Element of INTERPOLATE expression + */ +class ParserInterpolateElement : public IParserBase +{ +protected: + const char * getName() const override { return "element of INTERPOLATE expression"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + /** Parser for function with arguments like KEY VALUE (space separated) * no commas allowed, just space-separated pairs. */ diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index 13af308736b..1396e86df39 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -757,6 +757,13 @@ bool ParserOrderByExpressionList::parseImpl(Pos & pos, ASTPtr & node, Expected & } +bool ParserInterpolateExpressionList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + return ParserList(std::make_unique(), std::make_unique(TokenType::Comma), false) + .parse(pos, node, expected); +} + + bool ParserTTLExpressionList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { return ParserList(std::make_unique(), std::make_unique(TokenType::Comma), false) diff --git a/src/Parsers/ExpressionListParsers.h b/src/Parsers/ExpressionListParsers.h index 358fe778f91..d81aa8de380 100644 --- a/src/Parsers/ExpressionListParsers.h +++ b/src/Parsers/ExpressionListParsers.h @@ -517,6 +517,12 @@ protected: bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; }; +class ParserInterpolateExpressionList : public IParserBase +{ +protected: + const char * getName() const override { return "interpolate expression"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; /// Parser for key-value pair, where value can be list of pairs. class ParserKeyValuePair : public IParserBase diff --git a/src/Parsers/ParserSelectQuery.cpp b/src/Parsers/ParserSelectQuery.cpp index 90ab5911d6b..85f08ad11b0 100644 --- a/src/Parsers/ParserSelectQuery.cpp +++ b/src/Parsers/ParserSelectQuery.cpp @@ -10,6 +10,7 @@ #include #include #include +#include namespace DB @@ -59,12 +60,14 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserKeyword s_rows("ROWS"); ParserKeyword s_first("FIRST"); ParserKeyword s_next("NEXT"); + ParserKeyword s_interpolate("INTERPOLATE"); ParserNotEmptyExpressionList exp_list(false); ParserNotEmptyExpressionList exp_list_for_with_clause(false); ParserNotEmptyExpressionList exp_list_for_select_clause(true); /// Allows aliases without AS keyword. ParserExpressionWithOptionalAlias exp_elem(false); ParserOrderByExpressionList order_list; + ParserInterpolateExpressionList interpolate_list; ParserToken open_bracket(TokenType::OpeningRoundBracket); ParserToken close_bracket(TokenType::ClosingRoundBracket); @@ -78,6 +81,7 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ASTPtr having_expression; ASTPtr window_list; ASTPtr order_expression_list; + ASTPtr interpolate_expression_list; ASTPtr limit_by_length; ASTPtr limit_by_offset; ASTPtr limit_by_expression_list; @@ -239,6 +243,21 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { if (!order_list.parse(pos, order_expression_list, expected)) return false; + + /// if any WITH FILL parse possible INTERPOLATE list + if ( std::any_of(order_expression_list->children.begin(), order_expression_list->children.end(), + [](auto & child) { return child->template as()->with_fill; }) ) + { + if (s_interpolate.ignore(pos, expected)) + { + if (!open_bracket.ignore(pos, expected)) + return false; + if (!interpolate_list.parse(pos, interpolate_expression_list, expected)) + return false; + if (!close_bracket.ignore(pos, expected)) + return false; + } + } } /// This is needed for TOP expression, because it can also use WITH TIES. @@ -430,6 +449,7 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) select_query->setExpression(ASTSelectQuery::Expression::LIMIT_OFFSET, std::move(limit_offset)); select_query->setExpression(ASTSelectQuery::Expression::LIMIT_LENGTH, std::move(limit_length)); select_query->setExpression(ASTSelectQuery::Expression::SETTINGS, std::move(settings)); + select_query->setExpression(ASTSelectQuery::Expression::INTERPOLATE, std::move(interpolate_expression_list)); return true; } diff --git a/src/Processors/QueryPlan/FillingStep.cpp b/src/Processors/QueryPlan/FillingStep.cpp index 223892aa528..47540bb2725 100644 --- a/src/Processors/QueryPlan/FillingStep.cpp +++ b/src/Processors/QueryPlan/FillingStep.cpp @@ -28,9 +28,9 @@ static ITransformingStep::Traits getTraits() }; } -FillingStep::FillingStep(const DataStream & input_stream_, SortDescription sort_description_) - : ITransformingStep(input_stream_, FillingTransform::transformHeader(input_stream_.header, sort_description_), getTraits()) - , sort_description(std::move(sort_description_)) +FillingStep::FillingStep(const DataStream & input_stream_, SortDescription sort_description_, InterpolateDescription interpolate_description_) + : ITransformingStep(input_stream_, FillingTransform::transformHeader(input_stream_.header, sort_description_/*, interpolate_description_*/), getTraits()) + , sort_description(std::move(sort_description_)), interpolate_description(std::move(interpolate_description_)) { if (!input_stream_.has_single_port) throw Exception("FillingStep expects single input", ErrorCodes::LOGICAL_ERROR); @@ -41,7 +41,7 @@ void FillingStep::transformPipeline(QueryPipelineBuilder & pipeline, const Build pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr { bool on_totals = stream_type == QueryPipelineBuilder::StreamType::Totals; - return std::make_shared(header, sort_description, on_totals); + return std::make_shared(header, sort_description, interpolate_description, on_totals); }); } @@ -50,11 +50,14 @@ void FillingStep::describeActions(FormatSettings & settings) const settings.out << String(settings.offset, ' '); dumpSortDescription(sort_description, input_streams.front().header, settings.out); settings.out << '\n'; + dumpInterpolateDescription(interpolate_description, input_streams.front().header, settings.out); + settings.out << '\n'; } void FillingStep::describeActions(JSONBuilder::JSONMap & map) const { map.add("Sort Description", explainSortDescription(sort_description, input_streams.front().header)); + map.add("Interpolate Description", explainInterpolateDescription(interpolate_description, input_streams.front().header)); } } diff --git a/src/Processors/QueryPlan/FillingStep.h b/src/Processors/QueryPlan/FillingStep.h index 6a5bca1890d..252f1ba3fd5 100644 --- a/src/Processors/QueryPlan/FillingStep.h +++ b/src/Processors/QueryPlan/FillingStep.h @@ -1,6 +1,7 @@ #pragma once #include #include +#include namespace DB { @@ -9,7 +10,7 @@ namespace DB class FillingStep : public ITransformingStep { public: - FillingStep(const DataStream & input_stream_, SortDescription sort_description_); + FillingStep(const DataStream & input_stream_, SortDescription sort_description_, InterpolateDescription interpolate_description_); String getName() const override { return "Filling"; } @@ -22,6 +23,7 @@ public: private: SortDescription sort_description; + InterpolateDescription interpolate_description; }; } diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 1276157cc91..6113b4479a8 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -18,7 +18,7 @@ namespace ErrorCodes extern const int INVALID_WITH_FILL_EXPRESSION; } -Block FillingTransform::transformHeader(Block header, const SortDescription & sort_description) +Block FillingTransform::transformHeader(Block header, const SortDescription & sort_description/*, const InterpolateDescription & interpolate_description*/) { NameSet sort_keys; for (const auto & key : sort_description) @@ -140,12 +140,13 @@ static bool tryConvertFields(FillColumnDescription & descr, const DataTypePtr & } FillingTransform::FillingTransform( - const Block & header_, const SortDescription & sort_description_, bool on_totals_) - : ISimpleTransform(header_, transformHeader(header_, sort_description_), true) + const Block & header_, const SortDescription & sort_description_, const InterpolateDescription & interpolate_description_, bool on_totals_) + : ISimpleTransform(header_, transformHeader(header_, sort_description_/*, interpolate_description_*/), true) , sort_description(sort_description_) + , interpolate_description(interpolate_description_) , on_totals(on_totals_) - , filling_row(sort_description_) - , next_row(sort_description_) + , filling_row(sort_description_, interpolate_description_) + , next_row(sort_description_, interpolate_description_) { if (on_totals) return; @@ -173,6 +174,29 @@ FillingTransform::FillingTransform( } } + for (size_t i = 0, size = interpolate_description.size(); i < size; ++i) + { + size_t block_position = header_.getPositionByName(interpolate_description[i].column.name); + is_fill_column[block_position] = true; + fill_column_positions.push_back(block_position); +/* TODO JOO check types? + auto & descr = filling_row.getFillDescription(i); + const auto & type = header_.getByPosition(block_position).type; + + if (!tryConvertFields(descr, type)) + throw Exception("Incompatible types of WITH FILL expression values with column type " + + type->getName(), ErrorCodes::INVALID_WITH_FILL_EXPRESSION); + + if (type->isValueRepresentedByUnsignedInteger() && + ((!descr.fill_from.isNull() && less(descr.fill_from, Field{0}, 1)) || + (!descr.fill_to.isNull() && less(descr.fill_to, Field{0}, 1)))) + { + throw Exception("WITH FILL bound values cannot be negative for unsigned type " + + type->getName(), ErrorCodes::INVALID_WITH_FILL_EXPRESSION); + } +*/ + } + std::set unique_positions; for (auto pos : fill_column_positions) if (!unique_positions.insert(pos).second) @@ -189,8 +213,8 @@ IProcessor::Status FillingTransform::prepare() { should_insert_first = next_row < filling_row || first; - for (size_t i = 0, size = filling_row.size(); i < size; ++i) - next_row[i] = filling_row.getFillDescription(i).fill_to; + for (size_t i = 0, size = filling_row.sort.size(); i < size; ++i) + next_row.sort[i] = filling_row.getFillDescription(i).fill_to; if (first || filling_row < next_row) { @@ -251,7 +275,7 @@ void FillingTransform::transform(Chunk & chunk) if (first) { - for (size_t i = 0; i < filling_row.size(); ++i) + for (size_t i = 0; i < filling_row.sort.size(); ++i) { auto current_value = (*old_fill_columns[i])[0]; const auto & fill_from = filling_row.getFillDescription(i).fill_from; @@ -272,7 +296,7 @@ void FillingTransform::transform(Chunk & chunk) { should_insert_first = next_row < filling_row; - for (size_t i = 0; i < filling_row.size(); ++i) + for (size_t i = 0; i < filling_row.sort.size(); ++i) { auto current_value = (*old_fill_columns[i])[row_ind]; const auto & fill_to = filling_row.getFillDescription(i).fill_to; @@ -288,9 +312,23 @@ void FillingTransform::transform(Chunk & chunk) if (should_insert_first && filling_row < next_row) insertFromFillingRow(res_fill_columns, res_other_columns, filling_row); + /// Update interpolate fields + for (size_t i = filling_row.sort.size(); i < filling_row.size(); ++i) + filling_row.getInterpolateDescription(i - filling_row.sort.size()).interpolate(next_row[i]); + /// Insert generated filling row to block, while it is less than current row in block. while (filling_row.next(next_row)) + { + /// Update interpolate fields + for (size_t i = filling_row.sort.size(); i < filling_row.size(); ++i) + filling_row.getInterpolateDescription(i - filling_row.sort.size()).interpolate(next_row[i]); + insertFromFillingRow(res_fill_columns, res_other_columns, filling_row); + } + + /// Reset interpolate fields + for (size_t i = filling_row.sort.size(); i < filling_row.size(); ++i) + next_row[i] = (*old_fill_columns[i])[row_ind]; copyRowFromColumns(res_fill_columns, old_fill_columns, row_ind); copyRowFromColumns(res_other_columns, old_other_columns, row_ind); diff --git a/src/Processors/Transforms/FillingTransform.h b/src/Processors/Transforms/FillingTransform.h index 7ccebadfb6d..1a8a6c4919c 100644 --- a/src/Processors/Transforms/FillingTransform.h +++ b/src/Processors/Transforms/FillingTransform.h @@ -1,6 +1,7 @@ #pragma once #include #include +#include #include namespace DB @@ -13,13 +14,13 @@ namespace DB class FillingTransform : public ISimpleTransform { public: - FillingTransform(const Block & header_, const SortDescription & sort_description_, bool on_totals_); + FillingTransform(const Block & header_, const SortDescription & sort_description_, const InterpolateDescription & interpolate_description_, bool on_totals_); String getName() const override { return "FillingTransform"; } Status prepare() override; - static Block transformHeader(Block header, const SortDescription & sort_description); + static Block transformHeader(Block header, const SortDescription & sort_description/*, const InterpolateDescription & interpolate_description_*/); protected: void transform(Chunk & Chunk) override; @@ -27,7 +28,8 @@ protected: private: void setResultColumns(Chunk & chunk, MutableColumns & fill_columns, MutableColumns & other_columns) const; - const SortDescription sort_description; /// Contains only rows with WITH FILL. + const SortDescription sort_description; /// Contains only columns with WITH FILL. + const InterpolateDescription interpolate_description; /// Contains INTERPOLATE columns const bool on_totals; /// FillingTransform does nothing on totals. FillingRow filling_row; /// Current row, which is used to fill gaps. From 00700988ee40928fc25d589c4557fdca1fd41924 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 17 Mar 2022 02:31:01 -0400 Subject: [PATCH 003/101] style fix --- src/Core/InterpolateDescription.cpp | 2 +- src/Core/InterpolateDescription.h | 11 +---------- src/Interpreters/FillingRow.cpp | 4 +--- src/Interpreters/FillingRow.h | 6 ++++-- src/Parsers/ASTInterpolateElement.cpp | 2 +- src/Parsers/ExpressionElementParsers.cpp | 2 +- src/Parsers/ParserSelectQuery.cpp | 4 ++-- src/Processors/Transforms/FillingTransform.cpp | 5 ++--- 8 files changed, 13 insertions(+), 23 deletions(-) diff --git a/src/Core/InterpolateDescription.cpp b/src/Core/InterpolateDescription.cpp index 1f5c343fddd..3545880b5bf 100644 --- a/src/Core/InterpolateDescription.cpp +++ b/src/Core/InterpolateDescription.cpp @@ -25,7 +25,7 @@ void dumpInterpolateDescription(const InterpolateDescription & description, cons void InterpolateColumnDescription::interpolate(Field & field) const { - if(field.isNull()) + if (field.isNull()) return; Block expr_columns; expr_columns.insert({column.type->createColumnConst(1, field), column.type, column.name}); diff --git a/src/Core/InterpolateDescription.h b/src/Core/InterpolateDescription.h index 7cd9f86890e..bb2b5febdc9 100644 --- a/src/Core/InterpolateDescription.h +++ b/src/Core/InterpolateDescription.h @@ -33,22 +33,13 @@ struct InterpolateColumnDescription ColumnWithTypeAndName column; ExpressionActionsPtr actions; - // ExecutableFunctionExpression interpolate_func; - // std::string column_name; /// The name of the column. - // size_t column_number; /// Column number (used if no name is given). - - explicit InterpolateColumnDescription(const ColumnWithTypeAndName & column_, ExpressionActionsPtr actions_) : column(column_), actions(actions_) {} - // explicit InterpolateColumnDescription(size_t column_number_, const ASTInterpolateElement & /*ast*/) : column_number(column_number_) {} - - // explicit InterpolateColumnDescription(const std::string & column_name_, const ASTInterpolateElement & /*ast*/) : column_name(column_name_), column_number(0) {} - bool operator == (const InterpolateColumnDescription & other) const { - return column == other.column;// && column_number == other.column_number; + return column == other.column; } bool operator != (const InterpolateColumnDescription & other) const diff --git a/src/Interpreters/FillingRow.cpp b/src/Interpreters/FillingRow.cpp index 8cc918d2c72..873c48702ae 100644 --- a/src/Interpreters/FillingRow.cpp +++ b/src/Interpreters/FillingRow.cpp @@ -51,10 +51,8 @@ bool FillingRow::next(const FillingRow & to_row) { size_t pos = 0; - for(size_t i = 0; i < to_row.interpolate.size(); ++i) { - std::cout << to_row.interpolate[i] <<" : "; + for (size_t i = 0; i < to_row.interpolate.size(); ++i) interpolate[i] = to_row.interpolate[i]; - } /// Find position we need to increment for generating next row. for (; pos < sort.size(); ++pos) diff --git a/src/Interpreters/FillingRow.h b/src/Interpreters/FillingRow.h index 6081effe703..10adb6b72a8 100644 --- a/src/Interpreters/FillingRow.h +++ b/src/Interpreters/FillingRow.h @@ -18,7 +18,8 @@ bool equals(const Field & lhs, const Field & rhs); class FillingRow { public: - struct { + struct + { FillingRow & filling_row; Field & operator[](size_t index) { return filling_row.row[index]; } @@ -26,7 +27,8 @@ public: size_t size() const { return filling_row.sort_description.size(); } } sort; - struct { + struct + { FillingRow & filling_row; Field & operator[](size_t index) { return filling_row.row[filling_row.sort_description.size() + index]; } diff --git a/src/Parsers/ASTInterpolateElement.cpp b/src/Parsers/ASTInterpolateElement.cpp index 14aa5f7e672..07d4321e69e 100644 --- a/src/Parsers/ASTInterpolateElement.cpp +++ b/src/Parsers/ASTInterpolateElement.cpp @@ -7,9 +7,9 @@ namespace DB { +/// TODO JOO void ASTInterpolateElement::formatImpl(const FormatSettings & /*settings*/, FormatState & /*state*/, FormatStateStacked /*frame*/) const { - } } diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index b0cd2f5e4db..dbcfeb8e8b5 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -2331,7 +2331,7 @@ bool ParserInterpolateElement::parseImpl(Pos & pos, ASTPtr & node, Expected & ex ASTPtr expr; if (!element_p.parse(pos, expr, expected)) return false; - + auto elem = std::make_shared(); elem->column = ident; elem->expr = expr; diff --git a/src/Parsers/ParserSelectQuery.cpp b/src/Parsers/ParserSelectQuery.cpp index 85f08ad11b0..6cc00da1db0 100644 --- a/src/Parsers/ParserSelectQuery.cpp +++ b/src/Parsers/ParserSelectQuery.cpp @@ -245,8 +245,8 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) return false; /// if any WITH FILL parse possible INTERPOLATE list - if ( std::any_of(order_expression_list->children.begin(), order_expression_list->children.end(), - [](auto & child) { return child->template as()->with_fill; }) ) + if (std::any_of(order_expression_list->children.begin(), order_expression_list->children.end(), + [](auto & child) { return child->template as()->with_fill; })) { if (s_interpolate.ignore(pos, expected)) { diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 6113b4479a8..ca50c2fa236 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -179,7 +179,7 @@ FillingTransform::FillingTransform( size_t block_position = header_.getPositionByName(interpolate_description[i].column.name); is_fill_column[block_position] = true; fill_column_positions.push_back(block_position); -/* TODO JOO check types? + /* TODO JOO check types? auto & descr = filling_row.getFillDescription(i); const auto & type = header_.getByPosition(block_position).type; @@ -193,8 +193,7 @@ FillingTransform::FillingTransform( { throw Exception("WITH FILL bound values cannot be negative for unsigned type " + type->getName(), ErrorCodes::INVALID_WITH_FILL_EXPRESSION); - } -*/ + } */ } std::set unique_positions; From a8e1671a767d0f346121a8f33b36af48505c9c88 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 18 Mar 2022 16:44:27 -0400 Subject: [PATCH 004/101] type match check for INTERPOLATE expressions added, bugfix, printout fixed --- src/Interpreters/InterpreterSelectQuery.cpp | 22 +++++++++----- src/Parsers/ASTInterpolateElement.cpp | 6 ++-- src/Parsers/ASTSelectQuery.cpp | 9 ++++++ .../Transforms/FillingTransform.cpp | 29 +++++++++++-------- 4 files changed, 44 insertions(+), 22 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 6a4456b1caf..eb7faaf6bb5 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -831,15 +831,21 @@ static SortDescription getSortDescription(const ASTSelectQuery & query, ContextP static InterpolateDescription getInterpolateDescription(const ASTSelectQuery & query, Block block, ContextPtr context) { InterpolateDescription interpolate_descr; - interpolate_descr.reserve(query.interpolate()->children.size()); - - for (const auto & elem : query.interpolate()->children) + if (query.interpolate()) { - auto interpolate = elem->as(); - auto syntax_result = TreeRewriter(context).analyze(interpolate.expr, block.getNamesAndTypesList()); - ExpressionAnalyzer analyzer(interpolate.expr, syntax_result, context); - ExpressionActionsPtr actions = analyzer.getActions(true, true, CompileExpressions::yes); - interpolate_descr.emplace_back(block.findByName(interpolate.column->getColumnName())->cloneEmpty(), actions); + interpolate_descr.reserve(query.interpolate()->children.size()); + + for (const auto & elem : query.interpolate()->children) + { + auto interpolate = elem->as(); + ColumnWithTypeAndName column = block.findByName(interpolate.column->getColumnName())->cloneEmpty(); + + auto syntax_result = TreeRewriter(context).analyze(interpolate.expr, block.getNamesAndTypesList()); + ExpressionAnalyzer analyzer(interpolate.expr, syntax_result, context); + ExpressionActionsPtr actions = analyzer.getActions(true, true, CompileExpressions::yes); + + interpolate_descr.emplace_back(column, actions); + } } return interpolate_descr; diff --git a/src/Parsers/ASTInterpolateElement.cpp b/src/Parsers/ASTInterpolateElement.cpp index 07d4321e69e..3da94f772f2 100644 --- a/src/Parsers/ASTInterpolateElement.cpp +++ b/src/Parsers/ASTInterpolateElement.cpp @@ -7,9 +7,11 @@ namespace DB { -/// TODO JOO -void ASTInterpolateElement::formatImpl(const FormatSettings & /*settings*/, FormatState & /*state*/, FormatStateStacked /*frame*/) const +void ASTInterpolateElement::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { + column->formatImpl(settings, state, frame); + settings.ostr << (settings.hilite ? hilite_keyword : "") << " AS " << (settings.hilite ? hilite_none : ""); + expr->formatImpl(settings, state, frame); } } diff --git a/src/Parsers/ASTSelectQuery.cpp b/src/Parsers/ASTSelectQuery.cpp index 3f40167b1d1..22b34150d7a 100644 --- a/src/Parsers/ASTSelectQuery.cpp +++ b/src/Parsers/ASTSelectQuery.cpp @@ -129,6 +129,15 @@ void ASTSelectQuery::formatImpl(const FormatSettings & s, FormatState & state, F s.one_line ? orderBy()->formatImpl(s, state, frame) : orderBy()->as().formatImplMultiline(s, state, frame); + + if (interpolate()) + { + s.ostr + << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "INTERPOLATE" << (s.hilite ? hilite_none : "") + << " ("; + interpolate()->formatImpl(s, state, frame); + s.ostr << " )"; + } } if (limitByLength()) diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index ca50c2fa236..d9de72de16d 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -179,21 +179,26 @@ FillingTransform::FillingTransform( size_t block_position = header_.getPositionByName(interpolate_description[i].column.name); is_fill_column[block_position] = true; fill_column_positions.push_back(block_position); - /* TODO JOO check types? - auto & descr = filling_row.getFillDescription(i); - const auto & type = header_.getByPosition(block_position).type; - if (!tryConvertFields(descr, type)) - throw Exception("Incompatible types of WITH FILL expression values with column type " - + type->getName(), ErrorCodes::INVALID_WITH_FILL_EXPRESSION); + /// Check column-expression compatibility + auto column = interpolate_description[i].column; + auto exp_type = interpolate_description[i].actions->getActionsDAG().getResultColumns()[0].type; + auto exp_column = exp_type->createColumn(); + exp_column->insertDefault(); - if (type->isValueRepresentedByUnsignedInteger() && - ((!descr.fill_from.isNull() && less(descr.fill_from, Field{0}, 1)) || - (!descr.fill_to.isNull() && less(descr.fill_to, Field{0}, 1)))) + try { - throw Exception("WITH FILL bound values cannot be negative for unsigned type " - + type->getName(), ErrorCodes::INVALID_WITH_FILL_EXPRESSION); - } */ + if (auto exp_field = (*exp_column)[0]; convertFieldToType(exp_field, *column.type).isNull()) + throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, + "Incompatible types of INTERPOLATE expression type {} with column '{}' of type {}", + exp_type->getName(), column.name, column.type->getName()); + } + catch (const Exception &) + { + throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, + "Incompatible types of INTERPOLATE expression type {} with column '{}' of type {}", + exp_type->getName(), column.name, column.type->getName()); + } } std::set unique_positions; From ecf05ecc49547b9dddfeb4877a0db5d0ba0e5b6e Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 19 Mar 2022 02:41:12 -0400 Subject: [PATCH 005/101] tests are added, bugfix --- src/Core/InterpolateDescription.cpp | 6 ++- .../0_stateless/02233_interpolate_1.reference | 52 +++++++++++++++++++ .../0_stateless/02233_interpolate_1.sql | 39 ++++++++++++++ 3 files changed, 95 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02233_interpolate_1.reference create mode 100644 tests/queries/0_stateless/02233_interpolate_1.sql diff --git a/src/Core/InterpolateDescription.cpp b/src/Core/InterpolateDescription.cpp index 3545880b5bf..176909ca929 100644 --- a/src/Core/InterpolateDescription.cpp +++ b/src/Core/InterpolateDescription.cpp @@ -2,6 +2,7 @@ #include #include #include +#include namespace DB { @@ -28,9 +29,10 @@ void InterpolateColumnDescription::interpolate(Field & field) const if (field.isNull()) return; Block expr_columns; - expr_columns.insert({column.type->createColumnConst(1, field), column.type, column.name}); + Field column_field = convertFieldToType(field, *column.type.get()); + expr_columns.insert({column.type->createColumnConst(1, column_field), column.type, column.name}); actions->execute(expr_columns); - expr_columns.getByPosition(0).column->get(0, field); + field = convertFieldToType((*expr_columns.getByPosition(0).column)[0], *column.type.get()); } void InterpolateColumnDescription::explain(JSONBuilder::JSONMap & map, const Block & /*header*/) const diff --git a/tests/queries/0_stateless/02233_interpolate_1.reference b/tests/queries/0_stateless/02233_interpolate_1.reference new file mode 100644 index 00000000000..45377d486f9 --- /dev/null +++ b/tests/queries/0_stateless/02233_interpolate_1.reference @@ -0,0 +1,52 @@ +0 0 +0.5 0 +1 original 1 +1.5 0 +2 0 +2.5 0 +3 0 +3.5 0 +4 original 4 +4.5 0 +5 0 +5.5 0 +7 original 7 +0 0 +0.5 0 +1 original 1 +1.5 42 +2 42 +2.5 42 +3 42 +3.5 42 +4 original 4 +4.5 42 +5 42 +5.5 42 +7 original 7 +0 0 +0.5 0 +1 original 1 +1.5 1 +2 1 +2.5 1 +3 1 +3.5 1 +4 original 4 +4.5 4 +5 4 +5.5 4 +7 original 7 +0 0 +0.5 0 +1 original 1 +1.5 2 +2 3 +2.5 4 +3 5 +3.5 6 +4 original 4 +4.5 5 +5 6 +5.5 7 +7 original 7 diff --git a/tests/queries/0_stateless/02233_interpolate_1.sql b/tests/queries/0_stateless/02233_interpolate_1.sql new file mode 100644 index 00000000000..dacfbd068ae --- /dev/null +++ b/tests/queries/0_stateless/02233_interpolate_1.sql @@ -0,0 +1,39 @@ +# Test WITH FILL without INTERPOLATE +SELECT n, source, inter FROM ( + SELECT toFloat32(number % 10) AS n, 'original' AS source, number as inter FROM numbers(10) WHERE number % 3 = 1 +) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5; + +# Test INTERPOLATE with const +SELECT n, source, inter FROM ( + SELECT toFloat32(number % 10) AS n, 'original' AS source, number as inter FROM numbers(10) WHERE number % 3 = 1 +) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (inter AS 42); + +# Test INTERPOLATE with field value +SELECT n, source, inter FROM ( + SELECT toFloat32(number % 10) AS n, 'original' AS source, number as inter FROM numbers(10) WHERE number % 3 = 1 +) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (inter AS inter); + +# Test INTERPOLATE with expression +SELECT n, source, inter FROM ( + SELECT toFloat32(number % 10) AS n, 'original' AS source, number as inter FROM numbers(10) WHERE number % 3 = 1 +) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (inter AS inter + 1); + +# Test INTERPOLATE with incompatible const - should produce error +SELECT n, source, inter FROM ( + SELECT toFloat32(number % 10) AS n, 'original' AS source, number as inter FROM numbers(10) WHERE number % 3 = 1 +) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (inter AS 'inter'); -- { serverError 475 } + +# Test INTERPOLATE with incompatible expression - should produce error +SELECT n, source, inter FROM ( + SELECT toFloat32(number % 10) AS n, 'original' AS source, number as inter FROM numbers(10) WHERE number % 3 = 1 +) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (inter AS inter||'inter'); -- { serverError 475 } + +# Test INTERPOLATE with column from WITH FILL expression - should produce error +SELECT n, source, inter FROM ( + SELECT toFloat32(number % 10) AS n, 'original' AS source, number as inter FROM numbers(10) WHERE number % 3 = 1 +) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (n AS n); -- { serverError 475 } + +# Test INTERPOLATE with inconsistent column - should produce error +SELECT n, source, inter FROM ( + SELECT toFloat32(number % 10) AS n, 'original' AS source, number as inter FROM numbers(10) WHERE number % 3 = 1 +) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (inter AS source); -- { serverError 475 } From 90888ea95b4a11a792dd9d67761922a1c676d45c Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Fri, 18 Mar 2022 20:18:12 -0400 Subject: [PATCH 006/101] Update index.md INTERPOLATE is added --- docs/en/sql-reference/statements/select/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/select/index.md b/docs/en/sql-reference/statements/select/index.md index 33644133153..0386083b5df 100644 --- a/docs/en/sql-reference/statements/select/index.md +++ b/docs/en/sql-reference/statements/select/index.md @@ -22,7 +22,7 @@ SELECT [DISTINCT [ON (column1, column2, ...)]] expr_list [WHERE expr] [GROUP BY expr_list] [WITH ROLLUP|WITH CUBE] [WITH TOTALS] [HAVING expr] -[ORDER BY expr_list] [WITH FILL] [FROM expr] [TO expr] [STEP expr] +[ORDER BY expr_list] [WITH FILL] [FROM expr] [TO expr] [STEP expr] [INTERPOLATE (expr_list)] [LIMIT [offset_value, ]n BY columns] [LIMIT [n, ]m] [WITH TIES] [SETTINGS ...] From 40c91c3f981d0c7abd7fe4b684da07530bce8b52 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Fri, 18 Mar 2022 20:21:10 -0400 Subject: [PATCH 007/101] Update index.md INTERPOLATE is added --- docs/ru/sql-reference/statements/select/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/statements/select/index.md b/docs/ru/sql-reference/statements/select/index.md index 4a409dc7743..240a0fff7b6 100644 --- a/docs/ru/sql-reference/statements/select/index.md +++ b/docs/ru/sql-reference/statements/select/index.md @@ -20,7 +20,7 @@ SELECT [DISTINCT [ON (column1, column2, ...)]] expr_list [WHERE expr] [GROUP BY expr_list] [WITH ROLLUP|WITH CUBE] [WITH TOTALS] [HAVING expr] -[ORDER BY expr_list] [WITH FILL] [FROM expr] [TO expr] [STEP expr] +[ORDER BY expr_list] [WITH FILL] [FROM expr] [TO expr] [STEP expr] [INTERPOLATE (expr_list)] [LIMIT [offset_value, ]n BY columns] [LIMIT [n, ]m] [WITH TIES] [SETTINGS ...] From 4f892dc9daac9a7704b444b004ceb777a4e57645 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Fri, 18 Mar 2022 20:55:02 -0400 Subject: [PATCH 008/101] Update order-by.md INTERPOLATE is added --- .../statements/select/order-by.md | 60 +++++++++++++++++++ 1 file changed, 60 insertions(+) diff --git a/docs/en/sql-reference/statements/select/order-by.md b/docs/en/sql-reference/statements/select/order-by.md index b24f0213e4e..b7e84173284 100644 --- a/docs/en/sql-reference/statements/select/order-by.md +++ b/docs/en/sql-reference/statements/select/order-by.md @@ -280,6 +280,7 @@ To fill multiple columns, add `WITH FILL` modifier with optional parameters afte ``` sql ORDER BY expr [WITH FILL] [FROM const_expr] [TO const_expr] [STEP const_numeric_expr], ... exprN [WITH FILL] [FROM expr] [TO expr] [STEP numeric_expr] +[INTERPOLATE (col AS expr, ... colN AS exprN)] ``` `WITH FILL` can be applied for fields with Numeric (all kinds of float, decimal, int) or Date/DateTime types. When applied for `String` fields, missed values are filled with empty strings. @@ -287,6 +288,7 @@ When `FROM const_expr` not defined sequence of filling use minimal `expr` field When `TO const_expr` not defined sequence of filling use maximum `expr` field value from `ORDER BY`. When `STEP const_numeric_expr` defined then `const_numeric_expr` interprets `as is` for numeric types, as `days` for Date type, as `seconds` for DateTime type. It also supports [INTERVAL](https://clickhouse.com/docs/en/sql-reference/data-types/special-data-types/interval/) data type representing time and date intervals. When `STEP const_numeric_expr` omitted then sequence of filling use `1.0` for numeric type, `1 day` for Date type and `1 second` for DateTime type. +`INTERPOLATE` can be applied to columns not participating in `ORDER BY WITH FILL`. Such columns are filled based on previous field value by applying `expr`. Example of a query without `WITH FILL`: @@ -483,4 +485,62 @@ Result: └────────────┴────────────┴──────────┘ ``` +Example of a query without `INTERPOLATE`: + +``` sql +SELECT n, source, inter FROM ( + SELECT toFloat32(number % 10) AS n, 'original' AS source, n as inter + FROM numbers(10) WHERE number % 3 = 1 +) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5; +``` + +Result: + +``` text +┌───n─┬─source───┬─inter─┐ +│ 0 │ │ 0 │ +│ 0.5 │ │ 0 │ +│ 1 │ original │ 1 │ +│ 1.5 │ │ 0 │ +│ 2 │ │ 0 │ +│ 2.5 │ │ 0 │ +│ 3 │ │ 0 │ +│ 3.5 │ │ 0 │ +│ 4 │ original │ 4 │ +│ 4.5 │ │ 0 │ +│ 5 │ │ 0 │ +│ 5.5 │ │ 0 │ +│ 7 │ original │ 7 │ +└─────┴──────────┴───────┘ +``` + +Same query after applying `INTERPOLATE`: + +``` sql +SELECT n, source, inter FROM ( + SELECT toFloat32(number % 10) AS n, 'original' AS source, n as inter + FROM numbers(10) WHERE number % 3 = 1 +) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (inter AS inter + 1); +``` + +Result: + +``` text +┌───n─┬─source───┬─inter─┐ +│ 0 │ │ 0 │ +│ 0.5 │ │ 0 │ +│ 1 │ original │ 1 │ +│ 1.5 │ │ 2 │ +│ 2 │ │ 3 │ +│ 2.5 │ │ 4 │ +│ 3 │ │ 5 │ +│ 3.5 │ │ 6 │ +│ 4 │ original │ 4 │ +│ 4.5 │ │ 5 │ +│ 5 │ │ 6 │ +│ 5.5 │ │ 7 │ +│ 7 │ original │ 7 │ +└─────┴──────────┴───────┘ +``` + [Original article](https://clickhouse.com/docs/en/sql-reference/statements/select/order-by/) From f9ed659959cc893a8cb19ef27b010d3aa70db522 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Fri, 18 Mar 2022 21:04:54 -0400 Subject: [PATCH 009/101] Update order-by.md INTERPOLATE is added --- .../statements/select/order-by.md | 58 +++++++++++++++++++ 1 file changed, 58 insertions(+) diff --git a/docs/ru/sql-reference/statements/select/order-by.md b/docs/ru/sql-reference/statements/select/order-by.md index 3f52b260423..71f12ec73b0 100644 --- a/docs/ru/sql-reference/statements/select/order-by.md +++ b/docs/ru/sql-reference/statements/select/order-by.md @@ -280,6 +280,7 @@ SELECT * FROM collate_test ORDER BY s ASC COLLATE 'en'; ```sql ORDER BY expr [WITH FILL] [FROM const_expr] [TO const_expr] [STEP const_numeric_expr], ... exprN [WITH FILL] [FROM expr] [TO expr] [STEP numeric_expr] +[INTERPOLATE (col AS expr, ... colN AS exprN)] ``` `WITH FILL` может быть применен к полям с числовыми (все разновидности float, int, decimal) или временными (все разновидности Date, DateTime) типами. В случае применения к полям типа `String` недостающие значения заполняются пустой строкой. @@ -289,6 +290,8 @@ ORDER BY expr [WITH FILL] [FROM const_expr] [TO const_expr] [STEP const_numeric_ Когда `STEP const_numeric_expr` не указан, тогда используется `1.0` для числовых типов, `1 день` для типа Date и `1 секунда` для типа DateTime. +`INTERPOLATE` может быть применен к колонкам, не участвующим в `ORDER BY WITH FILL`. Такие колонки заполняются значениями, вычисляемыми применением `expr` к предыдущему значению. + Пример запроса без использования `WITH FILL`: ```sql SELECT n, source FROM ( @@ -395,3 +398,58 @@ ORDER BY │ 1970-03-12 │ 1970-01-08 │ original │ └────────────┴────────────┴──────────┘ ``` + +Пример запроса без `INTERPOLATE`: + +``` sql +SELECT n, source, inter FROM ( + SELECT toFloat32(number % 10) AS n, 'original' AS source, n as inter + FROM numbers(10) WHERE number % 3 = 1 +) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5; +``` + +Результат: +``` text +┌───n─┬─source───┬─inter─┐ +│ 0 │ │ 0 │ +│ 0.5 │ │ 0 │ +│ 1 │ original │ 1 │ +│ 1.5 │ │ 0 │ +│ 2 │ │ 0 │ +│ 2.5 │ │ 0 │ +│ 3 │ │ 0 │ +│ 3.5 │ │ 0 │ +│ 4 │ original │ 4 │ +│ 4.5 │ │ 0 │ +│ 5 │ │ 0 │ +│ 5.5 │ │ 0 │ +│ 7 │ original │ 7 │ +└─────┴──────────┴───────┘ +``` + +Тот же запрос с `INTERPOLATE`: + +``` sql +SELECT n, source, inter FROM ( + SELECT toFloat32(number % 10) AS n, 'original' AS source, n as inter + FROM numbers(10) WHERE number % 3 = 1 +) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (inter AS inter + 1); +``` + +Результат: +``` text +┌───n─┬─source───┬─inter─┐ +│ 0 │ │ 0 │ +│ 0.5 │ │ 0 │ +│ 1 │ original │ 1 │ +│ 1.5 │ │ 2 │ +│ 2 │ │ 3 │ +│ 2.5 │ │ 4 │ +│ 3 │ │ 5 │ +│ 3.5 │ │ 6 │ +│ 4 │ original │ 4 │ +│ 4.5 │ │ 5 │ +│ 5 │ │ 6 │ +│ 5.5 │ │ 7 │ +│ 7 │ original │ 7 │ +└─────┴──────────┴───────┘ From 5c8a77dadeb60a3b2688601070a75e37559996fe Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Fri, 18 Mar 2022 23:37:58 -0400 Subject: [PATCH 010/101] Update order-by.md INTERPOLATE fix --- docs/en/sql-reference/statements/select/order-by.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/select/order-by.md b/docs/en/sql-reference/statements/select/order-by.md index b7e84173284..2ec1d23ef5a 100644 --- a/docs/en/sql-reference/statements/select/order-by.md +++ b/docs/en/sql-reference/statements/select/order-by.md @@ -518,7 +518,7 @@ Same query after applying `INTERPOLATE`: ``` sql SELECT n, source, inter FROM ( - SELECT toFloat32(number % 10) AS n, 'original' AS source, n as inter + SELECT toFloat32(number % 10) AS n, 'original' AS source, number as inter FROM numbers(10) WHERE number % 3 = 1 ) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (inter AS inter + 1); ``` From b01f965f31db5b5ed9591bbddebaed2963342e70 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Fri, 18 Mar 2022 23:39:30 -0400 Subject: [PATCH 011/101] Update order-by.md fix --- docs/ru/sql-reference/statements/select/order-by.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/sql-reference/statements/select/order-by.md b/docs/ru/sql-reference/statements/select/order-by.md index 71f12ec73b0..8a4a8d5d111 100644 --- a/docs/ru/sql-reference/statements/select/order-by.md +++ b/docs/ru/sql-reference/statements/select/order-by.md @@ -403,7 +403,7 @@ ORDER BY ``` sql SELECT n, source, inter FROM ( - SELECT toFloat32(number % 10) AS n, 'original' AS source, n as inter + SELECT toFloat32(number % 10) AS n, 'original' AS source, number as inter FROM numbers(10) WHERE number % 3 = 1 ) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5; ``` @@ -431,7 +431,7 @@ SELECT n, source, inter FROM ( ``` sql SELECT n, source, inter FROM ( - SELECT toFloat32(number % 10) AS n, 'original' AS source, n as inter + SELECT toFloat32(number % 10) AS n, 'original' AS source, number as inter FROM numbers(10) WHERE number % 3 = 1 ) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (inter AS inter + 1); ``` From 5ae6f800c6cebc78d9f1f1353e00534b62233550 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Fri, 18 Mar 2022 23:40:09 -0400 Subject: [PATCH 012/101] Update order-by.md INTERPOLATE fix --- docs/en/sql-reference/statements/select/order-by.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/select/order-by.md b/docs/en/sql-reference/statements/select/order-by.md index 2ec1d23ef5a..06d3b985854 100644 --- a/docs/en/sql-reference/statements/select/order-by.md +++ b/docs/en/sql-reference/statements/select/order-by.md @@ -489,7 +489,7 @@ Example of a query without `INTERPOLATE`: ``` sql SELECT n, source, inter FROM ( - SELECT toFloat32(number % 10) AS n, 'original' AS source, n as inter + SELECT toFloat32(number % 10) AS n, 'original' AS source, number as inter FROM numbers(10) WHERE number % 3 = 1 ) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5; ``` From 481ee8aad5a5b1d7d2b22a0891d8e32b03e0c8e4 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Sat, 19 Mar 2022 12:17:56 -0400 Subject: [PATCH 013/101] Update FillingTransform.cpp use range-based for loop --- src/Processors/Transforms/FillingTransform.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index d9de72de16d..c3991e393b9 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -174,15 +174,15 @@ FillingTransform::FillingTransform( } } - for (size_t i = 0, size = interpolate_description.size(); i < size; ++i) + for (const auto & descr : interpolate_description) { - size_t block_position = header_.getPositionByName(interpolate_description[i].column.name); + size_t block_position = header_.getPositionByName(descr.column.name); is_fill_column[block_position] = true; fill_column_positions.push_back(block_position); /// Check column-expression compatibility - auto column = interpolate_description[i].column; - auto exp_type = interpolate_description[i].actions->getActionsDAG().getResultColumns()[0].type; + auto column = descr.column; + auto exp_type = descr.actions->getActionsDAG().getResultColumns()[0].type; auto exp_column = exp_type->createColumn(); exp_column->insertDefault(); From c4daf514d6aa99cd9445846d549b0795bc3bdfbd Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Sat, 19 Mar 2022 14:12:29 -0400 Subject: [PATCH 014/101] Update InterpreterSelectQuery.cpp bugfix: check column existence for INTERPOLATE expression target --- src/Interpreters/InterpreterSelectQuery.cpp | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index fa3d2698cdc..4bef4c66ade 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -101,6 +101,7 @@ namespace ErrorCodes extern const int INVALID_LIMIT_EXPRESSION; extern const int INVALID_WITH_FILL_EXPRESSION; extern const int ACCESS_DENIED; + extern const int UNKNOWN_IDENTIFIER; } /// Assumes `storage` is set and the table filter (row-level security) is not empty. @@ -842,7 +843,11 @@ static InterpolateDescription getInterpolateDescription(const ASTSelectQuery & q for (const auto & elem : query.interpolate()->children) { auto interpolate = elem->as(); - ColumnWithTypeAndName column = block.findByName(interpolate.column->getColumnName())->cloneEmpty(); + ColumnWithTypeAndName *block_column = block.findByName(interpolate.column->getColumnName()); + if (!block_column) + throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, + "Missing column '{}' as an INTERPOLATE expression target", interpolate.column->getColumnName()); + ColumnWithTypeAndName column = block_column->cloneEmpty(); auto syntax_result = TreeRewriter(context).analyze(interpolate.expr, block.getNamesAndTypesList()); ExpressionAnalyzer analyzer(interpolate.expr, syntax_result, context); From 83f406b722e3a76fcd0adb582153055d37a69a0e Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 24 Mar 2022 15:29:29 -0400 Subject: [PATCH 015/101] optimization, INTERPOLATE without expr. list, any column is allowed except WITH FILL --- src/Core/InterpolateDescription.cpp | 63 +++------- src/Core/InterpolateDescription.h | 56 ++------- src/Interpreters/FillingRow.cpp | 93 ++++++++++----- src/Interpreters/FillingRow.h | 28 +---- src/Interpreters/InterpreterSelectQuery.cpp | 38 ++++-- src/Parsers/ExpressionElementParsers.cpp | 12 +- src/Parsers/ExpressionListParsers.cpp | 2 +- src/Parsers/ParserSelectQuery.cpp | 34 +++++- src/Processors/QueryPlan/FillingStep.cpp | 9 +- src/Processors/QueryPlan/FillingStep.h | 4 +- .../Transforms/FillingTransform.cpp | 110 ++++++++++-------- src/Processors/Transforms/FillingTransform.h | 6 +- .../0_stateless/02233_interpolate_1.sql | 6 +- 13 files changed, 229 insertions(+), 232 deletions(-) diff --git a/src/Core/InterpolateDescription.cpp b/src/Core/InterpolateDescription.cpp index 176909ca929..e0c3bebd45e 100644 --- a/src/Core/InterpolateDescription.cpp +++ b/src/Core/InterpolateDescription.cpp @@ -7,57 +7,22 @@ namespace DB { -void dumpInterpolateDescription(const InterpolateDescription & description, const Block & /*header*/, WriteBuffer & out) -{ - bool first = true; - - for (const auto & desc : description) + InterpolateDescription::InterpolateDescription(ExpressionActionsPtr actions_) + : actions(actions_) { - if (!first) - out << ", "; - first = false; + for (const auto & name_type : actions->getRequiredColumnsWithTypes()) + { + columns_full_set.insert(name_type.name); + required_columns_map[name_type.name] = name_type; + } - if (desc.column.name.empty()) - out << "?"; - else - out << desc.column.name; - } -} - -void InterpolateColumnDescription::interpolate(Field & field) const -{ - if (field.isNull()) - return; - Block expr_columns; - Field column_field = convertFieldToType(field, *column.type.get()); - expr_columns.insert({column.type->createColumnConst(1, column_field), column.type, column.name}); - actions->execute(expr_columns); - field = convertFieldToType((*expr_columns.getByPosition(0).column)[0], *column.type.get()); -} - -void InterpolateColumnDescription::explain(JSONBuilder::JSONMap & map, const Block & /*header*/) const -{ - map.add("Column", column.name); -} - -std::string dumpInterpolateDescription(const InterpolateDescription & description) -{ - WriteBufferFromOwnString wb; - dumpInterpolateDescription(description, Block{}, wb); - return wb.str(); -} - -JSONBuilder::ItemPtr explainInterpolateDescription(const InterpolateDescription & description, const Block & header) -{ - auto json_array = std::make_unique(); - for (const auto & descr : description) - { - auto json_map = std::make_unique(); - descr.explain(*json_map, header); - json_array->add(std::move(json_map)); + const Block & res_block = actions->getSampleBlock(); + size_t idx = 0; + for (const ColumnWithTypeAndName & column : res_block) + { + columns_full_set.insert(column.name); + result_columns_map[column.name] = idx++; + } } - return json_array; -} - } diff --git a/src/Core/InterpolateDescription.h b/src/Core/InterpolateDescription.h index bb2b5febdc9..827586aac62 100644 --- a/src/Core/InterpolateDescription.h +++ b/src/Core/InterpolateDescription.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include #include @@ -11,60 +11,26 @@ #include #include -class Collator; namespace DB { -namespace JSONBuilder -{ - class JSONMap; - class IItem; - using ItemPtr = std::unique_ptr; -} - -class Block; - - /// Interpolate description -struct InterpolateColumnDescription +struct InterpolateDescription { - using Signature = ExecutableFunctionExpression::Signature; + explicit InterpolateDescription(ExpressionActionsPtr actions); - ColumnWithTypeAndName column; ExpressionActionsPtr actions; + std::set columns_full_set; /// columns to add to row + std::unordered_map required_columns_map; /// input columns + std::unordered_map result_columns_map; /// result block column name -> block column index - explicit InterpolateColumnDescription(const ColumnWithTypeAndName & column_, ExpressionActionsPtr actions_) : - column(column_), actions(actions_) {} - - bool operator == (const InterpolateColumnDescription & other) const - { - return column == other.column; - } - - bool operator != (const InterpolateColumnDescription & other) const - { - return !(*this == other); - } - - void interpolate(Field & field) const; - - std::string dump() const - { - return fmt::format("{}", column.name); - } - - void explain(JSONBuilder::JSONMap & map, const Block & header) const; + /// filled externally in transform + std::unordered_map input_map; /// row index -> column name type + std::unordered_map output_map; /// result block column index -> row index + std::unordered_map reset_map; /// row index -> column type, columns not filled by fill or interpolate }; -/// Description of interpolation for several columns. -using InterpolateDescription = std::vector; - -/// Outputs user-readable description into `out`. -void dumpInterpolateDescription(const InterpolateDescription & description, const Block & header, WriteBuffer & out); - -std::string dumpInterpolateDescription(const InterpolateDescription & description); - -JSONBuilder::ItemPtr explainInterpolateDescription(const InterpolateDescription & description, const Block & header); +using InterpolateDescriptionPtr = std::shared_ptr; } diff --git a/src/Interpreters/FillingRow.cpp b/src/Interpreters/FillingRow.cpp index 873c48702ae..89879f80ffc 100644 --- a/src/Interpreters/FillingRow.cpp +++ b/src/Interpreters/FillingRow.cpp @@ -19,30 +19,34 @@ bool equals(const Field & lhs, const Field & rhs) } -FillingRow::FillingRow(const SortDescription & sort_description_, const InterpolateDescription & interpolate_description_) - : sort{*this} - , interpolate{*this} - , sort_description(sort_description_) +FillingRow::FillingRow(const SortDescription & sort_description_, InterpolateDescriptionPtr interpolate_description_) + : sort_description(sort_description_) , interpolate_description(interpolate_description_) { - row.resize(sort_description.size() + interpolate_description.size()); + std::unordered_set col_set; + for (const auto & col : sort_description) + col_set.insert(col.column_name); + if (interpolate_description) + for (const auto & name : interpolate_description->columns_full_set) + col_set.insert(name); + row.resize(col_set.size()); } bool FillingRow::operator<(const FillingRow & other) const { - for (size_t i = 0; i < sort.size(); ++i) + for (size_t i = 0; i < sort_description.size(); ++i) { - if (sort[i].isNull() || other.sort[i].isNull() || equals(sort[i], other.sort[i])) + if (row[i].isNull() || other.row[i].isNull() || equals(row[i], other.row[i])) continue; - return less(sort[i], other.sort[i], getDirection(i)); + return less(row[i], other.row[i], getDirection(i)); } return false; } bool FillingRow::operator==(const FillingRow & other) const { - for (size_t i = 0; i < sort.size(); ++i) - if (!equals(sort[i], other.sort[i])) + for (size_t i = 0; i < sort_description.size(); ++i) + if (!equals(row[i], other.row[i])) return false; return true; } @@ -51,52 +55,50 @@ bool FillingRow::next(const FillingRow & to_row) { size_t pos = 0; - for (size_t i = 0; i < to_row.interpolate.size(); ++i) - interpolate[i] = to_row.interpolate[i]; - /// Find position we need to increment for generating next row. - for (; pos < sort.size(); ++pos) - if (!sort[pos].isNull() && !to_row.sort[pos].isNull() && !equals(sort[pos], to_row.sort[pos])) + for (; pos < size(); ++pos) + if (!row[pos].isNull() && !to_row.row[pos].isNull() && !equals(row[pos], to_row.row[pos])) break; - if (pos == sort.size() || less(to_row.sort[pos], sort[pos], getDirection(pos))) + if (pos == size() || less(to_row.row[pos], row[pos], getDirection(pos))) return false; /// If we have any 'fill_to' value at position greater than 'pos', /// we need to generate rows up to 'fill_to' value. - for (size_t i = sort.size() - 1; i > pos; --i) + for (size_t i = size() - 1; i > pos; --i) { - if (getFillDescription(i).fill_to.isNull() || sort[i].isNull()) + if (getFillDescription(i).fill_to.isNull() || row[i].isNull()) continue; - auto next_value = sort[i]; + auto next_value = row[i]; getFillDescription(i).step_func(next_value); if (less(next_value, getFillDescription(i).fill_to, getDirection(i))) { - sort[i] = next_value; + row[i] = next_value; initFromDefaults(i + 1); return true; } } - auto next_value = sort[pos]; + auto next_value = row[pos]; getFillDescription(pos).step_func(next_value); - if (less(to_row.sort[pos], next_value, getDirection(pos))) + if (less(to_row.row[pos], next_value, getDirection(pos))) return false; - sort[pos] = next_value; - if (equals(sort[pos], to_row.sort[pos])) + row[pos] = next_value; + if (equals(row[pos], to_row.row[pos])) { bool is_less = false; - for (size_t i = pos + 1; i < sort.size(); ++i) + size_t i = pos + 1; + for (; i < size(); ++i) { const auto & fill_from = getFillDescription(i).fill_from; if (!fill_from.isNull()) - sort[i] = fill_from; + row[i] = fill_from; else - sort[i] = to_row.sort[i]; - is_less |= less(sort[i], to_row.sort[i], getDirection(i)); + row[i] = to_row.row[i]; + is_less |= less(row[i], to_row.row[i], getDirection(i)); } return is_less; @@ -108,8 +110,39 @@ bool FillingRow::next(const FillingRow & to_row) void FillingRow::initFromDefaults(size_t from_pos) { - for (size_t i = from_pos; i < sort.size(); ++i) - sort[i] = getFillDescription(i).fill_from; + for (size_t i = from_pos; i < sort_description.size(); ++i) + row[i] = getFillDescription(i).fill_from; +} + +void FillingRow::interpolate() +{ + if (!interpolate_description) + return; + + Block block; + if (interpolate_description->input_map.size()) + { + for (const auto & [row_pos, name_type] : interpolate_description->input_map) + { + if (row[row_pos].isNull()) + return; + block.insert({name_type.type->createColumnConst(1, row[row_pos]), name_type.type, name_type.name}); + } + } + else /// INTERPOLATE contains only constants + { + /// dirty hack - we need at least one column with one row to execute actions on block + DataTypePtr dt = std::make_shared(); + block.insert({dt->createColumnConst(1, dt->getDefault()), dt, "dummy"}); + } + + interpolate_description->actions->execute(block); + + for (auto [col_pos, row_pos] : interpolate_description->output_map) + block.getByPosition(col_pos).column->get(0, row[row_pos]); + + for (const auto & [row_pos, col_type] : interpolate_description->reset_map) + row[row_pos] = col_type->getDefault(); } diff --git a/src/Interpreters/FillingRow.h b/src/Interpreters/FillingRow.h index 10adb6b72a8..04e0b6a0a79 100644 --- a/src/Interpreters/FillingRow.h +++ b/src/Interpreters/FillingRow.h @@ -18,25 +18,7 @@ bool equals(const Field & lhs, const Field & rhs); class FillingRow { public: - struct - { - FillingRow & filling_row; - - Field & operator[](size_t index) { return filling_row.row[index]; } - const Field & operator[](size_t index) const { return filling_row.row[index]; } - size_t size() const { return filling_row.sort_description.size(); } - } sort; - - struct - { - FillingRow & filling_row; - - Field & operator[](size_t index) { return filling_row.row[filling_row.sort_description.size() + index]; } - const Field & operator[](size_t index) const { return filling_row.row[filling_row.sort_description.size() + index]; } - size_t size() const { return filling_row.interpolate_description.size(); } - } interpolate; -public: - FillingRow(const SortDescription & sort_description, const InterpolateDescription & interpolate_description); + FillingRow(const SortDescription & sort_description, InterpolateDescriptionPtr interpolate_description); /// Generates next row according to fill 'from', 'to' and 'step' values. bool next(const FillingRow & to_row); @@ -45,18 +27,20 @@ public: Field & operator[](size_t index) { return row[index]; } const Field & operator[](size_t index) const { return row[index]; } - size_t size() const { return row.size(); } + size_t size() const { return sort_description.size(); } + size_t row_size() const { return row.size(); } bool operator<(const FillingRow & other) const; bool operator==(const FillingRow & other) const; int getDirection(size_t index) const { return sort_description[index].direction; } FillColumnDescription & getFillDescription(size_t index) { return sort_description[index].fill_description; } - InterpolateColumnDescription & getInterpolateDescription(size_t index) { return interpolate_description[index]; } + + void interpolate(); private: Row row; SortDescription sort_description; - InterpolateDescription interpolate_description; + InterpolateDescriptionPtr interpolate_description; }; void insertFromFillingRow(MutableColumns & filling_columns, MutableColumns & other_columns, const FillingRow & filling_row); diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 4bef4c66ade..611087e1e99 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -833,13 +833,12 @@ static SortDescription getSortDescription(const ASTSelectQuery & query, ContextP return order_descr; } -static InterpolateDescription getInterpolateDescription(const ASTSelectQuery & query, Block block, ContextPtr context) +static InterpolateDescriptionPtr getInterpolateDescription(const ASTSelectQuery & query, Block block, ContextPtr context) { - InterpolateDescription interpolate_descr; + InterpolateDescriptionPtr interpolate_descr; if (query.interpolate()) { - interpolate_descr.reserve(query.interpolate()->children.size()); - + std::map> position_map; for (const auto & elem : query.interpolate()->children) { auto interpolate = elem->as(); @@ -847,14 +846,31 @@ static InterpolateDescription getInterpolateDescription(const ASTSelectQuery & q if (!block_column) throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, "Missing column '{}' as an INTERPOLATE expression target", interpolate.column->getColumnName()); - ColumnWithTypeAndName column = block_column->cloneEmpty(); - auto syntax_result = TreeRewriter(context).analyze(interpolate.expr, block.getNamesAndTypesList()); - ExpressionAnalyzer analyzer(interpolate.expr, syntax_result, context); - ExpressionActionsPtr actions = analyzer.getActions(true, true, CompileExpressions::yes); + size_t position = block.getPositionByName(interpolate.column->getColumnName()); + position_map[position] = { + ColumnWithTypeAndName(block_column->type, block_column->name), + interpolate.expr->clone() + }; - interpolate_descr.emplace_back(column, actions); } + + ColumnsWithTypeAndName columns; + ASTPtr exprs = std::make_shared(); + for (auto & p : position_map) + { + columns.emplace_back(std::move(p.second.first)); + exprs->children.emplace_back(std::move(p.second.second)); + } + + auto syntax_result = TreeRewriter(context).analyze(exprs, block.getNamesAndTypesList()); + ExpressionAnalyzer analyzer(exprs, syntax_result, context); + ExpressionActionsPtr actions = analyzer.getActions(true, true, CompileExpressions::yes); + ActionsDAGPtr convDAG = ActionsDAG::makeConvertingActions(actions->getActionsDAG().getResultColumns(), + columns, ActionsDAG::MatchColumnsMode::Position, true); + ActionsDAGPtr mergeDAG = ActionsDAG::merge(std::move(*actions->getActionsDAG().clone()), std::move(*convDAG)); + + interpolate_descr = std::make_shared(std::make_shared(mergeDAG)); } return interpolate_descr; @@ -2527,8 +2543,8 @@ void InterpreterSelectQuery::executeWithFill(QueryPlan & query_plan) if (fill_descr.empty()) return; - InterpolateDescription interpolate_descr = getInterpolateDescription(query, source_header, context); - auto filling_step = std::make_unique(query_plan.getCurrentDataStream(), std::move(fill_descr), std::move(interpolate_descr)); + InterpolateDescriptionPtr interpolate_descr = getInterpolateDescription(query, source_header, context); + auto filling_step = std::make_unique(query_plan.getCurrentDataStream(), std::move(fill_descr), interpolate_descr); query_plan.addStep(std::move(filling_step)); } } diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index dbcfeb8e8b5..a643cfb83ea 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -2325,12 +2325,14 @@ bool ParserInterpolateElement::parseImpl(Pos & pos, ASTPtr & node, Expected & ex if (!ident_p.parse(pos, ident, expected)) return false; - if (!as.ignore(pos, expected)) - return false; - ASTPtr expr; - if (!element_p.parse(pos, expr, expected)) - return false; + if (as.ignore(pos, expected)) + { + if (!element_p.parse(pos, expr, expected)) + return false; + } + else + expr = ident; auto elem = std::make_shared(); elem->column = ident; diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index 1396e86df39..e79bdc495c1 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -759,7 +759,7 @@ bool ParserOrderByExpressionList::parseImpl(Pos & pos, ASTPtr & node, Expected & bool ParserInterpolateExpressionList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { - return ParserList(std::make_unique(), std::make_unique(TokenType::Comma), false) + return ParserList(std::make_unique(), std::make_unique(TokenType::Comma), true) .parse(pos, node, expected); } diff --git a/src/Parsers/ParserSelectQuery.cpp b/src/Parsers/ParserSelectQuery.cpp index 6cc00da1db0..65c89bd6f9b 100644 --- a/src/Parsers/ParserSelectQuery.cpp +++ b/src/Parsers/ParserSelectQuery.cpp @@ -11,6 +11,8 @@ #include #include #include +#include +#include namespace DB @@ -250,12 +252,32 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { if (s_interpolate.ignore(pos, expected)) { - if (!open_bracket.ignore(pos, expected)) - return false; - if (!interpolate_list.parse(pos, interpolate_expression_list, expected)) - return false; - if (!close_bracket.ignore(pos, expected)) - return false; + if (open_bracket.ignore(pos, expected)) + { + if (!interpolate_list.parse(pos, interpolate_expression_list, expected)) + return false; + if (!close_bracket.ignore(pos, expected)) + return false; + } else + interpolate_expression_list = std::make_shared(); + + if (interpolate_expression_list->children.empty()) + { + std::unordered_map columns; + for (const auto & elem : select_expression_list->children) + columns[elem->getColumnName()] = elem; + for (const auto & elem : order_expression_list->children) + if (elem->as()->with_fill) + columns.erase(elem->as()->children.front()->getColumnName()); + + for (const auto & [column, ast] : columns) + { + auto elem = std::make_shared(); + elem->column = ast; + elem->expr = ast; + interpolate_expression_list->children.push_back(elem); + } + } } } } diff --git a/src/Processors/QueryPlan/FillingStep.cpp b/src/Processors/QueryPlan/FillingStep.cpp index 47540bb2725..cfff191b464 100644 --- a/src/Processors/QueryPlan/FillingStep.cpp +++ b/src/Processors/QueryPlan/FillingStep.cpp @@ -28,9 +28,9 @@ static ITransformingStep::Traits getTraits() }; } -FillingStep::FillingStep(const DataStream & input_stream_, SortDescription sort_description_, InterpolateDescription interpolate_description_) - : ITransformingStep(input_stream_, FillingTransform::transformHeader(input_stream_.header, sort_description_/*, interpolate_description_*/), getTraits()) - , sort_description(std::move(sort_description_)), interpolate_description(std::move(interpolate_description_)) +FillingStep::FillingStep(const DataStream & input_stream_, SortDescription sort_description_, InterpolateDescriptionPtr interpolate_description_) + : ITransformingStep(input_stream_, FillingTransform::transformHeader(input_stream_.header, sort_description_), getTraits()) + , sort_description(std::move(sort_description_)), interpolate_description(interpolate_description_) { if (!input_stream_.has_single_port) throw Exception("FillingStep expects single input", ErrorCodes::LOGICAL_ERROR); @@ -50,14 +50,11 @@ void FillingStep::describeActions(FormatSettings & settings) const settings.out << String(settings.offset, ' '); dumpSortDescription(sort_description, input_streams.front().header, settings.out); settings.out << '\n'; - dumpInterpolateDescription(interpolate_description, input_streams.front().header, settings.out); - settings.out << '\n'; } void FillingStep::describeActions(JSONBuilder::JSONMap & map) const { map.add("Sort Description", explainSortDescription(sort_description, input_streams.front().header)); - map.add("Interpolate Description", explainInterpolateDescription(interpolate_description, input_streams.front().header)); } } diff --git a/src/Processors/QueryPlan/FillingStep.h b/src/Processors/QueryPlan/FillingStep.h index 252f1ba3fd5..0393b2c525b 100644 --- a/src/Processors/QueryPlan/FillingStep.h +++ b/src/Processors/QueryPlan/FillingStep.h @@ -10,7 +10,7 @@ namespace DB class FillingStep : public ITransformingStep { public: - FillingStep(const DataStream & input_stream_, SortDescription sort_description_, InterpolateDescription interpolate_description_); + FillingStep(const DataStream & input_stream_, SortDescription sort_description_, InterpolateDescriptionPtr interpolate_description_); String getName() const override { return "Filling"; } @@ -23,7 +23,7 @@ public: private: SortDescription sort_description; - InterpolateDescription interpolate_description; + InterpolateDescriptionPtr interpolate_description; }; } diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index c3991e393b9..85d31ecda4c 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -140,7 +140,7 @@ static bool tryConvertFields(FillColumnDescription & descr, const DataTypePtr & } FillingTransform::FillingTransform( - const Block & header_, const SortDescription & sort_description_, const InterpolateDescription & interpolate_description_, bool on_totals_) + const Block & header_, const SortDescription & sort_description_, InterpolateDescriptionPtr interpolate_description_, bool on_totals_) : ISimpleTransform(header_, transformHeader(header_, sort_description_/*, interpolate_description_*/), true) , sort_description(sort_description_) , interpolate_description(interpolate_description_) @@ -154,6 +154,19 @@ FillingTransform::FillingTransform( std::vector is_fill_column(header_.columns()); for (size_t i = 0, size = sort_description.size(); i < size; ++i) { + if (interpolate_description && interpolate_description->columns_full_set.count(sort_description[i].column_name)) + { + if (interpolate_description->result_columns_map.find(sort_description[i].column_name) != + interpolate_description->result_columns_map.end()) + throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, + "Column '{}' is participating in ORDER BY ... WITH FILL expression and can't be INTERPOLATE output", + sort_description[i].column_name); + + if (const auto & p = interpolate_description->required_columns_map.find(sort_description[i].column_name); + p != interpolate_description->required_columns_map.end()) + interpolate_description->input_map[fill_column_positions.size()] = p->second; + } + size_t block_position = header_.getPositionByName(sort_description[i].column_name); is_fill_column[block_position] = true; fill_column_positions.push_back(block_position); @@ -174,41 +187,40 @@ FillingTransform::FillingTransform( } } - for (const auto & descr : interpolate_description) - { - size_t block_position = header_.getPositionByName(descr.column.name); - is_fill_column[block_position] = true; - fill_column_positions.push_back(block_position); - - /// Check column-expression compatibility - auto column = descr.column; - auto exp_type = descr.actions->getActionsDAG().getResultColumns()[0].type; - auto exp_column = exp_type->createColumn(); - exp_column->insertDefault(); - - try - { - if (auto exp_field = (*exp_column)[0]; convertFieldToType(exp_field, *column.type).isNull()) - throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, - "Incompatible types of INTERPOLATE expression type {} with column '{}' of type {}", - exp_type->getName(), column.name, column.type->getName()); - } - catch (const Exception &) - { - throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, - "Incompatible types of INTERPOLATE expression type {} with column '{}' of type {}", - exp_type->getName(), column.name, column.type->getName()); - } - } - std::set unique_positions; for (auto pos : fill_column_positions) if (!unique_positions.insert(pos).second) throw Exception("Multiple WITH FILL for identical expressions is not supported in ORDER BY", ErrorCodes::INVALID_WITH_FILL_EXPRESSION); - for (size_t i = 0; i < header_.columns(); ++i) - if (!is_fill_column[i]) - other_column_positions.push_back(i); + size_t idx = 0; + for (const ColumnWithTypeAndName & column : header_.getColumnsWithTypeAndName()) + { + if (!is_fill_column[idx]) + { + if (interpolate_description && interpolate_description->columns_full_set.count(column.name)) + { + if ( + const auto & p = interpolate_description->required_columns_map.find(column.name); + p != interpolate_description->required_columns_map.end() + ) + interpolate_description->input_map[fill_column_positions.size()] = p->second; + + if ( + const auto & p = interpolate_description->result_columns_map.find(column.name); + p != interpolate_description->result_columns_map.end() + ) + interpolate_description->output_map[p->second] = fill_column_positions.size(); + else + interpolate_description->reset_map[fill_column_positions.size()] = column.type; + + is_fill_column[idx] = true; + fill_column_positions.push_back(idx); + } + else + other_column_positions.push_back(idx); + } + ++idx; + } } IProcessor::Status FillingTransform::prepare() @@ -217,8 +229,8 @@ IProcessor::Status FillingTransform::prepare() { should_insert_first = next_row < filling_row || first; - for (size_t i = 0, size = filling_row.sort.size(); i < size; ++i) - next_row.sort[i] = filling_row.getFillDescription(i).fill_to; + for (size_t i = 0, size = filling_row.size(); i < size; ++i) + next_row[i] = filling_row.getFillDescription(i).fill_to; if (first || filling_row < next_row) { @@ -277,9 +289,11 @@ void FillingTransform::transform(Chunk & chunk) init_columns_by_positions(old_columns, old_fill_columns, res_fill_columns, fill_column_positions); init_columns_by_positions(old_columns, old_other_columns, res_other_columns, other_column_positions); + bool first_block = first; + if (first) { - for (size_t i = 0; i < filling_row.sort.size(); ++i) + for (size_t i = 0; i < filling_row.size(); ++i) { auto current_value = (*old_fill_columns[i])[0]; const auto & fill_from = filling_row.getFillDescription(i).fill_from; @@ -300,7 +314,8 @@ void FillingTransform::transform(Chunk & chunk) { should_insert_first = next_row < filling_row; - for (size_t i = 0; i < filling_row.sort.size(); ++i) + size_t i = 0; + for (; i < filling_row.size(); ++i) { auto current_value = (*old_fill_columns[i])[row_ind]; const auto & fill_to = filling_row.getFillDescription(i).fill_to; @@ -311,28 +326,25 @@ void FillingTransform::transform(Chunk & chunk) next_row[i] = fill_to; } + if (row_ind > 0) + for (; i < filling_row.row_size(); ++i) + filling_row[i] = (*old_fill_columns[i])[row_ind-1]; + /// A case, when at previous step row was initialized from defaults 'fill_from' values /// and probably we need to insert it to block. if (should_insert_first && filling_row < next_row) insertFromFillingRow(res_fill_columns, res_other_columns, filling_row); - /// Update interpolate fields - for (size_t i = filling_row.sort.size(); i < filling_row.size(); ++i) - filling_row.getInterpolateDescription(i - filling_row.sort.size()).interpolate(next_row[i]); - /// Insert generated filling row to block, while it is less than current row in block. - while (filling_row.next(next_row)) + if (first_block) { - /// Update interpolate fields - for (size_t i = filling_row.sort.size(); i < filling_row.size(); ++i) - filling_row.getInterpolateDescription(i - filling_row.sort.size()).interpolate(next_row[i]); - - insertFromFillingRow(res_fill_columns, res_other_columns, filling_row); + first_block = false; + while (filling_row.next(next_row)) + insertFromFillingRow(res_fill_columns, res_other_columns, filling_row); } - - /// Reset interpolate fields - for (size_t i = filling_row.sort.size(); i < filling_row.size(); ++i) - next_row[i] = (*old_fill_columns[i])[row_ind]; + else + for (filling_row.interpolate(); filling_row.next(next_row); filling_row.interpolate()) + insertFromFillingRow(res_fill_columns, res_other_columns, filling_row); copyRowFromColumns(res_fill_columns, old_fill_columns, row_ind); copyRowFromColumns(res_other_columns, old_other_columns, row_ind); diff --git a/src/Processors/Transforms/FillingTransform.h b/src/Processors/Transforms/FillingTransform.h index 1a8a6c4919c..9f233dcafc6 100644 --- a/src/Processors/Transforms/FillingTransform.h +++ b/src/Processors/Transforms/FillingTransform.h @@ -14,13 +14,13 @@ namespace DB class FillingTransform : public ISimpleTransform { public: - FillingTransform(const Block & header_, const SortDescription & sort_description_, const InterpolateDescription & interpolate_description_, bool on_totals_); + FillingTransform(const Block & header_, const SortDescription & sort_description_, InterpolateDescriptionPtr interpolate_description_, bool on_totals_); String getName() const override { return "FillingTransform"; } Status prepare() override; - static Block transformHeader(Block header, const SortDescription & sort_description/*, const InterpolateDescription & interpolate_description_*/); + static Block transformHeader(Block header, const SortDescription & sort_description); protected: void transform(Chunk & Chunk) override; @@ -29,7 +29,7 @@ private: void setResultColumns(Chunk & chunk, MutableColumns & fill_columns, MutableColumns & other_columns) const; const SortDescription sort_description; /// Contains only columns with WITH FILL. - const InterpolateDescription interpolate_description; /// Contains INTERPOLATE columns + const InterpolateDescriptionPtr interpolate_description; /// Contains INTERPOLATE columns const bool on_totals; /// FillingTransform does nothing on totals. FillingRow filling_row; /// Current row, which is used to fill gaps. diff --git a/tests/queries/0_stateless/02233_interpolate_1.sql b/tests/queries/0_stateless/02233_interpolate_1.sql index dacfbd068ae..c65be99bbe5 100644 --- a/tests/queries/0_stateless/02233_interpolate_1.sql +++ b/tests/queries/0_stateless/02233_interpolate_1.sql @@ -21,12 +21,12 @@ SELECT n, source, inter FROM ( # Test INTERPOLATE with incompatible const - should produce error SELECT n, source, inter FROM ( SELECT toFloat32(number % 10) AS n, 'original' AS source, number as inter FROM numbers(10) WHERE number % 3 = 1 -) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (inter AS 'inter'); -- { serverError 475 } +) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (inter AS 'inter'); -- { serverError 6 } # Test INTERPOLATE with incompatible expression - should produce error SELECT n, source, inter FROM ( SELECT toFloat32(number % 10) AS n, 'original' AS source, number as inter FROM numbers(10) WHERE number % 3 = 1 -) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (inter AS inter||'inter'); -- { serverError 475 } +) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (inter AS inter||'inter'); -- { serverError 44 } # Test INTERPOLATE with column from WITH FILL expression - should produce error SELECT n, source, inter FROM ( @@ -36,4 +36,4 @@ SELECT n, source, inter FROM ( # Test INTERPOLATE with inconsistent column - should produce error SELECT n, source, inter FROM ( SELECT toFloat32(number % 10) AS n, 'original' AS source, number as inter FROM numbers(10) WHERE number % 3 = 1 -) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (inter AS source); -- { serverError 475 } +) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (inter AS source); -- { serverError 6 } From 5a4694f340ff82b41bbc2651cf74b733388088c4 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sun, 27 Mar 2022 13:33:22 -0400 Subject: [PATCH 016/101] major refactoring, simplified, optimized, bugs fixed --- src/Core/InterpolateDescription.cpp | 14 +- src/Core/InterpolateDescription.h | 9 +- src/Core/SortDescription.h | 7 +- src/Interpreters/FillingRow.cpp | 58 ++---- src/Interpreters/FillingRow.h | 11 +- src/Interpreters/InterpreterSelectQuery.cpp | 23 ++- .../Transforms/FillingTransform.cpp | 179 ++++++++++++------ src/Processors/Transforms/FillingTransform.h | 6 +- .../0_stateless/02233_interpolate_1.reference | 52 ++++- .../0_stateless/02233_interpolate_1.sql | 16 +- 10 files changed, 221 insertions(+), 154 deletions(-) diff --git a/src/Core/InterpolateDescription.cpp b/src/Core/InterpolateDescription.cpp index e0c3bebd45e..817f66aa4bc 100644 --- a/src/Core/InterpolateDescription.cpp +++ b/src/Core/InterpolateDescription.cpp @@ -11,18 +11,10 @@ namespace DB : actions(actions_) { for (const auto & name_type : actions->getRequiredColumnsWithTypes()) - { - columns_full_set.insert(name_type.name); - required_columns_map[name_type.name] = name_type; - } + required_columns_map[name_type.name] = name_type.type; - const Block & res_block = actions->getSampleBlock(); - size_t idx = 0; - for (const ColumnWithTypeAndName & column : res_block) - { - columns_full_set.insert(column.name); - result_columns_map[column.name] = idx++; - } + for (const ColumnWithTypeAndName & column : actions->getSampleBlock()) + result_columns_map.insert(column.name); } } diff --git a/src/Core/InterpolateDescription.h b/src/Core/InterpolateDescription.h index 827586aac62..26ad4d1a11c 100644 --- a/src/Core/InterpolateDescription.h +++ b/src/Core/InterpolateDescription.h @@ -21,14 +21,11 @@ struct InterpolateDescription explicit InterpolateDescription(ExpressionActionsPtr actions); ExpressionActionsPtr actions; - std::set columns_full_set; /// columns to add to row - std::unordered_map required_columns_map; /// input columns - std::unordered_map result_columns_map; /// result block column name -> block column index + std::unordered_map required_columns_map; /// input columns + std::unordered_set result_columns_map; /// result block columns /// filled externally in transform - std::unordered_map input_map; /// row index -> column name type - std::unordered_map output_map; /// result block column index -> row index - std::unordered_map reset_map; /// row index -> column type, columns not filled by fill or interpolate + std::vector> input_positions; /// positions in result columns required for actions }; using InterpolateDescriptionPtr = std::shared_ptr; diff --git a/src/Core/SortDescription.h b/src/Core/SortDescription.h index db15f3a54db..ae5e9196d24 100644 --- a/src/Core/SortDescription.h +++ b/src/Core/SortDescription.h @@ -7,6 +7,7 @@ #include #include #include +#include class Collator; @@ -24,6 +25,8 @@ class Block; struct FillColumnDescription { + DataTypePtr type; + /// All missed values in range [FROM, TO) will be filled /// Range [FROM, TO) respects sorting direction Field fill_from; /// Fill value >= FILL_FROM @@ -50,14 +53,14 @@ struct SortColumnDescription explicit SortColumnDescription( size_t column_number_, int direction_ = 1, int nulls_direction_ = 1, const std::shared_ptr & collator_ = nullptr, - bool with_fill_ = false, const FillColumnDescription & fill_description_ = {}) + bool with_fill_ = false, const FillColumnDescription & fill_description_ = FillColumnDescription{}) : column_number(column_number_), direction(direction_), nulls_direction(nulls_direction_), collator(collator_) , with_fill(with_fill_), fill_description(fill_description_) {} explicit SortColumnDescription( const std::string & column_name_, int direction_ = 1, int nulls_direction_ = 1, const std::shared_ptr & collator_ = nullptr, - bool with_fill_ = false, const FillColumnDescription & fill_description_ = {}) + bool with_fill_ = false, const FillColumnDescription & fill_description_ = FillColumnDescription{}) : column_name(column_name_), column_number(0), direction(direction_), nulls_direction(nulls_direction_) , collator(collator_), with_fill(with_fill_), fill_description(fill_description_) {} diff --git a/src/Interpreters/FillingRow.cpp b/src/Interpreters/FillingRow.cpp index 89879f80ffc..f5bbf036dd4 100644 --- a/src/Interpreters/FillingRow.cpp +++ b/src/Interpreters/FillingRow.cpp @@ -19,24 +19,17 @@ bool equals(const Field & lhs, const Field & rhs) } -FillingRow::FillingRow(const SortDescription & sort_description_, InterpolateDescriptionPtr interpolate_description_) +FillingRow::FillingRow(const SortDescription & sort_description_) : sort_description(sort_description_) - , interpolate_description(interpolate_description_) { - std::unordered_set col_set; - for (const auto & col : sort_description) - col_set.insert(col.column_name); - if (interpolate_description) - for (const auto & name : interpolate_description->columns_full_set) - col_set.insert(name); - row.resize(col_set.size()); + row.resize(sort_description.size()); } bool FillingRow::operator<(const FillingRow & other) const { for (size_t i = 0; i < sort_description.size(); ++i) { - if (row[i].isNull() || other.row[i].isNull() || equals(row[i], other.row[i])) + if ((*this)[i].isNull() || other.row[i].isNull() || equals(row[i], other.row[i])) continue; return less(row[i], other.row[i], getDirection(i)); } @@ -114,39 +107,8 @@ void FillingRow::initFromDefaults(size_t from_pos) row[i] = getFillDescription(i).fill_from; } -void FillingRow::interpolate() -{ - if (!interpolate_description) - return; - - Block block; - if (interpolate_description->input_map.size()) - { - for (const auto & [row_pos, name_type] : interpolate_description->input_map) - { - if (row[row_pos].isNull()) - return; - block.insert({name_type.type->createColumnConst(1, row[row_pos]), name_type.type, name_type.name}); - } - } - else /// INTERPOLATE contains only constants - { - /// dirty hack - we need at least one column with one row to execute actions on block - DataTypePtr dt = std::make_shared(); - block.insert({dt->createColumnConst(1, dt->getDefault()), dt, "dummy"}); - } - - interpolate_description->actions->execute(block); - - for (auto [col_pos, row_pos] : interpolate_description->output_map) - block.getByPosition(col_pos).column->get(0, row[row_pos]); - - for (const auto & [row_pos, col_type] : interpolate_description->reset_map) - row[row_pos] = col_type->getDefault(); -} - - -void insertFromFillingRow(MutableColumns & filling_columns, MutableColumns & other_columns, const FillingRow & filling_row) +void insertFromFillingRow(MutableColumns & filling_columns, MutableColumns & interpolate_columns, MutableColumns & other_columns, + const FillingRow & filling_row, const Block & interpolate_block) { for (size_t i = 0; i < filling_columns.size(); ++i) { @@ -156,6 +118,16 @@ void insertFromFillingRow(MutableColumns & filling_columns, MutableColumns & oth filling_columns[i]->insert(filling_row[i]); } + if(size_t size = interpolate_block.columns()) + { + Columns columns = interpolate_block.getColumns(); + for (size_t i = 0; i < size; ++i) + interpolate_columns[i]->insertFrom(*columns[i], 0); + } + else + for (const auto & interpolate_column : interpolate_columns) + interpolate_column->insertDefault(); + for (const auto & other_column : other_columns) other_column->insertDefault(); } diff --git a/src/Interpreters/FillingRow.h b/src/Interpreters/FillingRow.h index 04e0b6a0a79..626bee70896 100644 --- a/src/Interpreters/FillingRow.h +++ b/src/Interpreters/FillingRow.h @@ -18,7 +18,7 @@ bool equals(const Field & lhs, const Field & rhs); class FillingRow { public: - FillingRow(const SortDescription & sort_description, InterpolateDescriptionPtr interpolate_description); + explicit FillingRow(const SortDescription & sort_description); /// Generates next row according to fill 'from', 'to' and 'step' values. bool next(const FillingRow & to_row); @@ -27,23 +27,20 @@ public: Field & operator[](size_t index) { return row[index]; } const Field & operator[](size_t index) const { return row[index]; } - size_t size() const { return sort_description.size(); } - size_t row_size() const { return row.size(); } + size_t size() const { return row.size(); } bool operator<(const FillingRow & other) const; bool operator==(const FillingRow & other) const; int getDirection(size_t index) const { return sort_description[index].direction; } FillColumnDescription & getFillDescription(size_t index) { return sort_description[index].fill_description; } - void interpolate(); - private: Row row; SortDescription sort_description; - InterpolateDescriptionPtr interpolate_description; }; -void insertFromFillingRow(MutableColumns & filling_columns, MutableColumns & other_columns, const FillingRow & filling_row); +void insertFromFillingRow(MutableColumns & filling_columns, MutableColumns & interpolate_columns, MutableColumns & other_columns, + const FillingRow & filling_row, const Block & interpolate_block); void copyRowFromColumns(MutableColumns & dest, const Columns & source, size_t row_num); } diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index ab09d8cd71d..b35bf31c81f 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -763,9 +763,11 @@ static std::pair> getWithFillStep(const ASTPt throw Exception("Illegal type " + type->getName() + " of WITH FILL expression, must be numeric type", ErrorCodes::INVALID_WITH_FILL_EXPRESSION); } -static FillColumnDescription getWithFillDescription(const ASTOrderByElement & order_by_elem, ContextPtr context) +static FillColumnDescription getWithFillDescription(const ASTOrderByElement & order_by_elem, DataTypePtr type, ContextPtr context) { FillColumnDescription descr; + descr.type = type; + if (order_by_elem.fill_from) descr.fill_from = getWithFillFieldValue(order_by_elem.fill_from, context); if (order_by_elem.fill_to) @@ -809,7 +811,7 @@ static FillColumnDescription getWithFillDescription(const ASTOrderByElement & or return descr; } -static SortDescription getSortDescription(const ASTSelectQuery & query, ContextPtr context) +static SortDescription getSortDescription(const ASTSelectQuery & query, Block block, ContextPtr context) { SortDescription order_descr; order_descr.reserve(query.orderBy()->children.size()); @@ -824,7 +826,8 @@ static SortDescription getSortDescription(const ASTSelectQuery & query, ContextP if (order_by_elem.with_fill) { - FillColumnDescription fill_desc = getWithFillDescription(order_by_elem, context); + FillColumnDescription fill_desc = + getWithFillDescription(order_by_elem, block.getByName(order_by_elem.children.front()->getColumnName()).type, context); order_descr.emplace_back(name, order_by_elem.direction, order_by_elem.nulls_direction, collator, true, fill_desc); } else @@ -1398,7 +1401,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

( query, analysis_result.order_by_elements_actions, - getSortDescription(query, context), + getSortDescription(query, source_header, context), query_info.syntax_analyzer_result); } } @@ -2387,7 +2390,7 @@ void InterpreterSelectQuery::executeOrderOptimized(QueryPlan & query_plan, Input void InterpreterSelectQuery::executeOrder(QueryPlan & query_plan, InputOrderInfoPtr input_sorting_info) { auto & query = getSelectQuery(); - SortDescription output_order_descr = getSortDescription(query, context); + SortDescription output_order_descr = getSortDescription(query, source_header, context); UInt64 limit = getLimitForSorting(query, context); if (input_sorting_info) @@ -2425,7 +2428,7 @@ void InterpreterSelectQuery::executeOrder(QueryPlan & query_plan, InputOrderInfo void InterpreterSelectQuery::executeMergeSorted(QueryPlan & query_plan, const std::string & description) { auto & query = getSelectQuery(); - SortDescription order_descr = getSortDescription(query, context); + SortDescription order_descr = getSortDescription(query, source_header,context); UInt64 limit = getLimitForSorting(query, context); executeMergeSorted(query_plan, order_descr, limit, description); @@ -2529,7 +2532,7 @@ void InterpreterSelectQuery::executeWithFill(QueryPlan & query_plan) auto & query = getSelectQuery(); if (query.orderBy()) { - SortDescription order_descr = getSortDescription(query, context); + SortDescription order_descr = getSortDescription(query, source_header, context); SortDescription fill_descr; for (auto & desc : order_descr) { @@ -2579,7 +2582,7 @@ void InterpreterSelectQuery::executeLimit(QueryPlan & query_plan) { if (!query.orderBy()) throw Exception("LIMIT WITH TIES without ORDER BY", ErrorCodes::LOGICAL_ERROR); - order_descr = getSortDescription(query, context); + order_descr = getSortDescription(query, source_header, context); } auto limit = std::make_unique( diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 088e85ca9d9..f71c4f4f511 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -139,12 +139,12 @@ static bool tryConvertFields(FillColumnDescription & descr, const DataTypePtr & FillingTransform::FillingTransform( const Block & header_, const SortDescription & sort_description_, InterpolateDescriptionPtr interpolate_description_, bool on_totals_) - : ISimpleTransform(header_, transformHeader(header_, sort_description_/*, interpolate_description_*/), true) + : ISimpleTransform(header_, transformHeader(header_, sort_description_), true) , sort_description(sort_description_) , interpolate_description(interpolate_description_) , on_totals(on_totals_) - , filling_row(sort_description_, interpolate_description_) - , next_row(sort_description_, interpolate_description_) + , filling_row(sort_description_) + , next_row(sort_description_) { if (on_totals) return; @@ -152,18 +152,10 @@ FillingTransform::FillingTransform( std::vector is_fill_column(header_.columns()); for (size_t i = 0, size = sort_description.size(); i < size; ++i) { - if (interpolate_description && interpolate_description->columns_full_set.count(sort_description[i].column_name)) - { - if (interpolate_description->result_columns_map.find(sort_description[i].column_name) != - interpolate_description->result_columns_map.end()) - throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, - "Column '{}' is participating in ORDER BY ... WITH FILL expression and can't be INTERPOLATE output", - sort_description[i].column_name); - - if (const auto & p = interpolate_description->required_columns_map.find(sort_description[i].column_name); - p != interpolate_description->required_columns_map.end()) - interpolate_description->input_map[fill_column_positions.size()] = p->second; - } + if (interpolate_description && interpolate_description->result_columns_map.count(sort_description[i].column_name)) + throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, + "Column '{}' is participating in ORDER BY ... WITH FILL expression and can't be INTERPOLATE output", + sort_description[i].column_name); size_t block_position = header_.getPositionByName(sort_description[i].column_name); is_fill_column[block_position] = true; @@ -193,27 +185,15 @@ FillingTransform::FillingTransform( size_t idx = 0; for (const ColumnWithTypeAndName & column : header_.getColumnsWithTypeAndName()) { + if (interpolate_description) + if (const auto & p = interpolate_description->required_columns_map.find(column.name); + p != interpolate_description->required_columns_map.end()) + interpolate_description->input_positions.emplace_back(idx, NameAndTypePair(column.name, p->second)); + if (!is_fill_column[idx]) { - if (interpolate_description && interpolate_description->columns_full_set.count(column.name)) - { - if ( - const auto & p = interpolate_description->required_columns_map.find(column.name); - p != interpolate_description->required_columns_map.end() - ) - interpolate_description->input_map[fill_column_positions.size()] = p->second; - - if ( - const auto & p = interpolate_description->result_columns_map.find(column.name); - p != interpolate_description->result_columns_map.end() - ) - interpolate_description->output_map[p->second] = fill_column_positions.size(); - else - interpolate_description->reset_map[fill_column_positions.size()] = column.type; - - is_fill_column[idx] = true; - fill_column_positions.push_back(idx); - } + if (interpolate_description && interpolate_description->result_columns_map.count(column.name)) + interpolate_column_positions.push_back(idx); else other_column_positions.push_back(idx); } @@ -247,37 +227,85 @@ void FillingTransform::transform(Chunk & chunk) return; Columns old_fill_columns; + Columns old_interpolate_columns; Columns old_other_columns; MutableColumns res_fill_columns; + MutableColumns res_interpolate_columns; MutableColumns res_other_columns; - auto init_columns_by_positions = [](const Columns & old_columns, Columns & new_columns, - MutableColumns & new_mutable_columns, const Positions & positions) + std::vector> res_map; + res_map.resize(input.getHeader().columns()); + + auto init_columns_by_positions = [&res_map](const Columns & old_columns, Columns & new_columns, + MutableColumns & new_mutable_columns, const Positions & positions) { for (size_t pos : positions) { auto old_column = old_columns[pos]->convertToFullColumnIfConst(); new_columns.push_back(old_column); + res_map[pos] = {&new_mutable_columns, new_mutable_columns.size()}; new_mutable_columns.push_back(old_column->cloneEmpty()->assumeMutable()); } }; + Block interpolate_block; + + auto interpolate = [&]() { + if (interpolate_description) + { + interpolate_block.clear(); + + if (interpolate_description->input_positions.size()) + { + for (const auto & [col_pos, name_type] : interpolate_description->input_positions) + { + MutableColumnPtr column = name_type.type->createColumn(); + auto [res_columns, pos] = res_map[col_pos]; + size_t size = (*res_columns)[pos]->size(); + if (size == 0) + { + if (last_row.size() > col_pos && last_row[col_pos]->size()) + column->insertFrom(*last_row[col_pos], 0); + else + column->insertDefault(); + } + else + column->insertFrom(*(*res_columns)[pos], size - 1); + + interpolate_block.insert({std::move(column), name_type.type, name_type.name}); + } + } + else /// all INTERPOLATE expressions are constants + { + /// dirty hack - we need at least one column with one row to execute actions on block + DataTypePtr dt = std::make_shared(); + interpolate_block.insert({dt->createColumnConst(1, dt->getDefault()), dt, "dummy"}); + } + + interpolate_description->actions->execute(interpolate_block); + } + }; + if (generate_suffix) { const auto & empty_columns = input.getHeader().getColumns(); init_columns_by_positions(empty_columns, old_fill_columns, res_fill_columns, fill_column_positions); + init_columns_by_positions(empty_columns, old_interpolate_columns, res_interpolate_columns, interpolate_column_positions); init_columns_by_positions(empty_columns, old_other_columns, res_other_columns, other_column_positions); if (first) filling_row.initFromDefaults(); if (should_insert_first && filling_row < next_row) - insertFromFillingRow(res_fill_columns, res_other_columns, filling_row); + { + interpolate(); + insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); + } - while (filling_row.next(next_row)) - insertFromFillingRow(res_fill_columns, res_other_columns, filling_row); + for (interpolate(); filling_row.next(next_row); interpolate()) + insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); - setResultColumns(chunk, res_fill_columns, res_other_columns); + setResultColumns(chunk, res_fill_columns, res_interpolate_columns, res_other_columns); return; } @@ -285,10 +313,9 @@ void FillingTransform::transform(Chunk & chunk) auto old_columns = chunk.detachColumns(); init_columns_by_positions(old_columns, old_fill_columns, res_fill_columns, fill_column_positions); + init_columns_by_positions(old_columns, old_interpolate_columns, res_interpolate_columns, interpolate_column_positions); init_columns_by_positions(old_columns, old_other_columns, res_other_columns, other_column_positions); - bool first_block = first; - if (first) { for (size_t i = 0; i < filling_row.size(); ++i) @@ -300,7 +327,10 @@ void FillingTransform::transform(Chunk & chunk) { filling_row.initFromDefaults(i); if (less(fill_from, current_value, filling_row.getDirection(i))) - insertFromFillingRow(res_fill_columns, res_other_columns, filling_row); + { + interpolate(); + insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); + } break; } filling_row[i] = current_value; @@ -312,8 +342,7 @@ void FillingTransform::transform(Chunk & chunk) { should_insert_first = next_row < filling_row; - size_t i = 0; - for (; i < filling_row.size(); ++i) + for (size_t i = 0; i < filling_row.size(); ++i) { auto current_value = (*old_fill_columns[i])[row_ind]; const auto & fill_to = filling_row.getFillDescription(i).fill_to; @@ -324,45 +353,71 @@ void FillingTransform::transform(Chunk & chunk) next_row[i] = fill_to; } - if (row_ind > 0) - for (; i < filling_row.row_size(); ++i) - filling_row[i] = (*old_fill_columns[i])[row_ind-1]; - /// A case, when at previous step row was initialized from defaults 'fill_from' values /// and probably we need to insert it to block. if (should_insert_first && filling_row < next_row) - insertFromFillingRow(res_fill_columns, res_other_columns, filling_row); - - /// Insert generated filling row to block, while it is less than current row in block. - if (first_block) { - first_block = false; - while (filling_row.next(next_row)) - insertFromFillingRow(res_fill_columns, res_other_columns, filling_row); + interpolate(); + insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); } - else - for (filling_row.interpolate(); filling_row.next(next_row); filling_row.interpolate()) - insertFromFillingRow(res_fill_columns, res_other_columns, filling_row); + + for (interpolate(); filling_row.next(next_row); interpolate()) + insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); copyRowFromColumns(res_fill_columns, old_fill_columns, row_ind); + copyRowFromColumns(res_interpolate_columns, old_interpolate_columns, row_ind); copyRowFromColumns(res_other_columns, old_other_columns, row_ind); } - setResultColumns(chunk, res_fill_columns, res_other_columns); + saveLastRow(res_fill_columns, res_interpolate_columns, res_other_columns); + setResultColumns(chunk, res_fill_columns, res_interpolate_columns, res_other_columns); } -void FillingTransform::setResultColumns(Chunk & chunk, MutableColumns & fill_columns, MutableColumns & other_columns) const +void FillingTransform::setResultColumns(Chunk & chunk, MutableColumns & fill_columns, MutableColumns & interpolate_columns, MutableColumns & other_columns) const { - MutableColumns result_columns(fill_columns.size() + other_columns.size()); + MutableColumns result_columns(fill_columns.size() + interpolate_columns.size() + other_columns.size()); /// fill_columns always non-empty. size_t num_rows = fill_columns[0]->size(); for (size_t i = 0, size = fill_columns.size(); i < size; ++i) result_columns[fill_column_positions[i]] = std::move(fill_columns[i]); + for (size_t i = 0, size = interpolate_columns.size(); i < size; ++i) + result_columns[interpolate_column_positions[i]] = std::move(interpolate_columns[i]); for (size_t i = 0, size = other_columns.size(); i < size; ++i) result_columns[other_column_positions[i]] = std::move(other_columns[i]); chunk.setColumns(std::move(result_columns), num_rows); } +void FillingTransform::saveLastRow(const MutableColumns & fill_columns, const MutableColumns & interpolate_columns, const MutableColumns & other_columns) +{ + last_row.clear(); + last_row.resize(fill_columns.size() + interpolate_columns.size() + other_columns.size()); + + size_t num_rows = fill_columns[0]->size(); + if (num_rows == 0) + return; + + for (size_t i = 0, size = fill_columns.size(); i < size; ++i) + { + auto column = fill_columns[i]->cloneEmpty(); + column->insertFrom(*fill_columns[i], num_rows - 1); + last_row[fill_column_positions[i]] = std::move(column); + } + + for (size_t i = 0, size = interpolate_columns.size(); i < size; ++i) + { + auto column = interpolate_columns[i]->cloneEmpty(); + column->insertFrom(*interpolate_columns[i], num_rows - 1); + last_row[interpolate_column_positions[i]] = std::move(column); + } + + for (size_t i = 0, size = other_columns.size(); i < size; ++i) + { + auto column = other_columns[i]->cloneEmpty(); + column->insertFrom(*other_columns[i], num_rows - 1); + last_row[other_column_positions[i]] = std::move(column); + } +} + } diff --git a/src/Processors/Transforms/FillingTransform.h b/src/Processors/Transforms/FillingTransform.h index 9f233dcafc6..90f8d61ceb6 100644 --- a/src/Processors/Transforms/FillingTransform.h +++ b/src/Processors/Transforms/FillingTransform.h @@ -26,7 +26,8 @@ protected: void transform(Chunk & Chunk) override; private: - void setResultColumns(Chunk & chunk, MutableColumns & fill_columns, MutableColumns & other_columns) const; + void setResultColumns(Chunk & chunk, MutableColumns & fill_columns, MutableColumns & interpolate_columns, MutableColumns & other_columns) const; + void saveLastRow(const MutableColumns & fill_columns, const MutableColumns & interpolate_columns, const MutableColumns & other_columns); const SortDescription sort_description; /// Contains only columns with WITH FILL. const InterpolateDescriptionPtr interpolate_description; /// Contains INTERPOLATE columns @@ -37,10 +38,13 @@ private: using Positions = std::vector; Positions fill_column_positions; + Positions interpolate_column_positions; Positions other_column_positions; bool first = true; bool generate_suffix = false; + Columns last_row; + /// Determines should we insert filling row before start generating next rows. bool should_insert_first = false; }; diff --git a/tests/queries/0_stateless/02233_interpolate_1.reference b/tests/queries/0_stateless/02233_interpolate_1.reference index 45377d486f9..c7774fe05ff 100644 --- a/tests/queries/0_stateless/02233_interpolate_1.reference +++ b/tests/queries/0_stateless/02233_interpolate_1.reference @@ -10,9 +10,20 @@ 4.5 0 5 0 5.5 0 +6 0 +6.5 0 7 original 7 -0 0 -0.5 0 +7.5 0 +8 0 +8.5 0 +9 0 +9.5 0 +10 0 +10.5 0 +11 0 +11.5 0 +0 42 +0.5 42 1 original 1 1.5 42 2 42 @@ -23,7 +34,18 @@ 4.5 42 5 42 5.5 42 +6 42 +6.5 42 7 original 7 +7.5 42 +8 42 +8.5 42 +9 42 +9.5 42 +10 42 +10.5 42 +11 42 +11.5 42 0 0 0.5 0 1 original 1 @@ -36,9 +58,20 @@ 4.5 4 5 4 5.5 4 +6 4 +6.5 4 7 original 7 -0 0 -0.5 0 +7.5 7 +8 7 +8.5 7 +9 7 +9.5 7 +10 7 +10.5 7 +11 7 +11.5 7 +0 1 +0.5 2 1 original 1 1.5 2 2 3 @@ -49,4 +82,15 @@ 4.5 5 5 6 5.5 7 +6 8 +6.5 9 7 original 7 +7.5 8 +8 9 +8.5 10 +9 11 +9.5 12 +10 13 +10.5 14 +11 15 +11.5 16 diff --git a/tests/queries/0_stateless/02233_interpolate_1.sql b/tests/queries/0_stateless/02233_interpolate_1.sql index c65be99bbe5..a355e903dae 100644 --- a/tests/queries/0_stateless/02233_interpolate_1.sql +++ b/tests/queries/0_stateless/02233_interpolate_1.sql @@ -1,39 +1,39 @@ # Test WITH FILL without INTERPOLATE SELECT n, source, inter FROM ( SELECT toFloat32(number % 10) AS n, 'original' AS source, number as inter FROM numbers(10) WHERE number % 3 = 1 -) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5; +) ORDER BY n WITH FILL FROM 0 TO 11.51 STEP 0.5; # Test INTERPOLATE with const SELECT n, source, inter FROM ( SELECT toFloat32(number % 10) AS n, 'original' AS source, number as inter FROM numbers(10) WHERE number % 3 = 1 -) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (inter AS 42); +) ORDER BY n WITH FILL FROM 0 TO 11.51 STEP 0.5 INTERPOLATE (inter AS 42); # Test INTERPOLATE with field value SELECT n, source, inter FROM ( SELECT toFloat32(number % 10) AS n, 'original' AS source, number as inter FROM numbers(10) WHERE number % 3 = 1 -) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (inter AS inter); +) ORDER BY n WITH FILL FROM 0 TO 11.51 STEP 0.5 INTERPOLATE (inter AS inter); # Test INTERPOLATE with expression SELECT n, source, inter FROM ( SELECT toFloat32(number % 10) AS n, 'original' AS source, number as inter FROM numbers(10) WHERE number % 3 = 1 -) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (inter AS inter + 1); +) ORDER BY n WITH FILL FROM 0 TO 11.51 STEP 0.5 INTERPOLATE (inter AS inter + 1); # Test INTERPOLATE with incompatible const - should produce error SELECT n, source, inter FROM ( SELECT toFloat32(number % 10) AS n, 'original' AS source, number as inter FROM numbers(10) WHERE number % 3 = 1 -) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (inter AS 'inter'); -- { serverError 6 } +) ORDER BY n WITH FILL FROM 0 TO 11.51 STEP 0.5 INTERPOLATE (inter AS 'inter'); -- { serverError 6 } # Test INTERPOLATE with incompatible expression - should produce error SELECT n, source, inter FROM ( SELECT toFloat32(number % 10) AS n, 'original' AS source, number as inter FROM numbers(10) WHERE number % 3 = 1 -) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (inter AS inter||'inter'); -- { serverError 44 } +) ORDER BY n WITH FILL FROM 0 TO 11.51 STEP 0.5 INTERPOLATE (inter AS inter||'inter'); -- { serverError 44 } # Test INTERPOLATE with column from WITH FILL expression - should produce error SELECT n, source, inter FROM ( SELECT toFloat32(number % 10) AS n, 'original' AS source, number as inter FROM numbers(10) WHERE number % 3 = 1 -) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (n AS n); -- { serverError 475 } +) ORDER BY n WITH FILL FROM 0 TO 11.51 STEP 0.5 INTERPOLATE (n AS n); -- { serverError 475 } # Test INTERPOLATE with inconsistent column - should produce error SELECT n, source, inter FROM ( SELECT toFloat32(number % 10) AS n, 'original' AS source, number as inter FROM numbers(10) WHERE number % 3 = 1 -) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (inter AS source); -- { serverError 6 } +) ORDER BY n WITH FILL FROM 0 TO 11.51 STEP 0.5 INTERPOLATE (inter AS source); -- { serverError 32 } From 615efa1381b2afe824cb993b8ad26c2e9d94bbf2 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 28 Mar 2022 19:15:53 -0400 Subject: [PATCH 017/101] aliases processing fixed --- src/Core/InterpolateDescription.cpp | 16 ++++- src/Core/InterpolateDescription.h | 6 +- src/Core/SortDescription.h | 2 - src/Interpreters/FillingRow.cpp | 2 +- src/Interpreters/InterpreterSelectQuery.cpp | 58 +++++++------------ src/Parsers/ASTInterpolateElement.cpp | 3 +- src/Parsers/ASTInterpolateElement.h | 2 +- src/Parsers/ExpressionElementParsers.cpp | 2 +- src/Parsers/ParserSelectQuery.cpp | 11 +++- .../Transforms/FillingTransform.cpp | 7 ++- 10 files changed, 55 insertions(+), 54 deletions(-) diff --git a/src/Core/InterpolateDescription.cpp b/src/Core/InterpolateDescription.cpp index 817f66aa4bc..5a19c51cc9f 100644 --- a/src/Core/InterpolateDescription.cpp +++ b/src/Core/InterpolateDescription.cpp @@ -7,14 +7,24 @@ namespace DB { - InterpolateDescription::InterpolateDescription(ExpressionActionsPtr actions_) + InterpolateDescription::InterpolateDescription(ExpressionActionsPtr actions_, const Aliases & aliases) : actions(actions_) { for (const auto & name_type : actions->getRequiredColumnsWithTypes()) - required_columns_map[name_type.name] = name_type.type; + { + if (const auto & p = aliases.find(name_type.name); p != aliases.end()) + required_columns_map[p->second->getColumnName()] = name_type; + else + required_columns_map[name_type.name] = name_type; + } for (const ColumnWithTypeAndName & column : actions->getSampleBlock()) - result_columns_map.insert(column.name); + { + if (const auto & p = aliases.find(column.name); p != aliases.end()) + result_columns_map.insert(p->second->getColumnName()); + else + result_columns_map.insert(column.name); + } } } diff --git a/src/Core/InterpolateDescription.h b/src/Core/InterpolateDescription.h index 26ad4d1a11c..553c236b116 100644 --- a/src/Core/InterpolateDescription.h +++ b/src/Core/InterpolateDescription.h @@ -10,6 +10,7 @@ #include #include #include +#include namespace DB @@ -18,10 +19,11 @@ namespace DB /// Interpolate description struct InterpolateDescription { - explicit InterpolateDescription(ExpressionActionsPtr actions); + explicit InterpolateDescription(ExpressionActionsPtr actions, const Aliases & aliases); ExpressionActionsPtr actions; - std::unordered_map required_columns_map; /// input columns + + std::unordered_map required_columns_map; /// input column name -> {alias, type} std::unordered_set result_columns_map; /// result block columns /// filled externally in transform diff --git a/src/Core/SortDescription.h b/src/Core/SortDescription.h index ae5e9196d24..ffdff0ecd4c 100644 --- a/src/Core/SortDescription.h +++ b/src/Core/SortDescription.h @@ -25,8 +25,6 @@ class Block; struct FillColumnDescription { - DataTypePtr type; - /// All missed values in range [FROM, TO) will be filled /// Range [FROM, TO) respects sorting direction Field fill_from; /// Fill value >= FILL_FROM diff --git a/src/Interpreters/FillingRow.cpp b/src/Interpreters/FillingRow.cpp index f5bbf036dd4..9679923f10b 100644 --- a/src/Interpreters/FillingRow.cpp +++ b/src/Interpreters/FillingRow.cpp @@ -118,7 +118,7 @@ void insertFromFillingRow(MutableColumns & filling_columns, MutableColumns & int filling_columns[i]->insert(filling_row[i]); } - if(size_t size = interpolate_block.columns()) + if (size_t size = interpolate_block.columns()) { Columns columns = interpolate_block.getColumns(); for (size_t i = 0; i < size; ++i) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index b35bf31c81f..a2926347c24 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -763,10 +763,9 @@ static std::pair> getWithFillStep(const ASTPt throw Exception("Illegal type " + type->getName() + " of WITH FILL expression, must be numeric type", ErrorCodes::INVALID_WITH_FILL_EXPRESSION); } -static FillColumnDescription getWithFillDescription(const ASTOrderByElement & order_by_elem, DataTypePtr type, ContextPtr context) +static FillColumnDescription getWithFillDescription(const ASTOrderByElement & order_by_elem, ContextPtr context) { FillColumnDescription descr; - descr.type = type; if (order_by_elem.fill_from) descr.fill_from = getWithFillFieldValue(order_by_elem.fill_from, context); @@ -811,7 +810,7 @@ static FillColumnDescription getWithFillDescription(const ASTOrderByElement & or return descr; } -static SortDescription getSortDescription(const ASTSelectQuery & query, Block block, ContextPtr context) +static SortDescription getSortDescription(const ASTSelectQuery & query, ContextPtr context) { SortDescription order_descr; order_descr.reserve(query.orderBy()->children.size()); @@ -823,11 +822,9 @@ static SortDescription getSortDescription(const ASTSelectQuery & query, Block bl std::shared_ptr collator; if (order_by_elem.collation) collator = std::make_shared(order_by_elem.collation->as().value.get()); - if (order_by_elem.with_fill) { - FillColumnDescription fill_desc = - getWithFillDescription(order_by_elem, block.getByName(order_by_elem.children.front()->getColumnName()).type, context); + FillColumnDescription fill_desc = getWithFillDescription(order_by_elem, context); order_descr.emplace_back(name, order_by_elem.direction, order_by_elem.nulls_direction, collator, true, fill_desc); } else @@ -837,34 +834,23 @@ static SortDescription getSortDescription(const ASTSelectQuery & query, Block bl return order_descr; } -static InterpolateDescriptionPtr getInterpolateDescription(const ASTSelectQuery & query, Block block, ContextPtr context) +static InterpolateDescriptionPtr getInterpolateDescription(const ASTSelectQuery & query, Block block, const Aliases & aliases, ContextPtr context) { InterpolateDescriptionPtr interpolate_descr; if (query.interpolate()) { - std::map> position_map; - for (const auto & elem : query.interpolate()->children) - { - auto interpolate = elem->as(); - ColumnWithTypeAndName *block_column = block.findByName(interpolate.column->getColumnName()); - if (!block_column) - throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, - "Missing column '{}' as an INTERPOLATE expression target", interpolate.column->getColumnName()); - - size_t position = block.getPositionByName(interpolate.column->getColumnName()); - position_map[position] = { - ColumnWithTypeAndName(block_column->type, block_column->name), - interpolate.expr->clone() - }; - - } - ColumnsWithTypeAndName columns; ASTPtr exprs = std::make_shared(); - for (auto & p : position_map) + for (const auto & elem : query.interpolate()->children) { - columns.emplace_back(std::move(p.second.first)); - exprs->children.emplace_back(std::move(p.second.second)); + const auto & interpolate = elem->as(); + ColumnWithTypeAndName *block_column = block.findByName(interpolate.column); + if (!block_column) + throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, + "Missing column '{}' as an INTERPOLATE expression target", interpolate.column); + + columns.emplace_back(block_column->type, block_column->name); + exprs->children.emplace_back(interpolate.expr->clone()); } auto syntax_result = TreeRewriter(context).analyze(exprs, block.getNamesAndTypesList()); @@ -874,7 +860,7 @@ static InterpolateDescriptionPtr getInterpolateDescription(const ASTSelectQuery columns, ActionsDAG::MatchColumnsMode::Position, true); ActionsDAGPtr mergeDAG = ActionsDAG::merge(std::move(*actions->getActionsDAG().clone()), std::move(*convDAG)); - interpolate_descr = std::make_shared(std::make_shared(mergeDAG)); + interpolate_descr = std::make_shared(std::make_shared(mergeDAG), aliases); } return interpolate_descr; @@ -1401,7 +1387,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

( query, analysis_result.order_by_elements_actions, - getSortDescription(query, source_header, context), + getSortDescription(query, context), query_info.syntax_analyzer_result); } } @@ -2390,7 +2376,7 @@ void InterpreterSelectQuery::executeOrderOptimized(QueryPlan & query_plan, Input void InterpreterSelectQuery::executeOrder(QueryPlan & query_plan, InputOrderInfoPtr input_sorting_info) { auto & query = getSelectQuery(); - SortDescription output_order_descr = getSortDescription(query, source_header, context); + SortDescription output_order_descr = getSortDescription(query, context); UInt64 limit = getLimitForSorting(query, context); if (input_sorting_info) @@ -2428,7 +2414,7 @@ void InterpreterSelectQuery::executeOrder(QueryPlan & query_plan, InputOrderInfo void InterpreterSelectQuery::executeMergeSorted(QueryPlan & query_plan, const std::string & description) { auto & query = getSelectQuery(); - SortDescription order_descr = getSortDescription(query, source_header,context); + SortDescription order_descr = getSortDescription(query, context); UInt64 limit = getLimitForSorting(query, context); executeMergeSorted(query_plan, order_descr, limit, description); @@ -2532,7 +2518,7 @@ void InterpreterSelectQuery::executeWithFill(QueryPlan & query_plan) auto & query = getSelectQuery(); if (query.orderBy()) { - SortDescription order_descr = getSortDescription(query, source_header, context); + SortDescription order_descr = getSortDescription(query, context); SortDescription fill_descr; for (auto & desc : order_descr) { @@ -2543,7 +2529,7 @@ void InterpreterSelectQuery::executeWithFill(QueryPlan & query_plan) if (fill_descr.empty()) return; - InterpolateDescriptionPtr interpolate_descr = getInterpolateDescription(query, source_header, context); + InterpolateDescriptionPtr interpolate_descr = getInterpolateDescription(query, result_header, syntax_analyzer_result->aliases, context); auto filling_step = std::make_unique(query_plan.getCurrentDataStream(), std::move(fill_descr), interpolate_descr); query_plan.addStep(std::move(filling_step)); } @@ -2582,7 +2568,7 @@ void InterpreterSelectQuery::executeLimit(QueryPlan & query_plan) { if (!query.orderBy()) throw Exception("LIMIT WITH TIES without ORDER BY", ErrorCodes::LOGICAL_ERROR); - order_descr = getSortDescription(query, source_header, context); + order_descr = getSortDescription(query, context); } auto limit = std::make_unique( diff --git a/src/Parsers/ASTInterpolateElement.cpp b/src/Parsers/ASTInterpolateElement.cpp index 3da94f772f2..15531962eb9 100644 --- a/src/Parsers/ASTInterpolateElement.cpp +++ b/src/Parsers/ASTInterpolateElement.cpp @@ -9,8 +9,7 @@ namespace DB void ASTInterpolateElement::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { - column->formatImpl(settings, state, frame); - settings.ostr << (settings.hilite ? hilite_keyword : "") << " AS " << (settings.hilite ? hilite_none : ""); + settings.ostr << column << (settings.hilite ? hilite_keyword : "") << " AS " << (settings.hilite ? hilite_none : ""); expr->formatImpl(settings, state, frame); } diff --git a/src/Parsers/ASTInterpolateElement.h b/src/Parsers/ASTInterpolateElement.h index e23b10cd434..ffb492787c9 100644 --- a/src/Parsers/ASTInterpolateElement.h +++ b/src/Parsers/ASTInterpolateElement.h @@ -9,7 +9,7 @@ namespace DB class ASTInterpolateElement : public IAST { public: - ASTPtr column; + String column; ASTPtr expr; String getID(char) const override { return "InterpolateElement"; } diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index a643cfb83ea..8eda7bba54a 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -2335,7 +2335,7 @@ bool ParserInterpolateElement::parseImpl(Pos & pos, ASTPtr & node, Expected & ex expr = ident; auto elem = std::make_shared(); - elem->column = ident; + elem->column = ident->getColumnName(); elem->expr = expr; node = elem; diff --git a/src/Parsers/ParserSelectQuery.cpp b/src/Parsers/ParserSelectQuery.cpp index 65c89bd6f9b..a232451fdb9 100644 --- a/src/Parsers/ParserSelectQuery.cpp +++ b/src/Parsers/ParserSelectQuery.cpp @@ -13,6 +13,7 @@ #include #include #include +#include namespace DB @@ -265,7 +266,11 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { std::unordered_map columns; for (const auto & elem : select_expression_list->children) - columns[elem->getColumnName()] = elem; + { + std::string alias = elem->tryGetAlias(); + columns[alias.empty() ? elem->getColumnName() : alias] = elem; + } + for (const auto & elem : order_expression_list->children) if (elem->as()->with_fill) columns.erase(elem->as()->children.front()->getColumnName()); @@ -273,8 +278,8 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) for (const auto & [column, ast] : columns) { auto elem = std::make_shared(); - elem->column = ast; - elem->expr = ast; + elem->column = column; + elem->expr = std::make_shared(column); interpolate_expression_list->children.push_back(elem); } } diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index f71c4f4f511..e2af77c9790 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -188,7 +188,7 @@ FillingTransform::FillingTransform( if (interpolate_description) if (const auto & p = interpolate_description->required_columns_map.find(column.name); p != interpolate_description->required_columns_map.end()) - interpolate_description->input_positions.emplace_back(idx, NameAndTypePair(column.name, p->second)); + interpolate_description->input_positions.emplace_back(idx, p->second); if (!is_fill_column[idx]) { @@ -250,7 +250,8 @@ void FillingTransform::transform(Chunk & chunk) Block interpolate_block; - auto interpolate = [&]() { + auto interpolate = [&]() + { if (interpolate_description) { interpolate_block.clear(); @@ -267,7 +268,7 @@ void FillingTransform::transform(Chunk & chunk) if (last_row.size() > col_pos && last_row[col_pos]->size()) column->insertFrom(*last_row[col_pos], 0); else - column->insertDefault(); + column->insertDefault(); } else column->insertFrom(*(*res_columns)[pos], size - 1); From 08e1fdc6b8a4d6fb74f76d29861e97a1accdf26e Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Wed, 30 Mar 2022 16:19:16 +0800 Subject: [PATCH 018/101] Fixed :a elimate AMBIGUOUS_COLUMN_NAME exception elimate AMBIGUOUS_COLUMN_NAME exception in hdfs/s3 cluster when run a query with join --- src/Storages/HDFS/StorageHDFSCluster.cpp | 2 +- src/Storages/StorageS3Cluster.cpp | 2 +- tests/integration/test_s3_cluster/test.py | 19 +++++++++++++++++++ tests/integration/test_storage_hdfs/test.py | 10 ++++++++++ 4 files changed, 31 insertions(+), 2 deletions(-) diff --git a/src/Storages/HDFS/StorageHDFSCluster.cpp b/src/Storages/HDFS/StorageHDFSCluster.cpp index b039caa4330..3e02fd46596 100644 --- a/src/Storages/HDFS/StorageHDFSCluster.cpp +++ b/src/Storages/HDFS/StorageHDFSCluster.cpp @@ -93,7 +93,7 @@ Pipe StorageHDFSCluster::read( /// So, task_identifier is passed as constructor argument. It is more obvious. auto remote_query_executor = std::make_shared( connection, - queryToString(query_info.query), + queryToString(query_info.original_query), header, context, /*throttler=*/nullptr, diff --git a/src/Storages/StorageS3Cluster.cpp b/src/Storages/StorageS3Cluster.cpp index 3c031d545fe..9e8fbfca04c 100644 --- a/src/Storages/StorageS3Cluster.cpp +++ b/src/Storages/StorageS3Cluster.cpp @@ -134,7 +134,7 @@ Pipe StorageS3Cluster::read( /// So, task_identifier is passed as constructor argument. It is more obvious. auto remote_query_executor = std::make_shared( connection, - queryToString(query_info.query), + queryToString(query_info.original_query), header, context, /*throttler=*/nullptr, diff --git a/tests/integration/test_s3_cluster/test.py b/tests/integration/test_s3_cluster/test.py index 561d3e3ed28..be0715e069e 100644 --- a/tests/integration/test_s3_cluster/test.py +++ b/tests/integration/test_s3_cluster/test.py @@ -153,3 +153,22 @@ def test_wrong_cluster(started_cluster): ) assert "not found" in error + + +def test_ambiguous_join(started_cluster): + node = started_cluster.instances["s0_0_0"] + result = node.query( + """ + SELECT l.name, r.value from s3Cluster( + 'cluster_simple', + 'http://minio1:9001/root/data/{clickhouse,database}/*', 'minio', 'minio123', 'CSV', + 'name String, value UInt32, polygon Array(Array(Tuple(Float64, Float64)))') as l + JOIN s3Cluster( + 'cluster_simple', + 'http://minio1:9001/root/data/{clickhouse,database}/*', 'minio', 'minio123', 'CSV', + 'name String, value UInt32, polygon Array(Array(Tuple(Float64, Float64)))') as r + ON l.name = r.name + """ + ) + assert "AMBIGUOUS_COLUMN_NAME" not in result + \ No newline at end of file diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index 7f340424ccf..85f1f79dee0 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -553,6 +553,16 @@ def test_insert_select_schema_inference(started_cluster): result = node1.query(f"select * from hdfs('hdfs://hdfs1:9000/test.native.zst')") assert int(result) == 1 +def test_cluster_join(started_cluster): + result = node1.query( + ''' + SELECT l.id,r.id FROM hdfsCluster('test_cluster_two_shards', 'hdfs://hdfs1:9000/test_hdfsCluster/file*', 'TSV', 'id UInt32') as l + JOIN hdfsCluster('test_cluster_two_shards', 'hdfs://hdfs1:9000/test_hdfsCluster/file*', 'TSV', 'id UInt32') as r + ON l.id = r.id + ''' + ) + assert "AMBIGUOUS_COLUMN_NAME" not in result + if __name__ == "__main__": cluster.start() From 6a1e116c467b9dfe055f3b01a4c3656507ae0c63 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 30 Mar 2022 16:34:19 -0400 Subject: [PATCH 019/101] refactoring --- src/Core/InterpolateDescription.cpp | 10 ++--- src/Core/InterpolateDescription.h | 9 ++--- src/Interpreters/InterpreterSelectQuery.cpp | 8 ++-- src/Processors/QueryPlan/FillingStep.cpp | 2 +- .../Transforms/FillingTransform.cpp | 40 ++++++++++++------- src/Processors/Transforms/FillingTransform.h | 2 + 6 files changed, 41 insertions(+), 30 deletions(-) diff --git a/src/Core/InterpolateDescription.cpp b/src/Core/InterpolateDescription.cpp index 5a19c51cc9f..4147d254033 100644 --- a/src/Core/InterpolateDescription.cpp +++ b/src/Core/InterpolateDescription.cpp @@ -7,10 +7,10 @@ namespace DB { - InterpolateDescription::InterpolateDescription(ExpressionActionsPtr actions_, const Aliases & aliases) + InterpolateDescription::InterpolateDescription(ActionsDAGPtr actions_, const Aliases & aliases) : actions(actions_) { - for (const auto & name_type : actions->getRequiredColumnsWithTypes()) + for (const auto & name_type : actions->getRequiredColumns()) { if (const auto & p = aliases.find(name_type.name); p != aliases.end()) required_columns_map[p->second->getColumnName()] = name_type; @@ -18,12 +18,12 @@ namespace DB required_columns_map[name_type.name] = name_type; } - for (const ColumnWithTypeAndName & column : actions->getSampleBlock()) + for (const ColumnWithTypeAndName & column : actions->getResultColumns()) { if (const auto & p = aliases.find(column.name); p != aliases.end()) - result_columns_map.insert(p->second->getColumnName()); + result_columns_set.insert(p->second->getColumnName()); else - result_columns_map.insert(column.name); + result_columns_set.insert(column.name); } } diff --git a/src/Core/InterpolateDescription.h b/src/Core/InterpolateDescription.h index 553c236b116..b19674eb590 100644 --- a/src/Core/InterpolateDescription.h +++ b/src/Core/InterpolateDescription.h @@ -19,15 +19,12 @@ namespace DB /// Interpolate description struct InterpolateDescription { - explicit InterpolateDescription(ExpressionActionsPtr actions, const Aliases & aliases); + explicit InterpolateDescription(ActionsDAGPtr actions, const Aliases & aliases); - ExpressionActionsPtr actions; + ActionsDAGPtr actions; std::unordered_map required_columns_map; /// input column name -> {alias, type} - std::unordered_set result_columns_map; /// result block columns - - /// filled externally in transform - std::vector> input_positions; /// positions in result columns required for actions + std::unordered_set result_columns_set; /// result block columns }; using InterpolateDescriptionPtr = std::shared_ptr; diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index a2926347c24..6a9fc2cef43 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -855,12 +855,12 @@ static InterpolateDescriptionPtr getInterpolateDescription(const ASTSelectQuery auto syntax_result = TreeRewriter(context).analyze(exprs, block.getNamesAndTypesList()); ExpressionAnalyzer analyzer(exprs, syntax_result, context); - ExpressionActionsPtr actions = analyzer.getActions(true, true, CompileExpressions::yes); - ActionsDAGPtr convDAG = ActionsDAG::makeConvertingActions(actions->getActionsDAG().getResultColumns(), + ActionsDAGPtr actions = analyzer.getActionsDAG(true); + ActionsDAGPtr convDAG = ActionsDAG::makeConvertingActions(actions->getResultColumns(), columns, ActionsDAG::MatchColumnsMode::Position, true); - ActionsDAGPtr mergeDAG = ActionsDAG::merge(std::move(*actions->getActionsDAG().clone()), std::move(*convDAG)); + ActionsDAGPtr mergeDAG = ActionsDAG::merge(std::move(*actions->clone()), std::move(*convDAG)); - interpolate_descr = std::make_shared(std::make_shared(mergeDAG), aliases); + interpolate_descr = std::make_shared(mergeDAG, aliases); } return interpolate_descr; diff --git a/src/Processors/QueryPlan/FillingStep.cpp b/src/Processors/QueryPlan/FillingStep.cpp index cfff191b464..fe8abcd3750 100644 --- a/src/Processors/QueryPlan/FillingStep.cpp +++ b/src/Processors/QueryPlan/FillingStep.cpp @@ -41,7 +41,7 @@ void FillingStep::transformPipeline(QueryPipelineBuilder & pipeline, const Build pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr { bool on_totals = stream_type == QueryPipelineBuilder::StreamType::Totals; - return std::make_shared(header, sort_description, interpolate_description, on_totals); + return std::make_shared(header, sort_description, std::move(interpolate_description), on_totals); }); } diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index e2af77c9790..b8dcf1f0bf9 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -149,10 +149,13 @@ FillingTransform::FillingTransform( if (on_totals) return; + if (interpolate_description) + interpolate_actions = std::make_shared(interpolate_description->actions); + std::vector is_fill_column(header_.columns()); for (size_t i = 0, size = sort_description.size(); i < size; ++i) { - if (interpolate_description && interpolate_description->result_columns_map.count(sort_description[i].column_name)) + if (interpolate_description && interpolate_description->result_columns_set.count(sort_description[i].column_name)) throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, "Column '{}' is participating in ORDER BY ... WITH FILL expression and can't be INTERPOLATE output", sort_description[i].column_name); @@ -188,11 +191,11 @@ FillingTransform::FillingTransform( if (interpolate_description) if (const auto & p = interpolate_description->required_columns_map.find(column.name); p != interpolate_description->required_columns_map.end()) - interpolate_description->input_positions.emplace_back(idx, p->second); + input_positions.emplace_back(idx, p->second); if (!is_fill_column[idx]) { - if (interpolate_description && interpolate_description->result_columns_map.count(column.name)) + if (interpolate_description && interpolate_description->result_columns_set.count(column.name)) interpolate_column_positions.push_back(idx); else other_column_positions.push_back(idx); @@ -256,34 +259,35 @@ void FillingTransform::transform(Chunk & chunk) { interpolate_block.clear(); - if (interpolate_description->input_positions.size()) + if (input_positions.size()) { - for (const auto & [col_pos, name_type] : interpolate_description->input_positions) + /// populate calculation block with required columns with values from previous row + for (const auto & [col_pos, name_type] : input_positions) { MutableColumnPtr column = name_type.type->createColumn(); auto [res_columns, pos] = res_map[col_pos]; size_t size = (*res_columns)[pos]->size(); - if (size == 0) + if (size == 0) /// this is the first row in current chunk { + /// take value from last row of previous chunk if exists, else use default if (last_row.size() > col_pos && last_row[col_pos]->size()) column->insertFrom(*last_row[col_pos], 0); else column->insertDefault(); } - else + else /// take value from previous row of current chunk column->insertFrom(*(*res_columns)[pos], size - 1); interpolate_block.insert({std::move(column), name_type.type, name_type.name}); } + interpolate_actions->execute(interpolate_block); } else /// all INTERPOLATE expressions are constants { - /// dirty hack - we need at least one column with one row to execute actions on block - DataTypePtr dt = std::make_shared(); - interpolate_block.insert({dt->createColumnConst(1, dt->getDefault()), dt, "dummy"}); + /// LOL :) + size_t n = 1; + interpolate_actions->execute(interpolate_block, n); } - - interpolate_description->actions->execute(interpolate_block); } }; @@ -303,8 +307,12 @@ void FillingTransform::transform(Chunk & chunk) insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); } - for (interpolate(); filling_row.next(next_row); interpolate()) + interpolate(); + while (filling_row.next(next_row)) + { insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); + interpolate(); + } setResultColumns(chunk, res_fill_columns, res_interpolate_columns, res_other_columns); return; @@ -362,8 +370,12 @@ void FillingTransform::transform(Chunk & chunk) insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); } - for (interpolate(); filling_row.next(next_row); interpolate()) + interpolate(); + while (filling_row.next(next_row)) + { insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); + interpolate(); + } copyRowFromColumns(res_fill_columns, old_fill_columns, row_ind); copyRowFromColumns(res_interpolate_columns, old_interpolate_columns, row_ind); diff --git a/src/Processors/Transforms/FillingTransform.h b/src/Processors/Transforms/FillingTransform.h index 90f8d61ceb6..5e1068e18e2 100644 --- a/src/Processors/Transforms/FillingTransform.h +++ b/src/Processors/Transforms/FillingTransform.h @@ -40,6 +40,8 @@ private: Positions fill_column_positions; Positions interpolate_column_positions; Positions other_column_positions; + std::vector> input_positions; /// positions in result columns required for actions + ExpressionActionsPtr interpolate_actions; bool first = true; bool generate_suffix = false; From d0e5b3571cf6c0455e60312177694f13ef8bac57 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 31 Mar 2022 09:28:07 +0800 Subject: [PATCH 020/101] fixed code style --- tests/ci/clickhouse_helper.py | 2 +- tests/integration/test_s3_cluster/test.py | 6 +++--- tests/integration/test_storage_hdfs/test.py | 4 ++-- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/ci/clickhouse_helper.py b/tests/ci/clickhouse_helper.py index 218aaca8b91..d52b6262a78 100644 --- a/tests/ci/clickhouse_helper.py +++ b/tests/ci/clickhouse_helper.py @@ -15,7 +15,7 @@ class ClickHouseHelper: self.url = url self.auth = { "X-ClickHouse-User": get_parameter_from_ssm("clickhouse-test-stat-login"), - "X-ClickHouse-Key": get_parameter_from_ssm("clickhouse-test-stat-password") + "X-ClickHouse-Key": get_parameter_from_ssm("clickhouse-test-stat-password"), } @staticmethod diff --git a/tests/integration/test_s3_cluster/test.py b/tests/integration/test_s3_cluster/test.py index be0715e069e..93708acd49c 100644 --- a/tests/integration/test_s3_cluster/test.py +++ b/tests/integration/test_s3_cluster/test.py @@ -149,7 +149,8 @@ def test_wrong_cluster(started_cluster): SELECT count(*) from s3Cluster( 'non_existent_cluster', 'http://minio1:9001/root/data/{clickhouse,database}/*', - 'minio', 'minio123', 'CSV', 'name String, value UInt32, polygon Array(Array(Tuple(Float64, Float64)))')""" + 'minio', 'minio123', 'CSV', 'name String, value UInt32, polygon Array(Array(Tuple(Float64, Float64)))') + """ ) assert "not found" in error @@ -158,7 +159,7 @@ def test_wrong_cluster(started_cluster): def test_ambiguous_join(started_cluster): node = started_cluster.instances["s0_0_0"] result = node.query( - """ + """ SELECT l.name, r.value from s3Cluster( 'cluster_simple', 'http://minio1:9001/root/data/{clickhouse,database}/*', 'minio', 'minio123', 'CSV', @@ -171,4 +172,3 @@ def test_ambiguous_join(started_cluster): """ ) assert "AMBIGUOUS_COLUMN_NAME" not in result - \ No newline at end of file diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index 85f1f79dee0..b6dcff6d174 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -555,11 +555,11 @@ def test_insert_select_schema_inference(started_cluster): def test_cluster_join(started_cluster): result = node1.query( - ''' + """ SELECT l.id,r.id FROM hdfsCluster('test_cluster_two_shards', 'hdfs://hdfs1:9000/test_hdfsCluster/file*', 'TSV', 'id UInt32') as l JOIN hdfsCluster('test_cluster_two_shards', 'hdfs://hdfs1:9000/test_hdfsCluster/file*', 'TSV', 'id UInt32') as r ON l.id = r.id - ''' + """ ) assert "AMBIGUOUS_COLUMN_NAME" not in result From b5682c1f02eaaf15be2f85977417fc8f1b16c904 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 31 Mar 2022 08:33:50 -0400 Subject: [PATCH 021/101] minor refactoring --- src/Interpreters/InterpreterSelectQuery.cpp | 6 +++--- src/Processors/Transforms/FillingTransform.cpp | 5 ++--- 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 6a9fc2cef43..f788cc98c72 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -856,11 +856,11 @@ static InterpolateDescriptionPtr getInterpolateDescription(const ASTSelectQuery auto syntax_result = TreeRewriter(context).analyze(exprs, block.getNamesAndTypesList()); ExpressionAnalyzer analyzer(exprs, syntax_result, context); ActionsDAGPtr actions = analyzer.getActionsDAG(true); - ActionsDAGPtr convDAG = ActionsDAG::makeConvertingActions(actions->getResultColumns(), + ActionsDAGPtr conv_dag = ActionsDAG::makeConvertingActions(actions->getResultColumns(), columns, ActionsDAG::MatchColumnsMode::Position, true); - ActionsDAGPtr mergeDAG = ActionsDAG::merge(std::move(*actions->clone()), std::move(*convDAG)); + ActionsDAGPtr merge_dag = ActionsDAG::merge(std::move(*actions->clone()), std::move(*conv_dag)); - interpolate_descr = std::make_shared(mergeDAG, aliases); + interpolate_descr = std::make_shared(merge_dag, aliases); } return interpolate_descr; diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index b8dcf1f0bf9..99fdd6b19f0 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -259,7 +259,7 @@ void FillingTransform::transform(Chunk & chunk) { interpolate_block.clear(); - if (input_positions.size()) + if (!input_positions.empty()) { /// populate calculation block with required columns with values from previous row for (const auto & [col_pos, name_type] : input_positions) @@ -270,7 +270,7 @@ void FillingTransform::transform(Chunk & chunk) if (size == 0) /// this is the first row in current chunk { /// take value from last row of previous chunk if exists, else use default - if (last_row.size() > col_pos && last_row[col_pos]->size()) + if (last_row.size() > col_pos && !last_row[col_pos]->empty()) column->insertFrom(*last_row[col_pos], 0); else column->insertDefault(); @@ -284,7 +284,6 @@ void FillingTransform::transform(Chunk & chunk) } else /// all INTERPOLATE expressions are constants { - /// LOL :) size_t n = 1; interpolate_actions->execute(interpolate_block, n); } From a15996315e5b01b7a09128310ff1e76182940fab Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 31 Mar 2022 11:51:13 -0400 Subject: [PATCH 022/101] bugfix - columns order tracking --- src/Core/InterpolateDescription.cpp | 10 ++++++---- src/Core/InterpolateDescription.h | 1 + src/Interpreters/InterpreterSelectQuery.cpp | 4 ++++ src/Processors/Transforms/FillingTransform.cpp | 12 ++++++------ 4 files changed, 17 insertions(+), 10 deletions(-) diff --git a/src/Core/InterpolateDescription.cpp b/src/Core/InterpolateDescription.cpp index 4147d254033..e7b74716b79 100644 --- a/src/Core/InterpolateDescription.cpp +++ b/src/Core/InterpolateDescription.cpp @@ -20,10 +20,12 @@ namespace DB for (const ColumnWithTypeAndName & column : actions->getResultColumns()) { - if (const auto & p = aliases.find(column.name); p != aliases.end()) - result_columns_set.insert(p->second->getColumnName()); - else - result_columns_set.insert(column.name); + std::string name = column.name; + if (const auto & p = aliases.find(name); p != aliases.end()) + name = p->second->getColumnName(); + + result_columns_set.insert(name); + result_columns_order.push_back(name); } } diff --git a/src/Core/InterpolateDescription.h b/src/Core/InterpolateDescription.h index b19674eb590..8aabce1470e 100644 --- a/src/Core/InterpolateDescription.h +++ b/src/Core/InterpolateDescription.h @@ -25,6 +25,7 @@ struct InterpolateDescription std::unordered_map required_columns_map; /// input column name -> {alias, type} std::unordered_set result_columns_set; /// result block columns + std::vector result_columns_order; /// result block columns order }; using InterpolateDescriptionPtr = std::shared_ptr; diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index f788cc98c72..6bcedc5b4f7 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -839,6 +839,7 @@ static InterpolateDescriptionPtr getInterpolateDescription(const ASTSelectQuery InterpolateDescriptionPtr interpolate_descr; if (query.interpolate()) { + std::unordered_set col_set; ColumnsWithTypeAndName columns; ASTPtr exprs = std::make_shared(); for (const auto & elem : query.interpolate()->children) @@ -848,6 +849,9 @@ static InterpolateDescriptionPtr getInterpolateDescription(const ASTSelectQuery if (!block_column) throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, "Missing column '{}' as an INTERPOLATE expression target", interpolate.column); + if (!col_set.insert(block_column->name).second) + throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, + "Duplicate INTERPOLATE column '{}'", interpolate.column); columns.emplace_back(block_column->type, block_column->name); exprs->children.emplace_back(interpolate.expr->clone()); diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 99fdd6b19f0..c13bdfe2d84 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -193,15 +193,15 @@ FillingTransform::FillingTransform( p != interpolate_description->required_columns_map.end()) input_positions.emplace_back(idx, p->second); - if (!is_fill_column[idx]) - { - if (interpolate_description && interpolate_description->result_columns_set.count(column.name)) - interpolate_column_positions.push_back(idx); - else + if (!is_fill_column[idx] && !(interpolate_description && interpolate_description->result_columns_set.count(column.name))) other_column_positions.push_back(idx); - } + ++idx; } + + if(interpolate_description) + for (const auto & name : interpolate_description->result_columns_order) + interpolate_column_positions.push_back(header_.getPositionByName(name)); } IProcessor::Status FillingTransform::prepare() From 538373a79b51cd8ec7f2ad9701a067cc9f6eae75 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 31 Mar 2022 12:13:49 -0400 Subject: [PATCH 023/101] style fix --- src/Processors/Transforms/FillingTransform.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index c13bdfe2d84..c7bcd03ec08 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -199,7 +199,7 @@ FillingTransform::FillingTransform( ++idx; } - if(interpolate_description) + if (interpolate_description) for (const auto & name : interpolate_description->result_columns_order) interpolate_column_positions.push_back(header_.getPositionByName(name)); } From 0116233d36cf78ad505401495a3433399042afbc Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 1 Apr 2022 16:18:19 -0400 Subject: [PATCH 024/101] allow INTERPOLATE to reference optimized out columns --- src/Interpreters/InterpreterSelectQuery.cpp | 13 +++++++++++-- src/Interpreters/InterpreterSelectQuery.h | 2 ++ 2 files changed, 13 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 6bcedc5b4f7..cb15b4fd404 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -286,6 +286,9 @@ InterpreterSelectQuery::InterpreterSelectQuery( { checkStackSize(); + for(const auto & column : query_ptr->as().select()->children) + original_select_set.insert(column->getColumnName()); + query_info.ignore_projections = options.ignore_projections; query_info.is_projection_query = options.is_projection_query; @@ -834,7 +837,8 @@ static SortDescription getSortDescription(const ASTSelectQuery & query, ContextP return order_descr; } -static InterpolateDescriptionPtr getInterpolateDescription(const ASTSelectQuery & query, Block block, const Aliases & aliases, ContextPtr context) +static InterpolateDescriptionPtr getInterpolateDescription( + const ASTSelectQuery & query, Block block, const Aliases & aliases, const NameSet & original_select_set, ContextPtr context) { InterpolateDescriptionPtr interpolate_descr; if (query.interpolate()) @@ -847,8 +851,12 @@ static InterpolateDescriptionPtr getInterpolateDescription(const ASTSelectQuery const auto & interpolate = elem->as(); ColumnWithTypeAndName *block_column = block.findByName(interpolate.column); if (!block_column) + { + if (original_select_set.count(interpolate.column)) /// column was removed + continue; throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, "Missing column '{}' as an INTERPOLATE expression target", interpolate.column); + } if (!col_set.insert(block_column->name).second) throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, "Duplicate INTERPOLATE column '{}'", interpolate.column); @@ -2533,7 +2541,8 @@ void InterpreterSelectQuery::executeWithFill(QueryPlan & query_plan) if (fill_descr.empty()) return; - InterpolateDescriptionPtr interpolate_descr = getInterpolateDescription(query, result_header, syntax_analyzer_result->aliases, context); + InterpolateDescriptionPtr interpolate_descr = + getInterpolateDescription(query, result_header, syntax_analyzer_result->aliases, original_select_set, context); auto filling_step = std::make_unique(query_plan.getCurrentDataStream(), std::move(fill_descr), interpolate_descr); query_plan.addStep(std::move(filling_step)); } diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index 6bb12caff7d..ec3775ab848 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -185,6 +185,8 @@ private: Names required_columns; /// Structure of query source (table, subquery, etc). Block source_header; + /// Original set of selected columns + NameSet original_select_set; /// Actions to calculate ALIAS if required. ActionsDAGPtr alias_actions; From ec0ad8804a45efa91505daf56912f5cf5a21e9fd Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 1 Apr 2022 21:45:58 -0400 Subject: [PATCH 025/101] style fix --- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index cb15b4fd404..cddaa51b8b7 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -286,7 +286,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( { checkStackSize(); - for(const auto & column : query_ptr->as().select()->children) + for (const auto & column : query_ptr->as().select()->children) original_select_set.insert(column->getColumnName()); query_info.ignore_projections = options.ignore_projections; From 95ad1bf6e1a3cbb904a0b3ff9cc9447893a9be2a Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sun, 3 Apr 2022 22:10:36 -0400 Subject: [PATCH 026/101] use aliases if exist for original_select_set --- src/Interpreters/InterpreterSelectQuery.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index cddaa51b8b7..8d2a6d21994 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -286,8 +286,10 @@ InterpreterSelectQuery::InterpreterSelectQuery( { checkStackSize(); - for (const auto & column : query_ptr->as().select()->children) - original_select_set.insert(column->getColumnName()); + for (const auto & column : query_ptr->as().select()->children) { + std::string alias = column->tryGetAlias(); + original_select_set.insert(alias.empty() ? column->getColumnName() : alias); + } query_info.ignore_projections = options.ignore_projections; query_info.is_projection_query = options.is_projection_query; From ff4d29526541901dc80a7a08602c9b89b032feab Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sun, 3 Apr 2022 22:19:35 -0400 Subject: [PATCH 027/101] style fix --- src/Interpreters/InterpreterSelectQuery.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 8d2a6d21994..d050b378916 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -286,7 +286,8 @@ InterpreterSelectQuery::InterpreterSelectQuery( { checkStackSize(); - for (const auto & column : query_ptr->as().select()->children) { + for (const auto & column : query_ptr->as().select()->children) + { std::string alias = column->tryGetAlias(); original_select_set.insert(alias.empty() ? column->getColumnName() : alias); } From cbfc0f6bacd33fe5d99ed13ffb21671c79898fe3 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 4 Apr 2022 10:42:22 +0800 Subject: [PATCH 028/101] fix typo --- src/Storages/Hive/HiveCommon.cpp | 1 + src/Storages/Hive/StorageHive.cpp | 7 ++++--- src/Storages/Hive/StorageHive.h | 1 - 3 files changed, 5 insertions(+), 4 deletions(-) diff --git a/src/Storages/Hive/HiveCommon.cpp b/src/Storages/Hive/HiveCommon.cpp index a9d0c22d6a5..60f471c1568 100644 --- a/src/Storages/Hive/HiveCommon.cpp +++ b/src/Storages/Hive/HiveCommon.cpp @@ -172,6 +172,7 @@ std::vector HiveMetastoreClient::HiveTableMetad std::vector result; std::lock_guard lock{mutex}; + result.reserve(partition_infos.size()); for (const auto & partition_info : partition_infos) result.emplace_back(partition_info.second.partition); return result; diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 61bbc3ce4f6..c5e50e6b480 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -60,7 +60,7 @@ public: struct SourcesInfo { HiveMetastoreClientPtr hive_metastore_client; - std::string database; + std::string database_name; std::string table_name; HiveFiles hive_files; NamesAndTypesList partition_name_types; @@ -159,7 +159,7 @@ public: { if (e.code() == ErrorCodes::CANNOT_OPEN_FILE) { - source_info->hive_metastore_client->clearTableMetadata(source_info->database, source_info->table_name); + source_info->hive_metastore_client->clearTableMetadata(source_info->database_name, source_info->table_name); throw; } } @@ -562,6 +562,7 @@ HiveFilePtr StorageHive::createHiveFileIfNeeded( } return hive_file; } + bool StorageHive::isColumnOriented() const { return format_name == "Parquet" || format_name == "ORC"; @@ -651,7 +652,7 @@ Pipe StorageHive::read( auto sources_info = std::make_shared(); sources_info->hive_files = std::move(hive_files); - sources_info->database = hive_database; + sources_info->database_name = hive_database; sources_info->table_name = hive_table; sources_info->hive_metastore_client = hive_metastore_client; sources_info->partition_name_types = partition_name_types; diff --git a/src/Storages/Hive/StorageHive.h b/src/Storages/Hive/StorageHive.h index 376aab311d0..bdf37cc9f04 100644 --- a/src/Storages/Hive/StorageHive.h +++ b/src/Storages/Hive/StorageHive.h @@ -39,7 +39,6 @@ public: return true; } - Pipe read( const Names & column_names, const StorageSnapshotPtr & storage_snapshot, From 38f149b533607d13d406c4efac4d8398ad4a1eb3 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 4 Apr 2022 15:28:26 +0800 Subject: [PATCH 029/101] optimize trivial count hive query --- src/Storages/Hive/HiveFile.cpp | 44 ++++- src/Storages/Hive/HiveFile.h | 23 ++- src/Storages/Hive/StorageHive.cpp | 260 +++++++++++++++++++----------- src/Storages/Hive/StorageHive.h | 51 ++++-- 4 files changed, 266 insertions(+), 112 deletions(-) diff --git a/src/Storages/Hive/HiveFile.cpp b/src/Storages/Hive/HiveFile.cpp index 407d9602b61..12d60c4d1b3 100644 --- a/src/Storages/Hive/HiveFile.cpp +++ b/src/Storages/Hive/HiveFile.cpp @@ -77,7 +77,14 @@ Range createRangeFromParquetStatistics(std::shared_ptr IHiveFile::getRows() +{ + if (!rows) + rows = getRowsImpl(); + return rows; +} + +Range HiveORCFile::buildRange(const orc::ColumnStatistics * col_stats) { if (!col_stats || col_stats->hasNull()) return {}; @@ -122,7 +129,7 @@ Range HiveOrcFile::buildRange(const orc::ColumnStatistics * col_stats) return {}; } -void HiveOrcFile::prepareReader() +void HiveORCFile::prepareReader() { in = std::make_unique(namenode_url, path, getContext()->getGlobalContext()->getConfigRef()); auto format_settings = getFormatSettings(getContext()); @@ -132,7 +139,7 @@ void HiveOrcFile::prepareReader() reader = std::move(result).ValueOrDie(); } -void HiveOrcFile::prepareColumnMapping() +void HiveORCFile::prepareColumnMapping() { const orc::Type & type = reader->GetRawORCReader()->getType(); size_t count = type.getSubtypeCount(); @@ -145,13 +152,13 @@ void HiveOrcFile::prepareColumnMapping() } } -bool HiveOrcFile::hasMinMaxIndex() const +bool HiveORCFile::hasMinMaxIndex() const { return storage_settings->enable_orc_file_minmax_index; } -std::unique_ptr HiveOrcFile::buildMinMaxIndex(const orc::Statistics * statistics) +std::unique_ptr HiveORCFile::buildMinMaxIndex(const orc::Statistics * statistics) { if (!statistics) return nullptr; @@ -184,7 +191,7 @@ std::unique_ptr HiveOrcFile::buildMinMaxIndex(c } -void HiveOrcFile::loadMinMaxIndex() +void HiveORCFile::loadMinMaxIndex() { if (!reader) { @@ -196,13 +203,13 @@ void HiveOrcFile::loadMinMaxIndex() minmax_idx = buildMinMaxIndex(statistics.get()); } -bool HiveOrcFile::hasSubMinMaxIndex() const +bool HiveORCFile::hasSubMinMaxIndex() const { return storage_settings->enable_orc_stripe_minmax_index; } -void HiveOrcFile::loadSubMinMaxIndex() +void HiveORCFile::loadSubMinMaxIndex() { if (!reader) { @@ -226,6 +233,18 @@ void HiveOrcFile::loadSubMinMaxIndex() } } +std::optional HiveORCFile::getRowsImpl() +{ + if (!reader) + { + prepareReader(); + prepareColumnMapping(); + } + + auto * raw_reader = reader->GetRawORCReader(); + return raw_reader->getNumberOfRows(); +} + bool HiveParquetFile::hasSubMinMaxIndex() const { return storage_settings->enable_parquet_rowgroup_minmax_index; @@ -312,5 +331,14 @@ void HiveParquetFile::loadSubMinMaxIndex() } } +std::optional HiveParquetFile::getRowsImpl() +{ + if (!reader) + prepareReader(); + + auto meta = reader->parquet_reader()->metadata(); + return meta->num_rows(); +} + } #endif diff --git a/src/Storages/Hive/HiveFile.h b/src/Storages/Hive/HiveFile.h index aef9d72755a..21dddc1b68f 100644 --- a/src/Storages/Hive/HiveFile.h +++ b/src/Storages/Hive/HiveFile.h @@ -102,6 +102,12 @@ public: virtual String getPath() const { return path; } + virtual UInt64 getLastModTs() const { return last_modify_time; } + + virtual size_t getSize() const { return size; } + + std::optional getRows(); + virtual FieldVector getPartitionValues() const { return partition_values; } virtual String getNamenodeUrl() { return namenode_url; } @@ -144,15 +150,16 @@ public: return boost::algorithm::join(strs, "|"); } - inline UInt64 getLastModTs() const { return last_modify_time; } - inline size_t getSize() const { return size; } - protected: + virtual std::optional getRowsImpl() = 0; + FieldVector partition_values; String namenode_url; String path; UInt64 last_modify_time; size_t size; + std::optional rows; + NamesAndTypesList index_names_and_types; MinMaxIndexPtr minmax_idx; std::vector sub_minmax_idxes; @@ -182,12 +189,15 @@ public: virtual FileFormat getFormat() const override { return FileFormat::TEXT; } virtual String getName() const override { return "TEXT"; } + +protected: + std::optional getRowsImpl() override { return {}; } }; -class HiveOrcFile : public IHiveFile +class HiveORCFile : public IHiveFile { public: - HiveOrcFile( + HiveORCFile( const FieldVector & values_, const String & namenode_url_, const String & path_, @@ -214,6 +224,8 @@ protected: virtual void prepareReader(); virtual void prepareColumnMapping(); + std::optional getRowsImpl() override; + std::unique_ptr in; std::unique_ptr reader; std::map orc_column_positions; @@ -243,6 +255,7 @@ public: protected: virtual void prepareReader(); + std::optional getRowsImpl() override; std::unique_ptr in; std::unique_ptr reader; diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 97d735d4fc1..4d2a01f2c94 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -44,6 +44,7 @@ namespace ErrorCodes extern const int INVALID_PARTITION_VALUE; extern const int BAD_ARGUMENTS; extern const int CANNOT_OPEN_FILE; + extern const int LOGICAL_ERROR; } @@ -419,7 +420,7 @@ HiveFilePtr createHiveFile( } else if (format_name == "ORC") { - hive_file = std::make_shared(fields, namenode_url, path, ts, size, index_names_and_types, hive_settings, context); + hive_file = std::make_shared(fields, namenode_url, path, ts, size, index_names_and_types, hive_settings, context); } else if (format_name == "Parquet") { @@ -432,24 +433,26 @@ HiveFilePtr createHiveFile( return hive_file; } -std::vector StorageHive::collectHiveFilesFromPartition( +HiveFiles StorageHive::collectHiveFilesFromPartition( const Apache::Hadoop::Hive::Partition & partition, - SelectQueryInfo & query_info, + const SelectQueryInfo & query_info, HiveTableMetadataPtr hive_table_metadata, const HDFSFSPtr & fs, - ContextPtr context_) + ContextPtr context_, + PruneLevel prune_level) const { - LOG_DEBUG(log, "Collect hive files from partition {}", boost::join(partition.values, ",")); + LOG_DEBUG( + log, "Collect hive files from partition {}, prune_level:{}", boost::join(partition.values, ","), pruneLevelToString(prune_level)); - /// Skip partition "__HIVE_DEFAULT_PARTITION__" - bool has_default_partition = false; - for (const auto & value : partition.values) - { - if (value == "__HIVE_DEFAULT_PARTITION__") - { - has_default_partition = true; - break; - } + /// Skip partition "__HIVE_DEFAULT_PARTITION__" + bool has_default_partition = false; + for (const auto & value : partition.values) + { + if (value == "__HIVE_DEFAULT_PARTITION__") + { + has_default_partition = true; + break; + } } if (has_default_partition) return {}; @@ -479,25 +482,29 @@ std::vector StorageHive::collectHiveFilesFromPartition( if (!reader->pull(block) || !block.rows()) throw Exception("Could not parse partition value: " + wb.str(), ErrorCodes::INVALID_PARTITION_VALUE); - std::vector ranges; - ranges.reserve(partition_names.size()); + /// Get partition values FieldVector fields(partition_names.size()); for (size_t i = 0; i < partition_names.size(); ++i) - { block.getByPosition(i).column->get(0, fields[i]); - ranges.emplace_back(fields[i]); + + if (prune_level >= PruneLevel::Partition) + { + std::vector ranges; + ranges.reserve(partition_names.size()); + for (size_t i = 0; i < partition_names.size(); ++i) + ranges.emplace_back(fields[i]); + + const KeyCondition partition_key_condition(query_info, getContext(), partition_names, partition_minmax_idx_expr); + if (!partition_key_condition.checkInHyperrectangle(ranges, partition_types).can_be_true) + return {}; } - const KeyCondition partition_key_condition(query_info, getContext(), partition_names, partition_minmax_idx_expr); - if (!partition_key_condition.checkInHyperrectangle(ranges, partition_types).can_be_true) - return {}; - + HiveFiles hive_files; auto file_infos = listDirectory(partition.sd.location, hive_table_metadata, fs); - std::vector hive_files; hive_files.reserve(file_infos.size()); for (const auto & file_info : file_infos) { - auto hive_file = createHiveFileIfNeeded(file_info, fields, query_info, context_); + auto hive_file = createHiveFileIfNeeded(file_info, fields, query_info, context_, prune_level); if (hive_file) hive_files.push_back(hive_file); } @@ -511,12 +518,17 @@ StorageHive::listDirectory(const String & path, HiveTableMetadataPtr hive_table_ } HiveFilePtr StorageHive::createHiveFileIfNeeded( - const FileInfo & file_info, const FieldVector & fields, SelectQueryInfo & query_info, ContextPtr context_) + const FileInfo & file_info, + const FieldVector & fields, + const SelectQueryInfo & query_info, + ContextPtr context_, + PruneLevel prune_level) const { - LOG_TRACE(log, "Append hive file {}", file_info.path); + LOG_TRACE(log, "create hive file {} if needed, prune_level:{}", file_info.path, pruneLevelToString(prune_level)); + String filename = getBaseName(file_info.path); /// Skip temporary files starts with '.' - if (filename.find('.') == 0) + if (startsWith(filename, ".") == 0) return {}; auto hive_file = createHiveFile( @@ -531,34 +543,44 @@ HiveFilePtr StorageHive::createHiveFileIfNeeded( context_); /// Load file level minmax index and apply - const KeyCondition hivefile_key_condition(query_info, getContext(), hivefile_name_types.getNames(), hivefile_minmax_idx_expr); - if (hive_file->hasMinMaxIndex()) + if (prune_level >= PruneLevel::File) { - hive_file->loadMinMaxIndex(); - if (!hivefile_key_condition.checkInHyperrectangle(hive_file->getMinMaxIndex()->hyperrectangle, hivefile_name_types.getTypes()) - .can_be_true) + const KeyCondition hivefile_key_condition(query_info, getContext(), hivefile_name_types.getNames(), hivefile_minmax_idx_expr); + if (hive_file->hasMinMaxIndex()) { - LOG_TRACE(log, "Skip hive file {} by index {}", hive_file->getPath(), hive_file->describeMinMaxIndex(hive_file->getMinMaxIndex())); - return {}; - } - } - - /// Load sub-file level minmax index and apply - if (hive_file->hasSubMinMaxIndex()) - { - std::set skip_splits; - hive_file->loadSubMinMaxIndex(); - const auto & sub_minmax_idxes = hive_file->getSubMinMaxIndexes(); - for (size_t i = 0; i < sub_minmax_idxes.size(); ++i) - { - if (!hivefile_key_condition.checkInHyperrectangle(sub_minmax_idxes[i]->hyperrectangle, hivefile_name_types.getTypes()) + hive_file->loadMinMaxIndex(); + if (!hivefile_key_condition.checkInHyperrectangle(hive_file->getMinMaxIndex()->hyperrectangle, hivefile_name_types.getTypes()) .can_be_true) { - LOG_TRACE(log, "Skip split {} of hive file {}", i, hive_file->getPath()); - skip_splits.insert(i); + LOG_TRACE( + log, + "Skip hive file {} by index {}", + hive_file->getPath(), + hive_file->describeMinMaxIndex(hive_file->getMinMaxIndex())); + return {}; + } + } + + if (prune_level >= PruneLevel::Split) + { + /// Load sub-file level minmax index and apply + if (hive_file->hasSubMinMaxIndex()) + { + std::set skip_splits; + hive_file->loadSubMinMaxIndex(); + const auto & sub_minmax_idxes = hive_file->getSubMinMaxIndexes(); + for (size_t i = 0; i < sub_minmax_idxes.size(); ++i) + { + if (!hivefile_key_condition.checkInHyperrectangle(sub_minmax_idxes[i]->hyperrectangle, hivefile_name_types.getTypes()) + .can_be_true) + { + LOG_TRACE(log, "Skip split {} of hive file {}", i, hive_file->getPath()); + skip_splits.insert(i); + } + } + hive_file->setSkipSplits(skip_splits); } } - hive_file->setSkipSplits(skip_splits); } return hive_file; } @@ -591,6 +613,7 @@ void StorageHive::getActualColumnsToRead(Block & sample_block, const Block & hea } } } + Pipe StorageHive::read( const Names & column_names, const StorageSnapshotPtr & storage_snapshot, @@ -607,47 +630,9 @@ Pipe StorageHive::read( auto hive_metastore_client = HiveMetastoreClientFactory::instance().getOrCreate(hive_metastore_url, getContext()); auto hive_table_metadata = hive_metastore_client->getTableMetadata(hive_database, hive_table); - std::vector partitions = hive_table_metadata->getPartitions(); - /// Hive files to read - HiveFiles hive_files; - /// Mutext to protect hive_files, which maybe appended in multiple threads - std::mutex hive_files_mutex; - - ThreadPool pool{num_streams}; - if (!partitions.empty()) - { - for (const auto & partition : partitions) - { - pool.scheduleOrThrowOnError([&]() - { - auto hive_files_in_partition = collectHiveFilesFromPartition(partition, query_info, hive_table_metadata, fs, context_); - if (!hive_files_in_partition.empty()) - { - std::lock_guard lock(hive_files_mutex); - hive_files.insert(std::end(hive_files), std::begin(hive_files_in_partition), std::end(hive_files_in_partition)); - } - }); - } - pool.wait(); - } - else if (partition_name_types.empty()) /// Partition keys is empty - { - auto file_infos = listDirectory(hive_table_metadata->getTable()->sd.location, hive_table_metadata, fs); - for (const auto & file_info : file_infos) - { - pool.scheduleOrThrowOnError([&] - { - auto hive_file = createHiveFileIfNeeded(file_info, {}, query_info, context_); - if (hive_file) - { - std::lock_guard lock(hive_files_mutex); - hive_files.push_back(hive_file); - } - }); - } - pool.wait(); - } - else /// Partition keys is not empty but partitions is empty + /// Collect Hive files to read + HiveFiles hive_files = collectHiveFiles(num_streams, query_info, hive_table_metadata, fs, context_); + if (hive_files.empty()) return {}; auto sources_info = std::make_shared(); @@ -689,6 +674,63 @@ Pipe StorageHive::read( return Pipe::unitePipes(std::move(pipes)); } +HiveFiles StorageHive::collectHiveFiles( + unsigned max_threads, + const SelectQueryInfo & query_info, + HiveTableMetadataPtr hive_table_metadata, + const HDFSFSPtr & fs, + ContextPtr context_, + PruneLevel prune_level) const +{ + + std::vector partitions = hive_table_metadata->getPartitions(); + /// Hive table have no partition + if (!partition_name_types.empty() && partitions.empty()) + return {}; + + /// Hive files to collect + HiveFiles hive_files; + /// Mutext to protect hive_files, which maybe appended in multiple threads + std::mutex hive_files_mutex; + ThreadPool pool{max_threads}; + if (!partitions.empty()) + { + for (const auto & partition : partitions) + { + pool.scheduleOrThrowOnError( + [&]() + { + auto hive_files_in_partition + = collectHiveFilesFromPartition(partition, query_info, hive_table_metadata, fs, context_, prune_level); + if (!hive_files_in_partition.empty()) + { + std::lock_guard lock(hive_files_mutex); + hive_files.insert(std::end(hive_files), std::begin(hive_files_in_partition), std::end(hive_files_in_partition)); + } + }); + } + } + else /// Partition keys is empty but still have files + { + auto file_infos = listDirectory(hive_table_metadata->getTable()->sd.location, hive_table_metadata, fs); + for (const auto & file_info : file_infos) + { + pool.scheduleOrThrowOnError( + [&]() + { + auto hive_file = createHiveFileIfNeeded(file_info, {}, query_info, context_, prune_level); + if (hive_file) + { + std::lock_guard lock(hive_files_mutex); + hive_files.push_back(hive_file); + } + }); + } + } + pool.wait(); + return hive_files; +} + SinkToStoragePtr StorageHive::write(const ASTPtr & /*query*/, const StorageMetadataPtr & /* metadata_snapshot*/, ContextPtr /*context*/) { throw Exception("Method write is not implemented for StorageHive", ErrorCodes::NOT_IMPLEMENTED); @@ -701,6 +743,44 @@ NamesAndTypesList StorageHive::getVirtuals() const {"_file", std::make_shared(std::make_shared())}}; } +std::optional StorageHive::totalRows(const Settings & settings) const +{ + /// query_info is not used when prune_level == PruneLevel::None + SelectQueryInfo query_info; + return totalRowsImpl(settings, query_info, getContext(), PruneLevel::None); +} + +std::optional StorageHive::totalRowsByPartitionPredicate(const SelectQueryInfo & query_info, ContextPtr context_) const +{ + return totalRowsImpl(context_->getSettingsRef(), query_info, context_, PruneLevel::Partition); +} + +std::optional +StorageHive::totalRowsImpl(const Settings & settings, const SelectQueryInfo & query_info, ContextPtr context_, PruneLevel prune_level) const +{ + /// Row-based format like Text doesn't support totalRowsByPartitionPredicate + if (!isColumnOriented()) + return {}; + + auto hive_metastore_client = HiveMetastoreClientFactory::instance().getOrCreate(hive_metastore_url, getContext()); + auto hive_table_metadata = hive_metastore_client->getTableMetadata(hive_database, hive_table); + HDFSBuilderWrapper builder = createHDFSBuilder(hdfs_namenode_url, getContext()->getGlobalContext()->getConfigRef()); + HDFSFSPtr fs = createHDFSFS(builder.get()); + HiveFiles hive_files = collectHiveFiles(settings.max_threads, query_info, hive_table_metadata, fs, context_, prune_level); + + UInt64 total_rows = 0; + for (const auto & hive_file : hive_files) + { + auto file_rows = hive_file->getRows(); + if (!file_rows) + throw Exception( + ErrorCodes::LOGICAL_ERROR, "Rows of hive file:{} with format:{} not initialized", hive_file->getPath(), format_name); + total_rows += *file_rows; + } + return total_rows; +} + + void registerStorageHive(StorageFactory & factory) { factory.registerStorage( diff --git a/src/Storages/Hive/StorageHive.h b/src/Storages/Hive/StorageHive.h index bdf37cc9f04..1470b883585 100644 --- a/src/Storages/Hive/StorageHive.h +++ b/src/Storages/Hive/StorageHive.h @@ -26,7 +26,6 @@ class HiveSettings; class StorageHive final : public shared_ptr_helper, public IStorage, WithContext { friend struct shared_ptr_helper; - public: String getName() const override { return "Hive"; } @@ -54,6 +53,9 @@ public: bool isColumnOriented() const override; + std::optional totalRows(const Settings & settings) const override; + std::optional totalRowsByPartitionPredicate(const SelectQueryInfo & query_info, ContextPtr context_) const override; + protected: friend class StorageHiveSource; StorageHive( @@ -73,24 +75,56 @@ private: using FileInfo = HiveMetastoreClient::FileInfo; using HiveTableMetadataPtr = HiveMetastoreClient::HiveTableMetadataPtr; + enum class PruneLevel + { + None, /// Do not prune + Partition, + File, + Split, + Max = Split, + }; + + static String pruneLevelToString(PruneLevel level) + { + return String(magic_enum::enum_name(level)); + } + static ASTPtr extractKeyExpressionList(const ASTPtr & node); static std::vector listDirectory(const String & path, HiveTableMetadataPtr hive_table_metadata, const HDFSFSPtr & fs); void initMinMaxIndexExpression(); - std::vector collectHiveFilesFromPartition( - const Apache::Hadoop::Hive::Partition & partition, - SelectQueryInfo & query_info, + HiveFiles collectHiveFiles( + unsigned max_threads, + const SelectQueryInfo & query_info, HiveTableMetadataPtr hive_table_metadata, const HDFSFSPtr & fs, - ContextPtr context_); + ContextPtr context_, + PruneLevel prune_level = PruneLevel::Max) const; - HiveFilePtr - createHiveFileIfNeeded(const FileInfo & file_info, const FieldVector & fields, SelectQueryInfo & query_info, ContextPtr context_); + HiveFiles collectHiveFilesFromPartition( + const Apache::Hadoop::Hive::Partition & partition, + const SelectQueryInfo & query_info, + HiveTableMetadataPtr hive_table_metadata, + const HDFSFSPtr & fs, + ContextPtr context_, + PruneLevel prune_level = PruneLevel::Max) const; + + HiveFilePtr createHiveFileIfNeeded( + const FileInfo & file_info, + const FieldVector & fields, + const SelectQueryInfo & query_info, + ContextPtr context_, + PruneLevel prune_level = PruneLevel::Max) const; void getActualColumnsToRead(Block & sample_block, const Block & header_block, const NameSet & partition_columns) const; + void lazyInitialize(); + + std::optional + totalRowsImpl(const Settings & settings, const SelectQueryInfo & query_info, ContextPtr context_, PruneLevel prune_level) const; + String hive_metastore_url; /// Hive database and table @@ -122,9 +156,8 @@ private: std::shared_ptr storage_settings; Poco::Logger * log = &Poco::Logger::get("StorageHive"); - - void lazyInitialize(); }; + } #endif From f0de8eb62506adad8c939215090026bd81109d3c Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Mon, 4 Apr 2022 19:57:04 +0200 Subject: [PATCH 030/101] Extracted argument handling into a separate class to reuse it for makeDateTime() and makeDateTime64() --- src/Functions/makeDate.cpp | 97 +++++++++++++++++++++++--------------- 1 file changed, 60 insertions(+), 37 deletions(-) diff --git a/src/Functions/makeDate.cpp b/src/Functions/makeDate.cpp index 19fb14efc89..4ec765f0191 100644 --- a/src/Functions/makeDate.cpp +++ b/src/Functions/makeDate.cpp @@ -3,6 +3,8 @@ #include #include #include +#include +#include #include #include @@ -17,20 +19,69 @@ namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int ARGUMENT_OUT_OF_BOUND; } namespace { -// A helper function to simplify comparisons of valid YYYY-MM-DD values for <,>,= +/// A helper function to simplify comparisons of valid YYYY-MM-DD values for <,>,= inline constexpr Int64 YearMonthDayToSingleInt(Int64 year, Int64 month, Int64 day) { return year * 512 + month * 32 + day; } -// Common implementation for makeDate, makeDate32 +/// Common logic to handle numeric arguments like year, month, day, hour, minute, second +class FunctionWithNumericParamsBase : public IFunction +{ +public: + bool isInjective(const ColumnsWithTypeAndName &) const override + { + return false; /// invalid argument values and timestamps that are out of supported range are converted into a default value + } + + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + + bool useDefaultImplementationForNulls() const override { return true; } + + bool useDefaultImplementationForConstants() const override { return true; } + +protected: + template + void checkRequiredArguments(const ColumnsWithTypeAndName & arguments, const AgrumentNames & argument_names, const size_t optional_argument_count) const + { + if (arguments.size() < argument_names.size() || arguments.size() > argument_names.size() + optional_argument_count) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Function {} requires {} to {} arguments, but {} given", + getName(), argument_names.size(), argument_names.size() + optional_argument_count, arguments.size()); + + for (size_t i = 0; i < argument_names.size(); ++i) + { + DataTypePtr argument_type = arguments[i].type; + if (!isNumber(argument_type)) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Argument '{}' for function {} must be number", std::string(argument_names[i]), getName()); + } + } + + template + void convertRequiredArguments(const ColumnsWithTypeAndName & arguments, const AgrumentNames & argument_names, Columns & converted_arguments) const + { + const DataTypePtr converted_argument_type = std::make_shared(); + converted_arguments.clear(); + converted_arguments.reserve(arguments.size()); + for (size_t i = 0; i < argument_names.size(); ++i) + { + ColumnPtr argument_column = castColumn(arguments[i], converted_argument_type); + argument_column = argument_column->convertToFullColumnIfConst(); + converted_arguments.push_back(argument_column); + } + } +}; + +/// Common implementation for makeDate, makeDate32 template -class FunctionMakeDate : public IFunction +class FunctionMakeDate : public FunctionWithNumericParamsBase { private: static constexpr std::array argument_names = {"year", "month", "day"}; @@ -46,45 +97,17 @@ public: size_t getNumberOfArguments() const override { return argument_names.size(); } - bool isInjective(const ColumnsWithTypeAndName &) const override + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - return false; // {year,month,day} that are out of supported range are converted into a default value - } - - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } - - bool useDefaultImplementationForNulls() const override { return true; } - - bool useDefaultImplementationForConstants() const override { return true; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - if (arguments.size() != argument_names.size()) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Function {} requires 3 arguments, but {} given", getName(), arguments.size()); - - for (size_t i = 0; i < argument_names.size(); ++i) - { - DataTypePtr argument_type = arguments[i]; - if (!isNumber(argument_type)) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Argument '{}' for function {} must be number", std::string(argument_names[i]), getName()); - } + checkRequiredArguments(arguments, argument_names, 0); return std::make_shared(); } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { - const DataTypePtr converted_argument_type = std::make_shared(); Columns converted_arguments; - converted_arguments.reserve(arguments.size()); - for (const auto & argument : arguments) - { - ColumnPtr argument_column = castColumn(argument, converted_argument_type); - argument_column = argument_column->convertToFullColumnIfConst(); - converted_arguments.push_back(argument_column); - } + convertRequiredArguments(arguments, argument_names, converted_arguments); auto res_column = Traits::ReturnColumnType::create(input_rows_count); auto & result_data = res_column->getData(); @@ -119,7 +142,7 @@ public: } }; -// makeDate(year, month, day) +/// makeDate(year, month, day) struct MakeDateTraits { static constexpr auto name = "makeDate"; @@ -128,11 +151,11 @@ struct MakeDateTraits static constexpr auto MIN_YEAR = 1970; static constexpr auto MAX_YEAR = 2149; - // This date has the maximum day number that fits in 16-bit uint + /// This date has the maximum day number that fits in 16-bit uint static constexpr auto MAX_DATE = YearMonthDayToSingleInt(MAX_YEAR, 6, 6); }; -// makeDate32(year, month, day) +/// makeDate32(year, month, day) struct MakeDate32Traits { static constexpr auto name = "makeDate32"; From e0d6033c396e20f31d00b973f1a0c9f780888ee7 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 5 Apr 2022 14:26:49 -0400 Subject: [PATCH 031/101] all columns can participate in interpolate expression despite if they are selected or not, some optimization on expressionless INTERPOLATE --- src/Interpreters/ExpressionAnalyzer.cpp | 24 ++++++++ src/Interpreters/InterpreterSelectQuery.cpp | 59 +++++++++++-------- src/Interpreters/InterpreterSelectQuery.h | 2 - .../RequiredSourceColumnsVisitor.cpp | 24 ++++++++ src/Parsers/ASTSelectQuery.cpp | 12 ++-- src/Parsers/ParserSelectQuery.cpp | 22 ------- 6 files changed, 89 insertions(+), 54 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 841d7bc567f..de2b49e8509 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include @@ -1342,6 +1343,29 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendOrderBy(ExpressionActionsChai with_fill = true; } + if (auto interpolate_list = select_query->interpolate()) + { + auto find_columns = [&step](IAST * function) + { + auto fImpl = [&step](IAST * fn, auto fi) + { + if (auto ident = fn->as()) + { + step.addRequiredOutput(ident->getColumnName()); + return; + } + if (fn->as() || fn->as()) + for (auto ch : fn->children) + fi(ch.get(), fi); + return; + }; + fImpl(function, fImpl); + }; + + for (const auto & interpolate : interpolate_list->children) + find_columns(interpolate->as()->expr.get()); + } + if (optimize_read_in_order) { for (auto & child : select_query->orderBy()->children) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index d050b378916..0e46879d6c0 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -286,12 +286,6 @@ InterpreterSelectQuery::InterpreterSelectQuery( { checkStackSize(); - for (const auto & column : query_ptr->as().select()->children) - { - std::string alias = column->tryGetAlias(); - original_select_set.insert(alias.empty() ? column->getColumnName() : alias); - } - query_info.ignore_projections = options.ignore_projections; query_info.is_projection_query = options.is_projection_query; @@ -841,7 +835,7 @@ static SortDescription getSortDescription(const ASTSelectQuery & query, ContextP } static InterpolateDescriptionPtr getInterpolateDescription( - const ASTSelectQuery & query, Block block, const Aliases & aliases, const NameSet & original_select_set, ContextPtr context) + const ASTSelectQuery & query, const Block & source_block, const Block & result_block, const Aliases & aliases, ContextPtr context) { InterpolateDescriptionPtr interpolate_descr; if (query.interpolate()) @@ -849,26 +843,41 @@ static InterpolateDescriptionPtr getInterpolateDescription( std::unordered_set col_set; ColumnsWithTypeAndName columns; ASTPtr exprs = std::make_shared(); - for (const auto & elem : query.interpolate()->children) - { - const auto & interpolate = elem->as(); - ColumnWithTypeAndName *block_column = block.findByName(interpolate.column); - if (!block_column) - { - if (original_select_set.count(interpolate.column)) /// column was removed - continue; - throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, - "Missing column '{}' as an INTERPOLATE expression target", interpolate.column); - } - if (!col_set.insert(block_column->name).second) - throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, - "Duplicate INTERPOLATE column '{}'", interpolate.column); - columns.emplace_back(block_column->type, block_column->name); - exprs->children.emplace_back(interpolate.expr->clone()); + if (query.interpolate()->children.empty()) + { + std::unordered_map column_names; + for (const auto & column : result_block.getColumnsWithTypeAndName()) + column_names[column.name] = column.type; + for (const auto & elem : query.orderBy()->children) + if (elem->as()->with_fill) + column_names.erase(elem->as()->children.front()->getColumnName()); + for (const auto & [name, type] : column_names) + { + columns.emplace_back(type, name); + exprs->children.emplace_back(std::make_shared(name)); + } + } + else + { + for (const auto & elem : query.interpolate()->children) + { + const auto & interpolate = elem->as(); + const ColumnWithTypeAndName *block_column = source_block.findByName(interpolate.column); + if (!block_column) + throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, + "Missing column '{}' as an INTERPOLATE expression target", interpolate.column); + + if (!col_set.insert(block_column->name).second) + throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, + "Duplicate INTERPOLATE column '{}'", interpolate.column); + + columns.emplace_back(block_column->type, block_column->name); + exprs->children.emplace_back(interpolate.expr->clone()); + } } - auto syntax_result = TreeRewriter(context).analyze(exprs, block.getNamesAndTypesList()); + auto syntax_result = TreeRewriter(context).analyze(exprs, source_block.getNamesAndTypesList()); ExpressionAnalyzer analyzer(exprs, syntax_result, context); ActionsDAGPtr actions = analyzer.getActionsDAG(true); ActionsDAGPtr conv_dag = ActionsDAG::makeConvertingActions(actions->getResultColumns(), @@ -2545,7 +2554,7 @@ void InterpreterSelectQuery::executeWithFill(QueryPlan & query_plan) return; InterpolateDescriptionPtr interpolate_descr = - getInterpolateDescription(query, result_header, syntax_analyzer_result->aliases, original_select_set, context); + getInterpolateDescription(query, source_header, result_header, syntax_analyzer_result->aliases, context); auto filling_step = std::make_unique(query_plan.getCurrentDataStream(), std::move(fill_descr), interpolate_descr); query_plan.addStep(std::move(filling_step)); } diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index ec3775ab848..6bb12caff7d 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -185,8 +185,6 @@ private: Names required_columns; /// Structure of query source (table, subquery, etc). Block source_header; - /// Original set of selected columns - NameSet original_select_set; /// Actions to calculate ALIAS if required. ActionsDAGPtr alias_actions; diff --git a/src/Interpreters/RequiredSourceColumnsVisitor.cpp b/src/Interpreters/RequiredSourceColumnsVisitor.cpp index 21ec94a6917..f6907aaafd7 100644 --- a/src/Interpreters/RequiredSourceColumnsVisitor.cpp +++ b/src/Interpreters/RequiredSourceColumnsVisitor.cpp @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB { @@ -123,6 +124,29 @@ void RequiredSourceColumnsMatcher::visit(const ASTSelectQuery & select, const AS data.addColumnAliasIfAny(*node); } + if (auto interpolate_list = select.interpolate()) + { + auto find_columns = [&data](IAST * function) + { + auto fImpl = [&data](IAST * fn, auto fi) + { + if (auto ident = fn->as()) + { + data.addColumnIdentifier(*ident); + return; + } + if (fn->as() || fn->as()) + for (auto ch : fn->children) + fi(ch.get(), fi); + return; + }; + fImpl(function, fImpl); + }; + + for (const auto & interpolate : interpolate_list->children) + find_columns(interpolate->as()->expr.get()); + } + if (const auto & with = select.with()) { for (auto & node : with->children) diff --git a/src/Parsers/ASTSelectQuery.cpp b/src/Parsers/ASTSelectQuery.cpp index 22b34150d7a..1a12877c1c8 100644 --- a/src/Parsers/ASTSelectQuery.cpp +++ b/src/Parsers/ASTSelectQuery.cpp @@ -132,11 +132,13 @@ void ASTSelectQuery::formatImpl(const FormatSettings & s, FormatState & state, F if (interpolate()) { - s.ostr - << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "INTERPOLATE" << (s.hilite ? hilite_none : "") - << " ("; - interpolate()->formatImpl(s, state, frame); - s.ostr << " )"; + s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "INTERPOLATE" << (s.hilite ? hilite_none : ""); + if (interpolate()->children.size()) + { + s.ostr << " ("; + interpolate()->formatImpl(s, state, frame); + s.ostr << " )"; + } } } diff --git a/src/Parsers/ParserSelectQuery.cpp b/src/Parsers/ParserSelectQuery.cpp index a232451fdb9..572edd93e39 100644 --- a/src/Parsers/ParserSelectQuery.cpp +++ b/src/Parsers/ParserSelectQuery.cpp @@ -261,28 +261,6 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) return false; } else interpolate_expression_list = std::make_shared(); - - if (interpolate_expression_list->children.empty()) - { - std::unordered_map columns; - for (const auto & elem : select_expression_list->children) - { - std::string alias = elem->tryGetAlias(); - columns[alias.empty() ? elem->getColumnName() : alias] = elem; - } - - for (const auto & elem : order_expression_list->children) - if (elem->as()->with_fill) - columns.erase(elem->as()->children.front()->getColumnName()); - - for (const auto & [column, ast] : columns) - { - auto elem = std::make_shared(); - elem->column = column; - elem->expr = std::make_shared(column); - interpolate_expression_list->children.push_back(elem); - } - } } } } From 6b9a34908332d5bcc9aed6a9a426c1466d378eba Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Tue, 5 Apr 2022 15:02:34 -0400 Subject: [PATCH 032/101] Update SortDescription.h wrong merge fix --- src/Core/SortDescription.h | 26 ++++++++++++++------------ 1 file changed, 14 insertions(+), 12 deletions(-) diff --git a/src/Core/SortDescription.h b/src/Core/SortDescription.h index b057c371e5b..b86706e665a 100644 --- a/src/Core/SortDescription.h +++ b/src/Core/SortDescription.h @@ -48,18 +48,20 @@ struct SortColumnDescription FillColumnDescription fill_description; explicit SortColumnDescription( - size_t column_number_, int direction_ = 1, int nulls_direction_ = 1, - const std::shared_ptr & collator_ = nullptr, - bool with_fill_ = false, const FillColumnDescription & fill_description_ = FillColumnDescription{}) - : column_number(column_number_), direction(direction_), nulls_direction(nulls_direction_), collator(collator_) - , with_fill(with_fill_), fill_description(fill_description_) {} - - explicit SortColumnDescription( - const std::string & column_name_, int direction_ = 1, int nulls_direction_ = 1, - const std::shared_ptr & collator_ = nullptr, - bool with_fill_ = false, const FillColumnDescription & fill_description_ = FillColumnDescription{}) - : column_name(column_name_), column_number(0), direction(direction_), nulls_direction(nulls_direction_) - , collator(collator_), with_fill(with_fill_), fill_description(fill_description_) {} + const std::string & column_name_, + int direction_ = 1, + int nulls_direction_ = 1, + const std::shared_ptr & collator_ = nullptr, + bool with_fill_ = false, + const FillColumnDescription & fill_description_ = {}) + : column_name(column_name_) + , direction(direction_) + , nulls_direction(nulls_direction_) + , collator(collator_) + , with_fill(with_fill_) + , fill_description(fill_description_) + { + } bool operator == (const SortColumnDescription & other) const { From f9a4cd12d4a2932a72828b25ab71b4bc497a205f Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Tue, 5 Apr 2022 21:46:17 +0200 Subject: [PATCH 033/101] Make parallel reading of kafka consumers work again (issue #35153) --- src/Storages/Kafka/StorageKafka.cpp | 5 ++ tests/integration/test_storage_kafka/test.py | 69 ++++++++++++++++++++ 2 files changed, 74 insertions(+) diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 4c7465d587d..79c967ce9c0 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -664,6 +664,11 @@ bool StorageKafka::streamToViews() size_t rows = 0; { block_io.pipeline.complete(std::move(pipe)); + + // we need to read all consumers in parallel (sequential read may lead to situation + // when some of consumers are not used, and will break some Kafka consumer invariants) + block_io.pipeline.setNumThreads(stream_count); + block_io.pipeline.setProgressCallback([&](const Progress & progress) { rows += progress.read_rows.load(); }); CompletedPipelineExecutor executor(block_io.pipeline); executor.execute(); diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index e451e15a5d6..a74a045b9a3 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -1135,6 +1135,75 @@ def test_kafka_consumer_hang2(kafka_cluster): kafka_delete_topic(admin_client, topic_name) +# sequential read from different consumers leads to breaking lot of kafka invariants +# (first consumer will get all partitions initially, and may have problems in doing polls every 60 sec) +def test_kafka_read_consumers_in_parallel(kafka_cluster): + admin_client = KafkaAdminClient( + bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) + ) + + topic_name = "read_consumers_in_parallel" + kafka_create_topic(admin_client, topic_name, num_partitions=8) + + cancel = threading.Event() + def produce(): + while not cancel.is_set(): + messages = [] + for _ in range(100): + messages.append(json.dumps({"key": 0, "value": 0})) + kafka_produce(kafka_cluster, "read_consumers_in_parallel", messages) + time.sleep(1) + + kafka_thread = threading.Thread(target=produce) + kafka_thread.start() + + # when we have more than 1 consumer in a single table, + # and kafka_thread_per_consumer=0 + # all the consumers should be read in parallel, not in sequence. + # then reading in parallel 8 consumers with 1 seconds kafka_poll_timeout_ms and less than 1 sec limit + # we should have exactly 1 poll per consumer (i.e. 8 polls) every 1 seconds (from different threads) + # in case parallel consuming is not working we will have only 1 poll every 1 seconds (from the same thread). + instance.query( + f""" + DROP TABLE IF EXISTS test.kafka; + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.consumer; + + CREATE TABLE test.kafka (key UInt64, value UInt64) + ENGINE = Kafka + SETTINGS kafka_broker_list = 'kafka1:19092', + kafka_topic_list = '{topic_name}', + kafka_group_name = '{topic_name}', + kafka_format = 'JSONEachRow', + kafka_num_consumers = 8, + kafka_thread_per_consumer = 0, + kafka_poll_timeout_ms = 1000, + kafka_flush_interval_ms = 999; + CREATE TABLE test.view (key UInt64, value UInt64) ENGINE = Memory(); + CREATE MATERIALIZED VIEW test.consumer TO test.view AS SELECT * FROM test.kafka; + """ + ) + + instance.wait_for_log_line( + "kafka.*Polled batch of [0-9]+.*read_consumers_in_parallel", + repetitions=64, + look_behind_lines=100, + timeout=30 # we should get 64 polls in ~8 seconds, but when read sequentially it will take more than 64 sec + ) + + cancel.set() + kafka_thread.join() + + instance.query( + """ + DROP TABLE test.consumer; + DROP TABLE test.view; + DROP TABLE test.kafka; + """ + ) + kafka_delete_topic(admin_client, topic_name) + + def test_kafka_csv_with_delimiter(kafka_cluster): messages = [] for i in range(50): From a34d9f9cc3e9e19be65b01c6e3f12dd01b4a246b Mon Sep 17 00:00:00 2001 From: Roman Nikonov Date: Wed, 6 Apr 2022 02:40:56 +0300 Subject: [PATCH 034/101] Remove arg duplication and fix a typo --- tests/integration/runner | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/runner b/tests/integration/runner index 737eaeef683..522fdb7b745 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -76,7 +76,7 @@ def check_args_and_update_paths(args): args.src_dir = os.path.abspath(os.path.join(CURRENT_WORK_DIR, args.dockerd_volume)) if (not os.path.exists(os.path.join(args.base_configs_dir, "config.xml"))) and (not os.path.exists(os.path.join(args.base_configs_dir, "config.yaml"))): - raise Exception("No configs.xml or configs.yaml in {}".format(args.base_configs_dir)) + raise Exception("No config.xml or config.yaml in {}".format(args.base_configs_dir)) if (not os.path.exists(os.path.join(args.base_configs_dir, "users.xml"))) and (not os.path.exists(os.path.join(args.base_configs_dir, "users.yaml"))): raise Exception("No users.xml or users.yaml in {}".format(args.base_configs_dir)) @@ -279,7 +279,7 @@ if __name__ == "__main__": cmd = "docker run {net} {tty} --rm --name {name} --privileged \ --volume={odbc_bridge_bin}:/clickhouse-odbc-bridge --volume={bin}:/clickhouse \ - --volume={library_bridge_bin}:/clickhouse-library-bridge --volume={bin}:/clickhouse \ + --volume={library_bridge_bin}:/clickhouse-library-bridge \ --volume={base_cfg}:/clickhouse-config --volume={cases_dir}:/ClickHouse/tests/integration \ --volume={src_dir}/Server/grpc_protos:/ClickHouse/src/Server/grpc_protos \ --volume=/run:/run/host:ro \ From ac441b9dcf96120139be4efaa79def31369795e4 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 6 Apr 2022 00:04:51 -0400 Subject: [PATCH 035/101] compiler suggestions --- src/Interpreters/ExpressionAnalyzer.cpp | 8 ++++---- src/Interpreters/RequiredSourceColumnsVisitor.cpp | 8 ++++---- src/Parsers/ASTSelectQuery.cpp | 2 +- 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 40539fdaf76..4494ce82157 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1338,19 +1338,19 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendOrderBy(ExpressionActionsChai { auto find_columns = [&step](IAST * function) { - auto fImpl = [&step](IAST * fn, auto fi) + auto f_impl = [&step](IAST * fn, auto fi) { - if (auto ident = fn->as()) + if (auto * ident = fn->as()) { step.addRequiredOutput(ident->getColumnName()); return; } if (fn->as() || fn->as()) - for (auto ch : fn->children) + for (const auto & ch : fn->children) fi(ch.get(), fi); return; }; - fImpl(function, fImpl); + f_impl(function, f_impl); }; for (const auto & interpolate : interpolate_list->children) diff --git a/src/Interpreters/RequiredSourceColumnsVisitor.cpp b/src/Interpreters/RequiredSourceColumnsVisitor.cpp index f6907aaafd7..8ff47ee7972 100644 --- a/src/Interpreters/RequiredSourceColumnsVisitor.cpp +++ b/src/Interpreters/RequiredSourceColumnsVisitor.cpp @@ -128,19 +128,19 @@ void RequiredSourceColumnsMatcher::visit(const ASTSelectQuery & select, const AS { auto find_columns = [&data](IAST * function) { - auto fImpl = [&data](IAST * fn, auto fi) + auto f_impl = [&data](IAST * fn, auto fi) { - if (auto ident = fn->as()) + if (auto * ident = fn->as()) { data.addColumnIdentifier(*ident); return; } if (fn->as() || fn->as()) - for (auto ch : fn->children) + for (const auto & ch : fn->children) fi(ch.get(), fi); return; }; - fImpl(function, fImpl); + f_impl(function, f_impl); }; for (const auto & interpolate : interpolate_list->children) diff --git a/src/Parsers/ASTSelectQuery.cpp b/src/Parsers/ASTSelectQuery.cpp index 1a12877c1c8..048b9dfc38f 100644 --- a/src/Parsers/ASTSelectQuery.cpp +++ b/src/Parsers/ASTSelectQuery.cpp @@ -133,7 +133,7 @@ void ASTSelectQuery::formatImpl(const FormatSettings & s, FormatState & state, F if (interpolate()) { s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "INTERPOLATE" << (s.hilite ? hilite_none : ""); - if (interpolate()->children.size()) + if (!interpolate()->children.empty()) { s.ostr << " ("; interpolate()->formatImpl(s, state, frame); From 81d150ed432138481debc29aa78be74fdfee0971 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Mon, 4 Apr 2022 20:12:10 +0200 Subject: [PATCH 036/101] Implementation of makeDateTime() and makeDateTime64() --- src/Functions/makeDate.cpp | 239 ++++++++++++++++++ .../0_stateless/02244_make_datetime.reference | 33 +++ .../0_stateless/02244_make_datetime.sql | 39 +++ .../02245_make_datetime64.reference | 47 ++++ .../0_stateless/02245_make_datetime64.sql | 64 +++++ 5 files changed, 422 insertions(+) create mode 100644 tests/queries/0_stateless/02244_make_datetime.reference create mode 100644 tests/queries/0_stateless/02244_make_datetime.sql create mode 100644 tests/queries/0_stateless/02245_make_datetime64.reference create mode 100644 tests/queries/0_stateless/02245_make_datetime64.sql diff --git a/src/Functions/makeDate.cpp b/src/Functions/makeDate.cpp index 4ec765f0191..675d22d7f00 100644 --- a/src/Functions/makeDate.cpp +++ b/src/Functions/makeDate.cpp @@ -2,6 +2,8 @@ #include #include #include +#include +#include #include #include #include @@ -167,12 +169,249 @@ struct MakeDate32Traits static constexpr auto MAX_DATE = YearMonthDayToSingleInt(MAX_YEAR, 11, 11); }; +/// Common implementation for makeDateTime, makeDateTime64 +class FunctionMakeDateTimeBase : public FunctionWithNumericParamsBase +{ +protected: + static constexpr std::array argument_names = {"year", "month", "day", "hour", "minute", "second"}; + +public: + bool isVariadic() const override { return true; } + + size_t getNumberOfArguments() const override { return 0; } + +protected: + void checkRequiredArguments(const ColumnsWithTypeAndName & arguments, const size_t optional_argument_count) const + { + FunctionWithNumericParamsBase::checkRequiredArguments(arguments, argument_names, optional_argument_count); + } + + void convertRequiredArguments(const ColumnsWithTypeAndName & arguments, Columns & converted_arguments) const + { + FunctionWithNumericParamsBase::convertRequiredArguments(arguments, argument_names, converted_arguments); + } + + template + static Int64 dateTime(T year, T month, T day_of_month, T hour, T minute, T second, const DateLUTImpl & lut) + { + /// Note that hour, minute and second are checked against 99 to behave consistently with parsing DateTime from String + /// E.g. "select cast('1984-01-01 99:99:99' as DateTime);" returns "1984-01-05 04:40:39" + if (unlikely(year < DATE_LUT_MIN_YEAR || month < 1 || month > 12 || day_of_month < 1 || day_of_month > 31 || + hour < 0 || hour > 99 || minute < 0 || minute > 99 || second < 0 || second > 99)) + return lut.makeDateTime(DATE_LUT_MIN_YEAR-1, 1, 1, 0, 0, 0); + + if (unlikely(year > DATE_LUT_MAX_YEAR)) + return lut.makeDateTime(DATE_LUT_MAX_YEAR+1, 1, 1, 23, 59, 59); + + return lut.makeDateTime(year, month, day_of_month, hour, minute, second); + } + + std::string extractTimezone(const ColumnWithTypeAndName & timezone_argument) const + { + std::string timezone; + if (!isStringOrFixedString(timezone_argument.type) || !timezone_argument.column || (timezone_argument.column->size() != 1 && !typeid_cast(timezone_argument.column.get()))) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Argument 'timezone' for function {} must be const string", getName()); + timezone = timezone_argument.column->getDataAt(0).toString(); + + return timezone; + } +}; + +/// makeDateTime(year, month, day, hour, minute, second, [timezone]) +class FunctionMakeDateTime : public FunctionMakeDateTimeBase +{ +private: + static constexpr std::array optional_argument_names = {"timezone"}; + +public: + static constexpr auto name = "makeDateTime"; + + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + + String getName() const override { return name; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + checkRequiredArguments(arguments, optional_argument_names.size()); + + /// Optional timezone argument + std::string timezone; + if (arguments.size() == argument_names.size() + 1) + timezone = extractTimezone(arguments.back()); + + return std::make_shared(timezone); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + /// Optional timezone argument + std::string timezone; + if (arguments.size() == argument_names.size() + 1) + timezone = extractTimezone(arguments.back()); + + Columns converted_arguments; + convertRequiredArguments(arguments, converted_arguments); + + auto res_column = ColumnUInt32::create(input_rows_count); + auto & result_data = res_column->getData(); + + const auto & year_data = typeid_cast(*converted_arguments[0]).getData(); + const auto & month_data = typeid_cast(*converted_arguments[1]).getData(); + const auto & day_data = typeid_cast(*converted_arguments[2]).getData(); + const auto & hour_data = typeid_cast(*converted_arguments[3]).getData(); + const auto & minute_data = typeid_cast(*converted_arguments[4]).getData(); + const auto & second_data = typeid_cast(*converted_arguments[5]).getData(); + + const auto & date_lut = DateLUT::instance(timezone); + + for (size_t i = 0; i < input_rows_count; ++i) + { + const auto year = year_data[i]; + const auto month = month_data[i]; + const auto day = day_data[i]; + const auto hour = hour_data[i]; + const auto minute = minute_data[i]; + const auto second = second_data[i]; + + auto date_time = dateTime(year, month, day, hour, minute, second, date_lut); + if (unlikely(date_time < 0)) + date_time = 0; + else if (unlikely(date_time > 0x0ffffffffll)) + date_time = 0x0ffffffffll; + + result_data[i] = date_time; + } + + return res_column; + } +}; + +/// makeDateTime64(year, month, day, hour, minute, second, [fraction], [precision], [timezone]) +class FunctionMakeDateTime64 : public FunctionMakeDateTimeBase +{ +private: + static constexpr std::array optional_argument_names = {"fraction", "precision", "timezone"}; + static constexpr UInt8 DEFAULT_PRECISION = 3; + +public: + static constexpr auto name = "makeDateTime64"; + + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + + String getName() const override { return name; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + checkRequiredArguments(arguments, optional_argument_names.size()); + + if (arguments.size() >= argument_names.size() + 1) + { + const auto& fraction_argument = arguments[argument_names.size()]; + if (!isNumber(fraction_argument.type)) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Argument 'fraction' for function {} must be number", getName()); + } + + /// Optional precision argument + Int64 precision = DEFAULT_PRECISION; + if (arguments.size() >= argument_names.size() + 2) + precision = extractPrecision(arguments[argument_names.size() + 1]); + + /// Optional timezone argument + std::string timezone; + if (arguments.size() == argument_names.size() + 3) + timezone = extractTimezone(arguments.back()); + + return std::make_shared(precision, timezone); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + /// Optional precision argument + Int64 precision = DEFAULT_PRECISION; + if (arguments.size() >= argument_names.size() + 2) + precision = extractPrecision(arguments[argument_names.size() + 1]); + + /// Optional timezone argument + std::string timezone; + if (arguments.size() == argument_names.size() + 3) + timezone = extractTimezone(arguments.back()); + + Columns converted_arguments; + convertRequiredArguments(arguments, converted_arguments); + + /// Optional fraction argument + const ColumnVector::Container * fraction_data = nullptr; + if (arguments.size() >= argument_names.size() + 1) + { + ColumnPtr fraction_column = castColumn(arguments[argument_names.size()], std::make_shared()); + fraction_column = fraction_column->convertToFullColumnIfConst(); + converted_arguments.push_back(fraction_column); + fraction_data = &typeid_cast(*converted_arguments[6]).getData(); + } + + auto res_column = ColumnDecimal::create(input_rows_count, precision); + auto & result_data = res_column->getData(); + + const auto & year_data = typeid_cast(*converted_arguments[0]).getData(); + const auto & month_data = typeid_cast(*converted_arguments[1]).getData(); + const auto & day_data = typeid_cast(*converted_arguments[2]).getData(); + const auto & hour_data = typeid_cast(*converted_arguments[3]).getData(); + const auto & minute_data = typeid_cast(*converted_arguments[4]).getData(); + const auto & second_data = typeid_cast(*converted_arguments[5]).getData(); + + const auto & date_lut = DateLUT::instance(timezone); + + const auto max_fraction = pow(10, precision) - 1; + + for (size_t i = 0; i < input_rows_count; ++i) + { + const auto year = year_data[i]; + const auto month = month_data[i]; + const auto day = day_data[i]; + const auto hour = hour_data[i]; + const auto minute = minute_data[i]; + const auto second = second_data[i]; + + auto date_time = dateTime(year, month, day, hour, minute, second, date_lut); + + auto fraction = fraction_data ? (*fraction_data)[i] : 0; + if (unlikely(fraction < 0)) + fraction = 0; + else if (unlikely(fraction > max_fraction)) + fraction = max_fraction; + + result_data[i] = DecimalUtils::decimalFromComponents(date_time, fraction, precision); + } + + return res_column; + } + +private: + UInt8 extractPrecision(const ColumnWithTypeAndName & precision_argument) const + { + Int64 precision = DEFAULT_PRECISION; + if (!isNumber(precision_argument.type) || !precision_argument.column || (precision_argument.column->size() != 1 && !typeid_cast(precision_argument.column.get()))) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Argument 'precision' for function {} must be constant number", getName()); + precision = precision_argument.column->getInt(0); + if (precision < 0 || precision > 9) + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, + "Argument 'precision' for function {} must be in range [0, 9]", getName()); + + return precision; + } +}; + } void registerFunctionsMakeDate(FunctionFactory & factory) { factory.registerFunction>(); factory.registerFunction>(); + factory.registerFunction(); + factory.registerFunction(); } } diff --git a/tests/queries/0_stateless/02244_make_datetime.reference b/tests/queries/0_stateless/02244_make_datetime.reference new file mode 100644 index 00000000000..57524c26254 --- /dev/null +++ b/tests/queries/0_stateless/02244_make_datetime.reference @@ -0,0 +1,33 @@ +1991-08-24 21:04:00 +1991-08-24 21:04:00 +1991-08-24 19:04:00 +DateTime +DateTime(\'CET\') +1970-01-01 00:00:00 +1970-01-01 00:00:00 +2106-02-07 06:28:15 +2106-02-07 06:28:15 +2106-02-07 06:28:15 +2106-02-07 06:28:15 +2106-02-07 06:28:15 +1970-01-01 00:00:00 +1970-01-01 00:00:00 +1970-01-01 00:00:00 +1970-01-01 00:00:00 +1984-01-02 01:00:00 +1984-01-01 01:10:00 +1984-01-01 00:01:10 +1984-01-01 00:00:00 +1983-03-01 00:00:00 +1970-01-01 00:00:00 +1970-01-01 00:00:00 +1970-01-01 00:00:00 +1970-01-01 00:00:00 +1970-01-01 00:00:00 +1970-01-01 00:00:00 +2106-02-07 06:28:15 +1970-01-01 00:00:00 +1970-01-01 00:00:00 +1970-01-01 00:00:00 +1970-01-01 00:00:00 +1970-01-01 00:00:00 diff --git a/tests/queries/0_stateless/02244_make_datetime.sql b/tests/queries/0_stateless/02244_make_datetime.sql new file mode 100644 index 00000000000..9b8f561994b --- /dev/null +++ b/tests/queries/0_stateless/02244_make_datetime.sql @@ -0,0 +1,39 @@ +select makeDateTime(1991, 8, 24, 21, 4, 0); +select makeDateTime(1991, 8, 24, 21, 4, 0, 'CET'); +select cast(makeDateTime(1991, 8, 24, 21, 4, 0, 'CET') as DateTime('UTC')); + +select toTypeName(makeDateTime(1991, 8, 24, 21, 4, 0)); +select toTypeName(makeDateTime(1991, 8, 24, 21, 4, 0, 'CET')); + +select makeDateTime(1925, 1, 1, 0, 0, 0, 'UTC'); +select makeDateTime(1924, 12, 31, 23, 59, 59, 'UTC'); +select makeDateTime(2283, 11, 11, 23, 59, 59, 'UTC'); +select makeDateTime(2283, 11, 12, 0, 0, 0, 'UTC'); +select makeDateTime(2262, 4, 11, 23, 47, 16, 'UTC'); +select makeDateTime(2262, 4, 11, 23, 47, 17, 'UTC'); +select makeDateTime(2262, 4, 11, 23, 47, 16, 'UTC'); + +select makeDateTime(1984, 0, 1, 0, 0, 0, 'UTC'); +select makeDateTime(1984, 1, 0, 0, 0, 0, 'UTC'); +select makeDateTime(1984, 13, 1, 0, 0, 0, 'UTC'); +select makeDateTime(1984, 1, 41, 0, 0, 0, 'UTC'); +select makeDateTime(1984, 1, 1, 25, 0, 0, 'UTC'); +select makeDateTime(1984, 1, 1, 0, 70, 0, 'UTC'); +select makeDateTime(1984, 1, 1, 0, 0, 70, 'UTC'); +select makeDateTime(1984, 1, 1, 0, 0, 0, 'not a timezone'); -- { serverError 1000 } + +select makeDateTime(1984, 1, 1, 0, 0, 0, 'UTC'); +select makeDateTime(1983, 2, 29, 0, 0, 0, 'UTC'); +select makeDateTime(-1984, 1, 1, 0, 0, 0, 'UTC'); +select makeDateTime(1984, -1, 1, 0, 0, 0, 'UTC'); +select makeDateTime(1984, 1, -1, 0, 0, 0, 'UTC'); +select makeDateTime(1984, 1, 1, -1, 0, 0, 'UTC'); +select makeDateTime(1984, 1, 1, 0, -1, 0, 'UTC'); +select makeDateTime(1984, 1, 1, 0, 0, -1, 'UTC'); + +select makeDateTime(65537, 8, 24, 21, 4, 0, 'UTC'); +select makeDateTime(1991, 65537, 24, 21, 4, 0, 'UTC'); +select makeDateTime(1991, 8, 65537, 21, 4, 0, 'UTC'); +select makeDateTime(1991, 8, 24, 65537, 4, 0, 'UTC'); +select makeDateTime(1991, 8, 24, 21, 65537, 0, 'UTC'); +select makeDateTime(1991, 8, 24, 21, 4, 65537, 'UTC'); \ No newline at end of file diff --git a/tests/queries/0_stateless/02245_make_datetime64.reference b/tests/queries/0_stateless/02245_make_datetime64.reference new file mode 100644 index 00000000000..bcdf7dacccc --- /dev/null +++ b/tests/queries/0_stateless/02245_make_datetime64.reference @@ -0,0 +1,47 @@ +1991-08-24 21:04:00.000 +1991-08-24 21:04:00.123 +1991-08-24 21:04:00.001234 +1991-08-24 21:04:00.0001234 +1991-08-24 19:04:00.0001234 +DateTime64(3) +DateTime64(3) +DateTime64(6) +DateTime64(7, \'CET\') +DateTime64(7, \'UTC\') +1925-01-01 00:00:00.000000000 +1925-01-01 00:00:00.999999999 +2283-11-11 23:59:59.99999999 +2262-04-11 23:47:16.854775807 +2262-04-11 23:47:16.85477581 +1991-08-24 21:04:00 +1991-08-24 21:04:00.9 +1991-08-24 21:04:00.99 +1991-08-24 21:04:00.999 +1991-08-24 21:04:00.1234 +1991-08-24 21:04:00.01234 +1991-08-24 21:04:00.001234 +1991-08-24 21:04:00.0001234 +1991-08-24 21:04:00.00001234 +1991-08-24 21:04:00.000001234 +1925-01-01 00:00:00.000000000 +1925-01-01 00:00:00.000000000 +1925-01-01 00:00:00.000000000 +1925-01-01 00:00:00.000000000 +1984-01-02 01:00:00.000000000 +1984-01-01 01:10:00.000000000 +1984-01-01 00:01:10.000000000 +1984-01-01 00:00:00.000000000 +1983-03-01 00:00:00.000000000 +1925-01-01 00:00:00.000000000 +1925-01-01 00:00:00.000000000 +1925-01-01 00:00:00.000000000 +1925-01-01 00:00:00.000000000 +1925-01-01 00:00:00.000000000 +1925-01-01 00:00:00.000000000 +1984-01-01 00:00:00.000000000 +2283-11-11 23:59:59.000 +1925-01-01 00:00:00.000 +1925-01-01 00:00:00.000 +1925-01-01 00:00:00.000 +1925-01-01 00:00:00.000 +1925-01-01 00:00:00.000 diff --git a/tests/queries/0_stateless/02245_make_datetime64.sql b/tests/queries/0_stateless/02245_make_datetime64.sql new file mode 100644 index 00000000000..7bd3c1df2da --- /dev/null +++ b/tests/queries/0_stateless/02245_make_datetime64.sql @@ -0,0 +1,64 @@ +select makeDateTime64(1991, 8, 24, 21, 4, 0); +select makeDateTime64(1991, 8, 24, 21, 4, 0, 123); +select makeDateTime64(1991, 8, 24, 21, 4, 0, 1234, 6); +select makeDateTime64(1991, 8, 24, 21, 4, 0, 1234, 7, 'CET'); +select cast(makeDateTime64(1991, 8, 24, 21, 4, 0, 1234, 7, 'CET') as DateTime64(7, 'UTC')); + +select toTypeName(makeDateTime64(1991, 8, 24, 21, 4, 0)); +select toTypeName(makeDateTime64(1991, 8, 24, 21, 4, 0, 123)); +select toTypeName(makeDateTime64(1991, 8, 24, 21, 4, 0, 1234, 6)); +select toTypeName(makeDateTime64(1991, 8, 24, 21, 4, 0, 1234, 7, 'CET')); +select toTypeName(cast(makeDateTime64(1991, 8, 24, 21, 4, 0, 1234, 7, 'CET') as DateTime64(7, 'UTC'))); + +select makeDateTime64(1925, 1, 1, 0, 0, 0, 0, 9, 'UTC'); +select makeDateTime64(1924, 12, 31, 23, 59, 59, 999999999, 9, 'UTC'); +select makeDateTime64(2283, 11, 11, 23, 59, 59, 99999999, 8, 'UTC'); +select makeDateTime64(2283, 11, 11, 23, 59, 59, 999999999, 9, 'UTC'); -- { serverError 407 } +select makeDateTime64(2262, 4, 11, 23, 47, 16, 854775807, 9, 'UTC'); +select makeDateTime64(2262, 4, 11, 23, 47, 16, 854775808, 9, 'UTC'); -- { serverError 407 } +select makeDateTime64(2262, 4, 11, 23, 47, 16, 85477581, 8, 'UTC'); + +select makeDateTime64(1991, 8, 24, 21, 4, 0, 1234, 0, 'CET'); +select makeDateTime64(1991, 8, 24, 21, 4, 0, 1234, 1, 'CET'); +select makeDateTime64(1991, 8, 24, 21, 4, 0, 1234, 2, 'CET'); +select makeDateTime64(1991, 8, 24, 21, 4, 0, 1234, 3, 'CET'); +select makeDateTime64(1991, 8, 24, 21, 4, 0, 1234, 4, 'CET'); +select makeDateTime64(1991, 8, 24, 21, 4, 0, 1234, 5, 'CET'); +select makeDateTime64(1991, 8, 24, 21, 4, 0, 1234, 6, 'CET'); +select makeDateTime64(1991, 8, 24, 21, 4, 0, 1234, 7, 'CET'); +select makeDateTime64(1991, 8, 24, 21, 4, 0, 1234, 8, 'CET'); +select makeDateTime64(1991, 8, 24, 21, 4, 0, 1234, 9, 'CET'); +select makeDateTime64(1991, 8, 24, 21, 4, 0, 1234, 10, 'CET'); -- { serverError 69 } +select makeDateTime64(1991, 8, 24, 21, 4, 0, 1234, -1, 'CET'); -- { serverError 69 } + +select makeDateTime64(1984, 0, 1, 0, 0, 0, 0, 9, 'UTC'); +select makeDateTime64(1984, 1, 0, 0, 0, 0, 0, 9, 'UTC'); +select makeDateTime64(1984, 13, 1, 0, 0, 0, 0, 9, 'UTC'); +select makeDateTime64(1984, 1, 41, 0, 0, 0, 0, 9, 'UTC'); +select makeDateTime64(1984, 1, 1, 25, 0, 0, 0, 9, 'UTC'); +select makeDateTime64(1984, 1, 1, 0, 70, 0, 0, 9, 'UTC'); +select makeDateTime64(1984, 1, 1, 0, 0, 70, 0, 9, 'UTC'); +select makeDateTime64(1984, 1, 1, 0, 0, 0, 0, 9, 'not a timezone'); -- { serverError 1000 } + +select makeDateTime64(1984, 1, 1, 0, 0, 0, 0, 9, 'UTC'); +select makeDateTime64(1983, 2, 29, 0, 0, 0, 0, 9, 'UTC'); +select makeDateTime64(-1984, 1, 1, 0, 0, 0, 0, 9, 'UTC'); +select makeDateTime64(1984, -1, 1, 0, 0, 0, 0, 9, 'UTC'); +select makeDateTime64(1984, 1, -1, 0, 0, 0, 0, 9, 'UTC'); +select makeDateTime64(1984, 1, 1, -1, 0, 0, 0, 9, 'UTC'); +select makeDateTime64(1984, 1, 1, 0, -1, 0, 0, 9, 'UTC'); +select makeDateTime64(1984, 1, 1, 0, 0, -1, 0, 9, 'UTC'); +select makeDateTime64(1984, 1, 1, 0, 0, 0, -1, 9, 'UTC'); + +select makeDateTime64(65537, 8, 24, 21, 4, 0); +select makeDateTime64(1991, 65537, 24, 21, 4, 0); +select makeDateTime64(1991, 8, 65537, 21, 4, 0); +select makeDateTime64(1991, 8, 24, 65537, 4, 0); +select makeDateTime64(1991, 8, 24, 21, 65537, 0); +select makeDateTime64(1991, 8, 24, 21, 4, 65537); + +select makeDateTime64(year, 1, 1, 1, 0, 0, 0, precision, timezone) from ( + select 1984 as year, 5 as precision, 'UTC' as timezone + union all + select 1985 as year, 5 as precision, 'UTC' as timezone +); -- { serverError 43 } From 7dbe8bc2dcff387503945e4ac4ad34e0aa5071cb Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 7 Apr 2022 01:21:24 -0400 Subject: [PATCH 037/101] major bugs fixed, tests added, docs updated --- .../sql-reference/statements/select/index.md | 2 +- .../statements/select/order-by.md | 4 +- .../sql-reference/statements/select/index.md | 2 +- .../statements/select/order-by.md | 4 +- src/Interpreters/ExpressionAnalyzer.cpp | 12 ++- src/Interpreters/InterpreterSelectQuery.cpp | 39 +++++--- .../RequiredSourceColumnsVisitor.cpp | 10 +- src/Interpreters/TreeRewriter.cpp | 26 ++++- src/Parsers/ASTInterpolateElement.h | 3 +- src/Parsers/ExpressionElementParsers.cpp | 1 + .../0_stateless/02233_interpolate_1.reference | 96 +++++++++++++++++++ .../0_stateless/02233_interpolate_1.sql | 23 +++++ 12 files changed, 196 insertions(+), 26 deletions(-) diff --git a/docs/en/sql-reference/statements/select/index.md b/docs/en/sql-reference/statements/select/index.md index 0386083b5df..c5421c83091 100644 --- a/docs/en/sql-reference/statements/select/index.md +++ b/docs/en/sql-reference/statements/select/index.md @@ -22,7 +22,7 @@ SELECT [DISTINCT [ON (column1, column2, ...)]] expr_list [WHERE expr] [GROUP BY expr_list] [WITH ROLLUP|WITH CUBE] [WITH TOTALS] [HAVING expr] -[ORDER BY expr_list] [WITH FILL] [FROM expr] [TO expr] [STEP expr] [INTERPOLATE (expr_list)] +[ORDER BY expr_list] [WITH FILL] [FROM expr] [TO expr] [STEP expr] [INTERPOLATE [(expr_list)]] [LIMIT [offset_value, ]n BY columns] [LIMIT [n, ]m] [WITH TIES] [SETTINGS ...] diff --git a/docs/en/sql-reference/statements/select/order-by.md b/docs/en/sql-reference/statements/select/order-by.md index 06d3b985854..04630ba1075 100644 --- a/docs/en/sql-reference/statements/select/order-by.md +++ b/docs/en/sql-reference/statements/select/order-by.md @@ -280,7 +280,7 @@ To fill multiple columns, add `WITH FILL` modifier with optional parameters afte ``` sql ORDER BY expr [WITH FILL] [FROM const_expr] [TO const_expr] [STEP const_numeric_expr], ... exprN [WITH FILL] [FROM expr] [TO expr] [STEP numeric_expr] -[INTERPOLATE (col AS expr, ... colN AS exprN)] +[INTERPOLATE [(col [AS expr], ... colN [AS exprN])]] ``` `WITH FILL` can be applied for fields with Numeric (all kinds of float, decimal, int) or Date/DateTime types. When applied for `String` fields, missed values are filled with empty strings. @@ -288,7 +288,7 @@ When `FROM const_expr` not defined sequence of filling use minimal `expr` field When `TO const_expr` not defined sequence of filling use maximum `expr` field value from `ORDER BY`. When `STEP const_numeric_expr` defined then `const_numeric_expr` interprets `as is` for numeric types, as `days` for Date type, as `seconds` for DateTime type. It also supports [INTERVAL](https://clickhouse.com/docs/en/sql-reference/data-types/special-data-types/interval/) data type representing time and date intervals. When `STEP const_numeric_expr` omitted then sequence of filling use `1.0` for numeric type, `1 day` for Date type and `1 second` for DateTime type. -`INTERPOLATE` can be applied to columns not participating in `ORDER BY WITH FILL`. Such columns are filled based on previous field value by applying `expr`. +`INTERPOLATE` can be applied to columns not participating in `ORDER BY WITH FILL`. Such columns are filled based on previous fields values by applying `expr`. If `expr` is not present will repeate previous value. Omitted list will result in including all allowed columns. Example of a query without `WITH FILL`: diff --git a/docs/ru/sql-reference/statements/select/index.md b/docs/ru/sql-reference/statements/select/index.md index 240a0fff7b6..fead3c11060 100644 --- a/docs/ru/sql-reference/statements/select/index.md +++ b/docs/ru/sql-reference/statements/select/index.md @@ -20,7 +20,7 @@ SELECT [DISTINCT [ON (column1, column2, ...)]] expr_list [WHERE expr] [GROUP BY expr_list] [WITH ROLLUP|WITH CUBE] [WITH TOTALS] [HAVING expr] -[ORDER BY expr_list] [WITH FILL] [FROM expr] [TO expr] [STEP expr] [INTERPOLATE (expr_list)] +[ORDER BY expr_list] [WITH FILL] [FROM expr] [TO expr] [STEP expr] [INTERPOLATE [(expr_list)]] [LIMIT [offset_value, ]n BY columns] [LIMIT [n, ]m] [WITH TIES] [SETTINGS ...] diff --git a/docs/ru/sql-reference/statements/select/order-by.md b/docs/ru/sql-reference/statements/select/order-by.md index 8a4a8d5d111..e293e62e34c 100644 --- a/docs/ru/sql-reference/statements/select/order-by.md +++ b/docs/ru/sql-reference/statements/select/order-by.md @@ -280,7 +280,7 @@ SELECT * FROM collate_test ORDER BY s ASC COLLATE 'en'; ```sql ORDER BY expr [WITH FILL] [FROM const_expr] [TO const_expr] [STEP const_numeric_expr], ... exprN [WITH FILL] [FROM expr] [TO expr] [STEP numeric_expr] -[INTERPOLATE (col AS expr, ... colN AS exprN)] +[INTERPOLATE [(col [AS expr], ... colN [AS exprN])]] ``` `WITH FILL` может быть применен к полям с числовыми (все разновидности float, int, decimal) или временными (все разновидности Date, DateTime) типами. В случае применения к полям типа `String` недостающие значения заполняются пустой строкой. @@ -290,7 +290,7 @@ ORDER BY expr [WITH FILL] [FROM const_expr] [TO const_expr] [STEP const_numeric_ Когда `STEP const_numeric_expr` не указан, тогда используется `1.0` для числовых типов, `1 день` для типа Date и `1 секунда` для типа DateTime. -`INTERPOLATE` может быть применен к колонкам, не участвующим в `ORDER BY WITH FILL`. Такие колонки заполняются значениями, вычисляемыми применением `expr` к предыдущему значению. +`INTERPOLATE` может быть применен к колонкам, не участвующим в `ORDER BY WITH FILL`. Такие колонки заполняются значениями, вычисляемыми применением `expr` к предыдущему значению. Если `expr` опущен, то колонка заполняется предыдущим значением. Если список колонок не указан, то включаются все разрешенные колонки. Пример запроса без использования `WITH FILL`: ```sql diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 4494ce82157..dd3162aa8a4 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1336,13 +1336,19 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendOrderBy(ExpressionActionsChai if (auto interpolate_list = select_query->interpolate()) { - auto find_columns = [&step](IAST * function) + + NameSet select; + for (const auto & child : select_query->select()->children) + select.insert(child->getAliasOrColumnName()); + + auto find_columns = [&step, &select](IAST * function) { - auto f_impl = [&step](IAST * fn, auto fi) + auto f_impl = [&step, &select](IAST * fn, auto fi) { if (auto * ident = fn->as()) { - step.addRequiredOutput(ident->getColumnName()); + if (select.count(ident->getColumnName()) == 0) + step.addRequiredOutput(ident->getColumnName()); return; } if (fn->as() || fn->as()) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index dcd2f2556a9..2c3c2dcf23d 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -840,8 +840,8 @@ static InterpolateDescriptionPtr getInterpolateDescription( InterpolateDescriptionPtr interpolate_descr; if (query.interpolate()) { - std::unordered_set col_set; - ColumnsWithTypeAndName columns; + NamesAndTypesList source_columns; + ColumnsWithTypeAndName result_columns; ASTPtr exprs = std::make_shared(); if (query.interpolate()->children.empty()) @@ -854,34 +854,49 @@ static InterpolateDescriptionPtr getInterpolateDescription( column_names.erase(elem->as()->children.front()->getColumnName()); for (const auto & [name, type] : column_names) { - columns.emplace_back(type, name); + source_columns.emplace_back(name, type); + result_columns.emplace_back(type, name); exprs->children.emplace_back(std::make_shared(name)); } } else { + NameSet col_set; for (const auto & elem : query.interpolate()->children) { const auto & interpolate = elem->as(); - const ColumnWithTypeAndName *block_column = source_block.findByName(interpolate.column); - if (!block_column) + + if (const ColumnWithTypeAndName *result_block_column = result_block.findByName(interpolate.column)) + { + if (!col_set.insert(result_block_column->name).second) + throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, + "Duplicate INTERPOLATE column '{}'", interpolate.column); + + result_columns.emplace_back(result_block_column->type, result_block_column->name); + } + else throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, "Missing column '{}' as an INTERPOLATE expression target", interpolate.column); - if (!col_set.insert(block_column->name).second) - throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, - "Duplicate INTERPOLATE column '{}'", interpolate.column); - - columns.emplace_back(block_column->type, block_column->name); exprs->children.emplace_back(interpolate.expr->clone()); } + + col_set.clear(); + for (const auto & column : source_block) + { + source_columns.emplace_back(column.name, column.type); + col_set.insert(column.name); + } + for (const auto & column : result_block) + if( col_set.count(column.name) == 0) + source_columns.emplace_back(column.name, column.type); } - auto syntax_result = TreeRewriter(context).analyze(exprs, source_block.getNamesAndTypesList()); + auto syntax_result = TreeRewriter(context).analyze(exprs, source_columns); ExpressionAnalyzer analyzer(exprs, syntax_result, context); ActionsDAGPtr actions = analyzer.getActionsDAG(true); ActionsDAGPtr conv_dag = ActionsDAG::makeConvertingActions(actions->getResultColumns(), - columns, ActionsDAG::MatchColumnsMode::Position, true); + result_columns, ActionsDAG::MatchColumnsMode::Position, true); ActionsDAGPtr merge_dag = ActionsDAG::merge(std::move(*actions->clone()), std::move(*conv_dag)); interpolate_descr = std::make_shared(merge_dag, aliases); diff --git a/src/Interpreters/RequiredSourceColumnsVisitor.cpp b/src/Interpreters/RequiredSourceColumnsVisitor.cpp index 8ff47ee7972..c46d0ac7bf0 100644 --- a/src/Interpreters/RequiredSourceColumnsVisitor.cpp +++ b/src/Interpreters/RequiredSourceColumnsVisitor.cpp @@ -115,9 +115,12 @@ void RequiredSourceColumnsMatcher::visit(const ASTPtr & ast, Data & data) void RequiredSourceColumnsMatcher::visit(const ASTSelectQuery & select, const ASTPtr &, Data & data) { + NameSet select_columns; /// special case for top-level SELECT items: they are publics for (auto & node : select.select()->children) { + select_columns.insert(node->getAliasOrColumnName()); + if (const auto * identifier = node->as()) data.addColumnIdentifier(*identifier); else @@ -126,13 +129,14 @@ void RequiredSourceColumnsMatcher::visit(const ASTSelectQuery & select, const AS if (auto interpolate_list = select.interpolate()) { - auto find_columns = [&data](IAST * function) + auto find_columns = [&data, &select_columns](IAST * function) { - auto f_impl = [&data](IAST * fn, auto fi) + auto f_impl = [&data, &select_columns](IAST * fn, auto fi) { if (auto * ident = fn->as()) { - data.addColumnIdentifier(*ident); + if (select_columns.count(ident->getColumnName()) == 0) + data.addColumnIdentifier(*ident); return; } if (fn->as() || fn->as()) diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 929e516f687..d0e54f9f598 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -32,6 +32,7 @@ #include #include #include +#include #include #include @@ -420,7 +421,8 @@ void renameDuplicatedColumns(const ASTSelectQuery * select_query) /// Sometimes we have to calculate more columns in SELECT clause than will be returned from query. /// This is the case when we have DISTINCT or arrayJoin: we require more columns in SELECT even if we need less columns in result. /// Also we have to remove duplicates in case of GLOBAL subqueries. Their results are placed into tables so duplicates are impossible. -void removeUnneededColumnsFromSelectClause(const ASTSelectQuery * select_query, const Names & required_result_columns, bool remove_dups) +/// Also remove all INTERPOLATE columns which are not in SELECT anymore. +void removeUnneededColumnsFromSelectClause(ASTSelectQuery * select_query, const Names & required_result_columns, bool remove_dups) { ASTs & elements = select_query->select()->children; @@ -449,6 +451,8 @@ void removeUnneededColumnsFromSelectClause(const ASTSelectQuery * select_query, ASTs new_elements; new_elements.reserve(elements.size()); + NameSet remove_columns; + for (const auto & elem : elements) { String name = elem->getAliasOrColumnName(); @@ -465,6 +469,8 @@ void removeUnneededColumnsFromSelectClause(const ASTSelectQuery * select_query, } else { + remove_columns.insert(name); + ASTFunction * func = elem->as(); /// Never remove untuple. It's result column may be in required columns. @@ -478,6 +484,24 @@ void removeUnneededColumnsFromSelectClause(const ASTSelectQuery * select_query, } } + if (select_query->interpolate()) + { + auto & children = select_query->interpolate()->children; + if (!children.empty()) + { + for (auto it = children.begin(); it != children.end();) + { + if (remove_columns.count((*it)->as()->column)) + it = select_query->interpolate()->children.erase(it); + else + ++it; + } + + if (children.empty()) + select_query->setExpression(ASTSelectQuery::Expression::INTERPOLATE, nullptr); + } + } + elements = std::move(new_elements); } diff --git a/src/Parsers/ASTInterpolateElement.h b/src/Parsers/ASTInterpolateElement.h index ffb492787c9..70178de053f 100644 --- a/src/Parsers/ASTInterpolateElement.h +++ b/src/Parsers/ASTInterpolateElement.h @@ -12,12 +12,13 @@ public: String column; ASTPtr expr; - String getID(char) const override { return "InterpolateElement"; } + String getID(char delim) const override { return String("InterpolateElement") + delim + "(column " + column + ")"; } ASTPtr clone() const override { auto clone = std::make_shared(*this); clone->cloneChildren(); + clone->expr = clone->expr->clone(); return clone; } diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 1be1e925ca3..cd07e304a39 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -2339,6 +2339,7 @@ bool ParserInterpolateElement::parseImpl(Pos & pos, ASTPtr & node, Expected & ex auto elem = std::make_shared(); elem->column = ident->getColumnName(); elem->expr = expr; + elem->children.push_back(expr); node = elem; diff --git a/tests/queries/0_stateless/02233_interpolate_1.reference b/tests/queries/0_stateless/02233_interpolate_1.reference index c7774fe05ff..bb1a93131e5 100644 --- a/tests/queries/0_stateless/02233_interpolate_1.reference +++ b/tests/queries/0_stateless/02233_interpolate_1.reference @@ -94,3 +94,99 @@ 10.5 14 11 15 11.5 16 +0 1 +0.5 2 +1 original 2 +1.5 3 +2 4 +2.5 5 +3 6 +3.5 7 +4 original 5 +4.5 6 +5 7 +5.5 8 +6 9 +6.5 10 +7 original 8 +7.5 9 +8 10 +8.5 11 +9 12 +9.5 13 +10 14 +10.5 15 +11 16 +11.5 17 + 0 + 0 +original 1 + 3 + 3 + 3 + 3 + 3 +original 4 + 9 + 9 + 9 + 9 + 9 +original 7 + 15 + 15 + 15 + 15 + 15 + 15 + 15 + 15 + 15 +0 0 +0.5 0 +1 original 1 +1.5 3 +2 3 +2.5 3 +3 3 +3.5 3 +4 original 4 +4.5 9 +5 9 +5.5 9 +6 9 +6.5 9 +7 original 7 +7.5 15 +8 15 +8.5 15 +9 15 +9.5 15 +10 15 +10.5 15 +11 15 +11.5 15 +0 1 +0.5 2 +1 original 1 +1.5 2 +2 3 +2.5 4 +3 5 +3.5 6 +4 original 4 +4.5 5 +5 6 +5.5 7 +6 8 +6.5 9 +7 original 7 +7.5 8 +8 9 +8.5 10 +9 11 +9.5 12 +10 13 +10.5 14 +11 15 +11.5 16 diff --git a/tests/queries/0_stateless/02233_interpolate_1.sql b/tests/queries/0_stateless/02233_interpolate_1.sql index a355e903dae..52b597e06e0 100644 --- a/tests/queries/0_stateless/02233_interpolate_1.sql +++ b/tests/queries/0_stateless/02233_interpolate_1.sql @@ -37,3 +37,26 @@ SELECT n, source, inter FROM ( SELECT n, source, inter FROM ( SELECT toFloat32(number % 10) AS n, 'original' AS source, number as inter FROM numbers(10) WHERE number % 3 = 1 ) ORDER BY n WITH FILL FROM 0 TO 11.51 STEP 0.5 INTERPOLATE (inter AS source); -- { serverError 32 } + +# Test INTERPOLATE with aliased column +SELECT n, source, inter + 1 AS inter_p FROM ( + SELECT toFloat32(number % 10) AS n, 'original' AS source, number AS inter FROM numbers(10) WHERE (number % 3) = 1 +) ORDER BY n ASC WITH FILL FROM 0 TO 11.51 STEP 0.5 INTERPOLATE ( inter_p AS inter_p + 1 ); + +# Test INTERPOLATE with column not present in select +SELECT source, inter FROM ( + SELECT toFloat32(number % 10) AS n, 'original' AS source, number AS inter, number + 1 AS inter2 FROM numbers(10) WHERE (number % 3) = 1 +) ORDER BY n ASC WITH FILL FROM 0 TO 11.51 STEP 0.5 INTERPOLATE ( inter AS inter2 + inter ); + +# Test INTERPOLATE in sub-select +SELECT n, source, inter FROM ( + SELECT n, source, inter, inter2 FROM ( + SELECT toFloat32(number % 10) AS n, 'original' AS source, number AS inter, number + 1 AS inter2 FROM numbers(10) WHERE (number % 3) = 1 + ) ORDER BY n ASC WITH FILL FROM 0 TO 11.51 STEP 0.5 INTERPOLATE ( inter AS inter + inter2 ) +); + +# Test INTERPOLATE with aggregates +SELECT n, any(source), sum(inter) AS inter_s FROM ( + SELECT toFloat32(number % 10) AS n, 'original' AS source, number AS inter FROM numbers(10) WHERE (number % 3) = 1 +) GROUP BY n +ORDER BY n ASC WITH FILL FROM 0 TO 11.51 STEP 0.5 INTERPOLATE ( inter_s AS inter_s + 1 ); \ No newline at end of file From dbdc6322436b85bcbc0dd246a456454c95306ae4 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 7 Apr 2022 18:11:49 +0800 Subject: [PATCH 038/101] fixed code style --- tests/integration/test_storage_hdfs/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index b6dcff6d174..c1176e40e42 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -553,6 +553,7 @@ def test_insert_select_schema_inference(started_cluster): result = node1.query(f"select * from hdfs('hdfs://hdfs1:9000/test.native.zst')") assert int(result) == 1 + def test_cluster_join(started_cluster): result = node1.query( """ From 42d4a84a6f845e169194fcac64872541c27c22ab Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Thu, 7 Apr 2022 10:20:38 +0200 Subject: [PATCH 039/101] More tests for corner cases --- src/Functions/makeDate.cpp | 41 +++++++++++++++--- .../02245_make_datetime64.reference | 30 +++++++++++-- .../0_stateless/02245_make_datetime64.sql | 43 +++++++++++++++---- 3 files changed, 94 insertions(+), 20 deletions(-) diff --git a/src/Functions/makeDate.cpp b/src/Functions/makeDate.cpp index 675d22d7f00..dbf29322787 100644 --- a/src/Functions/makeDate.cpp +++ b/src/Functions/makeDate.cpp @@ -196,16 +196,28 @@ protected: { /// Note that hour, minute and second are checked against 99 to behave consistently with parsing DateTime from String /// E.g. "select cast('1984-01-01 99:99:99' as DateTime);" returns "1984-01-05 04:40:39" - if (unlikely(year < DATE_LUT_MIN_YEAR || month < 1 || month > 12 || day_of_month < 1 || day_of_month > 31 || + if (unlikely(std::isnan(year) || std::isnan(month) || std::isnan(day_of_month) || + std::isnan(hour) || std::isnan(minute) || std::isnan(second) || + year < DATE_LUT_MIN_YEAR || month < 1 || month > 12 || day_of_month < 1 || day_of_month > 31 || hour < 0 || hour > 99 || minute < 0 || minute > 99 || second < 0 || second > 99)) - return lut.makeDateTime(DATE_LUT_MIN_YEAR-1, 1, 1, 0, 0, 0); + return minDateTime(lut); if (unlikely(year > DATE_LUT_MAX_YEAR)) - return lut.makeDateTime(DATE_LUT_MAX_YEAR+1, 1, 1, 23, 59, 59); + return maxDateTime(lut); return lut.makeDateTime(year, month, day_of_month, hour, minute, second); } + static Int64 minDateTime(const DateLUTImpl & lut) + { + return lut.makeDateTime(DATE_LUT_MIN_YEAR - 1, 1, 1, 0, 0, 0); + } + + static Int64 maxDateTime(const DateLUTImpl & lut) + { + return lut.makeDateTime(DATE_LUT_MAX_YEAR + 1, 1, 1, 23, 59, 59); + } + std::string extractTimezone(const ColumnWithTypeAndName & timezone_argument) const { std::string timezone; @@ -364,6 +376,8 @@ public: const auto & date_lut = DateLUT::instance(timezone); const auto max_fraction = pow(10, precision) - 1; + const auto min_date_time = minDateTime(date_lut); + const auto max_date_time = maxDateTime(date_lut); for (size_t i = 0; i < input_rows_count; ++i) { @@ -376,11 +390,24 @@ public: auto date_time = dateTime(year, month, day, hour, minute, second, date_lut); - auto fraction = fraction_data ? (*fraction_data)[i] : 0; - if (unlikely(fraction < 0)) + double fraction = 0; + if (unlikely(date_time == min_date_time)) fraction = 0; - else if (unlikely(fraction > max_fraction)) - fraction = max_fraction; + else if (unlikely(date_time == max_date_time)) + fraction = 999999999ll; + else + { + fraction = fraction_data ? (*fraction_data)[i] : 0; + if (unlikely(std::isnan(fraction))) + { + date_time = min_date_time; + fraction = 0; + } + else if (unlikely(fraction < 0)) + fraction = 0; + else if (unlikely(fraction > max_fraction)) + fraction = max_fraction; + } result_data[i] = DecimalUtils::decimalFromComponents(date_time, fraction, precision); } diff --git a/tests/queries/0_stateless/02245_make_datetime64.reference b/tests/queries/0_stateless/02245_make_datetime64.reference index bcdf7dacccc..eea768f1dc7 100644 --- a/tests/queries/0_stateless/02245_make_datetime64.reference +++ b/tests/queries/0_stateless/02245_make_datetime64.reference @@ -9,7 +9,7 @@ DateTime64(6) DateTime64(7, \'CET\') DateTime64(7, \'UTC\') 1925-01-01 00:00:00.000000000 -1925-01-01 00:00:00.999999999 +1925-01-01 00:00:00.000000000 2283-11-11 23:59:59.99999999 2262-04-11 23:47:16.854775807 2262-04-11 23:47:16.85477581 @@ -30,8 +30,23 @@ DateTime64(7, \'UTC\') 1984-01-02 01:00:00.000000000 1984-01-01 01:10:00.000000000 1984-01-01 00:01:10.000000000 -1984-01-01 00:00:00.000000000 -1983-03-01 00:00:00.000000000 +1984-01-01 02:03:04.000000005 +1984-02-29 02:03:04.000000005 +1983-03-01 02:03:04.000000005 +1984-03-01 02:03:04.000000005 +1983-03-02 02:03:04.000000005 +1984-03-02 02:03:04.000000005 +1983-03-03 02:03:04.000000005 +1925-01-01 00:00:00.000000000 +1925-01-01 00:00:00.000000000 +1925-01-01 00:00:00.000000000 +1925-01-01 00:00:00.000000000 +1925-01-01 00:00:00.000000000 +1925-01-01 00:00:00.000000000 +1925-01-01 00:00:00.000000000 +1925-01-01 00:00:00.000000000 +1984-01-01 02:03:04.000000000 +1925-01-01 00:00:00.000000000 1925-01-01 00:00:00.000000000 1925-01-01 00:00:00.000000000 1925-01-01 00:00:00.000000000 @@ -39,7 +54,14 @@ DateTime64(7, \'UTC\') 1925-01-01 00:00:00.000000000 1925-01-01 00:00:00.000000000 1984-01-01 00:00:00.000000000 -2283-11-11 23:59:59.000 +1984-01-01 00:00:00.000000000 +1984-01-01 00:00:00.000000000 +1984-01-01 00:00:00.000000000 +1984-01-01 00:00:00.000000000 +1984-01-01 00:00:00.000000000 +1984-01-01 00:00:00.000000000 +1984-01-01 00:00:00.000000000 +2283-11-11 23:59:59.999 1925-01-01 00:00:00.000 1925-01-01 00:00:00.000 1925-01-01 00:00:00.000 diff --git a/tests/queries/0_stateless/02245_make_datetime64.sql b/tests/queries/0_stateless/02245_make_datetime64.sql index 7bd3c1df2da..2e2d81fa363 100644 --- a/tests/queries/0_stateless/02245_make_datetime64.sql +++ b/tests/queries/0_stateless/02245_make_datetime64.sql @@ -40,15 +40,40 @@ select makeDateTime64(1984, 1, 1, 0, 70, 0, 0, 9, 'UTC'); select makeDateTime64(1984, 1, 1, 0, 0, 70, 0, 9, 'UTC'); select makeDateTime64(1984, 1, 1, 0, 0, 0, 0, 9, 'not a timezone'); -- { serverError 1000 } -select makeDateTime64(1984, 1, 1, 0, 0, 0, 0, 9, 'UTC'); -select makeDateTime64(1983, 2, 29, 0, 0, 0, 0, 9, 'UTC'); -select makeDateTime64(-1984, 1, 1, 0, 0, 0, 0, 9, 'UTC'); -select makeDateTime64(1984, -1, 1, 0, 0, 0, 0, 9, 'UTC'); -select makeDateTime64(1984, 1, -1, 0, 0, 0, 0, 9, 'UTC'); -select makeDateTime64(1984, 1, 1, -1, 0, 0, 0, 9, 'UTC'); -select makeDateTime64(1984, 1, 1, 0, -1, 0, 0, 9, 'UTC'); -select makeDateTime64(1984, 1, 1, 0, 0, -1, 0, 9, 'UTC'); -select makeDateTime64(1984, 1, 1, 0, 0, 0, -1, 9, 'UTC'); +select makeDateTime64(1984, 1, 1, 2, 3, 4, 5, 9, 'UTC'); +select makeDateTime64(1984, 2, 29, 2, 3, 4, 5, 9, 'UTC'); +select makeDateTime64(1983, 2, 29, 2, 3, 4, 5, 9, 'UTC'); +select makeDateTime64(1984, 2, 30, 2, 3, 4, 5, 9, 'UTC'); +select makeDateTime64(1983, 2, 30, 2, 3, 4, 5, 9, 'UTC'); +select makeDateTime64(1984, 2, 31, 2, 3, 4, 5, 9, 'UTC'); +select makeDateTime64(1983, 2, 31, 2, 3, 4, 5, 9, 'UTC'); +select makeDateTime64(1984, 2, 32, 2, 3, 4, 5, 9, 'UTC'); +select makeDateTime64(1983, 2, 32, 2, 3, 4, 5, 9, 'UTC'); + +select makeDateTime64(-1984, 1, 1, 2, 3, 4, 5, 9, 'UTC'); +select makeDateTime64(1984, -1, 1, 2, 3, 4, 5, 9, 'UTC'); +select makeDateTime64(1984, 1, -1, 2, 3, 4, 5, 9, 'UTC'); +select makeDateTime64(1984, 1, 1, -1, 3, 4, 5, 9, 'UTC'); +select makeDateTime64(1984, 1, 1, 2, -1, 4, 5, 9, 'UTC'); +select makeDateTime64(1984, 1, 1, 2, 3, -1, 5, 9, 'UTC'); +select makeDateTime64(1984, 1, 1, 2, 3, 4, -1, 9, 'UTC'); + +select makeDateTime64(NaN, 1, 1, 2, 3, 4, 5, 9, 'UTC'); +select makeDateTime64(1984, NaN, 1, 2, 3, 4, 5, 9, 'UTC'); +select makeDateTime64(1984, 1, NaN, 2, 3, 4, 5, 9, 'UTC'); +select makeDateTime64(1984, 1, 1, NaN, 3, 4, 5, 9, 'UTC'); +select makeDateTime64(1984, 1, 1, 2, NaN, 4, 5, 9, 'UTC'); +select makeDateTime64(1984, 1, 1, 2, 3, NaN, 5, 9, 'UTC'); +select makeDateTime64(1984, 1, 1, 2, 3, 4, NaN, 9, 'UTC'); + +select makeDateTime64(1984.5, 1, 1, 0, 0, 0, 0, 9, 'UTC'); +select makeDateTime64(1984, 1.5, 1, 0, 0, 0, 0, 9, 'UTC'); +select makeDateTime64(1984, 1, 1.5, 0, 0, 0, 0, 9, 'UTC'); +select makeDateTime64(1984, 1, 1, 0.5, 0, 0, 0, 9, 'UTC'); +select makeDateTime64(1984, 1, 1, 0, 0.5, 0, 0, 9, 'UTC'); +select makeDateTime64(1984, 1, 1, 0, 0, 0.5, 0, 9, 'UTC'); +select makeDateTime64(1984, 1, 1, 0, 0, 0, 0.5, 9, 'UTC'); +select makeDateTime64(1984, 1, 1, 0, 0, 0, 0, 9.5, 'UTC'); select makeDateTime64(65537, 8, 24, 21, 4, 0); select makeDateTime64(1991, 65537, 24, 21, 4, 0); From a998d73ee22dcf02d0388d3adaf13c3af6742c9a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 7 Apr 2022 12:37:11 +0200 Subject: [PATCH 040/101] Update StorageKafka.cpp --- src/Storages/Kafka/StorageKafka.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 79c967ce9c0..127911c9f60 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -661,7 +661,7 @@ bool StorageKafka::streamToViews() // We can't cancel during copyData, as it's not aware of commits and other kafka-related stuff. // It will be cancelled on underlying layer (kafka buffer) - size_t rows = 0; + std::atomic_size_t rows = 0; { block_io.pipeline.complete(std::move(pipe)); From ebfdadd6f8491fba128af05194213d951935df88 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 7 Apr 2022 12:05:21 +0000 Subject: [PATCH 041/101] Fix firmatting. --- tests/integration/test_storage_kafka/test.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index a74a045b9a3..a27b5a134e4 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -1146,6 +1146,7 @@ def test_kafka_read_consumers_in_parallel(kafka_cluster): kafka_create_topic(admin_client, topic_name, num_partitions=8) cancel = threading.Event() + def produce(): while not cancel.is_set(): messages = [] @@ -1188,14 +1189,14 @@ def test_kafka_read_consumers_in_parallel(kafka_cluster): "kafka.*Polled batch of [0-9]+.*read_consumers_in_parallel", repetitions=64, look_behind_lines=100, - timeout=30 # we should get 64 polls in ~8 seconds, but when read sequentially it will take more than 64 sec + timeout=30, # we should get 64 polls in ~8 seconds, but when read sequentially it will take more than 64 sec ) cancel.set() kafka_thread.join() instance.query( - """ + """ DROP TABLE test.consumer; DROP TABLE test.view; DROP TABLE test.kafka; From e9de38c52bc876b43ab44458f5cd1cede1a7ceba Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 7 Apr 2022 20:45:07 +0800 Subject: [PATCH 042/101] fix bug --- src/Storages/Hive/StorageHive.cpp | 18 ++++++++++++++---- 1 file changed, 14 insertions(+), 4 deletions(-) diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 659bf44e3f4..90e5dd59722 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -308,6 +308,8 @@ StorageHive::StorageHive( storage_metadata.setColumns(columns_); storage_metadata.setConstraints(constraints_); storage_metadata.setComment(comment_); + storage_metadata.partition_key = KeyDescription::getKeyFromAST(partition_by_ast, storage_metadata.columns, getContext()); + setInMemoryMetadata(storage_metadata); } @@ -517,7 +519,15 @@ HiveFiles StorageHive::collectHiveFilesFromPartition( { auto hive_file = createHiveFileIfNeeded(file_info, fields, query_info, context_, prune_level); if (hive_file) + { + LOG_TRACE( + log, + "Append hive file {} from partition {}, prune_level:{}", + hive_file->getPath(), + boost::join(partition.values, ","), + pruneLevelToString(prune_level)); hive_files.push_back(hive_file); + } } return hive_files; } @@ -535,11 +545,11 @@ HiveFilePtr StorageHive::createHiveFileIfNeeded( ContextPtr context_, PruneLevel prune_level) const { - LOG_TRACE(log, "create hive file {} if needed, prune_level:{}", file_info.path, pruneLevelToString(prune_level)); + LOG_TRACE(log, "Create hive file {} if needed, prune_level:{}", file_info.path, pruneLevelToString(prune_level)); String filename = getBaseName(file_info.path); /// Skip temporary files starts with '.' - if (startsWith(filename, ".") == 0) + if (startsWith(filename, ".")) return {}; auto hive_file = createHiveFile( @@ -553,12 +563,12 @@ HiveFilePtr StorageHive::createHiveFileIfNeeded( storage_settings, context_); - /// Load file level minmax index and apply if (prune_level >= PruneLevel::File) { const KeyCondition hivefile_key_condition(query_info, getContext(), hivefile_name_types.getNames(), hivefile_minmax_idx_expr); if (hive_file->useFileMinMaxIndex()) { + /// Load file level minmax index and apply hive_file->loadFileMinMaxIndex(); if (!hivefile_key_condition.checkInHyperrectangle(hive_file->getMinMaxIndex()->hyperrectangle, hivefile_name_types.getTypes()) .can_be_true) @@ -574,9 +584,9 @@ HiveFilePtr StorageHive::createHiveFileIfNeeded( if (prune_level >= PruneLevel::Split) { - /// Load sub-file level minmax index and apply if (hive_file->useSplitMinMaxIndex()) { + /// Load sub-file level minmax index and apply std::unordered_set skip_splits; hive_file->loadSplitMinMaxIndex(); const auto & sub_minmax_idxes = hive_file->getSubMinMaxIndexes(); From 87507ec9e8e428d00395ecc6b1e27d3d4119cbdc Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 7 Apr 2022 20:52:54 +0800 Subject: [PATCH 043/101] fix conflicts --- src/Storages/Hive/HiveFile.cpp | 4 ---- 1 file changed, 4 deletions(-) diff --git a/src/Storages/Hive/HiveFile.cpp b/src/Storages/Hive/HiveFile.cpp index 1d965350e16..7b85a2f4f91 100644 --- a/src/Storages/Hive/HiveFile.cpp +++ b/src/Storages/Hive/HiveFile.cpp @@ -190,10 +190,6 @@ std::unique_ptr HiveORCFile::buildMinMaxIndex(c return idx; } -<<<<<<< HEAD -======= - ->>>>>>> master void HiveORCFile::loadFileMinMaxIndex() { if (!reader) From 64dcddc6e3c58d6aa2bca27d3ce4d699a15b1414 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 7 Apr 2022 17:41:05 -0400 Subject: [PATCH 044/101] fixed ASTInterpolateElement::clone, fixed QueryNormalizer to exclude ASTInterpolateElement::children --- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- src/Interpreters/QueryNormalizer.cpp | 3 ++- src/Interpreters/RequiredSourceColumnsVisitor.cpp | 2 +- src/Parsers/ASTInterpolateElement.h | 3 ++- 4 files changed, 6 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 2c3c2dcf23d..d4f0f3994a7 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -888,7 +888,7 @@ static InterpolateDescriptionPtr getInterpolateDescription( col_set.insert(column.name); } for (const auto & column : result_block) - if( col_set.count(column.name) == 0) + if (col_set.count(column.name) == 0) source_columns.emplace_back(column.name, column.type); } diff --git a/src/Interpreters/QueryNormalizer.cpp b/src/Interpreters/QueryNormalizer.cpp index 7c820622c37..3c0a965dfa4 100644 --- a/src/Interpreters/QueryNormalizer.cpp +++ b/src/Interpreters/QueryNormalizer.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -134,7 +135,7 @@ void QueryNormalizer::visit(ASTTablesInSelectQueryElement & node, const ASTPtr & static bool needVisitChild(const ASTPtr & child) { - return !(child->as() || child->as()); + return !(child->as() || child->as() || child->as()); } /// special visitChildren() for ASTSelectQuery diff --git a/src/Interpreters/RequiredSourceColumnsVisitor.cpp b/src/Interpreters/RequiredSourceColumnsVisitor.cpp index c46d0ac7bf0..cc9a2d86d28 100644 --- a/src/Interpreters/RequiredSourceColumnsVisitor.cpp +++ b/src/Interpreters/RequiredSourceColumnsVisitor.cpp @@ -47,7 +47,7 @@ bool RequiredSourceColumnsMatcher::needChildVisit(const ASTPtr & node, const AST return false; /// Processed. Do not need children. - if (node->as() || node->as() || node->as()) + if (node->as() || node->as() || node->as() || node->as()) return false; if (const auto * f = node->as()) diff --git a/src/Parsers/ASTInterpolateElement.h b/src/Parsers/ASTInterpolateElement.h index 70178de053f..a278755de88 100644 --- a/src/Parsers/ASTInterpolateElement.h +++ b/src/Parsers/ASTInterpolateElement.h @@ -17,8 +17,9 @@ public: ASTPtr clone() const override { auto clone = std::make_shared(*this); - clone->cloneChildren(); clone->expr = clone->expr->clone(); + clone->children.clear(); + clone->children.push_back(clone->expr); return clone; } From 2e6f0db8255abaaa2da459d7dd2c2b4b7f9dd7a9 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 8 Apr 2022 15:12:24 +0800 Subject: [PATCH 045/101] first commit --- src/Storages/Hive/HiveCommon.cpp | 6 ++--- src/Storages/Hive/HiveCommon.h | 18 ++++++--------- src/Storages/Hive/HiveFile.cpp | 22 +++++++++++++++--- src/Storages/Hive/HiveFile.h | 38 ++++++++++++++++++------------- src/Storages/Hive/StorageHive.cpp | 17 ++++++++++---- src/Storages/Hive/StorageHive.h | 4 +++- 6 files changed, 66 insertions(+), 39 deletions(-) diff --git a/src/Storages/Hive/HiveCommon.cpp b/src/Storages/Hive/HiveCommon.cpp index 60f471c1568..510ac2a98d6 100644 --- a/src/Storages/Hive/HiveCommon.cpp +++ b/src/Storages/Hive/HiveCommon.cpp @@ -128,7 +128,7 @@ HiveMetastoreClient::HiveTableMetadataPtr HiveMetastoreClient::getTableMetadata( } metadata = std::make_shared( - db_name, table_name, table, std::move(new_partition_infos), getContext()); + db_name, table_name, table, std::move(new_partition_infos)); table_metadata_cache.set(cache_key, metadata); } return metadata; @@ -232,7 +232,7 @@ using namespace apache::thrift::protocol; using namespace apache::thrift::transport; using namespace Apache::Hadoop::Hive; -HiveMetastoreClientPtr HiveMetastoreClientFactory::getOrCreate(const String & name, ContextPtr context) +HiveMetastoreClientPtr HiveMetastoreClientFactory::getOrCreate(const String & name) { std::lock_guard lock(mutex); @@ -243,7 +243,7 @@ HiveMetastoreClientPtr HiveMetastoreClientFactory::getOrCreate(const String & na { return createThriftHiveMetastoreClient(name); }; - auto client = std::make_shared(builder, context->getGlobalContext()); + auto client = std::make_shared(builder); clients[name] = client; return client; } diff --git a/src/Storages/Hive/HiveCommon.h b/src/Storages/Hive/HiveCommon.h index b8075457a02..bce7d291351 100644 --- a/src/Storages/Hive/HiveCommon.h +++ b/src/Storages/Hive/HiveCommon.h @@ -37,7 +37,7 @@ protected: private: ThriftHiveMetastoreClientBuilder builder; }; -class HiveMetastoreClient : public WithContext +class HiveMetastoreClient { public: @@ -68,17 +68,15 @@ public: /// Used for speeding up metadata query process. - struct HiveTableMetadata : public WithContext + struct HiveTableMetadata { public: HiveTableMetadata( const String & db_name_, const String & table_name_, std::shared_ptr table_, - const std::map & partition_infos_, - ContextPtr context_) - : WithContext(context_) - , db_name(db_name_) + const std::map & partition_infos_) + : db_name(db_name_) , table_name(table_name_) , table(table_) , partition_infos(partition_infos_) @@ -117,14 +115,12 @@ public: using HiveTableMetadataPtr = std::shared_ptr; - explicit HiveMetastoreClient(ThriftHiveMetastoreClientBuilder builder_, ContextPtr context_) - : WithContext(context_) - , table_metadata_cache(1000) + explicit HiveMetastoreClient(ThriftHiveMetastoreClientBuilder builder_) + : table_metadata_cache(1000) , client_pool(builder_) { } - HiveTableMetadataPtr getTableMetadata(const String & db_name, const String & table_name); // Access hive table information by hive client std::shared_ptr getHiveTable(const String & db_name, const String & table_name); @@ -150,7 +146,7 @@ class HiveMetastoreClientFactory final : private boost::noncopyable public: static HiveMetastoreClientFactory & instance(); - HiveMetastoreClientPtr getOrCreate(const String & name, ContextPtr context); + HiveMetastoreClientPtr getOrCreate(const String & name); static std::shared_ptr createThriftHiveMetastoreClient(const String & name); diff --git a/src/Storages/Hive/HiveFile.cpp b/src/Storages/Hive/HiveFile.cpp index 7b85a2f4f91..02c92770274 100644 --- a/src/Storages/Hive/HiveFile.cpp +++ b/src/Storages/Hive/HiveFile.cpp @@ -84,6 +84,22 @@ std::optional IHiveFile::getRows() return rows; } +void IHiveFile::loadFileMinMaxIndex() +{ + if (file_minmax_idx_loaded) + return; + loadFileMinMaxIndexImpl(); + file_minmax_idx_loaded = true; +} + +void IHiveFile::loadSplitMinMaxIndexes() +{ + if (split_minmax_idxes_loaded) + return; + loadSplitMinMaxIndexesImpl(); + split_minmax_idxes_loaded = true; +} + Range HiveORCFile::buildRange(const orc::ColumnStatistics * col_stats) { if (!col_stats || col_stats->hasNull()) @@ -190,7 +206,7 @@ std::unique_ptr HiveORCFile::buildMinMaxIndex(c return idx; } -void HiveORCFile::loadFileMinMaxIndex() +void HiveORCFile::loadFileMinMaxIndexImpl() { if (!reader) { @@ -208,7 +224,7 @@ bool HiveORCFile::useSplitMinMaxIndex() const } -void HiveORCFile::loadSplitMinMaxIndex() +void HiveORCFile::loadSplitMinMaxIndexesImpl() { if (!reader) { @@ -257,7 +273,7 @@ void HiveParquetFile::prepareReader() THROW_ARROW_NOT_OK(parquet::arrow::OpenFile(asArrowFile(*in, format_settings, is_stopped), arrow::default_memory_pool(), &reader)); } -void HiveParquetFile::loadSplitMinMaxIndex() +void HiveParquetFile::loadSplitMinMaxIndexesImpl() { if (!reader) prepareReader(); diff --git a/src/Storages/Hive/HiveFile.h b/src/Storages/Hive/HiveFile.h index f7f50063268..0faf2a190b7 100644 --- a/src/Storages/Hive/HiveFile.h +++ b/src/Storages/Hive/HiveFile.h @@ -123,22 +123,25 @@ public: virtual FileFormat getFormat() const = 0; + /// If hive query could use file level minmax index? virtual bool useFileMinMaxIndex() const { return false; } + void loadFileMinMaxIndex(); - virtual void loadFileMinMaxIndex() - { - throw Exception("Method loadFileMinMaxIndex is not supported by hive file:" + getFormatName(), ErrorCodes::NOT_IMPLEMENTED); - } - - /// If hive query could use contains sub-file level minmax index? + /// If hive query could use sub-file level minmax index? virtual bool useSplitMinMaxIndex() const { return false; } - - virtual void loadSplitMinMaxIndex() - { - throw Exception("Method loadSplitMinMaxIndex is not supported by hive file:" + getFormatName(), ErrorCodes::NOT_IMPLEMENTED); - } + void loadSplitMinMaxIndexes(); protected: + virtual void loadFileMinMaxIndexImpl() + { + throw Exception("Method loadFileMinMaxIndexImpl is not supported by hive file:" + getFormatName(), ErrorCodes::NOT_IMPLEMENTED); + } + + virtual void loadSplitMinMaxIndexesImpl() + { + throw Exception("Method loadSplitMinMaxIndexesImpl is not supported by hive file:" + getFormatName(), ErrorCodes::NOT_IMPLEMENTED); + } + virtual std::optional getRowsImpl() = 0; FieldVector partition_values; @@ -149,8 +152,13 @@ protected: std::optional rows; NamesAndTypesList index_names_and_types; + MinMaxIndexPtr file_minmax_idx; + std::atomic file_minmax_idx_loaded{false}; + std::vector split_minmax_idxes; + std::atomic split_minmax_idxes_loaded{false}; + /// Skip splits for this file after applying minmax index (if any) std::unordered_set skip_splits; std::shared_ptr storage_settings; @@ -199,14 +207,13 @@ public: FileFormat getFormat() const override { return FileFormat::ORC; } bool useFileMinMaxIndex() const override; - void loadFileMinMaxIndex() override; - bool useSplitMinMaxIndex() const override; - void loadSplitMinMaxIndex() override; private: static Range buildRange(const orc::ColumnStatistics * col_stats); + void loadFileMinMaxIndexImpl() override; + void loadSplitMinMaxIndexesImpl() override; std::unique_ptr buildMinMaxIndex(const orc::Statistics * statistics); void prepareReader(); void prepareColumnMapping(); @@ -235,11 +242,10 @@ public: } FileFormat getFormat() const override { return FileFormat::PARQUET; } - bool useSplitMinMaxIndex() const override; - void loadSplitMinMaxIndex() override; private: + void loadSplitMinMaxIndexesImpl() override; std::optional getRowsImpl() override; void prepareReader(); diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 90e5dd59722..a91ecf6e280 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -319,7 +319,7 @@ void StorageHive::lazyInitialize() if (has_initialized) return; - auto hive_metastore_client = HiveMetastoreClientFactory::instance().getOrCreate(hive_metastore_url, getContext()); + auto hive_metastore_client = HiveMetastoreClientFactory::instance().getOrCreate(hive_metastore_url); auto hive_table_metadata = hive_metastore_client->getHiveTable(hive_database, hive_table); hdfs_namenode_url = getNameNodeUrl(hive_table_metadata->sd.location); @@ -415,7 +415,7 @@ ASTPtr StorageHive::extractKeyExpressionList(const ASTPtr & node) } -HiveFilePtr createHiveFile( +static HiveFilePtr createHiveFile( const String & format_name, const FieldVector & fields, const String & namenode_url, @@ -562,6 +562,13 @@ HiveFilePtr StorageHive::createHiveFileIfNeeded( hivefile_name_types, storage_settings, context_); + /* + { + std::lock_guard lock{init_mutex}; + hive_files_by_path[file_info.path] = hive_file; + std::cout << "size:" << hive_files_by_path.size() << std::endl; + } + */ if (prune_level >= PruneLevel::File) { @@ -588,7 +595,7 @@ HiveFilePtr StorageHive::createHiveFileIfNeeded( { /// Load sub-file level minmax index and apply std::unordered_set skip_splits; - hive_file->loadSplitMinMaxIndex(); + hive_file->loadSplitMinMaxIndexes(); const auto & sub_minmax_idxes = hive_file->getSubMinMaxIndexes(); for (size_t i = 0; i < sub_minmax_idxes.size(); ++i) { @@ -654,7 +661,7 @@ Pipe StorageHive::read( HDFSBuilderWrapper builder = createHDFSBuilder(hdfs_namenode_url, context_->getGlobalContext()->getConfigRef()); HDFSFSPtr fs = createHDFSFS(builder.get()); - auto hive_metastore_client = HiveMetastoreClientFactory::instance().getOrCreate(hive_metastore_url, getContext()); + auto hive_metastore_client = HiveMetastoreClientFactory::instance().getOrCreate(hive_metastore_url); auto hive_table_metadata = hive_metastore_client->getTableMetadata(hive_database, hive_table); /// Collect Hive files to read @@ -788,7 +795,7 @@ StorageHive::totalRowsImpl(const Settings & settings, const SelectQueryInfo & qu if (!isColumnOriented()) return {}; - auto hive_metastore_client = HiveMetastoreClientFactory::instance().getOrCreate(hive_metastore_url, getContext()); + auto hive_metastore_client = HiveMetastoreClientFactory::instance().getOrCreate(hive_metastore_url); auto hive_table_metadata = hive_metastore_client->getTableMetadata(hive_database, hive_table); HDFSBuilderWrapper builder = createHDFSBuilder(hdfs_namenode_url, getContext()->getGlobalContext()->getConfigRef()); HDFSFSPtr fs = createHDFSFS(builder.get()); diff --git a/src/Storages/Hive/StorageHive.h b/src/Storages/Hive/StorageHive.h index 0f381803156..b9841d28252 100644 --- a/src/Storages/Hive/StorageHive.h +++ b/src/Storages/Hive/StorageHive.h @@ -131,7 +131,7 @@ private: String hive_database; String hive_table; - std::mutex init_mutex; + mutable std::mutex init_mutex; bool has_initialized = false; /// Hive table meta @@ -155,6 +155,8 @@ private: std::shared_ptr storage_settings; + // mutable std::map hive_files_by_path; + Poco::Logger * log = &Poco::Logger::get("StorageHive"); }; From 213b7bb222322bd1d7c207868ce2a2827c3cf5f6 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 8 Apr 2022 08:48:20 +0300 Subject: [PATCH 046/101] clickhouse-client: fix query cancellation if any result was not received yet This should fix issues with queries left after tests like in: - https://s3.amazonaws.com/clickhouse-test-reports/35865/10b9f38d8215cb57783125efe51a8c7aa48590a5/stateless_tests__debug__actions__[2/3].html - https://s3.amazonaws.com/clickhouse-test-reports/35865/10b9f38d8215cb57783125efe51a8c7aa48590a5/stateless_tests__debug__actions__[3/3].html Signed-off-by: Azat Khuzhin --- src/Client/ClientBase.cpp | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 5409971c5de..93ed1780e3b 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -225,17 +225,16 @@ std::atomic_flag exit_on_signal; class QueryInterruptHandler : private boost::noncopyable { public: - QueryInterruptHandler() { exit_on_signal.clear(); } - - ~QueryInterruptHandler() { exit_on_signal.test_and_set(); } - + static void start() { exit_on_signal.clear(); } + /// Return true if the query was stopped. + static bool stop() { return exit_on_signal.test_and_set(); } static bool cancelled() { return exit_on_signal.test(); } }; /// This signal handler is set only for SIGINT. void interruptSignalHandler(int signum) { - if (exit_on_signal.test_and_set()) + if (QueryInterruptHandler::stop()) safeExit(128 + signum); } @@ -254,7 +253,7 @@ ClientBase::ClientBase() = default; void ClientBase::setupSignalHandler() { - exit_on_signal.test_and_set(); + QueryInterruptHandler::stop(); struct sigaction new_act; memset(&new_act, 0, sizeof(new_act)); @@ -685,6 +684,9 @@ void ClientBase::processOrdinaryQuery(const String & query_to_execute, ASTPtr pa { try { + QueryInterruptHandler::start(); + SCOPE_EXIT({ QueryInterruptHandler::stop(); }); + connection->sendQuery( connection_parameters.timeouts, query, @@ -724,8 +726,6 @@ void ClientBase::processOrdinaryQuery(const String & query_to_execute, ASTPtr pa /// Also checks if query execution should be cancelled. void ClientBase::receiveResult(ASTPtr parsed_query) { - QueryInterruptHandler query_interrupt_handler; - // TODO: get the poll_interval from commandline. const auto receive_timeout = connection_parameters.timeouts.receive_timeout; constexpr size_t default_poll_interval = 1000000; /// in microseconds @@ -760,7 +760,7 @@ void ClientBase::receiveResult(ASTPtr parsed_query) }; /// handler received sigint - if (query_interrupt_handler.cancelled()) + if (QueryInterruptHandler::cancelled()) { cancel_query(); } From eaad99bd48f483ce45c77eb91ed5bc3f97786b40 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 8 Apr 2022 13:55:10 +0200 Subject: [PATCH 047/101] Try to turn off always green backward compatibility checks --- docker/test/stress/run.sh | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 27d5f9c4be4..0f5139f5b4d 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -329,8 +329,8 @@ then -e "Code: 1000, e.code() = 111, Connection refused" \ -e "UNFINISHED" \ -e "Renaming unexpected part" \ - /var/log/clickhouse-server/clickhouse-server.backward.*.log | zgrep -Fa "" > /test_output/bc_check_error_messages.txt \ - && echo -e 'Backward compatibility check: Error message in clickhouse-server.log (see bc_check_error_messages.txt)\tOK' >> /test_output/test_results.tsv \ + /var/log/clickhouse-server/clickhouse-server.backward.clean.log | zgrep -Fa "" > /test_output/bc_check_error_messages.txt \ + && echo -e 'Backward compatibility check: Error message in clickhouse-server.log (see bc_check_error_messages.txt)\tFAIL' >> /test_output/test_results.tsv \ || echo -e 'Backward compatibility check: No Error messages in clickhouse-server.log\tOK' >> /test_output/test_results.tsv # Remove file bc_check_error_messages.txt if it's empty @@ -346,7 +346,7 @@ then # OOM zgrep -Fa " Application: Child process was terminated by signal 9" /var/log/clickhouse-server/clickhouse-server.backward.*.log > /dev/null \ - && echo -e 'Backward compatibility check: OOM killer (or signal 9) in clickhouse-server.log\tOK' >> /test_output/test_results.tsv \ + && echo -e 'Backward compatibility check: OOM killer (or signal 9) in clickhouse-server.log\tFAIL' >> /test_output/test_results.tsv \ || echo -e 'Backward compatibility check: No OOM messages in clickhouse-server.log\tOK' >> /test_output/test_results.tsv # Logical errors @@ -366,7 +366,7 @@ then # It also checks for crash without stacktrace (printed by watchdog) echo "Check for Fatal message in server log:" zgrep -Fa " " /var/log/clickhouse-server/clickhouse-server.backward.*.log > /test_output/bc_check_fatal_messages.txt \ - && echo -e 'Backward compatibility check: Fatal message in clickhouse-server.log (see bc_check_fatal_messages.txt)\tOK' >> /test_output/test_results.tsv \ + && echo -e 'Backward compatibility check: Fatal message in clickhouse-server.log (see bc_check_fatal_messages.txt)\tFAIL' >> /test_output/test_results.tsv \ || echo -e 'Backward compatibility check: No fatal messages in clickhouse-server.log\tOK' >> /test_output/test_results.tsv # Remove file bc_check_fatal_messages.txt if it's empty From 2deec53162cc3c090335dd9714746743404b3374 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 8 Apr 2022 13:39:54 +0000 Subject: [PATCH 048/101] Disable LIMIT push down through WINDOW functions. --- .../QueryPlan/Optimizations/limitPushDown.cpp | 6 ++++++ ...push_down_over_window_functions_bug.reference | 10 ++++++++++ ...limit_push_down_over_window_functions_bug.sql | 16 ++++++++++++++++ 3 files changed, 32 insertions(+) create mode 100644 tests/queries/0_stateless/02265_limit_push_down_over_window_functions_bug.reference create mode 100644 tests/queries/0_stateless/02265_limit_push_down_over_window_functions_bug.sql diff --git a/src/Processors/QueryPlan/Optimizations/limitPushDown.cpp b/src/Processors/QueryPlan/Optimizations/limitPushDown.cpp index eb65f49103b..cb731376473 100644 --- a/src/Processors/QueryPlan/Optimizations/limitPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/limitPushDown.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include namespace DB::QueryPlanOptimizations @@ -66,6 +67,11 @@ size_t tryPushDownLimit(QueryPlan::Node * parent_node, QueryPlan::Nodes &) if (typeid_cast(child.get())) return 0; + /// Disable for WindowStep. + /// TODO: we can push down limit in some cases if increase the limit value. + if (typeid_cast(child.get())) + return 0; + /// Now we should decide if pushing down limit possible for this step. const auto & transform_traits = transforming->getTransformTraits(); diff --git a/tests/queries/0_stateless/02265_limit_push_down_over_window_functions_bug.reference b/tests/queries/0_stateless/02265_limit_push_down_over_window_functions_bug.reference new file mode 100644 index 00000000000..592637a91fc --- /dev/null +++ b/tests/queries/0_stateless/02265_limit_push_down_over_window_functions_bug.reference @@ -0,0 +1,10 @@ +0 1 +1 2 +2 3 +a 2 +0 10000000 +1 10000000 +2 10000000 +0 10000000 +1 10000000 +2 10000000 diff --git a/tests/queries/0_stateless/02265_limit_push_down_over_window_functions_bug.sql b/tests/queries/0_stateless/02265_limit_push_down_over_window_functions_bug.sql new file mode 100644 index 00000000000..208ec8ef11f --- /dev/null +++ b/tests/queries/0_stateless/02265_limit_push_down_over_window_functions_bug.sql @@ -0,0 +1,16 @@ +SELECT + number, + leadInFrame(number) OVER w AS W +FROM numbers(10) +WINDOW w AS (ORDER BY number ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) +LIMIT 3; + +WITH arrayJoin(['a', 'a', 'b', 'b']) AS field +SELECT + field, + count() OVER (PARTITION BY field) +ORDER BY field ASC +LIMIT 1; + +select * from ( ( select *, count() over () cnt from ( select * from numbers(10000000) ) ) ) limit 3 ; +select * from ( ( select *, count() over () cnt from ( select * from numbers(10000000) ) ) ) order by number limit 3 ; From 87c2b3e2cf5ccff068bef788690e17fc49c93264 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 8 Apr 2022 10:52:10 -0400 Subject: [PATCH 049/101] fixed Nullable, tests added --- src/Interpreters/FillingRow.cpp | 2 +- .../0_stateless/02233_interpolate_1.reference | 48 +++++++++++++++++++ .../0_stateless/02233_interpolate_1.sql | 12 ++++- 3 files changed, 60 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/FillingRow.cpp b/src/Interpreters/FillingRow.cpp index 9679923f10b..bb8661d0ef9 100644 --- a/src/Interpreters/FillingRow.cpp +++ b/src/Interpreters/FillingRow.cpp @@ -122,7 +122,7 @@ void insertFromFillingRow(MutableColumns & filling_columns, MutableColumns & int { Columns columns = interpolate_block.getColumns(); for (size_t i = 0; i < size; ++i) - interpolate_columns[i]->insertFrom(*columns[i], 0); + interpolate_columns[i]->insertFrom(*columns[i]->convertToFullColumnIfConst(), 0); } else for (const auto & interpolate_column : interpolate_columns) diff --git a/tests/queries/0_stateless/02233_interpolate_1.reference b/tests/queries/0_stateless/02233_interpolate_1.reference index bb1a93131e5..64f5a8308bf 100644 --- a/tests/queries/0_stateless/02233_interpolate_1.reference +++ b/tests/queries/0_stateless/02233_interpolate_1.reference @@ -190,3 +190,51 @@ original 7 10.5 14 11 15 11.5 16 +0 \N +0.5 \N +1 original \N +1.5 \N +2 \N +2.5 \N +3 \N +3.5 \N +4 original \N +4.5 \N +5 \N +5.5 \N +6 \N +6.5 \N +7 original \N +7.5 \N +8 \N +8.5 \N +9 \N +9.5 \N +10 \N +10.5 \N +11 \N +11.5 \N +0 \N +0.5 \N +1 original \N +1.5 \N +2 \N +2.5 \N +3 \N +3.5 \N +4 original \N +4.5 \N +5 \N +5.5 \N +6 \N +6.5 \N +7 original \N +7.5 \N +8 \N +8.5 \N +9 \N +9.5 \N +10 \N +10.5 \N +11 \N +11.5 \N diff --git a/tests/queries/0_stateless/02233_interpolate_1.sql b/tests/queries/0_stateless/02233_interpolate_1.sql index 52b597e06e0..b11385e17b6 100644 --- a/tests/queries/0_stateless/02233_interpolate_1.sql +++ b/tests/queries/0_stateless/02233_interpolate_1.sql @@ -59,4 +59,14 @@ SELECT n, source, inter FROM ( SELECT n, any(source), sum(inter) AS inter_s FROM ( SELECT toFloat32(number % 10) AS n, 'original' AS source, number AS inter FROM numbers(10) WHERE (number % 3) = 1 ) GROUP BY n -ORDER BY n ASC WITH FILL FROM 0 TO 11.51 STEP 0.5 INTERPOLATE ( inter_s AS inter_s + 1 ); \ No newline at end of file +ORDER BY n ASC WITH FILL FROM 0 TO 11.51 STEP 0.5 INTERPOLATE ( inter_s AS inter_s + 1 ); + +# Test INTERPOLATE with Nullable in result +SELECT n, source, inter + NULL AS inter_p FROM ( + SELECT toFloat32(number % 10) AS n, 'original' AS source, number AS inter FROM numbers(10) WHERE (number % 3) = 1 +) ORDER BY n ASC WITH FILL FROM 0 TO 11.51 STEP 0.5 INTERPOLATE ( inter_p AS inter_p + 1 ); + +# Test INTERPOLATE with Nullable in source +SELECT n, source, inter AS inter_p FROM ( + SELECT toFloat32(number % 10) AS n, 'original' AS source, number + NULL AS inter FROM numbers(10) WHERE (number % 3) = 1 +) ORDER BY n ASC WITH FILL FROM 0 TO 11.51 STEP 0.5 INTERPOLATE ( inter_p AS inter_p + 1 ); From 2c99ef0eccbc914e0004498458109feac5bef49c Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 8 Apr 2022 23:04:24 +0800 Subject: [PATCH 050/101] refactor HiveTableMetadata --- src/Storages/Hive/HiveCommon.cpp | 131 +++++++++++++++---------------- src/Storages/Hive/HiveCommon.h | 53 +++++++------ src/Storages/Hive/HiveFile.h | 2 + 3 files changed, 94 insertions(+), 92 deletions(-) diff --git a/src/Storages/Hive/HiveCommon.cpp b/src/Storages/Hive/HiveCommon.cpp index 510ac2a98d6..12b5a7ba7ce 100644 --- a/src/Storages/Hive/HiveCommon.cpp +++ b/src/Storages/Hive/HiveCommon.cpp @@ -29,31 +29,6 @@ ThriftHiveMetastoreClientPool::ThriftHiveMetastoreClientPool(ThriftHiveMetastore { } -bool HiveMetastoreClient::shouldUpdateTableMetadata( - const String & db_name, const String & table_name, const std::vector & partitions) -{ - String cache_key = getCacheKey(db_name, table_name); - HiveTableMetadataPtr metadata = table_metadata_cache.get(cache_key); - if (!metadata) - return true; - - auto old_partiton_infos = metadata->getPartitionInfos(); - if (old_partiton_infos.size() != partitions.size()) - return true; - - for (const auto & partition : partitions) - { - auto it = old_partiton_infos.find(partition.sd.location); - if (it == old_partiton_infos.end()) - return true; - - const auto & old_partition_info = it->second; - if (!old_partition_info.haveSameParameters(partition)) - return true; - } - return false; -} - void HiveMetastoreClient::tryCallHiveClient(std::function func) { int i = 0; @@ -91,44 +66,17 @@ HiveMetastoreClient::HiveTableMetadataPtr HiveMetastoreClient::getTableMetadata( }; tryCallHiveClient(client_call); - bool update_cache = shouldUpdateTableMetadata(db_name, table_name, partitions); + // bool update_cache = shouldUpdateTableMetadata(db_name, table_name, partitions); String cache_key = getCacheKey(db_name, table_name); HiveTableMetadataPtr metadata = table_metadata_cache.get(cache_key); - - if (update_cache) + if (metadata) { - LOG_INFO(log, "Reload hive partition metadata info for {}.{}", db_name, table_name); - - /// Generate partition infos from partitions and old partition infos(if exists). - std::map new_partition_infos; - if (metadata) - { - auto & old_partiton_infos = metadata->getPartitionInfos(); - for (const auto & partition : partitions) - { - auto it = old_partiton_infos.find(partition.sd.location); - if (it == old_partiton_infos.end() || !it->second.haveSameParameters(partition) || !it->second.initialized) - { - new_partition_infos.emplace(partition.sd.location, PartitionInfo(partition)); - continue; - } - else - { - PartitionInfo new_partition_info(partition); - new_partition_info.files = std::move(it->second.files); - new_partition_info.initialized = true; - } - } - } - else - { - for (const auto & partition : partitions) - new_partition_infos.emplace(partition.sd.location, PartitionInfo(partition)); - } - - metadata = std::make_shared( - db_name, table_name, table, std::move(new_partition_infos)); + metadata->updateIfNeeded(partitions); + } + else + { + metadata = std::make_shared(db_name, table_name, table, partitions); table_metadata_cache.set(cache_key, metadata); } return metadata; @@ -157,14 +105,14 @@ void HiveMetastoreClient::clearTableMetadata(const String & db_name, const Strin bool HiveMetastoreClient::PartitionInfo::haveSameParameters(const Apache::Hadoop::Hive::Partition & other) const { /// Parameters include keys:numRows,numFiles,rawDataSize,totalSize,transient_lastDdlTime - auto it1 = partition.parameters.cbegin(); - auto it2 = other.parameters.cbegin(); - for (; it1 != partition.parameters.cend() && it2 != other.parameters.cend(); ++it1, ++it2) + auto it = partition.parameters.cbegin(); + auto oit = other.parameters.cbegin(); + for (; it != partition.parameters.cend() && oit != other.parameters.cend(); ++it, ++oit) { - if (it1->first != it2->first || it1->second != it2->second) + if (it->first != oit->first || it->second != oit->second) return false; } - return (it1 == partition.parameters.cend() && it2 == other.parameters.cend()); + return (it == partition.parameters.cend() && oit == other.parameters.cend()); } std::vector HiveMetastoreClient::HiveTableMetadata::getPartitions() const @@ -221,6 +169,57 @@ std::vector HiveMetastoreClient::HiveTableMetadat return result; } +HiveFilesCachePtr HiveMetastoreClient::HiveTableMetadata::getHiveFilesCache() const +{ + return hive_files_cache; +} + +void HiveMetastoreClient::HiveTableMetadata::updateIfNeeded(const std::vector & partitions) +{ + std::lock_guard lock{mutex}; + + if (!shouldUpdate(partitions)) + return; + + std::map new_partition_infos; + auto & old_partiton_infos = partition_infos; + for (const auto & partition : partitions) + { + auto it = old_partiton_infos.find(partition.sd.location); + if (it == old_partiton_infos.end() || !it->second.haveSameParameters(partition) || !it->second.initialized) + { + new_partition_infos.emplace(partition.sd.location, PartitionInfo(partition)); + continue; + } + else + { + new_partition_infos.emplace(partition.sd.location, std::move(it->second)); + } + } + + partition_infos.swap(new_partition_infos); +} + +bool HiveMetastoreClient::HiveTableMetadata::shouldUpdate(const std::vector & partitions) +{ + const auto & old_partiton_infos = partition_infos; + if (old_partiton_infos.size() != partitions.size()) + return false; + + for (const auto & partition : partitions) + { + auto it = old_partiton_infos.find(partition.sd.location); + if (it == old_partiton_infos.end()) + return true; + + const auto & old_partition_info = it->second; + if (!old_partition_info.haveSameParameters(partition)) + return true; + } + return false; +} + + HiveMetastoreClientFactory & HiveMetastoreClientFactory::instance() { static HiveMetastoreClientFactory factory; @@ -234,7 +233,6 @@ using namespace Apache::Hadoop::Hive; HiveMetastoreClientPtr HiveMetastoreClientFactory::getOrCreate(const String & name) { - std::lock_guard lock(mutex); auto it = clients.find(name); if (it == clients.end()) @@ -244,11 +242,12 @@ HiveMetastoreClientPtr HiveMetastoreClientFactory::getOrCreate(const String & na return createThriftHiveMetastoreClient(name); }; auto client = std::make_shared(builder); - clients[name] = client; + clients.emplace(name, client); return client; } return it->second; } + std::shared_ptr HiveMetastoreClientFactory::createThriftHiveMetastoreClient(const String &name) { Poco::URI hive_metastore_url(name); diff --git a/src/Storages/Hive/HiveCommon.h b/src/Storages/Hive/HiveCommon.h index bce7d291351..985d0a0a716 100644 --- a/src/Storages/Hive/HiveCommon.h +++ b/src/Storages/Hive/HiveCommon.h @@ -13,6 +13,7 @@ #include #include #include +#include namespace DB @@ -40,7 +41,6 @@ private: class HiveMetastoreClient { public: - struct FileInfo { String path; @@ -63,57 +63,61 @@ public: bool initialized = false; /// If true, files are initialized. explicit PartitionInfo(const Apache::Hadoop::Hive::Partition & partition_): partition(partition_) {} + PartitionInfo(PartitionInfo &&) = default; + bool haveSameParameters(const Apache::Hadoop::Hive::Partition & other) const; }; + class HiveTableMetadata; + using HiveTableMetadataPtr = std::shared_ptr; /// Used for speeding up metadata query process. - struct HiveTableMetadata + class HiveTableMetadata : boost::noncopyable { public: HiveTableMetadata( const String & db_name_, const String & table_name_, std::shared_ptr table_, - const std::map & partition_infos_) + const std::vector & partitions_) : db_name(db_name_) , table_name(table_name_) - , table(table_) - , partition_infos(partition_infos_) + , table(std::move(table_)) , empty_partition_keys(table->partitionKeys.empty()) + , hive_files_cache(std::make_shared(10000)) { + std::lock_guard lock(mutex); + for (const auto & partition : partitions_) + partition_infos.emplace(partition.sd.location, PartitionInfo(partition)); } - - std::map & getPartitionInfos() - { - std::lock_guard lock{mutex}; - return partition_infos; - } - - std::shared_ptr getTable() const - { - std::lock_guard lock{mutex}; - return table; - } + std::shared_ptr getTable() const { return table; } std::vector getPartitions() const; std::vector getFilesByLocation(const HDFSFSPtr & fs, const String & location); - private: - String db_name; - String table_name; + HiveFilesCachePtr getHiveFilesCache() const; + void updateIfNeeded(const std::vector & partitions); + + private: + bool shouldUpdate(const std::vector & partitions); + + const String db_name; + const String table_name; + const std::shared_ptr table; + + /// Mutex to protect partition_infos. mutable std::mutex mutex; - std::shared_ptr table; std::map partition_infos; + const bool empty_partition_keys; + const HiveFilesCachePtr hive_files_cache; Poco::Logger * log = &Poco::Logger::get("HiveMetastoreClient"); }; - using HiveTableMetadataPtr = std::shared_ptr; explicit HiveMetastoreClient(ThriftHiveMetastoreClientBuilder builder_) : table_metadata_cache(1000) @@ -129,9 +133,6 @@ public: private: static String getCacheKey(const String & db_name, const String & table_name) { return db_name + "." + table_name; } - bool shouldUpdateTableMetadata( - const String & db_name, const String & table_name, const std::vector & partitions); - void tryCallHiveClient(std::function func); LRUCache table_metadata_cache; @@ -148,9 +149,9 @@ public: HiveMetastoreClientPtr getOrCreate(const String & name); +private: static std::shared_ptr createThriftHiveMetastoreClient(const String & name); -private: std::mutex mutex; std::map clients; }; diff --git a/src/Storages/Hive/HiveFile.h b/src/Storages/Hive/HiveFile.h index 0faf2a190b7..6752da15a5d 100644 --- a/src/Storages/Hive/HiveFile.h +++ b/src/Storages/Hive/HiveFile.h @@ -166,6 +166,8 @@ protected: using HiveFilePtr = std::shared_ptr; using HiveFiles = std::vector; +using HiveFilesCache = LRUCache; +using HiveFilesCachePtr = std::shared_ptr; class HiveTextFile : public IHiveFile { From 50a32a74b70fbf7460bef948b14e742513b83020 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 8 Apr 2022 15:38:35 +0200 Subject: [PATCH 051/101] add thread_id and query_id to zookeeper_log --- src/Common/ZooKeeper/ZooKeeperCommon.h | 2 ++ src/Common/ZooKeeper/ZooKeeperImpl.cpp | 11 +++++++++++ src/Common/ZooKeeper/ZooKeeperImpl.h | 1 + src/Interpreters/ZooKeeperLog.cpp | 4 ++++ src/Interpreters/ZooKeeperLog.h | 2 ++ 5 files changed, 20 insertions(+) diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.h b/src/Common/ZooKeeper/ZooKeeperCommon.h index 80e38dd74d5..e51bea3f7f8 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.h +++ b/src/Common/ZooKeeper/ZooKeeperCommon.h @@ -57,6 +57,8 @@ struct ZooKeeperRequest : virtual Request bool restored_from_zookeeper_log = false; UInt64 request_created_time_ns = 0; + UInt64 thread_id = 0; + String query_id; ZooKeeperRequest() = default; ZooKeeperRequest(const ZooKeeperRequest &) = default; diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 935df255843..b8a6e040bc8 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include @@ -1016,6 +1017,11 @@ void ZooKeeper::pushRequest(RequestInfo && info) try { info.time = clock::now(); + if (zk_log) + { + info.request->thread_id = getThreadId(); + info.request->query_id = String(CurrentThread::getQueryId()); + } if (!info.request->xid) { @@ -1269,6 +1275,11 @@ void ZooKeeper::logOperationIfNeeded(const ZooKeeperRequestPtr & request, const elem.event_time = event_time; elem.address = socket_address; elem.session_id = session_id; + if (request) + { + elem.thread_id = request->thread_id; + elem.query_id = request->query_id; + } maybe_zk_log->add(elem); } } diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.h b/src/Common/ZooKeeper/ZooKeeperImpl.h index 58c5947e8ea..faa777c33aa 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -219,6 +219,7 @@ private: ResponseCallback callback; WatchCallback watch; clock::time_point time; + UInt64 thread_id = 0; }; using RequestsQueue = ConcurrentBoundedQueue; diff --git a/src/Interpreters/ZooKeeperLog.cpp b/src/Interpreters/ZooKeeperLog.cpp index fdcbe430834..2828c3e2fa8 100644 --- a/src/Interpreters/ZooKeeperLog.cpp +++ b/src/Interpreters/ZooKeeperLog.cpp @@ -116,6 +116,8 @@ NamesAndTypesList ZooKeeperLogElement::getNamesAndTypes() {"type", std::move(type_enum)}, {"event_date", std::make_shared()}, {"event_time", std::make_shared(6)}, + {"thread_id", std::make_shared()}, + {"query_id", std::make_shared()}, {"address", DataTypeFactory::instance().get("IPv6")}, {"port", std::make_shared()}, {"session_id", std::make_shared()}, @@ -164,6 +166,8 @@ void ZooKeeperLogElement::appendToBlock(MutableColumns & columns) const auto event_time_seconds = event_time / 1000000; columns[i++]->insert(DateLUT::instance().toDayNum(event_time_seconds).toUnderType()); columns[i++]->insert(event_time); + columns[i++]->insert(thread_id); + columns[i++]->insert(query_id); columns[i++]->insertData(IPv6ToBinary(address.host()).data(), 16); columns[i++]->insert(address.port()); columns[i++]->insert(session_id); diff --git a/src/Interpreters/ZooKeeperLog.h b/src/Interpreters/ZooKeeperLog.h index 284675a7ff5..733c61ba8fb 100644 --- a/src/Interpreters/ZooKeeperLog.h +++ b/src/Interpreters/ZooKeeperLog.h @@ -22,6 +22,8 @@ struct ZooKeeperLogElement Type type = UNKNOWN; Decimal64 event_time = 0; + UInt64 thread_id = 0; + String query_id; Poco::Net::SocketAddress address; Int64 session_id = 0; From e319df17993d75d12992b28888dbafe3b641a7fa Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 8 Apr 2022 23:58:56 +0800 Subject: [PATCH 052/101] finish dev --- src/Storages/Hive/HiveFile.h | 10 +++--- src/Storages/Hive/StorageHive.cpp | 57 +++++++++++++++++-------------- src/Storages/Hive/StorageHive.h | 15 ++++---- 3 files changed, 44 insertions(+), 38 deletions(-) diff --git a/src/Storages/Hive/HiveFile.h b/src/Storages/Hive/HiveFile.h index 6752da15a5d..cbdf17bd5b5 100644 --- a/src/Storages/Hive/HiveFile.h +++ b/src/Storages/Hive/HiveFile.h @@ -83,7 +83,7 @@ public: size_t size_, const NamesAndTypesList & index_names_and_types_, const std::shared_ptr & storage_settings_, - ContextPtr context_) + const ContextPtr & context_) : WithContext(context_) , partition_values(partition_values_) , namenode_url(namenode_url_) @@ -166,7 +166,7 @@ protected: using HiveFilePtr = std::shared_ptr; using HiveFiles = std::vector; -using HiveFilesCache = LRUCache; +using HiveFilesCache = LRUCache; using HiveFilesCachePtr = std::shared_ptr; class HiveTextFile : public IHiveFile @@ -180,7 +180,7 @@ public: size_t size_, const NamesAndTypesList & index_names_and_types_, const std::shared_ptr & hive_settings_, - ContextPtr context_) + const ContextPtr & context_) : IHiveFile(partition_values_, namenode_url_, path_, last_modify_time_, size_, index_names_and_types_, hive_settings_, context_) { } @@ -202,7 +202,7 @@ public: size_t size_, const NamesAndTypesList & index_names_and_types_, const std::shared_ptr & hive_settings_, - ContextPtr context_) + const ContextPtr & context_) : IHiveFile(partition_values_, namenode_url_, path_, last_modify_time_, size_, index_names_and_types_, hive_settings_, context_) { } @@ -238,7 +238,7 @@ public: size_t size_, const NamesAndTypesList & index_names_and_types_, const std::shared_ptr & hive_settings_, - ContextPtr context_) + const ContextPtr & context_) : IHiveFile(partition_values_, namenode_url_, path_, last_modify_time_, size_, index_names_and_types_, hive_settings_, context_) { } diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index a91ecf6e280..9bc0a3bd2ec 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -424,7 +424,7 @@ static HiveFilePtr createHiveFile( size_t size, const NamesAndTypesList & index_names_and_types, const std::shared_ptr & hive_settings, - ContextPtr context) + const ContextPtr & context) { HiveFilePtr hive_file; if (format_name == "HiveText") @@ -449,9 +449,9 @@ static HiveFilePtr createHiveFile( HiveFiles StorageHive::collectHiveFilesFromPartition( const Apache::Hadoop::Hive::Partition & partition, const SelectQueryInfo & query_info, - HiveTableMetadataPtr hive_table_metadata, + const HiveTableMetadataPtr & hive_table_metadata, const HDFSFSPtr & fs, - ContextPtr context_, + const ContextPtr & context_, PruneLevel prune_level) const { LOG_DEBUG( @@ -517,7 +517,7 @@ HiveFiles StorageHive::collectHiveFilesFromPartition( hive_files.reserve(file_infos.size()); for (const auto & file_info : file_infos) { - auto hive_file = createHiveFileIfNeeded(file_info, fields, query_info, context_, prune_level); + auto hive_file = getHiveFileIfNeeded(file_info, fields, query_info, hive_table_metadata, context_, prune_level); if (hive_file) { LOG_TRACE( @@ -533,42 +533,47 @@ HiveFiles StorageHive::collectHiveFilesFromPartition( } std::vector -StorageHive::listDirectory(const String & path, HiveTableMetadataPtr hive_table_metadata, const HDFSFSPtr & fs) +StorageHive::listDirectory(const String & path, const HiveTableMetadataPtr & hive_table_metadata, const HDFSFSPtr & fs) { return hive_table_metadata->getFilesByLocation(fs, path); } -HiveFilePtr StorageHive::createHiveFileIfNeeded( +HiveFilePtr StorageHive::getHiveFileIfNeeded( const FileInfo & file_info, const FieldVector & fields, const SelectQueryInfo & query_info, - ContextPtr context_, + const HiveTableMetadataPtr & hive_table_metadata, + const ContextPtr & context_, PruneLevel prune_level) const { - LOG_TRACE(log, "Create hive file {} if needed, prune_level:{}", file_info.path, pruneLevelToString(prune_level)); + LOG_TRACE(log, "Get hive file {} if needed, prune_level:{}", file_info.path, pruneLevelToString(prune_level)); String filename = getBaseName(file_info.path); /// Skip temporary files starts with '.' if (startsWith(filename, ".")) return {}; - auto hive_file = createHiveFile( - format_name, - fields, - hdfs_namenode_url, - file_info.path, - file_info.last_modify_time, - file_info.size, - hivefile_name_types, - storage_settings, - context_); - /* + auto cache = hive_table_metadata->getHiveFilesCache(); + auto hive_file = cache->get(file_info.path); + if (!hive_file || hive_file->getLastModTs() < file_info.last_modify_time) { - std::lock_guard lock{init_mutex}; - hive_files_by_path[file_info.path] = hive_file; - std::cout << "size:" << hive_files_by_path.size() << std::endl; + LOG_TRACE(log, "Create hive file {}", file_info.path); + hive_file = createHiveFile( + format_name, + fields, + hdfs_namenode_url, + file_info.path, + file_info.last_modify_time, + file_info.size, + hivefile_name_types, + storage_settings, + context_); + cache->set(file_info.path, hive_file); + } + else + { + LOG_TRACE(log, "Get hive file {} from cache", file_info.path); } - */ if (prune_level >= PruneLevel::File) { @@ -711,9 +716,9 @@ Pipe StorageHive::read( HiveFiles StorageHive::collectHiveFiles( unsigned max_threads, const SelectQueryInfo & query_info, - HiveTableMetadataPtr hive_table_metadata, + const HiveTableMetadataPtr & hive_table_metadata, const HDFSFSPtr & fs, - ContextPtr context_, + const ContextPtr & context_, PruneLevel prune_level) const { std::vector partitions = hive_table_metadata->getPartitions(); @@ -751,7 +756,7 @@ HiveFiles StorageHive::collectHiveFiles( pool.scheduleOrThrowOnError( [&]() { - auto hive_file = createHiveFileIfNeeded(file_info, {}, query_info, context_, prune_level); + auto hive_file = getHiveFileIfNeeded(file_info, {}, query_info, hive_table_metadata, context_, prune_level); if (hive_file) { std::lock_guard lock(hive_files_mutex); diff --git a/src/Storages/Hive/StorageHive.h b/src/Storages/Hive/StorageHive.h index b9841d28252..f7cd32c2162 100644 --- a/src/Storages/Hive/StorageHive.h +++ b/src/Storages/Hive/StorageHive.h @@ -91,31 +91,32 @@ private: static ASTPtr extractKeyExpressionList(const ASTPtr & node); - static std::vector listDirectory(const String & path, HiveTableMetadataPtr hive_table_metadata, const HDFSFSPtr & fs); + static std::vector listDirectory(const String & path, const HiveTableMetadataPtr & hive_table_metadata, const HDFSFSPtr & fs); void initMinMaxIndexExpression(); HiveFiles collectHiveFiles( unsigned max_threads, const SelectQueryInfo & query_info, - HiveTableMetadataPtr hive_table_metadata, + const HiveTableMetadataPtr & hive_table_metadata, const HDFSFSPtr & fs, - ContextPtr context_, + const ContextPtr & context_, PruneLevel prune_level = PruneLevel::Max) const; HiveFiles collectHiveFilesFromPartition( const Apache::Hadoop::Hive::Partition & partition, const SelectQueryInfo & query_info, - HiveTableMetadataPtr hive_table_metadata, + const HiveTableMetadataPtr & hive_table_metadata, const HDFSFSPtr & fs, - ContextPtr context_, + const ContextPtr & context_, PruneLevel prune_level = PruneLevel::Max) const; - HiveFilePtr createHiveFileIfNeeded( + HiveFilePtr getHiveFileIfNeeded( const FileInfo & file_info, const FieldVector & fields, const SelectQueryInfo & query_info, - ContextPtr context_, + const HiveTableMetadataPtr & hive_table_metadata, + const ContextPtr & context_, PruneLevel prune_level = PruneLevel::Max) const; void getActualColumnsToRead(Block & sample_block, const Block & header_block, const NameSet & partition_columns) const; From cd807da83890608b5d30b5cec1ee0b8f4f359e30 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Sat, 9 Apr 2022 00:15:33 +0800 Subject: [PATCH 053/101] finish test --- src/Storages/Hive/StorageHive.cpp | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 9bc0a3bd2ec..3302df7da3d 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -546,8 +546,6 @@ HiveFilePtr StorageHive::getHiveFileIfNeeded( const ContextPtr & context_, PruneLevel prune_level) const { - LOG_TRACE(log, "Get hive file {} if needed, prune_level:{}", file_info.path, pruneLevelToString(prune_level)); - String filename = getBaseName(file_info.path); /// Skip temporary files starts with '.' if (startsWith(filename, ".")) @@ -557,7 +555,7 @@ HiveFilePtr StorageHive::getHiveFileIfNeeded( auto hive_file = cache->get(file_info.path); if (!hive_file || hive_file->getLastModTs() < file_info.last_modify_time) { - LOG_TRACE(log, "Create hive file {}", file_info.path); + LOG_TRACE(log, "Create hive file {}, prune_level {}", file_info.path, pruneLevelToString(prune_level)); hive_file = createHiveFile( format_name, fields, @@ -572,7 +570,7 @@ HiveFilePtr StorageHive::getHiveFileIfNeeded( } else { - LOG_TRACE(log, "Get hive file {} from cache", file_info.path); + LOG_TRACE(log, "Get hive file {} from cache, prune_level {}", file_info.path, pruneLevelToString(prune_level)); } if (prune_level >= PruneLevel::File) From 70f4503ba5d2e5bb73348773bc451bb026838437 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Sat, 9 Apr 2022 00:28:07 +0800 Subject: [PATCH 054/101] use global context for cache --- src/Storages/Hive/StorageHive.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 3302df7da3d..633816bb195 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -565,7 +565,7 @@ HiveFilePtr StorageHive::getHiveFileIfNeeded( file_info.size, hivefile_name_types, storage_settings, - context_); + context_->getGlobalContext()); cache->set(file_info.path, hive_file); } else From 833836e8da5842b822425bd7a2bd6b811469c740 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 8 Apr 2022 16:46:11 +0000 Subject: [PATCH 055/101] Try to avoid timeoutes in parallel parsing tests --- .../00159_parallel_formatting_csv_and_friends.sh | 4 ++-- .../00159_parallel_formatting_json_and_friends.sh | 4 ++-- .../1_stateful/00161_parallel_parsing_with_names.sh | 8 ++++---- .../00167_parallel_parsing_with_names_and_types.sh | 8 ++++---- 4 files changed, 12 insertions(+), 12 deletions(-) diff --git a/tests/queries/1_stateful/00159_parallel_formatting_csv_and_friends.sh b/tests/queries/1_stateful/00159_parallel_formatting_csv_and_friends.sh index 1476d2892bf..6f251e7f95a 100755 --- a/tests/queries/1_stateful/00159_parallel_formatting_csv_and_friends.sh +++ b/tests/queries/1_stateful/00159_parallel_formatting_csv_and_friends.sh @@ -9,11 +9,11 @@ FORMATS=('CSV' 'CSVWithNames') for format in "${FORMATS[@]}" do echo "$format, false"; - $CLICKHOUSE_CLIENT --output_format_parallel_formatting=false -q \ + $CLICKHOUSE_CLIENT --max_threads=0 --output_format_parallel_formatting=false -q \ "SELECT ClientEventTime::DateTime('Asia/Dubai') as a, MobilePhoneModel as b, ClientIP6 as c FROM test.hits ORDER BY a, b, c Format $format" | md5sum echo "$format, true"; - $CLICKHOUSE_CLIENT --output_format_parallel_formatting=true -q \ + $CLICKHOUSE_CLIENT --max_threads=0 --output_format_parallel_formatting=true -q \ "SELECT ClientEventTime::DateTime('Asia/Dubai') as a, MobilePhoneModel as b, ClientIP6 as c FROM test.hits ORDER BY a, b, c Format $format" | md5sum done diff --git a/tests/queries/1_stateful/00159_parallel_formatting_json_and_friends.sh b/tests/queries/1_stateful/00159_parallel_formatting_json_and_friends.sh index a96ed0c9b96..036eb9aecb3 100755 --- a/tests/queries/1_stateful/00159_parallel_formatting_json_and_friends.sh +++ b/tests/queries/1_stateful/00159_parallel_formatting_json_and_friends.sh @@ -11,10 +11,10 @@ FORMATS=('JSONEachRow' 'JSONCompactEachRow' 'JSONCompactStringsEachRow' 'JSONCom for format in "${FORMATS[@]}" do echo "$format, false"; - $CLICKHOUSE_CLIENT --output_format_parallel_formatting=false -q \ + $CLICKHOUSE_CLIENT --max_therads=0 --output_format_parallel_formatting=false -q \ "SELECT ClientEventTime::DateTime('Asia/Dubai') as a, MobilePhoneModel as b, ClientIP6 as c FROM test.hits ORDER BY a, b, c LIMIT 3000000 Format $format" | md5sum echo "$format, true"; - $CLICKHOUSE_CLIENT --output_format_parallel_formatting=true -q \ + $CLICKHOUSE_CLIENT --max_threads=0 --output_format_parallel_formatting=true -q \ "SELECT ClientEventTime::DateTime('Asia/Dubai') as a, MobilePhoneModel as b, ClientIP6 as c FROM test.hits ORDER BY a, b, c LIMIT 3000000 Format $format" | md5sum done diff --git a/tests/queries/1_stateful/00161_parallel_parsing_with_names.sh b/tests/queries/1_stateful/00161_parallel_parsing_with_names.sh index d14a174d3a0..18cd8d6caa4 100755 --- a/tests/queries/1_stateful/00161_parallel_parsing_with_names.sh +++ b/tests/queries/1_stateful/00161_parallel_parsing_with_names.sh @@ -13,9 +13,9 @@ do $CLICKHOUSE_CLIENT -q "CREATE TABLE parsing_with_names(c FixedString(16), a DateTime('Asia/Dubai'), b String) ENGINE=Memory()" echo "$format, false"; - $CLICKHOUSE_CLIENT --max_block_size=65505 --output_format_parallel_formatting=false -q \ + $CLICKHOUSE_CLIENT --max_therads=0 --max_block_size=65505 --output_format_parallel_formatting=false -q \ "SELECT URLRegions as d, toTimeZone(ClientEventTime, 'Asia/Dubai') as a, MobilePhoneModel as b, ParamPrice as e, ClientIP6 as c FROM test.hits LIMIT 50000 Format $format" | \ - $CLICKHOUSE_CLIENT --max_block_size=65505 --input_format_skip_unknown_fields=1 --input_format_parallel_parsing=false -q "INSERT INTO parsing_with_names FORMAT $format" + $CLICKHOUSE_CLIENT --max_threads=0 --max_block_size=65505 --input_format_skip_unknown_fields=1 --input_format_parallel_parsing=false -q "INSERT INTO parsing_with_names FORMAT $format" $CLICKHOUSE_CLIENT -q "SELECT * FROM parsing_with_names;" | md5sum $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parsing_with_names" @@ -23,9 +23,9 @@ do $CLICKHOUSE_CLIENT -q "CREATE TABLE parsing_with_names(c FixedString(16), a DateTime('Asia/Dubai'), b String) ENGINE=Memory()" echo "$format, true"; - $CLICKHOUSE_CLIENT --max_block_size=65505 --output_format_parallel_formatting=false -q \ + $CLICKHOUSE_CLIENT --max_threads=0 --max_block_size=65505 --output_format_parallel_formatting=false -q \ "SELECT URLRegions as d, toTimeZone(ClientEventTime, 'Asia/Dubai') as a, MobilePhoneModel as b, ParamPrice as e, ClientIP6 as c FROM test.hits LIMIT 50000 Format $format" | \ - $CLICKHOUSE_CLIENT --max_block_size=65505 --input_format_skip_unknown_fields=1 --input_format_parallel_parsing=true -q "INSERT INTO parsing_with_names FORMAT $format" + $CLICKHOUSE_CLIENT --max_threads=0 --max_block_size=65505 --input_format_skip_unknown_fields=1 --input_format_parallel_parsing=true -q "INSERT INTO parsing_with_names FORMAT $format" $CLICKHOUSE_CLIENT -q "SELECT * FROM parsing_with_names;" | md5sum $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parsing_with_names" diff --git a/tests/queries/1_stateful/00167_parallel_parsing_with_names_and_types.sh b/tests/queries/1_stateful/00167_parallel_parsing_with_names_and_types.sh index 433d51a3036..29546ca1814 100755 --- a/tests/queries/1_stateful/00167_parallel_parsing_with_names_and_types.sh +++ b/tests/queries/1_stateful/00167_parallel_parsing_with_names_and_types.sh @@ -13,9 +13,9 @@ do $CLICKHOUSE_CLIENT -q "CREATE TABLE parsing_with_names(c FixedString(16), a DateTime('Asia/Dubai'), b String) ENGINE=Memory()" echo "$format, false"; - $CLICKHOUSE_CLIENT --output_format_parallel_formatting=false -q \ + $CLICKHOUSE_CLIENT --max_threads=0 --output_format_parallel_formatting=false -q \ "SELECT URLRegions as d, toTimeZone(ClientEventTime, 'Asia/Dubai') as a, MobilePhoneModel as b, ParamPrice as e, ClientIP6 as c FROM test.hits LIMIT 5000 Format $format" | \ - $CLICKHOUSE_CLIENT --input_format_skip_unknown_fields=1 --input_format_parallel_parsing=false -q "INSERT INTO parsing_with_names SETTINGS input_format_null_as_default=0 FORMAT $format" + $CLICKHOUSE_CLIENT --max_threads=0 --input_format_skip_unknown_fields=1 --input_format_parallel_parsing=false -q "INSERT INTO parsing_with_names SETTINGS input_format_null_as_default=0 FORMAT $format" $CLICKHOUSE_CLIENT -q "SELECT * FROM parsing_with_names;" | md5sum $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parsing_with_names" @@ -23,9 +23,9 @@ do $CLICKHOUSE_CLIENT -q "CREATE TABLE parsing_with_names(c FixedString(16), a DateTime('Asia/Dubai'), b String) ENGINE=Memory()" echo "$format, true"; - $CLICKHOUSE_CLIENT --output_format_parallel_formatting=false -q \ + $CLICKHOUSE_CLIENT --max_threads=0 --output_format_parallel_formatting=false -q \ "SELECT URLRegions as d, toTimeZone(ClientEventTime, 'Asia/Dubai') as a, MobilePhoneModel as b, ParamPrice as e, ClientIP6 as c FROM test.hits LIMIT 5000 Format $format" | \ - $CLICKHOUSE_CLIENT --input_format_skip_unknown_fields=1 --input_format_parallel_parsing=true -q "INSERT INTO parsing_with_names SETTINGS input_format_null_as_default=0 FORMAT $format" + $CLICKHOUSE_CLIENT --max_threads=0 --input_format_skip_unknown_fields=1 --input_format_parallel_parsing=true -q "INSERT INTO parsing_with_names SETTINGS input_format_null_as_default=0 FORMAT $format" $CLICKHOUSE_CLIENT -q "SELECT * FROM parsing_with_names;" | md5sum $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parsing_with_names" From 7e89f760f36d70756b2076cbf14ffd304fd5f703 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Sat, 9 Apr 2022 10:43:58 +0800 Subject: [PATCH 056/101] remove useless code --- src/Storages/Hive/StorageHive.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Storages/Hive/StorageHive.h b/src/Storages/Hive/StorageHive.h index f7cd32c2162..63b42d68a82 100644 --- a/src/Storages/Hive/StorageHive.h +++ b/src/Storages/Hive/StorageHive.h @@ -156,8 +156,6 @@ private: std::shared_ptr storage_settings; - // mutable std::map hive_files_by_path; - Poco::Logger * log = &Poco::Logger::get("StorageHive"); }; From 090fd728842b9cc20800df0ace80d9f59ba673cc Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 11 Apr 2022 11:19:31 +0800 Subject: [PATCH 057/101] fix bug --- src/Storages/Hive/HiveCommon.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/Hive/HiveCommon.cpp b/src/Storages/Hive/HiveCommon.cpp index 12b5a7ba7ce..7b2f04f7073 100644 --- a/src/Storages/Hive/HiveCommon.cpp +++ b/src/Storages/Hive/HiveCommon.cpp @@ -204,7 +204,7 @@ bool HiveMetastoreClient::HiveTableMetadata::shouldUpdate(const std::vector Date: Mon, 11 Apr 2022 09:42:00 +0200 Subject: [PATCH 058/101] fix --- src/Common/ZooKeeper/ZooKeeperImpl.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.h b/src/Common/ZooKeeper/ZooKeeperImpl.h index faa777c33aa..58c5947e8ea 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -219,7 +219,6 @@ private: ResponseCallback callback; WatchCallback watch; clock::time_point time; - UInt64 thread_id = 0; }; using RequestsQueue = ConcurrentBoundedQueue; From 5c0d15e0170cf15abc53174220d320ab2453742c Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 11 Apr 2022 12:52:26 +0200 Subject: [PATCH 059/101] fix flaky tests --- tests/queries/0_stateless/00971_query_id_in_logs.sh | 3 ++- .../queries/0_stateless/01003_kill_query_race_condition.sh | 6 +++--- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/00971_query_id_in_logs.sh b/tests/queries/0_stateless/00971_query_id_in_logs.sh index 9e927f36a9c..4de6e02d10d 100755 --- a/tests/queries/0_stateless/00971_query_id_in_logs.sh +++ b/tests/queries/0_stateless/00971_query_id_in_logs.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-parallel CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=trace @@ -9,4 +10,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) set -e # No log lines without query id -$CLICKHOUSE_CLIENT --query_id=hello --query="SELECT count() FROM numbers(10)" 2>&1 | grep -vF ' {hello} ' | grep -P '<\w+>' ||: +$CLICKHOUSE_CLIENT --query_id=hello_00971 --query="SELECT count() FROM numbers(10)" 2>&1 | grep -vF ' {hello_00971} ' | grep -P '<\w+>' ||: diff --git a/tests/queries/0_stateless/01003_kill_query_race_condition.sh b/tests/queries/0_stateless/01003_kill_query_race_condition.sh index 64caf0f88d1..f98897b1544 100755 --- a/tests/queries/0_stateless/01003_kill_query_race_condition.sh +++ b/tests/queries/0_stateless/01003_kill_query_race_condition.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: race +# Tags: race, no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -9,12 +9,12 @@ set -e function thread1() { - $CLICKHOUSE_CLIENT --query_id=hello --query "SELECT count() FROM numbers(1000000000)" --format Null; + $CLICKHOUSE_CLIENT --query_id=hello_01003 --query "SELECT count() FROM numbers(1000000000)" --format Null; } function thread2() { - $CLICKHOUSE_CLIENT --query "KILL QUERY WHERE query_id = 'hello'" --format Null + $CLICKHOUSE_CLIENT --query "KILL QUERY WHERE query_id = 'hello_01003'" --format Null sleep 0.$RANDOM } From 6e346a7fc38d683a198a506b205f562b4085c7d9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 11 Apr 2022 14:32:55 +0200 Subject: [PATCH 060/101] Support atomic replace under OSX --- .../{renameat2.cpp => atomicRename.cpp} | 65 +++++++++++++++++-- src/Common/{renameat2.h => atomicRename.h} | 2 +- src/Databases/DatabaseAtomic.cpp | 4 +- src/Disks/DiskLocal.cpp | 2 +- src/Interpreters/DatabaseCatalog.cpp | 2 +- src/Interpreters/InterpreterCreateQuery.cpp | 2 +- 6 files changed, 65 insertions(+), 12 deletions(-) rename src/Common/{renameat2.cpp => atomicRename.cpp} (69%) rename src/Common/{renameat2.h => atomicRename.h} (95%) diff --git a/src/Common/renameat2.cpp b/src/Common/atomicRename.cpp similarity index 69% rename from src/Common/renameat2.cpp rename to src/Common/atomicRename.cpp index 8ee9081af56..fb6b0ada40b 100644 --- a/src/Common/renameat2.cpp +++ b/src/Common/atomicRename.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include @@ -55,7 +55,7 @@ namespace ErrorCodes namespace DB { -static bool supportsRenameat2Impl() +static bool supportsAtomicRenameImpl() { VersionNumber renameat2_minimal_version(3, 15, 0); VersionNumber linux_version(Poco::Environment::osVersion()); @@ -64,7 +64,7 @@ static bool supportsRenameat2Impl() static bool renameat2(const std::string & old_path, const std::string & new_path, int flags) { - if (!supportsRenameat2()) + if (!supportsAtomicRename()) return false; if (old_path.empty() || new_path.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot rename {} to {}: path is empty", old_path, new_path); @@ -93,14 +93,67 @@ static bool renameat2(const std::string & old_path, const std::string & new_path throwFromErrnoWithPath(fmt::format("Cannot rename {} to {}", old_path, new_path), new_path, ErrorCodes::SYSTEM_ERROR); } -bool supportsRenameat2() +bool supportsAtomicRename() { - static bool supports = supportsRenameat2Impl(); + static bool supports = supportsAtomicRenameImpl(); return supports; } } +#elif defined(__APPLE__) + +// Includes +#include // For dlsym +#include // For renamex_np +#include // For stderror + +#ifndef RENAME_SWAP + #define RENAME_SWAP 0x00000002 +#endif +#ifndef RENAME_EXCL + #define RENAME_EXCL 0x00000004 +#endif + + +#define RENAME_NOREPLACE RENAME_EXCL +#define RENAME_EXCHANGE RENAME_SWAP + + +static bool renameat2(const std::string & old_path, const std::string & new_path, int flags) +{ + auto fun = dlsym(RTLD_DEFAULT, "renamex_np"); + if (fun == NULL) + return false; + + if (old_path.empty() || new_path.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot rename {} to {}: path is empty", old_path, new_path); + + /// int olddirfd (ignored for absolute oldpath), const char *oldpath, + /// int newdirfd (ignored for absolute newpath), const char *newpath, + /// unsigned int flags + if (0 == (*fun)(old_path.c_str(), AT_FDCWD, new_path.c_str(), flags)) + return true; + int errnum = errno; + + if ((errnum == ENOTSUP) || (errnum == EINVAL) + return false; + if (errnum == EEXIST) + throwFromErrno(fmt::format("Cannot rename {} to {} because the second path already exists", old_path, new_path), ErrorCodes::ATOMIC_RENAME_FAIL); + if (errnum == ENOENT) + throwFromErrno(fmt::format("Paths cannot be exchanged because {} or {} does not exist", old_path, new_path), ErrorCodes::ATOMIC_RENAME_FAIL); + throwFromErrnoWithPath(fmt::format("Cannot rename {} to {}: {}", old_path, new_path, stderror(errnum)), new_path, ErrorCodes::SYSTEM_ERROR); +} + + + +static bool supportsAtomicRenameImpl() +{ + auto fun = dlsym(RTLD_DEFAULT, "renamex_np"); + return fun != NULL; +} + + #else #define RENAME_NOREPLACE -1 @@ -114,7 +167,7 @@ static bool renameat2(const std::string &, const std::string &, int) return false; } -bool supportsRenameat2() +bool supportsAtomicRename() { return false; } diff --git a/src/Common/renameat2.h b/src/Common/atomicRename.h similarity index 95% rename from src/Common/renameat2.h rename to src/Common/atomicRename.h index 141c5d385c5..6da8a8f623b 100644 --- a/src/Common/renameat2.h +++ b/src/Common/atomicRename.h @@ -6,7 +6,7 @@ namespace DB { /// Returns true, if the following functions supported by the system -bool supportsRenameat2(); +bool supportsAtomicRename(); /// Atomically rename old_path to new_path. If new_path exists, do not overwrite it and throw exception void renameNoReplace(const std::string & old_path, const std::string & new_path); diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index adfcd83f5a7..622d38e01bd 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include #include #include @@ -158,7 +158,7 @@ void DatabaseAtomic::renameTable(ContextPtr local_context, const String & table_ return; } - if (exchange && !supportsRenameat2()) + if (exchange && !supportsAtomicRename()) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "RENAME EXCHANGE is not supported"); auto & other_db = dynamic_cast(to_database); diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index 8aad42ab475..d81782a8af1 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -6,7 +6,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 2f51d942403..7513c3bf849 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -12,7 +12,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index f3953eb67fe..24c58c819a4 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -8,7 +8,7 @@ #include #include #include -#include +#include #include #include From 36e2809fd74df11e8f970817bd66fb21332e29a3 Mon Sep 17 00:00:00 2001 From: yuuch Date: Mon, 11 Apr 2022 20:46:24 +0800 Subject: [PATCH 061/101] support '\G;' at the end of query --- src/Client/ClientBase.cpp | 4 +- ...ient_interactive_vertical_multiline.expect | 39 +++++++++++++++++++ ...ent_interactive_vertical_singleline.expect | 15 +++++++ 3 files changed, 56 insertions(+), 2 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index e77d683d26f..b79a67b7526 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1838,7 +1838,7 @@ void ClientBase::runInteractive() } LineReader::Patterns query_extenders = {"\\"}; - LineReader::Patterns query_delimiters = {";", "\\G"}; + LineReader::Patterns query_delimiters = {";", "\\G", "\\G;"}; #if USE_REPLXX replxx::Replxx::highlighter_callback_t highlight_callback{}; @@ -1860,7 +1860,7 @@ void ClientBase::runInteractive() break; has_vertical_output_suffix = false; - if (input.ends_with("\\G")) + if (input.ends_with("\\G") || input.ends_with("\\G;")) { input.resize(input.size() - 2); has_vertical_output_suffix = true; diff --git a/tests/queries/0_stateless/01293_client_interactive_vertical_multiline.expect b/tests/queries/0_stateless/01293_client_interactive_vertical_multiline.expect index 5057ec44e8a..07bdbcdac76 100755 --- a/tests/queries/0_stateless/01293_client_interactive_vertical_multiline.expect +++ b/tests/queries/0_stateless/01293_client_interactive_vertical_multiline.expect @@ -27,6 +27,11 @@ expect "Row 1:" expect "1: 1" expect ":) " +send -- "SELECT 1\\G;\r" +expect "Row 1:" +expect "1: 1" +expect ":) " + send -- "SELECT 1\\\r" expect ":-] " send -- ", 2\r" @@ -41,6 +46,14 @@ expect "1: 1" expect "2: 2" expect ":) " +send -- "SELECT 1\\\r" +expect ":-] " +send -- ", 2\\G;\r" +expect "Row 1:" +expect "1: 1" +expect "2: 2" +expect ":) " + send -- "" expect eof @@ -56,6 +69,11 @@ expect "Row 1:" expect "1: 1" expect ":) " +send -- "SELECT 1\\G;\r" +expect "Row 1:" +expect "1: 1" +expect ":) " + send -- "SELECT 1; \r" expect "│ 1 │" expect ":) " @@ -65,6 +83,11 @@ expect "Row 1:" expect "1: 1" expect ":) " +send -- "SELECT 1\\G; \r" +expect "Row 1:" +expect "1: 1" +expect ":) " + send -- "SELECT 1\r" expect ":-] " send -- ";\r" @@ -78,6 +101,13 @@ expect "Row 1:" expect "1: 1" expect ":) " +send -- "SELECT 1\r" +expect ":-] " +send -- "\\G;\r" +expect "Row 1:" +expect "1: 1" +expect ":) " + send -- "SELECT 1\r" expect ":-] " send -- ", 2;\r" @@ -92,5 +122,14 @@ expect "1: 1" expect "2: 2" expect ":) " + +send -- "SELECT 1\r" +expect ":-] " +send -- ", 2\\G;\r" +expect "Row 1:" +expect "1: 1" +expect "2: 2" +expect ":) " + send -- "" expect eof diff --git a/tests/queries/0_stateless/01293_client_interactive_vertical_singleline.expect b/tests/queries/0_stateless/01293_client_interactive_vertical_singleline.expect index 2f871ab46d8..085a1140753 100755 --- a/tests/queries/0_stateless/01293_client_interactive_vertical_singleline.expect +++ b/tests/queries/0_stateless/01293_client_interactive_vertical_singleline.expect @@ -23,6 +23,12 @@ expect "Row 1:" expect "1: 1" expect ":) " + +send -- "SELECT 1\\G;\r" +expect "Row 1:" +expect "1: 1" +expect ":) " + send -- "SELECT 1\\\r" expect ":-] " send -- ", 2\r" @@ -37,5 +43,14 @@ expect "1: 1" expect "2: 2" expect ":) " + +send -- "SELECT 1\\\r" +expect ":-] " +send -- ", 2\\G;\r" +expect "Row 1:" +expect "1: 1" +expect "2: 2" +expect ":) " + send -- "" expect eof From 9ce069c8b548121bf88fb845333a3c438e745593 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 11 Apr 2022 14:56:32 +0200 Subject: [PATCH 062/101] Fix tests --- tests/queries/1_stateful/00161_parallel_parsing_with_names.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/1_stateful/00161_parallel_parsing_with_names.sh b/tests/queries/1_stateful/00161_parallel_parsing_with_names.sh index 18cd8d6caa4..0c930a9fb6e 100755 --- a/tests/queries/1_stateful/00161_parallel_parsing_with_names.sh +++ b/tests/queries/1_stateful/00161_parallel_parsing_with_names.sh @@ -13,7 +13,7 @@ do $CLICKHOUSE_CLIENT -q "CREATE TABLE parsing_with_names(c FixedString(16), a DateTime('Asia/Dubai'), b String) ENGINE=Memory()" echo "$format, false"; - $CLICKHOUSE_CLIENT --max_therads=0 --max_block_size=65505 --output_format_parallel_formatting=false -q \ + $CLICKHOUSE_CLIENT --max_threads=0 --max_block_size=65505 --output_format_parallel_formatting=false -q \ "SELECT URLRegions as d, toTimeZone(ClientEventTime, 'Asia/Dubai') as a, MobilePhoneModel as b, ParamPrice as e, ClientIP6 as c FROM test.hits LIMIT 50000 Format $format" | \ $CLICKHOUSE_CLIENT --max_threads=0 --max_block_size=65505 --input_format_skip_unknown_fields=1 --input_format_parallel_parsing=false -q "INSERT INTO parsing_with_names FORMAT $format" From 5ceb96be87a9454bdd66a218dd99f233366869b1 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 11 Apr 2022 14:56:49 +0200 Subject: [PATCH 063/101] Fix tests --- .../1_stateful/00159_parallel_formatting_json_and_friends.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/1_stateful/00159_parallel_formatting_json_and_friends.sh b/tests/queries/1_stateful/00159_parallel_formatting_json_and_friends.sh index 036eb9aecb3..77dd330099a 100755 --- a/tests/queries/1_stateful/00159_parallel_formatting_json_and_friends.sh +++ b/tests/queries/1_stateful/00159_parallel_formatting_json_and_friends.sh @@ -11,7 +11,7 @@ FORMATS=('JSONEachRow' 'JSONCompactEachRow' 'JSONCompactStringsEachRow' 'JSONCom for format in "${FORMATS[@]}" do echo "$format, false"; - $CLICKHOUSE_CLIENT --max_therads=0 --output_format_parallel_formatting=false -q \ + $CLICKHOUSE_CLIENT --max_threads=0 --output_format_parallel_formatting=false -q \ "SELECT ClientEventTime::DateTime('Asia/Dubai') as a, MobilePhoneModel as b, ClientIP6 as c FROM test.hits ORDER BY a, b, c LIMIT 3000000 Format $format" | md5sum echo "$format, true"; From 991d6d18dfcb65c905957d890c7b2b4ce36c83bb Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 11 Apr 2022 15:08:51 +0200 Subject: [PATCH 064/101] Update 01171_mv_select_insert_isolation_long.sh --- .../0_stateless/01171_mv_select_insert_isolation_long.sh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh index 3de63615bc4..5086515e9eb 100755 --- a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh +++ b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh @@ -11,6 +11,7 @@ set -e $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS src"; $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS dst"; $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS mv"; +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS tmp"; $CLICKHOUSE_CLIENT --query "CREATE TABLE src (n Int8, m Int8, CONSTRAINT c CHECK xxHash32(n+m) % 8 != 0) ENGINE=MergeTree ORDER BY n PARTITION BY 0 < n SETTINGS old_parts_lifetime=0"; $CLICKHOUSE_CLIENT --query "CREATE TABLE dst (nm Int16, CONSTRAINT c CHECK xxHash32(nm) % 8 != 0) ENGINE=MergeTree ORDER BY nm SETTINGS old_parts_lifetime=0"; $CLICKHOUSE_CLIENT --query "CREATE MATERIALIZED VIEW mv TO dst (nm Int16) AS SELECT n*m AS nm FROM src"; @@ -154,3 +155,4 @@ $CLICKHOUSE_CLIENT --query "SELECT count(), sum(nm) FROM mv" $CLICKHOUSE_CLIENT --query "DROP TABLE src"; $CLICKHOUSE_CLIENT --query "DROP TABLE dst"; $CLICKHOUSE_CLIENT --query "DROP TABLE mv"; +$CLICKHOUSE_CLIENT --query "DROP TABLE tmp"; From 83c814f65839c046bd16fad7bac472c6b493605d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 11 Apr 2022 15:25:20 +0200 Subject: [PATCH 065/101] OSX fixes --- src/Common/atomicRename.cpp | 27 +++++++++++++++++---------- 1 file changed, 17 insertions(+), 10 deletions(-) diff --git a/src/Common/atomicRename.cpp b/src/Common/atomicRename.cpp index fb6b0ada40b..c63c0e05899 100644 --- a/src/Common/atomicRename.cpp +++ b/src/Common/atomicRename.cpp @@ -119,40 +119,47 @@ bool supportsAtomicRename() #define RENAME_NOREPLACE RENAME_EXCL #define RENAME_EXCHANGE RENAME_SWAP +namespace DB +{ static bool renameat2(const std::string & old_path, const std::string & new_path, int flags) { - auto fun = dlsym(RTLD_DEFAULT, "renamex_np"); - if (fun == NULL) + using function_type = int (*)(const char * from, const char * to, unsigned int flags); + static function_type fun = reinterpret_cast(dlsym(RTLD_DEFAULT, "renamex_np")); + if (fun == nullptr) return false; if (old_path.empty() || new_path.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot rename {} to {}: path is empty", old_path, new_path); - /// int olddirfd (ignored for absolute oldpath), const char *oldpath, - /// int newdirfd (ignored for absolute newpath), const char *newpath, - /// unsigned int flags - if (0 == (*fun)(old_path.c_str(), AT_FDCWD, new_path.c_str(), flags)) + if (0 == (*fun)(old_path.c_str(), new_path.c_str(), flags)) return true; int errnum = errno; - if ((errnum == ENOTSUP) || (errnum == EINVAL) + if (errnum == ENOTSUP || errnum == EINVAL) return false; if (errnum == EEXIST) throwFromErrno(fmt::format("Cannot rename {} to {} because the second path already exists", old_path, new_path), ErrorCodes::ATOMIC_RENAME_FAIL); if (errnum == ENOENT) throwFromErrno(fmt::format("Paths cannot be exchanged because {} or {} does not exist", old_path, new_path), ErrorCodes::ATOMIC_RENAME_FAIL); - throwFromErrnoWithPath(fmt::format("Cannot rename {} to {}: {}", old_path, new_path, stderror(errnum)), new_path, ErrorCodes::SYSTEM_ERROR); + throwFromErrnoWithPath( + fmt::format("Cannot rename {} to {}: {}", old_path, new_path, strerror(errnum)), new_path, ErrorCodes::SYSTEM_ERROR); } - static bool supportsAtomicRenameImpl() { auto fun = dlsym(RTLD_DEFAULT, "renamex_np"); - return fun != NULL; + return fun != nullptr; } +bool supportsAtomicRename() +{ + static bool supports = supportsAtomicRenameImpl(); + return supports; +} + +} #else From df81d719068e5a094ef22d8a5fe8c0114ee211dc Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 11 Apr 2022 15:28:46 +0200 Subject: [PATCH 066/101] Remove duplicate --- .github/workflows/pull_request.yml | 1 - 1 file changed, 1 deletion(-) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 8072f816cb8..4881bed3f55 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -1052,7 +1052,6 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' CHECK_NAME=ClickHouse build check (actions) REPORTS_PATH=${{runner.temp}}/reports_dir - REPORTS_PATH=${{runner.temp}}/reports_dir TEMP_PATH=${{runner.temp}}/report_check EOF - name: Download json reports From 9a946c0213380bb3cff407b2016f982950cc5b23 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 11 Apr 2022 15:35:13 +0200 Subject: [PATCH 067/101] Add typing --- tests/ci/build_report_check.py | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/tests/ci/build_report_check.py b/tests/ci/build_report_check.py index 7ca958bd745..e47ca3a7811 100644 --- a/tests/ci/build_report_check.py +++ b/tests/ci/build_report_check.py @@ -4,6 +4,8 @@ import json import logging import os import sys +from typing import Dict, List, Tuple + from github import Github from env_helper import ( @@ -44,7 +46,7 @@ class BuildResult: self.with_coverage = with_coverage -def group_by_artifacts(build_urls): +def group_by_artifacts(build_urls: List[str]) -> Dict[str, List[str]]: groups = { "apk": [], "deb": [], @@ -52,7 +54,7 @@ def group_by_artifacts(build_urls): "tgz": [], "rpm": [], "performance": [], - } + } # type: Dict[str, List[str]] for url in build_urls: if url.endswith("performance.tgz"): groups["performance"].append(url) @@ -74,7 +76,9 @@ def group_by_artifacts(build_urls): return groups -def process_report(build_report): +def process_report( + build_report, +) -> Tuple[List[BuildResult], List[List[str]], List[str]]: build_config = build_report["build_config"] build_result = BuildResult( compiler=build_config["compiler"], @@ -98,6 +102,7 @@ def process_report(build_report): build_logs_urls.append(build_report["log_url"]) found_group = True + # No one group of urls is found, a failed report if not found_group: build_results.append(build_result) build_urls.append([""]) From 7293e0164729d41f530cfcae11452005dd0055fa Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 11 Apr 2022 09:47:27 -0400 Subject: [PATCH 068/101] some comments added --- src/Interpreters/ExpressionAnalyzer.cpp | 3 +++ src/Interpreters/QueryNormalizer.cpp | 1 + 2 files changed, 4 insertions(+) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index dd3162aa8a4..47e90e9c745 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1341,12 +1341,15 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendOrderBy(ExpressionActionsChai for (const auto & child : select_query->select()->children) select.insert(child->getAliasOrColumnName()); + /// collect columns required for interpolate expressions - + /// interpolate expression can use any available column auto find_columns = [&step, &select](IAST * function) { auto f_impl = [&step, &select](IAST * fn, auto fi) { if (auto * ident = fn->as()) { + /// exclude columns already in select expression - they are already in required list if (select.count(ident->getColumnName()) == 0) step.addRequiredOutput(ident->getColumnName()); return; diff --git a/src/Interpreters/QueryNormalizer.cpp b/src/Interpreters/QueryNormalizer.cpp index 3c0a965dfa4..28b2195c9c3 100644 --- a/src/Interpreters/QueryNormalizer.cpp +++ b/src/Interpreters/QueryNormalizer.cpp @@ -135,6 +135,7 @@ void QueryNormalizer::visit(ASTTablesInSelectQueryElement & node, const ASTPtr & static bool needVisitChild(const ASTPtr & child) { + /// exclude interpolate elements - they are not subject for normalization and will be processed in filling transform return !(child->as() || child->as() || child->as()); } From 38c80940fdafe6b52a6c2c6951d39b0ccb13b9d4 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 11 Apr 2022 16:25:59 +0200 Subject: [PATCH 069/101] slightly beter fix --- src/Interpreters/MergeTreeTransaction.cpp | 27 +++++++++---------- .../01172_transaction_counters.reference | 3 +-- 2 files changed, 13 insertions(+), 17 deletions(-) diff --git a/src/Interpreters/MergeTreeTransaction.cpp b/src/Interpreters/MergeTreeTransaction.cpp index 7c1feb579e2..c0d3cdfeb62 100644 --- a/src/Interpreters/MergeTreeTransaction.cpp +++ b/src/Interpreters/MergeTreeTransaction.cpp @@ -250,21 +250,6 @@ bool MergeTreeTransaction::rollback() noexcept /// Discard changes in active parts set /// Remove parts that were created, restore parts that were removed (except parts that were created by this transaction too) - for (const auto & part : parts_to_remove) - { - if (part->version.isRemovalTIDLocked()) - { - /// Don't need to remove part from working set if it was created and removed by this transaction - assert(part->version.removal_tid_lock == tid.getHash()); - continue; - } - /// FIXME do not lock removal_tid when rolling back part creation, it's ugly - const_cast(part->storage).removePartsFromWorkingSet(NO_TRANSACTION_RAW, {part}, true); - } - - for (const auto & part : parts_to_activate) - if (part->version.getCreationTID() != tid) - const_cast(part->storage).restoreAndActivatePart(part); /// Kind of optimization: cleanup thread can remove these parts immediately for (const auto & part : parts_to_remove) @@ -274,6 +259,18 @@ bool MergeTreeTransaction::rollback() noexcept part->appendCSNToVersionMetadata(VersionMetadata::CREATION); } + for (const auto & part : parts_to_remove) + { + /// NOTE It's possible that part is already removed from working set in the same transaction + /// (or, even worse, in a separate non-transactional query with PrehistoricTID), + /// but it's not a problem: removePartsFromWorkingSet(...) will do nothing in this case. + const_cast(part->storage).removePartsFromWorkingSet(NO_TRANSACTION_RAW, {part}, true); + } + + for (const auto & part : parts_to_activate) + if (part->version.getCreationTID() != tid) + const_cast(part->storage).restoreAndActivatePart(part); + for (const auto & part : parts_to_activate) { /// Clear removal_tid from version metadata file, so we will not need to distinguish TIDs that were not committed diff --git a/tests/queries/0_stateless/01172_transaction_counters.reference b/tests/queries/0_stateless/01172_transaction_counters.reference index 1aabf8a2a38..3a167e76817 100644 --- a/tests/queries/0_stateless/01172_transaction_counters.reference +++ b/tests/queries/0_stateless/01172_transaction_counters.reference @@ -6,7 +6,7 @@ 3 all_1_1_0 0 3 all_3_3_0 1 4 all_1_1_0 1 (0,0,'00000000-0000-0000-0000-000000000000') 0 -4 all_2_2_0 18446744073709551615 (1,1,'00000000-0000-0000-0000-000000000000') 0 +4 all_2_2_0 18446744073709551615 (0,0,'00000000-0000-0000-0000-000000000000') 0 4 all_3_3_0 0 (0,0,'00000000-0000-0000-0000-000000000000') 0 5 1 6 all_1_1_0 0 @@ -19,7 +19,6 @@ 1 1 AddPart 1 1 1 1 all_1_1_0 2 1 Begin 1 1 1 1 2 1 AddPart 1 1 1 1 all_2_2_0 -1 1 LockPart 1 1 1 1 all_2_2_0 2 1 Rollback 1 1 1 1 3 1 Begin 1 1 1 1 3 1 AddPart 1 1 1 1 all_3_3_0 From f7c17a8e866ba859979ccd7d1810654cdabd2364 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Mon, 11 Apr 2022 18:03:19 +0300 Subject: [PATCH 070/101] Update stress --- docker/test/stress/stress | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docker/test/stress/stress b/docker/test/stress/stress index 10c6088af75..9f3a51c1b6b 100755 --- a/docker/test/stress/stress +++ b/docker/test/stress/stress @@ -215,6 +215,8 @@ if __name__ == "__main__": "--client-option", "max_untracked_memory=1Gi", "--client-option", "max_memory_usage_for_user=0", "--client-option", "memory_profiler_step=1Gi", + # Use system database to avoid CREATE/DROP DATABASE queries + "--database=system", "--hung-check", "00001_select_1" ]) From 11897d56b911d1a98ecbc5625f24e0a5879210cb Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 11 Apr 2022 17:03:27 +0200 Subject: [PATCH 071/101] Upload ccache for the first run in PR --- tests/ci/ccache_utils.py | 7 ++++++- tests/ci/fast_test_check.py | 8 +++++++- 2 files changed, 13 insertions(+), 2 deletions(-) diff --git a/tests/ci/ccache_utils.py b/tests/ci/ccache_utils.py index 7b0b0f01aa3..bd155b02cb4 100644 --- a/tests/ci/ccache_utils.py +++ b/tests/ci/ccache_utils.py @@ -57,10 +57,12 @@ def dowload_file_with_progress(url, path): def get_ccache_if_not_exists( path_to_ccache_dir, s3_helper, current_pr_number, temp_path -): +) -> int: + """returns: number of PR for downloaded PR. -1 if ccache not found""" ccache_name = os.path.basename(path_to_ccache_dir) cache_found = False prs_to_check = [current_pr_number] + ccache_pr = -1 if current_pr_number != 0: prs_to_check.append(0) for pr_number in prs_to_check: @@ -87,6 +89,7 @@ def get_ccache_if_not_exists( decompress_fast(compressed_cache, path_to_decompress) logging.info("Files on path %s", os.listdir(path_to_decompress)) cache_found = True + ccache_pr = pr_number break if cache_found: break @@ -98,6 +101,8 @@ def get_ccache_if_not_exists( else: logging.info("ccache downloaded") + return ccache_pr + def upload_ccache(path_to_ccache_dir, s3_helper, current_pr_number, temp_path): logging.info("Uploading cache %s for pr %s", path_to_ccache_dir, current_pr_number) diff --git a/tests/ci/fast_test_check.py b/tests/ci/fast_test_check.py index 41f1f9a88e8..f4a695eac1a 100644 --- a/tests/ci/fast_test_check.py +++ b/tests/ci/fast_test_check.py @@ -113,7 +113,10 @@ if __name__ == "__main__": cache_path = os.path.join(caches_path, "fasttest") logging.info("Will try to fetch cache for our build") - get_ccache_if_not_exists(cache_path, s3_helper, pr_info.number, temp_path) + ccache_for_pr = get_ccache_if_not_exists( + cache_path, s3_helper, pr_info.number, temp_path + ) + upload_master_ccache = ccache_for_pr in (-1, 0) if not os.path.exists(cache_path): logging.info("cache was not fetched, will create empty dir") @@ -179,6 +182,9 @@ if __name__ == "__main__": logging.info("Will upload cache") upload_ccache(cache_path, s3_helper, pr_info.number, temp_path) + if upload_master_ccache: + logging.info("Will upload a fallback cache for master") + upload_ccache(cache_path, s3_helper, 0, temp_path) ch_helper = ClickHouseHelper() mark_flaky_tests(ch_helper, NAME, test_results) From 2588f8011f84e28ce76999f5392ce6c694dba737 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 11 Apr 2022 11:13:29 -0400 Subject: [PATCH 072/101] comment fix --- src/Interpreters/ExpressionAnalyzer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 47e90e9c745..5e5931713e0 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1349,7 +1349,7 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendOrderBy(ExpressionActionsChai { if (auto * ident = fn->as()) { - /// exclude columns already in select expression - they are already in required list + /// exclude columns from select expression - they are already available if (select.count(ident->getColumnName()) == 0) step.addRequiredOutput(ident->getColumnName()); return; From faa26b4377a1f5f1d27b3c7178918ab51037dcae Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 11 Apr 2022 15:36:18 +0200 Subject: [PATCH 073/101] Improve report and variables naming --- tests/ci/build_report_check.py | 91 ++++++++++++++++++---------------- 1 file changed, 48 insertions(+), 43 deletions(-) diff --git a/tests/ci/build_report_check.py b/tests/ci/build_report_check.py index e47ca3a7811..44cc45390a5 100644 --- a/tests/ci/build_report_check.py +++ b/tests/ci/build_report_check.py @@ -115,7 +115,7 @@ def get_build_name_from_file_name(file_name): return file_name.replace("build_urls_", "").replace(".json", "") -if __name__ == "__main__": +def main(): logging.basicConfig(level=logging.INFO) reports_path = REPORTS_PATH temp_path = TEMP_PATH @@ -125,7 +125,7 @@ if __name__ == "__main__": os.makedirs(temp_path) build_check_name = sys.argv[1] - reports_length = int(sys.argv[2]) if len(sys.argv) > 2 else 0 + required_builds = int(sys.argv[2]) if len(sys.argv) > 2 else 0 gh = Github(get_best_robot_token()) pr_info = PRInfo() @@ -134,19 +134,20 @@ if __name__ == "__main__": logging.info("Check is already finished according to github status, exiting") sys.exit(0) - reports_order = CI_CONFIG["builds_report_config"][build_check_name] - logging.info("My reports list %s", reports_order) + builds_for_check = CI_CONFIG["builds_report_config"][build_check_name] + logging.info("My reports list %s", builds_for_check) + required_builds = required_builds or len(builds_for_check) - build_reports_map = {} - for root, dirs, files in os.walk(reports_path): + # Collect reports from json artifacts + builds_report_map = {} + for root, _, files in os.walk(reports_path): for f in files: if f.startswith("build_urls_") and f.endswith(".json"): logging.info("Found build report json %s", f) build_name = get_build_name_from_file_name(f) - if build_name in reports_order: + if build_name in builds_for_check: with open(os.path.join(root, f), "rb") as file_handler: - build_report = json.load(file_handler) - build_reports_map[build_name] = build_report + builds_report_map[build_name] = json.load(file_handler) else: logging.info( "Skipping report %s for build %s, it's not in our reports list", @@ -154,44 +155,45 @@ if __name__ == "__main__": build_name, ) - reports_length = reports_length or len(reports_order) - some_builds_are_missing = len(build_reports_map) < reports_length - - if some_builds_are_missing: - logging.info( - "Expected to get %s build results, got %s", - len(reports_order), - len(build_reports_map), - ) - else: - logging.info("Got exactly %s builds", len(build_reports_map)) - + # Sort reports by config order build_reports = [ - build_reports_map[build_name] - for build_name in reports_order - if build_name in build_reports_map + builds_report_map[build_name] + for build_name in builds_for_check + if build_name in builds_report_map ] - build_results = [] - build_artifacts = [] + some_builds_are_missing = len(build_reports) < required_builds + if some_builds_are_missing: + logging.warning( + "Expected to get %s build results, got only %s", + required_builds, + len(build_reports), + ) + else: + logging.info("Got exactly %s builds", len(builds_report_map)) + + # Group build artifacts by groups + build_results = [] # type: List[BuildResult] + build_artifacts = [] # build_logs = [] for build_report in build_reports: build_result, build_artifacts_url, build_logs_url = process_report(build_report) - logging.info("Got %s result for report", len(build_result)) - build_results += build_result - build_artifacts += build_artifacts_url - build_logs += build_logs_url + logging.info( + "Got %s artifact groups for build report report", len(build_result) + ) + build_results.extend(build_result) + build_artifacts.extend(build_artifacts_url) + build_logs.extend(build_logs_url) - logging.info("Totally got %s results", len(build_results)) - if len(build_results) == 0: - logging.info("No builds, failing check") + total_groups = len(build_results) + logging.info("Totally got %s artifact groups", total_groups) + if total_groups == 0: + logging.error("No success builds, failing check") sys.exit(1) s3_helper = S3Helper("https://s3.amazonaws.com") - pr_info = PRInfo() - branch_url = f"{GITHUB_SERVER_URL}/{GITHUB_REPOSITORY}/commits/master" branch_name = "master" if pr_info.number != 0: @@ -224,9 +226,10 @@ if __name__ == "__main__": report_path, s3_path_prefix + "/report.html" ) logging.info("Report url %s", url) + print(f"::notice ::Report url: {url}") - total_builds = len(build_results) - ok_builds = 0 + # Prepare a commit status + ok_groups = 0 summary_status = "success" for build_result in build_results: if build_result.status == "failure" and summary_status != "error": @@ -235,18 +238,16 @@ if __name__ == "__main__": summary_status = "error" if build_result.status == "success": - ok_builds += 1 + ok_groups += 1 - if ok_builds == 0 or some_builds_are_missing: + if ok_groups == 0 or some_builds_are_missing: summary_status = "error" addition = "" if some_builds_are_missing: - addition = f"({len(build_reports_map)} < {reports_length})" + addition = f"({len(build_reports)} of {required_builds} builds are OK)" - description = f"{ok_builds}/{total_builds} builds are OK {addition}" - - print(f"::notice ::Report url: {url}") + description = f"{ok_groups}/{total_groups} artifact groups are OK {addition}" commit = get_commit(gh, pr_info.sha) commit.create_status( @@ -258,3 +259,7 @@ if __name__ == "__main__": if summary_status == "error": sys.exit(1) + + +if __name__ == "__main__": + main() From fc62e95d76de675099c4bed11c975526c0a8db19 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 11 Apr 2022 22:24:45 +0200 Subject: [PATCH 074/101] Fix --- src/Storages/System/StorageSystemRemoteDataPaths.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Storages/System/StorageSystemRemoteDataPaths.cpp b/src/Storages/System/StorageSystemRemoteDataPaths.cpp index 410d1ae6dd4..bf71f4f1f86 100644 --- a/src/Storages/System/StorageSystemRemoteDataPaths.cpp +++ b/src/Storages/System/StorageSystemRemoteDataPaths.cpp @@ -53,6 +53,8 @@ Pipe StorageSystemRemoteDataPaths::read( { std::vector remote_paths_by_local_path; disk->getRemotePathsRecursive("store", remote_paths_by_local_path); + disk->getRemotePathsRecursive("data", remote_paths_by_local_path); + disk->getRemotePathsRecursive("metadata", remote_paths_by_local_path); FileCachePtr cache; auto cache_base_path = disk->getCacheBasePath(); From 51dd39b42fee59079c2b09e235ba601cd000222c Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 11 Apr 2022 22:46:51 +0200 Subject: [PATCH 075/101] Update StorageSystemRemoteDataPaths.cpp --- src/Storages/System/StorageSystemRemoteDataPaths.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/System/StorageSystemRemoteDataPaths.cpp b/src/Storages/System/StorageSystemRemoteDataPaths.cpp index bf71f4f1f86..dbce4c25773 100644 --- a/src/Storages/System/StorageSystemRemoteDataPaths.cpp +++ b/src/Storages/System/StorageSystemRemoteDataPaths.cpp @@ -54,7 +54,6 @@ Pipe StorageSystemRemoteDataPaths::read( std::vector remote_paths_by_local_path; disk->getRemotePathsRecursive("store", remote_paths_by_local_path); disk->getRemotePathsRecursive("data", remote_paths_by_local_path); - disk->getRemotePathsRecursive("metadata", remote_paths_by_local_path); FileCachePtr cache; auto cache_base_path = disk->getCacheBasePath(); From 588b58243723ff7bbe66d5d28047bee44b2aa6a9 Mon Sep 17 00:00:00 2001 From: yuuch Date: Tue, 12 Apr 2022 11:01:20 +0800 Subject: [PATCH 076/101] detail about the input --- src/Client/ClientBase.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index b79a67b7526..dcebe01c5b9 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1862,7 +1862,10 @@ void ClientBase::runInteractive() has_vertical_output_suffix = false; if (input.ends_with("\\G") || input.ends_with("\\G;")) { - input.resize(input.size() - 2); + if (input.ends_with("\\G")) + input.resize(input.size() - 2); + if (input.ends_with("\\G;")) + input.resize(input.size() - 3); has_vertical_output_suffix = true; } From 9d31c44d39b16b56864bd7db935d43fcb82ac8dc Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 12 Apr 2022 12:01:51 +0800 Subject: [PATCH 077/101] fix bug of read buffer from hdfs --- src/Storages/HDFS/ReadBufferFromHDFS.cpp | 24 ++++++++++++------------ 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/src/Storages/HDFS/ReadBufferFromHDFS.cpp b/src/Storages/HDFS/ReadBufferFromHDFS.cpp index 1bafa49e55b..af4b324cd77 100644 --- a/src/Storages/HDFS/ReadBufferFromHDFS.cpp +++ b/src/Storages/HDFS/ReadBufferFromHDFS.cpp @@ -30,7 +30,7 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl : public BufferWithOwnMemory {})", offset, read_until_position - 1); + if (read_until_position < file_offset) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to read beyond right offset ({} > {})", file_offset, read_until_position - 1); - num_bytes_to_read = read_until_position - offset; + num_bytes_to_read = read_until_position - file_offset; } else { @@ -94,28 +94,28 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl : public BufferWithOwnMemorynext(); if (result) - BufferBase::set(impl->buffer().begin(), impl->buffer().size(), impl->offset); /// use the buffer returned by `impl` + BufferBase::set(impl->buffer().begin(), impl->buffer().size(), impl->offset()); /// use the buffer returned by `impl` return result; } From 6b07105de65f376d723025a85b178e30bf92ac76 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Mon, 11 Apr 2022 22:38:23 -0700 Subject: [PATCH 078/101] fix typo in cmake message --- cmake/strip_binary.cmake | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmake/strip_binary.cmake b/cmake/strip_binary.cmake index 1f24790a159..2d6a3888503 100644 --- a/cmake/strip_binary.cmake +++ b/cmake/strip_binary.cmake @@ -46,7 +46,7 @@ macro(clickhouse_make_empty_debug_info_for_nfpm) add_custom_command(TARGET ${EMPTY_DEBUG_TARGET} POST_BUILD COMMAND mkdir -p "${EMPTY_DEBUG_DESTINATION_DIR}/lib/debug" COMMAND touch "${EMPTY_DEBUG_DESTINATION_DIR}/lib/debug/${EMPTY_DEBUG_TARGET}.debug" - COMMENT "Addiding empty debug info for NFPM" VERBATIM + COMMENT "Adding empty debug info for NFPM" VERBATIM ) install(FILES "${EMPTY_DEBUG_DESTINATION_DIR}/lib/debug/${EMPTY_DEBUG_TARGET}.debug" DESTINATION "${CMAKE_INSTALL_LIBDIR}/debug/${CMAKE_INSTALL_FULL_BINDIR}" COMPONENT clickhouse) From 0ae820550d0355b00ed8b1fc8bf1447b2022fddc Mon Sep 17 00:00:00 2001 From: zhanglistar Date: Tue, 12 Apr 2022 14:30:53 +0800 Subject: [PATCH 079/101] fix ut error sometimes --- src/Coordination/tests/gtest_coordination.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 7dfd451e111..a64fde5ad05 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -407,7 +407,7 @@ TEST_P(CoordinationTest, ChangelogTestCompaction) EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); changelog.compact(6); - std::this_thread::sleep_for(std::chrono::microseconds(200)); + std::this_thread::sleep_for(std::chrono::microseconds(1000)); EXPECT_FALSE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); @@ -1469,7 +1469,7 @@ TEST_P(CoordinationTest, TestRotateIntervalChanges) } changelog_2.compact(105); - std::this_thread::sleep_for(std::chrono::microseconds(200)); + std::this_thread::sleep_for(std::chrono::microseconds(1000)); EXPECT_FALSE(fs::exists("./logs/changelog_1_100.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_101_110.bin" + params.extension)); @@ -1489,7 +1489,7 @@ TEST_P(CoordinationTest, TestRotateIntervalChanges) } changelog_3.compact(125); - std::this_thread::sleep_for(std::chrono::microseconds(200)); + std::this_thread::sleep_for(std::chrono::microseconds(1000)); EXPECT_FALSE(fs::exists("./logs/changelog_101_110.bin" + params.extension)); EXPECT_FALSE(fs::exists("./logs/changelog_111_117.bin" + params.extension)); EXPECT_FALSE(fs::exists("./logs/changelog_118_124.bin" + params.extension)); From abf8c3c5ad8dc302864e9e9cb5cf57fc709cc5a3 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 12 Apr 2022 09:51:26 +0200 Subject: [PATCH 080/101] Fix and improve cancel-rerun lambda - Fix rerun for DocsCheck - Additional check for workflow head sha1 - Fix workflow debug info - Fix event_data parsing - Always print debug info --- .../cancel_and_rerun_workflow_lambda/app.py | 23 ++++++++++++------- 1 file changed, 15 insertions(+), 8 deletions(-) diff --git a/tests/ci/cancel_and_rerun_workflow_lambda/app.py b/tests/ci/cancel_and_rerun_workflow_lambda/app.py index 8deed464012..4cce1e5024a 100644 --- a/tests/ci/cancel_and_rerun_workflow_lambda/app.py +++ b/tests/ci/cancel_and_rerun_workflow_lambda/app.py @@ -1,7 +1,7 @@ #!/usr/bin/env python3 from collections import namedtuple -from typing import Any, Dict +from typing import Any, Dict, List import json import time @@ -11,7 +11,7 @@ import boto3 # type: ignore NEED_RERUN_OR_CANCELL_WORKFLOWS = { "PullRequestCI", - "Docs", + "DocsCheck", "DocsRelease", "BackportPR", } @@ -93,7 +93,9 @@ WorkflowDescription = namedtuple( def get_workflows_description_for_pull_request(pull_request_event): + head_repo = pull_request_event["head"]["repo"]["full_name"] head_branch = pull_request_event["head"]["ref"] + head_sha = pull_request_event["head"]["sha"] print("PR", pull_request_event["number"], "has head ref", head_branch) workflows_data = [] workflows = _exec_get_with_retry( @@ -111,17 +113,23 @@ def get_workflows_description_for_pull_request(pull_request_event): print("Too many workflows found") break + DEBUG_INFO["workflows"] = [] # type: List[Dict[str, str]] workflow_descriptions = [] for workflow in workflows_data: - DEBUG_INFO["workflow"] = workflow # Some time workflow["head_repository"]["full_name"] is None if workflow["head_repository"] is None: continue + DEBUG_INFO["workflows"].append( + { + "full_name": workflow["head_repository"]["full_name"], + "name": workflow["name"], + } + ) # unfortunately we cannot filter workflows from forks in request to API # so doing it manually if ( - workflow["head_repository"]["full_name"] - == pull_request_event["head"]["repo"]["full_name"] + workflow["head_sha"] == head_sha + and workflow["head_repository"]["full_name"] == head_repo and workflow["name"] in NEED_RERUN_OR_CANCELL_WORKFLOWS ): workflow_descriptions.append( @@ -170,7 +178,7 @@ def exec_workflow_url(urls_to_cancel, token): def main(event): token = get_token_from_aws() DEBUG_INFO["event_body"] = event["body"] - event_data = event["body"] + event_data = json.loads(event["body"]) print("Got event for PR", event_data["number"]) action = event_data["action"] @@ -220,7 +228,6 @@ def main(event): def handler(event, _): try: main(event) - except Exception: + finally: for name, value in DEBUG_INFO.items(): print(f"Value of {name}: ", value) - raise From 30a9fce67ac46d11fcf2f5a0611bded1688abbd6 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 12 Apr 2022 10:59:44 +0200 Subject: [PATCH 081/101] Rebuild the least docker layer --- tests/ci/cancel_and_rerun_workflow_lambda/Dockerfile | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/ci/cancel_and_rerun_workflow_lambda/Dockerfile b/tests/ci/cancel_and_rerun_workflow_lambda/Dockerfile index f53be71a893..0d50224c51d 100644 --- a/tests/ci/cancel_and_rerun_workflow_lambda/Dockerfile +++ b/tests/ci/cancel_and_rerun_workflow_lambda/Dockerfile @@ -1,13 +1,13 @@ FROM public.ecr.aws/lambda/python:3.9 -# Copy function code -COPY app.py ${LAMBDA_TASK_ROOT} - # Install the function's dependencies using file requirements.txt # from your project folder. COPY requirements.txt . RUN pip3 install -r requirements.txt --target "${LAMBDA_TASK_ROOT}" +# Copy function code +COPY app.py ${LAMBDA_TASK_ROOT} + # Set the CMD to your handler (could also be done as a parameter override outside of the Dockerfile) CMD [ "app.handler" ] From 48f9b61bdb8e4051eba6d35eec836857ef1d187c Mon Sep 17 00:00:00 2001 From: tavplubix Date: Tue, 12 Apr 2022 12:22:03 +0300 Subject: [PATCH 082/101] Update stress --- docker/test/stress/stress | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/docker/test/stress/stress b/docker/test/stress/stress index 9f3a51c1b6b..5527ff74224 100755 --- a/docker/test/stress/stress +++ b/docker/test/stress/stress @@ -198,7 +198,11 @@ if __name__ == "__main__": logging.info("Logs compressed") if args.hung_check: - have_long_running_queries = prepare_for_hung_check(args.drop_databases) + try: + have_long_running_queries = prepare_for_hung_check(args.drop_databases) + except Exception as ex: + have_long_running_queries = True + print("Failed to prepare for hung check", ex) logging.info("Checking if some queries hung") cmd = ' '.join([args.test_cmd, # Do not track memory allocations up to 1Gi, From ccb45ae943c31a4758a2b8f6ebe2ad83509791b6 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Tue, 12 Apr 2022 12:42:45 +0300 Subject: [PATCH 083/101] Update stress --- docker/test/stress/stress | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/stress/stress b/docker/test/stress/stress index 5527ff74224..e42587e9a4d 100755 --- a/docker/test/stress/stress +++ b/docker/test/stress/stress @@ -131,7 +131,7 @@ def prepare_for_hung_check(drop_databases): Popen(command, shell=True) break except Exception as ex: - print("Failed to SHOW or DROP databasese, will retry", ex) + logging.error("Failed to SHOW or DROP databasese, will retry %s", str(ex)) time.sleep(i) else: raise Exception("Cannot drop databases after stress tests. Probably server consumed too much memory and cannot execute simple queries") @@ -202,7 +202,7 @@ if __name__ == "__main__": have_long_running_queries = prepare_for_hung_check(args.drop_databases) except Exception as ex: have_long_running_queries = True - print("Failed to prepare for hung check", ex) + logging.error("Failed to prepare for hung check %s", str(ex)) logging.info("Checking if some queries hung") cmd = ' '.join([args.test_cmd, # Do not track memory allocations up to 1Gi, From 6e4787560bc493b14f5ee5f2dd61db743c8a87f6 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 12 Apr 2022 11:39:21 +0200 Subject: [PATCH 084/101] fix another issue --- src/Interpreters/TransactionLog.cpp | 9 ++------- 1 file changed, 2 insertions(+), 7 deletions(-) diff --git a/src/Interpreters/TransactionLog.cpp b/src/Interpreters/TransactionLog.cpp index 393a8aa848b..7dd020d9a7a 100644 --- a/src/Interpreters/TransactionLog.cpp +++ b/src/Interpreters/TransactionLog.cpp @@ -212,7 +212,7 @@ void TransactionLog::runUpdatingThread() if (stop_flag.load()) return; - if (!zookeeper) + if (getZooKeeper()->expired()) { auto new_zookeeper = global_context->getZooKeeper(); std::lock_guard lock{mutex}; @@ -222,16 +222,11 @@ void TransactionLog::runUpdatingThread() loadNewEntries(); removeOldEntries(); } - catch (const Coordination::Exception & e) + catch (const Coordination::Exception &) { tryLogCurrentException(log); /// TODO better backoff std::this_thread::sleep_for(std::chrono::milliseconds(1000)); - if (Coordination::isHardwareError(e.code)) - { - std::lock_guard lock{mutex}; - zookeeper.reset(); - } log_updated_event->set(); } catch (...) From 8f822af3580c4d170a10e7d62687fb6b1c8d8a33 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 12 Apr 2022 11:47:22 +0200 Subject: [PATCH 085/101] Fix mongodb test with new cert --- tests/integration/helpers/mongo_cert.pem | 89 +++++++++++++----------- 1 file changed, 47 insertions(+), 42 deletions(-) diff --git a/tests/integration/helpers/mongo_cert.pem b/tests/integration/helpers/mongo_cert.pem index 9e18b1d4469..b9c3b83e473 100644 --- a/tests/integration/helpers/mongo_cert.pem +++ b/tests/integration/helpers/mongo_cert.pem @@ -1,44 +1,49 @@ ------BEGIN RSA PRIVATE KEY----- -MIIEpAIBAAKCAQEAtz2fpa8hyUff8u8jYlh20HbkOO8hQi64Ke2Prack2Br0lhOr -1MI6I8nVk5iDrt+7ix2Cnt+2aZKb6HJv0CG1V25yWg+jgsXeIT1KHTJf8rTmYxhb -t+ye+S1Z0h/Rt+xqSd9XXfzOLPGHYfyx6ZQ4AumO/HoEFD4IH/qiREjwtOfRXuhz -CohqtUTyYR7pJmZqBSuGac461WVRisnjfKRxeVa3itc84/RgktgYej2x4PQBFk13 -xAXKrWmHkwdgWklTuuK8Gtoqz65Y4/J9CSl+Bd08QDdRnaVvq1u1eNTZg1BVyeRv -jFYBMSathKASrng5nK66Fdilw6tO/9khaP0SDQIDAQABAoIBAAm/5qGrKtIJ1/mW -Dbzq1g+Lc+MvngZmc/gPIsjrjsNM09y0WT0txGgpEgsTX1ZLoy/otw16+7qsSU1Z -4WcilAJ95umx0VJg8suz9iCNkJtaUrPNFPw5Q9AgQJo0hTUTCCi8EGr4y4OKqlhl -WJYEA+LryGbYmyT0k/wXmtClTOFjKS09mK4deQ1DqbBxayR9MUZgRJzEODA8eGXs -Rc6fJUenMVNMzIVLgpossRtKImoZNcf5UtCKL3HECunndQeMu4zuqLMU+EzL1F/o -iHKF7v3CVmsK0OxNJfOfT0abN3XaJttFwTJyghQjgP8OX1IKjlj3vo9xwEDfVUEf -GVIER0UCgYEA2j+kjaT3Dw2lNlBEsA8uvVlLJHBc9JBtMGduGYsIEgsL/iStqXv4 -xoA9N3CwkN/rrQpDfi/16DMXAAYdjGulPwiMNFBY22TCzhtPC2mAnfaSForxwZCs -lwc3KkIloo3N5XvN78AuZf8ewiS+bOEj+HHHqqSb1+u/csuaXO9neesCgYEA1u/I -Mlt/pxJkH+c3yOskwCh/CNhq9szi8G9LXROIQ58BT2ydJSEPpt7AhUTtQGimQQTW -KLiffJSkjuVaFckR1GjCoAmFGYw9wUb+TmFNScz5pJ2dXse8aBysAMIQfEIcRAEa -gKnkLBH6nw3+/Hm3xwoBc35t8Pa2ek7LsWDfbecCgYBhilQW4gVw+t49uf4Y2ZBA -G+pTbMx+mRXTrkYssFB5D+raOLZMqxVyUdoKLxkahpkkCxRDD1hN4JeE8Ta/jVSb -KUzQDKDJ3OybhOT86rgK4SpFXO/TXL9l+FmVT17WmZ3N1Fkjr7aM60pp5lYc/zo+ -TUu5XjwwcjJsMcbZhj2u5QKBgQCDNuUn4PYAP9kCJPzIWs0XxmEvPDeorZIJqFgA -3XC9n2+EVlFlHlbYz3oGofqY7Io6fUJkn7k1q+T+G4QwcozA+KeAXe90lkoJGVcc -8IfnewwYc+RjvVoG0SIsYE0CHrX0yhus2oqiYON4gGnfJkuMZk5WfKOPjH4AEuSF -SBd+lwKBgQCHG/DA6u2mYmezPF9hebWFoyAVSr2PDXDhu8cNNHCpx9GewJXhuK/P -tW8mazHzUuJKBvmaUXDIXFh4K6FNhjH16p5jR1w3hsPE7NEZhjfVRaUYPmBqaOYR -jp8H+Sh5g4Rwbtfp6Qhu6UAKi/y6Vozs5GkJtSiNrjNDVrD+sGGrXA== ------END RSA PRIVATE KEY----- +-----BEGIN PRIVATE KEY----- +MIIEvQIBADANBgkqhkiG9w0BAQEFAASCBKcwggSjAgEAAoIBAQC3uaPiZMfjPBBE +yDEYJsJIoriu0SaC80uTmPM7bFpnOOXOBvbT4wD2q+uVaLQifKtPTgZAkP5Y3rX8 +S5TOzaLsNp68S1Ja/EzxQUolOSgb4A948TTiUTrTjfMxsPRhmxXTjozWV8CFtL9P +Lg6H+55oyQOJedWYe1kSWRJQayXSweBK5qjOPi2qDF/xdFRQuMivpBUar/b/E9GQ +RKpIaoqMYsl/WF/tReb4N658UxkVlFdR8s48UoA9LfJLMPr4N+QDTfvtcT2bYlpT +4a9b6IXa9BQKCw3AKfTqEPO1XunH//iLNkt1bLtqgZNyT/tY0tLY3EKMXIDuRBVn +KCbfVJ1RAgMBAAECggEAJFCjXiqBgB7tMEtJuPZgTK8tRhC9RgEFHUWMPmCqdeC/ +O7wQqc0i8Z8Fz+CESpTN370Sa0y9mZ9b5WSjI0VuQLaDJcDVpHpeUwmOuFDV5ryh +EkzLITjhIdPbECVkCK7433o7yFpMCaGydtopsSNBKoEhG9ljKOKotoG4pwCm10N5 +K9Qepj82OjRhLkpmuiMFb4/vvOm5dglYmkq5+n/fdUYFtrYr3NvMSCTlietPHDgV +Wb3strvk1g9ARWfa2j7Q6moF2sbyob9zVLoRiD9VgmNB60v7QAJxDctVkbOoDgKp +uN2fkxTHwlOPAO6Zhgnie11jnZr1711TFxmEfMkSKQKBgQDqpB8m0hSJsWLKWxQK +yx+5Xgs+Cr8gb0AYHJQ87obj2XqwXLpBSMrkzTn6vIGRv+NMSfiM/755RUm5aJPY +om+7F68JEIL26ZA7bIfjHhV5o9fvpo+6N6cJyR08Q/KkF8Tej9K4qQec0W/jtKeZ +KAJ1k7/BBuN82iTtEJ3GWBaaRwKBgQDIcwQrGlyyXqnBK25gl/E1Ng+V3p/2sy98 +1BpEshxen4KorHEXCJArydELtvK/ll6agil6QebrJN5dtYOOgvcDTu1mQjdUPN3C +VXpSQ0L8XxfyTNYQTWON9wJGL1pzlTiyHvlSrQFsFWMUoxrqndWIIRtrXjap1npp +HDrcqy2/pwKBgB5fHhUlTjlAd7wfq+l1v2Z8ENJ4C6NEIzS7xkhYy6cEiIf5iLZY +mMKi+eVFrzPRdbdzP7Poipwh5tgT/EcnR3UdLK/srjcNpni6pKA2TatQFOxVT/dX +qsxudtVNKkQpO3dfgHQclPqsdWIxCRye/CqB9Gkk3h9UEUGKTBHXZx2TAoGAF0tG +cLvfidr2Xzxs10zQ+x4NMZ1teX3ZRuhfJRyNr3FZ/cAMZGDaYDxTzsiz7Q/Mbqgx +qcN+0lS2gq1VXHpbukaxz/Bh/agVHUBRtr2aSznBzqafOcXEi/roiL94A3aT4B85 +WiJAyA60NPG/bwRojClMxm1sbNA/6XceYAaEioECgYEA3m88G3UwizfJAsfT5H5K +3HXNYzQ1XGrA8shI0kxeqfNP5qmTfH5q/K2VMWeShT3F/9Ytgc+H8c9XP1wKq7Zl +6AtmdDOeLzHkgwVK0p20/Wh2Qjw4ikJLdM+y8wnfMiwCXWQxoh1X905EwNtyBc2Z +9S3G5CXldFHC4NGdx0vetiE= +-----END PRIVATE KEY----- -----BEGIN CERTIFICATE----- -MIICqDCCAZACFBdaMnuT0pWhmrh05UT3HXJ+kI0yMA0GCSqGSIb3DQEBCwUAMA0x -CzAJBgNVBAMMAmNhMB4XDTIxMDQwNjE3MDQxNVoXDTIyMDQwNjE3MDQxNVowFDES -MBAGA1UEAwwJbG9jYWxob3N0MIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8AMIIBCgKC -AQEAtz2fpa8hyUff8u8jYlh20HbkOO8hQi64Ke2Prack2Br0lhOr1MI6I8nVk5iD -rt+7ix2Cnt+2aZKb6HJv0CG1V25yWg+jgsXeIT1KHTJf8rTmYxhbt+ye+S1Z0h/R -t+xqSd9XXfzOLPGHYfyx6ZQ4AumO/HoEFD4IH/qiREjwtOfRXuhzCohqtUTyYR7p -JmZqBSuGac461WVRisnjfKRxeVa3itc84/RgktgYej2x4PQBFk13xAXKrWmHkwdg -WklTuuK8Gtoqz65Y4/J9CSl+Bd08QDdRnaVvq1u1eNTZg1BVyeRvjFYBMSathKAS -rng5nK66Fdilw6tO/9khaP0SDQIDAQABMA0GCSqGSIb3DQEBCwUAA4IBAQAct2If -isMLHIqyL9GjY4b0xcxF4svFU/DUwNanStmoFMW1ifPf1cCqeMzyQOxBCDdMs0RT -hBbDYHW0BMXDqYIr3Ktbu38/3iVyr3pb56YOCKy8yHXpmKEaUBhCknSLcQyvNfeS -tM+DWsKFTZfyR5px+WwXbGKVMYwLaTON+/wcv1MeKMig3CxluaCpEJVYYwAiUc4K -sgvQNAunwGmPLPoXtUnpR2ZWiQA5R6yjS1oIe+8vpryFP6kjhWs0HR0jZEtLulV5 -WXUuxkqTXiBIvYpsmusoR44e9rptwLbV1wL/LUScRt9ttqFM3N5/Pof+2UwkSjGB -GAyPmw0Pkqtt+lva +MIIDazCCAlOgAwIBAgIUO9pfiBMsADdk9nBMHs10n8kaIr8wDQYJKoZIhvcNAQEL +BQAwRTELMAkGA1UEBhMCQVUxEzARBgNVBAgMClNvbWUtU3RhdGUxITAfBgNVBAoM +GEludGVybmV0IFdpZGdpdHMgUHR5IEx0ZDAeFw0yMjA0MTIwOTQxNDVaFw0yNTAx +MDUwOTQxNDVaMEUxCzAJBgNVBAYTAkFVMRMwEQYDVQQIDApTb21lLVN0YXRlMSEw +HwYDVQQKDBhJbnRlcm5ldCBXaWRnaXRzIFB0eSBMdGQwggEiMA0GCSqGSIb3DQEB +AQUAA4IBDwAwggEKAoIBAQC3uaPiZMfjPBBEyDEYJsJIoriu0SaC80uTmPM7bFpn +OOXOBvbT4wD2q+uVaLQifKtPTgZAkP5Y3rX8S5TOzaLsNp68S1Ja/EzxQUolOSgb +4A948TTiUTrTjfMxsPRhmxXTjozWV8CFtL9PLg6H+55oyQOJedWYe1kSWRJQayXS +weBK5qjOPi2qDF/xdFRQuMivpBUar/b/E9GQRKpIaoqMYsl/WF/tReb4N658UxkV +lFdR8s48UoA9LfJLMPr4N+QDTfvtcT2bYlpT4a9b6IXa9BQKCw3AKfTqEPO1XunH +//iLNkt1bLtqgZNyT/tY0tLY3EKMXIDuRBVnKCbfVJ1RAgMBAAGjUzBRMB0GA1Ud +DgQWBBSx7Tx8W4c6wjW0qkeG7CAMLY7YkjAfBgNVHSMEGDAWgBSx7Tx8W4c6wjW0 +qkeG7CAMLY7YkjAPBgNVHRMBAf8EBTADAQH/MA0GCSqGSIb3DQEBCwUAA4IBAQAb +/Up/LEIdwhiN/S3HolxY2D2BrTpKHLQuggBN4+gZlK5OksCkM46LYlP/ruHXCxbR +mQoRhmooj4TvkKyBwzvKq76O+OuRtBhXzRipnBbNTqFPLf9enJUrut8lsFrI+pdl +Nn4PSGGbFPpQ5vFRCktczwwYh0zLuZ/1DbFsbRWlDnZdvoWZdfV0qsvcBRK2DXDI +29xSfw897OpITIkaryZigQVsKv8TXhfsaq9PUuH0/z84S82QG5fR6FzULofgkylb +wXvwaSdcu3k4Lo8j77BEAEvlH8Ynja0eojx5Avl9h4iw/IOQKE4GAg56CzcequLv +clPlaBBWoD6yn+q4NhLF -----END CERTIFICATE----- From 82713f347b1db00a5ef1b9f896dc8f160ba44ab9 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Tue, 12 Apr 2022 12:55:57 +0300 Subject: [PATCH 086/101] Update clickhouse-test --- tests/clickhouse-test | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index b93416beda6..b52a9f4f026 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1173,6 +1173,9 @@ def check_server_started(args): retry_count -= 1 sleep(0.5) continue + except TimeoutError: + print("\nConnection timeout, will not retry") + break print('\nAll connection tries failed') sys.stdout.flush() From abb498c9a561fb06eba58b46a3d0016b765f34fe Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 12 Apr 2022 12:51:19 +0200 Subject: [PATCH 087/101] Update ClientBase.cpp --- src/Client/ClientBase.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index dcebe01c5b9..329fdc20cd9 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1864,8 +1864,9 @@ void ClientBase::runInteractive() { if (input.ends_with("\\G")) input.resize(input.size() - 2); - if (input.ends_with("\\G;")) + else if (input.ends_with("\\G;")) input.resize(input.size() - 3); + has_vertical_output_suffix = true; } From b91281455cb874d1ee856c182c62804ec0122aff Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 12 Apr 2022 15:15:21 +0300 Subject: [PATCH 088/101] clickhouse-client: properly cancel query in case of error during formatting data One example of such error is NO_ROW_DELIMITER [1]: $ clickhouse-client --stacktrace --multiquery <<<"SELECT * FROM no_length_delimiter_protobuf_00825 FORMAT ProtobufSingle SETTINGS format_schema = '$PWD/tests/queries/0_stateless/format_schemas/00825_protobuf_format_no_length_delimiter:Message'" Error on processing query: Code: 546. DB::Exception: The ProtobufSingle format can't be used to write multiple rows because this format doesn't have any row delimiter. (NO_ROW_DELIMITER), Stack trace (when copying this message, always include the lines below): ... 3. /build/build_docker/../src/Common/Exception.cpp:56: DB::Exception::Exception() 4. /build/build_docker/../src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp:43: DB::ProtobufRowOutputFormat::write() 5. /build/build_docker/../src/Processors/Formats/IRowOutputFormat.cpp:34: DB::IRowOutputFormat::consume() 6. /build/build_docker/../src/Processors/Formats/IOutputFormat.cpp:115: DB::IOutputFormat::write() 7. /build/build_docker/../src/Client/ClientBase.cpp:398: DB::ClientBase::onData() [1]: https://s3.amazonaws.com/clickhouse-test-reports/35865/b862fa7f3ede7d30b7c606c06e7fe1e273b49d32/stateless_tests__thread__actions__[3/3].html Signed-off-by: Azat Khuzhin --- src/Client/ClientBase.cpp | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 9092cb16663..7e8a7341e73 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -394,8 +394,16 @@ void ClientBase::onData(Block & block, ASTPtr parsed_query) if (need_render_progress && (stdout_is_a_tty || is_interactive) && !select_into_file) progress_indication.clearProgressOutput(); - output_format->write(materializeBlock(block)); - written_first_block = true; + try + { + output_format->write(materializeBlock(block)); + written_first_block = true; + } + catch (const Exception &) + { + /// Catch client errors like NO_ROW_DELIMITER + throw LocalFormatError(getCurrentExceptionMessage(print_stack_trace), getCurrentExceptionCode()); + } /// Received data block is immediately displayed to the user. output_format->flush(); From de9c9462e5e6aaf106fafbb8b1c91a304f691535 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 12 Apr 2022 13:54:53 +0000 Subject: [PATCH 089/101] Fix crash in ParallelReadBuffer --- src/IO/ParallelReadBuffer.cpp | 6 +----- tests/integration/test_storage_s3/test.py | 21 +++++++++++++++++++++ 2 files changed, 22 insertions(+), 5 deletions(-) diff --git a/src/IO/ParallelReadBuffer.cpp b/src/IO/ParallelReadBuffer.cpp index f036d6a08c8..79a16347094 100644 --- a/src/IO/ParallelReadBuffer.cpp +++ b/src/IO/ParallelReadBuffer.cpp @@ -33,6 +33,7 @@ bool ParallelReadBuffer::addReaderToPool(std::unique_lock & /*buffer auto worker = read_workers.emplace_back(std::make_shared(std::move(reader))); + ++active_working_reader; schedule([this, worker = std::move(worker)]() mutable { readerThreadFunction(std::move(worker)); }); return true; @@ -203,11 +204,6 @@ bool ParallelReadBuffer::nextImpl() void ParallelReadBuffer::readerThreadFunction(ReadWorkerPtr read_worker) { - { - std::lock_guard lock{mutex}; - ++active_working_reader; - } - SCOPE_EXIT({ std::lock_guard lock{mutex}; --active_working_reader; diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index e32ddd2782b..18cf7cc67ab 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -1407,3 +1407,24 @@ def test_insert_select_schema_inference(started_cluster): f"select * from s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_insert_select.native')" ) assert int(result) == 1 + + +def test_parallel_reading_with_memory_limit(started_cluster): + bucket = started_cluster.minio_bucket + instance = started_cluster.instances["dummy"] + + instance.query( + f"insert into function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_memory_limit.native') select * from numbers(100000)" + ) + + result = instance.query_and_get_error( + f"select * from url('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_memory_limit.native') settings max_memory_usage=10000" + ) + + assert "Memory limit (for query) exceeded" in result + + sleep(5) + + # Check that server didn't crash + result = instance.query("select 1") + assert int(result) == 1 From cefa64a8461b1c9017b3baf8fe34e05cce9dbf07 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 12 Apr 2022 17:56:22 +0200 Subject: [PATCH 090/101] Fix --- tests/queries/0_stateless/01161_all_system_tables.sh | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01161_all_system_tables.sh b/tests/queries/0_stateless/01161_all_system_tables.sh index 1a653763ad3..a5ed2ea7e6d 100755 --- a/tests/queries/0_stateless/01161_all_system_tables.sh +++ b/tests/queries/0_stateless/01161_all_system_tables.sh @@ -1,5 +1,7 @@ #!/usr/bin/env bash +# Tags: no-parallel + CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh @@ -12,7 +14,7 @@ function run_selects() { thread_num=$1 readarray -t tables_arr < <(${CLICKHOUSE_CLIENT} -q "SELECT database || '.' || name FROM system.tables - WHERE database in ('system', 'information_schema', 'INFORMATION_SCHEMA') and name!='zookeeper' and name!='merge_tree_metadata_cache' + WHERE database in ('system', 'information_schema', 'INFORMATION_SCHEMA') and name!='zookeeper' and name!='merge_tree_metadata_cache' AND sipHash64(name || toString($RAND)) % $THREADS = $thread_num") for t in "${tables_arr[@]}" From 75789eb9a773330314e0a92a6369bb542fa62928 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 12 Apr 2022 20:18:27 +0300 Subject: [PATCH 091/101] tests: improve 02050_client_profile_events Signed-off-by: Azat Khuzhin --- .../0_stateless/02050_client_profile_events.reference | 4 ++++ .../0_stateless/02050_client_profile_events.sh | 11 +++++++---- 2 files changed, 11 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02050_client_profile_events.reference b/tests/queries/0_stateless/02050_client_profile_events.reference index 2451417ddf0..e622780b7ec 100644 --- a/tests/queries/0_stateless/02050_client_profile_events.reference +++ b/tests/queries/0_stateless/02050_client_profile_events.reference @@ -1,5 +1,9 @@ +do not print any ProfileEvents packets 0 +print only last (and also number of rows to provide more info in case of failures) 100000 +print everything [ 0 ] SelectedRows: 131010 (increment) OK +print each 100 ms OK diff --git a/tests/queries/0_stateless/02050_client_profile_events.sh b/tests/queries/0_stateless/02050_client_profile_events.sh index f8bcea0d1bb..f0176f31c26 100755 --- a/tests/queries/0_stateless/02050_client_profile_events.sh +++ b/tests/queries/0_stateless/02050_client_profile_events.sh @@ -4,13 +4,16 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -# do not print any ProfileEvents packets +echo 'do not print any ProfileEvents packets' $CLICKHOUSE_CLIENT -q 'select * from numbers(1e5) format Null' |& grep -c 'SelectedRows' -# print only last (and also number of rows to provide more info in case of failures) + +echo 'print only last (and also number of rows to provide more info in case of failures)' $CLICKHOUSE_CLIENT --max_block_size=65505 --print-profile-events --profile-events-delay-ms=-1 -q 'select * from numbers(1e5)' 2> >(grep -o -e '\[ 0 \] SelectedRows: .*$' -e Exception) 1> >(wc -l) -# print everything + +echo 'print everything' profile_events="$($CLICKHOUSE_CLIENT --max_block_size 1 --print-profile-events -q 'select sleep(1) from numbers(2) format Null' |& grep -c 'SelectedRows')" test "$profile_events" -gt 1 && echo OK || echo "FAIL ($profile_events)" -# print each 100 ms + +echo 'print each 100 ms' profile_events="$($CLICKHOUSE_CLIENT --max_block_size 1 --print-profile-events --profile-events-delay-ms=100 -q 'select sleep(1) from numbers(2) format Null' |& grep -c 'SelectedRows')" test "$profile_events" -gt 1 && echo OK || echo "FAIL ($profile_events)" From d46eaadacc6a3b6b77737b4db0dcdf23a0a75b78 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 12 Apr 2022 20:38:16 +0300 Subject: [PATCH 092/101] tests: do not use process substution in 02050_client_profile_events bash does not waits then and so if the next line will be executed fast enough, result of process substituted command can appers in stdout after it. Signed-off-by: Azat Khuzhin --- .../queries/0_stateless/02050_client_profile_events.reference | 3 +-- tests/queries/0_stateless/02050_client_profile_events.sh | 2 +- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02050_client_profile_events.reference b/tests/queries/0_stateless/02050_client_profile_events.reference index e622780b7ec..cf7c14d9fdd 100644 --- a/tests/queries/0_stateless/02050_client_profile_events.reference +++ b/tests/queries/0_stateless/02050_client_profile_events.reference @@ -1,9 +1,8 @@ do not print any ProfileEvents packets 0 print only last (and also number of rows to provide more info in case of failures) -100000 -print everything [ 0 ] SelectedRows: 131010 (increment) +print everything OK print each 100 ms OK diff --git a/tests/queries/0_stateless/02050_client_profile_events.sh b/tests/queries/0_stateless/02050_client_profile_events.sh index f0176f31c26..01a25244546 100755 --- a/tests/queries/0_stateless/02050_client_profile_events.sh +++ b/tests/queries/0_stateless/02050_client_profile_events.sh @@ -8,7 +8,7 @@ echo 'do not print any ProfileEvents packets' $CLICKHOUSE_CLIENT -q 'select * from numbers(1e5) format Null' |& grep -c 'SelectedRows' echo 'print only last (and also number of rows to provide more info in case of failures)' -$CLICKHOUSE_CLIENT --max_block_size=65505 --print-profile-events --profile-events-delay-ms=-1 -q 'select * from numbers(1e5)' 2> >(grep -o -e '\[ 0 \] SelectedRows: .*$' -e Exception) 1> >(wc -l) +$CLICKHOUSE_CLIENT --max_block_size=65505 --print-profile-events --profile-events-delay-ms=-1 -q 'select * from numbers(1e5)' |& grep -o -e '\[ 0 \] SelectedRows: .*$' -e Exception echo 'print everything' profile_events="$($CLICKHOUSE_CLIENT --max_block_size 1 --print-profile-events -q 'select sleep(1) from numbers(2) format Null' |& grep -c 'SelectedRows')" From 1fa3e6a4260f65350bcab1f8093e12136d64eea9 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 11 Apr 2022 17:42:45 +0300 Subject: [PATCH 093/101] Fix printing ProfileEvents on client for multiple queries Signed-off-by: Azat Khuzhin --- src/Client/ClientBase.cpp | 2 ++ .../queries/0_stateless/02050_client_profile_events.reference | 2 ++ tests/queries/0_stateless/02050_client_profile_events.sh | 4 ++++ 3 files changed, 8 insertions(+) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 9092cb16663..49a17e8910d 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1413,6 +1413,8 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin progress_indication.clearProgressOutput(); logs_out_stream->writeProfileEvents(profile_events.last_block); logs_out_stream->flush(); + + profile_events.last_block = {}; } if (is_interactive) diff --git a/tests/queries/0_stateless/02050_client_profile_events.reference b/tests/queries/0_stateless/02050_client_profile_events.reference index cf7c14d9fdd..ac7978506a2 100644 --- a/tests/queries/0_stateless/02050_client_profile_events.reference +++ b/tests/queries/0_stateless/02050_client_profile_events.reference @@ -6,3 +6,5 @@ print everything OK print each 100 ms OK +check that ProfileEvents is new for each query +OK diff --git a/tests/queries/0_stateless/02050_client_profile_events.sh b/tests/queries/0_stateless/02050_client_profile_events.sh index 01a25244546..5a9a7c42180 100755 --- a/tests/queries/0_stateless/02050_client_profile_events.sh +++ b/tests/queries/0_stateless/02050_client_profile_events.sh @@ -17,3 +17,7 @@ test "$profile_events" -gt 1 && echo OK || echo "FAIL ($profile_events)" echo 'print each 100 ms' profile_events="$($CLICKHOUSE_CLIENT --max_block_size 1 --print-profile-events --profile-events-delay-ms=100 -q 'select sleep(1) from numbers(2) format Null' |& grep -c 'SelectedRows')" test "$profile_events" -gt 1 && echo OK || echo "FAIL ($profile_events)" + +echo 'check that ProfileEvents is new for each query' +sleep_function_calls=$($CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -n -q 'select sleep(1); select 1' |& grep -c 'SleepFunctionCalls') +test "$sleep_function_calls" -eq 1 && echo OK || echo "FAIL ($sleep_function_calls)" From d9dd8f5f655282dbae0e90e7cfe323d4a38aa198 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 12 Apr 2022 19:51:39 +0300 Subject: [PATCH 094/101] Fix filtering out snapshots from profile events This fixes POSITION_OUT_OF_BOUND error: $ clickhouse-client --print-profile-events --profile-events-delay-ms=-1 -n -q 'select sleep(1); select 1' 0 [p1.azat.localdomain] 2022.04.12 19:31:48 [ 0 ] ContextLock: 9 (increment) [p1.azat.localdomain] 2022.04.12 19:31:48 [ 0 ] FunctionExecute: 1 (increment) [p1.azat.localdomain] 2022.04.12 19:31:49 [ 0 ] MemoryTrackerUsage: 2132102 (gauge) [p1.azat.localdomain] 2022.04.12 19:31:49 [ 0 ] NetworkReceiveElapsedMicroseconds: 139 (increment) [p1.azat.localdomain] 2022.04.12 19:31:49 [ 0 ] NetworkSendBytes: 4850 (increment) [p1.azat.localdomain] 2022.04.12 19:31:49 [ 0 ] NetworkSendElapsedMicroseconds: 1844 (increment) [p1.azat.localdomain] 2022.04.12 19:31:48 [ 0 ] Query: 1 (increment) [p1.azat.localdomain] 2022.04.12 19:31:48 [ 0 ] RWLockAcquiredReadLocks: 1 (increment) [p1.azat.localdomain] 2022.04.12 19:31:48 [ 0 ] SelectQuery: 1 (increment) [p1.azat.localdomain] 2022.04.12 19:31:48 [ 0 ] SelectedBytes: 1 (increment) [p1.azat.localdomain] 2022.04.12 19:31:48 [ 0 ] SelectedRows: 1 (increment) [p1.azat.localdomain] 2022.04.12 19:31:49 [ 0 ] OSCPUVirtualTimeMicroseconds: 1842 (increment) [p1.azat.localdomain] 2022.04.12 19:31:49 [ 0 ] OSReadChars: 426 (increment) [p1.azat.localdomain] 2022.04.12 19:31:49 [ 0 ] OSWriteChars: 322 (increment) [p1.azat.localdomain] 2022.04.12 19:31:49 [ 0 ] RealTimeMicroseconds: 1002689 (increment) [p1.azat.localdomain] 2022.04.12 19:31:49 [ 0 ] SleepFunctionCalls: 1 (increment) [p1.azat.localdomain] 2022.04.12 19:31:49 [ 0 ] SleepFunctionMicroseconds: 1000000 (increment) [p1.azat.localdomain] 2022.04.12 19:31:49 [ 0 ] UserTimeMicroseconds: 1843 (increment) 1 Error on processing query: Code: 11. DB::Exception: Position out of bound in Block::erase(), max position = 5. (POSITION_OUT_OF_BOUND) (version 22.4.1.1) (query: select 1) Signed-off-by: Azat Khuzhin --- src/Client/ClientBase.cpp | 35 ++++++++++++------- .../02050_client_profile_events.reference | 2 ++ .../02050_client_profile_events.sh | 4 +++ 3 files changed, 28 insertions(+), 13 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 49a17e8910d..3a31a4b21c8 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -137,14 +137,14 @@ static void incrementProfileEventsBlock(Block & dst, const Block & src) auto & dst_column_host_name = typeid_cast(*mutable_columns[name_pos["host_name"]]); auto & dst_array_current_time = typeid_cast(*mutable_columns[name_pos["current_time"]]).getData(); - auto & dst_array_thread_id = typeid_cast(*mutable_columns[name_pos["thread_id"]]).getData(); + // auto & dst_array_thread_id = typeid_cast(*mutable_columns[name_pos["thread_id"]]).getData(); auto & dst_array_type = typeid_cast(*mutable_columns[name_pos["type"]]).getData(); auto & dst_column_name = typeid_cast(*mutable_columns[name_pos["name"]]); auto & dst_array_value = typeid_cast(*mutable_columns[name_pos["value"]]).getData(); const auto & src_column_host_name = typeid_cast(*src.getByName("host_name").column); const auto & src_array_current_time = typeid_cast(*src.getByName("current_time").column).getData(); - // const auto & src_array_thread_id = typeid_cast(*src.getByName("thread_id").column).getData(); + const auto & src_array_thread_id = typeid_cast(*src.getByName("thread_id").column).getData(); const auto & src_column_name = typeid_cast(*src.getByName("name").column); const auto & src_array_value = typeid_cast(*src.getByName("value").column).getData(); @@ -169,6 +169,16 @@ static void incrementProfileEventsBlock(Block & dst, const Block & src) rows_by_name[id] = src_row; } + /// Filter out snapshots + std::set thread_id_filter_mask; + for (size_t i = 0; i < src_array_thread_id.size(); ++i) + { + if (src_array_thread_id[i] != 0) + { + thread_id_filter_mask.emplace(i); + } + } + /// Merge src into dst. for (size_t dst_row = 0; dst_row < dst_rows; ++dst_row) { @@ -180,6 +190,11 @@ static void incrementProfileEventsBlock(Block & dst, const Block & src) if (auto it = rows_by_name.find(id); it != rows_by_name.end()) { size_t src_row = it->second; + if (thread_id_filter_mask.contains(src_row)) + { + continue; + } + dst_array_current_time[dst_row] = src_array_current_time[src_row]; switch (dst_array_type[dst_row]) @@ -199,24 +214,18 @@ static void incrementProfileEventsBlock(Block & dst, const Block & src) /// Copy rows from src that dst does not contains. for (const auto & [id, pos] : rows_by_name) { + if (thread_id_filter_mask.contains(pos)) + { + continue; + } + for (size_t col = 0; col < src.columns(); ++col) { mutable_columns[col]->insert((*src.getByPosition(col).column)[pos]); } } - /// Filter out snapshots - std::set thread_id_filter_mask; - for (size_t i = 0; i < dst_array_thread_id.size(); ++i) - { - if (dst_array_thread_id[i] != 0) - { - thread_id_filter_mask.emplace(i); - } - } - dst.setColumns(std::move(mutable_columns)); - dst.erase(thread_id_filter_mask); } diff --git a/tests/queries/0_stateless/02050_client_profile_events.reference b/tests/queries/0_stateless/02050_client_profile_events.reference index ac7978506a2..c8e031d3562 100644 --- a/tests/queries/0_stateless/02050_client_profile_events.reference +++ b/tests/queries/0_stateless/02050_client_profile_events.reference @@ -2,6 +2,8 @@ do not print any ProfileEvents packets 0 print only last (and also number of rows to provide more info in case of failures) [ 0 ] SelectedRows: 131010 (increment) +regression test for incorrect filtering out snapshots +0 print everything OK print each 100 ms diff --git a/tests/queries/0_stateless/02050_client_profile_events.sh b/tests/queries/0_stateless/02050_client_profile_events.sh index 5a9a7c42180..5470c44594c 100755 --- a/tests/queries/0_stateless/02050_client_profile_events.sh +++ b/tests/queries/0_stateless/02050_client_profile_events.sh @@ -10,6 +10,10 @@ $CLICKHOUSE_CLIENT -q 'select * from numbers(1e5) format Null' |& grep -c 'Selec echo 'print only last (and also number of rows to provide more info in case of failures)' $CLICKHOUSE_CLIENT --max_block_size=65505 --print-profile-events --profile-events-delay-ms=-1 -q 'select * from numbers(1e5)' |& grep -o -e '\[ 0 \] SelectedRows: .*$' -e Exception +echo 'regression test for incorrect filtering out snapshots' +$CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -n -q 'select 1; select 1' >& /dev/null +echo $? + echo 'print everything' profile_events="$($CLICKHOUSE_CLIENT --max_block_size 1 --print-profile-events -q 'select sleep(1) from numbers(2) format Null' |& grep -c 'SelectedRows')" test "$profile_events" -gt 1 && echo OK || echo "FAIL ($profile_events)" From e38057ddb0cfd13776586611eaee11fef4723cd4 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 12 Apr 2022 20:14:28 +0300 Subject: [PATCH 095/101] Fix code alignment in ProfileEventsExt Signed-off-by: Azat Khuzhin --- src/Interpreters/ProfileEventsExt.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ProfileEventsExt.cpp b/src/Interpreters/ProfileEventsExt.cpp index ea87d565854..d7add73dbf0 100644 --- a/src/Interpreters/ProfileEventsExt.cpp +++ b/src/Interpreters/ProfileEventsExt.cpp @@ -105,7 +105,7 @@ void getProfileEvents( {"value", std::make_shared()}, }; - ColumnsWithTypeAndName temp_columns; + ColumnsWithTypeAndName temp_columns; for (auto const & name_and_type : column_names_and_types) temp_columns.emplace_back(name_and_type.type, name_and_type.name); From 097e2f626d39ca21584a1f7d83d954b3ebb24c5a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 12 Apr 2022 20:14:41 +0300 Subject: [PATCH 096/101] Do not overlap profile events snapshots for queries Signed-off-by: Azat Khuzhin --- src/Server/TCPHandler.cpp | 1 + .../queries/0_stateless/02050_client_profile_events.reference | 3 +++ tests/queries/0_stateless/02050_client_profile_events.sh | 3 +++ 3 files changed, 7 insertions(+) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 119f601563a..981a59caefe 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -728,6 +728,7 @@ void TCPHandler::processOrdinaryQueryWithProcessors() return; sendData({}); + last_sent_snapshots.clear(); } sendProgress(); diff --git a/tests/queries/0_stateless/02050_client_profile_events.reference b/tests/queries/0_stateless/02050_client_profile_events.reference index c8e031d3562..7ca88730e16 100644 --- a/tests/queries/0_stateless/02050_client_profile_events.reference +++ b/tests/queries/0_stateless/02050_client_profile_events.reference @@ -4,6 +4,9 @@ print only last (and also number of rows to provide more info in case of failure [ 0 ] SelectedRows: 131010 (increment) regression test for incorrect filtering out snapshots 0 +regression test for overlap profile events snapshots between queries +[ 0 ] SelectedRows: 1 (increment) +[ 0 ] SelectedRows: 1 (increment) print everything OK print each 100 ms diff --git a/tests/queries/0_stateless/02050_client_profile_events.sh b/tests/queries/0_stateless/02050_client_profile_events.sh index 5470c44594c..6117f971fd5 100755 --- a/tests/queries/0_stateless/02050_client_profile_events.sh +++ b/tests/queries/0_stateless/02050_client_profile_events.sh @@ -14,6 +14,9 @@ echo 'regression test for incorrect filtering out snapshots' $CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -n -q 'select 1; select 1' >& /dev/null echo $? +echo 'regression test for overlap profile events snapshots between queries' +$CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -n -q 'select 1; select 1' |& grep -F -o '[ 0 ] SelectedRows: 1 (increment)' + echo 'print everything' profile_events="$($CLICKHOUSE_CLIENT --max_block_size 1 --print-profile-events -q 'select sleep(1) from numbers(2) format Null' |& grep -c 'SelectedRows')" test "$profile_events" -gt 1 && echo OK || echo "FAIL ($profile_events)" From 1a8b49b5357079f11e8ad31a0b64db69b0a76895 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 12 Apr 2022 21:13:59 +0300 Subject: [PATCH 097/101] Remove redundant check from LocalConnection::finishQuery() It is called from only one place, and under state->is_finished, so state is there. Signed-off-by: Azat Khuzhin --- src/Client/LocalConnection.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Client/LocalConnection.cpp b/src/Client/LocalConnection.cpp index a57086810bf..e983bda5d57 100644 --- a/src/Client/LocalConnection.cpp +++ b/src/Client/LocalConnection.cpp @@ -201,9 +201,6 @@ void LocalConnection::finishQuery() { next_packet_type = Protocol::Server::EndOfStream; - if (!state) - return; - if (state->executor) { state->executor.reset(); From 54a1cea7fd8d33144b95862e88af6392f187f2cd Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 12 Apr 2022 21:18:16 +0300 Subject: [PATCH 098/101] Send ProfileEvents in clickhouse-local on query finish (like server does) v2: send only for pulling pipelines Signed-off-by: Azat Khuzhin --- src/Client/LocalConnection.cpp | 15 +++++++++++++++ src/Client/LocalConnection.h | 1 + 2 files changed, 16 insertions(+) diff --git a/src/Client/LocalConnection.cpp b/src/Client/LocalConnection.cpp index e983bda5d57..b62bbe86b79 100644 --- a/src/Client/LocalConnection.cpp +++ b/src/Client/LocalConnection.cpp @@ -323,6 +323,21 @@ bool LocalConnection::poll(size_t) } } + if (state->is_finished && !state->sent_profile_events) + { + state->sent_profile_events = true; + + if (send_profile_events && state->executor) + { + Block block; + state->after_send_profile_events.restart(); + next_packet_type = Protocol::Server::ProfileEvents; + getProfileEvents(block); + state->block.emplace(std::move(block)); + return true; + } + } + if (state->is_finished) { finishQuery(); diff --git a/src/Client/LocalConnection.h b/src/Client/LocalConnection.h index 62e95cdfee6..32af8a4cfa1 100644 --- a/src/Client/LocalConnection.h +++ b/src/Client/LocalConnection.h @@ -47,6 +47,7 @@ struct LocalQueryState bool sent_extremes = false; bool sent_progress = false; bool sent_profile_info = false; + bool sent_profile_events = false; /// To output progress, the difference after the previous sending of progress. Progress progress; From 891c601589ec117d32a29a5ec79a91d8a3f6941f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 12 Apr 2022 21:11:39 +0300 Subject: [PATCH 099/101] Fix ProfileEvents overlaps (for multiple queries) in clickhouse-local Signed-off-by: Azat Khuzhin --- src/Client/LocalConnection.cpp | 1 + .../queries/0_stateless/02050_client_profile_events.reference | 3 +++ tests/queries/0_stateless/02050_client_profile_events.sh | 3 +++ 3 files changed, 7 insertions(+) diff --git a/src/Client/LocalConnection.cpp b/src/Client/LocalConnection.cpp index b62bbe86b79..b1dacedb042 100644 --- a/src/Client/LocalConnection.cpp +++ b/src/Client/LocalConnection.cpp @@ -216,6 +216,7 @@ void LocalConnection::finishQuery() state->io.onFinish(); state.reset(); + last_sent_snapshots.clear(); } bool LocalConnection::poll(size_t) diff --git a/tests/queries/0_stateless/02050_client_profile_events.reference b/tests/queries/0_stateless/02050_client_profile_events.reference index 7ca88730e16..99b0fa6c981 100644 --- a/tests/queries/0_stateless/02050_client_profile_events.reference +++ b/tests/queries/0_stateless/02050_client_profile_events.reference @@ -7,6 +7,9 @@ regression test for incorrect filtering out snapshots regression test for overlap profile events snapshots between queries [ 0 ] SelectedRows: 1 (increment) [ 0 ] SelectedRows: 1 (increment) +regression test for overlap profile events snapshots between queries (clickhouse-local) +[ 0 ] SelectedRows: 1 (increment) +[ 0 ] SelectedRows: 1 (increment) print everything OK print each 100 ms diff --git a/tests/queries/0_stateless/02050_client_profile_events.sh b/tests/queries/0_stateless/02050_client_profile_events.sh index 6117f971fd5..dce0c80525a 100755 --- a/tests/queries/0_stateless/02050_client_profile_events.sh +++ b/tests/queries/0_stateless/02050_client_profile_events.sh @@ -17,6 +17,9 @@ echo $? echo 'regression test for overlap profile events snapshots between queries' $CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -n -q 'select 1; select 1' |& grep -F -o '[ 0 ] SelectedRows: 1 (increment)' +echo 'regression test for overlap profile events snapshots between queries (clickhouse-local)' +$CLICKHOUSE_LOCAL --print-profile-events --profile-events-delay-ms=-1 -n -q 'select 1; select 1' |& grep -F -o '[ 0 ] SelectedRows: 1 (increment)' + echo 'print everything' profile_events="$($CLICKHOUSE_CLIENT --max_block_size 1 --print-profile-events -q 'select sleep(1) from numbers(2) format Null' |& grep -c 'SelectedRows')" test "$profile_events" -gt 1 && echo OK || echo "FAIL ($profile_events)" From 51685e095638c9b2f6f1b7bd70ba540e6b9eb64b Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 13 Apr 2022 13:36:28 +0200 Subject: [PATCH 100/101] Dirty fix fasttest after addressing CVE-2022-24765 --- docker/test/fasttest/run.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 079d2872204..31ec52b1d5a 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -115,6 +115,7 @@ function start_server function clone_root { + git config --global --add safe.directory "$FASTTEST_SOURCE" git clone --depth 1 https://github.com/ClickHouse/ClickHouse.git -- "$FASTTEST_SOURCE" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/clone_log.txt" ( From 693711544c8863df17d9bc7e6884a96a0ec05b13 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 13 Apr 2022 13:45:54 +0200 Subject: [PATCH 101/101] Use a proper image in fasttest --- .github/workflows/pull_request.yml | 16 +++++++++++----- tests/ci/docker_pull_helper.py | 1 + 2 files changed, 12 insertions(+), 5 deletions(-) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 4881bed3f55..0e68a68e8f5 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -153,13 +153,19 @@ jobs: EOF - name: Clear repository run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Fast Test - run: | + sudo rm -fr "$GITHUB_WORKSPACE" + mkdir "$GITHUB_WORKSPACE" sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ env.TEMP_PATH }} + - name: Fast Test + run: | cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" cd "$REPO_COPY/tests/ci" && python3 fast_test_check.py - name: Cleanup diff --git a/tests/ci/docker_pull_helper.py b/tests/ci/docker_pull_helper.py index 717327ded41..04817ed7de3 100644 --- a/tests/ci/docker_pull_helper.py +++ b/tests/ci/docker_pull_helper.py @@ -76,4 +76,5 @@ def get_images_with_versions( def get_image_with_version(reports_path, image, pull=True, version=None): + logging.info("Looking for images file in %s", reports_path) return get_images_with_versions(reports_path, [image], pull, version=version)[0]